@@ -508,10 +508,12 @@ def client_table(self):
508508            if  message  is  None :
509509                return  []
510510
511-             node_info  =  [] 
511+             node_info  =  {} 
512512            gcs_entry  =  ray .gcs_utils .GcsTableEntry .GetRootAsGcsTableEntry (
513513                message , 0 )
514514
515+             # Since GCS entries are append-only, we override so that 
516+             # only the latest entries are kept. 
515517            for  i  in  range (gcs_entry .EntriesLength ()):
516518                client  =  (
517519                    ray .gcs_utils .ClientTableData .GetRootAsClientTableData (
@@ -522,8 +524,18 @@ def client_table(self):
522524                    client .ResourcesTotalCapacity (i )
523525                    for  i  in  range (client .ResourcesTotalLabelLength ())
524526                }
525-                 node_info .append ({
526-                     "ClientID" : ray .utils .binary_to_hex (client .ClientId ()),
527+                 client_id  =  ray .utils .binary_to_hex (client .ClientId ())
528+ 
529+                 # If this client is being removed, then it must 
530+                 # have previously been inserted, and 
531+                 # it cannot have previously been removed. 
532+                 if  not  client .IsInsertion ():
533+                     assert  client_id  in  node_info , "Client removed not found!" 
534+                     assert  node_info [client_id ]["IsInsertion" ], (
535+                         "Unexpected duplicate removal of client." )
536+ 
537+                 node_info [client_id ] =  {
538+                     "ClientID" : client_id ,
527539                    "IsInsertion" : client .IsInsertion (),
528540                    "NodeManagerAddress" : decode (client .NodeManagerAddress ()),
529541                    "NodeManagerPort" : client .NodeManagerPort (),
@@ -532,8 +544,8 @@ def client_table(self):
532544                        client .ObjectStoreSocketName ()),
533545                    "RayletSocketName" : decode (client .RayletSocketName ()),
534546                    "Resources" : resources 
535-                 }) 
536-             return  node_info 
547+                 }
548+             return  list ( node_info . values ()) 
537549
538550    def  log_files (self ):
539551        """Fetch and return a dictionary of log file names to outputs. 
0 commit comments