@@ -314,12 +314,17 @@ class ConsolidateChunks(beam.PTransform):
314314 target_chunks : Mapping [str , int ]
315315
316316 def _prepend_chunk_key (self , key , chunk ):
317+ core .inc_counter (self .__class__ , 'in-chunks' )
318+ core .inc_counter (self .__class__ , 'in-bytes' , chunk .nbytes )
317319 rounded_key = _round_chunk_key (key , self .target_chunks )
318320 return rounded_key , (key , chunk )
319321
320322 def _consolidate (self , key , inputs ):
321- ((consolidated_key , dataset ),) = consolidate_chunks (inputs )
323+ with core .inc_timer_msec (self .__class__ , 'consolidate-msec' ):
324+ ((consolidated_key , dataset ),) = consolidate_chunks (inputs )
322325 assert key == consolidated_key , (key , consolidated_key )
326+ core .inc_counter (self .__class__ , 'out-chunks' )
327+ core .inc_counter (self .__class__ , 'out-bytes' , dataset .nbytes )
323328 return consolidated_key , dataset
324329
325330 def expand (self , pcoll ):
@@ -339,17 +344,22 @@ class ConsolidateVariables(beam.PTransform):
339344 # of variables.
340345
341346 def _prepend_chunk_key (self , key , chunk ):
347+ core .inc_counter (self .__class__ , 'in-chunks' )
348+ core .inc_counter (self .__class__ , 'in-bytes' , chunk .nbytes )
342349 return key .replace (vars = None ), (key , chunk )
343350
344351 def _consolidate (self , key , inputs ):
345- ((consolidated_key , dataset ),) = consolidate_variables (inputs )
352+ with core .inc_timer_msec (self .__class__ , 'consolidate-msec' ):
353+ ((consolidated_key , dataset ),) = consolidate_variables (inputs )
346354 assert key .offsets == consolidated_key .offsets , (key , consolidated_key )
347355 assert key .vars is None
348356 # TODO(shoyer): consider carefully whether it is better to return key or
349357 # consolidated_key. They are both valid in the xarray-beam data model -- the
350358 # difference is whether vars=None or is an explicit set of variables.
351359 # For now, conservatively return the version of key with vars=None so
352360 # users don't rely on it.
361+ core .inc_counter (self .__class__ , 'out-chunks' )
362+ core .inc_counter (self .__class__ , 'out-bytes' , dataset .nbytes )
353363 return key , dataset
354364
355365 def expand (self , pcoll ):
@@ -432,7 +442,13 @@ def _split_chunks(
432442 target_chunks = {
433443 k : v for k , v in self .target_chunks .items () if k in dataset .dims
434444 }
435- yield from split_chunks (key , dataset , target_chunks )
445+ core .inc_counter (self .__class__ , 'in-chunks' )
446+ core .inc_counter (self .__class__ , 'in-bytes' , dataset .nbytes )
447+ with core .inc_timer_msec (self .__class__ , 'split-msec' ):
448+ for new_key , new_dataset in split_chunks (key , dataset , target_chunks ):
449+ yield new_key , new_dataset
450+ core .inc_counter (self .__class__ , 'out-chunks' )
451+ core .inc_counter (self .__class__ , 'out-bytes' , new_dataset .nbytes )
436452
437453 def expand (self , pcoll ):
438454 return pcoll | beam .FlatMapTuple (self ._split_chunks )
@@ -458,8 +474,19 @@ def split_variables(
458474class SplitVariables (beam .PTransform ):
459475 """Split existing chunks into a separate chunk per data variable."""
460476
477+ def _split_variables (
478+ self , key : core .Key , dataset : xarray .Dataset
479+ ) -> Iterator [tuple [core .Key , xarray .Dataset ]]:
480+ core .inc_counter (self .__class__ , 'in-chunks' )
481+ core .inc_counter (self .__class__ , 'in-bytes' , dataset .nbytes )
482+ with core .inc_timer_msec (self .__class__ , 'split-msec' ):
483+ for new_key , new_dataset in split_variables (key , dataset ):
484+ yield new_key , new_dataset
485+ core .inc_counter (self .__class__ , 'out-chunks' )
486+ core .inc_counter (self .__class__ , 'out-bytes' , new_dataset .nbytes )
487+
461488 def expand (self , pcoll ):
462- return pcoll | beam .FlatMapTuple (split_variables )
489+ return pcoll | beam .FlatMapTuple (self . _split_variables )
463490
464491
465492@core .export
0 commit comments