You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@bookkeeper.apache.org by si...@apache.org on 2018/09/24 06:08:47 UTC

[bookkeeper] branch master updated: [TABLE SERVICE] [PYTHON] introduce a python client for table service

This is an automated email from the ASF dual-hosted git repository.

sijie pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/bookkeeper.git


The following commit(s) were added to refs/heads/master by this push:
     new 4deb4d1  [TABLE SERVICE] [PYTHON] introduce a python client for table service
4deb4d1 is described below

commit 4deb4d15390f4b633021539b3c92806291411f57
Author: Sijie Guo <gu...@gmail.com>
AuthorDate: Sun Sep 23 23:08:42 2018 -0700

    [TABLE SERVICE] [PYTHON] introduce a python client for table service
    
    Descriptions of the changes in this PR:
    
    *Motivation*
    
    We need a python client for accessing table service.
    
    *Changes*
    
    This is the first change for introducing a python client for table service.
    
    - setup the directory structure for the python client
    - setup build, lint, test for the client
    - implement the admin client for creating/deleting/getting namespaces/streams
    - add examples for admin client
    
    Master Issue: #1690
    
    
    
    
    Author: Sijie Guo <si...@apache.org>
    
    Reviewers: Jia Zhai <None>
    
    This closes #1691 from sijie/stream_python_client
---
 pom.xml                                            |   17 +
 stream/clients/python/.coveragerc                  |   30 +
 stream/clients/python/.flake8                      |   23 +
 stream/clients/python/.gitignore                   |   22 +
 stream/clients/python/README.md                    |    4 +
 stream/clients/python/bookkeeper/__init__.py       |   20 +
 stream/clients/python/bookkeeper/admin/__init__.py |   19 +
 stream/clients/python/bookkeeper/admin/client.py   |   72 ++
 .../clients/python/bookkeeper/admin/exceptions.py  |   19 +
 .../clients/python/bookkeeper/admin/namespace.py   |   60 +
 .../clients/python/bookkeeper/admin/namespaces.py  |   53 +
 .../clients/python/bookkeeper/common/__init__.py   |   13 +
 .../clients/python/bookkeeper/common/constants.py  |   45 +
 .../python/bookkeeper/common/datetime_helpers.py   |  224 ++++
 .../clients/python/bookkeeper/common/exceptions.py |  185 +++
 .../python/bookkeeper/common/future/__init__.py    |   19 +
 .../python/bookkeeper/common/future/_helpers.py    |   37 +
 .../python/bookkeeper/common/future/base.py        |   64 +
 .../python/bookkeeper/common/future/polling.py     |  161 +++
 .../python/bookkeeper/common/general_helpers.py    |   30 +
 .../python/bookkeeper/common/grpc_helpers.py       |  313 +++++
 .../python/bookkeeper/common/protobuf_helpers.py   |  321 +++++
 stream/clients/python/bookkeeper/common/retry.py   |  282 +++++
 .../python/bookkeeper/common/service_uri.py        |   31 +
 stream/clients/python/bookkeeper/kv/__init__.py    |   19 +
 stream/clients/python/bookkeeper/kv/client.py      |   60 +
 stream/clients/python/bookkeeper/kv/exceptions.py  |   19 +
 stream/clients/python/bookkeeper/kv/futures.py     |  170 +++
 .../clients/python/bookkeeper/proto/cluster_pb2.py |  199 +++
 .../clients/python/bookkeeper/proto/common_pb2.py  |   77 ++
 stream/clients/python/bookkeeper/proto/kv_pb2.py   |  192 +++
 .../clients/python/bookkeeper/proto/kv_rpc_pb2.py  | 1232 ++++++++++++++++++
 .../python/bookkeeper/proto/kv_rpc_pb2_grpc.py     |  122 ++
 .../python/bookkeeper/proto/kv_store_pb2.py        |  297 +++++
 .../clients/python/bookkeeper/proto/storage_pb2.py | 1276 +++++++++++++++++++
 .../python/bookkeeper/proto/storage_pb2_grpc.py    |  219 ++++
 .../clients/python/bookkeeper/proto/stream_pb2.py  | 1323 ++++++++++++++++++++
 stream/clients/python/bookkeeper/types.py          |   71 ++
 stream/clients/python/examples/admin.py            |   58 +
 stream/clients/python/noxfile.py                   |   86 ++
 stream/clients/python/pylint.config.py             |   29 +
 stream/clients/python/setup.cfg                    |   14 +
 stream/clients/python/setup.py                     |   87 ++
 stream/clients/python/tests/unit/__init__.py       |   11 +
 .../unit/bookkeeper/common/future/test__helpers.py |   35 +
 .../unit/bookkeeper/common/future/test_polling.py  |  155 +++
 .../unit/bookkeeper/common/test_grpc_helpers.py    |  172 +++
 .../bookkeeper/common/test_protobuf_helpers.py     |  274 ++++
 .../python/tests/unit/bookkeeper/test_futures.py   |  145 +++
 49 files changed, 8406 insertions(+)

diff --git a/pom.xml b/pom.xml
index a788fcf..e5a3ffd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -824,6 +824,9 @@
         <version>${apache-rat-plugin.version}</version>
         <configuration>
           <excludes>
+            <!-- Intellij -->
+            <exclude>**/.idea/**</exclude>
+
             <!-- Git -->
             <exclude>.git/**/*</exclude>
             <exclude>.github/**/*</exclude>
@@ -867,6 +870,20 @@
 
             <!-- vargrant -->
             <exclude>dev/.vagrant/**</exclude>
+
+            <!-- protobuf generated python files-->
+            <exclude>**/proto/**.py</exclude>
+            <!-- python build/test env -->
+            <exclude>**/python/.coverage</exclude>
+            <exclude>**/python/.Python</exclude>
+            <exclude>**/python/bin/**</exclude>
+            <exclude>**/python/include/**</exclude>
+            <exclude>**/python/lib/**</exclude>
+            <exclude>**/.nox/**</exclude>
+            <exclude>**/.pytest_cache/**</exclude>
+            <exclude>**/__pycache__/**</exclude>
+            <exclude>**/bookkeeper.egg-info/**</exclude>
+            <exclude>**/pip-selfcheck.json</exclude>
           </excludes>
           <consoleOutput>true</consoleOutput>
         </configuration>
diff --git a/stream/clients/python/.coveragerc b/stream/clients/python/.coveragerc
new file mode 100644
index 0000000..2a6c4e7
--- /dev/null
+++ b/stream/clients/python/.coveragerc
@@ -0,0 +1,30 @@
+# 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.
+
+[run]
+branch = True
+source =
+  bookkeeper
+  tests.unit
+
+[report]
+omit =
+  */proto/*
+show_missing = True
+
+exclude_lines =
+    # Re-enable the standard pragma
+    pragma: NO COVER
+    # Ignore debug-only repr
+    def __repr__
+    # Ignore abstract methods
+    raise NotImplementedError
diff --git a/stream/clients/python/.flake8 b/stream/clients/python/.flake8
new file mode 100644
index 0000000..6c93f20
--- /dev/null
+++ b/stream/clients/python/.flake8
@@ -0,0 +1,23 @@
+# 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.
+
+[flake8]
+exclude =
+  # Exclude generated code.
+  **/proto/**
+  *_pb2.py
+
+  # Standard linting exemptions.
+  __pycache__,
+  .git,
+  *.pyc,
+  conf.py
diff --git a/stream/clients/python/.gitignore b/stream/clients/python/.gitignore
new file mode 100644
index 0000000..a520e65
--- /dev/null
+++ b/stream/clients/python/.gitignore
@@ -0,0 +1,22 @@
+# virtualenv
+.Python
+bin/
+include/
+
+# coverage
+.coverage
+
+# nox
+.nox/
+
+# pytest
+.pytest_cache/
+
+# pycache
+__pycache__/
+
+# build files
+bookkeeper.egg-info/
+
+# pip
+pip-selfcheck.json
diff --git a/stream/clients/python/README.md b/stream/clients/python/README.md
new file mode 100644
index 0000000..df1b7ef
--- /dev/null
+++ b/stream/clients/python/README.md
@@ -0,0 +1,4 @@
+Python Client for Apache BookKeeper
+===================================
+
+|pypi| |versions|
diff --git a/stream/clients/python/bookkeeper/__init__.py b/stream/clients/python/bookkeeper/__init__.py
new file mode 100644
index 0000000..c1052c1
--- /dev/null
+++ b/stream/clients/python/bookkeeper/__init__.py
@@ -0,0 +1,20 @@
+# -*- coding: utf-8 -*-
+#
+# 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
+#
+#     https://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.
+
+try:
+    import pkg_resources
+    pkg_resources.declare_namespace(__name__)
+except ImportError:
+    import pkgutil
+    __path__ = pkgutil.extend_path(__path__, __name__)
diff --git a/stream/clients/python/bookkeeper/admin/__init__.py b/stream/clients/python/bookkeeper/admin/__init__.py
new file mode 100644
index 0000000..dce620f
--- /dev/null
+++ b/stream/clients/python/bookkeeper/admin/__init__.py
@@ -0,0 +1,19 @@
+# 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 __future__ import absolute_import
+
+from bookkeeper.admin.client import Client
+
+__all__ = (
+    'Client',
+)
diff --git a/stream/clients/python/bookkeeper/admin/client.py b/stream/clients/python/bookkeeper/admin/client.py
new file mode 100644
index 0000000..d3dff3f
--- /dev/null
+++ b/stream/clients/python/bookkeeper/admin/client.py
@@ -0,0 +1,72 @@
+# -*- coding: utf-8 -*-
+#
+# 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
+#
+#     https://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 __future__ import absolute_import
+
+import grpc
+import logging
+import pkg_resources
+
+from bookkeeper import types
+from bookkeeper.admin.namespace import Namespace
+from bookkeeper.admin.namespaces import Namespaces
+from bookkeeper.common.service_uri import ServiceURI
+from bookkeeper.proto.storage_pb2_grpc import RootRangeServiceStub
+
+__version__ = pkg_resources.get_distribution('bookkeeper').version
+__logger__ = logging.getLogger("bookkeeper.admin.Client")
+
+
+class Client(object):
+    """An admin client for Apache BookKeeper.
+
+    This creates an object that is capable of administrating bookkeeper
+    resources. Generally, you can instantiate this client with no arguments,
+    and you get sensible defaults.
+
+    Args:
+        storage_client_settings (~bookkeeper.types.StorageClientSettings): The
+            settings for bookkeeper storage client .
+        kwargs (dict): Any additional arguments provided are sent as keyword
+            arguments to the underlying grpc client.
+    """
+
+    def __init__(self, storage_client_settings=(), **kwargs):
+        # init the storage client settings
+        self.storage_client_settings =\
+            types.StorageClientSettings(*storage_client_settings)
+        __logger__.info("Creating an admin client to cluster '%s'",
+                        self.storage_client_settings.service_uri)
+
+        service_uri = ServiceURI(self.storage_client_settings.service_uri)
+        assert service_uri.service_name.lower() == 'bk'
+
+        # create channel
+        self.channel = grpc.insecure_channel(
+            target=service_uri.service_location
+        )
+        __logger__.info("Successfully created an admin client to cluster '%s'",
+                        self.storage_client_settings.service_uri)
+
+        # create the rpc stub
+        self.root_range = RootRangeServiceStub(channel=self.channel)
+
+        # services
+        self.__namespaces__ = Namespaces(client=self)
+
+    def namespaces(self):
+        return self.__namespaces__
+
+    def namespace(self, namespace):
+        return Namespace(self, namespace)
diff --git a/stream/clients/python/bookkeeper/admin/exceptions.py b/stream/clients/python/bookkeeper/admin/exceptions.py
new file mode 100644
index 0000000..3bc01a858
--- /dev/null
+++ b/stream/clients/python/bookkeeper/admin/exceptions.py
@@ -0,0 +1,19 @@
+# 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 __future__ import absolute_import
+
+from concurrent.futures import TimeoutError
+
+__all__ = (
+    'TimeoutError',
+)
diff --git a/stream/clients/python/bookkeeper/admin/namespace.py b/stream/clients/python/bookkeeper/admin/namespace.py
new file mode 100644
index 0000000..029c2c4
--- /dev/null
+++ b/stream/clients/python/bookkeeper/admin/namespace.py
@@ -0,0 +1,60 @@
+# -*- coding: utf-8 -*-
+#
+# 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
+#
+#     https://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 __future__ import absolute_import
+
+from bookkeeper.common.constants import __DEFAULT_STREAM_CONF__
+from bookkeeper.common.constants import __ROOT_RANGE_METADATA__
+from bookkeeper.proto import storage_pb2
+from bookkeeper.proto import stream_pb2
+
+
+class Namespace(object):
+
+    def __init__(self, client, namespace):
+        self.client = client
+        self.namespace = namespace
+
+    def create(self, stream_name, stream_config=__DEFAULT_STREAM_CONF__):
+        create_stream_req = storage_pb2.CreateStreamRequest(
+            ns_name=self.namespace,
+            name=stream_name,
+            stream_conf=stream_config
+        )
+        return self.client.root_range.CreateStream(
+            request=create_stream_req,
+            metadata=__ROOT_RANGE_METADATA__
+        )
+
+    def get(self, stream_name):
+        get_stream_req = storage_pb2.GetStreamRequest(
+            stream_name=stream_pb2.StreamName(
+                namespace_name=self.namespace,
+                stream_name=stream_name
+            )
+        )
+        return self.client.root_range.GetStream(
+            request=get_stream_req,
+            metadata=__ROOT_RANGE_METADATA__
+        )
+
+    def delete(self, stream_name):
+        del_stream_req = storage_pb2.DeleteStreamRequest(
+            ns_name=self.namespace,
+            name=stream_name
+        )
+        return self.client.root_range.DeleteStream(
+            request=del_stream_req,
+            metadata=__ROOT_RANGE_METADATA__
+        )
diff --git a/stream/clients/python/bookkeeper/admin/namespaces.py b/stream/clients/python/bookkeeper/admin/namespaces.py
new file mode 100644
index 0000000..7589d08
--- /dev/null
+++ b/stream/clients/python/bookkeeper/admin/namespaces.py
@@ -0,0 +1,53 @@
+# -*- coding: utf-8 -*-
+#
+# 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
+#
+#     https://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 __future__ import absolute_import
+
+from bookkeeper.common.constants import __DEFAULT_NS_CONF__
+from bookkeeper.common.constants import __ROOT_RANGE_METADATA__
+from bookkeeper.proto import storage_pb2
+
+
+class Namespaces(object):
+
+    def __init__(self, client):
+        self.client = client
+
+    def create(self, namespace, namespace_config=__DEFAULT_NS_CONF__):
+        create_ns_req = storage_pb2.CreateNamespaceRequest(
+            name=namespace,
+            ns_conf=namespace_config
+        )
+        return self.client.root_range.CreateNamespace(
+            request=create_ns_req,
+            metadata=__ROOT_RANGE_METADATA__
+        )
+
+    def get(self, namespace):
+        get_ns_req = storage_pb2.GetNamespaceRequest(
+            name=namespace
+        )
+        return self.client.root_range.GetNamespace(
+            request=get_ns_req,
+            metadata=__ROOT_RANGE_METADATA__
+        )
+
+    def delete(self, namespace):
+        del_ns_req = storage_pb2.DeleteNamespaceRequest(
+            name=namespace
+        )
+        return self.client.root_range.DeleteNamespace(
+            request=del_ns_req,
+            metadata=__ROOT_RANGE_METADATA__
+        )
diff --git a/stream/clients/python/bookkeeper/common/__init__.py b/stream/clients/python/bookkeeper/common/__init__.py
new file mode 100644
index 0000000..89c7831
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/__init__.py
@@ -0,0 +1,13 @@
+# -*- coding: utf-8 -*-
+#
+# 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
+#
+#     https://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/stream/clients/python/bookkeeper/common/constants.py b/stream/clients/python/bookkeeper/common/constants.py
new file mode 100644
index 0000000..bec9e83
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/constants.py
@@ -0,0 +1,45 @@
+# -*- coding: utf-8 -*-
+#
+# 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
+#
+#     https://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 bookkeeper.proto import stream_pb2
+
+__ROOT_RANGE_ID__ = 0
+__ROOT_RANGE_METADATA__ = [
+        ('sc-id-bin', __ROOT_RANGE_ID__.to_bytes(8, "little"))
+]
+__DEFAULT_STREAM_CONF__ = stream_pb2.StreamConfiguration(
+        key_type=stream_pb2.RangeKeyType.values()[0],
+        min_num_ranges=24,
+        initial_num_ranges=4,
+        split_policy=stream_pb2.SplitPolicy(
+                type=stream_pb2.SplitPolicyType.values()[0],
+                fixed_range_policy=stream_pb2.FixedRangeSplitPolicy(
+                        num_ranges=2
+                )
+        ),
+        rolling_policy=stream_pb2.SegmentRollingPolicy(
+                size_policy=stream_pb2.SizeBasedSegmentRollingPolicy(
+                        max_segment_size=128*1024*1024
+                )
+        ),
+        retention_policy=stream_pb2.RetentionPolicy(
+                time_policy=stream_pb2.TimeBasedRetentionPolicy(
+                        retention_minutes=-1
+                )
+        ),
+        storage_type=stream_pb2.StorageType.values()[1]
+)
+__DEFAULT_NS_CONF__ = stream_pb2.NamespaceConfiguration(
+        default_stream_conf=__DEFAULT_STREAM_CONF__
+)
diff --git a/stream/clients/python/bookkeeper/common/datetime_helpers.py b/stream/clients/python/bookkeeper/common/datetime_helpers.py
new file mode 100644
index 0000000..a467164
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/datetime_helpers.py
@@ -0,0 +1,224 @@
+# 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.
+
+"""Helpers for :mod:`datetime`."""
+
+import calendar
+import datetime
+import re
+
+import pytz
+
+
+_UTC_EPOCH = datetime.datetime.utcfromtimestamp(0).replace(tzinfo=pytz.utc)
+_RFC3339_MICROS = '%Y-%m-%dT%H:%M:%S.%fZ'
+_RFC3339_NO_FRACTION = '%Y-%m-%dT%H:%M:%S'
+# datetime.strptime cannot handle nanosecond precision:  parse w/ regex
+_RFC3339_NANOS = re.compile(r"""
+    (?P<no_fraction>
+        \d{4}-\d{2}-\d{2}T\d{2}:\d{2}:\d{2}  # YYYY-MM-DDTHH:MM:SS
+    )
+    (                                        # Optional decimal part
+     \.                                      # decimal point
+     (?P<nanos>\d{1,9})                      # nanoseconds, maybe truncated
+    )?
+    Z                                        # Zulu
+""", re.VERBOSE)
+
+
+def utcnow():
+    """A :meth:`datetime.datetime.utcnow()` alias to allow mocking in tests."""
+    return datetime.datetime.utcnow()
+
+
+def to_milliseconds(value):
+    """Convert a zone-aware datetime to milliseconds since the unix epoch.
+    Args:
+        value (datetime.datetime): The datetime to covert.
+    Returns:
+        int: Milliseconds since the unix epoch.
+    """
+    micros = to_microseconds(value)
+    return micros // 1000
+
+
+def from_microseconds(value):
+    """Convert timestamp in microseconds since the unix epoch to datetime.
+    Args:
+        value (float): The timestamp to convert, in microseconds.
+    Returns:
+        datetime.datetime: The datetime object equivalent to the timestamp in
+            UTC.
+    """
+    return _UTC_EPOCH + datetime.timedelta(microseconds=value)
+
+
+def to_microseconds(value):
+    """Convert a datetime to microseconds since the unix epoch.
+    Args:
+        value (datetime.datetime): The datetime to covert.
+    Returns:
+        int: Microseconds since the unix epoch.
+    """
+    if not value.tzinfo:
+        value = value.replace(tzinfo=pytz.utc)
+    # Regardless of what timezone is on the value, convert it to UTC.
+    value = value.astimezone(pytz.utc)
+    # Convert the datetime to a microsecond timestamp.
+    return int(calendar.timegm(value.timetuple()) * 1e6) + value.microsecond
+
+
+def from_iso8601_date(value):
+    """Convert a ISO8601 date string to a date.
+    Args:
+        value (str): The ISO8601 date string.
+    Returns:
+        datetime.date: A date equivalent to the date string.
+    """
+    return datetime.datetime.strptime(value, '%Y-%m-%d').date()
+
+
+def from_iso8601_time(value):
+    """Convert a zoneless ISO8601 time string to a time.
+    Args:
+        value (str): The ISO8601 time string.
+    Returns:
+        datetime.time: A time equivalent to the time string.
+    """
+    return datetime.datetime.strptime(value, '%H:%M:%S').time()
+
+
+def from_rfc3339(value):
+    """Convert a microsecond-precision timestamp to datetime.
+    Args:
+        value (str): The RFC3339 string to convert.
+    Returns:
+        datetime.datetime: The datetime object equivalent to the timestamp in
+            UTC.
+    """
+    return datetime.datetime.strptime(
+        value, _RFC3339_MICROS).replace(tzinfo=pytz.utc)
+
+
+def from_rfc3339_nanos(value):
+    """Convert a nanosecond-precision timestamp to a native datetime.
+    .. note::
+        Python datetimes do not support nanosecond precision; this function
+        therefore truncates such values to microseconds.
+    Args:
+        value (str): The RFC3339 string to convert.
+    Returns:
+        datetime.datetime: The datetime object equivalent to the timestamp in
+            UTC.
+    Raises:
+        ValueError: If the timestamp does not match the RFC 3339
+            regular expression.
+    """
+    with_nanos = _RFC3339_NANOS.match(value)
+
+    if with_nanos is None:
+        raise ValueError(
+            'Timestamp: {!r}, does not match pattern: {!r}'.format(
+                value, _RFC3339_NANOS.pattern))
+
+    bare_seconds = datetime.datetime.strptime(
+        with_nanos.group('no_fraction'), _RFC3339_NO_FRACTION)
+    fraction = with_nanos.group('nanos')
+
+    if fraction is None:
+        micros = 0
+    else:
+        scale = 9 - len(fraction)
+        nanos = int(fraction) * (10 ** scale)
+        micros = nanos // 1000
+
+    return bare_seconds.replace(microsecond=micros, tzinfo=pytz.utc)
+
+
+def to_rfc3339(value, ignore_zone=True):
+    """Convert a datetime to an RFC3339 timestamp string.
+    Args:
+        value (datetime.datetime):
+            The datetime object to be converted to a string.
+        ignore_zone (bool): If True, then the timezone (if any) of the
+            datetime object is ignored and the datetime is treated as UTC.
+    Returns:
+        str: The RFC3339 formated string representing the datetime.
+    """
+    if not ignore_zone and value.tzinfo is not None:
+        # Convert to UTC and remove the time zone info.
+        value = value.replace(tzinfo=None) - value.utcoffset()
+
+    return value.strftime(_RFC3339_MICROS)
+
+
+class DatetimeWithNanoseconds(datetime.datetime):
+    """Track nanosecond in addition to normal datetime attrs.
+    Nanosecond can be passed only as a keyword argument.
+    """
+    __slots__ = ('_nanosecond',)
+
+    # pylint: disable=arguments-differ
+    def __new__(cls, *args, **kw):
+        nanos = kw.pop('nanosecond', 0)
+        if nanos > 0:
+            if 'microsecond' in kw:
+                raise TypeError(
+                    "Specify only one of 'microsecond' or 'nanosecond'")
+            kw['microsecond'] = nanos // 1000
+        inst = datetime.datetime.__new__(cls, *args, **kw)
+        inst._nanosecond = nanos or 0
+        return inst
+    # pylint: disable=arguments-differ
+
+    @property
+    def nanosecond(self):
+        """Read-only: nanosecond precision."""
+        return self._nanosecond
+
+    def rfc3339(self):
+        """Return an RFC 3339-compliant timestamp.
+        Returns:
+            (str): Timestamp string according to RFC 3339 spec.
+        """
+        if self._nanosecond == 0:
+            return to_rfc3339(self)
+        nanos = str(self._nanosecond).rstrip('0')
+        return '{}.{}Z'.format(self.strftime(_RFC3339_NO_FRACTION), nanos)
+
+    @classmethod
+    def from_rfc3339(cls, stamp):
+        """Parse RFC 3339-compliant timestamp, preserving nanoseconds.
+        Args:
+            stamp (str): RFC 3339 stamp, with up to nanosecond precision
+        Returns:
+            :class:`DatetimeWithNanoseconds`:
+                an instance matching the timestamp string
+        Raises:
+            ValueError: if `stamp` does not match the expected format
+        """
+        with_nanos = _RFC3339_NANOS.match(stamp)
+        if with_nanos is None:
+            raise ValueError(
+                'Timestamp: {}, does not match pattern: {}'.format(
+                    stamp, _RFC3339_NANOS.pattern))
+        bare = datetime.datetime.strptime(
+            with_nanos.group('no_fraction'), _RFC3339_NO_FRACTION)
+        fraction = with_nanos.group('nanos')
+        if fraction is None:
+            nanos = 0
+        else:
+            scale = 9 - len(fraction)
+            nanos = int(fraction) * (10 ** scale)
+        return cls(bare.year, bare.month, bare.day,
+                   bare.hour, bare.minute, bare.second,
+                   nanosecond=nanos, tzinfo=pytz.UTC)
diff --git a/stream/clients/python/bookkeeper/common/exceptions.py b/stream/clients/python/bookkeeper/common/exceptions.py
new file mode 100644
index 0000000..34ff122
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/exceptions.py
@@ -0,0 +1,185 @@
+# 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.
+
+"""Exceptions raised by bookkeeper clients.
+This module provides base classes for all errors raised by libraries based
+on :mod:`bookkeeper.common`.
+"""
+
+from __future__ import absolute_import
+from __future__ import unicode_literals
+from bookkeeper.proto.storage_pb2 import BAD_REQUEST
+from bookkeeper.proto.storage_pb2 import INTERNAL_SERVER_ERROR
+
+import six
+
+try:
+    import grpc
+except ImportError:  # pragma: NO COVER
+    grpc = None
+
+# Lookup tables for mapping exceptions from gRPC transports.
+# Populated by _APICallErrorMeta
+_GRPC_CODE_TO_EXCEPTION = {}
+
+
+class BKError(Exception):
+    """Base class for all exceptions raised by bookkeeper Clients."""
+    pass
+
+
+@six.python_2_unicode_compatible
+class RetryError(BKError):
+    """Raised when a function has exhausted all of its available retries.
+    Args:
+        message (str): The exception message.
+        cause (Exception): The last exception raised when retring the
+            function.
+    """
+    def __init__(self, message, cause):
+        super(RetryError, self).__init__(message)
+        self.message = message
+        self._cause = cause
+
+    @property
+    def cause(self):
+        """The last exception raised when retrying the function."""
+        return self._cause
+
+    def __str__(self):
+        return '{}, last exception: {}'.format(self.message, self.cause)
+
+
+class _BKGRpcCallErrorMeta(type):
+    """Metaclass for registering BKGRpcCallError subclasses."""
+    def __new__(mcs, name, bases, class_dict):
+        cls = type.__new__(mcs, name, bases, class_dict)
+        if cls.grpc_status_code is not None:
+            _GRPC_CODE_TO_EXCEPTION.setdefault(cls.grpc_status_code, cls)
+        return cls
+
+
+@six.python_2_unicode_compatible
+@six.add_metaclass(_BKGRpcCallErrorMeta)
+class BKGrpcCallError(BKError):
+    """Base class for exceptions raised by calling API methods.
+    Args:
+        message (str): The exception message.
+        errors (Sequence[Any]): An optional list of error details.
+        response (Union[requests.Request, grpc.Call]): The response or
+            gRPC call metadata.
+    """
+
+    grpc_status_code = None
+    """Optional[grpc.StatusCode]: The gRPC status code associated with this
+    error.
+    This may be ``None`` if the exception does not match up to a gRPC error.
+    """
+
+    bk_status_code = None
+    """Optional[bookkeeper.proto.StatusCode]: The bookkeeper storage status code
+    associated with this error.
+    This may be ```None` if the exception is a gRPC channel error.
+    """
+
+    def __init__(self, message, errors=(), response=None):
+        super(BKGrpcCallError, self).__init__(message)
+        self.message = message
+        """str: The exception message."""
+        self._errors = errors
+        self._response = response
+
+    def __str__(self):
+        return '{} {}'.format(self.code, self.message)
+
+    @property
+    def errors(self):
+        """Detailed error information.
+        Returns:
+            Sequence[Any]: A list of additional error details.
+        """
+        return list(self._errors)
+
+    @property
+    def response(self):
+        """Optional[Union[requests.Request, grpc.Call]]: The response or
+        gRPC call metadata."""
+        return self._response
+
+
+class ClientError(BKGrpcCallError):
+    """Base class for all client error responses."""
+
+
+class BadRequest(ClientError):
+    """Exception mapping a ``400 Bad Request`` response."""
+    code = BAD_REQUEST
+
+
+class ServerError(BKGrpcCallError):
+    """Base for 5xx responses."""
+
+
+class InternalServerError(ServerError):
+    """Exception mapping a ``500 Internal Server Error`` response. or a
+    :attr:`grpc.StatusCode.INTERNAL` error."""
+    code = INTERNAL_SERVER_ERROR
+    grpc_status_code = grpc.StatusCode.INTERNAL if grpc is not None else None
+
+
+def exception_class_for_grpc_status(status_code):
+    """Return the exception class for a specific :class:`grpc.StatusCode`.
+    Args:
+        status_code (grpc.StatusCode): The gRPC status code.
+    Returns:
+        :func:`type`: the appropriate subclass of :class:`BKGrpcCallError`.
+    """
+    return _GRPC_CODE_TO_EXCEPTION.get(status_code, BKGrpcCallError)
+
+
+def from_grpc_status(status_code, message, **kwargs):
+    """Create a :class:`BKGrpcCallError` from a :class:`grpc.StatusCode`.
+    Args:
+        status_code (grpc.StatusCode): The gRPC status code.
+        message (str): The exception message.
+        kwargs: Additional arguments passed to the :class:`BKGrpcCallError`
+            constructor.
+    Returns:
+        BKGrpcCallError: An instance of the appropriate subclass of
+            :class:`BKGrpcCallError`.
+    """
+    error_class = exception_class_for_grpc_status(status_code)
+    error = error_class(message, **kwargs)
+
+    if error.grpc_status_code is None:
+        error.grpc_status_code = status_code
+
+    return error
+
+
+def from_grpc_error(rpc_exc):
+    """Create a :class:`BKGrpcCallError` from a :class:`grpc.RpcError`.
+    Args:
+        rpc_exc (grpc.RpcError): The gRPC error.
+    Returns:
+        BKGrpcCallError: An instance of the appropriate subclass of
+            :class:`BKGrpcError`.
+    """
+    if isinstance(rpc_exc, grpc.Call):
+        return from_grpc_status(
+            rpc_exc.code(),
+            rpc_exc.details(),
+            errors=(rpc_exc,),
+            response=rpc_exc)
+    else:
+        return BKGrpcCallError(
+            str(rpc_exc), errors=(rpc_exc,), response=rpc_exc)
diff --git a/stream/clients/python/bookkeeper/common/future/__init__.py b/stream/clients/python/bookkeeper/common/future/__init__.py
new file mode 100644
index 0000000..f9d985c
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/future/__init__.py
@@ -0,0 +1,19 @@
+# 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.
+
+"""Futures for dealing with asynchronous operations."""
+
+from bookkeeper.common.future.base import Future
+
+__all__ = [
+    'Future',
+]
diff --git a/stream/clients/python/bookkeeper/common/future/_helpers.py b/stream/clients/python/bookkeeper/common/future/_helpers.py
new file mode 100644
index 0000000..0aa841d
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/future/_helpers.py
@@ -0,0 +1,37 @@
+# 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.
+
+"""Private helpers for futures."""
+
+import logging
+import threading
+
+
+_LOGGER = logging.getLogger(__name__)
+
+
+def start_daemon_thread(*args, **kwargs):
+    """Starts a thread and marks it as a daemon thread."""
+    thread = threading.Thread(*args, **kwargs)
+    thread.daemon = True
+    thread.start()
+    return thread
+
+
+def safe_invoke_callback(callback, *args, **kwargs):
+    """Invoke a callback, swallowing and logging any exceptions."""
+    # pylint: disable=bare-except
+    # We intentionally want to swallow all exceptions.
+    try:
+        return callback(*args, **kwargs)
+    except Exception:
+        _LOGGER.exception('Error while executing Future callback.')
diff --git a/stream/clients/python/bookkeeper/common/future/base.py b/stream/clients/python/bookkeeper/common/future/base.py
new file mode 100644
index 0000000..b548c43
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/future/base.py
@@ -0,0 +1,64 @@
+# 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.
+
+"""Abstract and helper bases for Future implementations."""
+
+import abc
+
+import six
+
+
+@six.add_metaclass(abc.ABCMeta)
+class Future(object):
+    # pylint: disable=missing-docstring
+    # We inherit the interfaces here from concurrent.futures.
+
+    """Future interface.
+    This interface is based on :class:`concurrent.futures.Future`.
+    """
+
+    @abc.abstractmethod
+    def cancel(self):
+        raise NotImplementedError()
+
+    @abc.abstractmethod
+    def cancelled(self):
+        raise NotImplementedError()
+
+    @abc.abstractmethod
+    def running(self):
+        raise NotImplementedError()
+
+    @abc.abstractmethod
+    def done(self):
+        raise NotImplementedError()
+
+    @abc.abstractmethod
+    def result(self, timeout=None):
+        raise NotImplementedError()
+
+    @abc.abstractmethod
+    def exception(self, timeout=None):
+        raise NotImplementedError()
+
+    @abc.abstractmethod
+    def add_done_callback(self, fn):
+        # pylint: disable=invalid-name
+        raise NotImplementedError()
+
+    @abc.abstractmethod
+    def set_result(self, result):
+        raise NotImplementedError()
+
+    @abc.abstractmethod
+    def set_exception(self, exception):
+        raise NotImplementedError()
diff --git a/stream/clients/python/bookkeeper/common/future/polling.py b/stream/clients/python/bookkeeper/common/future/polling.py
new file mode 100644
index 0000000..2bde3a8
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/future/polling.py
@@ -0,0 +1,161 @@
+# 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.
+
+"""Abstract and helper bases for Future implementations."""
+
+import abc
+import concurrent.futures
+
+from bookkeeper.common import exceptions
+from bookkeeper.common import retry
+from bookkeeper.common.future import _helpers
+from bookkeeper.common.future import base
+
+
+class _OperationNotComplete(Exception):
+    """Private exception used for polling via retry."""
+    pass
+
+
+RETRY_PREDICATE = retry.if_exception_type(_OperationNotComplete)
+DEFAULT_RETRY = retry.Retry(predicate=RETRY_PREDICATE)
+
+
+class PollingFuture(base.Future):
+    """A Future that needs to poll some service to check its status.
+    The :meth:`done` method should be implemented by subclasses. The polling
+    behavior will repeatedly call ``done`` until it returns True.
+    .. note: Privacy here is intended to prevent the final class from
+    overexposing, not to prevent subclasses from accessing methods.
+    Args:
+        retry (bookkeeper.common.retry.Retry): The retry configuration used
+            when polling. This can be used to control how often :meth:`done`
+            is polled. Regardless of the retry's ``deadline``, it will be
+            overridden by the ``timeout`` argument to :meth:`result`.
+    """
+    def __init__(self, retry=DEFAULT_RETRY):
+        super(PollingFuture, self).__init__()
+        self._retry = retry
+        self._result = None
+        self._exception = None
+        self._result_set = False
+        """bool: Set to True when the result has been set via set_result or
+        set_exception."""
+        self._polling_thread = None
+        self._done_callbacks = []
+
+    @abc.abstractmethod
+    def done(self):
+        """Checks to see if the operation is complete.
+        Returns:
+            bool: True if the operation is complete, False otherwise.
+        """
+        # pylint: disable=redundant-returns-doc, missing-raises-doc
+        raise NotImplementedError()
+
+    def _done_or_raise(self):
+        """Check if the future is done and raise if it's not."""
+        if not self.done():
+            raise _OperationNotComplete()
+
+    def running(self):
+        """True if the operation is currently running."""
+        return not self.done()
+
+    def _blocking_poll(self, timeout=None):
+        """Poll and wait for the Future to be resolved.
+        Args:
+            timeout (int):
+                How long (in seconds) to wait for the operation to complete.
+                If None, wait indefinitely.
+        """
+        if self._result_set:
+            return
+
+        retry_ = self._retry.with_deadline(timeout)
+
+        try:
+            retry_(self._done_or_raise)()
+        except exceptions.RetryError:
+            raise concurrent.futures.TimeoutError(
+                'Operation did not complete within the designated '
+                'timeout.')
+
+    def result(self, timeout=None):
+        """Get the result of the operation, blocking if necessary.
+        Args:
+            timeout (int):
+                How long (in seconds) to wait for the operation to complete.
+                If None, wait indefinitely.
+        Returns:
+            google.protobuf.Message: The Operation's result.
+        Raises:
+            bookkeeper.common.exceptions.BKGrpcAPICallError: If the operation errors or if
+                the timeout is reached before the operation completes.
+        """
+        self._blocking_poll(timeout=timeout)
+
+        if self._exception is not None:
+            # pylint: disable=raising-bad-type
+            # Pylint doesn't recognize that this is valid in this case.
+            raise self._exception
+
+        return self._result
+
+    def exception(self, timeout=None):
+        """Get the exception from the operation, blocking if necessary.
+        Args:
+            timeout (int): How long to wait for the operation to complete.
+                If None, wait indefinitely.
+        Returns:
+            Optional[bookkeeper.common.exceptions.BKGrpcAPICallError]: The operation's
+                error.
+        """
+        self._blocking_poll()
+        return self._exception
+
+    def add_done_callback(self, fn):
+        """Add a callback to be executed when the operation is complete.
+        If the operation is not already complete, this will start a helper
+        thread to poll for the status of the operation in the background.
+        Args:
+            fn (Callable[Future]): The callback to execute when the operation
+                is complete.
+        """
+        if self._result_set:
+            _helpers.safe_invoke_callback(fn, self)
+            return
+
+        self._done_callbacks.append(fn)
+
+        if self._polling_thread is None:
+            # The polling thread will exit on its own as soon as the operation
+            # is done.
+            self._polling_thread = _helpers.start_daemon_thread(
+                target=self._blocking_poll)
+
+    def _invoke_callbacks(self, *args, **kwargs):
+        """Invoke all done callbacks."""
+        for callback in self._done_callbacks:
+            _helpers.safe_invoke_callback(callback, *args, **kwargs)
+
+    def set_result(self, result):
+        """Set the Future's result."""
+        self._result = result
+        self._result_set = True
+        self._invoke_callbacks(self)
+
+    def set_exception(self, exception):
+        """Set the Future's exception."""
+        self._exception = exception
+        self._result_set = True
+        self._invoke_callbacks(self)
diff --git a/stream/clients/python/bookkeeper/common/general_helpers.py b/stream/clients/python/bookkeeper/common/general_helpers.py
new file mode 100644
index 0000000..353f5e9
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/general_helpers.py
@@ -0,0 +1,30 @@
+# 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.
+
+"""Helpers for general Python functionality."""
+
+import functools
+
+import six
+
+
+# functools.partial objects lack several attributes present on real function
+# objects. In Python 2 wraps fails on this so use a restricted set instead.
+_PARTIAL_VALID_ASSIGNMENTS = ('__doc__',)
+
+
+def wraps(wrapped):
+    """A functools.wraps helper that handles partial objects on Python 2."""
+    if isinstance(wrapped, functools.partial):
+        return six.wraps(wrapped, assigned=_PARTIAL_VALID_ASSIGNMENTS)
+    else:
+        return six.wraps(wrapped)
diff --git a/stream/clients/python/bookkeeper/common/grpc_helpers.py b/stream/clients/python/bookkeeper/common/grpc_helpers.py
new file mode 100644
index 0000000..9e7cfd0
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/grpc_helpers.py
@@ -0,0 +1,313 @@
+# 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.
+
+"""Helpers for :mod:`grpc`."""
+
+import collections
+
+import grpc
+import six
+
+from bookkeeper.common import exceptions
+from bookkeeper.common import general_helpers
+
+# The list of gRPC Callable interfaces that return iterators.
+_STREAM_WRAP_CLASSES = (
+    grpc.UnaryStreamMultiCallable,
+    grpc.StreamStreamMultiCallable,
+)
+
+
+def _patch_callable_name(callable_):
+    """Fix-up gRPC callable attributes.
+    gRPC callable lack the ``__name__`` attribute which causes
+    :func:`functools.wraps` to error. This adds the attribute if needed.
+    """
+    if not hasattr(callable_, '__name__'):
+        callable_.__name__ = callable_.__class__.__name__
+
+
+def _wrap_unary_errors(callable_):
+    """Map errors for Unary-Unary and Stream-Unary gRPC callables."""
+    _patch_callable_name(callable_)
+
+    @six.wraps(callable_)
+    def error_remapped_callable(*args, **kwargs):
+        try:
+            return callable_(*args, **kwargs)
+        except grpc.RpcError as exc:
+            six.raise_from(exceptions.from_grpc_error(exc), exc)
+
+    return error_remapped_callable
+
+
+class _StreamingResponseIterator(grpc.Call):
+    def __init__(self, wrapped):
+        self._wrapped = wrapped
+
+    def __iter__(self):
+        """This iterator is also an iterable that returns itself."""
+        return self
+
+    def next(self):
+        """Get the next response from the stream.
+        Returns:
+            protobuf.Message: A single response from the stream.
+        """
+        try:
+            return six.next(self._wrapped)
+        except grpc.RpcError as exc:
+            six.raise_from(exceptions.from_grpc_error(exc), exc)
+
+    # Alias needed for Python 2/3 support.
+    __next__ = next
+
+    # grpc.Call & grpc.RpcContext interface
+
+    def add_callback(self, callback):
+        return self._wrapped.add_callback(callback)
+
+    def cancel(self):
+        return self._wrapped.cancel()
+
+    def code(self):
+        return self._wrapped.code()
+
+    def details(self):
+        return self._wrapped.details()
+
+    def initial_metadata(self):
+        return self._wrapped.initial_metadata()
+
+    def is_active(self):
+        return self._wrapped.is_active()
+
+    def time_remaining(self):
+        return self._wrapped.time_remaining()
+
+    def trailing_metadata(self):
+        return self._wrapped.trailing_metadata()
+
+
+def _wrap_stream_errors(callable_):
+    """Wrap errors for Unary-Stream and Stream-Stream gRPC callables.
+    The callables that return iterators require a bit more logic to re-map
+    errors when iterating. This wraps both the initial invocation and the
+    iterator of the return value to re-map errors.
+    """
+    _patch_callable_name(callable_)
+
+    @general_helpers.wraps(callable_)
+    def error_remapped_callable(*args, **kwargs):
+        try:
+            result = callable_(*args, **kwargs)
+            return _StreamingResponseIterator(result)
+        except grpc.RpcError as exc:
+            six.raise_from(exceptions.from_grpc_error(exc), exc)
+
+    return error_remapped_callable
+
+
+def wrap_errors(callable_):
+    """Wrap a gRPC callable and map :class:`grpc.RpcErrors` to friendly error
+    classes.
+    Errors raised by the gRPC callable are mapped to the appropriate
+    :class:`bookkeeper.common.exceptions.BKGrpcAPICallError` subclasses.
+    The original `grpc.RpcError` (which is usually also a `grpc.Call`) is
+    available from the ``response`` property on the mapped exception. This
+    is useful for extracting metadata from the original error.
+    Args:
+        callable_ (Callable): A gRPC callable.
+    Returns:
+        Callable: The wrapped gRPC callable.
+    """
+    if isinstance(callable_, _STREAM_WRAP_CLASSES):
+        return _wrap_stream_errors(callable_)
+    else:
+        return _wrap_unary_errors(callable_)
+
+
+def create_channel(target,
+                   **kwargs):
+    """Create a secure channel with credentials.
+    Args:
+        target (str): The target service address in the format 'hostname:port'.
+        kwargs: Additional key-word args passed to
+            :func:`grpc_gcp.secure_channel` or :func:`grpc.secure_channel`.
+    Returns:
+        grpc.Channel: The created channel.
+    """
+    return grpc.secure_channel(target, None, **kwargs)
+
+
+_MethodCall = collections.namedtuple(
+    '_MethodCall', ('request', 'timeout', 'metadata', 'credentials'))
+
+_ChannelRequest = collections.namedtuple(
+    '_ChannelRequest', ('method', 'request'))
+
+
+class _CallableStub(object):
+    """Stub for the grpc.*MultiCallable interfaces."""
+
+    def __init__(self, method, channel):
+        self._method = method
+        self._channel = channel
+        self.response = None
+        """Union[protobuf.Message, Callable[protobuf.Message], exception]:
+        The response to give when invoking this callable. If this is a
+        callable, it will be invoked with the request protobuf. If it's an
+        exception, the exception will be raised when this is invoked.
+        """
+        self.responses = None
+        """Iterator[
+            Union[protobuf.Message, Callable[protobuf.Message], exception]]:
+        An iterator of responses. If specified, self.response will be populated
+        on each invocation by calling ``next(self.responses)``."""
+        self.requests = []
+        """List[protobuf.Message]: All requests sent to this callable."""
+        self.calls = []
+        """List[Tuple]: All invocations of this callable. Each tuple is the
+        request, timeout, metadata, and credentials."""
+
+    def __call__(self, request, timeout=None, metadata=None, credentials=None):
+        self._channel.requests.append(
+            _ChannelRequest(self._method, request))
+        self.calls.append(
+            _MethodCall(request, timeout, metadata, credentials))
+        self.requests.append(request)
+
+        response = self.response
+        if self.responses is not None:
+            if response is None:
+                response = next(self.responses)
+            else:
+                raise ValueError(
+                    '{method}.response and {method}.responses are mutually '
+                    'exclusive.'.format(method=self._method))
+
+        if callable(response):
+            return response(request)
+
+        if isinstance(response, Exception):
+            raise response
+
+        if response is not None:
+            return response
+
+        raise ValueError(
+            'Method stub for "{}" has no response.'.format(self._method))
+
+
+def _simplify_method_name(method):
+    """Simplifies a gRPC method name.
+    When gRPC invokes the channel to create a callable, it gives a full
+    method name like "/org.apache.bookkeeper.Table/Put". This
+    returns just the name of the method, in this case "Put".
+    Args:
+        method (str): The name of the method.
+    Returns:
+        str: The simplified name of the method.
+    """
+    return method.rsplit('/', 1).pop()
+
+
+class ChannelStub(grpc.Channel):
+    """A testing stub for the grpc.Channel interface.
+    This can be used to test any client that eventually uses a gRPC channel
+    to communicate. By passing in a channel stub, you can configure which
+    responses are returned and track which requests are made.
+    For example:
+    .. code-block:: python
+        channel_stub = grpc_helpers.ChannelStub()
+        client = FooClient(channel=channel_stub)
+        channel_stub.GetFoo.response = foo_pb2.Foo(name='bar')
+        foo = client.get_foo(labels=['baz'])
+        assert foo.name == 'bar'
+        assert channel_stub.GetFoo.requests[0].labels = ['baz']
+    Each method on the stub can be accessed and configured on the channel.
+    Here's some examples of various configurations:
+    .. code-block:: python
+        # Return a basic response:
+        channel_stub.GetFoo.response = foo_pb2.Foo(name='bar')
+        assert client.get_foo().name == 'bar'
+        # Raise an exception:
+        channel_stub.GetFoo.response = NotFound('...')
+        with pytest.raises(NotFound):
+            client.get_foo()
+        # Use a sequence of responses:
+        channel_stub.GetFoo.responses = iter([
+            foo_pb2.Foo(name='bar'),
+            foo_pb2.Foo(name='baz'),
+        ])
+        assert client.get_foo().name == 'bar'
+        assert client.get_foo().name == 'baz'
+        # Use a callable
+        def on_get_foo(request):
+            return foo_pb2.Foo(name='bar' + request.id)
+        channel_stub.GetFoo.response = on_get_foo
+        assert client.get_foo(id='123').name == 'bar123'
+    """
+
+    def __init__(self, responses=[]):
+        self.requests = []
+        """Sequence[Tuple[str, protobuf.Message]]: A list of all requests made
+        on this channel in order. The tuple is of method name, request
+        message."""
+        self._method_stubs = {}
+
+    def _stub_for_method(self, method):
+        method = _simplify_method_name(method)
+        self._method_stubs[method] = _CallableStub(method, self)
+        return self._method_stubs[method]
+
+    def __getattr__(self, key):
+        try:
+            return self._method_stubs[key]
+        except KeyError:
+            raise AttributeError
+
+    def unary_unary(
+            self, method,
+            request_serializer=None, response_deserializer=None):
+        """grpc.Channel.unary_unary implementation."""
+        return self._stub_for_method(method)
+
+    def unary_stream(
+            self, method,
+            request_serializer=None, response_deserializer=None):
+        """grpc.Channel.unary_stream implementation."""
+        return self._stub_for_method(method)
+
+    def stream_unary(
+            self, method,
+            request_serializer=None, response_deserializer=None):
+        """grpc.Channel.stream_unary implementation."""
+        return self._stub_for_method(method)
+
+    def stream_stream(
+            self, method,
+            request_serializer=None, response_deserializer=None):
+        """grpc.Channel.stream_stream implementation."""
+        return self._stub_for_method(method)
+
+    def subscribe(self, callback, try_to_connect=False):
+        """grpc.Channel.subscribe implementation."""
+        pass
+
+    def unsubscribe(self, callback):
+        """grpc.Channel.unsubscribe implementation."""
+        pass
+
+    def close(self):
+        """grpc.Channel.close implementation."""
+        pass
diff --git a/stream/clients/python/bookkeeper/common/protobuf_helpers.py b/stream/clients/python/bookkeeper/common/protobuf_helpers.py
new file mode 100644
index 0000000..fcba501
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/protobuf_helpers.py
@@ -0,0 +1,321 @@
+# 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.
+
+"""Helpers for :mod:`protobuf`."""
+
+import collections
+import copy
+import inspect
+
+from google.protobuf import field_mask_pb2
+from google.protobuf import message
+from google.protobuf import wrappers_pb2
+
+_SENTINEL = object()
+_WRAPPER_TYPES = (
+    wrappers_pb2.BoolValue,
+    wrappers_pb2.BytesValue,
+    wrappers_pb2.DoubleValue,
+    wrappers_pb2.FloatValue,
+    wrappers_pb2.Int32Value,
+    wrappers_pb2.Int64Value,
+    wrappers_pb2.StringValue,
+    wrappers_pb2.UInt32Value,
+    wrappers_pb2.UInt64Value,
+)
+
+
+def from_any_pb(pb_type, any_pb):
+    """Converts an ``Any`` protobuf to the specified message type.
+    Args:
+        pb_type (type): the type of the message that any_pb stores an instance
+            of.
+        any_pb (google.protobuf.any_pb2.Any): the object to be converted.
+    Returns:
+        pb_type: An instance of the pb_type message.
+    Raises:
+        TypeError: if the message could not be converted.
+    """
+    msg = pb_type()
+    if not any_pb.Unpack(msg):
+        raise TypeError(
+            'Could not convert {} to {}'.format(
+                any_pb.__class__.__name__, pb_type.__name__))
+
+    return msg
+
+
+def check_oneof(**kwargs):
+    """Raise ValueError if more than one keyword argument is not ``None``.
+    Args:
+        kwargs (dict): The keyword arguments sent to the function.
+    Raises:
+        ValueError: If more than one entry in ``kwargs`` is not ``None``.
+    """
+    # Sanity check: If no keyword arguments were sent, this is fine.
+    if not kwargs:
+        return
+
+    not_nones = [val for val in kwargs.values() if val is not None]
+    if len(not_nones) > 1:
+        raise ValueError('Only one of {fields} should be set.'.format(
+            fields=', '.join(sorted(kwargs.keys())),
+        ))
+
+
+def get_messages(module):
+    """Discovers all protobuf Message classes in a given import module.
+    Args:
+        module (module): A Python module; :func:`dir` will be run against this
+            module to find Message subclasses.
+    Returns:
+        dict[str, google.protobuf.message.Message]: A dictionary with the
+            Message class names as keys, and the Message subclasses themselves
+            as values.
+    """
+    answer = collections.OrderedDict()
+    for name in dir(module):
+        candidate = getattr(module, name)
+        if (inspect.isclass(candidate) and
+                issubclass(candidate, message.Message)):
+            answer[name] = candidate
+    return answer
+
+
+def _resolve_subkeys(key, separator='.'):
+    """Resolve a potentially nested key.
+    If the key contains the ``separator`` (e.g. ``.``) then the key will be
+    split on the first instance of the subkey::
+       >>> _resolve_subkeys('a.b.c')
+       ('a', 'b.c')
+       >>> _resolve_subkeys('d|e|f', separator='|')
+       ('d', 'e|f')
+    If not, the subkey will be :data:`None`::
+        >>> _resolve_subkeys('foo')
+        ('foo', None)
+    Args:
+        key (str): A string that may or may not contain the separator.
+        separator (str): The namespace separator. Defaults to `.`.
+    Returns:
+        Tuple[str, str]: The key and subkey(s).
+    """
+    parts = key.split(separator, 1)
+
+    if len(parts) > 1:
+        return parts
+    else:
+        return parts[0], None
+
+
+def get(msg_or_dict, key, default=_SENTINEL):
+    """Retrieve a key's value from a protobuf Message or dictionary.
+    Args:
+        mdg_or_dict (Union[~google.protobuf.message.Message, Mapping]): the
+            object.
+        key (str): The key to retrieve from the object.
+        default (Any): If the key is not present on the object, and a default
+            is set, returns that default instead. A type-appropriate falsy
+            default is generally recommended, as protobuf messages almost
+            always have default values for unset values and it is not always
+            possible to tell the difference between a falsy value and an
+            unset one. If no default is set then :class:`KeyError` will be
+            raised if the key is not present in the object.
+    Returns:
+        Any: The return value from the underlying Message or dict.
+    Raises:
+        KeyError: If the key is not found. Note that, for unset values,
+            messages and dictionaries may not have consistent behavior.
+        TypeError: If ``msg_or_dict`` is not a Message or Mapping.
+    """
+    # We may need to get a nested key. Resolve this.
+    key, subkey = _resolve_subkeys(key)
+
+    # Attempt to get the value from the two types of objects we know about.
+    # If we get something else, complain.
+    if isinstance(msg_or_dict, message.Message):
+        answer = getattr(msg_or_dict, key, default)
+    elif isinstance(msg_or_dict, collections.Mapping):
+        answer = msg_or_dict.get(key, default)
+    else:
+        raise TypeError(
+            'get() expected a dict or protobuf message, got {!r}.'.format(
+                type(msg_or_dict)))
+
+    # If the object we got back is our sentinel, raise KeyError; this is
+    # a "not found" case.
+    if answer is _SENTINEL:
+        raise KeyError(key)
+
+    # If a subkey exists, call this method recursively against the answer.
+    if subkey is not None and answer is not default:
+        return get(answer, subkey, default=default)
+
+    return answer
+
+
+def _set_field_on_message(msg, key, value):
+    """Set helper for protobuf Messages."""
+    # Attempt to set the value on the types of objects we know how to deal
+    # with.
+    if isinstance(value, (collections.MutableSequence, tuple)):
+        # Clear the existing repeated protobuf message of any elements
+        # currently inside it.
+        while getattr(msg, key):
+            getattr(msg, key).pop()
+
+        # Write our new elements to the repeated field.
+        for item in value:
+            if isinstance(item, collections.Mapping):
+                getattr(msg, key).add(**item)
+            else:
+                # protobuf's RepeatedCompositeContainer doesn't support
+                # append.
+                getattr(msg, key).extend([item])
+    elif isinstance(value, collections.Mapping):
+        # Assign the dictionary values to the protobuf message.
+        for item_key, item_value in value.items():
+            set(getattr(msg, key), item_key, item_value)
+    elif isinstance(value, message.Message):
+        getattr(msg, key).CopyFrom(value)
+    else:
+        setattr(msg, key, value)
+
+
+def set(msg_or_dict, key, value):
+    """Set a key's value on a protobuf Message or dictionary.
+    Args:
+        msg_or_dict (Union[~google.protobuf.message.Message, Mapping]): the
+            object.
+        key (str): The key to set.
+        value (Any): The value to set.
+    Raises:
+        TypeError: If ``msg_or_dict`` is not a Message or dictionary.
+    """
+    # Sanity check: Is our target object valid?
+    if (not isinstance(msg_or_dict,
+                       (collections.MutableMapping, message.Message))):
+        raise TypeError(
+            'set() expected a dict or protobuf message, got {!r}.'.format(
+                type(msg_or_dict)))
+
+    # We may be setting a nested key. Resolve this.
+    basekey, subkey = _resolve_subkeys(key)
+
+    # If a subkey exists, then get that object and call this method
+    # recursively against it using the subkey.
+    if subkey is not None:
+        if isinstance(msg_or_dict, collections.MutableMapping):
+            msg_or_dict.setdefault(basekey, {})
+        set(get(msg_or_dict, basekey), subkey, value)
+        return
+
+    if isinstance(msg_or_dict, collections.MutableMapping):
+        msg_or_dict[key] = value
+    else:
+        _set_field_on_message(msg_or_dict, key, value)
+
+
+def setdefault(msg_or_dict, key, value):
+    """Set the key on a protobuf Message or dictionary to a given value if the
+    current value is falsy.
+    Because protobuf Messages do not distinguish between unset values and
+    falsy ones particularly well (by design), this method treats any falsy
+    value (e.g. 0, empty list) as a target to be overwritten, on both Messages
+    and dictionaries.
+    Args:
+        msg_or_dict (Union[~google.protobuf.message.Message, Mapping]): the
+            object.
+        key (str): The key on the object in question.
+        value (Any): The value to set.
+    Raises:
+        TypeError: If ``msg_or_dict`` is not a Message or dictionary.
+    """
+    if not get(msg_or_dict, key, default=None):
+        set(msg_or_dict, key, value)
+
+
+def field_mask(original, modified):
+    """Create a field mask by comparing two messages.
+    Args:
+        original (~google.protobuf.message.Message): the original message.
+            If set to None, this field will be interpretted as an empty
+            message.
+        modified (~google.protobuf.message.Message): the modified message.
+            If set to None, this field will be interpretted as an empty
+            message.
+    Returns:
+        google.protobuf.field_mask_pb2.FieldMask: field mask that contains
+        the list of field names that have different values between the two
+        messages. If the messages are equivalent, then the field mask is empty.
+    Raises:
+        ValueError: If the ``original`` or ``modified`` are not the same type.
+    """
+    if original is None and modified is None:
+        return field_mask_pb2.FieldMask()
+
+    if original is None and modified is not None:
+        original = copy.deepcopy(modified)
+        original.Clear()
+
+    if modified is None and original is not None:
+        modified = copy.deepcopy(original)
+        modified.Clear()
+
+    if type(original) != type(modified):
+        raise ValueError(
+                'expected that both original and modified should be of the '
+                'same type, received "{!r}" and "{!r}".'.
+                format(type(original), type(modified)))
+
+    return field_mask_pb2.FieldMask(
+        paths=_field_mask_helper(original, modified))
+
+
+def _field_mask_helper(original, modified, current=''):
+    answer = []
+
+    for name in original.DESCRIPTOR.fields_by_name:
+        field_path = _get_path(current, name)
+
+        original_val = getattr(original, name)
+        modified_val = getattr(modified, name)
+
+        if _is_message(original_val) or _is_message(modified_val):
+            if original_val != modified_val:
+                # Wrapper types do not need to include the .value part of the
+                # path.
+                if _is_wrapper(original_val) or _is_wrapper(modified_val):
+                    answer.append(field_path)
+                elif not modified_val.ListFields():
+                    answer.append(field_path)
+                else:
+                    answer.extend(_field_mask_helper(original_val,
+                                                     modified_val, field_path))
+        else:
+            if original_val != modified_val:
+                answer.append(field_path)
+
+    return answer
+
+
+def _get_path(current, name):
+    if not current:
+        return name
+    return '%s.%s' % (current, name)
+
+
+def _is_message(value):
+    return isinstance(value, message.Message)
+
+
+def _is_wrapper(value):
+    return type(value) in _WRAPPER_TYPES
diff --git a/stream/clients/python/bookkeeper/common/retry.py b/stream/clients/python/bookkeeper/common/retry.py
new file mode 100644
index 0000000..f75905a
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/retry.py
@@ -0,0 +1,282 @@
+# 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.
+
+"""Helpers for retrying functions with exponential back-off.
+The :class:`Retry` decorator can be used to retry functions that raise
+exceptions using exponential backoff. Because a exponential sleep algorithm is
+used, the retry is limited by a `deadline`. The deadline is the maxmimum amount
+of time a method can block. This is used instead of total number of retries
+because it is difficult to ascertain the amount of time a function can block
+when using total number of retries and exponential backoff.
+By default, this decorator will retry transient
+API errors (see :func:`if_transient_error`). For example:
+.. code-block:: python
+    @retry.Retry()
+    def call_flaky_rpc():
+        return client.flaky_rpc()
+    # Will retry flaky_rpc() if it raises transient API errors.
+    result = call_flaky_rpc()
+You can pass a custom predicate to retry on different exceptions, such as
+waiting for an eventually consistent item to be available:
+.. code-block:: python
+    @retry.Retry(predicate=if_exception_type(exceptions.NotFound))
+    def check_if_exists():
+        return client.does_thing_exist()
+    is_available = check_if_exists()
+Some client library methods apply retry automatically. These methods can accept
+a ``retry`` parameter that allows you to configure the behavior:
+.. code-block:: python
+    my_retry = retry.Retry(deadline=60)
+    result = client.some_method(retry=my_retry)
+"""
+
+from __future__ import unicode_literals
+
+import datetime
+import functools
+import logging
+import random
+import time
+
+import six
+
+from bookkeeper.common import datetime_helpers
+from bookkeeper.common import exceptions
+from bookkeeper.common import general_helpers
+
+_LOGGER = logging.getLogger(__name__)
+_DEFAULT_INITIAL_DELAY = 1.0  # seconds
+_DEFAULT_MAXIMUM_DELAY = 60.0  # seconds
+_DEFAULT_DELAY_MULTIPLIER = 2.0
+_DEFAULT_DEADLINE = 60.0 * 2.0  # seconds
+
+
+def if_exception_type(*exception_types):
+    """Creates a predicate to check if the exception is of a given type.
+    Args:
+        exception_types (Sequence[:func:`type`]): The exception types to check
+            for.
+    Returns:
+        Callable[Exception]: A predicate that returns True if the provided
+            exception is of the given type(s).
+    """
+    def if_exception_type_predicate(exception):
+        """Bound predicate for checking an exception type."""
+        return isinstance(exception, exception_types)
+    return if_exception_type_predicate
+
+
+# pylint: disable=invalid-name
+# Pylint sees this as a constant, but it is also an alias that should be
+# considered a function.
+if_transient_error = if_exception_type((
+    exceptions.InternalServerError))
+"""A predicate that checks if an exception is a transient API error.
+The following server errors are considered transient:
+- :class:`bookkeeper.common.exceptions.InternalServerError` - gRPC
+    ``INTERNAL(13)``, bk ``INTERNAL_SERVER_ERROR`` and its subclasses.
+"""
+# pylint: enable=invalid-name
+
+
+def exponential_sleep_generator(
+        initial, maximum, multiplier=_DEFAULT_DELAY_MULTIPLIER):
+    """Generates sleep intervals based on the exponential back-off algorithm.
+    This implements the `Truncated Exponential Back-off`_ algorithm.
+    .. _Truncated Exponential Back-off:
+        https://cloud.google.com/storage/docs/exponential-backoff
+    Args:
+        initial (float): The minimum about of time to delay. This must
+            be greater than 0.
+        maximum (float): The maximum about of time to delay.
+        multiplier (float): The multiplier applied to the delay.
+    Yields:
+        float: successive sleep intervals.
+    """
+    delay = initial
+    while True:
+        # Introduce jitter by yielding a delay that is uniformly distributed
+        # to average out to the delay time.
+        yield min(random.uniform(0.0, delay * 2.0), maximum)
+        delay = delay * multiplier
+
+
+def retry_target(target, predicate, sleep_generator, deadline, on_error=None):
+    """Call a function and retry if it fails.
+    This is the lowest-level retry helper. Generally, you'll use the
+    higher-level retry helper :class:`Retry`.
+    Args:
+        target(Callable): The function to call and retry. This must be a
+            nullary function - apply arguments with `functools.partial`.
+        predicate (Callable[Exception]): A callable used to determine if an
+            exception raised by the target should be considered retryable.
+            It should return True to retry or False otherwise.
+        sleep_generator (Iterable[float]): An infinite iterator that determines
+            how long to sleep between retries.
+        deadline (float): How long to keep retrying the target.
+        on_error (Callable): A function to call while processing a retryable
+            exception.  Any error raised by this function will *not* be
+            caught.
+    Returns:
+        Any: the return value of the target function.
+    Raises:
+        bookkeeper.common.RetryError: If the deadline is exceeded while
+            retrying.
+        ValueError: If the sleep generator stops yielding values.
+        Exception: If the target raises a method that isn't retryable.
+    """
+    if deadline is not None:
+        deadline_datetime = (
+            datetime_helpers.utcnow() + datetime.timedelta(seconds=deadline))
+    else:
+        deadline_datetime = None
+
+    last_exc = None
+
+    for sleep in sleep_generator:
+        try:
+            return target()
+
+        # pylint: disable=broad-except
+        # This function explicitly must deal with broad exceptions.
+        except Exception as exc:
+            if not predicate(exc):
+                raise
+            last_exc = exc
+            if on_error is not None:
+                on_error(exc)
+
+        now = datetime_helpers.utcnow()
+        if deadline_datetime is not None and deadline_datetime < now:
+            six.raise_from(
+                exceptions.RetryError(
+                    'Deadline of {:.1f}s exceeded while calling {}'.format(
+                        deadline, target),
+                    last_exc),
+                last_exc)
+
+        _LOGGER.debug('Retrying due to {}, sleeping {:.1f}s ...'.format(
+            last_exc, sleep))
+        time.sleep(sleep)
+
+    raise ValueError('Sleep generator stopped yielding sleep values.')
+
+
+@six.python_2_unicode_compatible
+class Retry(object):
+    """Exponential retry decorator.
+    This class is a decorator used to add exponential back-off retry behavior
+    to an RPC call.
+    Although the default behavior is to retry transient API errors, a
+    different predicate can be provided to retry other exceptions.
+    Args:
+        predicate (Callable[Exception]): A callable that should return ``True``
+            if the given exception is retryable.
+        initial (float): The minimum about of time to delay in seconds. This
+            must be greater than 0.
+        maximum (float): The maximum about of time to delay in seconds.
+        multiplier (float): The multiplier applied to the delay.
+        deadline (float): How long to keep retrying in seconds.
+    """
+    def __init__(
+            self,
+            predicate=if_transient_error,
+            initial=_DEFAULT_INITIAL_DELAY,
+            maximum=_DEFAULT_MAXIMUM_DELAY,
+            multiplier=_DEFAULT_DELAY_MULTIPLIER,
+            deadline=_DEFAULT_DEADLINE):
+        self._predicate = predicate
+        self._initial = initial
+        self._multiplier = multiplier
+        self._maximum = maximum
+        self._deadline = deadline
+
+    def __call__(self, func, on_error=None):
+        """Wrap a callable with retry behavior.
+        Args:
+            func (Callable): The callable to add retry behavior to.
+            on_error (Callable): A function to call while processing a
+                retryable exception.  Any error raised by this function will
+                *not* be caught.
+        Returns:
+            Callable: A callable that will invoke ``func`` with retry
+                behavior.
+        """
+        @general_helpers.wraps(func)
+        def retry_wrapped_func(*args, **kwargs):
+            """A wrapper that calls target function with retry."""
+            target = functools.partial(func, *args, **kwargs)
+            sleep_generator = exponential_sleep_generator(
+                self._initial, self._maximum, multiplier=self._multiplier)
+            return retry_target(
+                target,
+                self._predicate,
+                sleep_generator,
+                self._deadline,
+                on_error=on_error,
+            )
+
+        return retry_wrapped_func
+
+    def with_deadline(self, deadline):
+        """Return a copy of this retry with the given deadline.
+        Args:
+            deadline (float): How long to keep retrying.
+        Returns:
+            Retry: A new retry instance with the given deadline.
+        """
+        return Retry(
+            predicate=self._predicate,
+            initial=self._initial,
+            maximum=self._maximum,
+            multiplier=self._multiplier,
+            deadline=deadline)
+
+    def with_predicate(self, predicate):
+        """Return a copy of this retry with the given predicate.
+        Args:
+            predicate (Callable[Exception]): A callable that should return
+                ``True`` if the given exception is retryable.
+        Returns:
+            Retry: A new retry instance with the given predicate.
+        """
+        return Retry(
+            predicate=predicate,
+            initial=self._initial,
+            maximum=self._maximum,
+            multiplier=self._multiplier,
+            deadline=self._deadline)
+
+    def with_delay(
+            self, initial=None, maximum=None, multiplier=None):
+        """Return a copy of this retry with the given delay options.
+        Args:
+            initial (float): The minimum about of time to delay. This must
+                be greater than 0.
+            maximum (float): The maximum about of time to delay.
+            multiplier (float): The multiplier applied to the delay.
+        Returns:
+            Retry: A new retry instance with the given predicate.
+        """
+        return Retry(
+            predicate=self._predicate,
+            initial=initial if initial is not None else self._initial,
+            maximum=maximum if maximum is not None else self._maximum,
+            multiplier=multiplier if maximum is not None else self._multiplier,
+            deadline=self._deadline)
+
+    def __str__(self):
+        return (
+            '<Retry predicate={}, initial={:.1f}, maximum={:.1f}, '
+            'multiplier={:.1f}, deadline={:.1f}>'.format(
+                self._predicate, self._initial, self._maximum,
+                self._multiplier, self._deadline))
diff --git a/stream/clients/python/bookkeeper/common/service_uri.py b/stream/clients/python/bookkeeper/common/service_uri.py
new file mode 100644
index 0000000..ee8cd93
--- /dev/null
+++ b/stream/clients/python/bookkeeper/common/service_uri.py
@@ -0,0 +1,31 @@
+# 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.
+
+import sys
+
+if sys.version_info[0] < 3:
+    USE_PYTHON3 = False
+    from urlparse import urlparse
+else:
+    USE_PYTHON3 = True
+    from urllib.parse import urlparse
+
+
+class ServiceURI(object):
+
+    def __init__(self, service_uri):
+        self.uri = urlparse(service_uri)
+        self.service_name = self.uri.scheme
+        self.service_user = self.uri.username
+        self.service_path = self.uri.path
+        self.service_location = self.uri.netloc
+        self.service_hosts = self.uri.netloc.split(',')
diff --git a/stream/clients/python/bookkeeper/kv/__init__.py b/stream/clients/python/bookkeeper/kv/__init__.py
new file mode 100644
index 0000000..459fe51
--- /dev/null
+++ b/stream/clients/python/bookkeeper/kv/__init__.py
@@ -0,0 +1,19 @@
+# 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 __future__ import absolute_import
+
+from bookkeeper.kv.client import Client
+
+__all__ = (
+    'Client',
+)
diff --git a/stream/clients/python/bookkeeper/kv/client.py b/stream/clients/python/bookkeeper/kv/client.py
new file mode 100644
index 0000000..314fab5
--- /dev/null
+++ b/stream/clients/python/bookkeeper/kv/client.py
@@ -0,0 +1,60 @@
+# -*- coding: utf-8 -*-
+#
+# 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
+#
+#     https://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 __future__ import absolute_import
+
+import grpc
+import logging
+import pkg_resources
+
+from bookkeeper import types
+from bookkeeper.common.service_uri import ServiceURI
+from bookkeeper.proto.storage_pb2_grpc import RootRangeServiceStub
+
+__version__ = pkg_resources.get_distribution('bookkeeper').version
+__logger__ = logging.getLogger("bookkeeper.kv.Client")
+
+
+class Client(object):
+    """A key/value client for Apache BookKeeper Table Service.
+
+    This creates an object that is capable of interacting with bookkeeper
+    table service. Generally, you can instantiate this client with no
+    arguments, and you get sensible defaults.
+
+    Args:
+        storage_client_settings (~bookkeeper.types.StorageClientSettings): The
+            settings for bookkeeper storage client .
+        kwargs (dict): Any additional arguments provided are sent as keyword
+            arguments to the underlying grpc client.
+    """
+    def __init__(self, storage_client_settings=(), **kwargs):
+        # init the storage client settings
+        self.storage_client_settings =\
+            types.StorageClientSettings(*storage_client_settings)
+        __logger__.info("Creating an admin client to cluster '%s'",
+                        self.storage_client_settings.service_uri)
+
+        service_uri = ServiceURI(self.storage_client_settings.service_uri)
+        assert service_uri.service_name.lower() == 'bk'
+
+        # create channel
+        self.channel = grpc.insecure_channel(
+            target=service_uri.service_location
+        )
+        __logger__.info("Successfully created an admin client to cluster '%s'",
+                        self.storage_client_settings.service_uri)
+
+        # create the rpc stub
+        self.root_range = RootRangeServiceStub(channel=self.channel)
diff --git a/stream/clients/python/bookkeeper/kv/exceptions.py b/stream/clients/python/bookkeeper/kv/exceptions.py
new file mode 100644
index 0000000..3bc01a858
--- /dev/null
+++ b/stream/clients/python/bookkeeper/kv/exceptions.py
@@ -0,0 +1,19 @@
+# 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 __future__ import absolute_import
+
+from concurrent.futures import TimeoutError
+
+__all__ = (
+    'TimeoutError',
+)
diff --git a/stream/clients/python/bookkeeper/kv/futures.py b/stream/clients/python/bookkeeper/kv/futures.py
new file mode 100644
index 0000000..ae626a7
--- /dev/null
+++ b/stream/clients/python/bookkeeper/kv/futures.py
@@ -0,0 +1,170 @@
+# 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 __future__ import absolute_import
+
+import threading
+import uuid
+
+import bookkeeper.common.future
+from bookkeeper.kv import exceptions
+
+
+class Future(bookkeeper.common.future.Future):
+    """Encapsulation of the asynchronous execution of an action.
+    This object is returned from asychronous bookkeeper calls, and is the
+    interface to determine the status of those calls.
+    This object should not be created directly, but is returned by other
+    methods in this library.
+    Args:
+        completed (Optional[Any]): An event, with the same interface as
+            :class:`threading.Event`. This is provided so that callers
+            with different concurrency models (e.g. ``threading`` or
+            ``multiprocessing``) can supply an event that is compatible
+            with that model. The ``wait()`` and ``set()`` methods will be
+            used. If this argument is not provided, then a new
+            :class:`threading.Event` will be created and used.
+    """
+
+    # This could be a sentinel object or None, but the sentinel object's ID
+    # can change if the process is forked, and None has the possibility of
+    # actually being a result.
+    _SENTINEL = uuid.uuid4()
+
+    def __init__(self, completed=None):
+        self._result = self._SENTINEL
+        self._exception = self._SENTINEL
+        self._callbacks = []
+        if completed is None:
+            completed = threading.Event()
+        self._completed = completed
+
+    def cancel(self):
+        """Actions in bookkeeper generally may not be canceled.
+        This method always returns False.
+        """
+        return False
+
+    def cancelled(self):
+        """Actions in bookkeeper generally may not be canceled.
+        This method always returns False.
+        """
+        return False
+
+    def running(self):
+        """Actions in bookkeeper generally may not be canceled.
+        Returns:
+            bool: ``True`` if this method has not yet completed, or
+                ``False`` if it has completed.
+        """
+        if self.done():
+            return False
+        return True
+
+    def done(self):
+        """Return True the future is done, False otherwise.
+        This still returns True in failure cases; checking :meth:`result` or
+        :meth:`exception` is the canonical way to assess success or failure.
+        """
+        return (self._exception != self._SENTINEL or
+                self._result != self._SENTINEL)
+
+    def result(self, timeout=None):
+        """Return the message ID, or raise an exception.
+        This blocks until the message has successfully been published, and
+        returns the message ID.
+        Args:
+            timeout (Union[int, float]): The number of seconds before this call
+                times out and raises TimeoutError.
+        Returns:
+            str: The message ID.
+        Raises:
+            ~.pubsub_v1.TimeoutError: If the request times out.
+            Exception: For undefined exceptions in the underlying
+                call execution.
+        """
+        # Attempt to get the exception if there is one.
+        # If there is not one, then we know everything worked, and we can
+        # return an appropriate value.
+        err = self.exception(timeout=timeout)
+        if err is None:
+            return self._result
+        raise err
+
+    def exception(self, timeout=None):
+        """Return the exception raised by the call, if any.
+        This blocks until the message has successfully been published, and
+        returns the exception. If the call succeeded, return None.
+        Args:
+            timeout (Union[int, float]): The number of seconds before this call
+                times out and raises TimeoutError.
+        Raises:
+            TimeoutError: If the request times out.
+        Returns:
+            Exception: The exception raised by the call, if any.
+        """
+        # Wait until the future is done.
+        if not self._completed.wait(timeout=timeout):
+            raise exceptions.TimeoutError('Timed out waiting for result.')
+
+        # If the batch completed successfully, this should return None.
+        if self._result != self._SENTINEL:
+            return None
+
+        # Okay, this batch had an error; this should return it.
+        return self._exception
+
+    def add_done_callback(self, fn):
+        """Attach the provided callable to the future.
+        The provided function is called, with this future as its only argument,
+        when the future finishes running.
+        """
+        if self.done():
+            return fn(self)
+        self._callbacks.append(fn)
+
+    def set_result(self, result):
+        """Set the result of the future to the provided result.
+        Args:
+            result (Any): The result
+        """
+        # Sanity check: A future can only complete once.
+        if self.done():
+            raise RuntimeError('set_result can only be called once.')
+
+        # Set the result and trigger the future.
+        self._result = result
+        self._trigger()
+
+    def set_exception(self, exception):
+        """Set the result of the future to the given exception.
+        Args:
+            exception (:exc:`Exception`): The exception raised.
+        """
+        # Sanity check: A future can only complete once.
+        if self.done():
+            raise RuntimeError('set_exception can only be called once.')
+
+        # Set the exception and trigger the future.
+        self._exception = exception
+        self._trigger()
+
+    def _trigger(self):
+        """Trigger all callbacks registered to this Future.
+        This method is called internally by the batch once the batch
+        completes.
+        Args:
+            message_id (str): The message ID, as a string.
+        """
+        self._completed.set()
+        for callback in self._callbacks:
+            callback(self)
diff --git a/stream/clients/python/bookkeeper/proto/cluster_pb2.py b/stream/clients/python/bookkeeper/proto/cluster_pb2.py
new file mode 100644
index 0000000..65f5d6d
--- /dev/null
+++ b/stream/clients/python/bookkeeper/proto/cluster_pb2.py
@@ -0,0 +1,199 @@
+# Generated by the protocol buffer compiler.  DO NOT EDIT!
+# source: cluster.proto
+
+import sys
+_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1'))
+from google.protobuf import descriptor as _descriptor
+from google.protobuf import message as _message
+from google.protobuf import reflection as _reflection
+from google.protobuf import symbol_database as _symbol_database
+# @@protoc_insertion_point(imports)
+
+_sym_db = _symbol_database.Default()
+
+
+import bookkeeper.proto.common_pb2 as common__pb2
+
+
+DESCRIPTOR = _descriptor.FileDescriptor(
+  name='cluster.proto',
+  package='bookkeeper.proto.cluster',
+  syntax='proto3',
+  serialized_options=_b('\n*org.apache.bookkeeper.stream.proto.clusterP\001'),
+  serialized_pb=_b('\n\rcluster.proto\x12\x18\x62ookkeeper.proto.cluster\x1a\x0c\x63ommon.proto\"*\n\x14ServerAssignmentData\x12\x12\n\ncontainers\x18\x01 \x03(\x03\"\xc6\x01\n\x15\x43lusterAssignmentData\x12M\n\x07servers\x18\x01 \x03(\x0b\x32<.bookkeeper.proto.cluster.ClusterAssignmentData.ServersEntry\x1a^\n\x0cServersEntry\x12\x0b\n\x03key\x18\x01 \x01(\t\x12=\n\x05value\x18\x02 \x01(\x0b\x32..bookkeeper.proto.cluster.ServerAssignmentData:\x02\x38\x01\"1\n\x0f\x43lusterMetadata\x12\x [...]
+  ,
+  dependencies=[common__pb2.DESCRIPTOR,])
+
+
+
+
+_SERVERASSIGNMENTDATA = _descriptor.Descriptor(
+  name='ServerAssignmentData',
+  full_name='bookkeeper.proto.cluster.ServerAssignmentData',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='containers', full_name='bookkeeper.proto.cluster.ServerAssignmentData.containers', index=0,
+      number=1, type=3, cpp_type=2, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=57,
+  serialized_end=99,
+)
+
+
+_CLUSTERASSIGNMENTDATA_SERVERSENTRY = _descriptor.Descriptor(
+  name='ServersEntry',
+  full_name='bookkeeper.proto.cluster.ClusterAssignmentData.ServersEntry',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='key', full_name='bookkeeper.proto.cluster.ClusterAssignmentData.ServersEntry.key', index=0,
+      number=1, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='value', full_name='bookkeeper.proto.cluster.ClusterAssignmentData.ServersEntry.value', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=_b('8\001'),
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=206,
+  serialized_end=300,
+)
+
+_CLUSTERASSIGNMENTDATA = _descriptor.Descriptor(
+  name='ClusterAssignmentData',
+  full_name='bookkeeper.proto.cluster.ClusterAssignmentData',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='servers', full_name='bookkeeper.proto.cluster.ClusterAssignmentData.servers', index=0,
+      number=1, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[_CLUSTERASSIGNMENTDATA_SERVERSENTRY, ],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=102,
+  serialized_end=300,
+)
+
+
+_CLUSTERMETADATA = _descriptor.Descriptor(
+  name='ClusterMetadata',
+  full_name='bookkeeper.proto.cluster.ClusterMetadata',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='num_storage_containers', full_name='bookkeeper.proto.cluster.ClusterMetadata.num_storage_containers', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=302,
+  serialized_end=351,
+)
+
+_CLUSTERASSIGNMENTDATA_SERVERSENTRY.fields_by_name['value'].message_type = _SERVERASSIGNMENTDATA
+_CLUSTERASSIGNMENTDATA_SERVERSENTRY.containing_type = _CLUSTERASSIGNMENTDATA
+_CLUSTERASSIGNMENTDATA.fields_by_name['servers'].message_type = _CLUSTERASSIGNMENTDATA_SERVERSENTRY
+DESCRIPTOR.message_types_by_name['ServerAssignmentData'] = _SERVERASSIGNMENTDATA
+DESCRIPTOR.message_types_by_name['ClusterAssignmentData'] = _CLUSTERASSIGNMENTDATA
+DESCRIPTOR.message_types_by_name['ClusterMetadata'] = _CLUSTERMETADATA
+_sym_db.RegisterFileDescriptor(DESCRIPTOR)
+
+ServerAssignmentData = _reflection.GeneratedProtocolMessageType('ServerAssignmentData', (_message.Message,), dict(
+  DESCRIPTOR = _SERVERASSIGNMENTDATA,
+  __module__ = 'cluster_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.cluster.ServerAssignmentData)
+  ))
+_sym_db.RegisterMessage(ServerAssignmentData)
+
+ClusterAssignmentData = _reflection.GeneratedProtocolMessageType('ClusterAssignmentData', (_message.Message,), dict(
+
+  ServersEntry = _reflection.GeneratedProtocolMessageType('ServersEntry', (_message.Message,), dict(
+    DESCRIPTOR = _CLUSTERASSIGNMENTDATA_SERVERSENTRY,
+    __module__ = 'cluster_pb2'
+    # @@protoc_insertion_point(class_scope:bookkeeper.proto.cluster.ClusterAssignmentData.ServersEntry)
+    ))
+  ,
+  DESCRIPTOR = _CLUSTERASSIGNMENTDATA,
+  __module__ = 'cluster_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.cluster.ClusterAssignmentData)
+  ))
+_sym_db.RegisterMessage(ClusterAssignmentData)
+_sym_db.RegisterMessage(ClusterAssignmentData.ServersEntry)
+
+ClusterMetadata = _reflection.GeneratedProtocolMessageType('ClusterMetadata', (_message.Message,), dict(
+  DESCRIPTOR = _CLUSTERMETADATA,
+  __module__ = 'cluster_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.cluster.ClusterMetadata)
+  ))
+_sym_db.RegisterMessage(ClusterMetadata)
+
+
+DESCRIPTOR._options = None
+_CLUSTERASSIGNMENTDATA_SERVERSENTRY._options = None
+# @@protoc_insertion_point(module_scope)
diff --git a/stream/clients/python/bookkeeper/proto/common_pb2.py b/stream/clients/python/bookkeeper/proto/common_pb2.py
new file mode 100644
index 0000000..84bb846
--- /dev/null
+++ b/stream/clients/python/bookkeeper/proto/common_pb2.py
@@ -0,0 +1,77 @@
+# Generated by the protocol buffer compiler.  DO NOT EDIT!
+# source: common.proto
+
+import sys
+_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1'))
+from google.protobuf import descriptor as _descriptor
+from google.protobuf import message as _message
+from google.protobuf import reflection as _reflection
+from google.protobuf import symbol_database as _symbol_database
+# @@protoc_insertion_point(imports)
+
+_sym_db = _symbol_database.Default()
+
+
+
+
+DESCRIPTOR = _descriptor.FileDescriptor(
+  name='common.proto',
+  package='bookkeeper.proto.common',
+  syntax='proto3',
+  serialized_options=_b('\n)org.apache.bookkeeper.stream.proto.commonP\001'),
+  serialized_pb=_b('\n\x0c\x63ommon.proto\x12\x17\x62ookkeeper.proto.common\"*\n\x08\x45ndpoint\x12\x10\n\x08hostname\x18\x01 \x01(\t\x12\x0c\n\x04port\x18\x02 \x01(\x05\x42-\n)org.apache.bookkeeper.stream.proto.commonP\x01\x62\x06proto3')
+)
+
+
+
+
+_ENDPOINT = _descriptor.Descriptor(
+  name='Endpoint',
+  full_name='bookkeeper.proto.common.Endpoint',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='hostname', full_name='bookkeeper.proto.common.Endpoint.hostname', index=0,
+      number=1, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='port', full_name='bookkeeper.proto.common.Endpoint.port', index=1,
+      number=2, type=5, cpp_type=1, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=41,
+  serialized_end=83,
+)
+
+DESCRIPTOR.message_types_by_name['Endpoint'] = _ENDPOINT
+_sym_db.RegisterFileDescriptor(DESCRIPTOR)
+
+Endpoint = _reflection.GeneratedProtocolMessageType('Endpoint', (_message.Message,), dict(
+  DESCRIPTOR = _ENDPOINT,
+  __module__ = 'common_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.common.Endpoint)
+  ))
+_sym_db.RegisterMessage(Endpoint)
+
+
+DESCRIPTOR._options = None
+# @@protoc_insertion_point(module_scope)
diff --git a/stream/clients/python/bookkeeper/proto/kv_pb2.py b/stream/clients/python/bookkeeper/proto/kv_pb2.py
new file mode 100644
index 0000000..51fedc3
--- /dev/null
+++ b/stream/clients/python/bookkeeper/proto/kv_pb2.py
@@ -0,0 +1,192 @@
+# Generated by the protocol buffer compiler.  DO NOT EDIT!
+# source: kv.proto
+
+import sys
+_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1'))
+from google.protobuf import descriptor as _descriptor
+from google.protobuf import message as _message
+from google.protobuf import reflection as _reflection
+from google.protobuf import symbol_database as _symbol_database
+# @@protoc_insertion_point(imports)
+
+_sym_db = _symbol_database.Default()
+
+
+
+
+DESCRIPTOR = _descriptor.FileDescriptor(
+  name='kv.proto',
+  package='bookkeeper.proto.kv',
+  syntax='proto3',
+  serialized_options=_b('\n%org.apache.bookkeeper.stream.proto.kvP\001'),
+  serialized_pb=_b('\n\x08kv.proto\x12\x13\x62ookkeeper.proto.kv\"\x8f\x01\n\x08KeyValue\x12\x0b\n\x03key\x18\x01 \x01(\x0c\x12\x17\n\x0f\x63reate_revision\x18\x02 \x01(\x03\x12\x14\n\x0cmod_revision\x18\x03 \x01(\x03\x12\x0f\n\x07version\x18\x04 \x01(\x03\x12\r\n\x05value\x18\x05 \x01(\x0c\x12\x11\n\tis_number\x18\x06 \x01(\x08\x12\x14\n\x0cnumber_value\x18\x07 \x01(\x03\"\xb8\x01\n\x05\x45vent\x12\x32\n\x04type\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.kv.Event.EventType\x12)\n\x02kv\x18 [...]
+)
+
+
+
+_EVENT_EVENTTYPE = _descriptor.EnumDescriptor(
+  name='EventType',
+  full_name='bookkeeper.proto.kv.Event.EventType',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='PUT', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='DELETE', index=1, number=1,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=332,
+  serialized_end=364,
+)
+_sym_db.RegisterEnumDescriptor(_EVENT_EVENTTYPE)
+
+
+_KEYVALUE = _descriptor.Descriptor(
+  name='KeyValue',
+  full_name='bookkeeper.proto.kv.KeyValue',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='key', full_name='bookkeeper.proto.kv.KeyValue.key', index=0,
+      number=1, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='create_revision', full_name='bookkeeper.proto.kv.KeyValue.create_revision', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='mod_revision', full_name='bookkeeper.proto.kv.KeyValue.mod_revision', index=2,
+      number=3, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='version', full_name='bookkeeper.proto.kv.KeyValue.version', index=3,
+      number=4, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='value', full_name='bookkeeper.proto.kv.KeyValue.value', index=4,
+      number=5, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='is_number', full_name='bookkeeper.proto.kv.KeyValue.is_number', index=5,
+      number=6, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='number_value', full_name='bookkeeper.proto.kv.KeyValue.number_value', index=6,
+      number=7, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=34,
+  serialized_end=177,
+)
+
+
+_EVENT = _descriptor.Descriptor(
+  name='Event',
+  full_name='bookkeeper.proto.kv.Event',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='type', full_name='bookkeeper.proto.kv.Event.type', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='kv', full_name='bookkeeper.proto.kv.Event.kv', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='prev_kv', full_name='bookkeeper.proto.kv.Event.prev_kv', index=2,
+      number=3, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+    _EVENT_EVENTTYPE,
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=180,
+  serialized_end=364,
+)
+
+_EVENT.fields_by_name['type'].enum_type = _EVENT_EVENTTYPE
+_EVENT.fields_by_name['kv'].message_type = _KEYVALUE
+_EVENT.fields_by_name['prev_kv'].message_type = _KEYVALUE
+_EVENT_EVENTTYPE.containing_type = _EVENT
+DESCRIPTOR.message_types_by_name['KeyValue'] = _KEYVALUE
+DESCRIPTOR.message_types_by_name['Event'] = _EVENT
+_sym_db.RegisterFileDescriptor(DESCRIPTOR)
+
+KeyValue = _reflection.GeneratedProtocolMessageType('KeyValue', (_message.Message,), dict(
+  DESCRIPTOR = _KEYVALUE,
+  __module__ = 'kv_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.KeyValue)
+  ))
+_sym_db.RegisterMessage(KeyValue)
+
+Event = _reflection.GeneratedProtocolMessageType('Event', (_message.Message,), dict(
+  DESCRIPTOR = _EVENT,
+  __module__ = 'kv_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.Event)
+  ))
+_sym_db.RegisterMessage(Event)
+
+
+DESCRIPTOR._options = None
+# @@protoc_insertion_point(module_scope)
diff --git a/stream/clients/python/bookkeeper/proto/kv_rpc_pb2.py b/stream/clients/python/bookkeeper/proto/kv_rpc_pb2.py
new file mode 100644
index 0000000..2a3f7a9
--- /dev/null
+++ b/stream/clients/python/bookkeeper/proto/kv_rpc_pb2.py
@@ -0,0 +1,1232 @@
+# Generated by the protocol buffer compiler.  DO NOT EDIT!
+# source: kv_rpc.proto
+
+import sys
+_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1'))
+from google.protobuf import descriptor as _descriptor
+from google.protobuf import message as _message
+from google.protobuf import reflection as _reflection
+from google.protobuf import symbol_database as _symbol_database
+# @@protoc_insertion_point(imports)
+
+_sym_db = _symbol_database.Default()
+
+
+import bookkeeper.proto.kv_pb2 as kv__pb2
+import bookkeeper.proto.storage_pb2 as storage__pb2
+
+
+DESCRIPTOR = _descriptor.FileDescriptor(
+  name='kv_rpc.proto',
+  package='bookkeeper.proto.kv.rpc',
+  syntax='proto3',
+  serialized_options=_b('\n)org.apache.bookkeeper.stream.proto.kv.rpcP\001'),
+  serialized_pb=_b('\n\x0ckv_rpc.proto\x12\x17\x62ookkeeper.proto.kv.rpc\x1a\x08kv.proto\x1a\rstorage.proto\"C\n\rRoutingHeader\x12\x11\n\tstream_id\x18\x01 \x01(\x03\x12\x10\n\x08range_id\x18\x02 \x01(\x03\x12\r\n\x05r_key\x18\x03 \x01(\x0c\"\x84\x01\n\x0eResponseHeader\x12\x32\n\x04\x63ode\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\x12>\n\x0erouting_header\x18\x63 \x01(\x0b\x32&.bookkeeper.proto.kv.rpc.RoutingHeader\"\xb2\x04\n\x0cRangeRequest\x12\x0b\n\x03key\x18\x01 \ [...]
+  ,
+  dependencies=[kv__pb2.DESCRIPTOR,storage__pb2.DESCRIPTOR,])
+
+
+
+_RANGEREQUEST_SORTORDER = _descriptor.EnumDescriptor(
+  name='SortOrder',
+  full_name='bookkeeper.proto.kv.rpc.RangeRequest.SortOrder',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='NONE', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='ASCEND', index=1, number=1,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='DESCEND', index=2, number=2,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=719,
+  serialized_end=765,
+)
+_sym_db.RegisterEnumDescriptor(_RANGEREQUEST_SORTORDER)
+
+_RANGEREQUEST_SORTTARGET = _descriptor.EnumDescriptor(
+  name='SortTarget',
+  full_name='bookkeeper.proto.kv.rpc.RangeRequest.SortTarget',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='KEY', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='VERSION', index=1, number=1,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='CREATE', index=2, number=2,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='MOD', index=3, number=3,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='VALUE', index=4, number=4,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=767,
+  serialized_end=833,
+)
+_sym_db.RegisterEnumDescriptor(_RANGEREQUEST_SORTTARGET)
+
+_COMPARE_COMPARERESULT = _descriptor.EnumDescriptor(
+  name='CompareResult',
+  full_name='bookkeeper.proto.kv.rpc.Compare.CompareResult',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='EQUAL', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='GREATER', index=1, number=1,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='LESS', index=2, number=2,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='NOT_EQUAL', index=3, number=3,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=2502,
+  serialized_end=2566,
+)
+_sym_db.RegisterEnumDescriptor(_COMPARE_COMPARERESULT)
+
+_COMPARE_COMPARETARGET = _descriptor.EnumDescriptor(
+  name='CompareTarget',
+  full_name='bookkeeper.proto.kv.rpc.Compare.CompareTarget',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='VERSION', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='CREATE', index=1, number=1,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='MOD', index=2, number=2,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='VALUE', index=3, number=3,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=2568,
+  serialized_end=2628,
+)
+_sym_db.RegisterEnumDescriptor(_COMPARE_COMPARETARGET)
+
+
+_ROUTINGHEADER = _descriptor.Descriptor(
+  name='RoutingHeader',
+  full_name='bookkeeper.proto.kv.rpc.RoutingHeader',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='stream_id', full_name='bookkeeper.proto.kv.rpc.RoutingHeader.stream_id', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='range_id', full_name='bookkeeper.proto.kv.rpc.RoutingHeader.range_id', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='r_key', full_name='bookkeeper.proto.kv.rpc.RoutingHeader.r_key', index=2,
+      number=3, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=66,
+  serialized_end=133,
+)
+
+
+_RESPONSEHEADER = _descriptor.Descriptor(
+  name='ResponseHeader',
+  full_name='bookkeeper.proto.kv.rpc.ResponseHeader',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='code', full_name='bookkeeper.proto.kv.rpc.ResponseHeader.code', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='routing_header', full_name='bookkeeper.proto.kv.rpc.ResponseHeader.routing_header', index=1,
+      number=99, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=136,
+  serialized_end=268,
+)
+
+
+_RANGEREQUEST = _descriptor.Descriptor(
+  name='RangeRequest',
+  full_name='bookkeeper.proto.kv.rpc.RangeRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='key', full_name='bookkeeper.proto.kv.rpc.RangeRequest.key', index=0,
+      number=1, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='range_end', full_name='bookkeeper.proto.kv.rpc.RangeRequest.range_end', index=1,
+      number=2, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='limit', full_name='bookkeeper.proto.kv.rpc.RangeRequest.limit', index=2,
+      number=3, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='revision', full_name='bookkeeper.proto.kv.rpc.RangeRequest.revision', index=3,
+      number=4, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='sort_order', full_name='bookkeeper.proto.kv.rpc.RangeRequest.sort_order', index=4,
+      number=5, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='sort_target', full_name='bookkeeper.proto.kv.rpc.RangeRequest.sort_target', index=5,
+      number=6, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='serializable', full_name='bookkeeper.proto.kv.rpc.RangeRequest.serializable', index=6,
+      number=7, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='keys_only', full_name='bookkeeper.proto.kv.rpc.RangeRequest.keys_only', index=7,
+      number=8, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='count_only', full_name='bookkeeper.proto.kv.rpc.RangeRequest.count_only', index=8,
+      number=9, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='min_mod_revision', full_name='bookkeeper.proto.kv.rpc.RangeRequest.min_mod_revision', index=9,
+      number=10, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='max_mod_revision', full_name='bookkeeper.proto.kv.rpc.RangeRequest.max_mod_revision', index=10,
+      number=11, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='min_create_revision', full_name='bookkeeper.proto.kv.rpc.RangeRequest.min_create_revision', index=11,
+      number=12, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='max_create_revision', full_name='bookkeeper.proto.kv.rpc.RangeRequest.max_create_revision', index=12,
+      number=13, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='header', full_name='bookkeeper.proto.kv.rpc.RangeRequest.header', index=13,
+      number=99, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+    _RANGEREQUEST_SORTORDER,
+    _RANGEREQUEST_SORTTARGET,
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=271,
+  serialized_end=833,
+)
+
+
+_RANGERESPONSE = _descriptor.Descriptor(
+  name='RangeResponse',
+  full_name='bookkeeper.proto.kv.rpc.RangeResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='header', full_name='bookkeeper.proto.kv.rpc.RangeResponse.header', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='kvs', full_name='bookkeeper.proto.kv.rpc.RangeResponse.kvs', index=1,
+      number=2, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='more', full_name='bookkeeper.proto.kv.rpc.RangeResponse.more', index=2,
+      number=3, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='count', full_name='bookkeeper.proto.kv.rpc.RangeResponse.count', index=3,
+      number=4, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=836,
+  serialized_end=981,
+)
+
+
+_PUTREQUEST = _descriptor.Descriptor(
+  name='PutRequest',
+  full_name='bookkeeper.proto.kv.rpc.PutRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='key', full_name='bookkeeper.proto.kv.rpc.PutRequest.key', index=0,
+      number=1, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='value', full_name='bookkeeper.proto.kv.rpc.PutRequest.value', index=1,
+      number=2, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='lease', full_name='bookkeeper.proto.kv.rpc.PutRequest.lease', index=2,
+      number=3, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='prev_kv', full_name='bookkeeper.proto.kv.rpc.PutRequest.prev_kv', index=3,
+      number=4, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='ignore_value', full_name='bookkeeper.proto.kv.rpc.PutRequest.ignore_value', index=4,
+      number=5, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='ignore_lease', full_name='bookkeeper.proto.kv.rpc.PutRequest.ignore_lease', index=5,
+      number=6, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='header', full_name='bookkeeper.proto.kv.rpc.PutRequest.header', index=6,
+      number=99, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='expected_version', full_name='bookkeeper.proto.kv.rpc.PutRequest.expected_version', index=7,
+      number=100, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=984,
+  serialized_end=1182,
+)
+
+
+_PUTRESPONSE = _descriptor.Descriptor(
+  name='PutResponse',
+  full_name='bookkeeper.proto.kv.rpc.PutResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='header', full_name='bookkeeper.proto.kv.rpc.PutResponse.header', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='prev_kv', full_name='bookkeeper.proto.kv.rpc.PutResponse.prev_kv', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1184,
+  serialized_end=1302,
+)
+
+
+_INCREMENTREQUEST = _descriptor.Descriptor(
+  name='IncrementRequest',
+  full_name='bookkeeper.proto.kv.rpc.IncrementRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='key', full_name='bookkeeper.proto.kv.rpc.IncrementRequest.key', index=0,
+      number=1, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='amount', full_name='bookkeeper.proto.kv.rpc.IncrementRequest.amount', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='get_total', full_name='bookkeeper.proto.kv.rpc.IncrementRequest.get_total', index=2,
+      number=3, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='header', full_name='bookkeeper.proto.kv.rpc.IncrementRequest.header', index=3,
+      number=99, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1304,
+  serialized_end=1426,
+)
+
+
+_INCREMENTRESPONSE = _descriptor.Descriptor(
+  name='IncrementResponse',
+  full_name='bookkeeper.proto.kv.rpc.IncrementResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='header', full_name='bookkeeper.proto.kv.rpc.IncrementResponse.header', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='total_amount', full_name='bookkeeper.proto.kv.rpc.IncrementResponse.total_amount', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1428,
+  serialized_end=1526,
+)
+
+
+_DELETERANGEREQUEST = _descriptor.Descriptor(
+  name='DeleteRangeRequest',
+  full_name='bookkeeper.proto.kv.rpc.DeleteRangeRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='key', full_name='bookkeeper.proto.kv.rpc.DeleteRangeRequest.key', index=0,
+      number=1, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='range_end', full_name='bookkeeper.proto.kv.rpc.DeleteRangeRequest.range_end', index=1,
+      number=2, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='prev_kv', full_name='bookkeeper.proto.kv.rpc.DeleteRangeRequest.prev_kv', index=2,
+      number=3, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='header', full_name='bookkeeper.proto.kv.rpc.DeleteRangeRequest.header', index=3,
+      number=99, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1528,
+  serialized_end=1653,
+)
+
+
+_DELETERANGERESPONSE = _descriptor.Descriptor(
+  name='DeleteRangeResponse',
+  full_name='bookkeeper.proto.kv.rpc.DeleteRangeResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='header', full_name='bookkeeper.proto.kv.rpc.DeleteRangeResponse.header', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='deleted', full_name='bookkeeper.proto.kv.rpc.DeleteRangeResponse.deleted', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='prev_kvs', full_name='bookkeeper.proto.kv.rpc.DeleteRangeResponse.prev_kvs', index=2,
+      number=3, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1656,
+  serialized_end=1800,
+)
+
+
+_REQUESTOP = _descriptor.Descriptor(
+  name='RequestOp',
+  full_name='bookkeeper.proto.kv.rpc.RequestOp',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='request_range', full_name='bookkeeper.proto.kv.rpc.RequestOp.request_range', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='request_put', full_name='bookkeeper.proto.kv.rpc.RequestOp.request_put', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='request_delete_range', full_name='bookkeeper.proto.kv.rpc.RequestOp.request_delete_range', index=2,
+      number=3, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+    _descriptor.OneofDescriptor(
+      name='request', full_name='bookkeeper.proto.kv.rpc.RequestOp.request',
+      index=0, containing_type=None, fields=[]),
+  ],
+  serialized_start=1803,
+  serialized_end=2026,
+)
+
+
+_RESPONSEOP = _descriptor.Descriptor(
+  name='ResponseOp',
+  full_name='bookkeeper.proto.kv.rpc.ResponseOp',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='response_range', full_name='bookkeeper.proto.kv.rpc.ResponseOp.response_range', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='response_put', full_name='bookkeeper.proto.kv.rpc.ResponseOp.response_put', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='response_delete_range', full_name='bookkeeper.proto.kv.rpc.ResponseOp.response_delete_range', index=2,
+      number=3, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+    _descriptor.OneofDescriptor(
+      name='response', full_name='bookkeeper.proto.kv.rpc.ResponseOp.response',
+      index=0, containing_type=None, fields=[]),
+  ],
+  serialized_start=2029,
+  serialized_end=2260,
+)
+
+
+_COMPARE = _descriptor.Descriptor(
+  name='Compare',
+  full_name='bookkeeper.proto.kv.rpc.Compare',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='result', full_name='bookkeeper.proto.kv.rpc.Compare.result', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='target', full_name='bookkeeper.proto.kv.rpc.Compare.target', index=1,
+      number=2, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='key', full_name='bookkeeper.proto.kv.rpc.Compare.key', index=2,
+      number=3, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='version', full_name='bookkeeper.proto.kv.rpc.Compare.version', index=3,
+      number=4, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='create_revision', full_name='bookkeeper.proto.kv.rpc.Compare.create_revision', index=4,
+      number=5, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='mod_revision', full_name='bookkeeper.proto.kv.rpc.Compare.mod_revision', index=5,
+      number=6, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='value', full_name='bookkeeper.proto.kv.rpc.Compare.value', index=6,
+      number=7, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+    _COMPARE_COMPARERESULT,
+    _COMPARE_COMPARETARGET,
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+    _descriptor.OneofDescriptor(
+      name='target_union', full_name='bookkeeper.proto.kv.rpc.Compare.target_union',
+      index=0, containing_type=None, fields=[]),
+  ],
+  serialized_start=2263,
+  serialized_end=2644,
+)
+
+
+_TXNREQUEST = _descriptor.Descriptor(
+  name='TxnRequest',
+  full_name='bookkeeper.proto.kv.rpc.TxnRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='compare', full_name='bookkeeper.proto.kv.rpc.TxnRequest.compare', index=0,
+      number=1, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='success', full_name='bookkeeper.proto.kv.rpc.TxnRequest.success', index=1,
+      number=2, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='failure', full_name='bookkeeper.proto.kv.rpc.TxnRequest.failure', index=2,
+      number=3, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='header', full_name='bookkeeper.proto.kv.rpc.TxnRequest.header', index=3,
+      number=99, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=2647,
+  serialized_end=2872,
+)
+
+
+_TXNRESPONSE = _descriptor.Descriptor(
+  name='TxnResponse',
+  full_name='bookkeeper.proto.kv.rpc.TxnResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='header', full_name='bookkeeper.proto.kv.rpc.TxnResponse.header', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='succeeded', full_name='bookkeeper.proto.kv.rpc.TxnResponse.succeeded', index=1,
+      number=2, type=8, cpp_type=7, label=1,
+      has_default_value=False, default_value=False,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='responses', full_name='bookkeeper.proto.kv.rpc.TxnResponse.responses', index=2,
+      number=3, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=2875,
+  serialized_end=3020,
+)
+
+_RESPONSEHEADER.fields_by_name['code'].enum_type = storage__pb2._STATUSCODE
+_RESPONSEHEADER.fields_by_name['routing_header'].message_type = _ROUTINGHEADER
+_RANGEREQUEST.fields_by_name['sort_order'].enum_type = _RANGEREQUEST_SORTORDER
+_RANGEREQUEST.fields_by_name['sort_target'].enum_type = _RANGEREQUEST_SORTTARGET
+_RANGEREQUEST.fields_by_name['header'].message_type = _ROUTINGHEADER
+_RANGEREQUEST_SORTORDER.containing_type = _RANGEREQUEST
+_RANGEREQUEST_SORTTARGET.containing_type = _RANGEREQUEST
+_RANGERESPONSE.fields_by_name['header'].message_type = _RESPONSEHEADER
+_RANGERESPONSE.fields_by_name['kvs'].message_type = kv__pb2._KEYVALUE
+_PUTREQUEST.fields_by_name['header'].message_type = _ROUTINGHEADER
+_PUTRESPONSE.fields_by_name['header'].message_type = _RESPONSEHEADER
+_PUTRESPONSE.fields_by_name['prev_kv'].message_type = kv__pb2._KEYVALUE
+_INCREMENTREQUEST.fields_by_name['header'].message_type = _ROUTINGHEADER
+_INCREMENTRESPONSE.fields_by_name['header'].message_type = _RESPONSEHEADER
+_DELETERANGEREQUEST.fields_by_name['header'].message_type = _ROUTINGHEADER
+_DELETERANGERESPONSE.fields_by_name['header'].message_type = _RESPONSEHEADER
+_DELETERANGERESPONSE.fields_by_name['prev_kvs'].message_type = kv__pb2._KEYVALUE
+_REQUESTOP.fields_by_name['request_range'].message_type = _RANGEREQUEST
+_REQUESTOP.fields_by_name['request_put'].message_type = _PUTREQUEST
+_REQUESTOP.fields_by_name['request_delete_range'].message_type = _DELETERANGEREQUEST
+_REQUESTOP.oneofs_by_name['request'].fields.append(
+  _REQUESTOP.fields_by_name['request_range'])
+_REQUESTOP.fields_by_name['request_range'].containing_oneof = _REQUESTOP.oneofs_by_name['request']
+_REQUESTOP.oneofs_by_name['request'].fields.append(
+  _REQUESTOP.fields_by_name['request_put'])
+_REQUESTOP.fields_by_name['request_put'].containing_oneof = _REQUESTOP.oneofs_by_name['request']
+_REQUESTOP.oneofs_by_name['request'].fields.append(
+  _REQUESTOP.fields_by_name['request_delete_range'])
+_REQUESTOP.fields_by_name['request_delete_range'].containing_oneof = _REQUESTOP.oneofs_by_name['request']
+_RESPONSEOP.fields_by_name['response_range'].message_type = _RANGERESPONSE
+_RESPONSEOP.fields_by_name['response_put'].message_type = _PUTRESPONSE
+_RESPONSEOP.fields_by_name['response_delete_range'].message_type = _DELETERANGERESPONSE
+_RESPONSEOP.oneofs_by_name['response'].fields.append(
+  _RESPONSEOP.fields_by_name['response_range'])
+_RESPONSEOP.fields_by_name['response_range'].containing_oneof = _RESPONSEOP.oneofs_by_name['response']
+_RESPONSEOP.oneofs_by_name['response'].fields.append(
+  _RESPONSEOP.fields_by_name['response_put'])
+_RESPONSEOP.fields_by_name['response_put'].containing_oneof = _RESPONSEOP.oneofs_by_name['response']
+_RESPONSEOP.oneofs_by_name['response'].fields.append(
+  _RESPONSEOP.fields_by_name['response_delete_range'])
+_RESPONSEOP.fields_by_name['response_delete_range'].containing_oneof = _RESPONSEOP.oneofs_by_name['response']
+_COMPARE.fields_by_name['result'].enum_type = _COMPARE_COMPARERESULT
+_COMPARE.fields_by_name['target'].enum_type = _COMPARE_COMPARETARGET
+_COMPARE_COMPARERESULT.containing_type = _COMPARE
+_COMPARE_COMPARETARGET.containing_type = _COMPARE
+_COMPARE.oneofs_by_name['target_union'].fields.append(
+  _COMPARE.fields_by_name['version'])
+_COMPARE.fields_by_name['version'].containing_oneof = _COMPARE.oneofs_by_name['target_union']
+_COMPARE.oneofs_by_name['target_union'].fields.append(
+  _COMPARE.fields_by_name['create_revision'])
+_COMPARE.fields_by_name['create_revision'].containing_oneof = _COMPARE.oneofs_by_name['target_union']
+_COMPARE.oneofs_by_name['target_union'].fields.append(
+  _COMPARE.fields_by_name['mod_revision'])
+_COMPARE.fields_by_name['mod_revision'].containing_oneof = _COMPARE.oneofs_by_name['target_union']
+_COMPARE.oneofs_by_name['target_union'].fields.append(
+  _COMPARE.fields_by_name['value'])
+_COMPARE.fields_by_name['value'].containing_oneof = _COMPARE.oneofs_by_name['target_union']
+_TXNREQUEST.fields_by_name['compare'].message_type = _COMPARE
+_TXNREQUEST.fields_by_name['success'].message_type = _REQUESTOP
+_TXNREQUEST.fields_by_name['failure'].message_type = _REQUESTOP
+_TXNREQUEST.fields_by_name['header'].message_type = _ROUTINGHEADER
+_TXNRESPONSE.fields_by_name['header'].message_type = _RESPONSEHEADER
+_TXNRESPONSE.fields_by_name['responses'].message_type = _RESPONSEOP
+DESCRIPTOR.message_types_by_name['RoutingHeader'] = _ROUTINGHEADER
+DESCRIPTOR.message_types_by_name['ResponseHeader'] = _RESPONSEHEADER
+DESCRIPTOR.message_types_by_name['RangeRequest'] = _RANGEREQUEST
+DESCRIPTOR.message_types_by_name['RangeResponse'] = _RANGERESPONSE
+DESCRIPTOR.message_types_by_name['PutRequest'] = _PUTREQUEST
+DESCRIPTOR.message_types_by_name['PutResponse'] = _PUTRESPONSE
+DESCRIPTOR.message_types_by_name['IncrementRequest'] = _INCREMENTREQUEST
+DESCRIPTOR.message_types_by_name['IncrementResponse'] = _INCREMENTRESPONSE
+DESCRIPTOR.message_types_by_name['DeleteRangeRequest'] = _DELETERANGEREQUEST
+DESCRIPTOR.message_types_by_name['DeleteRangeResponse'] = _DELETERANGERESPONSE
+DESCRIPTOR.message_types_by_name['RequestOp'] = _REQUESTOP
+DESCRIPTOR.message_types_by_name['ResponseOp'] = _RESPONSEOP
+DESCRIPTOR.message_types_by_name['Compare'] = _COMPARE
+DESCRIPTOR.message_types_by_name['TxnRequest'] = _TXNREQUEST
+DESCRIPTOR.message_types_by_name['TxnResponse'] = _TXNRESPONSE
+_sym_db.RegisterFileDescriptor(DESCRIPTOR)
+
+RoutingHeader = _reflection.GeneratedProtocolMessageType('RoutingHeader', (_message.Message,), dict(
+  DESCRIPTOR = _ROUTINGHEADER,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.RoutingHeader)
+  ))
+_sym_db.RegisterMessage(RoutingHeader)
+
+ResponseHeader = _reflection.GeneratedProtocolMessageType('ResponseHeader', (_message.Message,), dict(
+  DESCRIPTOR = _RESPONSEHEADER,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.ResponseHeader)
+  ))
+_sym_db.RegisterMessage(ResponseHeader)
+
+RangeRequest = _reflection.GeneratedProtocolMessageType('RangeRequest', (_message.Message,), dict(
+  DESCRIPTOR = _RANGEREQUEST,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.RangeRequest)
+  ))
+_sym_db.RegisterMessage(RangeRequest)
+
+RangeResponse = _reflection.GeneratedProtocolMessageType('RangeResponse', (_message.Message,), dict(
+  DESCRIPTOR = _RANGERESPONSE,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.RangeResponse)
+  ))
+_sym_db.RegisterMessage(RangeResponse)
+
+PutRequest = _reflection.GeneratedProtocolMessageType('PutRequest', (_message.Message,), dict(
+  DESCRIPTOR = _PUTREQUEST,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.PutRequest)
+  ))
+_sym_db.RegisterMessage(PutRequest)
+
+PutResponse = _reflection.GeneratedProtocolMessageType('PutResponse', (_message.Message,), dict(
+  DESCRIPTOR = _PUTRESPONSE,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.PutResponse)
+  ))
+_sym_db.RegisterMessage(PutResponse)
+
+IncrementRequest = _reflection.GeneratedProtocolMessageType('IncrementRequest', (_message.Message,), dict(
+  DESCRIPTOR = _INCREMENTREQUEST,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.IncrementRequest)
+  ))
+_sym_db.RegisterMessage(IncrementRequest)
+
+IncrementResponse = _reflection.GeneratedProtocolMessageType('IncrementResponse', (_message.Message,), dict(
+  DESCRIPTOR = _INCREMENTRESPONSE,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.IncrementResponse)
+  ))
+_sym_db.RegisterMessage(IncrementResponse)
+
+DeleteRangeRequest = _reflection.GeneratedProtocolMessageType('DeleteRangeRequest', (_message.Message,), dict(
+  DESCRIPTOR = _DELETERANGEREQUEST,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.DeleteRangeRequest)
+  ))
+_sym_db.RegisterMessage(DeleteRangeRequest)
+
+DeleteRangeResponse = _reflection.GeneratedProtocolMessageType('DeleteRangeResponse', (_message.Message,), dict(
+  DESCRIPTOR = _DELETERANGERESPONSE,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.DeleteRangeResponse)
+  ))
+_sym_db.RegisterMessage(DeleteRangeResponse)
+
+RequestOp = _reflection.GeneratedProtocolMessageType('RequestOp', (_message.Message,), dict(
+  DESCRIPTOR = _REQUESTOP,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.RequestOp)
+  ))
+_sym_db.RegisterMessage(RequestOp)
+
+ResponseOp = _reflection.GeneratedProtocolMessageType('ResponseOp', (_message.Message,), dict(
+  DESCRIPTOR = _RESPONSEOP,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.ResponseOp)
+  ))
+_sym_db.RegisterMessage(ResponseOp)
+
+Compare = _reflection.GeneratedProtocolMessageType('Compare', (_message.Message,), dict(
+  DESCRIPTOR = _COMPARE,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.Compare)
+  ))
+_sym_db.RegisterMessage(Compare)
+
+TxnRequest = _reflection.GeneratedProtocolMessageType('TxnRequest', (_message.Message,), dict(
+  DESCRIPTOR = _TXNREQUEST,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.TxnRequest)
+  ))
+_sym_db.RegisterMessage(TxnRequest)
+
+TxnResponse = _reflection.GeneratedProtocolMessageType('TxnResponse', (_message.Message,), dict(
+  DESCRIPTOR = _TXNRESPONSE,
+  __module__ = 'kv_rpc_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.rpc.TxnResponse)
+  ))
+_sym_db.RegisterMessage(TxnResponse)
+
+
+DESCRIPTOR._options = None
+
+_TABLESERVICE = _descriptor.ServiceDescriptor(
+  name='TableService',
+  full_name='bookkeeper.proto.kv.rpc.TableService',
+  file=DESCRIPTOR,
+  index=0,
+  serialized_options=None,
+  serialized_start=3023,
+  serialized_end=3500,
+  methods=[
+  _descriptor.MethodDescriptor(
+    name='Range',
+    full_name='bookkeeper.proto.kv.rpc.TableService.Range',
+    index=0,
+    containing_service=None,
+    input_type=_RANGEREQUEST,
+    output_type=_RANGERESPONSE,
+    serialized_options=None,
+  ),
+  _descriptor.MethodDescriptor(
+    name='Put',
+    full_name='bookkeeper.proto.kv.rpc.TableService.Put',
+    index=1,
+    containing_service=None,
+    input_type=_PUTREQUEST,
+    output_type=_PUTRESPONSE,
+    serialized_options=None,
+  ),
+  _descriptor.MethodDescriptor(
+    name='Delete',
+    full_name='bookkeeper.proto.kv.rpc.TableService.Delete',
+    index=2,
+    containing_service=None,
+    input_type=_DELETERANGEREQUEST,
+    output_type=_DELETERANGERESPONSE,
+    serialized_options=None,
+  ),
+  _descriptor.MethodDescriptor(
+    name='Txn',
+    full_name='bookkeeper.proto.kv.rpc.TableService.Txn',
+    index=3,
+    containing_service=None,
+    input_type=_TXNREQUEST,
+    output_type=_TXNRESPONSE,
+    serialized_options=None,
+  ),
+  _descriptor.MethodDescriptor(
+    name='Increment',
+    full_name='bookkeeper.proto.kv.rpc.TableService.Increment',
+    index=4,
+    containing_service=None,
+    input_type=_INCREMENTREQUEST,
+    output_type=_INCREMENTRESPONSE,
+    serialized_options=None,
+  ),
+])
+_sym_db.RegisterServiceDescriptor(_TABLESERVICE)
+
+DESCRIPTOR.services_by_name['TableService'] = _TABLESERVICE
+
+# @@protoc_insertion_point(module_scope)
diff --git a/stream/clients/python/bookkeeper/proto/kv_rpc_pb2_grpc.py b/stream/clients/python/bookkeeper/proto/kv_rpc_pb2_grpc.py
new file mode 100644
index 0000000..364ab74
--- /dev/null
+++ b/stream/clients/python/bookkeeper/proto/kv_rpc_pb2_grpc.py
@@ -0,0 +1,122 @@
+# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT!
+import grpc
+
+import bookkeeper.proto.kv_rpc_pb2 as kv__rpc__pb2
+
+
+class TableServiceStub(object):
+  # missing associated documentation comment in .proto file
+  pass
+
+  def __init__(self, channel):
+    """Constructor.
+
+    Args:
+      channel: A grpc.Channel.
+    """
+    self.Range = channel.unary_unary(
+        '/bookkeeper.proto.kv.rpc.TableService/Range',
+        request_serializer=kv__rpc__pb2.RangeRequest.SerializeToString,
+        response_deserializer=kv__rpc__pb2.RangeResponse.FromString,
+        )
+    self.Put = channel.unary_unary(
+        '/bookkeeper.proto.kv.rpc.TableService/Put',
+        request_serializer=kv__rpc__pb2.PutRequest.SerializeToString,
+        response_deserializer=kv__rpc__pb2.PutResponse.FromString,
+        )
+    self.Delete = channel.unary_unary(
+        '/bookkeeper.proto.kv.rpc.TableService/Delete',
+        request_serializer=kv__rpc__pb2.DeleteRangeRequest.SerializeToString,
+        response_deserializer=kv__rpc__pb2.DeleteRangeResponse.FromString,
+        )
+    self.Txn = channel.unary_unary(
+        '/bookkeeper.proto.kv.rpc.TableService/Txn',
+        request_serializer=kv__rpc__pb2.TxnRequest.SerializeToString,
+        response_deserializer=kv__rpc__pb2.TxnResponse.FromString,
+        )
+    self.Increment = channel.unary_unary(
+        '/bookkeeper.proto.kv.rpc.TableService/Increment',
+        request_serializer=kv__rpc__pb2.IncrementRequest.SerializeToString,
+        response_deserializer=kv__rpc__pb2.IncrementResponse.FromString,
+        )
+
+
+class TableServiceServicer(object):
+  # missing associated documentation comment in .proto file
+  pass
+
+  def Range(self, request, context):
+    """Range gets the keys in the range from the key-value store.
+    NOT supported yet.
+    """
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+  def Put(self, request, context):
+    """Put puts the given key into the key-value store.
+    A put request increments the revision of the key-value store
+    and generates one event in the event history.
+    """
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+  def Delete(self, request, context):
+    """DeleteRange deletes the given range from the key-value store.
+    A delete request increments the revision of the key-value store
+    and generates a delete event in the event history for every deleted key.
+    """
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+  def Txn(self, request, context):
+    """Txn processes multiple requests in a single transaction.
+    A txn request increments the revision of the key-value store
+    and generates events with the same revision for every completed request.
+    It is not allowed to modify the same key several times within one txn.
+    """
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+  def Increment(self, request, context):
+    """Increment increments the amount associated with the keys
+    """
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+
+def add_TableServiceServicer_to_server(servicer, server):
+  rpc_method_handlers = {
+      'Range': grpc.unary_unary_rpc_method_handler(
+          servicer.Range,
+          request_deserializer=kv__rpc__pb2.RangeRequest.FromString,
+          response_serializer=kv__rpc__pb2.RangeResponse.SerializeToString,
+      ),
+      'Put': grpc.unary_unary_rpc_method_handler(
+          servicer.Put,
+          request_deserializer=kv__rpc__pb2.PutRequest.FromString,
+          response_serializer=kv__rpc__pb2.PutResponse.SerializeToString,
+      ),
+      'Delete': grpc.unary_unary_rpc_method_handler(
+          servicer.Delete,
+          request_deserializer=kv__rpc__pb2.DeleteRangeRequest.FromString,
+          response_serializer=kv__rpc__pb2.DeleteRangeResponse.SerializeToString,
+      ),
+      'Txn': grpc.unary_unary_rpc_method_handler(
+          servicer.Txn,
+          request_deserializer=kv__rpc__pb2.TxnRequest.FromString,
+          response_serializer=kv__rpc__pb2.TxnResponse.SerializeToString,
+      ),
+      'Increment': grpc.unary_unary_rpc_method_handler(
+          servicer.Increment,
+          request_deserializer=kv__rpc__pb2.IncrementRequest.FromString,
+          response_serializer=kv__rpc__pb2.IncrementResponse.SerializeToString,
+      ),
+  }
+  generic_handler = grpc.method_handlers_generic_handler(
+      'bookkeeper.proto.kv.rpc.TableService', rpc_method_handlers)
+  server.add_generic_rpc_handlers((generic_handler,))
diff --git a/stream/clients/python/bookkeeper/proto/kv_store_pb2.py b/stream/clients/python/bookkeeper/proto/kv_store_pb2.py
new file mode 100644
index 0000000..7c4ea29
--- /dev/null
+++ b/stream/clients/python/bookkeeper/proto/kv_store_pb2.py
@@ -0,0 +1,297 @@
+# Generated by the protocol buffer compiler.  DO NOT EDIT!
+# source: kv_store.proto
+
+import sys
+_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1'))
+from google.protobuf.internal import enum_type_wrapper
+from google.protobuf import descriptor as _descriptor
+from google.protobuf import message as _message
+from google.protobuf import reflection as _reflection
+from google.protobuf import symbol_database as _symbol_database
+# @@protoc_insertion_point(imports)
+
+_sym_db = _symbol_database.Default()
+
+
+import bookkeeper.proto.kv_pb2 as kv__pb2
+import bookkeeper.proto.kv_rpc_pb2 as kv__rpc__pb2
+
+
+DESCRIPTOR = _descriptor.FileDescriptor(
+  name='kv_store.proto',
+  package='bookkeeper.proto.kv.store',
+  syntax='proto3',
+  serialized_options=_b('\n+org.apache.bookkeeper.stream.proto.kv.storeP\001'),
+  serialized_pb=_b('\n\x0ekv_store.proto\x12\x19\x62ookkeeper.proto.kv.store\x1a\x08kv.proto\x1a\x0ckv_rpc.proto\"\x83\x01\n\x07KeyMeta\x12\x17\n\x0f\x63reate_revision\x18\x01 \x01(\x03\x12\x14\n\x0cmod_revision\x18\x02 \x01(\x03\x12\x0f\n\x07version\x18\x03 \x01(\x03\x12\x38\n\nvalue_type\x18\x04 \x01(\x0e\x32$.bookkeeper.proto.kv.store.ValueType\"\x0c\n\nNopRequest\"\xbc\x02\n\x07\x43ommand\x12\x38\n\x07nop_req\x18\x01 \x01(\x0b\x32%.bookkeeper.proto.kv.store.NopRequestH\x00\x12\x36\n\ [...]
+  ,
+  dependencies=[kv__pb2.DESCRIPTOR,kv__rpc__pb2.DESCRIPTOR,])
+
+_VALUETYPE = _descriptor.EnumDescriptor(
+  name='ValueType',
+  full_name='bookkeeper.proto.kv.store.ValueType',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='BYTES', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='NUMBER', index=1, number=1,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=607,
+  serialized_end=641,
+)
+_sym_db.RegisterEnumDescriptor(_VALUETYPE)
+
+ValueType = enum_type_wrapper.EnumTypeWrapper(_VALUETYPE)
+BYTES = 0
+NUMBER = 1
+
+
+
+_KEYMETA = _descriptor.Descriptor(
+  name='KeyMeta',
+  full_name='bookkeeper.proto.kv.store.KeyMeta',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='create_revision', full_name='bookkeeper.proto.kv.store.KeyMeta.create_revision', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='mod_revision', full_name='bookkeeper.proto.kv.store.KeyMeta.mod_revision', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='version', full_name='bookkeeper.proto.kv.store.KeyMeta.version', index=2,
+      number=3, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='value_type', full_name='bookkeeper.proto.kv.store.KeyMeta.value_type', index=3,
+      number=4, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=70,
+  serialized_end=201,
+)
+
+
+_NOPREQUEST = _descriptor.Descriptor(
+  name='NopRequest',
+  full_name='bookkeeper.proto.kv.store.NopRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=203,
+  serialized_end=215,
+)
+
+
+_COMMAND = _descriptor.Descriptor(
+  name='Command',
+  full_name='bookkeeper.proto.kv.store.Command',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='nop_req', full_name='bookkeeper.proto.kv.store.Command.nop_req', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='put_req', full_name='bookkeeper.proto.kv.store.Command.put_req', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='delete_req', full_name='bookkeeper.proto.kv.store.Command.delete_req', index=2,
+      number=3, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='txn_req', full_name='bookkeeper.proto.kv.store.Command.txn_req', index=3,
+      number=4, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='incr_req', full_name='bookkeeper.proto.kv.store.Command.incr_req', index=4,
+      number=5, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+    _descriptor.OneofDescriptor(
+      name='req', full_name='bookkeeper.proto.kv.store.Command.req',
+      index=0, containing_type=None, fields=[]),
+  ],
+  serialized_start=218,
+  serialized_end=534,
+)
+
+
+_CHECKPOINTMETADATA = _descriptor.Descriptor(
+  name='CheckpointMetadata',
+  full_name='bookkeeper.proto.kv.store.CheckpointMetadata',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='files', full_name='bookkeeper.proto.kv.store.CheckpointMetadata.files', index=0,
+      number=1, type=9, cpp_type=9, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='txid', full_name='bookkeeper.proto.kv.store.CheckpointMetadata.txid', index=1,
+      number=2, type=12, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b(""),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='created_at', full_name='bookkeeper.proto.kv.store.CheckpointMetadata.created_at', index=2,
+      number=3, type=4, cpp_type=4, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=536,
+  serialized_end=605,
+)
+
+_KEYMETA.fields_by_name['value_type'].enum_type = _VALUETYPE
+_COMMAND.fields_by_name['nop_req'].message_type = _NOPREQUEST
+_COMMAND.fields_by_name['put_req'].message_type = kv__rpc__pb2._PUTREQUEST
+_COMMAND.fields_by_name['delete_req'].message_type = kv__rpc__pb2._DELETERANGEREQUEST
+_COMMAND.fields_by_name['txn_req'].message_type = kv__rpc__pb2._TXNREQUEST
+_COMMAND.fields_by_name['incr_req'].message_type = kv__rpc__pb2._INCREMENTREQUEST
+_COMMAND.oneofs_by_name['req'].fields.append(
+  _COMMAND.fields_by_name['nop_req'])
+_COMMAND.fields_by_name['nop_req'].containing_oneof = _COMMAND.oneofs_by_name['req']
+_COMMAND.oneofs_by_name['req'].fields.append(
+  _COMMAND.fields_by_name['put_req'])
+_COMMAND.fields_by_name['put_req'].containing_oneof = _COMMAND.oneofs_by_name['req']
+_COMMAND.oneofs_by_name['req'].fields.append(
+  _COMMAND.fields_by_name['delete_req'])
+_COMMAND.fields_by_name['delete_req'].containing_oneof = _COMMAND.oneofs_by_name['req']
+_COMMAND.oneofs_by_name['req'].fields.append(
+  _COMMAND.fields_by_name['txn_req'])
+_COMMAND.fields_by_name['txn_req'].containing_oneof = _COMMAND.oneofs_by_name['req']
+_COMMAND.oneofs_by_name['req'].fields.append(
+  _COMMAND.fields_by_name['incr_req'])
+_COMMAND.fields_by_name['incr_req'].containing_oneof = _COMMAND.oneofs_by_name['req']
+DESCRIPTOR.message_types_by_name['KeyMeta'] = _KEYMETA
+DESCRIPTOR.message_types_by_name['NopRequest'] = _NOPREQUEST
+DESCRIPTOR.message_types_by_name['Command'] = _COMMAND
+DESCRIPTOR.message_types_by_name['CheckpointMetadata'] = _CHECKPOINTMETADATA
+DESCRIPTOR.enum_types_by_name['ValueType'] = _VALUETYPE
+_sym_db.RegisterFileDescriptor(DESCRIPTOR)
+
+KeyMeta = _reflection.GeneratedProtocolMessageType('KeyMeta', (_message.Message,), dict(
+  DESCRIPTOR = _KEYMETA,
+  __module__ = 'kv_store_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.store.KeyMeta)
+  ))
+_sym_db.RegisterMessage(KeyMeta)
+
+NopRequest = _reflection.GeneratedProtocolMessageType('NopRequest', (_message.Message,), dict(
+  DESCRIPTOR = _NOPREQUEST,
+  __module__ = 'kv_store_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.store.NopRequest)
+  ))
+_sym_db.RegisterMessage(NopRequest)
+
+Command = _reflection.GeneratedProtocolMessageType('Command', (_message.Message,), dict(
+  DESCRIPTOR = _COMMAND,
+  __module__ = 'kv_store_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.store.Command)
+  ))
+_sym_db.RegisterMessage(Command)
+
+CheckpointMetadata = _reflection.GeneratedProtocolMessageType('CheckpointMetadata', (_message.Message,), dict(
+  DESCRIPTOR = _CHECKPOINTMETADATA,
+  __module__ = 'kv_store_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.kv.store.CheckpointMetadata)
+  ))
+_sym_db.RegisterMessage(CheckpointMetadata)
+
+
+DESCRIPTOR._options = None
+# @@protoc_insertion_point(module_scope)
diff --git a/stream/clients/python/bookkeeper/proto/storage_pb2.py b/stream/clients/python/bookkeeper/proto/storage_pb2.py
new file mode 100644
index 0000000..7f7854f
--- /dev/null
+++ b/stream/clients/python/bookkeeper/proto/storage_pb2.py
@@ -0,0 +1,1276 @@
+# Generated by the protocol buffer compiler.  DO NOT EDIT!
+# source: storage.proto
+
+import sys
+_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1'))
+from google.protobuf.internal import enum_type_wrapper
+from google.protobuf import descriptor as _descriptor
+from google.protobuf import message as _message
+from google.protobuf import reflection as _reflection
+from google.protobuf import symbol_database as _symbol_database
+# @@protoc_insertion_point(imports)
+
+_sym_db = _symbol_database.Default()
+
+
+import bookkeeper.proto.common_pb2 as common__pb2
+import bookkeeper.proto.stream_pb2 as stream__pb2
+
+
+DESCRIPTOR = _descriptor.FileDescriptor(
+  name='storage.proto',
+  package='bookkeeper.proto.storage',
+  syntax='proto3',
+  serialized_options=_b('\n*org.apache.bookkeeper.stream.proto.storageP\001'),
+  serialized_pb=_b('\n\rstorage.proto\x12\x18\x62ookkeeper.proto.storage\x1a\x0c\x63ommon.proto\x1a\x0cstream.proto\"l\n\x16GetActiveRangesRequest\x12\x11\n\tstream_id\x18\x01 \x01(\x03\x12?\n\x0cstream_props\x18\x02 \x01(\x0b\x32).bookkeeper.proto.stream.StreamProperties\"\x86\x01\n\x17GetActiveRangesResponse\x12\x32\n\x04\x63ode\x18\x01 \x01(\x0e\x32$.bookkeeper.proto.storage.StatusCode\x12\x37\n\x06ranges\x18\x02 \x03(\x0b\x32\'.bookkeeper.proto.storage.RelatedRanges\"\x96\x01\n\rRela [...]
+  ,
+  dependencies=[common__pb2.DESCRIPTOR,stream__pb2.DESCRIPTOR,])
+
+_STATUSCODE = _descriptor.EnumDescriptor(
+  name='StatusCode',
+  full_name='bookkeeper.proto.storage.StatusCode',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='SUCCESS', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='FAILURE', index=1, number=1,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='BAD_REQUEST', index=2, number=400,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='ILLEGAL_OP', index=3, number=403,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='INTERNAL_SERVER_ERROR', index=4, number=500,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='NOT_IMPLEMENTED', index=5, number=501,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='UNEXPECTED', index=6, number=600,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='BAD_VERSION', index=7, number=900,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='BAD_REVISION', index=8, number=901,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='INVALID_NAMESPACE_NAME', index=9, number=2000,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='NAMESPACE_EXISTS', index=10, number=2001,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='NAMESPACE_NOT_FOUND', index=11, number=2002,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='INVALID_STREAM_NAME', index=12, number=2100,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='STREAM_EXISTS', index=13, number=2101,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='STREAM_NOT_FOUND', index=14, number=2102,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='INVALID_STREAMSNAPSHOT_NAME', index=15, number=2103,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='STREAMSNAPSHOT_EXISTS', index=16, number=2104,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='STREAMSNAPSHOT_NOT_FOUND', index=17, number=2105,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='WRONG_GROUP_LOCATION', index=18, number=3000,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='INVALID_GROUP_ID', index=19, number=3001,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='GROUP_EXISTS', index=20, number=3002,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='GROUP_NOT_FOUND', index=21, number=3003,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='STALE_GROUP_INFO', index=22, number=3004,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='INVALID_KEY', index=23, number=6000,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='KEY_EXISTS', index=24, number=6001,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='KEY_NOT_FOUND', index=25, number=6002,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=2398,
+  serialized_end=2984,
+)
+_sym_db.RegisterEnumDescriptor(_STATUSCODE)
+
+StatusCode = enum_type_wrapper.EnumTypeWrapper(_STATUSCODE)
+_RELATIONTYPE = _descriptor.EnumDescriptor(
+  name='RelationType',
+  full_name='bookkeeper.proto.storage.RelationType',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='CHILDREN', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='PARENTS', index=1, number=1,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='CHILDREN_PARENTS', index=2, number=2,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=2986,
+  serialized_end=3049,
+)
+_sym_db.RegisterEnumDescriptor(_RELATIONTYPE)
+
+RelationType = enum_type_wrapper.EnumTypeWrapper(_RELATIONTYPE)
+SUCCESS = 0
+FAILURE = 1
+BAD_REQUEST = 400
+ILLEGAL_OP = 403
+INTERNAL_SERVER_ERROR = 500
+NOT_IMPLEMENTED = 501
+UNEXPECTED = 600
+BAD_VERSION = 900
+BAD_REVISION = 901
+INVALID_NAMESPACE_NAME = 2000
+NAMESPACE_EXISTS = 2001
+NAMESPACE_NOT_FOUND = 2002
+INVALID_STREAM_NAME = 2100
+STREAM_EXISTS = 2101
+STREAM_NOT_FOUND = 2102
+INVALID_STREAMSNAPSHOT_NAME = 2103
+STREAMSNAPSHOT_EXISTS = 2104
+STREAMSNAPSHOT_NOT_FOUND = 2105
+WRONG_GROUP_LOCATION = 3000
+INVALID_GROUP_ID = 3001
+GROUP_EXISTS = 3002
+GROUP_NOT_FOUND = 3003
+STALE_GROUP_INFO = 3004
+INVALID_KEY = 6000
+KEY_EXISTS = 6001
+KEY_NOT_FOUND = 6002
+CHILDREN = 0
+PARENTS = 1
+CHILDREN_PARENTS = 2
+
+
+
+_GETACTIVERANGESREQUEST = _descriptor.Descriptor(
+  name='GetActiveRangesRequest',
+  full_name='bookkeeper.proto.storage.GetActiveRangesRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='stream_id', full_name='bookkeeper.proto.storage.GetActiveRangesRequest.stream_id', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='stream_props', full_name='bookkeeper.proto.storage.GetActiveRangesRequest.stream_props', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=71,
+  serialized_end=179,
+)
+
+
+_GETACTIVERANGESRESPONSE = _descriptor.Descriptor(
+  name='GetActiveRangesResponse',
+  full_name='bookkeeper.proto.storage.GetActiveRangesResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='code', full_name='bookkeeper.proto.storage.GetActiveRangesResponse.code', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='ranges', full_name='bookkeeper.proto.storage.GetActiveRangesResponse.ranges', index=1,
+      number=2, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=182,
+  serialized_end=316,
+)
+
+
+_RELATEDRANGES = _descriptor.Descriptor(
+  name='RelatedRanges',
+  full_name='bookkeeper.proto.storage.RelatedRanges',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='props', full_name='bookkeeper.proto.storage.RelatedRanges.props', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='type', full_name='bookkeeper.proto.storage.RelatedRanges.type', index=1,
+      number=2, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='related_ranges', full_name='bookkeeper.proto.storage.RelatedRanges.related_ranges', index=2,
+      number=3, type=3, cpp_type=2, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=319,
+  serialized_end=469,
+)
+
+
+_CREATENAMESPACEREQUEST = _descriptor.Descriptor(
+  name='CreateNamespaceRequest',
+  full_name='bookkeeper.proto.storage.CreateNamespaceRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='name', full_name='bookkeeper.proto.storage.CreateNamespaceRequest.name', index=0,
+      number=1, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='ns_conf', full_name='bookkeeper.proto.storage.CreateNamespaceRequest.ns_conf', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=471,
+  serialized_end=575,
+)
+
+
+_CREATENAMESPACERESPONSE = _descriptor.Descriptor(
+  name='CreateNamespaceResponse',
+  full_name='bookkeeper.proto.storage.CreateNamespaceResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='code', full_name='bookkeeper.proto.storage.CreateNamespaceResponse.code', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='ns_props', full_name='bookkeeper.proto.storage.CreateNamespaceResponse.ns_props', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=578,
+  serialized_end=719,
+)
+
+
+_DELETENAMESPACEREQUEST = _descriptor.Descriptor(
+  name='DeleteNamespaceRequest',
+  full_name='bookkeeper.proto.storage.DeleteNamespaceRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='name', full_name='bookkeeper.proto.storage.DeleteNamespaceRequest.name', index=0,
+      number=1, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=721,
+  serialized_end=759,
+)
+
+
+_DELETENAMESPACERESPONSE = _descriptor.Descriptor(
+  name='DeleteNamespaceResponse',
+  full_name='bookkeeper.proto.storage.DeleteNamespaceResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='code', full_name='bookkeeper.proto.storage.DeleteNamespaceResponse.code', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=761,
+  serialized_end=838,
+)
+
+
+_GETNAMESPACEREQUEST = _descriptor.Descriptor(
+  name='GetNamespaceRequest',
+  full_name='bookkeeper.proto.storage.GetNamespaceRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='name', full_name='bookkeeper.proto.storage.GetNamespaceRequest.name', index=0,
+      number=1, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=840,
+  serialized_end=875,
+)
+
+
+_GETNAMESPACERESPONSE = _descriptor.Descriptor(
+  name='GetNamespaceResponse',
+  full_name='bookkeeper.proto.storage.GetNamespaceResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='code', full_name='bookkeeper.proto.storage.GetNamespaceResponse.code', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='ns_props', full_name='bookkeeper.proto.storage.GetNamespaceResponse.ns_props', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=878,
+  serialized_end=1016,
+)
+
+
+_CREATESTREAMREQUEST = _descriptor.Descriptor(
+  name='CreateStreamRequest',
+  full_name='bookkeeper.proto.storage.CreateStreamRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='ns_name', full_name='bookkeeper.proto.storage.CreateStreamRequest.ns_name', index=0,
+      number=1, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='name', full_name='bookkeeper.proto.storage.CreateStreamRequest.name', index=1,
+      number=2, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='stream_conf', full_name='bookkeeper.proto.storage.CreateStreamRequest.stream_conf', index=2,
+      number=3, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1018,
+  serialized_end=1137,
+)
+
+
+_CREATESTREAMRESPONSE = _descriptor.Descriptor(
+  name='CreateStreamResponse',
+  full_name='bookkeeper.proto.storage.CreateStreamResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='code', full_name='bookkeeper.proto.storage.CreateStreamResponse.code', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='stream_props', full_name='bookkeeper.proto.storage.CreateStreamResponse.stream_props', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1140,
+  serialized_end=1279,
+)
+
+
+_DELETESTREAMREQUEST = _descriptor.Descriptor(
+  name='DeleteStreamRequest',
+  full_name='bookkeeper.proto.storage.DeleteStreamRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='ns_name', full_name='bookkeeper.proto.storage.DeleteStreamRequest.ns_name', index=0,
+      number=1, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='name', full_name='bookkeeper.proto.storage.DeleteStreamRequest.name', index=1,
+      number=2, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1281,
+  serialized_end=1333,
+)
+
+
+_DELETESTREAMRESPONSE = _descriptor.Descriptor(
+  name='DeleteStreamResponse',
+  full_name='bookkeeper.proto.storage.DeleteStreamResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='code', full_name='bookkeeper.proto.storage.DeleteStreamResponse.code', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1335,
+  serialized_end=1409,
+)
+
+
+_GETSTREAMREQUEST = _descriptor.Descriptor(
+  name='GetStreamRequest',
+  full_name='bookkeeper.proto.storage.GetStreamRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='stream_name', full_name='bookkeeper.proto.storage.GetStreamRequest.stream_name', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='stream_id', full_name='bookkeeper.proto.storage.GetStreamRequest.stream_id', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+    _descriptor.OneofDescriptor(
+      name='id', full_name='bookkeeper.proto.storage.GetStreamRequest.id',
+      index=0, containing_type=None, fields=[]),
+  ],
+  serialized_start=1411,
+  serialized_end=1516,
+)
+
+
+_GETSTREAMRESPONSE = _descriptor.Descriptor(
+  name='GetStreamResponse',
+  full_name='bookkeeper.proto.storage.GetStreamResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='code', full_name='bookkeeper.proto.storage.GetStreamResponse.code', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='stream_props', full_name='bookkeeper.proto.storage.GetStreamResponse.stream_props', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1519,
+  serialized_end=1655,
+)
+
+
+_STORAGECONTAINERENDPOINT = _descriptor.Descriptor(
+  name='StorageContainerEndpoint',
+  full_name='bookkeeper.proto.storage.StorageContainerEndpoint',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='storage_container_id', full_name='bookkeeper.proto.storage.StorageContainerEndpoint.storage_container_id', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='revision', full_name='bookkeeper.proto.storage.StorageContainerEndpoint.revision', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='rw_endpoint', full_name='bookkeeper.proto.storage.StorageContainerEndpoint.rw_endpoint', index=2,
+      number=3, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='ro_endpoint', full_name='bookkeeper.proto.storage.StorageContainerEndpoint.ro_endpoint', index=3,
+      number=4, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1658,
+  serialized_end=1844,
+)
+
+
+_ONESTORAGECONTAINERENDPOINTREQUEST = _descriptor.Descriptor(
+  name='OneStorageContainerEndpointRequest',
+  full_name='bookkeeper.proto.storage.OneStorageContainerEndpointRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='storage_container', full_name='bookkeeper.proto.storage.OneStorageContainerEndpointRequest.storage_container', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='revision', full_name='bookkeeper.proto.storage.OneStorageContainerEndpointRequest.revision', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1846,
+  serialized_end=1927,
+)
+
+
+_GETSTORAGECONTAINERENDPOINTREQUEST = _descriptor.Descriptor(
+  name='GetStorageContainerEndpointRequest',
+  full_name='bookkeeper.proto.storage.GetStorageContainerEndpointRequest',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='requests', full_name='bookkeeper.proto.storage.GetStorageContainerEndpointRequest.requests', index=0,
+      number=1, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1929,
+  serialized_end=2045,
+)
+
+
+_ONESTORAGECONTAINERENDPOINTRESPONSE = _descriptor.Descriptor(
+  name='OneStorageContainerEndpointResponse',
+  full_name='bookkeeper.proto.storage.OneStorageContainerEndpointResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='status_code', full_name='bookkeeper.proto.storage.OneStorageContainerEndpointResponse.status_code', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='endpoint', full_name='bookkeeper.proto.storage.OneStorageContainerEndpointResponse.endpoint', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=2048,
+  serialized_end=2214,
+)
+
+
+_GETSTORAGECONTAINERENDPOINTRESPONSE = _descriptor.Descriptor(
+  name='GetStorageContainerEndpointResponse',
+  full_name='bookkeeper.proto.storage.GetStorageContainerEndpointResponse',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='status_code', full_name='bookkeeper.proto.storage.GetStorageContainerEndpointResponse.status_code', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='responses', full_name='bookkeeper.proto.storage.GetStorageContainerEndpointResponse.responses', index=1,
+      number=2, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=2217,
+  serialized_end=2395,
+)
+
+_GETACTIVERANGESREQUEST.fields_by_name['stream_props'].message_type = stream__pb2._STREAMPROPERTIES
+_GETACTIVERANGESRESPONSE.fields_by_name['code'].enum_type = _STATUSCODE
+_GETACTIVERANGESRESPONSE.fields_by_name['ranges'].message_type = _RELATEDRANGES
+_RELATEDRANGES.fields_by_name['props'].message_type = stream__pb2._RANGEPROPERTIES
+_RELATEDRANGES.fields_by_name['type'].enum_type = _RELATIONTYPE
+_CREATENAMESPACEREQUEST.fields_by_name['ns_conf'].message_type = stream__pb2._NAMESPACECONFIGURATION
+_CREATENAMESPACERESPONSE.fields_by_name['code'].enum_type = _STATUSCODE
+_CREATENAMESPACERESPONSE.fields_by_name['ns_props'].message_type = stream__pb2._NAMESPACEPROPERTIES
+_DELETENAMESPACERESPONSE.fields_by_name['code'].enum_type = _STATUSCODE
+_GETNAMESPACERESPONSE.fields_by_name['code'].enum_type = _STATUSCODE
+_GETNAMESPACERESPONSE.fields_by_name['ns_props'].message_type = stream__pb2._NAMESPACEPROPERTIES
+_CREATESTREAMREQUEST.fields_by_name['stream_conf'].message_type = stream__pb2._STREAMCONFIGURATION
+_CREATESTREAMRESPONSE.fields_by_name['code'].enum_type = _STATUSCODE
+_CREATESTREAMRESPONSE.fields_by_name['stream_props'].message_type = stream__pb2._STREAMPROPERTIES
+_DELETESTREAMRESPONSE.fields_by_name['code'].enum_type = _STATUSCODE
+_GETSTREAMREQUEST.fields_by_name['stream_name'].message_type = stream__pb2._STREAMNAME
+_GETSTREAMREQUEST.oneofs_by_name['id'].fields.append(
+  _GETSTREAMREQUEST.fields_by_name['stream_name'])
+_GETSTREAMREQUEST.fields_by_name['stream_name'].containing_oneof = _GETSTREAMREQUEST.oneofs_by_name['id']
+_GETSTREAMREQUEST.oneofs_by_name['id'].fields.append(
+  _GETSTREAMREQUEST.fields_by_name['stream_id'])
+_GETSTREAMREQUEST.fields_by_name['stream_id'].containing_oneof = _GETSTREAMREQUEST.oneofs_by_name['id']
+_GETSTREAMRESPONSE.fields_by_name['code'].enum_type = _STATUSCODE
+_GETSTREAMRESPONSE.fields_by_name['stream_props'].message_type = stream__pb2._STREAMPROPERTIES
+_STORAGECONTAINERENDPOINT.fields_by_name['rw_endpoint'].message_type = common__pb2._ENDPOINT
+_STORAGECONTAINERENDPOINT.fields_by_name['ro_endpoint'].message_type = common__pb2._ENDPOINT
+_GETSTORAGECONTAINERENDPOINTREQUEST.fields_by_name['requests'].message_type = _ONESTORAGECONTAINERENDPOINTREQUEST
+_ONESTORAGECONTAINERENDPOINTRESPONSE.fields_by_name['status_code'].enum_type = _STATUSCODE
+_ONESTORAGECONTAINERENDPOINTRESPONSE.fields_by_name['endpoint'].message_type = _STORAGECONTAINERENDPOINT
+_GETSTORAGECONTAINERENDPOINTRESPONSE.fields_by_name['status_code'].enum_type = _STATUSCODE
+_GETSTORAGECONTAINERENDPOINTRESPONSE.fields_by_name['responses'].message_type = _ONESTORAGECONTAINERENDPOINTRESPONSE
+DESCRIPTOR.message_types_by_name['GetActiveRangesRequest'] = _GETACTIVERANGESREQUEST
+DESCRIPTOR.message_types_by_name['GetActiveRangesResponse'] = _GETACTIVERANGESRESPONSE
+DESCRIPTOR.message_types_by_name['RelatedRanges'] = _RELATEDRANGES
+DESCRIPTOR.message_types_by_name['CreateNamespaceRequest'] = _CREATENAMESPACEREQUEST
+DESCRIPTOR.message_types_by_name['CreateNamespaceResponse'] = _CREATENAMESPACERESPONSE
+DESCRIPTOR.message_types_by_name['DeleteNamespaceRequest'] = _DELETENAMESPACEREQUEST
+DESCRIPTOR.message_types_by_name['DeleteNamespaceResponse'] = _DELETENAMESPACERESPONSE
+DESCRIPTOR.message_types_by_name['GetNamespaceRequest'] = _GETNAMESPACEREQUEST
+DESCRIPTOR.message_types_by_name['GetNamespaceResponse'] = _GETNAMESPACERESPONSE
+DESCRIPTOR.message_types_by_name['CreateStreamRequest'] = _CREATESTREAMREQUEST
+DESCRIPTOR.message_types_by_name['CreateStreamResponse'] = _CREATESTREAMRESPONSE
+DESCRIPTOR.message_types_by_name['DeleteStreamRequest'] = _DELETESTREAMREQUEST
+DESCRIPTOR.message_types_by_name['DeleteStreamResponse'] = _DELETESTREAMRESPONSE
+DESCRIPTOR.message_types_by_name['GetStreamRequest'] = _GETSTREAMREQUEST
+DESCRIPTOR.message_types_by_name['GetStreamResponse'] = _GETSTREAMRESPONSE
+DESCRIPTOR.message_types_by_name['StorageContainerEndpoint'] = _STORAGECONTAINERENDPOINT
+DESCRIPTOR.message_types_by_name['OneStorageContainerEndpointRequest'] = _ONESTORAGECONTAINERENDPOINTREQUEST
+DESCRIPTOR.message_types_by_name['GetStorageContainerEndpointRequest'] = _GETSTORAGECONTAINERENDPOINTREQUEST
+DESCRIPTOR.message_types_by_name['OneStorageContainerEndpointResponse'] = _ONESTORAGECONTAINERENDPOINTRESPONSE
+DESCRIPTOR.message_types_by_name['GetStorageContainerEndpointResponse'] = _GETSTORAGECONTAINERENDPOINTRESPONSE
+DESCRIPTOR.enum_types_by_name['StatusCode'] = _STATUSCODE
+DESCRIPTOR.enum_types_by_name['RelationType'] = _RELATIONTYPE
+_sym_db.RegisterFileDescriptor(DESCRIPTOR)
+
+GetActiveRangesRequest = _reflection.GeneratedProtocolMessageType('GetActiveRangesRequest', (_message.Message,), dict(
+  DESCRIPTOR = _GETACTIVERANGESREQUEST,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetActiveRangesRequest)
+  ))
+_sym_db.RegisterMessage(GetActiveRangesRequest)
+
+GetActiveRangesResponse = _reflection.GeneratedProtocolMessageType('GetActiveRangesResponse', (_message.Message,), dict(
+  DESCRIPTOR = _GETACTIVERANGESRESPONSE,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetActiveRangesResponse)
+  ))
+_sym_db.RegisterMessage(GetActiveRangesResponse)
+
+RelatedRanges = _reflection.GeneratedProtocolMessageType('RelatedRanges', (_message.Message,), dict(
+  DESCRIPTOR = _RELATEDRANGES,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.RelatedRanges)
+  ))
+_sym_db.RegisterMessage(RelatedRanges)
+
+CreateNamespaceRequest = _reflection.GeneratedProtocolMessageType('CreateNamespaceRequest', (_message.Message,), dict(
+  DESCRIPTOR = _CREATENAMESPACEREQUEST,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.CreateNamespaceRequest)
+  ))
+_sym_db.RegisterMessage(CreateNamespaceRequest)
+
+CreateNamespaceResponse = _reflection.GeneratedProtocolMessageType('CreateNamespaceResponse', (_message.Message,), dict(
+  DESCRIPTOR = _CREATENAMESPACERESPONSE,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.CreateNamespaceResponse)
+  ))
+_sym_db.RegisterMessage(CreateNamespaceResponse)
+
+DeleteNamespaceRequest = _reflection.GeneratedProtocolMessageType('DeleteNamespaceRequest', (_message.Message,), dict(
+  DESCRIPTOR = _DELETENAMESPACEREQUEST,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.DeleteNamespaceRequest)
+  ))
+_sym_db.RegisterMessage(DeleteNamespaceRequest)
+
+DeleteNamespaceResponse = _reflection.GeneratedProtocolMessageType('DeleteNamespaceResponse', (_message.Message,), dict(
+  DESCRIPTOR = _DELETENAMESPACERESPONSE,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.DeleteNamespaceResponse)
+  ))
+_sym_db.RegisterMessage(DeleteNamespaceResponse)
+
+GetNamespaceRequest = _reflection.GeneratedProtocolMessageType('GetNamespaceRequest', (_message.Message,), dict(
+  DESCRIPTOR = _GETNAMESPACEREQUEST,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetNamespaceRequest)
+  ))
+_sym_db.RegisterMessage(GetNamespaceRequest)
+
+GetNamespaceResponse = _reflection.GeneratedProtocolMessageType('GetNamespaceResponse', (_message.Message,), dict(
+  DESCRIPTOR = _GETNAMESPACERESPONSE,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetNamespaceResponse)
+  ))
+_sym_db.RegisterMessage(GetNamespaceResponse)
+
+CreateStreamRequest = _reflection.GeneratedProtocolMessageType('CreateStreamRequest', (_message.Message,), dict(
+  DESCRIPTOR = _CREATESTREAMREQUEST,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.CreateStreamRequest)
+  ))
+_sym_db.RegisterMessage(CreateStreamRequest)
+
+CreateStreamResponse = _reflection.GeneratedProtocolMessageType('CreateStreamResponse', (_message.Message,), dict(
+  DESCRIPTOR = _CREATESTREAMRESPONSE,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.CreateStreamResponse)
+  ))
+_sym_db.RegisterMessage(CreateStreamResponse)
+
+DeleteStreamRequest = _reflection.GeneratedProtocolMessageType('DeleteStreamRequest', (_message.Message,), dict(
+  DESCRIPTOR = _DELETESTREAMREQUEST,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.DeleteStreamRequest)
+  ))
+_sym_db.RegisterMessage(DeleteStreamRequest)
+
+DeleteStreamResponse = _reflection.GeneratedProtocolMessageType('DeleteStreamResponse', (_message.Message,), dict(
+  DESCRIPTOR = _DELETESTREAMRESPONSE,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.DeleteStreamResponse)
+  ))
+_sym_db.RegisterMessage(DeleteStreamResponse)
+
+GetStreamRequest = _reflection.GeneratedProtocolMessageType('GetStreamRequest', (_message.Message,), dict(
+  DESCRIPTOR = _GETSTREAMREQUEST,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetStreamRequest)
+  ))
+_sym_db.RegisterMessage(GetStreamRequest)
+
+GetStreamResponse = _reflection.GeneratedProtocolMessageType('GetStreamResponse', (_message.Message,), dict(
+  DESCRIPTOR = _GETSTREAMRESPONSE,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetStreamResponse)
+  ))
+_sym_db.RegisterMessage(GetStreamResponse)
+
+StorageContainerEndpoint = _reflection.GeneratedProtocolMessageType('StorageContainerEndpoint', (_message.Message,), dict(
+  DESCRIPTOR = _STORAGECONTAINERENDPOINT,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.StorageContainerEndpoint)
+  ))
+_sym_db.RegisterMessage(StorageContainerEndpoint)
+
+OneStorageContainerEndpointRequest = _reflection.GeneratedProtocolMessageType('OneStorageContainerEndpointRequest', (_message.Message,), dict(
+  DESCRIPTOR = _ONESTORAGECONTAINERENDPOINTREQUEST,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.OneStorageContainerEndpointRequest)
+  ))
+_sym_db.RegisterMessage(OneStorageContainerEndpointRequest)
+
+GetStorageContainerEndpointRequest = _reflection.GeneratedProtocolMessageType('GetStorageContainerEndpointRequest', (_message.Message,), dict(
+  DESCRIPTOR = _GETSTORAGECONTAINERENDPOINTREQUEST,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetStorageContainerEndpointRequest)
+  ))
+_sym_db.RegisterMessage(GetStorageContainerEndpointRequest)
+
+OneStorageContainerEndpointResponse = _reflection.GeneratedProtocolMessageType('OneStorageContainerEndpointResponse', (_message.Message,), dict(
+  DESCRIPTOR = _ONESTORAGECONTAINERENDPOINTRESPONSE,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.OneStorageContainerEndpointResponse)
+  ))
+_sym_db.RegisterMessage(OneStorageContainerEndpointResponse)
+
+GetStorageContainerEndpointResponse = _reflection.GeneratedProtocolMessageType('GetStorageContainerEndpointResponse', (_message.Message,), dict(
+  DESCRIPTOR = _GETSTORAGECONTAINERENDPOINTRESPONSE,
+  __module__ = 'storage_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.storage.GetStorageContainerEndpointResponse)
+  ))
+_sym_db.RegisterMessage(GetStorageContainerEndpointResponse)
+
+
+DESCRIPTOR._options = None
+
+_METARANGESERVICE = _descriptor.ServiceDescriptor(
+  name='MetaRangeService',
+  full_name='bookkeeper.proto.storage.MetaRangeService',
+  file=DESCRIPTOR,
+  index=0,
+  serialized_options=None,
+  serialized_start=3052,
+  serialized_end=3190,
+  methods=[
+  _descriptor.MethodDescriptor(
+    name='GetActiveRanges',
+    full_name='bookkeeper.proto.storage.MetaRangeService.GetActiveRanges',
+    index=0,
+    containing_service=None,
+    input_type=_GETACTIVERANGESREQUEST,
+    output_type=_GETACTIVERANGESRESPONSE,
+    serialized_options=None,
+  ),
+])
+_sym_db.RegisterServiceDescriptor(_METARANGESERVICE)
+
+DESCRIPTOR.services_by_name['MetaRangeService'] = _METARANGESERVICE
+
+
+_ROOTRANGESERVICE = _descriptor.ServiceDescriptor(
+  name='RootRangeService',
+  full_name='bookkeeper.proto.storage.RootRangeService',
+  file=DESCRIPTOR,
+  index=1,
+  serialized_options=None,
+  serialized_start=3193,
+  serialized_end=3886,
+  methods=[
+  _descriptor.MethodDescriptor(
+    name='CreateNamespace',
+    full_name='bookkeeper.proto.storage.RootRangeService.CreateNamespace',
+    index=0,
+    containing_service=None,
+    input_type=_CREATENAMESPACEREQUEST,
+    output_type=_CREATENAMESPACERESPONSE,
+    serialized_options=None,
+  ),
+  _descriptor.MethodDescriptor(
+    name='DeleteNamespace',
+    full_name='bookkeeper.proto.storage.RootRangeService.DeleteNamespace',
+    index=1,
+    containing_service=None,
+    input_type=_DELETENAMESPACEREQUEST,
+    output_type=_DELETENAMESPACERESPONSE,
+    serialized_options=None,
+  ),
+  _descriptor.MethodDescriptor(
+    name='GetNamespace',
+    full_name='bookkeeper.proto.storage.RootRangeService.GetNamespace',
+    index=2,
+    containing_service=None,
+    input_type=_GETNAMESPACEREQUEST,
+    output_type=_GETNAMESPACERESPONSE,
+    serialized_options=None,
+  ),
+  _descriptor.MethodDescriptor(
+    name='CreateStream',
+    full_name='bookkeeper.proto.storage.RootRangeService.CreateStream',
+    index=3,
+    containing_service=None,
+    input_type=_CREATESTREAMREQUEST,
+    output_type=_CREATESTREAMRESPONSE,
+    serialized_options=None,
+  ),
+  _descriptor.MethodDescriptor(
+    name='DeleteStream',
+    full_name='bookkeeper.proto.storage.RootRangeService.DeleteStream',
+    index=4,
+    containing_service=None,
+    input_type=_DELETESTREAMREQUEST,
+    output_type=_DELETESTREAMRESPONSE,
+    serialized_options=None,
+  ),
+  _descriptor.MethodDescriptor(
+    name='GetStream',
+    full_name='bookkeeper.proto.storage.RootRangeService.GetStream',
+    index=5,
+    containing_service=None,
+    input_type=_GETSTREAMREQUEST,
+    output_type=_GETSTREAMRESPONSE,
+    serialized_options=None,
+  ),
+])
+_sym_db.RegisterServiceDescriptor(_ROOTRANGESERVICE)
+
+DESCRIPTOR.services_by_name['RootRangeService'] = _ROOTRANGESERVICE
+
+
+_STORAGECONTAINERSERVICE = _descriptor.ServiceDescriptor(
+  name='StorageContainerService',
+  full_name='bookkeeper.proto.storage.StorageContainerService',
+  file=DESCRIPTOR,
+  index=2,
+  serialized_options=None,
+  serialized_start=3889,
+  serialized_end=4071,
+  methods=[
+  _descriptor.MethodDescriptor(
+    name='GetStorageContainerEndpoint',
+    full_name='bookkeeper.proto.storage.StorageContainerService.GetStorageContainerEndpoint',
+    index=0,
+    containing_service=None,
+    input_type=_GETSTORAGECONTAINERENDPOINTREQUEST,
+    output_type=_GETSTORAGECONTAINERENDPOINTRESPONSE,
+    serialized_options=None,
+  ),
+])
+_sym_db.RegisterServiceDescriptor(_STORAGECONTAINERSERVICE)
+
+DESCRIPTOR.services_by_name['StorageContainerService'] = _STORAGECONTAINERSERVICE
+
+# @@protoc_insertion_point(module_scope)
diff --git a/stream/clients/python/bookkeeper/proto/storage_pb2_grpc.py b/stream/clients/python/bookkeeper/proto/storage_pb2_grpc.py
new file mode 100644
index 0000000..1f89bd3
--- /dev/null
+++ b/stream/clients/python/bookkeeper/proto/storage_pb2_grpc.py
@@ -0,0 +1,219 @@
+# Generated by the gRPC Python protocol compiler plugin. DO NOT EDIT!
+import grpc
+
+import bookkeeper.proto.storage_pb2 as storage__pb2
+
+
+class MetaRangeServiceStub(object):
+  """public service for other operations in range server
+  """
+
+  def __init__(self, channel):
+    """Constructor.
+
+    Args:
+      channel: A grpc.Channel.
+    """
+    self.GetActiveRanges = channel.unary_unary(
+        '/bookkeeper.proto.storage.MetaRangeService/GetActiveRanges',
+        request_serializer=storage__pb2.GetActiveRangesRequest.SerializeToString,
+        response_deserializer=storage__pb2.GetActiveRangesResponse.FromString,
+        )
+
+
+class MetaRangeServiceServicer(object):
+  """public service for other operations in range server
+  """
+
+  def GetActiveRanges(self, request, context):
+    # missing associated documentation comment in .proto file
+    pass
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+
+def add_MetaRangeServiceServicer_to_server(servicer, server):
+  rpc_method_handlers = {
+      'GetActiveRanges': grpc.unary_unary_rpc_method_handler(
+          servicer.GetActiveRanges,
+          request_deserializer=storage__pb2.GetActiveRangesRequest.FromString,
+          response_serializer=storage__pb2.GetActiveRangesResponse.SerializeToString,
+      ),
+  }
+  generic_handler = grpc.method_handlers_generic_handler(
+      'bookkeeper.proto.storage.MetaRangeService', rpc_method_handlers)
+  server.add_generic_rpc_handlers((generic_handler,))
+
+
+class RootRangeServiceStub(object):
+  """public service for metadata services
+  """
+
+  def __init__(self, channel):
+    """Constructor.
+
+    Args:
+      channel: A grpc.Channel.
+    """
+    self.CreateNamespace = channel.unary_unary(
+        '/bookkeeper.proto.storage.RootRangeService/CreateNamespace',
+        request_serializer=storage__pb2.CreateNamespaceRequest.SerializeToString,
+        response_deserializer=storage__pb2.CreateNamespaceResponse.FromString,
+        )
+    self.DeleteNamespace = channel.unary_unary(
+        '/bookkeeper.proto.storage.RootRangeService/DeleteNamespace',
+        request_serializer=storage__pb2.DeleteNamespaceRequest.SerializeToString,
+        response_deserializer=storage__pb2.DeleteNamespaceResponse.FromString,
+        )
+    self.GetNamespace = channel.unary_unary(
+        '/bookkeeper.proto.storage.RootRangeService/GetNamespace',
+        request_serializer=storage__pb2.GetNamespaceRequest.SerializeToString,
+        response_deserializer=storage__pb2.GetNamespaceResponse.FromString,
+        )
+    self.CreateStream = channel.unary_unary(
+        '/bookkeeper.proto.storage.RootRangeService/CreateStream',
+        request_serializer=storage__pb2.CreateStreamRequest.SerializeToString,
+        response_deserializer=storage__pb2.CreateStreamResponse.FromString,
+        )
+    self.DeleteStream = channel.unary_unary(
+        '/bookkeeper.proto.storage.RootRangeService/DeleteStream',
+        request_serializer=storage__pb2.DeleteStreamRequest.SerializeToString,
+        response_deserializer=storage__pb2.DeleteStreamResponse.FromString,
+        )
+    self.GetStream = channel.unary_unary(
+        '/bookkeeper.proto.storage.RootRangeService/GetStream',
+        request_serializer=storage__pb2.GetStreamRequest.SerializeToString,
+        response_deserializer=storage__pb2.GetStreamResponse.FromString,
+        )
+
+
+class RootRangeServiceServicer(object):
+  """public service for metadata services
+  """
+
+  def CreateNamespace(self, request, context):
+    """
+    Namespace Methods
+
+    """
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+  def DeleteNamespace(self, request, context):
+    # missing associated documentation comment in .proto file
+    pass
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+  def GetNamespace(self, request, context):
+    # missing associated documentation comment in .proto file
+    pass
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+  def CreateStream(self, request, context):
+    """
+    Stream Methods
+
+    """
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+  def DeleteStream(self, request, context):
+    # missing associated documentation comment in .proto file
+    pass
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+  def GetStream(self, request, context):
+    # missing associated documentation comment in .proto file
+    pass
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+
+def add_RootRangeServiceServicer_to_server(servicer, server):
+  rpc_method_handlers = {
+      'CreateNamespace': grpc.unary_unary_rpc_method_handler(
+          servicer.CreateNamespace,
+          request_deserializer=storage__pb2.CreateNamespaceRequest.FromString,
+          response_serializer=storage__pb2.CreateNamespaceResponse.SerializeToString,
+      ),
+      'DeleteNamespace': grpc.unary_unary_rpc_method_handler(
+          servicer.DeleteNamespace,
+          request_deserializer=storage__pb2.DeleteNamespaceRequest.FromString,
+          response_serializer=storage__pb2.DeleteNamespaceResponse.SerializeToString,
+      ),
+      'GetNamespace': grpc.unary_unary_rpc_method_handler(
+          servicer.GetNamespace,
+          request_deserializer=storage__pb2.GetNamespaceRequest.FromString,
+          response_serializer=storage__pb2.GetNamespaceResponse.SerializeToString,
+      ),
+      'CreateStream': grpc.unary_unary_rpc_method_handler(
+          servicer.CreateStream,
+          request_deserializer=storage__pb2.CreateStreamRequest.FromString,
+          response_serializer=storage__pb2.CreateStreamResponse.SerializeToString,
+      ),
+      'DeleteStream': grpc.unary_unary_rpc_method_handler(
+          servicer.DeleteStream,
+          request_deserializer=storage__pb2.DeleteStreamRequest.FromString,
+          response_serializer=storage__pb2.DeleteStreamResponse.SerializeToString,
+      ),
+      'GetStream': grpc.unary_unary_rpc_method_handler(
+          servicer.GetStream,
+          request_deserializer=storage__pb2.GetStreamRequest.FromString,
+          response_serializer=storage__pb2.GetStreamResponse.SerializeToString,
+      ),
+  }
+  generic_handler = grpc.method_handlers_generic_handler(
+      'bookkeeper.proto.storage.RootRangeService', rpc_method_handlers)
+  server.add_generic_rpc_handlers((generic_handler,))
+
+
+class StorageContainerServiceStub(object):
+  """A general range server service
+  """
+
+  def __init__(self, channel):
+    """Constructor.
+
+    Args:
+      channel: A grpc.Channel.
+    """
+    self.GetStorageContainerEndpoint = channel.unary_unary(
+        '/bookkeeper.proto.storage.StorageContainerService/GetStorageContainerEndpoint',
+        request_serializer=storage__pb2.GetStorageContainerEndpointRequest.SerializeToString,
+        response_deserializer=storage__pb2.GetStorageContainerEndpointResponse.FromString,
+        )
+
+
+class StorageContainerServiceServicer(object):
+  """A general range server service
+  """
+
+  def GetStorageContainerEndpoint(self, request, context):
+    """Get the storage container endpoints
+    """
+    context.set_code(grpc.StatusCode.UNIMPLEMENTED)
+    context.set_details('Method not implemented!')
+    raise NotImplementedError('Method not implemented!')
+
+
+def add_StorageContainerServiceServicer_to_server(servicer, server):
+  rpc_method_handlers = {
+      'GetStorageContainerEndpoint': grpc.unary_unary_rpc_method_handler(
+          servicer.GetStorageContainerEndpoint,
+          request_deserializer=storage__pb2.GetStorageContainerEndpointRequest.FromString,
+          response_serializer=storage__pb2.GetStorageContainerEndpointResponse.SerializeToString,
+      ),
+  }
+  generic_handler = grpc.method_handlers_generic_handler(
+      'bookkeeper.proto.storage.StorageContainerService', rpc_method_handlers)
+  server.add_generic_rpc_handlers((generic_handler,))
diff --git a/stream/clients/python/bookkeeper/proto/stream_pb2.py b/stream/clients/python/bookkeeper/proto/stream_pb2.py
new file mode 100644
index 0000000..2374de6
--- /dev/null
+++ b/stream/clients/python/bookkeeper/proto/stream_pb2.py
@@ -0,0 +1,1323 @@
+# Generated by the protocol buffer compiler.  DO NOT EDIT!
+# source: stream.proto
+
+import sys
+_b=sys.version_info[0]<3 and (lambda x:x) or (lambda x:x.encode('latin1'))
+from google.protobuf.internal import enum_type_wrapper
+from google.protobuf import descriptor as _descriptor
+from google.protobuf import message as _message
+from google.protobuf import reflection as _reflection
+from google.protobuf import symbol_database as _symbol_database
+# @@protoc_insertion_point(imports)
+
+_sym_db = _symbol_database.Default()
+
+
+import bookkeeper.proto.common_pb2 as common__pb2
+
+
+DESCRIPTOR = _descriptor.FileDescriptor(
+  name='stream.proto',
+  package='bookkeeper.proto.stream',
+  syntax='proto3',
+  serialized_options=_b('\n\"org.apache.bookkeeper.stream.protoP\001'),
+  serialized_pb=_b('\n\x0cstream.proto\x12\x17\x62ookkeeper.proto.stream\x1a\x0c\x63ommon.proto\"=\n\x07RangeId\x12\r\n\x05sc_id\x18\x01 \x01(\x03\x12\x11\n\tstream_id\x18\x02 \x01(\x03\x12\x10\n\x08range_id\x18\x03 \x01(\x03\"8\n\x08KeyRange\x12\x16\n\x0estart_hash_key\x18\x01 \x01(\x03\x12\x14\n\x0c\x65nd_hash_key\x18\x02 \x01(\x03\"o\n\x0fRangeProperties\x12\x16\n\x0estart_hash_key\x18\x01 \x01(\x03\x12\x14\n\x0c\x65nd_hash_key\x18\x02 \x01(\x03\x12\x10\n\x08range_id\x18\x03 \x01(\x03 [...]
+  ,
+  dependencies=[common__pb2.DESCRIPTOR,])
+
+_RANGESTATE = _descriptor.EnumDescriptor(
+  name='RangeState',
+  full_name='bookkeeper.proto.stream.RangeState',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='RANGE_ACTIVE', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='RANGE_FENCING', index=1, number=1,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='RANGE_FENCED', index=2, number=2,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=2924,
+  serialized_end=2991,
+)
+_sym_db.RegisterEnumDescriptor(_RANGESTATE)
+
+RangeState = enum_type_wrapper.EnumTypeWrapper(_RANGESTATE)
+_RANGEKEYTYPE = _descriptor.EnumDescriptor(
+  name='RangeKeyType',
+  full_name='bookkeeper.proto.stream.RangeKeyType',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='NULL', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='HASH', index=1, number=1,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='RAW', index=2, number=2,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=2993,
+  serialized_end=3036,
+)
+_sym_db.RegisterEnumDescriptor(_RANGEKEYTYPE)
+
+RangeKeyType = enum_type_wrapper.EnumTypeWrapper(_RANGEKEYTYPE)
+_STORAGETYPE = _descriptor.EnumDescriptor(
+  name='StorageType',
+  full_name='bookkeeper.proto.stream.StorageType',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='STREAM', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='TABLE', index=1, number=1,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=3038,
+  serialized_end=3074,
+)
+_sym_db.RegisterEnumDescriptor(_STORAGETYPE)
+
+StorageType = enum_type_wrapper.EnumTypeWrapper(_STORAGETYPE)
+_SPLITPOLICYTYPE = _descriptor.EnumDescriptor(
+  name='SplitPolicyType',
+  full_name='bookkeeper.proto.stream.SplitPolicyType',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='FIXED', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='BANDWIDTH', index=1, number=1,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=3076,
+  serialized_end=3119,
+)
+_sym_db.RegisterEnumDescriptor(_SPLITPOLICYTYPE)
+
+SplitPolicyType = enum_type_wrapper.EnumTypeWrapper(_SPLITPOLICYTYPE)
+RANGE_ACTIVE = 0
+RANGE_FENCING = 1
+RANGE_FENCED = 2
+NULL = 0
+HASH = 1
+RAW = 2
+STREAM = 0
+TABLE = 1
+FIXED = 0
+BANDWIDTH = 1
+
+
+_STREAMMETADATA_LIFECYCLESTATE = _descriptor.EnumDescriptor(
+  name='LifecycleState',
+  full_name='bookkeeper.proto.stream.StreamMetadata.LifecycleState',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='UNINIT', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='CREATING', index=1, number=1,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='CREATED', index=2, number=2,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='FENCING', index=3, number=3,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='FENCED', index=4, number=4,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=2470,
+  serialized_end=2550,
+)
+_sym_db.RegisterEnumDescriptor(_STREAMMETADATA_LIFECYCLESTATE)
+
+_STREAMMETADATA_SERVINGSTATE = _descriptor.EnumDescriptor(
+  name='ServingState',
+  full_name='bookkeeper.proto.stream.StreamMetadata.ServingState',
+  filename=None,
+  file=DESCRIPTOR,
+  values=[
+    _descriptor.EnumValueDescriptor(
+      name='WRITABLE', index=0, number=0,
+      serialized_options=None,
+      type=None),
+    _descriptor.EnumValueDescriptor(
+      name='READONLY', index=1, number=1,
+      serialized_options=None,
+      type=None),
+  ],
+  containing_type=None,
+  serialized_options=None,
+  serialized_start=2552,
+  serialized_end=2594,
+)
+_sym_db.RegisterEnumDescriptor(_STREAMMETADATA_SERVINGSTATE)
+
+
+_RANGEID = _descriptor.Descriptor(
+  name='RangeId',
+  full_name='bookkeeper.proto.stream.RangeId',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='sc_id', full_name='bookkeeper.proto.stream.RangeId.sc_id', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='stream_id', full_name='bookkeeper.proto.stream.RangeId.stream_id', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='range_id', full_name='bookkeeper.proto.stream.RangeId.range_id', index=2,
+      number=3, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=55,
+  serialized_end=116,
+)
+
+
+_KEYRANGE = _descriptor.Descriptor(
+  name='KeyRange',
+  full_name='bookkeeper.proto.stream.KeyRange',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='start_hash_key', full_name='bookkeeper.proto.stream.KeyRange.start_hash_key', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='end_hash_key', full_name='bookkeeper.proto.stream.KeyRange.end_hash_key', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=118,
+  serialized_end=174,
+)
+
+
+_RANGEPROPERTIES = _descriptor.Descriptor(
+  name='RangeProperties',
+  full_name='bookkeeper.proto.stream.RangeProperties',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='start_hash_key', full_name='bookkeeper.proto.stream.RangeProperties.start_hash_key', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='end_hash_key', full_name='bookkeeper.proto.stream.RangeProperties.end_hash_key', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='range_id', full_name='bookkeeper.proto.stream.RangeProperties.range_id', index=2,
+      number=3, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='storage_container_id', full_name='bookkeeper.proto.stream.RangeProperties.storage_container_id', index=3,
+      number=4, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=176,
+  serialized_end=287,
+)
+
+
+_RANGEMETADATA = _descriptor.Descriptor(
+  name='RangeMetadata',
+  full_name='bookkeeper.proto.stream.RangeMetadata',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='props', full_name='bookkeeper.proto.stream.RangeMetadata.props', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='revision', full_name='bookkeeper.proto.stream.RangeMetadata.revision', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='state', full_name='bookkeeper.proto.stream.RangeMetadata.state', index=2,
+      number=3, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='create_time', full_name='bookkeeper.proto.stream.RangeMetadata.create_time', index=3,
+      number=10, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='fence_time', full_name='bookkeeper.proto.stream.RangeMetadata.fence_time', index=4,
+      number=11, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='children', full_name='bookkeeper.proto.stream.RangeMetadata.children', index=5,
+      number=20, type=3, cpp_type=2, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='parents', full_name='bookkeeper.proto.stream.RangeMetadata.parents', index=6,
+      number=21, type=3, cpp_type=2, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=290,
+  serialized_end=508,
+)
+
+
+_PARENTRANGES = _descriptor.Descriptor(
+  name='ParentRanges',
+  full_name='bookkeeper.proto.stream.ParentRanges',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='range_id', full_name='bookkeeper.proto.stream.ParentRanges.range_id', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='parent_range_ids', full_name='bookkeeper.proto.stream.ParentRanges.parent_range_ids', index=1,
+      number=2, type=3, cpp_type=2, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=510,
+  serialized_end=568,
+)
+
+
+_PARENTRANGESLIST = _descriptor.Descriptor(
+  name='ParentRangesList',
+  full_name='bookkeeper.proto.stream.ParentRangesList',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='child_ranges', full_name='bookkeeper.proto.stream.ParentRangesList.child_ranges', index=0,
+      number=1, type=11, cpp_type=10, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=570,
+  serialized_end=649,
+)
+
+
+_FIXEDRANGESPLITPOLICY = _descriptor.Descriptor(
+  name='FixedRangeSplitPolicy',
+  full_name='bookkeeper.proto.stream.FixedRangeSplitPolicy',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='num_ranges', full_name='bookkeeper.proto.stream.FixedRangeSplitPolicy.num_ranges', index=0,
+      number=1, type=5, cpp_type=1, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=651,
+  serialized_end=694,
+)
+
+
+_BANDWIDTHBASEDSPLITPOLICY = _descriptor.Descriptor(
+  name='BandwidthBasedSplitPolicy',
+  full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='max_rate_in_bytes', full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy.max_rate_in_bytes', index=0,
+      number=1, type=5, cpp_type=1, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='min_rate_in_bytes', full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy.min_rate_in_bytes', index=1,
+      number=2, type=5, cpp_type=1, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='max_rate_in_records', full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy.max_rate_in_records', index=2,
+      number=3, type=5, cpp_type=1, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='min_rate_in_records', full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy.min_rate_in_records', index=3,
+      number=4, type=5, cpp_type=1, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='max_split_factor', full_name='bookkeeper.proto.stream.BandwidthBasedSplitPolicy.max_split_factor', index=4,
+      number=5, type=5, cpp_type=1, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=697,
+  serialized_end=862,
+)
+
+
+_SPLITPOLICY = _descriptor.Descriptor(
+  name='SplitPolicy',
+  full_name='bookkeeper.proto.stream.SplitPolicy',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='type', full_name='bookkeeper.proto.stream.SplitPolicy.type', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='fixed_range_policy', full_name='bookkeeper.proto.stream.SplitPolicy.fixed_range_policy', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='bandwidth_policy', full_name='bookkeeper.proto.stream.SplitPolicy.bandwidth_policy', index=2,
+      number=3, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+    _descriptor.OneofDescriptor(
+      name='policy', full_name='bookkeeper.proto.stream.SplitPolicy.policy',
+      index=0, containing_type=None, fields=[]),
+  ],
+  serialized_start=865,
+  serialized_end=1102,
+)
+
+
+_SIZEBASEDSEGMENTROLLINGPOLICY = _descriptor.Descriptor(
+  name='SizeBasedSegmentRollingPolicy',
+  full_name='bookkeeper.proto.stream.SizeBasedSegmentRollingPolicy',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='max_segment_size', full_name='bookkeeper.proto.stream.SizeBasedSegmentRollingPolicy.max_segment_size', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1104,
+  serialized_end=1161,
+)
+
+
+_TIMEBASEDSEGMENTROLLINGPOLICY = _descriptor.Descriptor(
+  name='TimeBasedSegmentRollingPolicy',
+  full_name='bookkeeper.proto.stream.TimeBasedSegmentRollingPolicy',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='interval_seconds', full_name='bookkeeper.proto.stream.TimeBasedSegmentRollingPolicy.interval_seconds', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1163,
+  serialized_end=1220,
+)
+
+
+_SEGMENTROLLINGPOLICY = _descriptor.Descriptor(
+  name='SegmentRollingPolicy',
+  full_name='bookkeeper.proto.stream.SegmentRollingPolicy',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='size_policy', full_name='bookkeeper.proto.stream.SegmentRollingPolicy.size_policy', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='time_policy', full_name='bookkeeper.proto.stream.SegmentRollingPolicy.time_policy', index=1,
+      number=2, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1223,
+  serialized_end=1399,
+)
+
+
+_TIMEBASEDRETENTIONPOLICY = _descriptor.Descriptor(
+  name='TimeBasedRetentionPolicy',
+  full_name='bookkeeper.proto.stream.TimeBasedRetentionPolicy',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='retention_minutes', full_name='bookkeeper.proto.stream.TimeBasedRetentionPolicy.retention_minutes', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1401,
+  serialized_end=1454,
+)
+
+
+_RETENTIONPOLICY = _descriptor.Descriptor(
+  name='RetentionPolicy',
+  full_name='bookkeeper.proto.stream.RetentionPolicy',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='time_policy', full_name='bookkeeper.proto.stream.RetentionPolicy.time_policy', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1456,
+  serialized_end=1545,
+)
+
+
+_STREAMCONFIGURATION = _descriptor.Descriptor(
+  name='StreamConfiguration',
+  full_name='bookkeeper.proto.stream.StreamConfiguration',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='key_type', full_name='bookkeeper.proto.stream.StreamConfiguration.key_type', index=0,
+      number=1, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='min_num_ranges', full_name='bookkeeper.proto.stream.StreamConfiguration.min_num_ranges', index=1,
+      number=2, type=5, cpp_type=1, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='initial_num_ranges', full_name='bookkeeper.proto.stream.StreamConfiguration.initial_num_ranges', index=2,
+      number=3, type=5, cpp_type=1, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='split_policy', full_name='bookkeeper.proto.stream.StreamConfiguration.split_policy', index=3,
+      number=4, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='rolling_policy', full_name='bookkeeper.proto.stream.StreamConfiguration.rolling_policy', index=4,
+      number=5, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='retention_policy', full_name='bookkeeper.proto.stream.StreamConfiguration.retention_policy', index=5,
+      number=6, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='storage_type', full_name='bookkeeper.proto.stream.StreamConfiguration.storage_type', index=6,
+      number=7, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1548,
+  serialized_end=1937,
+)
+
+
+_STREAMPROPERTIES = _descriptor.Descriptor(
+  name='StreamProperties',
+  full_name='bookkeeper.proto.stream.StreamProperties',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='stream_id', full_name='bookkeeper.proto.stream.StreamProperties.stream_id', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='storage_container_id', full_name='bookkeeper.proto.stream.StreamProperties.storage_container_id', index=1,
+      number=2, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='stream_name', full_name='bookkeeper.proto.stream.StreamProperties.stream_name', index=2,
+      number=3, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='stream_conf', full_name='bookkeeper.proto.stream.StreamProperties.stream_conf', index=3,
+      number=4, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=1940,
+  serialized_end=2095,
+)
+
+
+_STREAMNAME = _descriptor.Descriptor(
+  name='StreamName',
+  full_name='bookkeeper.proto.stream.StreamName',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='namespace_name', full_name='bookkeeper.proto.stream.StreamName.namespace_name', index=0,
+      number=1, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='stream_name', full_name='bookkeeper.proto.stream.StreamName.stream_name', index=1,
+      number=2, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=2097,
+  serialized_end=2154,
+)
+
+
+_STREAMMETADATA = _descriptor.Descriptor(
+  name='StreamMetadata',
+  full_name='bookkeeper.proto.stream.StreamMetadata',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='props', full_name='bookkeeper.proto.stream.StreamMetadata.props', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='lifecycle_state', full_name='bookkeeper.proto.stream.StreamMetadata.lifecycle_state', index=1,
+      number=2, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='serving_state', full_name='bookkeeper.proto.stream.StreamMetadata.serving_state', index=2,
+      number=3, type=14, cpp_type=8, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='c_time', full_name='bookkeeper.proto.stream.StreamMetadata.c_time', index=3,
+      number=4, type=4, cpp_type=4, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='m_time', full_name='bookkeeper.proto.stream.StreamMetadata.m_time', index=4,
+      number=5, type=4, cpp_type=4, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='next_range_id', full_name='bookkeeper.proto.stream.StreamMetadata.next_range_id', index=5,
+      number=100, type=4, cpp_type=4, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='current_ranges', full_name='bookkeeper.proto.stream.StreamMetadata.current_ranges', index=6,
+      number=101, type=4, cpp_type=4, label=3,
+      has_default_value=False, default_value=[],
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+    _STREAMMETADATA_LIFECYCLESTATE,
+    _STREAMMETADATA_SERVINGSTATE,
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=2157,
+  serialized_end=2594,
+)
+
+
+_NAMESPACECONFIGURATION = _descriptor.Descriptor(
+  name='NamespaceConfiguration',
+  full_name='bookkeeper.proto.stream.NamespaceConfiguration',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='default_stream_conf', full_name='bookkeeper.proto.stream.NamespaceConfiguration.default_stream_conf', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=2596,
+  serialized_end=2695,
+)
+
+
+_NAMESPACEPROPERTIES = _descriptor.Descriptor(
+  name='NamespaceProperties',
+  full_name='bookkeeper.proto.stream.NamespaceProperties',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='namespace_id', full_name='bookkeeper.proto.stream.NamespaceProperties.namespace_id', index=0,
+      number=1, type=3, cpp_type=2, label=1,
+      has_default_value=False, default_value=0,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='namespace_name', full_name='bookkeeper.proto.stream.NamespaceProperties.namespace_name', index=1,
+      number=2, type=9, cpp_type=9, label=1,
+      has_default_value=False, default_value=_b("").decode('utf-8'),
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+    _descriptor.FieldDescriptor(
+      name='default_stream_conf', full_name='bookkeeper.proto.stream.NamespaceProperties.default_stream_conf', index=2,
+      number=3, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=2698,
+  serialized_end=2840,
+)
+
+
+_NAMESPACEMETADATA = _descriptor.Descriptor(
+  name='NamespaceMetadata',
+  full_name='bookkeeper.proto.stream.NamespaceMetadata',
+  filename=None,
+  file=DESCRIPTOR,
+  containing_type=None,
+  fields=[
+    _descriptor.FieldDescriptor(
+      name='props', full_name='bookkeeper.proto.stream.NamespaceMetadata.props', index=0,
+      number=1, type=11, cpp_type=10, label=1,
+      has_default_value=False, default_value=None,
+      message_type=None, enum_type=None, containing_type=None,
+      is_extension=False, extension_scope=None,
+      serialized_options=None, file=DESCRIPTOR),
+  ],
+  extensions=[
+  ],
+  nested_types=[],
+  enum_types=[
+  ],
+  serialized_options=None,
+  is_extendable=False,
+  syntax='proto3',
+  extension_ranges=[],
+  oneofs=[
+  ],
+  serialized_start=2842,
+  serialized_end=2922,
+)
+
+_RANGEMETADATA.fields_by_name['props'].message_type = _RANGEPROPERTIES
+_RANGEMETADATA.fields_by_name['state'].enum_type = _RANGESTATE
+_PARENTRANGESLIST.fields_by_name['child_ranges'].message_type = _PARENTRANGES
+_SPLITPOLICY.fields_by_name['type'].enum_type = _SPLITPOLICYTYPE
+_SPLITPOLICY.fields_by_name['fixed_range_policy'].message_type = _FIXEDRANGESPLITPOLICY
+_SPLITPOLICY.fields_by_name['bandwidth_policy'].message_type = _BANDWIDTHBASEDSPLITPOLICY
+_SPLITPOLICY.oneofs_by_name['policy'].fields.append(
+  _SPLITPOLICY.fields_by_name['fixed_range_policy'])
+_SPLITPOLICY.fields_by_name['fixed_range_policy'].containing_oneof = _SPLITPOLICY.oneofs_by_name['policy']
+_SPLITPOLICY.oneofs_by_name['policy'].fields.append(
+  _SPLITPOLICY.fields_by_name['bandwidth_policy'])
+_SPLITPOLICY.fields_by_name['bandwidth_policy'].containing_oneof = _SPLITPOLICY.oneofs_by_name['policy']
+_SEGMENTROLLINGPOLICY.fields_by_name['size_policy'].message_type = _SIZEBASEDSEGMENTROLLINGPOLICY
+_SEGMENTROLLINGPOLICY.fields_by_name['time_policy'].message_type = _TIMEBASEDSEGMENTROLLINGPOLICY
+_RETENTIONPOLICY.fields_by_name['time_policy'].message_type = _TIMEBASEDRETENTIONPOLICY
+_STREAMCONFIGURATION.fields_by_name['key_type'].enum_type = _RANGEKEYTYPE
+_STREAMCONFIGURATION.fields_by_name['split_policy'].message_type = _SPLITPOLICY
+_STREAMCONFIGURATION.fields_by_name['rolling_policy'].message_type = _SEGMENTROLLINGPOLICY
+_STREAMCONFIGURATION.fields_by_name['retention_policy'].message_type = _RETENTIONPOLICY
+_STREAMCONFIGURATION.fields_by_name['storage_type'].enum_type = _STORAGETYPE
+_STREAMPROPERTIES.fields_by_name['stream_conf'].message_type = _STREAMCONFIGURATION
+_STREAMMETADATA.fields_by_name['props'].message_type = _STREAMPROPERTIES
+_STREAMMETADATA.fields_by_name['lifecycle_state'].enum_type = _STREAMMETADATA_LIFECYCLESTATE
+_STREAMMETADATA.fields_by_name['serving_state'].enum_type = _STREAMMETADATA_SERVINGSTATE
+_STREAMMETADATA_LIFECYCLESTATE.containing_type = _STREAMMETADATA
+_STREAMMETADATA_SERVINGSTATE.containing_type = _STREAMMETADATA
+_NAMESPACECONFIGURATION.fields_by_name['default_stream_conf'].message_type = _STREAMCONFIGURATION
+_NAMESPACEPROPERTIES.fields_by_name['default_stream_conf'].message_type = _STREAMCONFIGURATION
+_NAMESPACEMETADATA.fields_by_name['props'].message_type = _NAMESPACEPROPERTIES
+DESCRIPTOR.message_types_by_name['RangeId'] = _RANGEID
+DESCRIPTOR.message_types_by_name['KeyRange'] = _KEYRANGE
+DESCRIPTOR.message_types_by_name['RangeProperties'] = _RANGEPROPERTIES
+DESCRIPTOR.message_types_by_name['RangeMetadata'] = _RANGEMETADATA
+DESCRIPTOR.message_types_by_name['ParentRanges'] = _PARENTRANGES
+DESCRIPTOR.message_types_by_name['ParentRangesList'] = _PARENTRANGESLIST
+DESCRIPTOR.message_types_by_name['FixedRangeSplitPolicy'] = _FIXEDRANGESPLITPOLICY
+DESCRIPTOR.message_types_by_name['BandwidthBasedSplitPolicy'] = _BANDWIDTHBASEDSPLITPOLICY
+DESCRIPTOR.message_types_by_name['SplitPolicy'] = _SPLITPOLICY
+DESCRIPTOR.message_types_by_name['SizeBasedSegmentRollingPolicy'] = _SIZEBASEDSEGMENTROLLINGPOLICY
+DESCRIPTOR.message_types_by_name['TimeBasedSegmentRollingPolicy'] = _TIMEBASEDSEGMENTROLLINGPOLICY
+DESCRIPTOR.message_types_by_name['SegmentRollingPolicy'] = _SEGMENTROLLINGPOLICY
+DESCRIPTOR.message_types_by_name['TimeBasedRetentionPolicy'] = _TIMEBASEDRETENTIONPOLICY
+DESCRIPTOR.message_types_by_name['RetentionPolicy'] = _RETENTIONPOLICY
+DESCRIPTOR.message_types_by_name['StreamConfiguration'] = _STREAMCONFIGURATION
+DESCRIPTOR.message_types_by_name['StreamProperties'] = _STREAMPROPERTIES
+DESCRIPTOR.message_types_by_name['StreamName'] = _STREAMNAME
+DESCRIPTOR.message_types_by_name['StreamMetadata'] = _STREAMMETADATA
+DESCRIPTOR.message_types_by_name['NamespaceConfiguration'] = _NAMESPACECONFIGURATION
+DESCRIPTOR.message_types_by_name['NamespaceProperties'] = _NAMESPACEPROPERTIES
+DESCRIPTOR.message_types_by_name['NamespaceMetadata'] = _NAMESPACEMETADATA
+DESCRIPTOR.enum_types_by_name['RangeState'] = _RANGESTATE
+DESCRIPTOR.enum_types_by_name['RangeKeyType'] = _RANGEKEYTYPE
+DESCRIPTOR.enum_types_by_name['StorageType'] = _STORAGETYPE
+DESCRIPTOR.enum_types_by_name['SplitPolicyType'] = _SPLITPOLICYTYPE
+_sym_db.RegisterFileDescriptor(DESCRIPTOR)
+
+RangeId = _reflection.GeneratedProtocolMessageType('RangeId', (_message.Message,), dict(
+  DESCRIPTOR = _RANGEID,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.RangeId)
+  ))
+_sym_db.RegisterMessage(RangeId)
+
+KeyRange = _reflection.GeneratedProtocolMessageType('KeyRange', (_message.Message,), dict(
+  DESCRIPTOR = _KEYRANGE,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.KeyRange)
+  ))
+_sym_db.RegisterMessage(KeyRange)
+
+RangeProperties = _reflection.GeneratedProtocolMessageType('RangeProperties', (_message.Message,), dict(
+  DESCRIPTOR = _RANGEPROPERTIES,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.RangeProperties)
+  ))
+_sym_db.RegisterMessage(RangeProperties)
+
+RangeMetadata = _reflection.GeneratedProtocolMessageType('RangeMetadata', (_message.Message,), dict(
+  DESCRIPTOR = _RANGEMETADATA,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.RangeMetadata)
+  ))
+_sym_db.RegisterMessage(RangeMetadata)
+
+ParentRanges = _reflection.GeneratedProtocolMessageType('ParentRanges', (_message.Message,), dict(
+  DESCRIPTOR = _PARENTRANGES,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.ParentRanges)
+  ))
+_sym_db.RegisterMessage(ParentRanges)
+
+ParentRangesList = _reflection.GeneratedProtocolMessageType('ParentRangesList', (_message.Message,), dict(
+  DESCRIPTOR = _PARENTRANGESLIST,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.ParentRangesList)
+  ))
+_sym_db.RegisterMessage(ParentRangesList)
+
+FixedRangeSplitPolicy = _reflection.GeneratedProtocolMessageType('FixedRangeSplitPolicy', (_message.Message,), dict(
+  DESCRIPTOR = _FIXEDRANGESPLITPOLICY,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.FixedRangeSplitPolicy)
+  ))
+_sym_db.RegisterMessage(FixedRangeSplitPolicy)
+
+BandwidthBasedSplitPolicy = _reflection.GeneratedProtocolMessageType('BandwidthBasedSplitPolicy', (_message.Message,), dict(
+  DESCRIPTOR = _BANDWIDTHBASEDSPLITPOLICY,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.BandwidthBasedSplitPolicy)
+  ))
+_sym_db.RegisterMessage(BandwidthBasedSplitPolicy)
+
+SplitPolicy = _reflection.GeneratedProtocolMessageType('SplitPolicy', (_message.Message,), dict(
+  DESCRIPTOR = _SPLITPOLICY,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.SplitPolicy)
+  ))
+_sym_db.RegisterMessage(SplitPolicy)
+
+SizeBasedSegmentRollingPolicy = _reflection.GeneratedProtocolMessageType('SizeBasedSegmentRollingPolicy', (_message.Message,), dict(
+  DESCRIPTOR = _SIZEBASEDSEGMENTROLLINGPOLICY,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.SizeBasedSegmentRollingPolicy)
+  ))
+_sym_db.RegisterMessage(SizeBasedSegmentRollingPolicy)
+
+TimeBasedSegmentRollingPolicy = _reflection.GeneratedProtocolMessageType('TimeBasedSegmentRollingPolicy', (_message.Message,), dict(
+  DESCRIPTOR = _TIMEBASEDSEGMENTROLLINGPOLICY,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.TimeBasedSegmentRollingPolicy)
+  ))
+_sym_db.RegisterMessage(TimeBasedSegmentRollingPolicy)
+
+SegmentRollingPolicy = _reflection.GeneratedProtocolMessageType('SegmentRollingPolicy', (_message.Message,), dict(
+  DESCRIPTOR = _SEGMENTROLLINGPOLICY,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.SegmentRollingPolicy)
+  ))
+_sym_db.RegisterMessage(SegmentRollingPolicy)
+
+TimeBasedRetentionPolicy = _reflection.GeneratedProtocolMessageType('TimeBasedRetentionPolicy', (_message.Message,), dict(
+  DESCRIPTOR = _TIMEBASEDRETENTIONPOLICY,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.TimeBasedRetentionPolicy)
+  ))
+_sym_db.RegisterMessage(TimeBasedRetentionPolicy)
+
+RetentionPolicy = _reflection.GeneratedProtocolMessageType('RetentionPolicy', (_message.Message,), dict(
+  DESCRIPTOR = _RETENTIONPOLICY,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.RetentionPolicy)
+  ))
+_sym_db.RegisterMessage(RetentionPolicy)
+
+StreamConfiguration = _reflection.GeneratedProtocolMessageType('StreamConfiguration', (_message.Message,), dict(
+  DESCRIPTOR = _STREAMCONFIGURATION,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.StreamConfiguration)
+  ))
+_sym_db.RegisterMessage(StreamConfiguration)
+
+StreamProperties = _reflection.GeneratedProtocolMessageType('StreamProperties', (_message.Message,), dict(
+  DESCRIPTOR = _STREAMPROPERTIES,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.StreamProperties)
+  ))
+_sym_db.RegisterMessage(StreamProperties)
+
+StreamName = _reflection.GeneratedProtocolMessageType('StreamName', (_message.Message,), dict(
+  DESCRIPTOR = _STREAMNAME,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.StreamName)
+  ))
+_sym_db.RegisterMessage(StreamName)
+
+StreamMetadata = _reflection.GeneratedProtocolMessageType('StreamMetadata', (_message.Message,), dict(
+  DESCRIPTOR = _STREAMMETADATA,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.StreamMetadata)
+  ))
+_sym_db.RegisterMessage(StreamMetadata)
+
+NamespaceConfiguration = _reflection.GeneratedProtocolMessageType('NamespaceConfiguration', (_message.Message,), dict(
+  DESCRIPTOR = _NAMESPACECONFIGURATION,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.NamespaceConfiguration)
+  ))
+_sym_db.RegisterMessage(NamespaceConfiguration)
+
+NamespaceProperties = _reflection.GeneratedProtocolMessageType('NamespaceProperties', (_message.Message,), dict(
+  DESCRIPTOR = _NAMESPACEPROPERTIES,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.NamespaceProperties)
+  ))
+_sym_db.RegisterMessage(NamespaceProperties)
+
+NamespaceMetadata = _reflection.GeneratedProtocolMessageType('NamespaceMetadata', (_message.Message,), dict(
+  DESCRIPTOR = _NAMESPACEMETADATA,
+  __module__ = 'stream_pb2'
+  # @@protoc_insertion_point(class_scope:bookkeeper.proto.stream.NamespaceMetadata)
+  ))
+_sym_db.RegisterMessage(NamespaceMetadata)
+
+
+DESCRIPTOR._options = None
+# @@protoc_insertion_point(module_scope)
diff --git a/stream/clients/python/bookkeeper/types.py b/stream/clients/python/bookkeeper/types.py
new file mode 100644
index 0000000..e964329
--- /dev/null
+++ b/stream/clients/python/bookkeeper/types.py
@@ -0,0 +1,71 @@
+# 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 __future__ import absolute_import
+import collections
+import sys
+
+from google.protobuf import descriptor_pb2
+from google.protobuf import duration_pb2
+from google.protobuf import empty_pb2
+from google.protobuf import field_mask_pb2
+from google.protobuf import timestamp_pb2
+from bookkeeper.proto import common_pb2
+from bookkeeper.proto import kv_rpc_pb2
+from bookkeeper.proto import stream_pb2
+from bookkeeper.proto import storage_pb2
+
+from bookkeeper.common.protobuf_helpers import get_messages
+
+
+# Define the default values for storage client settings.
+#
+# This class is used when creating a bookkeeper client, and
+# these settings can be altered to tweak client behavior.
+# The defaults should be fine for most use cases.
+StorageClientSettings = collections.namedtuple(
+    'StorageClientSettings',
+    ['service_uri'],
+)
+StorageClientSettings.__new__.__defaults__ = (
+    "bk://localhost:4181",  # bookkeeper service uri
+)
+
+_shared_modules = [
+    descriptor_pb2,
+    duration_pb2,
+    empty_pb2,
+    field_mask_pb2,
+    timestamp_pb2,
+]
+
+_local_modules = [
+    common_pb2,
+    kv_rpc_pb2,
+    stream_pb2,
+    storage_pb2
+]
+
+names = ['StorageClientSettings']
+
+for module in _shared_modules:
+    for name, message in get_messages(module).items():
+        setattr(sys.modules[__name__], name, message)
+        names.append(name)
+
+for module in _local_modules:
+    for name, message in get_messages(module).items():
+        message.__module__ = 'bookkeeper.types'
+        setattr(sys.modules[__name__], name, message)
+        names.append(name)
+
+__all__ = tuple(sorted(names))
diff --git a/stream/clients/python/examples/admin.py b/stream/clients/python/examples/admin.py
new file mode 100644
index 0000000..d39d50c
--- /dev/null
+++ b/stream/clients/python/examples/admin.py
@@ -0,0 +1,58 @@
+# -*- coding: utf-8 -*-
+#
+# 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
+#
+#     https://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 bookkeeper import admin
+
+ns_name = "test"
+ns_name_2 = "test2"
+stream_name = "test_stream"
+stream_name_2 = "test_stream_2"
+
+client = admin.Client()
+
+# create first namespace
+ns_resp = client.namespaces().create(ns_name)
+print("Created first namespace '%s' : %s" % (ns_name, ns_resp))
+
+# create second namespace
+ns_resp = client.namespaces().create(ns_name_2)
+print("Created second namespace '%s' : %s" % (ns_name_2, ns_resp))
+
+# get first namespace
+ns_props = client.namespaces().get(ns_name)
+print("Get first namespace '%s' : %s" % (ns_name, ns_props))
+ns_props = client.namespaces().get(ns_name_2)
+print("Get second namespace '%s' : %s" % (ns_name_2, ns_props))
+
+# test operations on namespace 'test'
+ns = client.namespace(ns_name)
+stream_resp = ns.create(stream_name)
+print("Create first stream '%s' : %s" % (stream_name, stream_resp))
+stream_resp = ns.create(stream_name_2)
+print("Create second stream '%s' : %s" % (stream_name_2, stream_resp))
+
+stream_props = ns.get(stream_name)
+print("Get first stream '%s' : %s" % (stream_name, stream_props))
+stream_props = ns.get(stream_name_2)
+print("Get second stream '%s' : %s" % (stream_name_2, stream_props))
+
+del_resp = ns.delete(stream_name)
+print("Delete first stream '%s' : %s" % (stream_name, del_resp))
+del_resp = ns.delete(stream_name_2)
+print("Delete second stream '%s' : %s" % (stream_name_2, del_resp))
+
+del_resp = client.namespaces().delete(ns_name)
+print("Delete first namespace '%s' : %s" % (ns_name, del_resp))
+del_resp = client.namespaces().delete(ns_name_2)
+print("Delete second namespace '%s' : %s" % (ns_name_2, del_resp))
diff --git a/stream/clients/python/noxfile.py b/stream/clients/python/noxfile.py
new file mode 100644
index 0000000..0d25fa2
--- /dev/null
+++ b/stream/clients/python/noxfile.py
@@ -0,0 +1,86 @@
+# 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 __future__ import absolute_import
+
+import os
+
+import nox
+
+
+LOCAL_DEPS = (
+)
+
+
+@nox.session
+def default(session):
+    """Default unit test session.
+    This is intended to be run **without** an interpreter set, so
+    that the current ``python`` (on the ``PATH``) or the version of
+    Python corresponding to the ``nox`` binary the ``PATH`` can
+    run the tests.
+    """
+    # Install all test dependencies, then install local packages in-place.
+    session.install('mock', 'pytest', 'pytest-cov')
+    for local_dep in LOCAL_DEPS:
+        session.install('-e', local_dep)
+    session.install('-e', '.')
+
+    # Run py.test against the unit tests.
+    session.run(
+        'py.test',
+        '--quiet',
+        '--cov-append',
+        '--cov-report=',
+        '--cov=bookkeeper',
+        '--cov-config=.coveragerc',
+        os.path.join('tests', 'unit'),
+        *session.posargs
+    )
+
+
+@nox.session
+@nox.parametrize('py', ['3.7'])
+def unit(session, py):
+    """Run the unit test suite."""
+    default(session)
+
+
+@nox.session
+def lint(session):
+    """Run linters.
+    Returns a failure if the linters find linting errors or sufficiently
+    serious code quality issues.
+    """
+    session.install('flake8', *LOCAL_DEPS)
+    session.install('.')
+    session.run('flake8', 'bookkeeper', 'tests')
+
+
+@nox.session
+def lint_setup_py(session):
+    """Verify that setup.py is valid (including RST check)."""
+    session.install('docutils', 'Pygments')
+    session.run(
+        'python', 'setup.py', 'check', '--restructuredtext', '--strict')
+
+
+# TODO: Enable coverage report
+# @nox.session
+def cover(session):
+    """Run the final coverage report.
+    This outputs the coverage report aggregating coverage from the unit
+    test runs (not system test runs), and then erases coverage data.
+    """
+    session.install('coverage', 'pytest-cov')
+    session.run('coverage', 'report', '--show-missing', '--fail-under=100')
+    session.run('coverage', 'erase')
diff --git a/stream/clients/python/pylint.config.py b/stream/clients/python/pylint.config.py
new file mode 100644
index 0000000..a4f86b4
--- /dev/null
+++ b/stream/clients/python/pylint.config.py
@@ -0,0 +1,29 @@
+# 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.
+
+"""This module is used to configure gcp-devrel-py-tools run-pylint."""
+
+import copy
+
+from gcp_devrel.tools import pylint
+
+# Library configuration
+
+# library_additions = {}
+# Ignore generated code
+library_replacements = copy.deepcopy(pylint.DEFAULT_LIBRARY_RC_REPLACEMENTS)
+library_replacements['MASTER']['ignore'].append('proto')
+
+# Test configuration
+
+# test_additions = copy.deepcopy(library_additions)
+# test_replacements = copy.deepcopy(library_replacements)
\ No newline at end of file
diff --git a/stream/clients/python/setup.cfg b/stream/clients/python/setup.cfg
new file mode 100644
index 0000000..d3eef10
--- /dev/null
+++ b/stream/clients/python/setup.cfg
@@ -0,0 +1,14 @@
+# 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.
+
+[bdist_wheel]
+universal = 1
diff --git a/stream/clients/python/setup.py b/stream/clients/python/setup.py
new file mode 100644
index 0000000..5fd3e62
--- /dev/null
+++ b/stream/clients/python/setup.py
@@ -0,0 +1,87 @@
+# 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.
+
+import io
+import os
+
+import setuptools
+
+# Package metadata.
+
+name = 'bookkeeper'
+description = 'Apache BookKeeper client library'
+version = '4.9.0'
+# Should be one of:
+# 'Development Status :: 3 - Alpha'
+# 'Development Status :: 4 - Beta'
+# 'Development Status :: 5 - Production/Stable'
+release_status = 'Development Status :: 3 - Alpha'
+dependencies = [
+    'protobuf>=3.0.0',
+    'requests<3.0.0dev,>=2.18.0',
+    'setuptools>=34.0.0',
+    'six>=1.10.0',
+    'pytz',
+    'futures>=3.2.0;python_version<"3.2"',
+    'grpcio>=1.8.2',
+]
+extras = {
+}
+
+# Setup boilerplate below this line.
+
+package_root = os.path.abspath(os.path.dirname(__file__))
+
+readme_filename = os.path.join(package_root, 'README.md')
+with io.open(readme_filename, encoding='utf-8') as readme_file:
+    readme = readme_file.read()
+
+# Only include packages under the 'bookkeeper' namespace. Do not include tests,
+# benchmarks, etc.
+packages = [
+    package for package in setuptools.find_packages()
+    if package.startswith('bookkeeper')]
+
+# Determine which namespaces are needed.
+namespaces = ['bookkeeper']
+
+setuptools.setup(
+    name=name,
+    version=version,
+    description=description,
+    long_description=readme,
+    author='Apache BookKeeper',
+    author_email='dev@bookkeeper.apache.org',
+    license='Apache 2.0',
+    url='https://github.com/apache/bookkeeper/tree/master/stream/clients/python',
+    classifiers=[
+        release_status,
+        'Intended Audience :: Developers',
+        'License :: Apache Software License',
+        'Programming Language :: Python',
+        'Programming Language :: Python :: 2',
+        'Programming Language :: Python :: 2.7',
+        'Programming Language :: Python :: 3',
+        'Programming Language :: Python :: 3.4',
+        'Programming Language :: Python :: 3.5',
+        'Programming Language :: Python :: 3.6',
+        'Operating System :: OS Independent',
+        'Topic :: Internet',
+    ],
+    platforms='Posix; MacOS X; Windows',
+    packages=packages,
+    namespace_packages=namespaces,
+    install_requires=dependencies,
+    extras_require=extras,
+    include_package_data=True,
+    zip_safe=False,
+)
diff --git a/stream/clients/python/tests/unit/__init__.py b/stream/clients/python/tests/unit/__init__.py
new file mode 100644
index 0000000..4d9a924
--- /dev/null
+++ b/stream/clients/python/tests/unit/__init__.py
@@ -0,0 +1,11 @@
+# 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/stream/clients/python/tests/unit/bookkeeper/common/future/test__helpers.py b/stream/clients/python/tests/unit/bookkeeper/common/future/test__helpers.py
new file mode 100644
index 0000000..804e3be
--- /dev/null
+++ b/stream/clients/python/tests/unit/bookkeeper/common/future/test__helpers.py
@@ -0,0 +1,35 @@
+# 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.
+
+import mock
+
+from bookkeeper.common.future import _helpers
+
+
+@mock.patch('threading.Thread', autospec=True)
+def test_start_deamon_thread(unused_thread):
+    deamon_thread = _helpers.start_daemon_thread(target=mock.sentinel.target)
+    assert deamon_thread.daemon is True
+
+
+def test_safe_invoke_callback():
+    callback = mock.Mock(spec=['__call__'], return_value=42)
+    result = _helpers.safe_invoke_callback(callback, 'a', b='c')
+    assert result == 42
+    callback.assert_called_once_with('a', b='c')
+
+
+def test_safe_invoke_callback_exception():
+    callback = mock.Mock(spec=['__call__'], side_effect=ValueError())
+    result = _helpers.safe_invoke_callback(callback, 'a', b='c')
+    assert result is None
+    callback.assert_called_once_with('a', b='c')
diff --git a/stream/clients/python/tests/unit/bookkeeper/common/future/test_polling.py b/stream/clients/python/tests/unit/bookkeeper/common/future/test_polling.py
new file mode 100644
index 0000000..2ab7568
--- /dev/null
+++ b/stream/clients/python/tests/unit/bookkeeper/common/future/test_polling.py
@@ -0,0 +1,155 @@
+# 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.
+
+import concurrent.futures
+import threading
+import time
+
+import mock
+import pytest
+
+from bookkeeper.common.future import polling
+
+
+class PollingFutureImpl(polling.PollingFuture):
+    def done(self):
+        return False
+
+    def cancel(self):
+        return True
+
+    def cancelled(self):
+        return False
+
+    def running(self):
+        return True
+
+
+def test_polling_future_constructor():
+    future = PollingFutureImpl()
+    assert not future.done()
+    assert not future.cancelled()
+    assert future.running()
+    assert future.cancel()
+
+
+def test_set_result():
+    future = PollingFutureImpl()
+    callback = mock.Mock()
+
+    future.set_result(1)
+
+    assert future.result() == 1
+    future.add_done_callback(callback)
+    callback.assert_called_once_with(future)
+
+
+def test_set_exception():
+    future = PollingFutureImpl()
+    exception = ValueError('meep')
+
+    future.set_exception(exception)
+
+    assert future.exception() == exception
+    with pytest.raises(ValueError):
+        future.result()
+
+    callback = mock.Mock()
+    future.add_done_callback(callback)
+    callback.assert_called_once_with(future)
+
+
+def test_invoke_callback_exception():
+    future = PollingFutureImplWithPoll()
+    future.set_result(42)
+
+    # This should not raise, despite the callback causing an exception.
+    callback = mock.Mock(side_effect=ValueError)
+    future.add_done_callback(callback)
+    callback.assert_called_once_with(future)
+
+
+class PollingFutureImplWithPoll(PollingFutureImpl):
+    def __init__(self):
+        super(PollingFutureImplWithPoll, self).__init__()
+        self.poll_count = 0
+        self.event = threading.Event()
+
+    def done(self):
+        self.poll_count += 1
+        self.event.wait()
+        self.set_result(42)
+        return True
+
+
+def test_result_with_polling():
+    future = PollingFutureImplWithPoll()
+
+    future.event.set()
+    result = future.result()
+
+    assert result == 42
+    assert future.poll_count == 1
+    # Repeated calls should not cause additional polling
+    assert future.result() == result
+    assert future.poll_count == 1
+
+
+class PollingFutureImplTimeout(PollingFutureImplWithPoll):
+    def done(self):
+        time.sleep(1)
+        return False
+
+
+def test_result_timeout():
+    future = PollingFutureImplTimeout()
+    with pytest.raises(concurrent.futures.TimeoutError):
+        future.result(timeout=1)
+
+
+def test_callback_background_thread():
+    future = PollingFutureImplWithPoll()
+    callback = mock.Mock()
+
+    future.add_done_callback(callback)
+
+    assert future._polling_thread is not None
+
+    # Give the thread a second to poll
+    time.sleep(1)
+    assert future.poll_count == 1
+
+    future.event.set()
+    future._polling_thread.join()
+
+    callback.assert_called_once_with(future)
+
+
+def test_double_callback_background_thread():
+    future = PollingFutureImplWithPoll()
+    callback = mock.Mock()
+    callback2 = mock.Mock()
+
+    future.add_done_callback(callback)
+    current_thread = future._polling_thread
+    assert current_thread is not None
+
+    # only one polling thread should be created.
+    future.add_done_callback(callback2)
+    assert future._polling_thread is current_thread
+
+    future.event.set()
+    future._polling_thread.join()
+
+    assert future.poll_count == 1
+    callback.assert_called_once_with(future)
+    callback2.assert_called_once_with(future)
diff --git a/stream/clients/python/tests/unit/bookkeeper/common/test_grpc_helpers.py b/stream/clients/python/tests/unit/bookkeeper/common/test_grpc_helpers.py
new file mode 100644
index 0000000..ad9daf8
--- /dev/null
+++ b/stream/clients/python/tests/unit/bookkeeper/common/test_grpc_helpers.py
@@ -0,0 +1,172 @@
+# 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.
+
+import grpc
+import mock
+import pytest
+
+from bookkeeper.common import exceptions
+from bookkeeper.common import grpc_helpers
+
+
+def test__patch_callable_name():
+    callable = mock.Mock(spec=['__class__'])
+    callable.__class__ = mock.Mock(spec=['__name__'])
+    callable.__class__.__name__ = 'TestCallable'
+
+    grpc_helpers._patch_callable_name(callable)
+
+    assert callable.__name__ == 'TestCallable'
+
+
+def test__patch_callable_name_no_op():
+    callable = mock.Mock(spec=['__name__'])
+    callable.__name__ = 'test_callable'
+
+    grpc_helpers._patch_callable_name(callable)
+
+    assert callable.__name__ == 'test_callable'
+
+
+class RpcErrorImpl(grpc.RpcError, grpc.Call):
+    def __init__(self, code):
+        super(RpcErrorImpl, self).__init__()
+        self._code = code
+
+    def code(self):
+        return self._code
+
+    def details(self):
+        return None
+
+
+def test_wrap_unary_errors():
+    grpc_error = RpcErrorImpl(grpc.StatusCode.INTERNAL)
+    callable_ = mock.Mock(spec=['__call__'], side_effect=grpc_error)
+
+    wrapped_callable = grpc_helpers._wrap_unary_errors(callable_)
+
+    with pytest.raises(exceptions.InternalServerError) as exc_info:
+        wrapped_callable(1, 2, three='four')
+
+    callable_.assert_called_once_with(1, 2, three='four')
+    assert exc_info.value.response == grpc_error
+
+
+def test_wrap_stream_okay():
+    expected_responses = [1, 2, 3]
+    callable_ = mock.Mock(spec=[
+        '__call__'], return_value=iter(expected_responses))
+
+    wrapped_callable = grpc_helpers._wrap_stream_errors(callable_)
+
+    got_iterator = wrapped_callable(1, 2, three='four')
+
+    responses = list(got_iterator)
+
+    callable_.assert_called_once_with(1, 2, three='four')
+    assert responses == expected_responses
+
+
+def test_wrap_stream_iterable_iterface():
+    response_iter = mock.create_autospec(grpc.Call, instance=True)
+    callable_ = mock.Mock(spec=['__call__'], return_value=response_iter)
+
+    wrapped_callable = grpc_helpers._wrap_stream_errors(callable_)
+
+    got_iterator = wrapped_callable()
+
+    callable_.assert_called_once_with()
+
+    # Check each aliased method in the grpc.Call interface
+    got_iterator.add_callback(mock.sentinel.callback)
+    response_iter.add_callback.assert_called_once_with(mock.sentinel.callback)
+
+    got_iterator.cancel()
+    response_iter.cancel.assert_called_once_with()
+
+    got_iterator.code()
+    response_iter.code.assert_called_once_with()
+
+    got_iterator.details()
+    response_iter.details.assert_called_once_with()
+
+    got_iterator.initial_metadata()
+    response_iter.initial_metadata.assert_called_once_with()
+
+    got_iterator.is_active()
+    response_iter.is_active.assert_called_once_with()
+
+    got_iterator.time_remaining()
+    response_iter.time_remaining.assert_called_once_with()
+
+    got_iterator.trailing_metadata()
+    response_iter.trailing_metadata.assert_called_once_with()
+
+
+def test_wrap_stream_errors_invocation():
+    grpc_error = RpcErrorImpl(grpc.StatusCode.INTERNAL)
+    callable_ = mock.Mock(spec=['__call__'], side_effect=grpc_error)
+
+    wrapped_callable = grpc_helpers._wrap_stream_errors(callable_)
+
+    with pytest.raises(exceptions.InternalServerError) as exc_info:
+        wrapped_callable(1, 2, three='four')
+
+    callable_.assert_called_once_with(1, 2, three='four')
+    assert exc_info.value.response == grpc_error
+
+
+class RpcResponseIteratorImpl(object):
+    def __init__(self, exception):
+        self._exception = exception
+
+    def next(self):
+        raise self._exception
+
+    __next__ = next
+
+
+def test_wrap_stream_errors_iterator():
+    grpc_error = RpcErrorImpl(grpc.StatusCode.INTERNAL)
+    response_iter = RpcResponseIteratorImpl(grpc_error)
+    callable_ = mock.Mock(spec=['__call__'], return_value=response_iter)
+
+    wrapped_callable = grpc_helpers._wrap_stream_errors(callable_)
+
+    got_iterator = wrapped_callable(1, 2, three='four')
+
+    with pytest.raises(exceptions.InternalServerError) as exc_info:
+        next(got_iterator)
+
+    callable_.assert_called_once_with(1, 2, three='four')
+    assert exc_info.value.response == grpc_error
+
+
+@mock.patch('bookkeeper.common.grpc_helpers._wrap_unary_errors')
+def test_wrap_errors_non_streaming(wrap_unary_errors):
+    callable_ = mock.create_autospec(grpc.UnaryUnaryMultiCallable)
+
+    result = grpc_helpers.wrap_errors(callable_)
+
+    assert result == wrap_unary_errors.return_value
+    wrap_unary_errors.assert_called_once_with(callable_)
+
+
+@mock.patch('bookkeeper.common.grpc_helpers._wrap_stream_errors')
+def test_wrap_errors_streaming(wrap_stream_errors):
+    callable_ = mock.create_autospec(grpc.UnaryStreamMultiCallable)
+
+    result = grpc_helpers.wrap_errors(callable_)
+
+    assert result == wrap_stream_errors.return_value
+    wrap_stream_errors.assert_called_once_with(callable_)
diff --git a/stream/clients/python/tests/unit/bookkeeper/common/test_protobuf_helpers.py b/stream/clients/python/tests/unit/bookkeeper/common/test_protobuf_helpers.py
new file mode 100644
index 0000000..9851f18
--- /dev/null
+++ b/stream/clients/python/tests/unit/bookkeeper/common/test_protobuf_helpers.py
@@ -0,0 +1,274 @@
+# 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.
+
+import pytest
+
+from bookkeeper.common import protobuf_helpers
+from bookkeeper.proto import common_pb2
+from bookkeeper.proto import kv_pb2
+
+from google.protobuf import any_pb2
+from google.protobuf import message
+from google.protobuf import source_context_pb2
+from google.protobuf import struct_pb2
+from google.protobuf import timestamp_pb2
+from google.protobuf import type_pb2
+
+
+def test_from_any_pb_success():
+    in_message = common_pb2.Endpoint(port=5181)
+    in_message_any = any_pb2.Any()
+    in_message_any.Pack(in_message)
+    out_message =\
+        protobuf_helpers.from_any_pb(common_pb2.Endpoint, in_message_any)
+
+    assert in_message == out_message
+
+
+def test_from_any_pb_failure():
+    in_message = any_pb2.Any()
+    in_message.Pack(common_pb2.Endpoint(port=5181))
+
+    with pytest.raises(TypeError):
+        protobuf_helpers.from_any_pb(kv_pb2.KeyValue, in_message)
+
+
+def test_check_protobuf_helpers_ok():
+    assert protobuf_helpers.check_oneof() is None
+    assert protobuf_helpers.check_oneof(foo='bar') is None
+    assert protobuf_helpers.check_oneof(foo='bar', baz=None) is None
+    assert protobuf_helpers.check_oneof(foo=None, baz='bacon') is None
+    assert (protobuf_helpers.check_oneof(foo='bar', spam=None, eggs=None)
+            is None)
+
+
+def test_check_protobuf_helpers_failures():
+    with pytest.raises(ValueError):
+        protobuf_helpers.check_oneof(foo='bar', spam='eggs')
+    with pytest.raises(ValueError):
+        protobuf_helpers.check_oneof(foo='bar', baz='bacon', spam='eggs')
+    with pytest.raises(ValueError):
+        protobuf_helpers.check_oneof(foo='bar', spam=0, eggs=None)
+
+
+def test_get_messages():
+    kv = protobuf_helpers.get_messages(kv_pb2)
+
+    # Ensure that Date was exported properly.
+    assert kv['KeyValue'] is kv_pb2.KeyValue
+
+    # Ensure that no non-Message objects were exported.
+    for value in kv.values():
+        assert issubclass(value, message.Message)
+
+
+def test_get_dict_absent():
+    with pytest.raises(KeyError):
+        assert protobuf_helpers.get({}, 'foo')
+
+
+def test_get_dict_present():
+    assert protobuf_helpers.get({'foo': 'bar'}, 'foo') == 'bar'
+
+
+def test_get_dict_default():
+    assert protobuf_helpers.get({}, 'foo', default='bar') == 'bar'
+
+
+def test_get_dict_nested():
+    assert protobuf_helpers.get({'foo': {'bar': 'baz'}}, 'foo.bar') == 'baz'
+
+
+def test_get_dict_nested_default():
+    assert protobuf_helpers.get({}, 'foo.baz', default='bacon') == 'bacon'
+    assert (
+        protobuf_helpers.get({'foo': {}}, 'foo.baz', default='bacon') ==
+        'bacon')
+
+
+def test_get_msg_sentinel():
+    msg = timestamp_pb2.Timestamp()
+    with pytest.raises(KeyError):
+        assert protobuf_helpers.get(msg, 'foo')
+
+
+def test_get_msg_present():
+    msg = timestamp_pb2.Timestamp(seconds=42)
+    assert protobuf_helpers.get(msg, 'seconds') == 42
+
+
+def test_get_msg_default():
+    msg = timestamp_pb2.Timestamp()
+    assert protobuf_helpers.get(msg, 'foo', default='bar') == 'bar'
+
+
+def test_invalid_object():
+    with pytest.raises(TypeError):
+        protobuf_helpers.get(object(), 'foo', 'bar')
+
+
+def test_set_dict():
+    mapping = {}
+    protobuf_helpers.set(mapping, 'foo', 'bar')
+    assert mapping == {'foo': 'bar'}
+
+
+def test_set_msg():
+    msg = timestamp_pb2.Timestamp()
+    protobuf_helpers.set(msg, 'seconds', 42)
+    assert msg.seconds == 42
+
+
+def test_set_dict_nested():
+    mapping = {}
+    protobuf_helpers.set(mapping, 'foo.bar', 'baz')
+    assert mapping == {'foo': {'bar': 'baz'}}
+
+
+def test_set_invalid_object():
+    with pytest.raises(TypeError):
+        protobuf_helpers.set(object(), 'foo', 'bar')
+
+
+def test_setdefault_dict_unset():
+    mapping = {}
+    protobuf_helpers.setdefault(mapping, 'foo', 'bar')
+    assert mapping == {'foo': 'bar'}
+
+
+def test_setdefault_dict_falsy():
+    mapping = {'foo': None}
+    protobuf_helpers.setdefault(mapping, 'foo', 'bar')
+    assert mapping == {'foo': 'bar'}
+
+
+def test_setdefault_dict_truthy():
+    mapping = {'foo': 'bar'}
+    protobuf_helpers.setdefault(mapping, 'foo', 'baz')
+    assert mapping == {'foo': 'bar'}
+
+
+def test_field_mask_singular_field_diffs():
+    original = type_pb2.Type(name='name')
+    modified = type_pb2.Type()
+    assert (protobuf_helpers.field_mask(original, modified).paths ==
+            ['name'])
+
+    original = type_pb2.Type(name='name')
+    modified = type_pb2.Type()
+    assert (protobuf_helpers.field_mask(original, modified).paths ==
+            ['name'])
+
+    original = None
+    modified = type_pb2.Type(name='name')
+    assert (protobuf_helpers.field_mask(original, modified).paths ==
+            ['name'])
+
+    original = type_pb2.Type(name='name')
+    modified = None
+    assert (protobuf_helpers.field_mask(original, modified).paths ==
+            ['name'])
+
+
+def test_field_mask_message_diffs():
+    original = type_pb2.Type()
+    modified = type_pb2.Type(source_context=source_context_pb2.SourceContext(
+                            file_name='name'))
+    assert (protobuf_helpers.field_mask(original, modified).paths ==
+            ['source_context.file_name'])
+
+    original = type_pb2.Type(source_context=source_context_pb2.SourceContext(
+                             file_name='name'))
+    modified = type_pb2.Type()
+    assert (protobuf_helpers.field_mask(original, modified).paths ==
+            ['source_context'])
+
+    original = type_pb2.Type(source_context=source_context_pb2.SourceContext(
+                             file_name='name'))
+    modified = type_pb2.Type(source_context=source_context_pb2.SourceContext(
+                             file_name='other_name'))
+    assert (protobuf_helpers.field_mask(original, modified).paths ==
+            ['source_context.file_name'])
+
+    original = None
+    modified = type_pb2.Type(source_context=source_context_pb2.SourceContext(
+                             file_name='name'))
+    assert (protobuf_helpers.field_mask(original, modified).paths ==
+            ['source_context.file_name'])
+
+    original = type_pb2.Type(source_context=source_context_pb2.SourceContext(
+                             file_name='name'))
+    modified = None
+    assert (protobuf_helpers.field_mask(original, modified).paths ==
+            ['source_context'])
+
+
+def test_field_mask_repeated_diffs():
+    original = struct_pb2.ListValue()
+    modified = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=1.0),
+                                    struct_pb2.Value(number_value=2.0)])
+    assert protobuf_helpers.field_mask(original, modified).paths == ['values']
+
+    original = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=1.0),
+                                    struct_pb2.Value(number_value=2.0)])
+    modified = struct_pb2.ListValue()
+    assert protobuf_helpers.field_mask(original, modified).paths == ['values']
+
+    original = None
+    modified = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=1.0),
+                                    struct_pb2.Value(number_value=2.0)])
+    assert protobuf_helpers.field_mask(original, modified).paths == ['values']
+
+    original = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=1.0),
+                                    struct_pb2.Value(number_value=2.0)])
+    modified = None
+    assert protobuf_helpers.field_mask(original, modified).paths == ['values']
+
+    original = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=1.0),
+                                    struct_pb2.Value(number_value=2.0)])
+    modified = struct_pb2.ListValue(values=[struct_pb2.Value(number_value=2.0),
+                                    struct_pb2.Value(number_value=1.0)])
+    assert protobuf_helpers.field_mask(original, modified).paths == ['values']
+
+
+def test_field_mask_map_diffs():
+    original = struct_pb2.Struct()
+    modified = struct_pb2.Struct(
+            fields={'foo': struct_pb2.Value(number_value=1.0)})
+    assert protobuf_helpers.field_mask(original, modified).paths == ['fields']
+
+    original = struct_pb2.Struct(
+            fields={'foo': struct_pb2.Value(number_value=1.0)})
+    modified = struct_pb2.Struct()
+    assert protobuf_helpers.field_mask(original, modified).paths == ['fields']
+
+    original = None
+    modified = struct_pb2.Struct(
+            fields={'foo': struct_pb2.Value(number_value=1.0)})
+    assert protobuf_helpers.field_mask(original, modified).paths == ['fields']
+
+    original = struct_pb2.Struct(
+            fields={'foo': struct_pb2.Value(number_value=1.0)})
+    modified = None
+    assert protobuf_helpers.field_mask(original, modified).paths == ['fields']
+
+    original = struct_pb2.Struct(
+            fields={'foo': struct_pb2.Value(number_value=1.0)})
+    modified = struct_pb2.Struct(
+            fields={'foo': struct_pb2.Value(number_value=2.0)})
+    assert protobuf_helpers.field_mask(original, modified).paths == ['fields']
+
+    original = struct_pb2.Struct(
+            fields={'foo': struct_pb2.Value(number_value=1.0)})
+    modified = struct_pb2.Struct(
+            fields={'bar': struct_pb2.Value(number_value=1.0)})
+    assert protobuf_helpers.field_mask(original, modified).paths == ['fields']
diff --git a/stream/clients/python/tests/unit/bookkeeper/test_futures.py b/stream/clients/python/tests/unit/bookkeeper/test_futures.py
new file mode 100644
index 0000000..4f95c4c
--- /dev/null
+++ b/stream/clients/python/tests/unit/bookkeeper/test_futures.py
@@ -0,0 +1,145 @@
+# 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.
+
+import threading
+
+import mock
+import pytest
+
+from bookkeeper.kv import futures, exceptions
+
+
+def _future(*args, **kwargs):
+    return futures.Future(*args, **kwargs)
+
+
+def test_constructor_defaults():
+    with mock.patch.object(threading, 'Event', autospec=True) as Event:
+        future = _future()
+
+    assert future._result == futures.Future._SENTINEL
+    assert future._exception == futures.Future._SENTINEL
+    assert future._callbacks == []
+    assert future._completed is Event.return_value
+
+    Event.assert_called_once_with()
+
+
+def test_constructor_explicit_completed():
+    completed = mock.sentinel.completed
+    future = _future(completed=completed)
+
+    assert future._result == futures.Future._SENTINEL
+    assert future._exception == futures.Future._SENTINEL
+    assert future._callbacks == []
+    assert future._completed is completed
+
+
+def test_cancel():
+    assert _future().cancel() is False
+
+
+def test_cancelled():
+    assert _future().cancelled() is False
+
+
+def test_running():
+    future = _future()
+    assert future.running() is True
+    future.set_result('foobar')
+    assert future.running() is False
+
+
+def test_done():
+    future = _future()
+    assert future.done() is False
+    future.set_result('12345')
+    assert future.done() is True
+
+
+def test_exception_no_error():
+    future = _future()
+    future.set_result('12345')
+    assert future.exception() is None
+
+
+def test_exception_with_error():
+    future = _future()
+    error = RuntimeError('Something really bad happened.')
+    future.set_exception(error)
+
+    # Make sure that the exception that is returned is the batch's error.
+    # Also check the type to ensure the batch's error did not somehow
+    # change internally.
+    assert future.exception() is error
+    assert isinstance(future.exception(), RuntimeError)
+    with pytest.raises(RuntimeError):
+        future.result()
+
+
+def test_exception_timeout():
+    future = _future()
+    with pytest.raises(exceptions.TimeoutError):
+        future.exception(timeout=0.01)
+
+
+def test_result_no_error():
+    future = _future()
+    future.set_result('42')
+    assert future.result() == '42'
+
+
+def test_result_with_error():
+    future = _future()
+    future.set_exception(RuntimeError('Something really bad happened.'))
+    with pytest.raises(RuntimeError):
+        future.result()
+
+
+def test_add_done_callback_pending_batch():
+    future = _future()
+    callback = mock.Mock()
+    future.add_done_callback(callback)
+    assert len(future._callbacks) == 1
+    assert callback in future._callbacks
+    assert callback.call_count == 0
+
+
+def test_add_done_callback_completed_batch():
+    future = _future()
+    future.set_result('12345')
+    callback = mock.Mock(spec=())
+    future.add_done_callback(callback)
+    callback.assert_called_once_with(future)
+
+
+def test_trigger():
+    future = _future()
+    callback = mock.Mock(spec=())
+    future.add_done_callback(callback)
+    assert callback.call_count == 0
+    future.set_result('12345')
+    callback.assert_called_once_with(future)
+
+
+def test_set_result_once_only():
+    future = _future()
+    future.set_result('12345')
+    with pytest.raises(RuntimeError):
+        future.set_result('67890')
+
+
+def test_set_exception_once_only():
+    future = _future()
+    future.set_exception(ValueError('wah wah'))
+    with pytest.raises(RuntimeError):
+        future.set_exception(TypeError('other wah wah'))