File: distributed-schema.yaml

package info (click to toggle)
dask.distributed 2022.12.1%2Bds.1-3
  • links: PTS, VCS
  • area: main
  • in suites: bookworm
  • size: 10,164 kB
  • sloc: python: 81,938; javascript: 1,549; makefile: 228; sh: 100
file content (1153 lines) | stat: -rw-r--r-- 44,502 bytes parent folder | download
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
356
357
358
359
360
361
362
363
364
365
366
367
368
369
370
371
372
373
374
375
376
377
378
379
380
381
382
383
384
385
386
387
388
389
390
391
392
393
394
395
396
397
398
399
400
401
402
403
404
405
406
407
408
409
410
411
412
413
414
415
416
417
418
419
420
421
422
423
424
425
426
427
428
429
430
431
432
433
434
435
436
437
438
439
440
441
442
443
444
445
446
447
448
449
450
451
452
453
454
455
456
457
458
459
460
461
462
463
464
465
466
467
468
469
470
471
472
473
474
475
476
477
478
479
480
481
482
483
484
485
486
487
488
489
490
491
492
493
494
495
496
497
498
499
500
501
502
503
504
505
506
507
508
509
510
511
512
513
514
515
516
517
518
519
520
521
522
523
524
525
526
527
528
529
530
531
532
533
534
535
536
537
538
539
540
541
542
543
544
545
546
547
548
549
550
551
552
553
554
555
556
557
558
559
560
561
562
563
564
565
566
567
568
569
570
571
572
573
574
575
576
577
578
579
580
581
582
583
584
585
586
587
588
589
590
591
592
593
594
595
596
597
598
599
600
601
602
603
604
605
606
607
608
609
610
611
612
613
614
615
616
617
618
619
620
621
622
623
624
625
626
627
628
629
630
631
632
633
634
635
636
637
638
639
640
641
642
643
644
645
646
647
648
649
650
651
652
653
654
655
656
657
658
659
660
661
662
663
664
665
666
667
668
669
670
671
672
673
674
675
676
677
678
679
680
681
682
683
684
685
686
687
688
689
690
691
692
693
694
695
696
697
698
699
700
701
702
703
704
705
706
707
708
709
710
711
712
713
714
715
716
717
718
719
720
721
722
723
724
725
726
727
728
729
730
731
732
733
734
735
736
737
738
739
740
741
742
743
744
745
746
747
748
749
750
751
752
753
754
755
756
757
758
759
760
761
762
763
764
765
766
767
768
769
770
771
772
773
774
775
776
777
778
779
780
781
782
783
784
785
786
787
788
789
790
791
792
793
794
795
796
797
798
799
800
801
802
803
804
805
806
807
808
809
810
811
812
813
814
815
816
817
818
819
820
821
822
823
824
825
826
827
828
829
830
831
832
833
834
835
836
837
838
839
840
841
842
843
844
845
846
847
848
849
850
851
852
853
854
855
856
857
858
859
860
861
862
863
864
865
866
867
868
869
870
871
872
873
874
875
876
877
878
879
880
881
882
883
884
885
886
887
888
889
890
891
892
893
894
895
896
897
898
899
900
901
902
903
904
905
906
907
908
909
910
911
912
913
914
915
916
917
918
919
920
921
922
923
924
925
926
927
928
929
930
931
932
933
934
935
936
937
938
939
940
941
942
943
944
945
946
947
948
949
950
951
952
953
954
955
956
957
958
959
960
961
962
963
964
965
966
967
968
969
970
971
972
973
974
975
976
977
978
979
980
981
982
983
984
985
986
987
988
989
990
991
992
993
994
995
996
997
998
999
1000
1001
1002
1003
1004
1005
1006
1007
1008
1009
1010
1011
1012
1013
1014
1015
1016
1017
1018
1019
1020
1021
1022
1023
1024
1025
1026
1027
1028
1029
1030
1031
1032
1033
1034
1035
1036
1037
1038
1039
1040
1041
1042
1043
1044
1045
1046
1047
1048
1049
1050
1051
1052
1053
1054
1055
1056
1057
1058
1059
1060
1061
1062
1063
1064
1065
1066
1067
1068
1069
1070
1071
1072
1073
1074
1075
1076
1077
1078
1079
1080
1081
1082
1083
1084
1085
1086
1087
1088
1089
1090
1091
1092
1093
1094
1095
1096
1097
1098
1099
1100
1101
1102
1103
1104
1105
1106
1107
1108
1109
1110
1111
1112
1113
1114
1115
1116
1117
1118
1119
1120
1121
1122
1123
1124
1125
1126
1127
1128
1129
1130
1131
1132
1133
1134
1135
1136
1137
1138
1139
1140
1141
1142
1143
1144
1145
1146
1147
1148
1149
1150
1151
1152
1153
properties:
  distributed:
    type: object
    properties:

      version:
        type: integer

      scheduler:
        type: object
        properties:

          allowed-failures:
            type: integer
            minimum: 0
            description: |
              The number of retries before a task is considered bad

              When a worker dies when a task is running that task is rerun elsewhere.
              If many workers die while running this same task then we call the task bad, and raise a KilledWorker exception.
              This is the number of workers that are allowed to die before this task is marked as bad.

          bandwidth:
            type:
            - integer
            - string
            description: |
              The expected bandwidth between any pair of workers

              This is used when making scheduling decisions.
              The scheduler will use this value as a baseline, but also learn it over time.

          blocked-handlers:
            type: array
            description: |
              A list of handlers to exclude

              The scheduler operates by receiving messages from various workers and clients
              and then performing operations based on those messages.
              Each message has an operation like "close-worker" or "task-finished".
              In some high security situations administrators may choose to block certain handlers
              from running.  Those handlers can be listed here.

              For a list of handlers see the `dask.distributed.Scheduler.handlers` attribute.

          contact-address:
            type:
            - string
            - "null"
            description: |
              The address that the scheduler advertises to workers for communication with it.

              To be specified when the address to which the scheduler binds cannot be the same
              as the address that workers use to contact the scheduler (e.g. because the former
              is private and the scheduler is in a different network than the workers).

          default-data-size:
            type:
            - string
            - integer
            description: |
              The default size of a piece of data if we don't know anything about it.

              This is used by the scheduler in some scheduling decisions

          events-cleanup-delay:
            type: string
            description: |
              The amount of time to wait until workers or clients are removed from the event log
              after they have been removed from the scheduler

          idle-timeout:
            type:
            - string
            - "null"
            description: |
              Shut down the scheduler after this duration if no activity has occurred

              This can be helpful to reduce costs and stop zombie processes from roaming the earth.

          transition-log-length:
            type: integer
            minimum: 0
            description: |
              How long should we keep the transition log

              Every time a task transitions states (like "waiting", "processing", "memory", "released")
              we record that transition in a log.

              To make sure that we don't run out of memory
              we will clear out old entries after a certain length.
              This is that length.

          events-log-length:
            type: integer
            minimum: 0
            description: |
              How long should we keep the events log

              All events (e.g. worker heartbeat) are stored in the events log.

              To make sure that we don't run out of memory
              we will clear out old entries after a certain length.
              This is that length.

          work-stealing:
            type: boolean
            description: |
              Whether or not to balance work between workers dynamically

              Some times one worker has more work than we expected.
              The scheduler will move these tasks around as necessary by default.
              Set this to false to disable this behavior

          work-stealing-interval:
            type: string
            description: |
              How frequently to balance worker loads

          worker-saturation:
            oneOf:
              - type: number
                exclusiveMinimum: 0
              # String "inf", not to be confused with .inf which in YAML means float
              # infinity. This is necessary because there's no way to parse a float
              # infinity from a DASK_* environment variable.
              - enum: [inf]
            description: |
              Controls how many root tasks are sent to workers (like a `readahead`).

              Up to worker-saturation * nthreads root tasks are sent to a
              worker at a time. If `.inf`, all runnable tasks are immediately sent to workers.
              The target number is rounded up, so any `worker-saturation` value > 1.0 guarantees
              at least one extra task will be sent to workers.

              Allowing oversaturation (> 1.0) means a worker may start running a new root task as
              soon as it completes the previous, even if there is a higher-priority downstream task
              to run. This reduces worker idleness, by letting workers do something while waiting for
              further instructions from the scheduler, even if it's not the most efficient
              thing to do.

              This generally comes  at the expense of increased memory usage. It leads to "wider"
              (more breadth-first) execution of the graph.

              Compute-bound workloads may benefit from oversaturation. Memory-bound workloads should
              generally leave `worker-saturation` at 1.0, though 1.25-1.5 could slightly improve
              performance if ample memory is available.

          worker-ttl:
            type:
            - string
            - "null"
            description: |
              Time to live for workers.

              If we don't receive a heartbeat faster than this then we assume that the worker has died.

          pickle:
            type: boolean
            description: |
              Is the scheduler allowed to deserialize arbitrary bytestrings?

              The scheduler almost never deserializes user data.
              However there are some cases where the user can submit functions to run directly on the scheduler.
              This can be convenient for debugging, but also introduces some security risk.
              By setting this to false we ensure that the user is unable to run arbitrary code on the scheduler.

          preload:
            type: array
            description: |
              Run custom modules during the lifetime of the scheduler

              You can run custom modules when the scheduler starts up and closes down.
              See https://docs.dask.org/en/latest/how-to/customize-initialization.html for more information

          preload-argv:
            type: array
            description: |
              Arguments to pass into the preload scripts described above

              See https://docs.dask.org/en/latest/how-to/customize-initialization.html for more information

          unknown-task-duration:
            type: string
            description: |
              Default duration for all tasks with unknown durations

              Over time the scheduler learns a duration for tasks.
              However when it sees a new type of task for the first time it has to make a guess
              as to how long it will take.  This value is that guess.

          default-task-durations:
            type: object
            description: |
              How long we expect function names to run

              Over time the scheduler will learn these values, but these give it a good starting point.

          validate:
            type: boolean
            description: |
              Whether or not to run consistency checks during execution.
              This is typically only used for debugging.

          dashboard:
            type: object
            description: |
              Configuration options for Dask's real-time dashboard

            properties:
              status:
                type: object
                description: The main status page of the dashboard
                properties:
                  task-stream-length:
                    type: integer
                    minimum: 0
                    description: |
                      The maximum number of tasks to include in the task stream plot
              tasks:
                type: object
                description: |
                  The page which includes the full task stream history
                properties:
                  task-stream-length:
                    type: integer
                    minimum: 0
                    description: |
                      The maximum number of tasks to include in the task stream plot
              tls:
                type: object
                description: |
                  Settings around securing the dashboard
                properties:
                  ca-file:
                    type:
                    - string
                    - "null"
                  key:
                    type:
                    - string
                    - "null"
                  cert:
                    type:
                    - string
                    - "null"
              bokeh-application:
                type: object
                description: |
                  Keywords to pass to the BokehTornado application
          locks:
            type: object
            description: |
              Settings for Dask's distributed Lock object

              See https://docs.dask.org/en/latest/futures.html#locks for more information
            properties:
              lease-validation-interval:
                type: string
                description: |
                  The interval in which the scheduler validates staleness of all acquired leases. Must always be smaller than the lease-timeout itself.
              lease-timeout:
                type: string
                description: |
                  Maximum interval to wait for a Client refresh before a lease is invalidated and released.

          http:
            type: object
            description: Settings for Dask's embedded HTTP Server
            properties:
              routes:
                type: array
                description: |
                  A list of modules like "prometheus" and "health" that can be included or excluded as desired

                  These modules will have a ``routes`` keyword that gets added to the main HTTP Server.
                  This is also a list that can be extended with user defined modules.

          allowed-imports:
            type: array
            description: |
              A list of trusted root modules the schedular is allowed to import (incl. submodules). For security reasons, the
              scheduler does not import arbitrary Python modules.

          active-memory-manager:
            type: object
            required: [start, interval, measure, policies]
            additionalProperties: false
            properties:
              start:
                type: boolean
                description: set to true to auto-start the AMM on Scheduler init
              interval:
                type: string
                description:
                  Time expression, e.g. "2s". Run the AMM cycle every <interval>.
              measure:
                enum:
                  - process
                  - optimistic
                  - managed
                  - managed_in_memory
                description:
                  One of the attributes of distributed.scheduler.MemoryState
              policies:
                type: array
                items:
                  type: object
                  required: [class]
                  properties:
                    class:
                      type: string
                      description: fully qualified name of an ActiveMemoryManagerPolicy
                        subclass
                  additionalProperties:
                    description: keyword arguments to the policy constructor, if any

      worker:
        type: object
        description: |
          Configuration settings for Dask Workers
        properties:
          blocked-handlers:
            type: array
            description: |
              A list of handlers to exclude

              The scheduler operates by receiving messages from various workers and clients
              and then performing operations based on those messages.
              Each message has an operation like "close-worker" or "task-finished".
              In some high security situations administrators may choose to block certain handlers
              from running.  Those handlers can be listed here.

              For a list of handlers see the `dask.distributed.Scheduler.handlers` attribute.

          multiprocessing-method:
            enum:
              - spawn
              - fork
              - forkserver
            description: |
              How we create new workers, one of "spawn", "forkserver", or "fork"

              This is passed to the ``multiprocessing.get_context`` function.
          use-file-locking:
            type: boolean
            description: |
              Whether or not to use lock files when creating workers

              Workers create a local directory in which to place temporary files.
              When many workers are created on the same process at once
              these workers can conflict with each other by trying to create this directory all at the same time.

              To avoid this, Dask usually used a file-based lock.
              However, on some systems file-based locks don't work.
              This is particularly common on HPC NFS systems, where users may want to set this to false.
          transfer:
            type: object
            description: |
              Configuration setting for data transfer between workers
            properties:
              message-bytes-limit:
                type:
                  - string
                  - integer
                description: |
                  The maximum amount of data for a worker to request from another in a single gather operation

                  Tasks are gathered in batches, and if the first task in a batch is larger than this value,
                  the task will still be gathered to ensure progress. Hence, this limit is not absolute.
                  Note that this limit applies to a single gather operation and a worker may gather data from
                  multiple workers in parallel.
          connections:
            type: object
            description: |
              The number of concurrent connections to allow to other workers
            properties:
              incoming:
                type: integer
                minimum: 0
              outgoing:
                type: integer
                minimum: 0

          preload:
            type: array
            description: |
              Run custom modules during the lifetime of the worker

              You can run custom modules when the worker starts up and closes down.
              See https://docs.dask.org/en/latest/how-to/customize-initialization.html for more information

          preload-argv:
            type: array
            description: |
              Arguments to pass into the preload scripts described above

              See https://docs.dask.org/en/latest/how-to/customize-initialization.html for more information

          daemon:
            type: boolean
            description: |
              Whether or not to run our process as a daemon process

          validate:
            type: boolean
            description: |
              Whether or not to run consistency checks during execution.
              This is typically only used for debugging.

          resources:
            type: object
            description: |
              A dictionary specifying resources for workers.

              See https://distributed.dask.org/en/latest/resources.html for more information.
            properties: {}

          lifetime:
            type: object
            description: |
              The worker may choose to gracefully close itself down after some pre-determined time.

              This is particularly useful if you know that your worker job has a time limit on it.
              This is particularly common in HPC job schedulers.

              For example if your worker has a walltime of one hour,
              then you may want to set the lifetime.duration to "55 minutes"
            properties:
              duration:
                type:
                - string
                - "null"
                description: |
                  The time after creation to close the worker, like "1 hour"
              stagger:
                type: string
                description: |
                  Random amount by which to stagger lifetimes

                  If you create many workers at the same time,
                  you may want to avoid having them kill themselves all at the same time.
                  To avoid this you might want to set a stagger time,
                  so that they close themselves with some random variation, like "5 minutes"

                  That way some workers can die, new ones can be brought up,
                  and data can be transferred over smoothly.
              restart:
                type: boolean
                description: |
                  Do we try to resurrect the worker after the lifetime deadline?


          profile:
            type: object
            description: |
              The workers periodically poll every worker thread to see what they are working on.
              This data gets collected into statistical profiling information,
              which is then periodically bundled together and sent along to the scheduler.
            properties:
              enabled:
                type: boolean
                description: |
                  Whether or not to enable profiling
              interval:
                type: string
                description: |
                  The time between polling the worker threads, typically short like 10ms
              cycle:
                type: string
                description: |
                  The time between bundling together this data and sending it to the scheduler

                  This controls the granularity at which people can query the profile information
                  on the time axis.
              low-level:
                type: boolean
                description: |
                  Whether or not to use the libunwind and stacktrace libraries
                  to gather profiling information at the lower level (beneath Python)

                  To get this to work you will need to install the experimental stacktrace library at

                  conda install -c numba stacktrace

                  See https://github.com/numba/stacktrace

          memory:
            type: object
            description: >-
              Settings for memory management
            properties:
              recent-to-old-time:
                type: string
                description: >-
                  When there is an increase in process memory (as observed by the
                  operating system) that is not accounted for by the dask keys stored on
                  the worker, ignore it for this long before considering it in
                  non-time-sensitive heuristics. This should be set to be longer than
                  the duration of most dask tasks.
              rebalance:
                type: object
                description: >-
                  Settings for memory rebalance operations
                properties:
                  measure:
                    enum:
                      - process
                      - optimistic
                      - managed
                      - managed_in_memory
                    description: >-
                      Which of the properties of distributed.scheduler.MemoryState
                      should be used for measuring worker memory usage
                  sender-min:
                    type: number
                    minimum: 0
                    maximum: 1
                    description: >-
                      Fraction of worker process memory at which we start potentially
                      transferring data to other workers.
                  recipient-max:
                    type: number
                    minimum: 0
                    maximum: 1
                    description: >-
                      Fraction of worker process memory at which we stop potentially
                      receiving data from other workers. Ignored when max_memory is not
                      set.
                  sender-recipient-gap:
                    type: number
                    minimum: 0
                    maximum: 1
                    description: >-
                      Fraction of worker process memory, around the cluster mean, where
                      a worker is neither a sender nor a recipient of data during a
                      rebalance operation. E.g. if the mean cluster occupation is 50%,
                      sender-recipient-gap=0.1 means that only nodes above 55% will
                      donate data and only nodes below 45% will receive them. This helps
                      avoid data from bouncing around the cluster repeatedly.

              transfer:
                oneOf:
                  - {type: number, minimum: 0, maximum: 1}
                  - {enum: [false]}
                description: >-
                  When the total size of incoming data transfers gets above this amount,
                  we start throttling incoming data transfers

              target:
                oneOf:
                  - {type: number, minimum: 0, maximum: 1}
                  - {enum: [false]}
                description: >-
                  When the process memory (as observed by the operating system) gets
                  above this amount, we start spilling the dask keys holding the oldest
                  chunks of data to disk

              spill:
                oneOf:
                  - {type: number, minimum: 0, maximum: 1}
                  - {enum: [false]}
                description: >-
                  When the process memory (as observed by the operating system) gets
                  above this amount, we spill data to disk, starting from the dask keys
                  holding the oldest chunks of data, until the process memory falls below
                  the target threshold.

              pause:
                oneOf:
                  - {type: number, minimum: 0, maximum: 1}
                  - {enum: [false]}
                description: >-
                  When the process memory (as observed by the operating system) gets
                  above this amount, we no longer start new tasks or fetch new
                  data on the worker.

              terminate:
                oneOf:
                  - {type: number, minimum: 0, maximum: 1}
                  - {enum: [false]}
                description: >-
                  When the process memory reaches this level the nanny process will kill
                  the worker (if a nanny is present)

              max-spill:
                oneOf:
                  - type: string
                  - {type: number, minimum: 0}
                  - enum: [false]
                description: >-
                  Limit of number of bytes to be spilled on disk.

              monitor-interval:
                type: string
                description: >-
                  Interval between checks for the spill, pause, and terminate thresholds

          http:
            type: object
            description: Settings for Dask's embedded HTTP Server
            properties:
              routes:
                type: array
                description: |
                  A list of modules like "prometheus" and "health" that can be included or excluded as desired

                  These modules will have a ``routes`` keyword that gets added to the main HTTP Server.
                  This is also a list that can be extended with user defined modules.

      nanny:
        type: object
        description: |
          Configuration settings for Dask Nannies
        properties:

          preload:
            type: array
            description: |
              Run custom modules during the lifetime of the nanny

              You can run custom modules when the nanny starts up and closes down.
              See https://docs.dask.org/en/latest/how-to/customize-initialization.html for more information

          preload-argv:
            type: array
            description: |
              Arguments to pass into the preload scripts described above

              See https://docs.dask.org/en/latest/how-to/customize-initialization.html for more information

          environ:
            type: object
            description: |
              Environment variables to set on all worker processes started by nannies.
              These variables are set in the worker process after it has started.

          pre-spawn-environ:
            type: object
            description: |
              Environment variables to set on all worker processes started by nannies.
              These variables are set within the Nanny process,
              before spawning the worker process.
              Should be used for variables that must be set before
              process startup, interpreter startup, or imports.

      client:
        type: object
        description: |
          Configuration settings for Dask Clients

        properties:
          heartbeat:
            type: string
            description:
              This value is the time between heartbeats

              The client sends a periodic heartbeat message to the scheduler.
              If it misses enough of these then the scheduler assumes that it has gone.

          scheduler-info-interval:
            type: string
            description: Interval between scheduler-info updates

          security-loader:
            type: [string, 'null']
            description: |
              A fully qualified name (e.g. ``module.submodule.function``) of
              a callback to use for loading security credentials for the
              client. If no security object is explicitly passed when creating
              a ``Client``, this callback is called with a dict containing
              client information (currently just ``address``), and should
              return a ``Security`` object to use for this client, or ``None``
              to fallback to the default security configuration.
          preload:
            type: array
            description: |
              Run custom modules during the lifetime of the client

              You can run custom modules when the client starts up and closes down.
              See https://docs.dask.org/en/latest/how-to/customize-initialization.html for more information

          preload-argv:
            type: array
            description: |
              Arguments to pass into the preload scripts described above

              See https://docs.dask.org/en/latest/how-to/customize-initialization.html for more information


      deploy:
        type: object
        description: Configuration settings for general Dask deployment
        properties:
          lost-worker-timeout:
            type: string
            description: |
              Interval after which to hard-close a lost worker job

              Otherwise we wait for a while to see if a worker will reappear

          cluster-repr-interval:
            type: string
            description: Interval between calls to update cluster-repr for the widget

      adaptive:
        type: object
        description: Configuration settings for Dask's adaptive scheduling
        properties:
          interval:
            type: string
            description: |
              The duration between checking in with adaptive scheduling load

              The adaptive system periodically checks scheduler load and determines
              if it should scale the cluster up or down.
              This is the timing between those checks.

          target-duration:
            type: string
            description: |
              The desired time for the entire computation to run

              The adaptive system will try to start up enough workers to run
              the computation in about this time.

          minimum:
            type: integer
            minimum: 0
            description: |
              The minimum number of workers to keep around

          maximum:
            type: number
            minimum: 0
            description: |
              The maximum number of workers to keep around

          wait-count:
            type: integer
            minimum: 1
            description: |
              The number of times a worker should be suggested for removal before removing it

              This helps to smooth out the number of deployed workers

      comm:
        type: object
        description: Configuration settings for Dask communications
        properties:

          retry:
            type: object
            description: |
              Some operations (such as gathering data) are subject to re-tries with the below parameters
            properties:

              count:
                type: integer
                minimum: 0
                description: |
                  The number of times to retry a connection

              delay:
                type: object
                properties:
                  min:
                    type: string
                    description: The first non-zero delay between retry attempts
                  max:
                    type: string
                    description: The maximum delay between retries

          compression:
            type: string
            description: |
              The compression algorithm to use

              This could be one of lz4, snappy, zstd

          offload:
            type:
            - boolean
            - string
            description: |
              The size of message after which we choose to offload serialization to another thread

              In some cases, you may also choose to disable this altogether with the value false
              This is useful if you want to include serialization in profiling data,
              or if you have data types that are particularly sensitive to deserialization

          shard:
            type: string
            description: |
              The maximum size of a frame to send through a comm

              Some network infrastructure doesn't like sending through very large messages.
              Dask comms will cut up these large messages into many small ones.
              This attribute determines the maximum size of such a shard.

          socket-backlog:
            type: integer
            description: |
              When shuffling data between workers, there can
              really be O(cluster size) connection requests
              on a single worker socket, make sure the backlog
              is large enough not to lose any.

          zstd:
            type: object
            description: Options for the Z Standard compression scheme
            properties:
              level:
                type: integer
                minimum: 1
                maximum: 22
                description: Compression level, between 1 and 22.
              threads:
                type: integer
                minimum: -1
                description: |
                  Number of threads to use.

                  0 for single-threaded, -1 to infer from cpu count.

          timeouts:
            type: object
            properties:
              connect:
                type: string
              tcp:
                type: string

          require-encryption:
            type:
            - boolean
            - "null"
            description: |
              Whether to require encryption on non-local comms

          default-scheme:
            type: string
            description: The default protocol to use, like tcp or tls

          recent-messages-log-length:
            type: integer
            minimum: 0
            description: number of messages to keep for debugging

          tls:
            type: object
            properties:
              ciphers:
                type:
                - string
                - "null"
                description: Allowed ciphers, specified as an OpenSSL cipher string.

              min-version:
                enum: [null, 1.2, 1.3]
                description: The minimum TLS version to support. Defaults to TLS 1.2.

              max-version:
                enum: [null, 1.2, 1.3]
                description: |
                  The maximum TLS version to support. Defaults to the maximum
                  version supported by the platform.

              ca-file:
                type:
                - string
                - "null"
                description: Path to a CA file, in pem format

              scheduler:
                type: object
                description: TLS information for the scheduler
                properties:
                  cert:
                    type:
                    - string
                    - "null"
                    description: Path to certificate file
                  key:
                    type:
                    - string
                    - "null"
                    description: |
                      Path to key file.

                      Alternatively, the key can be appended to the cert file
                      above, and this field left blank

              worker:
                type: object
                description: TLS information for the worker
                properties:
                  cert:
                    type:
                    - string
                    - "null"
                    description: Path to certificate file
                  key:
                    type:
                    - string
                    - "null"
                    description: |
                      Path to key file.

                      Alternatively, the key can be appended to the cert file
                      above, and this field left blank

              client:
                type: object
                description: TLS information for the client
                properties:
                  cert:
                    type:
                    - string
                    - "null"
                    description: Path to certificate file
                  key:
                    type:
                    - string
                    - "null"
                    description: |
                      Path to key file.

                      Alternatively, the key can be appended to the cert file
                      above, and this field left blank

          ucx:
            type: object
            description: |
              UCX provides access to other transport methods including NVLink and InfiniBand.
            properties:
              cuda-copy:
                type: [boolean, 'null']
                description: |
                  Set environment variables to enable CUDA support over UCX. This may be used even if
                  InfiniBand and NVLink are not supported or disabled, then transferring data over TCP.
              tcp:
                type: [boolean, 'null']
                description: |
                  Set environment variables to enable TCP over UCX, even if InfiniBand and NVLink
                  are not supported or disabled.
              nvlink:
                type: [boolean, 'null']
                description: |
                  Set environment variables to enable UCX over NVLink, implies ``distributed.comm.ucx.tcp=True``.
              infiniband:
                type: [boolean, 'null']
                description: |
                  Set environment variables to enable UCX over InfiniBand, implies ``distributed.comm.ucx.tcp=True``.
              rdmacm:
                type: [boolean, 'null']
                description: |
                  Set environment variables to enable UCX RDMA connection manager support,
                  requires ``distributed.comm.ucx.infiniband=True``.
              create-cuda-context:
                type: [boolean, 'null']
                description: |
                  Creates a CUDA context before UCX is initialized. This is necessary to enable UCX to
                  properly identify connectivity of GPUs with specialized networking hardware, such as
                  InfiniBand. This permits UCX to choose transports automatically, without specifying
                  additional variables for each transport, while ensuring optimal connectivity. When
                  ``True``, a CUDA context will be created on the first device listed in
                  ``CUDA_VISIBLE_DEVICES``.
              environment:
                type: object
                description: |
                  Mapping for setting arbitrary UCX environment variables.
                  Names here are translated via the following rules to
                  map to the relevant UCX environment variable:
                    - hyphens are replaced with underscores
                    - words are uppercased
                    - UCX_ is prepended
                  So, for example, setting ``some-option=value`` is
                  equivalent to setting ``UCX_SOME_OPTION=value`` in
                  the calling environment.

                  For a full list of supported UCX environment
                  variables, run ``ucx_info -f``.
          tcp:
            type: object
            properties:
              backend:
                type: string
                description: |
                  The TCP backend implementation to use. Must be either `tornado` or `asyncio`.

          websockets:
            type: object
            properties:
              shard:
                type:
                - string
                description: |
                  The maximum size of a websocket frame to send through a comm.

                  This is somewhat duplicative of distributed.comm.shard, but websockets
                  often have much smaller maximum message sizes than other protocols, so
                  this attribute is used to set a smaller default shard size and to
                  allow separate control of websocket message sharding.

      diagnostics:
        type: object
        properties:
          nvml:
            type: boolean
            description: |
              If ``True``, enables GPU diagnostics with NVML. Generally leaving it enabled is
              not a problem and will be automatically disabled if no GPUs are found in the
              system, but in certain cases it may be desirable to completely disable NVML
              diagnostics.
          computations:
            type: object
            properties:
              max-history:
                type: integer
                minimum: 0
                description: |
                  The maximum number of Computations to remember.
              ignore-modules:
                type: array
                description: |
                  A list of modules which are ignored when trying to collect the
                  code context when submitting a computation. Accepts regular
                  expressions.
          erred-tasks:
            type: object
            properties:
              max-history:
                type: integer
                minimum: 0
                description: |
                  The maximum number of erred tasks to remember.

      dashboard:
        type: object
        properties:
          link:
            type: string
            description: |
              The form for the dashboard links

              This is used wherever we print out the link for the dashboard
              It is filled in with relevant information like the schema, host, and port number
          graph-max-items:
            type: integer
            minimum: 0
            description: maximum number of tasks to try to plot in "graph" view

          export-tool:
            type: boolean

          prometheus:
            type: object
            properties:
              namespace:
                type: string
                description: Namespace prefix to use for all prometheus metrics.

      admin:
        type: object
        description: |
          Options for logs, event loops, and so on
        properties:
          tick:
            type: object
            description: |
              Time between event loop health checks

              We set up a periodic callback to run on the event loop and check in fairly frequently.
              (by default, this is every 20 milliseconds)

              If this periodic callback sees that the last time it checked in was several seconds ago
              (by default, this is 3 seconds)
              then it logs a warning saying that something has been stopping the event loop from smooth operation.
              This is typically caused by GIL holding operations,
              but could also be several other things.

            properties:
              interval:
                type: string
                description: The time between ticks, default 20ms
              limit :
                type: string
                description: The time allowed before triggering a warning
              cycle :
                type: string
                description: The time in between verifying event loop speed

          max-error-length:
            type: integer
            minimum: 0
            description: |
              Maximum length of traceback as text

              Some Python tracebacks can be very very long
              (particularly in stack overflow errors)

              If the traceback is larger than this size (in bytes) then we truncate it.

          log-length:
            type: integer
            minimum: 0
            description: |
              Default length of logs to keep in memory

              The scheduler and workers keep the last 10000 or so log entries in memory.

          log-format:
            type: string
            description: |
              The log format to emit.

              See https://docs.python.org/3/library/logging.html#logrecord-attributes
          event-loop:
            type: string
            description: |
              The event loop to use,

              Must be one of tornado, asyncio, or uvloop

          pdb-on-err:
            type: boolean
            description: Enter Python Debugger on scheduling error

          system-monitor:
            type: object
            description: |
              Options for the periodic system monitor
            properties:
              interval:
                type: string
                description: Polling time to query cpu/memory statistics default 500ms
              disk:
                type: boolean
                description: Should we include disk metrics?  (they can cause issues in some systems)
              host-cpu:
                type: boolean
                description: Should we include host-wide CPU usage, with very granular breakdown?

      rmm:
        type: object
        description: |
          Configuration options for the RAPIDS Memory Manager.
        properties:
          pool-size:
            type: [integer, 'null']
            description: |
              The size of the memory pool in bytes.