-
Notifications
You must be signed in to change notification settings - Fork 9
Expand file tree
/
Copy pathconsumer.py
More file actions
920 lines (750 loc) · 39.3 KB
/
consumer.py
File metadata and controls
920 lines (750 loc) · 39.3 KB
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
import asyncio
import logging
from asyncio import TimeoutError
from datetime import datetime, timezone
from typing import Any, AsyncIterator, Dict, Optional
from aiobotocore.session import AioSession
from aiohttp import ClientConnectionError
from botocore.exceptions import ClientError
from .base import Base
from .checkpointers import CheckPointer, MemoryCheckPointer
from .processors import JsonProcessor, Processor
from .utils import Throttler
log = logging.getLogger(__name__)
class ShardStats:
def __init__(self) -> None:
self._throttled: int = 0
self._success: int = 0
def succeded(self) -> None:
self._success += 1
def throttled(self) -> None:
self._throttled += 1
def to_data(self) -> Dict[str, int]:
return {"throttled": self._throttled, "success": self._success}
class Consumer(Base):
def __init__(
self,
stream_name: str,
*,
session: Optional[AioSession] = None,
endpoint_url: Optional[str] = None,
region_name: Optional[str] = None,
max_queue_size: int = 10000,
max_shard_consumers: Optional[int] = None,
record_limit: int = 10000,
sleep_time_no_records: float = 2,
iterator_type: str = "TRIM_HORIZON",
shard_fetch_rate: int = 1,
checkpointer: Optional[CheckPointer] = None,
processor: Optional[Processor] = None,
retry_limit: Optional[int] = None,
expo_backoff: Optional[float] = None,
expo_backoff_limit: int = 120,
skip_describe_stream: bool = False,
use_list_shards: bool = False,
create_stream: bool = False,
create_stream_shards: int = 1,
describe_timeout: int = 60,
idle_timeout: float = 2.0,
timestamp: Optional[datetime] = None,
checkpoint_interval: Optional[float] = None,
) -> None:
super(Consumer, self).__init__(
stream_name,
session=session,
endpoint_url=endpoint_url,
region_name=region_name,
retry_limit=retry_limit,
expo_backoff=expo_backoff,
expo_backoff_limit=expo_backoff_limit,
skip_describe_stream=skip_describe_stream,
use_list_shards=use_list_shards,
create_stream=create_stream,
create_stream_shards=create_stream_shards,
describe_timeout=describe_timeout,
)
self.queue = asyncio.Queue(maxsize=max_queue_size)
self.sleep_time_no_records = sleep_time_no_records
self.idle_timeout = idle_timeout
self.max_shard_consumers = max_shard_consumers
self.record_limit = record_limit
self.is_fetching = True
self.checkpointer = checkpointer if checkpointer else MemoryCheckPointer()
self.processor = processor if processor else JsonProcessor()
self.iterator_type = iterator_type
self.fetch_task = None
self.shard_fetch_rate = shard_fetch_rate
self.timestamp = timestamp
self._ready = asyncio.Event()
# Deferred checkpoints awaiting implicit ack (per-shard)
self._deferred_checkpoints: Dict[str, str] = {} # shard_id → sequence
# Checkpoint interval state
if checkpoint_interval is not None and checkpoint_interval <= 0:
raise ValueError(f"checkpoint_interval must be positive, got {checkpoint_interval}")
self.checkpoint_interval = checkpoint_interval
self._pending_checkpoints: Dict[str, str] = {} # shard_id → sequence
self._checkpoint_flusher_task: Optional[asyncio.Task] = None
# Validate mutual exclusion: checkpoint_interval + auto_checkpoint=False
if checkpoint_interval is not None and checkpointer is not None:
if getattr(checkpointer, "auto_checkpoint", True) is False:
raise ValueError(
"checkpoint_interval and auto_checkpoint=False are mutually exclusive. "
"Consumer-level debounce calls checkpointer.checkpoint() which silently "
"buffers under auto_checkpoint=False, achieving nothing."
)
# Shard management
self._last_shard_refresh = 0
self._shard_refresh_interval = 60 # Refresh shards every 60 seconds
self._closed_shards = set() # Track shards that have been closed
self._shard_topology = {} # Track parent-child relationships
self._parent_shards = set() # Track shards that are parents
self._child_shards = set() # Track shards that are children
self._exhausted_parents = set() # Track parent shards that are fully consumed
def __aiter__(self) -> AsyncIterator[Any]:
return self
async def close(self):
log.debug("Closing Connection..")
self._ready.clear()
if not self.stream_status == self.RECONNECT:
await self.flush()
if self.fetch_task:
self.fetch_task.cancel()
self.fetch_task = None
# Flush deferred checkpoints (from __anext__ deferral)
if self._deferred_checkpoints and self.checkpointer:
for shard_id, sequence in self._deferred_checkpoints.items():
await self._maybe_checkpoint(shard_id, sequence)
self._deferred_checkpoints.clear()
# Cancel background flusher (triggers final flush via CancelledError)
if self._checkpoint_flusher_task is not None:
self._checkpoint_flusher_task.cancel()
try:
await self._checkpoint_flusher_task
except asyncio.CancelledError:
pass
self._checkpoint_flusher_task = None
# Final flush of any remaining interval-buffered checkpoints.
# Needed when the flusher task was cancelled before it started
# executing (CancelledError handler in _checkpoint_flusher never ran).
if self.checkpointer and self._pending_checkpoints:
await self._flush_pending_checkpoints()
if self.checkpointer:
await self.checkpointer.close()
if self.client is not None:
await self.client.close()
async def flush(self):
self.is_fetching = False
if not self.shards:
return
# Wait for shard fetches to finish
# todo: use gather
for shard in self.shards:
if shard.get("fetch"):
if not shard["fetch"].done():
await shard["fetch"]
async def _fetch(self):
error_count = 0
max_errors = 10
while self.is_fetching:
# Ensure fetch is performed at most 5 times per second (the limit per shard)
await asyncio.sleep(0.2)
try:
await self.fetch()
error_count = 0 # Reset error count on successful fetch
except asyncio.CancelledError:
log.debug("Fetch task cancelled")
self.is_fetching = False
break
except Exception as e:
log.exception(e)
error_count += 1
if error_count >= max_errors:
log.error(f"Too many fetch errors ({max_errors}), stopping fetch task")
self.is_fetching = False
break
await asyncio.sleep(min(2**error_count, 30)) # Exponential backoff
async def fetch(self):
if not self.is_fetching:
return
# Refresh shards to discover new ones and handle closed ones
await self.refresh_shards()
shards_in_use = [s for s in self.shards if self.checkpointer.is_allocated(s["ShardId"])]
# log.debug("shards in use: {}".format([s["ShardId"] for s in shards_in_use]))
for shard in self.shards:
if not self.is_fetching:
break
# Skip shards that are known to be closed
if shard["ShardId"] in self._closed_shards:
continue
if not self.checkpointer.is_allocated(shard["ShardId"]):
# Check parent-child ordering before allocation
if not self._should_allocate_shard(shard["ShardId"]):
log.debug(f"Skipping child shard {shard['ShardId']} - parent not exhausted")
continue
if self.max_shard_consumers and len(shards_in_use) >= self.max_shard_consumers:
continue
if self.checkpointer is None:
log.debug("Marking shard in use {}".format(shard["ShardId"]))
shard["ShardIterator"] = await self.get_shard_iterator(shard_id=shard["ShardId"])
else:
success, checkpoint = await self.checkpointer.allocate(shard["ShardId"])
if not success:
log.debug(
"Shard in use. Could not assign shard {} to checkpointer[{}]".format(
shard["ShardId"], self.checkpointer.get_ref()
)
)
continue
log.debug(
"Marking shard in use {} by checkpointer[{}] @ {}".format(
shard["ShardId"], self.checkpointer.get_ref(), checkpoint
)
)
shard["ShardIterator"] = await self.get_shard_iterator(
shard_id=shard["ShardId"], last_sequence_number=checkpoint
)
if "ShardIterator" in shard:
shard["stats"] = ShardStats()
shard["throttler"] = Throttler(rate_limit=self.shard_fetch_rate, period=1)
shards_in_use.append(shard)
log.debug("Shard count now at {}".format(len(shards_in_use)))
if shard.get("fetch"):
if shard["fetch"].done():
result = shard["fetch"].result()
if not result:
shard["fetch"] = None
continue
records = result["Records"]
if records:
log.debug("Shard {} got {} records".format(shard["ShardId"], len(records)))
total_items = 0
last_enqueued_sequence = None
for row in result["Records"]:
item_count = 0
dropped = False
for output in self.processor.parse(row["Data"]):
try:
await asyncio.wait_for(self.queue.put(output), timeout=30.0)
item_count += 1
except asyncio.TimeoutError:
log.warning("Queue put timed out, skipping record")
dropped = True
break
if item_count > 0 and not dropped:
last_enqueued_sequence = row["SequenceNumber"]
total_items += item_count
# Get approx minutes behind..
last_arrival = records[-1].get("ApproximateArrivalTimestamp")
if last_arrival:
last_arrival = round(((datetime.now(timezone.utc) - last_arrival).total_seconds() / 60))
log.debug(
"Shard {} added {} items from {} records. Consumer is {}m behind".format(
shard["ShardId"],
total_items,
len(records),
last_arrival,
),
extra={"consumer_behind_m": last_arrival},
)
else:
# ApproximateArrivalTimestamp not available in kinesis-lite
log.debug(
"Shard {} added {} items from {} records".format(
shard["ShardId"], total_items, len(records)
)
)
# Add checkpoint record — only for last *enqueued* sequence.
# Skip sentinel for exhausted shards; their terminal checkpoint
# is flushed synchronously in the deallocation block below.
if last_enqueued_sequence:
shard["LastSequenceNumber"] = last_enqueued_sequence
if result.get("NextShardIterator"):
try:
await asyncio.wait_for(
self.queue.put(
{
"__CHECKPOINT__": {
"ShardId": shard["ShardId"],
"SequenceNumber": last_enqueued_sequence,
}
}
),
timeout=30.0,
)
except asyncio.TimeoutError:
log.warning("Checkpoint queue put timed out")
else:
last_enqueued_sequence = None
log.debug(
"Shard {} caught up, sleeping {}s".format(shard["ShardId"], self.sleep_time_no_records)
)
await asyncio.sleep(self.sleep_time_no_records)
if not result["NextShardIterator"]:
# Shard is closed - this is normal during resharding
shard_id = shard["ShardId"]
log.info(f"Shard {shard_id} is closed (NextShardIterator is null)")
self._closed_shards.add(shard_id)
# If this is a parent shard, mark it as exhausted to allow child consumption
if shard_id in self._parent_shards:
self._exhausted_parents.add(shard_id)
children = self._shard_topology.get(shard_id, {}).get("children", set())
if children:
log.info(f"Parent shard {shard_id} exhausted, enabling child shards: {children}")
# Flush all pending checkpoints for this shard before deallocating
if self.checkpointer:
# Flush deferred checkpoint if it's for this shard
if shard_id in self._deferred_checkpoints:
await self._maybe_checkpoint(
shard_id, self._deferred_checkpoints.pop(shard_id)
)
# Flush interval-buffered checkpoint for this shard
if shard_id in self._pending_checkpoints:
await self.checkpointer.checkpoint(
shard_id, self._pending_checkpoints.pop(shard_id)
)
# Note: the terminal batch's records are enqueued but no
# checkpoint sentinel was added (would race with deallocate).
# Records will be processed normally; the batch replays on
# restart (at-least-once safe, no data loss).
await self.checkpointer.deallocate(shard_id)
# Remove shard iterator to stop fetching from this shard
shard.pop("ShardIterator", None)
shard["fetch"] = None
continue
shard["ShardIterator"] = result["NextShardIterator"]
shard["fetch"] = None
else:
# log.debug("shard {} fetch in progress..".format(shard['ShardId']))
continue
if "ShardIterator" in shard and shard["ShardIterator"] is not None:
shard["fetch"] = asyncio.create_task(self.get_records(shard=shard))
# Signal readiness once all allocated shards have iterators
# Guard with is_fetching to prevent close() race where _ready is cleared
# but a concurrent fetch() iteration re-sets it before cancellation propagates
if self.is_fetching and not self._ready.is_set():
allocated_shards = [s for s in self.shards if self.checkpointer.is_allocated(s["ShardId"])]
if allocated_shards and all(s.get("ShardIterator") is not None for s in allocated_shards):
log.info("Consumer ready: all %d allocated shards have iterators", len(allocated_shards))
self._ready.set()
async def get_records(self, shard):
# Note: "This operation has a limit of five transactions per second per account."
async with shard["throttler"]:
# log.debug("get_records shard={}".format(shard['ShardId']))
try:
result = await self.client.get_records(ShardIterator=shard["ShardIterator"], Limit=self.record_limit)
shard["stats"].succeded()
return result
except ClientConnectionError:
await self.get_conn()
except TimeoutError as e:
log.warning("Timeout {}. sleeping..".format(e))
await asyncio.sleep(3)
except ClientError as e:
code = e.response["Error"]["Code"]
if code == "ProvisionedThroughputExceededException":
log.warning("{} hit ProvisionedThroughputExceededException".format(shard["ShardId"]))
shard["stats"].throttled()
# todo: control the throttle ?
await asyncio.sleep(0.25)
elif code == "ExpiredIteratorException":
log.warning("{} hit ExpiredIteratorException, recreating iterator".format(shard["ShardId"]))
try:
# Try to get a new iterator from the last known sequence number
shard["ShardIterator"] = await self.get_shard_iterator(
shard_id=shard["ShardId"],
last_sequence_number=shard.get("LastSequenceNumber"),
)
log.debug(f"Successfully recreated iterator for shard {shard['ShardId']}")
except ClientError as iterator_error:
iterator_code = iterator_error.response["Error"]["Code"]
if iterator_code == "ResourceNotFoundException":
log.warning(f"Shard {shard['ShardId']} no longer exists, marking as closed")
self._closed_shards.add(shard["ShardId"])
if self.checkpointer:
await self.checkpointer.deallocate(shard["ShardId"])
shard.pop("ShardIterator", None)
else:
log.error(f"Failed to recreate iterator for shard {shard['ShardId']}: {iterator_code}")
# For other errors, remove the iterator to avoid infinite loops
shard.pop("ShardIterator", None)
except Exception as iterator_error:
log.error(
f"Unexpected error recreating iterator for shard {shard['ShardId']}: {iterator_error}"
)
shard.pop("ShardIterator", None)
elif code == "InternalFailure":
log.warning("Received InternalFailure from Kinesis, rebuilding connection.. ")
await self.get_conn()
else:
log.warning("ClientError {}. sleeping..".format(code))
await asyncio.sleep(3)
except Exception as e:
log.warning("Unknown error {}. sleeping..".format(e))
await asyncio.sleep(3)
# Connection or other issue
return None
async def refresh_shards(self):
"""
Refresh the shard list to discover new shards and handle closed ones.
This is important for handling Kinesis stream resharding events.
"""
import time
current_time = time.time()
if current_time - self._last_shard_refresh < self._shard_refresh_interval:
return # Too soon to refresh
# Skip shard refresh if skip_describe_stream is enabled
if self.skip_describe_stream:
log.debug("Skipping shard refresh due to skip_describe_stream setting")
self._last_shard_refresh = current_time
return
try:
log.debug("Refreshing shard list to check for new/closed shards")
if self.use_list_shards:
# Use ListShards API for better rate limits
log.debug("Using ListShards API for shard refresh")
try:
new_shards = await self.list_shards()
stream_status = self.ACTIVE # Assume active if ListShards succeeds
except Exception as e:
log.warning(f"ListShards failed ({e}), falling back to DescribeStream for refresh")
self.use_list_shards = False
if not self.use_list_shards:
# Use DescribeStream API
stream_info = await self.get_stream_description()
stream_status = stream_info.get("StreamStatus", "UNKNOWN")
# Handle stream that might be updating due to resharding
if stream_status == self.UPDATING:
log.info("Stream is currently UPDATING (possibly due to resharding)")
# Don't refresh shards during updating status to avoid inconsistent state
return
elif stream_status != self.ACTIVE:
log.warning(f"Stream is in unexpected status: {stream_status}")
return
new_shards = stream_info["Shards"]
# Get current shard IDs
current_shard_ids = {s["ShardId"] for s in self.shards} if self.shards else set()
new_shard_ids = {s["ShardId"] for s in new_shards}
# Build shard topology map for parent-child relationships
self._build_shard_topology(new_shards)
# Find newly discovered shards
discovered_shards = new_shard_ids - current_shard_ids
if discovered_shards:
log.info(f"Discovered new shards: {discovered_shards}")
# Check if this might be a resharding event
new_child_shards = discovered_shards & self._child_shards
if new_child_shards:
log.info(f"Resharding detected: new child shards {new_child_shards}")
# Find shards that no longer exist (though this is rare)
missing_shards = current_shard_ids - new_shard_ids
if missing_shards:
log.info(f"Shards no longer in stream description: {missing_shards}")
# Update the shards list
# Preserve existing shard state (iterators, stats, etc.) for shards that still exist
preserved_shards = {}
if self.shards:
for shard in self.shards:
if shard["ShardId"] in new_shard_ids:
preserved_shards[shard["ShardId"]] = shard
# Build new shards list, preserving state where possible
updated_shards = []
for new_shard in new_shards:
shard_id = new_shard["ShardId"]
if shard_id in preserved_shards:
# Keep existing shard with its state
existing_shard = preserved_shards[shard_id]
# Update shard metadata from AWS
for key in ["SequenceNumberRange", "ParentShardId", "HashKeyRange"]:
if key in new_shard:
existing_shard[key] = new_shard[key]
updated_shards.append(existing_shard)
else:
# New shard discovered
updated_shards.append(new_shard.copy())
self.shards = updated_shards
self._last_shard_refresh = current_time
log.debug(f"Shard refresh complete. Total shards: {len(self.shards)}")
except Exception as e:
log.warning(f"Failed to refresh shards: {e}")
def is_resharding_likely_in_progress(self) -> bool:
"""
Detect if resharding is likely in progress based on shard topology.
This helps consumers make informed decisions during potential resharding.
"""
# If we have parent-child relationships, resharding has occurred
if self._parent_shards and self._child_shards:
# Check if we have active parents with children (mid-resharding)
active_parents_with_children = 0
for parent_id in self._parent_shards:
if parent_id not in self._closed_shards and parent_id not in self._exhausted_parents:
if parent_id in self._shard_topology and self._shard_topology[parent_id].get("children"):
active_parents_with_children += 1
if active_parents_with_children > 0:
return True
# If we have many closed shards relative to active ones, might be post-resharding
# But only flag as "in progress" if we don't have clear parent-child completion
if len(self._closed_shards) > 0 and self.shards:
closed_ratio = len(self._closed_shards) / len(self.shards)
if closed_ratio > 0.3: # More than 30% of shards are closed
# Only consider it "in progress" if we don't have a clear completed resharding pattern
# (i.e., all parents are exhausted and we have active children)
if self._parent_shards and self._child_shards:
# If all parents are exhausted, resharding is complete, not in progress
all_parents_exhausted = all(p in self._exhausted_parents for p in self._parent_shards)
if all_parents_exhausted:
return False
return True
return False
def _build_shard_topology(self, shards):
"""
Build parent-child relationship topology from shard metadata.
Follows AWS best practice: consume parent shards before child shards.
"""
self._shard_topology.clear()
self._parent_shards.clear()
self._child_shards.clear()
# First pass: identify all parent shards
all_shard_ids = {s["ShardId"] for s in shards}
parent_shard_ids = set()
for shard in shards:
shard_id = shard["ShardId"]
parent_shard_id = shard.get("ParentShardId")
if parent_shard_id:
# This is a child shard
self._child_shards.add(shard_id)
parent_shard_ids.add(parent_shard_id)
# Build parent -> children mapping
if parent_shard_id not in self._shard_topology:
self._shard_topology[parent_shard_id] = {
"children": set(),
"parent": None,
}
self._shard_topology[parent_shard_id]["children"].add(shard_id)
# Build child -> parent mapping
if shard_id not in self._shard_topology:
self._shard_topology[shard_id] = {"children": set(), "parent": None}
self._shard_topology[shard_id]["parent"] = parent_shard_id
# Parent shards are those that have children OR are referenced as parents
# but might not be in the current shard list (if they're already closed)
self._parent_shards = parent_shard_ids & all_shard_ids
log.debug(f"Shard topology: {len(self._parent_shards)} parents, {len(self._child_shards)} children")
if self._parent_shards:
log.debug(f"Parent shards: {self._parent_shards}")
if self._child_shards:
log.debug(f"Child shards: {self._child_shards}")
def _should_allocate_shard(self, shard_id):
"""
Determine if a shard should be allocated based on parent-child ordering rules.
AWS Best Practice: Only allocate child shards after their parents are exhausted.
"""
# Always allow parent shards
if shard_id in self._parent_shards:
return True
# For child shards, check if parent is exhausted
if shard_id in self._child_shards:
parent_id = self._shard_topology.get(shard_id, {}).get("parent")
if parent_id:
# Parent must be exhausted (closed) before we can consume child
return parent_id in self._exhausted_parents or parent_id in self._closed_shards
# If not in any topology (independent shard), always allow
return True
async def get_shard_iterator(self, shard_id, last_sequence_number=None):
log.debug(
"getting shard iterator for {} @ {}".format(
shard_id,
last_sequence_number if last_sequence_number else self.iterator_type,
)
)
params = {
"ShardId": shard_id,
"ShardIteratorType": ("AFTER_SEQUENCE_NUMBER" if last_sequence_number else self.iterator_type),
}
params.update(self.address)
if last_sequence_number:
params["StartingSequenceNumber"] = last_sequence_number
if self.iterator_type == "AT_TIMESTAMP" and self.timestamp:
params["Timestamp"] = self.timestamp
response = await self.client.get_shard_iterator(**params)
return response["ShardIterator"]
def get_shard_status(self):
"""
Get current status of all shards being consumed.
Returns information about active, closed, and allocated shards.
"""
if not self.shards:
return {
"total_shards": 0,
"active_shards": 0,
"closed_shards": 0,
"allocated_shards": 0,
"shard_details": [],
}
# Generate comprehensive shard details list
shard_details = [
{
"shard_id": shard["ShardId"],
"is_allocated": self.checkpointer.is_allocated(shard["ShardId"]),
"is_closed": shard["ShardId"] in self._closed_shards,
"has_iterator": "ShardIterator" in shard and shard["ShardIterator"] is not None,
"sequence_range": shard.get("SequenceNumberRange", {}),
"parent_shard_id": shard.get("ParentShardId"),
"is_parent": shard["ShardId"] in self._parent_shards,
"is_child": shard["ShardId"] in self._child_shards,
"can_allocate": self._should_allocate_shard(shard["ShardId"]),
"stats": (shard.get("stats").to_data() if shard.get("stats") else None),
}
for shard in self.shards
]
# Calculate counts from shard_details
active_shards_count = len([s for s in shard_details if not s["is_closed"]])
allocated_shards_count = len([s for s in shard_details if s["is_allocated"]])
return {
"total_shards": len(self.shards),
"active_shards": active_shards_count,
"closed_shards": len(self._closed_shards),
"allocated_shards": allocated_shards_count,
"parent_shards": len(self._parent_shards),
"child_shards": len(self._child_shards),
"exhausted_parents": len(self._exhausted_parents),
"resharding_in_progress": self.is_resharding_likely_in_progress(),
"topology": {
"parent_child_map": {k: list(v["children"]) for k, v in self._shard_topology.items() if v["children"]},
"child_parent_map": {k: v["parent"] for k, v in self._shard_topology.items() if v["parent"]},
},
"shard_details": shard_details,
}
def _start_fetch_task(self):
self.fetch_task = asyncio.create_task(self._fetch())
async def wait_ready(self, timeout=30):
"""Wait until consumer has obtained shard iterators and is ready to receive records.
Useful with ``iterator_type="LATEST"`` where callers need to know when it's safe
to produce events. Starts the fetch task if not already running (e.g. when called
before the first ``async for`` iteration).
Args:
timeout: Maximum seconds to wait. Default 30s accounts for AWS API
retries and exponential backoff on get_shard_iterator().
Raises:
ValueError: If consumer was created with ``skip_describe_stream=True``.
asyncio.TimeoutError: If consumer doesn't become ready within timeout.
asyncio.CancelledError: If the fetch task is cancelled before consumer becomes ready.
BaseException: If the fetch task dies before consumer becomes ready,
its exception is re-raised directly.
RuntimeError: If the fetch task exits cleanly before consumer becomes ready.
"""
if self.skip_describe_stream:
raise ValueError("wait_ready() is not supported with skip_describe_stream=True")
loop = asyncio.get_running_loop()
deadline = loop.time() + timeout
# Ensure lifecycle is initialized (get_conn calls start() which describes the stream)
if self.shards is None:
try:
await asyncio.wait_for(self.get_conn(), timeout=timeout)
except asyncio.TimeoutError:
raise asyncio.TimeoutError(
f"Consumer did not become ready within {timeout}s (timed out connecting to stream)"
) from None
if not self.fetch_task:
self._start_fetch_task()
if self._ready.is_set():
return
remaining = deadline - loop.time()
if remaining <= 0:
raise asyncio.TimeoutError(f"Consumer did not become ready within {timeout}s (no time left after connect)")
# Race the ready signal against the fetch task dying
ready_fut = asyncio.ensure_future(self._ready.wait())
try:
done, pending = await asyncio.wait(
[ready_fut, self.fetch_task],
timeout=remaining,
return_when=asyncio.FIRST_COMPLETED,
)
except BaseException:
ready_fut.cancel()
raise
# Clean up if ready_fut didn't win (never cancel fetch_task)
if ready_fut in pending:
ready_fut.cancel()
if ready_fut in done:
return # Consumer is ready
if self.fetch_task in done:
# Fetch task died or exited before we became ready
if self.fetch_task.cancelled():
raise asyncio.CancelledError("Fetch task was cancelled before consumer became ready")
exc = self.fetch_task.exception()
if exc:
raise exc
raise RuntimeError("Fetch task exited before consumer became ready")
# Neither completed — timeout
raise asyncio.TimeoutError(f"Consumer did not become ready within {timeout}s")
@property
def is_ready(self) -> bool:
"""Non-blocking check whether consumer has obtained shard iterators."""
return self._ready.is_set()
async def _maybe_checkpoint(self, shard_id: str, sequence: str):
"""Commit a checkpoint, either immediately or via interval buffer."""
if self.checkpoint_interval is None:
# No debouncing — checkpoint immediately
await self.checkpointer.checkpoint(shard_id, sequence)
return
# Buffer for background flush
self._pending_checkpoints[shard_id] = sequence
# Start flusher on first use
if self._checkpoint_flusher_task is None:
self._checkpoint_flusher_task = asyncio.ensure_future(self._checkpoint_flusher())
async def _checkpoint_flusher(self):
"""Background task that flushes buffered checkpoints periodically."""
try:
while True:
await asyncio.sleep(self.checkpoint_interval)
try:
await self._flush_pending_checkpoints()
except Exception:
log.exception("Error flushing checkpoints, will retry next interval")
except asyncio.CancelledError:
# Final flush on shutdown — propagate errors so close() knows
await self._flush_pending_checkpoints()
async def _flush_pending_checkpoints(self):
"""Flush all interval-buffered checkpoints to the backend.
Entries are removed individually on success so that a failure
mid-loop preserves the remaining (unflushed) checkpoints for
the next attempt.
"""
for shard_id in list(self._pending_checkpoints):
await self.checkpointer.checkpoint(shard_id, self._pending_checkpoints[shard_id])
del self._pending_checkpoints[shard_id]
async def __anext__(self):
if not self.shards:
await self.get_conn()
if not self.fetch_task:
self._start_fetch_task()
# Raise exception from Fetch Task to main task otherwise raise exception inside
# Fetch Task will fail silently
if self.fetch_task.done():
exception = self.fetch_task.exception()
if exception:
raise exception
# 1. Commit deferred checkpoints from previous iteration (implicit ACK)
if self._deferred_checkpoints:
for shard_id, sequence in self._deferred_checkpoints.items():
await self._maybe_checkpoint(shard_id, sequence)
self._deferred_checkpoints.clear()
checkpoint_count = 0
max_checkpoints = 100 # Prevent infinite checkpoint processing
# 2. Get items from queue, deferring any checkpoint sentinels
while True:
try:
item = await asyncio.wait_for(self.queue.get(), timeout=self.idle_timeout)
except asyncio.TimeoutError:
log.debug(f"Queue idle for {self.idle_timeout}s, stopping iteration")
raise StopAsyncIteration from None
if item and isinstance(item, dict) and "__CHECKPOINT__" in item:
if self.checkpointer:
# Don't execute now — defer to next __anext__ call
self._deferred_checkpoints[item["__CHECKPOINT__"]["ShardId"]] = item[
"__CHECKPOINT__"
]["SequenceNumber"]
checkpoint_count += 1
if checkpoint_count >= max_checkpoints:
log.warning(f"Processed {max_checkpoints} checkpoints, stopping iteration")
raise StopAsyncIteration
continue
return item