-
Notifications
You must be signed in to change notification settings - Fork 115
/
sink.py
1027 lines (876 loc) · 38 KB
/
sink.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
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
#
# Copyright Elasticsearch B.V. and/or licensed to Elasticsearch B.V. under one
# or more contributor license agreements. Licensed under the Elastic License 2.0;
# you may not use this file except in compliance with the Elastic License 2.0.
#
"""
`SyncOrchestrator` is orchestrating a sync by:
- creating a queue
- launching a `Extractor`, a wrapper on the top of the documents' generator
- launching a `Sink`, a class that aggregates documents and run the bulk API
SyncOrchestrator.async_bulk(generator)
|
|
Elasticsearch <== Sink <== queue <== Extractor <== generator
"""
import asyncio
import copy
import functools
import logging
import time
from connectors.config import (
DEFAULT_ELASTICSEARCH_MAX_RETRIES,
DEFAULT_ELASTICSEARCH_RETRY_INTERVAL,
)
from connectors.es.management_client import ESManagementClient
from connectors.es.settings import TIMESTAMP_FIELD, Mappings
from connectors.filtering.basic_rule import BasicRuleEngine, parse
from connectors.logger import logger, tracer
from connectors.protocol import Filter, JobType
from connectors.protocol.connectors import (
DELETED_DOCUMENT_COUNT,
INDEXED_DOCUMENT_COUNT,
INDEXED_DOCUMENT_VOLUME,
)
from connectors.utils import (
DEFAULT_CHUNK_MEM_SIZE,
DEFAULT_CHUNK_SIZE,
DEFAULT_CONCURRENT_DOWNLOADS,
DEFAULT_DISPLAY_EVERY,
DEFAULT_MAX_CONCURRENCY,
DEFAULT_QUEUE_MEM_SIZE,
DEFAULT_QUEUE_SIZE,
ConcurrentTasks,
Counters,
MemQueue,
aenumerate,
get_size,
iso_utc,
retryable,
)
__all__ = ["SyncOrchestrator"]
EXTRACTOR_ERROR = "EXTRACTOR_ERROR"
END_DOCS = "END_DOCS"
OP_INDEX = "index"
OP_DELETE = "delete"
OP_CREATE = "create"
OP_UPDATE = "update"
OP_UNKNOWN = "operation_unknown"
CANCELATION_TIMEOUT = 5
# counter keys
BIN_DOCS_DOWNLOADED = "binary_docs_downloaded"
BULK_OPERATIONS = "bulk_operations"
BULK_RESPONSES = "bulk_item_responses"
CREATES_QUEUED = "doc_creates_queued"
UPDATES_QUEUED = "doc_updates_queued"
DELETES_QUEUED = "doc_deletes_queued"
DOCS_EXTRACTED = "docs_extracted"
DOCS_FILTERED = "docs_filtered"
DOCS_DROPPED = "docs_dropped"
ID_MISSING = "_ids_missing"
RESULT_ERROR = "result_errors"
RESULT_SUCCESS = "result_successes"
RESULT_UNDEFINED = "results_undefined"
ID_CHANGED_AFTER_REQUEST = "_ids_changed_after_request"
ID_DUPLICATE = "_id_duplicates"
# Successful results according to the docs: https://www.elastic.co/guide/en/elasticsearch/reference/current/docs-bulk.html#bulk-api-response-body
SUCCESSFUL_RESULTS = ("created", "deleted", "updated")
def get_mib_size(obj):
"""Returns the size of ob in MiB"""
return round(get_size(obj) / (1024 * 1024), 2)
class UnsupportedJobType(Exception):
pass
class ForceCanceledError(Exception):
pass
class ContentIndexDoesNotExistError(Exception):
pass
class ElasticsearchOverloadedError(Exception):
def __init__(self, cause=None):
msg = "Connector was unable to ingest data into overloaded Elasticsearch. Make sure Elasticsearch instance is healthy, has enough resources and content index is healthy."
super().__init__(msg)
self.__cause__ = cause
class Sink:
"""Send bulk operations in batches by consuming a queue.
This class runs a coroutine that gets operations out of a `queue` and collects them to
build and send bulk requests using a `client`
Arguments:
- `client` -- an instance of `connectors.es.ESManagementClient`
- `queue` -- an instance of `asyncio.Queue` to pull docs from
- `chunk_size` -- a maximum number of operations to send per request
- `pipeline` -- ingest pipeline settings to pass to the bulk API
- `chunk_mem_size` -- a maximum size in MiB for each bulk request
- `max_concurrency` -- a maximum number of concurrent bulk requests
"""
def __init__(
self,
client,
queue,
chunk_size,
pipeline,
chunk_mem_size,
max_concurrency,
max_retries,
retry_interval,
logger_=None,
enable_bulk_operations_logging=False,
):
self.client = client
self.queue = queue
self.chunk_size = chunk_size
self.pipeline = pipeline
self.chunk_mem_size = chunk_mem_size * 1024 * 1024
self.bulk_tasks = ConcurrentTasks(max_concurrency=max_concurrency)
self.max_retires = max_retries
self.retry_interval = retry_interval
self.error = None
self._logger = logger_ or logger
self._canceled = False
self._enable_bulk_operations_logging = enable_bulk_operations_logging
self.counters = Counters()
def _bulk_op(self, doc, operation=OP_INDEX):
doc_id = doc["_id"]
index = doc["_index"]
if operation == OP_INDEX:
return [{operation: {"_index": index, "_id": doc_id}}, doc["doc"]]
if operation == OP_UPDATE:
return [
{operation: {"_index": index, "_id": doc_id}},
{"doc": doc["doc"], "doc_as_upsert": True},
]
if operation == OP_DELETE:
return [{operation: {"_index": index, "_id": doc_id}}]
raise TypeError(operation)
@tracer.start_as_current_span("_bulk API call", slow_log=1.0)
async def _batch_bulk(self, operations, stats):
# TODO: make this retry policy work with unified retry strategy
@retryable(retries=self.max_retires, interval=self.retry_interval)
async def _bulk_api_call():
return await self.client.client.bulk(
operations=operations, pipeline=self.pipeline["name"]
)
# TODO: treat result to retry errors like in async_streaming_bulk
task_num = len(self.bulk_tasks)
if self._logger.isEnabledFor(logging.DEBUG):
self._logger.debug(
f"Task {task_num} - Sending a batch of {len(operations)} ops -- {get_mib_size(operations)}MiB"
)
# TODO: retry 429s for individual items here
res = await self.client.bulk_insert(operations, self.pipeline["name"])
ids_to_ops = self._map_id_to_op(operations)
await self._process_bulk_response(
res, ids_to_ops, do_log=self._enable_bulk_operations_logging
)
if res.get("errors"):
for item in res["items"]:
for op, data in item.items():
if "error" in data:
self._logger.error(f"operation {op} failed, {data['error']}")
self._populate_stats(stats, res)
return res
def _map_id_to_op(self, operations):
"""
Takes operations like: [{operation: {"_index": index, "_id": doc_id}}, doc["doc"]]
and turns them into { doc_id : operation }
"""
result = {}
for entry in operations:
if len(entry.keys()) == 1: # only looking at "operation" entries
for op, doc in entry.items():
if (
isinstance(doc, dict)
and "_id" in doc.keys()
and "_index" in doc.keys()
): # avoiding update bulk extra entries
result[doc["_id"]] = op
return result
async def _process_bulk_response(self, res, ids_to_ops, do_log=False):
for item in res.get("items", []):
if OP_INDEX in item:
action_item = OP_INDEX
elif OP_DELETE in item:
action_item = OP_DELETE
elif OP_CREATE in item:
action_item = OP_CREATE
elif OP_UPDATE in item:
action_item = OP_UPDATE
else:
# Should only happen, if the _bulk API changes
# Unlikely, but as this functionality could be used for audits we want to detect changes fast
if do_log:
self._logger.error(
f"Unknown action item returned from _bulk API for item {item}"
)
self.counters.increment(OP_UNKNOWN, namespace=BULK_RESPONSES)
continue
self.counters.increment(action_item, namespace=BULK_RESPONSES)
doc_id = item[action_item].get("_id")
if doc_id is None:
# Should only happen, if the _bulk API changes
# Unlikely, but as this functionality could be used for audits we want to detect changes fast
if do_log:
self._logger.error(f"Could not retrieve '_id' for document {item}")
self.counters.increment(ID_MISSING, namespace=BULK_RESPONSES)
continue
result = item[action_item].get("result")
requested_op = ids_to_ops.get(doc_id, None)
if requested_op is None:
# This ID wasn't in the request we sent, meaning that the ID was changed (probably via pipeline).
self.counters.increment(
ID_CHANGED_AFTER_REQUEST, namespace=BULK_RESPONSES
)
elif action_item != OP_UPDATE and result == "updated":
# This means we sent an `index` op, but didn't create a new doc. This could mean that there was a
# doc with this ID in the index before this sync, OR that this ID showed up more than once during
# this sync.
self.counters.increment(ID_DUPLICATE, namespace=BULK_RESPONSES)
if result == "noop":
# This means that whatever the requested op was, nothing happened. This is most likely to mean
# that the document was dropped during the ingest pipeline
self.counters.increment(DOCS_DROPPED, namespace=BULK_RESPONSES)
successful_result = result in SUCCESSFUL_RESULTS
if not successful_result:
if "error" in item[action_item]:
if do_log:
self._logger.debug(
f"Failed to execute '{action_item}' on document with id '{doc_id}'. Error: {item[action_item].get('error')}"
)
self.counters.increment(RESULT_ERROR, namespace=BULK_RESPONSES)
else:
if do_log:
self._logger.debug(
f"Executed '{action_item}' on document with id '{doc_id}', but got non-successful result: {result}"
)
self.counters.increment(RESULT_UNDEFINED, namespace=BULK_RESPONSES)
else:
if do_log:
self._logger.debug(
f"Successfully executed '{action_item}' on document with id '{doc_id}'. Result: {result}"
)
self.counters.increment(RESULT_SUCCESS)
def _populate_stats(self, stats, res):
for item in res["items"]:
for op, data in item.items():
# "result" is only present in successful operations
if "result" not in data:
del stats[op][data["_id"]]
self.counters.increment(
INDEXED_DOCUMENT_COUNT, len(stats[OP_INDEX]) + len(stats[OP_UPDATE])
)
self.counters.increment(
INDEXED_DOCUMENT_VOLUME,
sum(stats[OP_INDEX].values()) + sum(stats[OP_UPDATE].values()),
)
self.counters.increment(DELETED_DOCUMENT_COUNT, len(stats[OP_DELETE]))
self._logger.debug(
f"Sink stats - no. of docs indexed: {self.counters.get(INDEXED_DOCUMENT_COUNT)}, volume of docs indexed: {round(self.counters.get(INDEXED_DOCUMENT_VOLUME))} bytes, no. of docs deleted: {self.counters.get(DELETED_DOCUMENT_COUNT)}"
)
def force_cancel(self):
self._canceled = True
async def fetch_doc(self):
if self._canceled:
raise ForceCanceledError
return await self.queue.get()
async def run(self):
try:
await self._run()
except asyncio.CancelledError:
self._logger.info("Task is canceled, stop Sink...")
raise
except asyncio.QueueFull as e:
raise ElasticsearchOverloadedError from e
except Exception as e:
if isinstance(e, ForceCanceledError) or self._canceled:
self._logger.warning(
f"Sink did not stop within {CANCELATION_TIMEOUT} seconds of cancelation, force-canceling the task."
)
return
raise
async def _run(self):
"""Creates batches of bulk calls given a queue of items.
An item is a (size, object) tuple. Exits when the
item is the `END_DOCS` or `EXTRACTOR_ERROR` string.
Bulk calls are executed concurrently with a maximum number of concurrent
requests.
"""
try:
batch = []
# stats is a dictionary containing stats for 3 operations. In each sub-dictionary, it is a doc id to size map.
stats = {OP_INDEX: {}, OP_UPDATE: {}, OP_DELETE: {}}
bulk_size = 0
overhead_size = None
batch_num = 0
while True:
batch_num += 1
doc_size, doc = await self.fetch_doc()
if doc in (END_DOCS, EXTRACTOR_ERROR):
break
operation = doc["_op_type"]
doc_id = doc["_id"]
if not doc_id:
self._logger.warning(f"Skip document {doc} as '_id' is missing.")
continue
if operation == OP_DELETE:
stats[operation][doc_id] = 0
else:
# the doc_size also includes _op_type, _index and _id,
# which we want to exclude when calculating the size.
if overhead_size is None:
overhead = {
"_op_type": operation,
"_index": doc["_index"],
"_id": doc_id,
}
overhead_size = get_size(overhead)
stats[operation][doc_id] = max(doc_size - overhead_size, 0)
self.counters.increment(operation, namespace=BULK_OPERATIONS)
batch.extend(self._bulk_op(doc, operation))
bulk_size += doc_size
if len(batch) >= self.chunk_size or bulk_size > self.chunk_mem_size:
await self.bulk_tasks.put(
functools.partial(
self._batch_bulk,
copy.copy(batch),
copy.copy(stats),
),
name=f"Elasticsearch Sink: _bulk batch #{batch_num}",
)
batch.clear()
stats = {OP_INDEX: {}, OP_UPDATE: {}, OP_DELETE: {}}
bulk_size = 0
await asyncio.sleep(0)
self.bulk_tasks.raise_any_exception()
await self.bulk_tasks.join(raise_on_error=True)
if len(batch) > 0:
await self._batch_bulk(batch, stats)
except Exception as e:
self.error = e
raise
class Extractor:
"""Grabs data and adds them in the queue for the bulker.
This class runs a coroutine that puts docs in `queue`, given a document generator.
Arguments:
- client: an instance of `connectors.es.ESManagementClient`
- queue: an `asyncio.Queue` to put docs in
- index: the target Elasticsearch index
- filter_: an instance of `Filter` to apply on the fetched document -- default: `None`
- sync_rules_enabled: if `True`, we apply rules -- default: `False`
- content_extraction_enabled: if `True`, download content -- default `True`
- display_every -- display a log every `display_every` doc -- default: `DEFAULT_DISPLAY_EVERY`
- concurrent_downloads: -- concurrency level for downloads -- default: `DEFAULT_CONCURRENT_DOWNLOADS`
"""
def __init__(
self,
client,
queue,
index,
filter_=None,
sync_rules_enabled=False,
content_extraction_enabled=True,
display_every=DEFAULT_DISPLAY_EVERY,
concurrent_downloads=DEFAULT_CONCURRENT_DOWNLOADS,
logger_=None,
skip_unchanged_documents=False,
):
if filter_ is None:
filter_ = Filter()
self.client = client
self.queue = queue
self.index = index
self.loop = asyncio.get_event_loop()
self.counters = Counters()
self.error = None
self.filter_ = filter_
self.basic_rule_engine = (
BasicRuleEngine(parse(filter_.basic_rules)) if sync_rules_enabled else None
)
self.content_extraction_enabled = content_extraction_enabled
self.display_every = display_every
self.concurrent_downloads = concurrent_downloads
self._logger = logger_ or logger
self._canceled = False
self.skip_unchanged_documents = skip_unchanged_documents
async def _deferred_index(self, lazy_download, doc_id, doc, operation):
data = await lazy_download(doit=True, timestamp=doc[TIMESTAMP_FIELD])
if data is not None:
self.counters.increment(BIN_DOCS_DOWNLOADED)
data.pop("_id", None)
data.pop(TIMESTAMP_FIELD, None)
doc.update(data)
doc.pop("_original_filename", None)
await self.put_doc(
{
"_op_type": operation,
"_index": self.index,
"_id": doc_id,
"doc": doc,
}
)
def force_cancel(self):
self._canceled = True
async def put_doc(self, doc):
if self._canceled:
raise ForceCanceledError
await self.queue.put(doc)
async def run(self, generator, job_type):
try:
match job_type:
case JobType.FULL:
await self.get_docs(generator)
case JobType.INCREMENTAL:
if self.skip_unchanged_documents:
await self.get_docs(generator, skip_unchanged_documents=True)
else:
await self.get_docs_incrementally(generator)
case JobType.ACCESS_CONTROL:
await self.get_access_control_docs(generator)
case _:
raise UnsupportedJobType
except asyncio.CancelledError:
self._logger.info("Task is canceled, stop Extractor...")
raise
except asyncio.QueueFull as e:
self._logger.error("Sync was throttled by Elasticsearch")
# We clear the queue as we could not actually ingest anything.
# After that we indicate that we've encountered an error
self.queue.clear()
await self.put_doc(EXTRACTOR_ERROR)
self.error = ElasticsearchOverloadedError(e)
except Exception as e:
if isinstance(e, ForceCanceledError) or self._canceled:
self._logger.warning(
f"Extractor did not stop within {CANCELATION_TIMEOUT} seconds of cancelation, force-canceling the task."
)
return
self._logger.critical("Document extractor failed", exc_info=True)
await self.put_doc(EXTRACTOR_ERROR)
self.error = e
@tracer.start_as_current_span("get_doc call", slow_log=1.0)
async def _decorate_with_metrics_span(self, generator):
"""Wrapper for metrics"""
async for doc in generator:
yield doc
async def get_docs(self, generator, skip_unchanged_documents=False):
"""Iterate on a generator of documents to fill a queue of bulk operations for the `Sink` to consume.
Extraction happens in a separate task, when a document contains files.
Args:
generator (generator): BaseDataSource child get_docs or get_docs_incrementally
skip_unchanged_documents (bool): if True, will skip documents that have not changed since last sync
"""
generator = self._decorate_with_metrics_span(generator)
existing_ids = await self._load_existing_docs()
self._logger.info("Iterating on remote documents")
lazy_downloads = ConcurrentTasks(self.concurrent_downloads)
download_num = 0
try:
async for count, doc in aenumerate(generator):
self.counters.increment(DOCS_EXTRACTED)
doc, lazy_download, operation = doc
if count % self.display_every == 0:
self._log_progress()
doc_id = doc["id"] = doc.pop("_id")
if self.basic_rule_engine and not self.basic_rule_engine.should_ingest(
doc
):
self.counters.increment((DOCS_FILTERED))
continue
if doc_id in existing_ids:
# pop out of existing_ids, so they do not get deleted
ts = existing_ids.pop(doc_id)
if (
skip_unchanged_documents
and TIMESTAMP_FIELD in doc
and ts == doc[TIMESTAMP_FIELD]
):
# cancel the download
if (
self.content_extraction_enabled
and lazy_download is not None
):
await lazy_download(doit=False)
self._logger.debug(
f"Skipping document with id '{doc_id}' because field '{TIMESTAMP_FIELD}' has not changed since last sync"
)
continue
self.counters.increment(UPDATES_QUEUED)
else:
self.counters.increment(CREATES_QUEUED)
if TIMESTAMP_FIELD not in doc:
doc[TIMESTAMP_FIELD] = iso_utc()
# if we need to call lazy_download we push it in lazy_downloads
if self.content_extraction_enabled and lazy_download is not None:
download_num += 1
await lazy_downloads.put(
functools.partial(
self._deferred_index, lazy_download, doc_id, doc, operation
),
name=f"Extractor download #{download_num}",
)
else:
# we can push into the queue right away
await self.put_doc(
{
"_op_type": operation,
"_index": self.index,
"_id": doc_id,
"doc": doc,
}
)
await asyncio.sleep(0)
finally:
# wait for all downloads to be finished
await lazy_downloads.join()
await self.enqueue_docs_to_delete(existing_ids)
await self.put_doc(END_DOCS)
async def _load_existing_docs(self):
start = time.time()
self._logger.info("Collecting local document ids")
existing_ids = {
k: v
async for (k, v) in self.client.yield_existing_documents_metadata(
self.index
)
}
self._logger.debug(
f"Found {len(existing_ids)} docs in {self.index} (duration "
f"{int(time.time() - start)} seconds) "
)
if self._logger.isEnabledFor(logging.DEBUG):
self._logger.debug(
f"Size of ids in memory is {get_mib_size(existing_ids)}MiB"
)
return existing_ids
async def get_docs_incrementally(self, generator):
"""Iterate on a generator of documents to fill a queue with bulk operations for the `Sink` to consume.
A document might be discarded if its timestamp has not changed.
Extraction happens in a separate task, when a document contains files.
"""
generator = self._decorate_with_metrics_span(generator)
self._logger.info("Iterating on remote documents incrementally when possible")
lazy_downloads = ConcurrentTasks(self.concurrent_downloads)
num_downloads = 0
try:
async for count, doc in aenumerate(generator):
doc, lazy_download, operation = doc
if count % self.display_every == 0:
self._log_progress()
doc_id = doc["id"] = doc.pop("_id")
if self.basic_rule_engine and not self.basic_rule_engine.should_ingest(
doc
):
continue
if operation == OP_INDEX:
self.counters.increment(CREATES_QUEUED)
elif operation == OP_UPDATE:
self.counters.increment(UPDATES_QUEUED)
elif operation == OP_DELETE:
self.counters.increment(DELETES_QUEUED)
else:
self._logger.error(
f"unsupported operation {operation} for doc {doc_id}"
)
if TIMESTAMP_FIELD not in doc:
doc[TIMESTAMP_FIELD] = iso_utc()
# if we need to call lazy_download we push it in lazy_downloads
if self.content_extraction_enabled and lazy_download is not None:
num_downloads += 1
await lazy_downloads.put(
functools.partial(
self._deferred_index, lazy_download, doc_id, doc, operation
),
name=f"Extractor download #{num_downloads}",
)
else:
# we can push into the queue right away
item = {
"_op_type": operation,
"_index": self.index,
"_id": doc_id,
}
if operation in (OP_INDEX, OP_UPDATE):
item["doc"] = doc
await self.put_doc(item)
await asyncio.sleep(0)
finally:
# wait for all downloads to be finished
await lazy_downloads.join()
await self.put_doc(END_DOCS)
async def get_access_control_docs(self, generator):
"""Iterate on a generator of access control documents to fill a queue with bulk operations for the `Sink` to consume.
A document might be discarded if its timestamp has not changed.
"""
self._logger.info("Starting access control doc lookups")
generator = self._decorate_with_metrics_span(generator)
existing_ids = {
doc_id: last_update_timestamp
async for (
doc_id,
last_update_timestamp,
) in self.client.yield_existing_documents_metadata(self.index)
}
if self._logger.isEnabledFor(logging.DEBUG):
self._logger.debug(
f"Size of {len(existing_ids)} access control document ids in memory is {get_mib_size(existing_ids)}MiB"
)
count = 0
async for doc in generator:
doc, _, _ = doc
count += 1
if count % self.display_every == 0:
self._log_progress()
doc_id = doc["id"] = doc.pop("_id")
doc_exists = doc_id in existing_ids
if doc_exists:
last_update_timestamp = existing_ids.pop(doc_id)
doc_not_updated = (
TIMESTAMP_FIELD in doc
and last_update_timestamp == doc[TIMESTAMP_FIELD]
)
if doc_not_updated:
continue
self.counters.increment(UPDATES_QUEUED)
operation = OP_UPDATE
else:
self.counters.increment(CREATES_QUEUED)
if TIMESTAMP_FIELD not in doc:
doc[TIMESTAMP_FIELD] = iso_utc()
operation = OP_INDEX
await self.put_doc(
{
"_op_type": operation,
"_index": self.index,
"_id": doc_id,
"doc": doc,
}
)
await asyncio.sleep(0)
await self.enqueue_docs_to_delete(existing_ids)
await self.put_doc(END_DOCS)
async def enqueue_docs_to_delete(self, existing_ids):
self._logger.debug(f"Delete {len(existing_ids)} docs from index '{self.index}'")
for doc_id in existing_ids.keys():
await self.put_doc(
{
"_op_type": OP_DELETE,
"_index": self.index,
"_id": doc_id,
}
)
self.counters.increment(DELETES_QUEUED)
def _log_progress(
self,
):
self._logger.info(
"Sync progress -- "
f"created: {self.counters.get(CREATES_QUEUED)} | "
f"updated: {self.counters.get(UPDATES_QUEUED)} | "
f"deleted: {self.counters.get(DELETES_QUEUED)}"
)
class ContentIndexNameInvalid(Exception):
pass
class AsyncBulkRunningError(Exception):
pass
class SyncOrchestrator:
"""This class is the sync orchestrator.
It does the following in `async_bulk`
- grabs all ids on Elasticsearch for the index
- creates a MemQueue to hold documents to stream
- runs a `Extractor` (producer) and a `Sink` (consumer) against the queue
- once they are both over, returns totals
"""
def __init__(self, elastic_config, logger_=None):
self._logger = logger_ or logger
self._logger.debug(f"SyncOrchestrator connecting to {elastic_config['host']}")
self.es_management_client = ESManagementClient(elastic_config)
self.loop = asyncio.get_event_loop()
self._extractor = None
self._extractor_task = None
self._sink = None
self._sink_task = None
self.error = None
self.canceled = False
async def close(self):
await self.es_management_client.close()
async def has_active_license_enabled(self, license_):
# TODO: think how to make it not a proxy method to the client
return await self.es_management_client.has_active_license_enabled(license_)
async def prepare_content_index(self, index_name, language_code=None):
"""Creates the index, given a mapping/settings if it does not exist."""
self._logger.debug(f"Checking index {index_name}")
result = await self.es_management_client.get_index(
index_name, ignore_unavailable=True
)
index = result.get(index_name, None)
mappings = Mappings.default_text_fields_mappings(is_connectors_index=True)
if index:
# Update the index mappings if needed
self._logger.debug(f"{index_name} exists")
# Settings contain analyzers which are being used in the index mappings
# Therefore settings must be applied before mappings
await self.es_management_client.ensure_content_index_settings(
index_name=index_name, index=index, language_code=language_code
)
await self.es_management_client.ensure_content_index_mappings(
index_name, mappings
)
else:
# Create a new index
self._logger.info(f"Creating content index: {index_name}")
await self.es_management_client.create_content_index(
index_name, language_code
)
self._logger.info(f"Content index successfully created: {index_name}")
def done(self):
"""
An async task (which this mimics) should be "done" if:
- it was canceled
- it errored
- it completed successfully
:return: True if the orchestrator is "done", else False
"""
if self.get_error() is not None:
return True
extractor_done = (
True
if self._extractor_task is None or self._extractor_task.done()
else False
)
sink_done = True if self._sink_task is None or self._sink_task.done() else False
return extractor_done and sink_done
def _sink_task_running(self):
return self._sink_task is not None and not self._sink_task.done()
def _extractor_task_running(self):
return self._extractor_task is not None and not self._extractor_task.done()
async def cancel(self):
if self._sink_task_running():
self._sink_task.cancel()
if self._extractor_task_running():
self._extractor_task.cancel()
self.canceled = True
cancelation_timeout = CANCELATION_TIMEOUT
while cancelation_timeout > 0:
await asyncio.sleep(1)
cancelation_timeout -= 1
if not self._sink_task_running() and not self._extractor_task_running():
self._logger.info(
"Both Extractor and Sink tasks are successfully stopped."
)
return
self._logger.error(
f"Sync job did not stop within {CANCELATION_TIMEOUT} seconds of canceling. Force-canceling."
)
self._sink.force_cancel()
self._extractor.force_cancel()
def ingestion_stats(self):
stats = {}
if self._extractor is not None:
stats.update(self._extractor.counters.to_dict())
if self._sink is not None:
stats.update(self._sink.counters.to_dict())
stats[INDEXED_DOCUMENT_VOLUME] = round(
self._sink.counters.get(INDEXED_DOCUMENT_VOLUME) / (1024 * 1024)
) # return indexed_document_volume in number of MiB
return stats
def get_error(self):
return (
None
if self._extractor is None
else (self._extractor.error or self._sink.error or self.error)
)
async def async_bulk(
self,
index,
generator,
pipeline,
job_type,
filter_=None,
sync_rules_enabled=False,
content_extraction_enabled=True,
options=None,
skip_unchanged_documents=False,
enable_bulk_operations_logging=False,
):
"""Performs a batch of `_bulk` calls, given a generator of documents
Arguments:
- index: target index
- generator: documents generator
- pipeline: ingest pipeline settings to pass to the bulk API
- job_type: the job type of the sync job
- filter_: an instance of `Filter` to apply on the fetched document -- default: `None`
- sync_rules_enabled: if enabled, applies rules -- default: `False`
- content_extraction_enabled: if enabled, will download content -- default: `True`
- options: dict of options (from `elasticsearch.bulk` in the config file)
"""
if self._extractor_task is not None or self._sink_task is not None:
msg = "Async bulk task has already started."
raise AsyncBulkRunningError(msg)
if filter_ is None:
filter_ = Filter()
if options is None:
options = {}
queue_size = options.get("queue_max_size", DEFAULT_QUEUE_SIZE)
display_every = options.get("display_every", DEFAULT_DISPLAY_EVERY)
queue_mem_size = options.get("queue_max_mem_size", DEFAULT_QUEUE_MEM_SIZE)
chunk_mem_size = options.get("chunk_max_mem_size", DEFAULT_CHUNK_MEM_SIZE)
max_concurrency = options.get("max_concurrency", DEFAULT_MAX_CONCURRENCY)
chunk_size = options.get("chunk_size", DEFAULT_CHUNK_SIZE)
concurrent_downloads = options.get(
"concurrent_downloads", DEFAULT_CONCURRENT_DOWNLOADS
)
max_bulk_retries = options.get("max_retries", DEFAULT_ELASTICSEARCH_MAX_RETRIES)
retry_interval = options.get(
"retry_interval", DEFAULT_ELASTICSEARCH_RETRY_INTERVAL
)
mem_queue_refresh_timeout = options.get("queue_refresh_timeout", 60)
mem_queue_refresh_interval = options.get("queue_refresh_interval", 1)
stream = MemQueue(
maxsize=queue_size,
maxmemsize=queue_mem_size * 1024 * 1024,
refresh_timeout=mem_queue_refresh_timeout,
refresh_interval=mem_queue_refresh_interval,
)
# start the fetcher
self._extractor = Extractor(
self.es_management_client,
stream,
index,
filter_=filter_,
sync_rules_enabled=sync_rules_enabled,
content_extraction_enabled=content_extraction_enabled,
display_every=display_every,
concurrent_downloads=concurrent_downloads,
logger_=self._logger,
skip_unchanged_documents=skip_unchanged_documents,
)
self._extractor_task = asyncio.create_task(
self._extractor.run(generator, job_type)
)
self._extractor_task.add_done_callback(
functools.partial(self.extractor_task_callback)
)
# start the bulker
self._sink = Sink(