@@ -2717,7 +2717,7 @@ def execute_async(self, query, parameters=None, trace=False, custom_payload=None
2717
2717
if execute_as :
2718
2718
custom_payload [_proxy_execute_key ] = execute_as .encode ()
2719
2719
2720
- future = self ._create_response_future (
2720
+ future = self ._create_execute_response_future (
2721
2721
query , parameters , trace , custom_payload , timeout ,
2722
2722
execution_profile , paging_state , host )
2723
2723
future ._protocol_handler = self .client_protocol_handler
@@ -2782,8 +2782,8 @@ def execute_graph_async(self, query, parameters=None, trace=False, execution_pro
2782
2782
custom_payload [_proxy_execute_key ] = execute_as .encode ()
2783
2783
custom_payload [_request_timeout_key ] = int64_pack (int (execution_profile .request_timeout * 1000 ))
2784
2784
2785
- future = self ._create_response_future (query , parameters = None , trace = trace , custom_payload = custom_payload ,
2786
- timeout = _NOT_SET , execution_profile = execution_profile )
2785
+ future = self ._create_execute_response_future (query , parameters = None , trace = trace , custom_payload = custom_payload ,
2786
+ timeout = _NOT_SET , execution_profile = execution_profile )
2787
2787
2788
2788
future .message .query_params = graph_parameters
2789
2789
future ._protocol_handler = self .client_protocol_handler
@@ -2885,9 +2885,9 @@ def _transform_params(self, parameters, graph_options):
2885
2885
2886
2886
def _target_analytics_master (self , future ):
2887
2887
future ._start_timer ()
2888
- master_query_future = self ._create_response_future ("CALL DseClientTool.getAnalyticsGraphServer()" ,
2889
- parameters = None , trace = False ,
2890
- custom_payload = None , timeout = future .timeout )
2888
+ master_query_future = self ._create_execute_response_future ("CALL DseClientTool.getAnalyticsGraphServer()" ,
2889
+ parameters = None , trace = False ,
2890
+ custom_payload = None , timeout = future .timeout )
2891
2891
master_query_future .row_factory = tuple_factory
2892
2892
master_query_future .send_request ()
2893
2893
@@ -2910,9 +2910,37 @@ def _on_analytics_master_result(self, response, master_future, query_future):
2910
2910
2911
2911
self .submit (query_future .send_request )
2912
2912
2913
- def _create_response_future (self , query , parameters , trace , custom_payload ,
2914
- timeout , execution_profile = EXEC_PROFILE_DEFAULT ,
2915
- paging_state = None , host = None ):
2913
+ def prepare_async (self , query , custom_payload = None , keyspace = None ):
2914
+ """
2915
+ Prepare the given query and return a :class:`~.PrepareFuture`
2916
+ object. You may also call :meth:`~.PrepareFuture.result()`
2917
+ on the :class:`.PrepareFuture` to synchronously block for
2918
+ prepared statement object at any time.
2919
+
2920
+ See :meth:`Session.prepare` for parameter definitions.
2921
+
2922
+ Example usage::
2923
+
2924
+ >>> future = session.prepare_async("SELECT * FROM mycf")
2925
+ >>> # do other stuff...
2926
+
2927
+ >>> try:
2928
+ ... prepared_statement = future.result()
2929
+ ... except Exception:
2930
+ ... log.exception("Operation failed:")
2931
+ """
2932
+ future = self ._create_prepare_response_future (query , keyspace , custom_payload )
2933
+ future ._protocol_handler = self .client_protocol_handler
2934
+ self ._on_request (future )
2935
+ future .send_request ()
2936
+ return future
2937
+
2938
+ def _create_prepare_response_future (self , query , keyspace , custom_payload ):
2939
+ return PrepareFuture (self , query , keyspace , custom_payload , self .default_timeout )
2940
+
2941
+ def _create_execute_response_future (self , query , parameters , trace , custom_payload ,
2942
+ timeout , execution_profile = EXEC_PROFILE_DEFAULT ,
2943
+ paging_state = None , host = None ):
2916
2944
""" Returns the ResponseFuture before calling send_request() on it """
2917
2945
2918
2946
prepared_statement = None
@@ -3121,33 +3149,9 @@ def prepare(self, query, custom_payload=None, keyspace=None):
3121
3149
`custom_payload` is a key value map to be passed along with the prepare
3122
3150
message. See :ref:`custom_payload`.
3123
3151
"""
3124
- message = PrepareMessage (query = query , keyspace = keyspace )
3125
- future = ResponseFuture (self , message , query = None , timeout = self .default_timeout )
3126
- try :
3127
- future .send_request ()
3128
- response = future .result ().one ()
3129
- except Exception :
3130
- log .exception ("Error preparing query:" )
3131
- raise
3152
+ return self .prepare_async (query , custom_payload , keyspace ).result ()
3132
3153
3133
- prepared_keyspace = keyspace if keyspace else None
3134
- prepared_statement = PreparedStatement .from_message (
3135
- response .query_id , response .bind_metadata , response .pk_indexes , self .cluster .metadata , query , prepared_keyspace ,
3136
- self ._protocol_version , response .column_metadata , response .result_metadata_id , self .cluster .column_encryption_policy )
3137
- prepared_statement .custom_payload = future .custom_payload
3138
-
3139
- self .cluster .add_prepared (response .query_id , prepared_statement )
3140
-
3141
- if self .cluster .prepare_on_all_hosts :
3142
- host = future ._current_host
3143
- try :
3144
- self .prepare_on_all_hosts (prepared_statement .query_string , host , prepared_keyspace )
3145
- except Exception :
3146
- log .exception ("Error preparing query on all hosts:" )
3147
-
3148
- return prepared_statement
3149
-
3150
- def prepare_on_all_hosts (self , query , excluded_host , keyspace = None ):
3154
+ def prepare_on_all_nodes (self , query , excluded_host , keyspace = None ):
3151
3155
"""
3152
3156
Prepare the given query on all hosts, excluding ``excluded_host``.
3153
3157
Intended for internal use only.
@@ -5105,6 +5109,48 @@ def __str__(self):
5105
5109
__repr__ = __str__
5106
5110
5107
5111
5112
+ class PrepareFuture (ResponseFuture ):
5113
+ _final_prepare_result = _NOT_SET
5114
+
5115
+ def __init__ (self , session , query , keyspace , custom_payload , timeout ):
5116
+ super ().__init__ (session , PrepareMessage (query = query , keyspace = keyspace ), None , timeout )
5117
+ self .query_string = query
5118
+ self ._keyspace = keyspace
5119
+ self ._custom_payload = custom_payload
5120
+
5121
+ def _set_final_result (self , response ):
5122
+ session = self .session
5123
+ cluster = session .cluster
5124
+ prepared_statement = PreparedStatement .from_message (
5125
+ response .query_id , response .bind_metadata , response .pk_indexes , cluster .metadata , self .query_string ,
5126
+ self ._keyspace , session ._protocol_version , response .column_metadata , response .result_metadata_id ,
5127
+ cluster .column_encryption_policy )
5128
+ prepared_statement .custom_payload = response .custom_payload
5129
+ cluster .add_prepared (response .query_id , prepared_statement )
5130
+ self ._final_prepare_result = prepared_statement
5131
+
5132
+ if cluster .prepare_on_all_hosts :
5133
+ # trigger asynchronous preparation of query on other C* nodes,
5134
+ # we are on event loop thread, so do not execute those synchronously
5135
+ session .submit (
5136
+ session .prepare_on_all_nodes ,
5137
+ self .query_string , self ._current_host , self ._keyspace )
5138
+
5139
+ super ()._set_final_result (response )
5140
+
5141
+ def result (self ):
5142
+ self ._event .wait ()
5143
+ if self ._final_prepare_result is not _NOT_SET :
5144
+ return self ._final_prepare_result
5145
+ else :
5146
+ raise self ._final_exception
5147
+
5148
+ def __str__ (self ):
5149
+ result = "(no result yet)" if self ._final_result is _NOT_SET else self ._final_result
5150
+ return "<PrepareFuture: query='%s' request_id=%s result=%s exception=%s coordinator_host=%s>" \
5151
+ % (self .query_string , self ._req_id , result , self ._final_exception , self .coordinator_host )
5152
+ __repr__ = __str__
5153
+
5108
5154
class QueryExhausted (Exception ):
5109
5155
"""
5110
5156
Raised when :meth:`.ResponseFuture.start_fetching_next_page()` is called and
0 commit comments