async def _slurp_changelogs(self) -> None: changelog_queue = self.tables.changelog_queue tp_to_table = self.tp_to_table active_tps = self.active_tps standby_tps = self.standby_tps active_offsets = self.active_offsets standby_offsets = self.standby_offsets buffers = self.buffers buffer_sizes = self.buffer_sizes while not self.should_stop: event: EventT = await changelog_queue.get() message = event.message tp = message.tp offset = message.offset offsets: Counter[TP] bufsize = buffer_sizes.get(tp) if tp in active_tps: table = tp_to_table[tp] offsets = active_offsets if bufsize is None: bufsize = buffer_sizes[tp] = table.recovery_buffer_size elif tp in standby_tps: table = tp_to_table[tp] offsets = standby_offsets if bufsize is None: bufsize = buffer_sizes[tp] = table.standby_buffer_size else: continue seen_offset = offsets.get(tp, None) if seen_offset is None or offset > seen_offset: offsets[tp] = offset buf = buffers[table] buf.append(event) await table.on_changelog_event(event) if len(buf) >= bufsize: table.apply_changelog_batch(buf) buf.clear() if self.in_recovery and not self.active_remaining_total(): # apply anything stuck in the buffers self.flush_buffers() self.in_recovery = False if self._actives_span is not None: self._actives_span.set_tag('Actives-Ready', True) self.signal_recovery_end.set() if self.standbys_pending and not self.standby_remaining_total(): if self._standbys_span: finish_span(self._standbys_span) self._standbys_span = None self.tables.on_standbys_ready()
async def _restart_recovery(self) -> None: consumer = self.app.consumer active_tps = self.active_tps standby_tps = self.standby_tps standby_offsets = self.standby_offsets standby_highwaters = self.standby_highwaters assigned_active_tps = self.active_tps assigned_standby_tps = self.standby_tps active_offsets = self.active_offsets standby_offsets = self.standby_offsets active_highwaters = self.active_highwaters while not self.should_stop: self.log.dev('WAITING FOR NEXT RECOVERY TO START') self.signal_recovery_reset.clear() self.in_recovery = False if await self.wait_for_stopped(self.signal_recovery_start): self.signal_recovery_start.clear() break # service was stopped self.signal_recovery_start.clear() span: Any = None spans: list = [] tracer: Optional[opentracing.Tracer] = None if self.app.tracer: tracer = self.app.tracer.get_tracer('_faust') if tracer is not None and self._recovery_span: span = tracer.start_span('recovery-thread', child_of=self._recovery_span) self.app._span_add_default_tags(span) spans.extend([span, self._recovery_span]) T = traced_from_parent_span(span) try: await self._wait(T(asyncio.sleep)(self.recovery_delay)) if not self.tables: # If there are no tables -- simply resume streams await T(self._resume_streams)() for _span in spans: finish_span(_span) continue self.in_recovery = True self.standbys_pending = True # Must flush any buffers before starting rebalance. T(self.flush_buffers)() producer = cast(_App, self.app)._producer if producer is not None: await self._wait(T(producer.flush)()) self.log.dev('Build highwaters for active partitions') await self._wait( T(self._build_highwaters)(consumer, assigned_active_tps, active_highwaters, 'active')) self.log.dev('Build offsets for active partitions') await self._wait( T(self._build_offsets)(consumer, assigned_active_tps, active_offsets, 'active')) for tp in assigned_active_tps: if active_offsets[tp] > active_highwaters[tp]: raise ConsistencyError( E_PERSISTED_OFFSET.format( tp, active_offsets[tp], active_highwaters[tp], ), ) self.log.dev('Build offsets for standby partitions') await self._wait( T(self._build_offsets)(consumer, assigned_standby_tps, standby_offsets, 'standby')) self.log.dev('Seek offsets for active partitions') await self._wait( T(self._seek_offsets)(consumer, assigned_active_tps, active_offsets, 'active')) if self.need_recovery(): self.log.info('Restoring state from changelog topics...') T(consumer.resume_partitions)(active_tps) # Resume partitions and start fetching. self.log.info('Resuming flow...') T(consumer.resume_flow)() await T(cast(_App, self.app)._fetcher.maybe_start)() T(self.app.flow_control.resume)() # Wait for actives to be up to date. # This signal will be set by _slurp_changelogs if tracer is not None and span: self._actives_span = tracer.start_span( 'recovery-actives', child_of=span, tags={'Active-Stats': self.active_stats()}, ) self.app._span_add_default_tags(span) try: self.signal_recovery_end.clear() await self._wait(self.signal_recovery_end) except Exception as exc: finish_span(self._actives_span, error=exc) else: finish_span(self._actives_span) finally: self._actives_span = None # recovery done. self.log.info('Done reading from changelog topics') T(consumer.pause_partitions)(active_tps) else: self.log.info('Resuming flow...') T(consumer.resume_flow)() T(self.app.flow_control.resume)() self.log.info('Recovery complete') if span: span.set_tag('Recovery-Completed', True) self.in_recovery = False if standby_tps: self.log.info('Starting standby partitions...') self.log.dev('Seek standby offsets') await self._wait( T(self._seek_offsets)(consumer, standby_tps, standby_offsets, 'standby')) self.log.dev('Build standby highwaters') await self._wait( T(self._build_highwaters)( consumer, standby_tps, standby_highwaters, 'standby', ), ) for tp in standby_tps: if standby_offsets[tp] > standby_highwaters[tp]: raise ConsistencyError( E_PERSISTED_OFFSET.format( tp, standby_offsets[tp], standby_highwaters[tp], ), ) if tracer is not None and span: self._standbys_span = tracer.start_span( 'recovery-standbys', child_of=span, tags={'Standby-Stats': self.standby_stats()}, ) self.app._span_add_default_tags(span) self.log.dev('Resume standby partitions') T(consumer.resume_partitions)(standby_tps) # Pause all our topic partitions, # to make sure we don't fetch any more records from them. await self._wait(asyncio.sleep(0.1)) # still needed? await self._wait(T(self.on_recovery_completed)()) except RebalanceAgain as exc: self.log.dev('RAISED REBALANCE AGAIN') for _span in spans: finish_span(_span, error=exc) continue # another rebalance started except ServiceStopped as exc: self.log.dev('RAISED SERVICE STOPPED') for _span in spans: finish_span(_span, error=exc) break # service was stopped except Exception as exc: for _span in spans: finish_span(_span, error=exc) raise else: for _span in spans: finish_span(_span) # restart - wait for next rebalance. self.in_recovery = False
async def _restart_recovery(self) -> None: consumer = self.app.consumer active_tps = self.active_tps standby_tps = self.standby_tps standby_offsets = self.standby_offsets standby_highwaters = self.standby_highwaters assigned_active_tps = self.active_tps assigned_standby_tps = self.standby_tps active_offsets = self.active_offsets standby_offsets = self.standby_offsets active_highwaters = self.active_highwaters while not self.should_stop: self.log.dev("WAITING FOR NEXT RECOVERY TO START") if await self.wait_for_stopped(self.signal_recovery_start): self.signal_recovery_start.clear() break # service was stopped self.signal_recovery_start.clear() generation_id = self._generation_id span: Any = None spans: list = [] tracer: Optional[opentracing.Tracer] = None if self.app.tracer: tracer = self.app.tracer.get_tracer("_faust") if tracer is not None and self._recovery_span: span = tracer.start_span("recovery-thread", child_of=self._recovery_span) self.app._span_add_default_tags(span) spans.extend([span, self._recovery_span]) T = traced_from_parent_span(span) try: await self._wait(T(asyncio.sleep)(self.recovery_delay)) if not self.tables or self.app.conf.store == URL("aerospike:"): # If there are no tables -- simply resume streams await T(self._resume_streams)(generation_id=generation_id) for _span in spans: finish_span(_span) continue self._set_recovery_started() self.standbys_pending = True # Must flush any buffers before starting rebalance. T(self.flush_buffers)() producer = cast(_App, self.app)._producer if producer is not None: await self._wait( T(producer.flush)(), timeout=self.app.conf.broker_request_timeout, ) self.log.dev("Build highwaters for active partitions") await self._wait( T(self._build_highwaters)(consumer, assigned_active_tps, active_highwaters, "active"), timeout=self.app.conf.broker_request_timeout, ) self.log.dev("Build offsets for active partitions") await self._wait( T(self._build_offsets)(consumer, assigned_active_tps, active_offsets, "active"), timeout=self.app.conf.broker_request_timeout, ) if self.app.conf.recovery_consistency_check: for tp in assigned_active_tps: if (active_offsets[tp] and active_highwaters[tp] and active_offsets[tp] > active_highwaters[tp]): raise ConsistencyError( E_PERSISTED_OFFSET.format( tp, active_offsets[tp], active_highwaters[tp], ), ) self.log.dev("Build offsets for standby partitions") await self._wait( T(self._build_offsets)(consumer, assigned_standby_tps, standby_offsets, "standby"), timeout=self.app.conf.broker_request_timeout, ) self.log.dev("Seek offsets for active partitions") await self._wait( T(self._seek_offsets)(consumer, assigned_active_tps, active_offsets, "active"), timeout=self.app.conf.broker_request_timeout, ) if self.signal_recovery_start.is_set(): logger.info("Restarting Recovery") continue if self.need_recovery(): self._set_recovery_started() self.standbys_pending = True self.log.info("Restoring state from changelog topics...") T(consumer.resume_partitions)(active_tps) # Resume partitions and start fetching. self.log.info("Resuming flow...") T(self.app.flow_control.resume)() T(consumer.resume_flow)() await T(cast(_App, self.app)._fetcher.maybe_start)() # Wait for actives to be up to date. # This signal will be set by _slurp_changelogs if tracer is not None and span: self._actives_span = tracer.start_span( "recovery-actives", child_of=span, tags={"Active-Stats": self.active_stats()}, ) self.app._span_add_default_tags(span) try: await self._wait(self.signal_recovery_end.wait()) except Exception as exc: finish_span(self._actives_span, error=exc) else: finish_span(self._actives_span) finally: self._actives_span = None # recovery done. self.log.info("Done reading from changelog topics") T(consumer.pause_partitions)(active_tps) else: self.log.info("Resuming flow...") T(self.app.flow_control.resume)() T(consumer.resume_flow)() self._set_recovery_ended() # The changelog partitions only in the active_tps set need to be resumed active_only_partitions = active_tps - standby_tps if active_only_partitions: # Support for the specific scenario where recovery_buffer=1 tps_resuming = [ tp for tp in active_only_partitions if self.tp_to_table[tp].recovery_buffer_size == 1 ] if tps_resuming: T(consumer.resume_partitions)(tps_resuming) T(self.app.flow_control.resume)() T(consumer.resume_flow)() self.log.info("Recovery complete") if span: span.set_tag("Recovery-Completed", True) if standby_tps: self.log.info("Starting standby partitions...") self.log.dev("Seek standby offsets") await self._wait( T(self._seek_offsets)(consumer, standby_tps, standby_offsets, "standby"), timeout=self.app.conf.broker_request_timeout, ) self.log.dev("Build standby highwaters") await self._wait( T(self._build_highwaters)( consumer, standby_tps, standby_highwaters, "standby", ), timeout=self.app.conf.broker_request_timeout, ) if self.app.conf.recovery_consistency_check: for tp in standby_tps: if (standby_offsets[tp] and standby_highwaters[tp] and standby_offsets[tp] > standby_highwaters[tp]): raise ConsistencyError( E_PERSISTED_OFFSET.format( tp, standby_offsets[tp], standby_highwaters[tp], ), ) if tracer is not None and span: self._standbys_span = tracer.start_span( "recovery-standbys", child_of=span, tags={"Standby-Stats": self.standby_stats()}, ) self.app._span_add_default_tags(span) self.log.dev("Resume standby partitions") T(consumer.resume_partitions)(standby_tps) T(self.app.flow_control.resume)() T(consumer.resume_flow)() # Pause all our topic partitions, # to make sure we don't fetch any more records from them. await self._wait(T(self.on_recovery_completed)(generation_id)) except RebalanceAgain as exc: self.log.dev("RAISED REBALANCE AGAIN") for _span in spans: finish_span(_span, error=exc) continue # another rebalance started except IllegalStateError as exc: self.log.dev("RAISED REBALANCE AGAIN") for _span in spans: finish_span(_span, error=exc) continue # another rebalance started except ServiceStopped as exc: self.log.dev("RAISED SERVICE STOPPED") for _span in spans: finish_span(_span, error=exc) break # service was stopped except Exception as exc: for _span in spans: finish_span(_span, error=exc) raise else: for _span in spans: finish_span(_span)
async def _slurp_changelogs(self) -> None: changelog_queue = self.tables.changelog_queue tp_to_table = self.tp_to_table active_tps = self.active_tps standby_tps = self.standby_tps active_offsets = self.active_offsets standby_offsets = self.standby_offsets active_events_received_at = self._active_events_received_at standby_events_received_at = self._standby_events_received_at buffers = self.buffers buffer_sizes = self.buffer_sizes processing_times = self._processing_times async def _maybe_signal_recovery_end(timeout=False, timeout_count=0) -> None: # lets wait at least 2 consecutive cycles for the queue to be # empty to avoid race conditions between # the aiokafka consumer position and draining of the queue if timeout and self.app.in_transaction and timeout_count > 1: await detect_aborted_tx() if self.in_recovery and not self.need_recovery(): # apply anything stuck in the buffers self.flush_buffers() self._set_recovery_ended() if self._actives_span is not None: self._actives_span.set_tag("Actives-Ready", True) logger.debug("Setting recovery end") self.signal_recovery_end.set() async def detect_aborted_tx(): highwaters = self.active_highwaters offsets = self.active_offsets for tp, highwater in highwaters.items(): if (highwater is not None and offsets[tp] is not None and offsets[tp] < highwater): if await self.app.consumer.position(tp) >= highwater: logger.info(f"Aborted tx until highwater for {tp}") offsets[tp] = highwater timeout_count = 0 while not self.should_stop: try: self.signal_recovery_end.clear() try: event: EventT = await asyncio.wait_for( changelog_queue.get(), timeout=5.0) except asyncio.TimeoutError: timeout_count += 1 if self.should_stop: return await _maybe_signal_recovery_end( timeout=True, timeout_count=timeout_count) continue now = monotonic() timeout_count = 0 message = event.message tp = message.tp offset = message.offset logger.debug(f"Recovery message topic {tp} offset {offset}") offsets: Counter[TP] bufsize = buffer_sizes.get(tp) is_active = False if tp in active_tps: is_active = True table = tp_to_table[tp] offsets = active_offsets if bufsize is None: bufsize = buffer_sizes[tp] = table.recovery_buffer_size active_events_received_at[tp] = now elif tp in standby_tps: table = tp_to_table[tp] offsets = standby_offsets if bufsize is None: bufsize = buffer_sizes[tp] = table.standby_buffer_size standby_events_received_at[tp] = now else: logger.warning( f"recovery unknown topic {tp} offset {offset}") seen_offset = offsets.get(tp, None) logger.debug( f"seen offset for {tp} is {seen_offset} message offset {offset}" ) if seen_offset is None or offset > seen_offset: offsets[tp] = offset buf = buffers[table] buf.append(event) await table.on_changelog_event(event) if len(buf) >= bufsize: table.apply_changelog_batch(buf) buf.clear() self._last_flush_at = now now_after = monotonic() if is_active: last_processed_at = self._last_active_event_processed_at if last_processed_at is not None: processing_times.append(now_after - last_processed_at) max_samples = self.num_samples_required_for_estimate if len(processing_times) > max_samples: processing_times.popleft() self._last_active_event_processed_at = now_after await _maybe_signal_recovery_end() standby_ready = (self._standbys_span is None and self.tables.standbys_ready) if not standby_ready and not self.standby_remaining_total(): logger.debug("Completed standby partition fetch") if self._standbys_span: finish_span(self._standbys_span) self._standbys_span = None self.tables.on_standbys_ready() except Exception as ex: logger.warning(f"Error in recovery {ex}")
async def _slurp_changelogs(self) -> None: changelog_queue = self.tables.changelog_queue tp_to_table = self.tp_to_table active_tps = self.active_tps standby_tps = self.standby_tps active_offsets = self.active_offsets standby_offsets = self.standby_offsets active_events_received_at = self._active_events_received_at standby_events_received_at = self._standby_events_received_at buffers = self.buffers buffer_sizes = self.buffer_sizes processing_times = self._processing_times def _maybe_signal_recovery_end() -> None: if not self.active_remaining_total(): # apply anything stuck in the buffers self.flush_buffers() self._set_recovery_ended() if self._actives_span is not None: self._actives_span.set_tag("Actives-Ready", True) logger.debug("Setting recovery end") self.signal_recovery_end.set() while not self.should_stop: self.signal_recovery_end.clear() try: event: EventT = await asyncio.wait_for(changelog_queue.get(), timeout=5.0) except asyncio.TimeoutError: if self.should_stop: return _maybe_signal_recovery_end() continue now = monotonic() message = event.message tp = message.tp offset = message.offset offsets: Counter[TP] bufsize = buffer_sizes.get(tp) is_active = False if tp in active_tps: is_active = True table = tp_to_table[tp] offsets = active_offsets if bufsize is None: bufsize = buffer_sizes[tp] = table.recovery_buffer_size active_events_received_at[tp] = now elif tp in standby_tps: table = tp_to_table[tp] offsets = standby_offsets if bufsize is None: bufsize = buffer_sizes[tp] = table.standby_buffer_size standby_events_received_at[tp] = now else: continue seen_offset = offsets.get(tp, None) if seen_offset is None or offset > seen_offset: offsets[tp] = offset buf = buffers[table] buf.append(event) await table.on_changelog_event(event) if len(buf) >= bufsize: table.apply_changelog_batch(buf) buf.clear() self._last_flush_at = now now_after = monotonic() if is_active: last_processed_at = self._last_active_event_processed_at if last_processed_at is not None: processing_times.append(now_after - last_processed_at) max_samples = self.num_samples_required_for_estimate if len(processing_times) > max_samples: processing_times.popleft() self._last_active_event_processed_at = now_after _maybe_signal_recovery_end() if not self.standby_remaining_total(): logger.debug("Completed standby partition fetch") if self._standbys_span: finish_span(self._standbys_span) self._standbys_span = None self.tables.on_standbys_ready()