@@ -432,13 +432,18 @@ impl EagerSnapshot {
432432 }
433433
434434 /// Update the snapshot to the given version
435+ ///
436+ /// This will return a true value if the [LogStore] was read from. This can be helpful for
437+ /// understanding whether the snapshot loaded data or not
435438 pub async fn update (
436439 & mut self ,
437440 log_store : & dyn LogStore ,
438441 target_version : Option < i64 > ,
439- ) -> DeltaResult < ( ) > {
442+ ) -> DeltaResult < bool > {
443+ // Whether or not data has been read by this function
444+ let mut read_data = false ;
440445 if Some ( self . version ( ) ) == target_version {
441- return Ok ( ( ) ) ;
446+ return Ok ( read_data ) ;
442447 }
443448
444449 let new_slice = self
@@ -447,7 +452,7 @@ impl EagerSnapshot {
447452 . await ?;
448453
449454 if new_slice. is_none ( ) {
450- return Ok ( ( ) ) ;
455+ return Ok ( read_data ) ;
451456 }
452457 let new_slice = new_slice. unwrap ( ) ;
453458
@@ -457,9 +462,15 @@ impl EagerSnapshot {
457462 . flat_map ( get_visitor)
458463 . collect :: < Vec < _ > > ( ) ;
459464
465+ // If files is `None` then this can exit early because the snapshot has intentionally been
466+ // loaded _without_ files
467+ if self . files . is_none ( ) {
468+ self . process_visitors ( visitors) ?;
469+ return Ok ( read_data) ;
470+ }
471+
460472 let mut schema_actions: HashSet < _ > =
461473 visitors. iter ( ) . flat_map ( |v| v. required_actions ( ) ) . collect ( ) ;
462- let require_files = self . files . is_some ( ) ;
463474 let files = std:: mem:: take ( & mut self . files ) ;
464475
465476 schema_actions. insert ( ActionType :: Add ) ;
@@ -479,23 +490,17 @@ impl EagerSnapshot {
479490 let log_stream = new_slice. commit_stream ( log_store, & read_schema, & self . snapshot . config ) ?;
480491
481492 let mapper = LogMapper :: try_new ( & self . snapshot , None ) ?;
493+ let files =
494+ ReplayStream :: try_new ( log_stream, checkpoint_stream, & self . snapshot , & mut visitors) ?
495+ . map ( |batch| batch. and_then ( |b| mapper. map_batch ( b) ) )
496+ . try_collect ( )
497+ . await ?;
482498
483- if require_files {
484- let files = ReplayStream :: try_new (
485- log_stream,
486- checkpoint_stream,
487- & self . snapshot ,
488- & mut visitors,
489- ) ?
490- . map ( |batch| batch. and_then ( |b| mapper. map_batch ( b) ) )
491- . try_collect ( )
492- . await ?;
493-
494- self . files = Some ( files) ;
495- }
499+ self . files = Some ( files) ;
500+ read_data = true ;
496501 self . process_visitors ( visitors) ?;
497502
498- Ok ( ( ) )
503+ Ok ( read_data )
499504 }
500505
501506 /// Get the underlying snapshot
@@ -550,7 +555,7 @@ impl EagerSnapshot {
550555 pub fn log_data ( & self ) -> LogDataHandler < ' _ > {
551556 static EMPTY : Vec < RecordBatch > = vec ! [ ] ;
552557 LogDataHandler :: new (
553- & self . files . as_ref ( ) . unwrap_or ( & EMPTY ) ,
558+ self . files . as_ref ( ) . unwrap_or ( & EMPTY ) ,
554559 self . metadata ( ) ,
555560 self . schema ( ) ,
556561 )
0 commit comments