Documentation for auto paging
This commit is contained in:
@@ -2,6 +2,7 @@
|
||||
This module houses the main classes you will interact with,
|
||||
:class:`.Cluster` and :class:`.Session`.
|
||||
"""
|
||||
from __future__ import absolute_import
|
||||
|
||||
from concurrent.futures import ThreadPoolExecutor
|
||||
import logging
|
||||
@@ -147,8 +148,15 @@ class Cluster(object):
|
||||
server will be automatically used.
|
||||
"""
|
||||
|
||||
# TODO: docs
|
||||
protocol_version = 2
|
||||
"""
|
||||
The version of the native protocol to use. The protocol version 2
|
||||
add support for lightweight transactions, batch operations, and
|
||||
automatic query paging, but is only supported by Cassandra 2.0+. When
|
||||
working with Cassandra 1.2, this must be set to 1. You can also set
|
||||
this to 1 when working with Cassandra 2.0+, but features that require
|
||||
the version 2 protocol will not be enabled.
|
||||
"""
|
||||
|
||||
compression = True
|
||||
"""
|
||||
@@ -941,10 +949,10 @@ class Session(object):
|
||||
default_fetch_size = 5000
|
||||
"""
|
||||
By default, this many rows will be fetched at a time. This can be
|
||||
specified per-query through :attr:`~Statement.fetch_size`.
|
||||
specified per-query through :attr:`.Statement.fetch_size`.
|
||||
|
||||
This only takes effect when protocol version 2 or higher is used.
|
||||
See :attr:`~Cluster.protocol_version` for details.
|
||||
See :attr:`.Cluster.protocol_version` for details.
|
||||
"""
|
||||
|
||||
_lock = None
|
||||
@@ -1970,9 +1978,21 @@ class ResponseFuture(object):
|
||||
|
||||
@property
|
||||
def has_more_pages(self):
|
||||
"""
|
||||
Returns :const:`True` if there are more pages left in the
|
||||
query results, :const:`False` otherwise. This should only
|
||||
be checked after the first page has been returned.
|
||||
"""
|
||||
return self._paging_state is not None
|
||||
|
||||
def start_fetching_next_page(self):
|
||||
"""
|
||||
If there are more pages left in the query result, this asynchronously
|
||||
starts fetching the next page. If there are no pages left, :exc:`.QueryExhausted`
|
||||
is raised. Also see :attr:`.has_more_pages`.
|
||||
|
||||
This should only be called after the first page has been returned.
|
||||
"""
|
||||
if not self._paging_state:
|
||||
raise QueryExhausted()
|
||||
|
||||
@@ -2388,10 +2408,33 @@ class ResponseFuture(object):
|
||||
|
||||
|
||||
class QueryExhausted(Exception):
|
||||
"""
|
||||
Raised when :meth:`.ResultSet.start_fetching_next_page()` is called and
|
||||
there are no more pages. You can check :attr:`.ResultSet.has_more_pages`
|
||||
before calling to avoid this.
|
||||
"""
|
||||
pass
|
||||
|
||||
|
||||
class PagedResult(object):
|
||||
"""
|
||||
An iterator over the rows from a paged query result. Whenever the number
|
||||
of result rows for a query exceed the :attr:`~.query.Statement.fetch_size`
|
||||
(or :attr:`~.Session.default_fetch_size`, if not set) an instance of this
|
||||
class will be returned.
|
||||
|
||||
You can treat this as a normal iterator over rows::
|
||||
|
||||
>>> from cassandra.query import SimpleStatement
|
||||
>>> statement = SimpleStatement("SELECT * FROM users", fetch_size=10)
|
||||
>>> for user_row in session.execute(statement):
|
||||
... process_user(user_row)
|
||||
|
||||
Whenever there are no more rows in the current page, the next page will
|
||||
be fetched transparently. However, note that it _is_ possible for
|
||||
an :class:`Exception` to be raised while fetching the next page, just
|
||||
like you might see on a normal call to ``session.execute()``.
|
||||
"""
|
||||
|
||||
def __init__(self, response_future, initial_response):
|
||||
self.response_future = response_future
|
||||
|
||||
@@ -72,6 +72,13 @@ class Statement(object):
|
||||
"""
|
||||
|
||||
fetch_size = None
|
||||
"""
|
||||
How many rows will be fetched at a time. This overrides the default
|
||||
of :attr:`.Session.default_fetch_size`
|
||||
|
||||
This only takes effect when protocol version 2 or higher is used.
|
||||
See :attr:`.Cluster.protocol_version` for details.
|
||||
"""
|
||||
|
||||
_serial_consistency_level = None
|
||||
_routing_key = None
|
||||
|
||||
@@ -7,6 +7,8 @@
|
||||
|
||||
.. autoattribute:: cql_version
|
||||
|
||||
.. autoattribute:: protocol_version
|
||||
|
||||
.. autoattribute:: port
|
||||
|
||||
.. autoattribute:: compression
|
||||
@@ -59,6 +61,8 @@
|
||||
|
||||
.. autoattribute:: row_factory
|
||||
|
||||
.. autoattribute:: default_fetch_size
|
||||
|
||||
.. automethod:: execute(statement[, parameters][, timeout][, trace])
|
||||
|
||||
.. automethod:: execute_async(statement[, parameters][, trace])
|
||||
@@ -77,11 +81,20 @@
|
||||
|
||||
.. automethod:: get_query_trace()
|
||||
|
||||
.. autoattribute:: has_more_pages
|
||||
|
||||
.. automethod:: start_fetching_next_page()
|
||||
|
||||
.. automethod:: add_callback(fn, *args, **kwargs)
|
||||
|
||||
.. automethod:: add_errback(fn, *args, **kwargs)
|
||||
|
||||
.. automethod:: add_callbacks(callback, errback, callback_args=(), callback_kwargs=None, errback_args=(), errback_args=None)
|
||||
|
||||
.. autoclass:: PagedResult ()
|
||||
:members:
|
||||
|
||||
.. autoexception:: QueryExhausted ()
|
||||
|
||||
.. autoexception:: NoHostAvailable ()
|
||||
:members:
|
||||
|
||||
@@ -8,6 +8,7 @@ Python Cassandra Driver
|
||||
installation
|
||||
getting_started
|
||||
performance
|
||||
query_paging
|
||||
|
||||
Indices and Tables
|
||||
==================
|
||||
|
||||
74
docs/query_paging.rst
Normal file
74
docs/query_paging.rst
Normal file
@@ -0,0 +1,74 @@
|
||||
Paging Large Queries
|
||||
====================
|
||||
Cassandra 2.0+ offers support for automatic query paging. Starting with
|
||||
version 2.0 of the driver, if :attr:`~.Cluster.protocol_version` is set to
|
||||
:const:`2` (it is by default), queries returning large result sets will be
|
||||
automatically paged.
|
||||
|
||||
Controlling the Page Size
|
||||
-------------------------
|
||||
By default, :attr:`.Session.default_fetch_size` controls how many rows will
|
||||
be fetched per page. This can be overridden per-query by setting
|
||||
:attr:`~.fetch_size` on a :class:`~.Statement`. By default, each page
|
||||
will contain at most 5000 rows.
|
||||
|
||||
Handling Paged Results
|
||||
----------------------
|
||||
Whenever the number of result rows for are query exceed the page size, an
|
||||
instance of :class:`~.PagedResult` will be returned instead of a normal
|
||||
list. This class implements the iterator interface, so you can treat
|
||||
it like a normal iterator over rows::
|
||||
|
||||
from cassandra.query import SimpleStatement
|
||||
query = "SELECT * FROM users" # users contains 100 rows
|
||||
statement = SimpleStatement(query, fetch_size=10)
|
||||
for user_row in session.execute(statement):
|
||||
process_user(user_row)
|
||||
|
||||
Whenever there are no more rows in the current page, the next page will
|
||||
be fetched transparently. However, note that it *is* possible for
|
||||
an :class:`Exception` to be raised while fetching the next page, just
|
||||
like you might see on a normal call to ``session.execute()``.
|
||||
|
||||
If you use :meth:`.Session.execute_async()` along with,
|
||||
:meth:`.ResponseFuture.result()`, the first page will be fetched before
|
||||
:meth:`~.ResponseFuture.result()` returns, but latter pages will be
|
||||
transparently fetched synchronously while iterating the result.
|
||||
|
||||
Handling Paged Results with Callbacks
|
||||
-------------------------------------
|
||||
If callbacks are attached to a query that returns a paged result,
|
||||
the callback will be called once per page with a normal list of rows.
|
||||
|
||||
Use :attr:`.ResponseFuture.has_more_pages` and
|
||||
:meth:`.ResponseFuture.start_fetching_next_page()` to continue fetching
|
||||
pages. For example::
|
||||
|
||||
class PagedResultHandler(object):
|
||||
|
||||
def __init__(self, future):
|
||||
self.error = None
|
||||
self.finished_event = Event()
|
||||
self.future = future
|
||||
self.future.add_callbacks(
|
||||
callback=self.handle_page,
|
||||
errback=self.handle_err)
|
||||
|
||||
def handle_page(self, rows):
|
||||
for row in rows:
|
||||
process_row(row)
|
||||
|
||||
if self.future.has_more_pages:
|
||||
self.future.start_fetching_next_page()
|
||||
else:
|
||||
self.finished_event.set()
|
||||
|
||||
def handle_error(self, exc):
|
||||
self.error = exc
|
||||
self.finished_event.set()
|
||||
|
||||
future = session.execute_async("SELECT * FROM users")
|
||||
handler = PagedResultHandler(future)
|
||||
handler.finished_event.wait()
|
||||
if handler.error:
|
||||
raise handler.error
|
||||
Reference in New Issue
Block a user