-
Notifications
You must be signed in to change notification settings - Fork 64
/
api.py
347 lines (297 loc) · 12.2 KB
/
api.py
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
import time
import base64
import functools
import json
import logging
import requests
import urllib
from copy import deepcopy
from requests import Timeout
from urllib.parse import urlparse
from hyper import HTTPConnection
from ksql.builder import SQLBuilder
from ksql.errors import CreateError, InvalidQueryError, KSQLError
class BaseAPI(object):
def __init__(self, url, **kwargs):
self.url = url
self.max_retries = kwargs.get("max_retries", 3)
self.delay = kwargs.get("delay", 0)
self.timeout = kwargs.get("timeout", 15)
self.api_key = kwargs.get("api_key")
self.secret = kwargs.get("secret")
self.headers = {
'Content-Type': 'application/vnd.ksql.v1+json; charset=utf-8',
}
def get_timout(self):
return self.timeout
@staticmethod
def _validate_sql_string(sql_string):
if len(sql_string) > 0:
if sql_string[-1] != ";":
sql_string = sql_string + ";"
else:
raise InvalidQueryError(sql_string)
return sql_string
@staticmethod
def _raise_for_status(r, response):
r_json = json.loads(response)
if r.getcode() != 200:
# seems to be the new API behavior
if r_json.get("@type") == "statement_error" or r_json.get("@type") == "generic_error":
error_message = r_json["message"]
error_code = r_json["error_code"]
stackTrace = r_json["stack_trace"]
raise KSQLError(error_message, error_code, stackTrace)
else:
raise KSQLError("Unknown Error: {}".format(r.content))
else:
# seems to be the old API behavior, so some errors have status 200, bug??
if r_json and r_json[0]["@type"] == "currentStatus" and r_json[0]["commandStatus"]["status"] == "ERROR":
error_message = r_json[0]["commandStatus"]["message"]
error_code = None
stackTrace = None
raise KSQLError(error_message, error_code, stackTrace)
return True
def ksql(self, ksql_string, stream_properties=None):
r = self._request(endpoint="ksql", sql_string=ksql_string, stream_properties=stream_properties)
response = r.read().decode("utf-8")
self._raise_for_status(r, response)
res = json.loads(response)
return res
def query2(self, query_string, encoding="utf-8", chunk_size=128, stream_properties=None, idle_timeout=None):
"""
Process streaming incoming data with HTTP/2.
"""
parsed_uri = urlparse(self.url)
logging.debug("KSQL generated: {}".format(query_string))
sql_string = self._validate_sql_string(query_string)
body = {"sql": sql_string}
if stream_properties:
body["properties"] = stream_properties
else:
body["properties"] = {}
with HTTPConnection(parsed_uri.netloc) as connection:
streaming_response = self._request2(
endpoint="query-stream", body=body, connection=connection
)
start_idle = None
if streaming_response.status == 200:
for chunk in streaming_response.read_chunked():
if chunk != b"\n":
start_idle = None
yield chunk.decode(encoding)
else:
if not start_idle:
start_idle = time.time()
if idle_timeout and time.time() - start_idle > idle_timeout:
print("Ending query because of time out! ({} seconds)".format(idle_timeout))
return
else:
raise ValueError("Return code is {}.".format(streaming_response.status))
def query(self, query_string, encoding="utf-8", chunk_size=128, stream_properties=None, idle_timeout=None):
"""
Process streaming incoming data.
"""
streaming_response = self._request(
endpoint="query", sql_string=query_string, stream_properties=stream_properties
)
start_idle = None
if streaming_response.code == 200:
for chunk in streaming_response:
if chunk != b"\n":
start_idle = None
yield chunk.decode(encoding)
else:
if not start_idle:
start_idle = time.time()
if idle_timeout and time.time() - start_idle > idle_timeout:
print("Ending query because of time out! ({} seconds)".format(idle_timeout))
return
else:
raise ValueError("Return code is {}.".format(streaming_response.status_code))
def get_request(self, endpoint):
auth = (self.api_key, self.secret) if self.api_key or self.secret else None
return requests.get(endpoint, headers=self.headers, auth=auth)
def _request2(self, endpoint, connection, body, method="POST", encoding="utf-8"):
url = "{}/{}".format(self.url, endpoint)
data = json.dumps(body).encode(encoding)
headers = deepcopy(self.headers)
if self.api_key and self.secret:
base64string = base64.b64encode(bytes("{}:{}".format(self.api_key, self.secret), "utf-8")).decode("utf-8")
headers["Authorization"] = "Basic %s" % base64string
connection.request(method=method.upper(), url=url, headers=headers, body=data)
resp = connection.get_response()
return resp
def _request(self, endpoint, method="POST", sql_string="", stream_properties=None, encoding="utf-8"):
url = "{}/{}".format(self.url, endpoint)
logging.debug("KSQL generated: {}".format(sql_string))
sql_string = self._validate_sql_string(sql_string)
body = {"ksql": sql_string}
if stream_properties:
body["streamsProperties"] = stream_properties
else:
body["streamsProperties"] = {}
data = json.dumps(body).encode(encoding)
headers = deepcopy(self.headers)
if self.api_key and self.secret:
base64string = base64.b64encode(bytes("{}:{}".format(self.api_key, self.secret), "utf-8")).decode("utf-8")
headers["Authorization"] = "Basic %s" % base64string
req = urllib.request.Request(url=url, data=data, headers=headers, method=method.upper())
try:
r = urllib.request.urlopen(req, timeout=self.timeout)
except urllib.error.HTTPError as http_error:
try:
content = json.loads(http_error.read().decode(encoding))
except Exception as e:
raise http_error
else:
logging.debug("content: {}".format(content))
raise KSQLError(content.get("message"), content.get("error_code"), content.get("stackTrace"))
else:
return r
def close_query(self, query_id):
body = {"queryId": query_id}
data = json.dumps(body).encode("utf-8")
url = "{}/{}".format(self.url, "close-query")
response = requests.post(url=url, data=data)
if response.status_code == 200:
logging.debug("Successfully canceled Query ID: {}".format(query_id))
return True
elif response.status_code == 400:
message = json.loads(response.content)["message"]
logging.debug("Failed canceling Query ID: {}: {}".format(query_id, message))
return False
else:
raise ValueError("Return code is {}.".format(response.status_code))
def inserts_stream(self, stream_name, rows):
body = '{{"target":"{}"}}'.format(stream_name)
for row in rows:
body += '\n{}'.format(json.dumps(row))
parsed_uri = urlparse(self.url)
url = "{}/{}".format(self.url, "inserts-stream")
headers = deepcopy(self.headers)
with HTTPConnection(parsed_uri.netloc) as connection:
connection.request("POST", url, bytes(body, "utf-8"), headers)
response = connection.get_response()
result = response.read()
result_str = result.decode("utf-8")
result_chunks = result_str.split("\n")
return_arr = []
for chunk in result_chunks:
try:
return_arr.append(json.loads(chunk))
except:
pass
return return_arr
@staticmethod
def retry(exceptions, delay=1, max_retries=5):
"""
A decorator for retrying a function call with a specified delay in case of a set of exceptions
Parameter List
-------------
:param exceptions: A tuple of all exceptions that need to be caught for retry
e.g. retry(exception_list = (Timeout, Readtimeout))
:param delay: Amount of delay (seconds) needed between successive retries.
:param times: no of times the function should be retried
"""
def outer_wrapper(function):
@functools.wraps(function)
def inner_wrapper(*args, **kwargs):
final_excep = None
for counter in range(max_retries):
if counter > 0:
time.sleep(delay)
final_excep = None
try:
value = function(*args, **kwargs)
return value
except (exceptions) as e:
final_excep = e
pass # or log it
if final_excep is not None:
raise final_excep
return inner_wrapper
return outer_wrapper
class SimplifiedAPI(BaseAPI):
def __init__(self, url, **kwargs):
super(SimplifiedAPI, self).__init__(url, **kwargs)
def create_stream(self, table_name, columns_type, topic, value_format="JSON"):
return self._create(
table_type="stream",
table_name=table_name,
columns_type=columns_type,
topic=topic,
value_format=value_format,
)
def create_table(self, table_name, columns_type, topic, value_format, key):
if not key:
raise ValueError("key is required for creating a table.")
return self._create(
table_type="table",
table_name=table_name,
columns_type=columns_type,
topic=topic,
value_format=value_format,
key=key,
)
def create_stream_as(
self,
table_name,
select_columns,
src_table,
kafka_topic=None,
value_format="JSON",
conditions=[],
partition_by=None,
**kwargs
):
return self._create_as(
table_type="stream",
table_name=table_name,
select_columns=select_columns,
src_table=src_table,
kafka_topic=kafka_topic,
value_format=value_format,
conditions=conditions,
partition_by=partition_by,
**kwargs,
)
def _create(self, table_type, table_name, columns_type, topic, value_format="JSON", key=None):
ksql_string = SQLBuilder.build(
sql_type="create",
table_type=table_type,
table_name=table_name,
columns_type=columns_type,
topic=topic,
value_format=value_format,
key=key,
)
self.ksql(ksql_string)
return True
@BaseAPI.retry(exceptions=(Timeout, CreateError))
def _create_as(
self,
table_type,
table_name,
select_columns,
src_table,
kafka_topic=None,
value_format="JSON",
conditions=[],
partition_by=None,
**kwargs
):
ksql_string = SQLBuilder.build(
sql_type="create_as",
table_type=table_type,
table_name=table_name,
select_columns=select_columns,
src_table=src_table,
kafka_topic=kafka_topic,
value_format=value_format,
conditions=conditions,
partition_by=partition_by,
**kwargs,
)
self.ksql(ksql_string)
return True