|
37 | 37 | import time
|
38 | 38 | from threading import Lock, RLock, Thread, Event
|
39 | 39 | import uuid
|
| 40 | +import os |
| 41 | +import urllib.request, urllib.error |
| 42 | +import json |
40 | 43 |
|
41 | 44 | import weakref
|
42 | 45 | from weakref import WeakValueDictionary
|
|
81 | 84 | from cassandra.marshal import int64_pack
|
82 | 85 | from cassandra.timestamps import MonotonicTimestampGenerator
|
83 | 86 | from cassandra.util import _resolve_contact_points_to_string_map, Version
|
| 87 | +from cassandra.concurrent import execute_concurrent, execute_concurrent_with_args |
84 | 88 |
|
85 | 89 | from cassandra.datastax.insights.reporter import MonitorReporter
|
86 | 90 | from cassandra.datastax.insights.util import version_supports_insights
|
@@ -2725,6 +2729,79 @@ def execute_async(self, query, parameters=None, trace=False, custom_payload=None
|
2725 | 2729 | future.send_request()
|
2726 | 2730 | return future
|
2727 | 2731 |
|
| 2732 | + def execute_concurrent(self, statements_and_parameters, concurrency=100, raise_on_first_error=True, results_generator=False, execution_profile=EXEC_PROFILE_DEFAULT): |
| 2733 | + """ |
| 2734 | + Executes a sequence of (statement, parameters) tuples concurrently. Each |
| 2735 | + ``parameters`` item must be a sequence or :const:`None`. |
| 2736 | +
|
| 2737 | + The `concurrency` parameter controls how many statements will be executed |
| 2738 | + concurrently. When :attr:`.Cluster.protocol_version` is set to 1 or 2, |
| 2739 | + it is recommended that this be kept below 100 times the number of |
| 2740 | + core connections per host times the number of connected hosts (see |
| 2741 | + :meth:`.Cluster.set_core_connections_per_host`). If that amount is exceeded, |
| 2742 | + the event loop thread may attempt to block on new connection creation, |
| 2743 | + substantially impacting throughput. If :attr:`~.Cluster.protocol_version` |
| 2744 | + is 3 or higher, you can safely experiment with higher levels of concurrency. |
| 2745 | +
|
| 2746 | + If `raise_on_first_error` is left as :const:`True`, execution will stop |
| 2747 | + after the first failed statement and the corresponding exception will be |
| 2748 | + raised. |
| 2749 | +
|
| 2750 | + `results_generator` controls how the results are returned. |
| 2751 | +
|
| 2752 | + * If :const:`False`, the results are returned only after all requests have completed. |
| 2753 | + * If :const:`True`, a generator expression is returned. Using a generator results in a constrained |
| 2754 | + memory footprint when the results set will be large -- results are yielded |
| 2755 | + as they return instead of materializing the entire list at once. The trade for lower memory |
| 2756 | + footprint is marginal CPU overhead (more thread coordination and sorting out-of-order results |
| 2757 | + on-the-fly). |
| 2758 | +
|
| 2759 | + `execution_profile` argument is the execution profile to use for this |
| 2760 | + request, it is passed directly to :meth:`Session.execute_async`. |
| 2761 | +
|
| 2762 | + A sequence of ``ExecutionResult(success, result_or_exc)`` namedtuples is returned |
| 2763 | + in the same order that the statements were passed in. If ``success`` is :const:`False`, |
| 2764 | + there was an error executing the statement, and ``result_or_exc`` |
| 2765 | + will be an :class:`Exception`. If ``success`` is :const:`True`, ``result_or_exc`` |
| 2766 | + will be the query result. |
| 2767 | +
|
| 2768 | + Example usage:: |
| 2769 | +
|
| 2770 | + select_statement = session.prepare("SELECT * FROM users WHERE id=?") |
| 2771 | +
|
| 2772 | + statements_and_params = [] |
| 2773 | + for user_id in user_ids: |
| 2774 | + params = (user_id, ) |
| 2775 | + statements_and_params.append((select_statement, params)) |
| 2776 | +
|
| 2777 | + results = session.execute_concurrent(statements_and_params, raise_on_first_error=False) |
| 2778 | +
|
| 2779 | + for (success, result) in results: |
| 2780 | + if not success: |
| 2781 | + handle_error(result) # result will be an Exception |
| 2782 | + else: |
| 2783 | + process_user(result[0]) # result will be a list of rows |
| 2784 | +
|
| 2785 | + Note: in the case that `generators` are used, it is important to ensure the consumers do not |
| 2786 | + block or attempt further synchronous requests, because no further IO will be processed until |
| 2787 | + the consumer returns. This may also produce a deadlock in the IO event thread. |
| 2788 | + """ |
| 2789 | + return execute_concurrent(self, statements_and_parameters, concurrency, raise_on_first_error, results_generator, execution_profile) |
| 2790 | + |
| 2791 | + def execute_concurrent_with_args(self, statement, parameters, *args, **kwargs): |
| 2792 | + """ |
| 2793 | + Like :meth:`~cassandra.concurrent.execute_concurrent()`, but takes a single |
| 2794 | + statement and a sequence of parameters. Each item in ``parameters`` |
| 2795 | + should be a sequence or :const:`None`. |
| 2796 | +
|
| 2797 | + Example usage:: |
| 2798 | +
|
| 2799 | + statement = session.prepare("INSERT INTO mytable (a, b) VALUES (1, ?)") |
| 2800 | + parameters = [(x,) for x in range(1000)] |
| 2801 | + session.execute_concurrent_with_args(statement, parameters, concurrency=50) |
| 2802 | + """ |
| 2803 | + return execute_concurrent_with_args(self, statement, parameters, *args, **kwargs) |
| 2804 | + |
2728 | 2805 | def execute_graph(self, query, parameters=None, trace=False, execution_profile=EXEC_PROFILE_GRAPH_DEFAULT, execute_as=None):
|
2729 | 2806 | """
|
2730 | 2807 | Executes a Gremlin query string or GraphStatement synchronously,
|
|
0 commit comments