@@ -673,19 +673,19 @@ async def test_clean(c, s, a, b):
673
673
@gen_cluster (client = True )
674
674
async def test_message_breakup (c , s , a , b ):
675
675
n = 100_000
676
- a .state .target_message_size = 10 * n
677
- b .state .target_message_size = 10 * n
676
+ a .state .transfer_message_target_bytes = 10 * n
677
+ b .state .transfer_message_target_bytes = 10 * n
678
678
xs = [
679
679
c .submit (mul , b"%d" % i , n , key = f"x{ i } " , workers = [a .address ]) for i in range (30 )
680
680
]
681
681
y = c .submit (lambda _ : None , xs , key = "y" , workers = [b .address ])
682
682
await y
683
683
684
- assert 2 <= len (b .incoming_transfer_log ) <= 20
685
- assert 2 <= len (a .outgoing_transfer_log ) <= 20
684
+ assert 2 <= len (b .transfer_incoming_log ) <= 20
685
+ assert 2 <= len (a .transfer_outgoing_log ) <= 20
686
686
687
- assert all (msg ["who" ] == b .address for msg in a .outgoing_transfer_log )
688
- assert all (msg ["who" ] == a .address for msg in a .incoming_transfer_log )
687
+ assert all (msg ["who" ] == b .address for msg in a .transfer_outgoing_log )
688
+ assert all (msg ["who" ] == a .address for msg in a .transfer_incoming_log )
689
689
690
690
691
691
@gen_cluster (client = True )
@@ -764,7 +764,7 @@ async def test_gather_many_small(c, s, a, *snd_workers, as_deps):
764
764
concurrent outgoing connections. If multiple small fetches from the same worker are
765
765
scheduled all at once, they will result in a single call to gather_dep.
766
766
"""
767
- a .state .total_out_connections = 2
767
+ a .state .transfer_incoming_count_limit = 2
768
768
futures = await c .scatter (
769
769
{f"x{ i } " : i for i in range (100 )},
770
770
workers = [w .address for w in snd_workers ],
@@ -779,7 +779,7 @@ async def test_gather_many_small(c, s, a, *snd_workers, as_deps):
779
779
while len (a .data ) < 100 :
780
780
await asyncio .sleep (0.01 )
781
781
782
- assert a .state .comm_nbytes == 0
782
+ assert a .state .transfer_incoming_bytes == 0
783
783
784
784
story = a .state .story ("request-dep" , "receive-dep" )
785
785
assert len (story ) == 40 # 1 request-dep + 1 receive-dep per sender worker
@@ -808,27 +808,31 @@ async def test_multiple_transfers(c, s, w1, w2, w3):
808
808
@pytest .mark .xfail (reason = "very high flakiness" )
809
809
@gen_cluster (client = True , nthreads = [("127.0.0.1" , 1 )] * 3 )
810
810
async def test_share_communication (c , s , w1 , w2 , w3 ):
811
- x = c .submit (mul , b"1" , int (w3 .target_message_size + 1 ), workers = w1 .address )
812
- y = c .submit (mul , b"2" , int (w3 .target_message_size + 1 ), workers = w2 .address )
811
+ x = c .submit (
812
+ mul , b"1" , int (w3 .transfer_message_target_bytes + 1 ), workers = w1 .address
813
+ )
814
+ y = c .submit (
815
+ mul , b"2" , int (w3 .transfer_message_target_bytes + 1 ), workers = w2 .address
816
+ )
813
817
await wait ([x , y ])
814
818
await c ._replicate ([x , y ], workers = [w1 .address , w2 .address ])
815
819
z = c .submit (add , x , y , workers = w3 .address )
816
820
await wait (z )
817
- assert len (w3 .incoming_transfer_log ) == 2
818
- assert w1 .outgoing_transfer_log
819
- assert w2 .outgoing_transfer_log
821
+ assert len (w3 .transfer_incoming_log ) == 2
822
+ assert w1 .transfer_outgoing_log
823
+ assert w2 .transfer_outgoing_log
820
824
821
825
822
826
@pytest .mark .xfail (reason = "very high flakiness" )
823
827
@gen_cluster (client = True )
824
828
async def test_dont_overlap_communications_to_same_worker (c , s , a , b ):
825
- x = c .submit (mul , b"1" , int (b .target_message_size + 1 ), workers = a .address )
826
- y = c .submit (mul , b"2" , int (b .target_message_size + 1 ), workers = a .address )
829
+ x = c .submit (mul , b"1" , int (b .transfer_message_target_bytes + 1 ), workers = a .address )
830
+ y = c .submit (mul , b"2" , int (b .transfer_message_target_bytes + 1 ), workers = a .address )
827
831
await wait ([x , y ])
828
832
z = c .submit (add , x , y , workers = b .address )
829
833
await wait (z )
830
- assert len (b .incoming_transfer_log ) == 2
831
- l1 , l2 = b .incoming_transfer_log
834
+ assert len (b .transfer_incoming_log ) == 2
835
+ l1 , l2 = b .transfer_incoming_log
832
836
833
837
assert l1 ["stop" ] < l2 ["start" ]
834
838
@@ -1244,8 +1248,8 @@ async def test_wait_for_outgoing(c, s, a, b):
1244
1248
y = c .submit (inc , future , workers = b .address )
1245
1249
await wait (y )
1246
1250
1247
- assert len (b .incoming_transfer_log ) == len (a .outgoing_transfer_log ) == 1
1248
- bb = b .incoming_transfer_log [0 ]["duration" ]
1251
+ assert len (b .transfer_incoming_log ) == len (a .transfer_outgoing_log ) == 1
1252
+ bb = b .transfer_incoming_log [0 ]["duration" ]
1249
1253
aa = a .outgoing_transfer_log [0 ]["duration" ]
1250
1254
ratio = aa / bb
1251
1255
@@ -1262,8 +1266,8 @@ async def test_prefer_gather_from_local_address(c, s, w1, w2, w3):
1262
1266
y = c .submit (inc , x , workers = [w2 .address ])
1263
1267
await wait (y )
1264
1268
1265
- assert any (d ["who" ] == w2 .address for d in w1 .outgoing_transfer_log )
1266
- assert not any (d ["who" ] == w2 .address for d in w3 .outgoing_transfer_log )
1269
+ assert any (d ["who" ] == w2 .address for d in w1 .transfer_outgoing_log )
1270
+ assert not any (d ["who" ] == w2 .address for d in w3 .transfer_outgoing_log )
1267
1271
1268
1272
1269
1273
@gen_cluster (
@@ -1281,10 +1285,10 @@ async def test_avoid_oversubscription(c, s, *workers):
1281
1285
await wait (futures )
1282
1286
1283
1287
# Original worker not responsible for all transfers
1284
- assert len (workers [0 ].outgoing_transfer_log ) < len (workers ) - 2
1288
+ assert len (workers [0 ].transfer_outgoing_log ) < len (workers ) - 2
1285
1289
1286
1290
# Some other workers did some work
1287
- assert len ([w for w in workers if len (w .outgoing_transfer_log ) > 0 ]) >= 3
1291
+ assert len ([w for w in workers if len (w .transfer_outgoing_log ) > 0 ]) >= 3
1288
1292
1289
1293
1290
1294
@gen_cluster (client = True , worker_kwargs = {"metrics" : {"my_port" : lambda w : w .port }})
@@ -1959,7 +1963,7 @@ async def test_gather_dep_one_worker_always_busy(c, s, a, b):
1959
1963
# We will block A for any outgoing communication. This simulates an
1960
1964
# overloaded worker which will always return "busy" for get_data requests,
1961
1965
# effectively blocking H indefinitely
1962
- a .outgoing_current_count = 10000000
1966
+ a .transfer_outgoing_count = 10000000
1963
1967
1964
1968
h = c .submit (add , f , g , key = "h" , workers = [b .address ])
1965
1969
@@ -2021,7 +2025,7 @@ async def test_gather_dep_from_remote_workers_if_all_local_workers_are_busy(
2021
2025
)
2022
2026
)["f" ]
2023
2027
for w in lws :
2024
- w .outgoing_current_count = 10000000
2028
+ w .transfer_outgoing_count = 10000000
2025
2029
2026
2030
g = c .submit (inc , f , key = "g" , workers = [a .address ])
2027
2031
assert await g == 2
@@ -2718,7 +2722,7 @@ async def test_acquire_replicas_same_channel(c, s, a, b):
2718
2722
("request-dep" , a .address , {fut .key }),
2719
2723
],
2720
2724
)
2721
- assert any (fut .key in msg ["keys" ] for msg in b .incoming_transfer_log )
2725
+ assert any (fut .key in msg ["keys" ] for msg in b .transfer_incoming_log )
2722
2726
2723
2727
2724
2728
@gen_cluster (client = True , nthreads = [("127.0.0.1" , 1 )] * 3 )
@@ -2997,9 +3001,9 @@ async def test_acquire_replicas_with_no_priority(c, s, a, b):
2997
3001
@gen_cluster (client = True , nthreads = [("" , 1 )])
2998
3002
async def test_acquire_replicas_large_data (c , s , a ):
2999
3003
"""When acquire-replicas is used to acquire multiple sizeable tasks, it respects
3000
- target_message_size and acquires them over multiple iterations.
3004
+ transfer_message_target_bytes and acquires them over multiple iterations.
3001
3005
"""
3002
- size = a .state .target_message_size // 5 - 10_000
3006
+ size = a .state .transfer_message_target_bytes // 5 - 10_000
3003
3007
3004
3008
class C :
3005
3009
def __sizeof__ (self ):
@@ -3026,7 +3030,7 @@ async def test_missing_released_zombie_tasks(c, s, a, b):
3026
3030
Ensure that no fetch/flight tasks are left in the task dict of a
3027
3031
worker after everything was released
3028
3032
"""
3029
- a .total_in_connections = 0
3033
+ a .transfer_outgoing_count_limit = 0
3030
3034
f1 = c .submit (inc , 1 , key = "f1" , workers = [a .address ])
3031
3035
f2 = c .submit (inc , f1 , key = "f2" , workers = [b .address ])
3032
3036
key = f1 .key
@@ -3310,8 +3314,8 @@ async def test_Worker__to_dict(c, s, a):
3310
3314
"thread_id" ,
3311
3315
"logs" ,
3312
3316
"config" ,
3313
- "incoming_transfer_log " ,
3314
- "outgoing_transfer_log " ,
3317
+ "transfer_incoming_log " ,
3318
+ "transfer_outgoing_log " ,
3315
3319
# Attributes of WorkerMemoryManager
3316
3320
"data" ,
3317
3321
"max_spill" ,
@@ -3551,3 +3555,20 @@ async def test_execute_preamble_abort_retirement(c, s):
3551
3555
3552
3556
# Test that y does not get stuck.
3553
3557
assert await y == 2
3558
+
3559
+
3560
+ @gen_cluster ()
3561
+ async def test_deprecation_of_renamed_worker_attributes (s , a , b ):
3562
+ msg = (
3563
+ "The `Worker.outgoing_count` attribute has been renamed to "
3564
+ "`Worker.transfer_outgoing_count_total`"
3565
+ )
3566
+ with pytest .warns (DeprecationWarning , match = msg ):
3567
+ assert a .outgoing_count == a .transfer_outgoing_count_total
3568
+
3569
+ msg = (
3570
+ "The `Worker.outgoing_current_count` attribute has been renamed to "
3571
+ "`Worker.transfer_outgoing_count`"
3572
+ )
3573
+ with pytest .warns (DeprecationWarning , match = msg ):
3574
+ assert a .outgoing_current_count == a .transfer_outgoing_count
0 commit comments