dask-array 0.1.0__py3-none-any.whl
This diff represents the content of publicly available package versions that have been released to one of the supported registries. The information contained in this diff is provided for informational purposes only and reflects changes between package versions as they appear in their respective public registries.
- dask_array/__init__.py +228 -0
- dask_array/_backends.py +76 -0
- dask_array/_backends_array.py +99 -0
- dask_array/_blockwise.py +1410 -0
- dask_array/_broadcast.py +272 -0
- dask_array/_chunk.py +445 -0
- dask_array/_chunk_types.py +54 -0
- dask_array/_collection.py +1644 -0
- dask_array/_concatenate.py +331 -0
- dask_array/_core_utils.py +1365 -0
- dask_array/_dispatch.py +141 -0
- dask_array/_einsum.py +277 -0
- dask_array/_expr.py +544 -0
- dask_array/_expr_flow.py +586 -0
- dask_array/_gufunc.py +805 -0
- dask_array/_histogram.py +617 -0
- dask_array/_map_blocks.py +652 -0
- dask_array/_new_collection.py +10 -0
- dask_array/_numpy_compat.py +135 -0
- dask_array/_overlap.py +1159 -0
- dask_array/_rechunk.py +1050 -0
- dask_array/_reshape.py +710 -0
- dask_array/_routines.py +102 -0
- dask_array/_shuffle.py +448 -0
- dask_array/_stack.py +264 -0
- dask_array/_svg.py +291 -0
- dask_array/_templates.py +29 -0
- dask_array/_test_utils.py +257 -0
- dask_array/_ufunc.py +385 -0
- dask_array/_utils.py +349 -0
- dask_array/_visualize.py +223 -0
- dask_array/_xarray.py +337 -0
- dask_array/core/__init__.py +34 -0
- dask_array/core/_blockwise_funcs.py +312 -0
- dask_array/core/_conversion.py +422 -0
- dask_array/core/_from_graph.py +97 -0
- dask_array/creation/__init__.py +71 -0
- dask_array/creation/_arange.py +121 -0
- dask_array/creation/_diag.py +116 -0
- dask_array/creation/_diagonal.py +241 -0
- dask_array/creation/_eye.py +103 -0
- dask_array/creation/_linspace.py +102 -0
- dask_array/creation/_mesh.py +134 -0
- dask_array/creation/_ones_zeros.py +454 -0
- dask_array/creation/_pad.py +270 -0
- dask_array/creation/_repeat.py +55 -0
- dask_array/creation/_tile.py +36 -0
- dask_array/creation/_tri.py +28 -0
- dask_array/creation/_utils.py +296 -0
- dask_array/fft.py +320 -0
- dask_array/io/__init__.py +39 -0
- dask_array/io/_base.py +10 -0
- dask_array/io/_from_array.py +257 -0
- dask_array/io/_from_delayed.py +95 -0
- dask_array/io/_from_graph.py +54 -0
- dask_array/io/_from_npy_stack.py +67 -0
- dask_array/io/_store.py +336 -0
- dask_array/io/_tiledb.py +159 -0
- dask_array/io/_to_npy_stack.py +65 -0
- dask_array/io/_zarr.py +449 -0
- dask_array/linalg/__init__.py +39 -0
- dask_array/linalg/_cholesky.py +234 -0
- dask_array/linalg/_lu.py +300 -0
- dask_array/linalg/_norm.py +94 -0
- dask_array/linalg/_qr.py +601 -0
- dask_array/linalg/_solve.py +349 -0
- dask_array/linalg/_svd.py +394 -0
- dask_array/linalg/_tensordot.py +334 -0
- dask_array/linalg/_utils.py +74 -0
- dask_array/manipulation/__init__.py +45 -0
- dask_array/manipulation/_expand.py +321 -0
- dask_array/manipulation/_flip.py +92 -0
- dask_array/manipulation/_roll.py +78 -0
- dask_array/manipulation/_transpose.py +309 -0
- dask_array/random/__init__.py +125 -0
- dask_array/random/_choice.py +181 -0
- dask_array/random/_expr.py +256 -0
- dask_array/random/_generator.py +441 -0
- dask_array/random/_random_state.py +259 -0
- dask_array/random/_utils.py +84 -0
- dask_array/reductions/__init__.py +84 -0
- dask_array/reductions/_arg_reduction.py +130 -0
- dask_array/reductions/_common.py +1082 -0
- dask_array/reductions/_cumulative.py +522 -0
- dask_array/reductions/_percentile.py +261 -0
- dask_array/reductions/_reduction.py +725 -0
- dask_array/reductions/_trace.py +56 -0
- dask_array/routines/__init__.py +133 -0
- dask_array/routines/_apply.py +84 -0
- dask_array/routines/_bincount.py +112 -0
- dask_array/routines/_broadcast.py +111 -0
- dask_array/routines/_coarsen.py +115 -0
- dask_array/routines/_diff.py +79 -0
- dask_array/routines/_gradient.py +158 -0
- dask_array/routines/_indexing.py +65 -0
- dask_array/routines/_insert_delete.py +132 -0
- dask_array/routines/_misc.py +122 -0
- dask_array/routines/_nonzero.py +72 -0
- dask_array/routines/_search.py +123 -0
- dask_array/routines/_select.py +113 -0
- dask_array/routines/_statistics.py +171 -0
- dask_array/routines/_topk.py +82 -0
- dask_array/routines/_triangular.py +74 -0
- dask_array/routines/_unique.py +232 -0
- dask_array/routines/_where.py +62 -0
- dask_array/slicing/__init__.py +67 -0
- dask_array/slicing/_basic.py +550 -0
- dask_array/slicing/_blocks.py +138 -0
- dask_array/slicing/_bool_index.py +145 -0
- dask_array/slicing/_setitem.py +329 -0
- dask_array/slicing/_squeeze.py +101 -0
- dask_array/slicing/_utils.py +1133 -0
- dask_array/slicing/_vindex.py +282 -0
- dask_array/stacking/__init__.py +15 -0
- dask_array/stacking/_block.py +83 -0
- dask_array/stacking/_simple.py +58 -0
- dask_array/templates/array.html.j2 +48 -0
- dask_array/tests/__init__.py +0 -0
- dask_array/tests/conftest.py +22 -0
- dask_array/tests/test_api.py +40 -0
- dask_array/tests/test_binary_op_chunks.py +107 -0
- dask_array/tests/test_coarse_slice_through_blockwise.py +362 -0
- dask_array/tests/test_collection.py +799 -0
- dask_array/tests/test_creation.py +1102 -0
- dask_array/tests/test_expr_flow.py +143 -0
- dask_array/tests/test_linalg.py +1130 -0
- dask_array/tests/test_map_blocks_multi_output.py +104 -0
- dask_array/tests/test_rechunk_pushdown.py +214 -0
- dask_array/tests/test_reductions.py +1091 -0
- dask_array/tests/test_routines.py +2853 -0
- dask_array/tests/test_shuffle_chunks.py +67 -0
- dask_array/tests/test_slice_pushdown.py +968 -0
- dask_array/tests/test_slice_through_blockwise.py +678 -0
- dask_array/tests/test_slice_through_overlap.py +366 -0
- dask_array/tests/test_slice_through_reshape.py +272 -0
- dask_array/tests/test_slicing.py +839 -0
- dask_array/tests/test_transpose_slice_pushdown.py +208 -0
- dask_array/tests/test_visualize.py +94 -0
- dask_array/tests/test_xarray.py +193 -0
- dask_array-0.1.0.dist-info/METADATA +48 -0
- dask_array-0.1.0.dist-info/RECORD +144 -0
- dask_array-0.1.0.dist-info/WHEEL +4 -0
- dask_array-0.1.0.dist-info/entry_points.txt +2 -0
- dask_array-0.1.0.dist-info/licenses/LICENSE +29 -0
|
@@ -0,0 +1,256 @@
|
|
|
1
|
+
from __future__ import annotations
|
|
2
|
+
|
|
3
|
+
import importlib
|
|
4
|
+
from itertools import product
|
|
5
|
+
|
|
6
|
+
import numpy as np
|
|
7
|
+
|
|
8
|
+
from dask._task_spec import Dict as TaskDict
|
|
9
|
+
from dask._task_spec import Task, TaskRef
|
|
10
|
+
from dask._task_spec import Tuple as TaskTuple
|
|
11
|
+
from dask_array.io import IO
|
|
12
|
+
from dask_array._core_utils import broadcast_shapes, normalize_chunks
|
|
13
|
+
from dask_array._backends_array import array_creation_dispatch
|
|
14
|
+
from dask.tokenize import tokenize
|
|
15
|
+
from dask.utils import cached_property, random_state_data, typename
|
|
16
|
+
|
|
17
|
+
from ._generator import Generator
|
|
18
|
+
from ._random_state import RandomState
|
|
19
|
+
|
|
20
|
+
|
|
21
|
+
def _rng_from_bitgen(bitgen):
|
|
22
|
+
# Assumes typename(bitgen) starts with importable
|
|
23
|
+
# library name (e.g. "numpy" or "cupy")
|
|
24
|
+
backend_name = typename(bitgen).split(".")[0]
|
|
25
|
+
backend_lib = importlib.import_module(backend_name)
|
|
26
|
+
return backend_lib.random.default_rng(bitgen)
|
|
27
|
+
|
|
28
|
+
|
|
29
|
+
def _spawn_bitgens(bitgen, n_bitgens):
|
|
30
|
+
seeds = bitgen._seed_seq.spawn(n_bitgens)
|
|
31
|
+
bitgens = [type(bitgen)(seed) for seed in seeds]
|
|
32
|
+
return bitgens
|
|
33
|
+
|
|
34
|
+
|
|
35
|
+
def _apply_random_func(rng, funcname, bitgen, size, args, kwargs):
|
|
36
|
+
"""Apply random module method with seed"""
|
|
37
|
+
if isinstance(bitgen, np.random.SeedSequence):
|
|
38
|
+
bitgen = rng(bitgen)
|
|
39
|
+
rng = _rng_from_bitgen(bitgen)
|
|
40
|
+
func = getattr(rng, funcname)
|
|
41
|
+
return func(*args, size=size, **kwargs)
|
|
42
|
+
|
|
43
|
+
|
|
44
|
+
def _apply_random(RandomState, funcname, state_data, size, args, kwargs):
|
|
45
|
+
"""Apply RandomState method with seed"""
|
|
46
|
+
if RandomState is None:
|
|
47
|
+
RandomState = array_creation_dispatch.RandomState
|
|
48
|
+
state = RandomState(state_data)
|
|
49
|
+
func = getattr(state, funcname)
|
|
50
|
+
return func(*args, size=size, **kwargs)
|
|
51
|
+
|
|
52
|
+
|
|
53
|
+
class Random(IO):
|
|
54
|
+
_parameters = [
|
|
55
|
+
"rng",
|
|
56
|
+
"distribution",
|
|
57
|
+
"size",
|
|
58
|
+
"chunks",
|
|
59
|
+
"extra_chunks",
|
|
60
|
+
"args",
|
|
61
|
+
"kwargs",
|
|
62
|
+
]
|
|
63
|
+
_defaults = {"extra_chunks": ()}
|
|
64
|
+
_is_blockwise_fusable = True
|
|
65
|
+
|
|
66
|
+
@cached_property
|
|
67
|
+
def kwargs(self):
|
|
68
|
+
return self.operand("kwargs")
|
|
69
|
+
|
|
70
|
+
@cached_property
|
|
71
|
+
def _base_chunks(self):
|
|
72
|
+
"""Chunks for the size dimensions, excluding extra_chunks."""
|
|
73
|
+
size = self.operand("size")
|
|
74
|
+
chunks = self.operand("chunks")
|
|
75
|
+
shape = broadcast_shapes(size) if size is not None else ()
|
|
76
|
+
return normalize_chunks(
|
|
77
|
+
chunks,
|
|
78
|
+
shape,
|
|
79
|
+
dtype=self.kwargs.get("dtype", np.float64),
|
|
80
|
+
)
|
|
81
|
+
|
|
82
|
+
@property
|
|
83
|
+
def chunks(self):
|
|
84
|
+
return self._base_chunks + self.extra_chunks
|
|
85
|
+
|
|
86
|
+
@cached_property
|
|
87
|
+
def _info(self):
|
|
88
|
+
sizes = list(product(*self._base_chunks))
|
|
89
|
+
if isinstance(self.rng, Generator):
|
|
90
|
+
bitgens = _spawn_bitgens(self.rng._bit_generator, len(sizes))
|
|
91
|
+
bitgen_token = tokenize(bitgens)
|
|
92
|
+
bitgens = [_bitgen._seed_seq for _bitgen in bitgens]
|
|
93
|
+
func_applier = _apply_random_func
|
|
94
|
+
gen = type(self.rng._bit_generator)
|
|
95
|
+
elif isinstance(self.rng, RandomState):
|
|
96
|
+
bitgens = random_state_data(len(sizes), self.rng._numpy_state)
|
|
97
|
+
bitgen_token = tokenize(bitgens)
|
|
98
|
+
func_applier = _apply_random
|
|
99
|
+
gen = self.rng._RandomState
|
|
100
|
+
else:
|
|
101
|
+
raise TypeError("Unknown object type: Not a Generator and Not a RandomState")
|
|
102
|
+
token = tokenize(bitgen_token, self.size, self.chunks, self.args, self.kwargs)
|
|
103
|
+
name = f"{self.distribution}-{token}"
|
|
104
|
+
|
|
105
|
+
return bitgens, name, sizes, gen, func_applier
|
|
106
|
+
|
|
107
|
+
@property
|
|
108
|
+
def _name(self):
|
|
109
|
+
return self._info[1]
|
|
110
|
+
|
|
111
|
+
@property
|
|
112
|
+
def bitgens(self):
|
|
113
|
+
return self._info[0]
|
|
114
|
+
|
|
115
|
+
def _layer(self):
|
|
116
|
+
result = {}
|
|
117
|
+
for block_id in product(*[range(len(bd)) for bd in self.chunks]):
|
|
118
|
+
key = (self._name, *block_id)
|
|
119
|
+
result[key] = self._task(key, block_id)
|
|
120
|
+
return result
|
|
121
|
+
|
|
122
|
+
def _block_id_to_flat_index(self, block_id: tuple[int, ...]) -> int:
|
|
123
|
+
"""Convert N-D block_id to flat index for bitgens/sizes lookup."""
|
|
124
|
+
# Only use base_chunks dimensions (exclude extra_chunks which are always 0)
|
|
125
|
+
base_block_id = block_id[: len(self._base_chunks)]
|
|
126
|
+
flat_idx = 0
|
|
127
|
+
stride = 1
|
|
128
|
+
for i in reversed(range(len(base_block_id))):
|
|
129
|
+
flat_idx += base_block_id[i] * stride
|
|
130
|
+
stride *= len(self._base_chunks[i])
|
|
131
|
+
return flat_idx
|
|
132
|
+
|
|
133
|
+
def _task(self, key, block_id: tuple[int, ...]) -> Task:
|
|
134
|
+
"""Generate task for a specific output block."""
|
|
135
|
+
from dask_array._expr import ArrayExpr
|
|
136
|
+
|
|
137
|
+
bitgens, name, sizes, gen, func_applier = self._info
|
|
138
|
+
flat_idx = self._block_id_to_flat_index(block_id)
|
|
139
|
+
|
|
140
|
+
# Convert array expressions to TaskRefs
|
|
141
|
+
task_args = []
|
|
142
|
+
has_array_args = False
|
|
143
|
+
for arg in self.args:
|
|
144
|
+
if isinstance(arg, ArrayExpr):
|
|
145
|
+
# Reference the corresponding block of the dependency
|
|
146
|
+
task_args.append(TaskRef((arg._name,) + block_id))
|
|
147
|
+
has_array_args = True
|
|
148
|
+
else:
|
|
149
|
+
task_args.append(arg)
|
|
150
|
+
|
|
151
|
+
task_kwargs = {}
|
|
152
|
+
has_array_kwargs = False
|
|
153
|
+
for k, v in self.kwargs.items():
|
|
154
|
+
if isinstance(v, ArrayExpr):
|
|
155
|
+
task_kwargs[k] = TaskRef((v._name,) + block_id)
|
|
156
|
+
has_array_kwargs = True
|
|
157
|
+
else:
|
|
158
|
+
task_kwargs[k] = v
|
|
159
|
+
|
|
160
|
+
# Use TaskTuple/TaskDict to properly track dependencies inside containers
|
|
161
|
+
args_container = TaskTuple(*task_args) if has_array_args else tuple(task_args)
|
|
162
|
+
kwargs_container = TaskDict(task_kwargs) if has_array_kwargs else task_kwargs
|
|
163
|
+
|
|
164
|
+
return Task(
|
|
165
|
+
key,
|
|
166
|
+
func_applier,
|
|
167
|
+
gen,
|
|
168
|
+
self.distribution,
|
|
169
|
+
bitgens[flat_idx],
|
|
170
|
+
sizes[flat_idx],
|
|
171
|
+
args_container,
|
|
172
|
+
kwargs_container,
|
|
173
|
+
)
|
|
174
|
+
|
|
175
|
+
def dependencies(self):
|
|
176
|
+
"""Return array expression dependencies."""
|
|
177
|
+
from dask_array._expr import ArrayExpr
|
|
178
|
+
|
|
179
|
+
deps = []
|
|
180
|
+
for arg in self.args:
|
|
181
|
+
if isinstance(arg, ArrayExpr):
|
|
182
|
+
deps.append(arg)
|
|
183
|
+
for v in self.kwargs.values():
|
|
184
|
+
if isinstance(v, ArrayExpr):
|
|
185
|
+
deps.append(v)
|
|
186
|
+
return deps
|
|
187
|
+
|
|
188
|
+
def _input_block_id(self, dep, block_id: tuple[int, ...]) -> tuple[int, ...]:
|
|
189
|
+
"""Map output block_id to input block_id for dependencies."""
|
|
190
|
+
return block_id
|
|
191
|
+
|
|
192
|
+
@cached_property
|
|
193
|
+
def _meta(self):
|
|
194
|
+
from dask_array._expr import ArrayExpr
|
|
195
|
+
|
|
196
|
+
bitgens, name, sizes, gen, func_applier = self._info
|
|
197
|
+
size = self.operand("size")
|
|
198
|
+
meta_size = (0,) * len(size) if size is not None else ()
|
|
199
|
+
|
|
200
|
+
# Convert array arguments to scalars for meta computation
|
|
201
|
+
def to_scalar(x):
|
|
202
|
+
if isinstance(x, ArrayExpr):
|
|
203
|
+
# Array expression - use dtype to create a scalar
|
|
204
|
+
return x.dtype.type(1)
|
|
205
|
+
elif isinstance(x, np.ndarray):
|
|
206
|
+
# Numpy array
|
|
207
|
+
return x.flat[0]
|
|
208
|
+
return x
|
|
209
|
+
|
|
210
|
+
meta_args = tuple(to_scalar(arg) for arg in self.args)
|
|
211
|
+
meta_kwargs = {k: to_scalar(v) for k, v in self.kwargs.items()}
|
|
212
|
+
|
|
213
|
+
return func_applier(
|
|
214
|
+
gen,
|
|
215
|
+
self.distribution,
|
|
216
|
+
bitgens[0],
|
|
217
|
+
meta_size,
|
|
218
|
+
meta_args,
|
|
219
|
+
meta_kwargs,
|
|
220
|
+
)
|
|
221
|
+
|
|
222
|
+
|
|
223
|
+
# Distribution-specific subclasses with explicit parameters
|
|
224
|
+
# These avoid storing array dependencies inside tuple operands
|
|
225
|
+
|
|
226
|
+
|
|
227
|
+
class RandomNormal(Random):
|
|
228
|
+
"""Normal distribution with explicit loc and scale parameters."""
|
|
229
|
+
|
|
230
|
+
_parameters = ["rng", "size", "chunks", "extra_chunks", "loc", "scale"]
|
|
231
|
+
_defaults = {"extra_chunks": (), "loc": 0.0, "scale": 1.0}
|
|
232
|
+
distribution = "normal"
|
|
233
|
+
|
|
234
|
+
@property
|
|
235
|
+
def args(self):
|
|
236
|
+
return (self.loc, self.scale)
|
|
237
|
+
|
|
238
|
+
@property
|
|
239
|
+
def kwargs(self):
|
|
240
|
+
return {}
|
|
241
|
+
|
|
242
|
+
|
|
243
|
+
class RandomPoisson(Random):
|
|
244
|
+
"""Poisson distribution with explicit lam parameter."""
|
|
245
|
+
|
|
246
|
+
_parameters = ["rng", "size", "chunks", "extra_chunks", "lam"]
|
|
247
|
+
_defaults = {"extra_chunks": (), "lam": 1.0}
|
|
248
|
+
distribution = "poisson"
|
|
249
|
+
|
|
250
|
+
@property
|
|
251
|
+
def args(self):
|
|
252
|
+
return (self.lam,)
|
|
253
|
+
|
|
254
|
+
@property
|
|
255
|
+
def kwargs(self):
|
|
256
|
+
return {}
|
|
@@ -0,0 +1,441 @@
|
|
|
1
|
+
from __future__ import annotations
|
|
2
|
+
|
|
3
|
+
import importlib
|
|
4
|
+
import numbers
|
|
5
|
+
|
|
6
|
+
import numpy as np
|
|
7
|
+
|
|
8
|
+
from dask_array._new_collection import new_collection
|
|
9
|
+
from dask_array.creation import arange
|
|
10
|
+
from dask_array._backends_array import array_creation_dispatch
|
|
11
|
+
from dask.utils import derived_from, typename
|
|
12
|
+
|
|
13
|
+
from ._utils import _wrap_func
|
|
14
|
+
|
|
15
|
+
|
|
16
|
+
class Generator:
|
|
17
|
+
"""
|
|
18
|
+
Container for the BitGenerators.
|
|
19
|
+
|
|
20
|
+
``Generator`` exposes a number of methods for generating random
|
|
21
|
+
numbers drawn from a variety of probability distributions and serves
|
|
22
|
+
as a replacement for ``RandomState``. The main difference between the
|
|
23
|
+
two is that ``Generator`` relies on an additional ``BitGenerator`` to
|
|
24
|
+
manage state and generate the random bits, which are then transformed
|
|
25
|
+
into random values from useful distributions. The default ``BitGenerator``
|
|
26
|
+
used by ``Generator`` is ``PCG64``. The ``BitGenerator`` can be changed
|
|
27
|
+
by passing an instantiated ``BitGenerator`` to ``Generator``.
|
|
28
|
+
|
|
29
|
+
The function :func:`dask.array.random.default_rng` is the recommended way
|
|
30
|
+
to instantiate a ``Generator``.
|
|
31
|
+
|
|
32
|
+
.. warning::
|
|
33
|
+
|
|
34
|
+
No Compatibility Guarantee.
|
|
35
|
+
|
|
36
|
+
``Generator`` does not provide a version compatibility guarantee. In
|
|
37
|
+
particular, as better algorithms evolve the bit stream may change.
|
|
38
|
+
|
|
39
|
+
Parameters
|
|
40
|
+
----------
|
|
41
|
+
bit_generator : BitGenerator
|
|
42
|
+
BitGenerator to use as the core generator.
|
|
43
|
+
|
|
44
|
+
Notes
|
|
45
|
+
-----
|
|
46
|
+
In addition to the distribution-specific arguments, each ``Generator``
|
|
47
|
+
method takes a keyword argument `size` that defaults to ``None``. If
|
|
48
|
+
`size` is ``None``, then a single value is generated and returned. If
|
|
49
|
+
`size` is an integer, then a 1-D array filled with generated values is
|
|
50
|
+
returned. If `size` is a tuple, then an array with that shape is
|
|
51
|
+
filled and returned.
|
|
52
|
+
|
|
53
|
+
The Python stdlib module `random` contains pseudo-random number generator
|
|
54
|
+
with a number of methods that are similar to the ones available in
|
|
55
|
+
``Generator``. It uses Mersenne Twister, and this bit generator can
|
|
56
|
+
be accessed using ``MT19937``. ``Generator``, besides being
|
|
57
|
+
Dask-aware, has the advantage that it provides a much larger number
|
|
58
|
+
of probability distributions to choose from.
|
|
59
|
+
|
|
60
|
+
All ``Generator`` methods are identical to ``np.random.Generator`` except
|
|
61
|
+
that they also take a `chunks=` keyword argument.
|
|
62
|
+
|
|
63
|
+
``Generator`` does not guarantee parity in the generated numbers
|
|
64
|
+
with any third party library. In particular, numbers generated by
|
|
65
|
+
`Dask` and `NumPy` will differ even if they use the same seed.
|
|
66
|
+
|
|
67
|
+
Examples
|
|
68
|
+
--------
|
|
69
|
+
>>> from numpy.random import PCG64
|
|
70
|
+
>>> from dask_array.random import Generator
|
|
71
|
+
>>> rng = Generator(PCG64())
|
|
72
|
+
>>> rng.standard_normal().compute() # doctest: +SKIP
|
|
73
|
+
array(0.44595957) # random
|
|
74
|
+
|
|
75
|
+
See Also
|
|
76
|
+
--------
|
|
77
|
+
default_rng : Recommended constructor for `Generator`.
|
|
78
|
+
np.random.Generator
|
|
79
|
+
"""
|
|
80
|
+
|
|
81
|
+
def __init__(self, bit_generator):
|
|
82
|
+
self._bit_generator = bit_generator
|
|
83
|
+
|
|
84
|
+
def __str__(self):
|
|
85
|
+
return f"{self.__class__.__name__}({self._bit_generator.__class__.__name__})"
|
|
86
|
+
|
|
87
|
+
@property
|
|
88
|
+
def _backend_name(self):
|
|
89
|
+
# Assumes typename(self._RandomState) starts with an
|
|
90
|
+
# array-library name (e.g. "numpy" or "cupy")
|
|
91
|
+
return typename(self._bit_generator).split(".")[0]
|
|
92
|
+
|
|
93
|
+
@property
|
|
94
|
+
def _backend(self):
|
|
95
|
+
# Assumes `self._backend_name` is an importable
|
|
96
|
+
# array-library name (e.g. "numpy" or "cupy")
|
|
97
|
+
return importlib.import_module(self._backend_name)
|
|
98
|
+
|
|
99
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
100
|
+
def beta(self, a, b, size=None, chunks="auto", **kwargs):
|
|
101
|
+
return _wrap_func(self, "beta", a, b, size=size, chunks=chunks, **kwargs)
|
|
102
|
+
|
|
103
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
104
|
+
def binomial(self, n, p, size=None, chunks="auto", **kwargs):
|
|
105
|
+
return _wrap_func(self, "binomial", n, p, size=size, chunks=chunks, **kwargs)
|
|
106
|
+
|
|
107
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
108
|
+
def chisquare(self, df, size=None, chunks="auto", **kwargs):
|
|
109
|
+
return _wrap_func(self, "chisquare", df, size=size, chunks=chunks, **kwargs)
|
|
110
|
+
|
|
111
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
112
|
+
def choice(
|
|
113
|
+
self,
|
|
114
|
+
a,
|
|
115
|
+
size=None,
|
|
116
|
+
replace=True,
|
|
117
|
+
p=None,
|
|
118
|
+
axis=0,
|
|
119
|
+
shuffle=True,
|
|
120
|
+
chunks="auto",
|
|
121
|
+
):
|
|
122
|
+
from ._choice import RandomChoiceGenerator, _choice_validate_params
|
|
123
|
+
|
|
124
|
+
(
|
|
125
|
+
a_val,
|
|
126
|
+
a_expr,
|
|
127
|
+
size,
|
|
128
|
+
replace,
|
|
129
|
+
p_expr,
|
|
130
|
+
axis,
|
|
131
|
+
chunks,
|
|
132
|
+
meta,
|
|
133
|
+
) = _choice_validate_params(self, a, size, replace, p, axis, chunks)
|
|
134
|
+
|
|
135
|
+
return new_collection(
|
|
136
|
+
RandomChoiceGenerator(
|
|
137
|
+
a_val,
|
|
138
|
+
a_expr,
|
|
139
|
+
chunks,
|
|
140
|
+
meta,
|
|
141
|
+
self._bit_generator,
|
|
142
|
+
replace,
|
|
143
|
+
p_expr,
|
|
144
|
+
axis,
|
|
145
|
+
shuffle,
|
|
146
|
+
)
|
|
147
|
+
)
|
|
148
|
+
|
|
149
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
150
|
+
def exponential(self, scale=1.0, size=None, chunks="auto", **kwargs):
|
|
151
|
+
return _wrap_func(self, "exponential", scale, size=size, chunks=chunks, **kwargs)
|
|
152
|
+
|
|
153
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
154
|
+
def f(self, dfnum, dfden, size=None, chunks="auto", **kwargs):
|
|
155
|
+
return _wrap_func(self, "f", dfnum, dfden, size=size, chunks=chunks, **kwargs)
|
|
156
|
+
|
|
157
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
158
|
+
def gamma(self, shape, scale=1.0, size=None, chunks="auto", **kwargs):
|
|
159
|
+
return _wrap_func(self, "gamma", shape, scale, size=size, chunks=chunks, **kwargs)
|
|
160
|
+
|
|
161
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
162
|
+
def geometric(self, p, size=None, chunks="auto", **kwargs):
|
|
163
|
+
return _wrap_func(self, "geometric", p, size=size, chunks=chunks, **kwargs)
|
|
164
|
+
|
|
165
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
166
|
+
def gumbel(self, loc=0.0, scale=1.0, size=None, chunks="auto", **kwargs):
|
|
167
|
+
return _wrap_func(self, "gumbel", loc, scale, size=size, chunks=chunks, **kwargs)
|
|
168
|
+
|
|
169
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
170
|
+
def hypergeometric(self, ngood, nbad, nsample, size=None, chunks="auto", **kwargs):
|
|
171
|
+
return _wrap_func(
|
|
172
|
+
self,
|
|
173
|
+
"hypergeometric",
|
|
174
|
+
ngood,
|
|
175
|
+
nbad,
|
|
176
|
+
nsample,
|
|
177
|
+
size=size,
|
|
178
|
+
chunks=chunks,
|
|
179
|
+
**kwargs,
|
|
180
|
+
)
|
|
181
|
+
|
|
182
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
183
|
+
def integers(
|
|
184
|
+
self,
|
|
185
|
+
low,
|
|
186
|
+
high=None,
|
|
187
|
+
size=None,
|
|
188
|
+
dtype=np.int64,
|
|
189
|
+
endpoint=False,
|
|
190
|
+
chunks="auto",
|
|
191
|
+
**kwargs,
|
|
192
|
+
):
|
|
193
|
+
return _wrap_func(
|
|
194
|
+
self,
|
|
195
|
+
"integers",
|
|
196
|
+
low,
|
|
197
|
+
high=high,
|
|
198
|
+
size=size,
|
|
199
|
+
dtype=dtype,
|
|
200
|
+
endpoint=endpoint,
|
|
201
|
+
chunks=chunks,
|
|
202
|
+
**kwargs,
|
|
203
|
+
)
|
|
204
|
+
|
|
205
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
206
|
+
def laplace(self, loc=0.0, scale=1.0, size=None, chunks="auto", **kwargs):
|
|
207
|
+
return _wrap_func(self, "laplace", loc, scale, size=size, chunks=chunks, **kwargs)
|
|
208
|
+
|
|
209
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
210
|
+
def logistic(self, loc=0.0, scale=1.0, size=None, chunks="auto", **kwargs):
|
|
211
|
+
return _wrap_func(self, "logistic", loc, scale, size=size, chunks=chunks, **kwargs)
|
|
212
|
+
|
|
213
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
214
|
+
def lognormal(self, mean=0.0, sigma=1.0, size=None, chunks="auto", **kwargs):
|
|
215
|
+
return _wrap_func(self, "lognormal", mean, sigma, size=size, chunks=chunks, **kwargs)
|
|
216
|
+
|
|
217
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
218
|
+
def logseries(self, p, size=None, chunks="auto", **kwargs):
|
|
219
|
+
return _wrap_func(self, "logseries", p, size=size, chunks=chunks, **kwargs)
|
|
220
|
+
|
|
221
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
222
|
+
def multinomial(self, n, pvals, size=None, chunks="auto", **kwargs):
|
|
223
|
+
return _wrap_func(
|
|
224
|
+
self,
|
|
225
|
+
"multinomial",
|
|
226
|
+
n,
|
|
227
|
+
pvals,
|
|
228
|
+
size=size,
|
|
229
|
+
chunks=chunks,
|
|
230
|
+
extra_chunks=((len(pvals),),),
|
|
231
|
+
**kwargs,
|
|
232
|
+
)
|
|
233
|
+
|
|
234
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
235
|
+
def multivariate_hypergeometric(self, colors, nsample, size=None, method="marginals", chunks="auto", **kwargs):
|
|
236
|
+
return _wrap_func(
|
|
237
|
+
self,
|
|
238
|
+
"multivariate_hypergeometric",
|
|
239
|
+
colors,
|
|
240
|
+
nsample,
|
|
241
|
+
size=size,
|
|
242
|
+
method=method,
|
|
243
|
+
chunks=chunks,
|
|
244
|
+
**kwargs,
|
|
245
|
+
)
|
|
246
|
+
|
|
247
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
248
|
+
def negative_binomial(self, n, p, size=None, chunks="auto", **kwargs):
|
|
249
|
+
return _wrap_func(self, "negative_binomial", n, p, size=size, chunks=chunks, **kwargs)
|
|
250
|
+
|
|
251
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
252
|
+
def noncentral_chisquare(self, df, nonc, size=None, chunks="auto", **kwargs):
|
|
253
|
+
return _wrap_func(self, "noncentral_chisquare", df, nonc, size=size, chunks=chunks, **kwargs)
|
|
254
|
+
|
|
255
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
256
|
+
def noncentral_f(self, dfnum, dfden, nonc, size=None, chunks="auto", **kwargs):
|
|
257
|
+
return _wrap_func(self, "noncentral_f", dfnum, dfden, nonc, size=size, chunks=chunks, **kwargs)
|
|
258
|
+
|
|
259
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
260
|
+
def normal(self, loc=0.0, scale=1.0, size=None, chunks="auto", **kwargs):
|
|
261
|
+
return _wrap_func(self, "normal", loc, scale, size=size, chunks=chunks, **kwargs)
|
|
262
|
+
|
|
263
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
264
|
+
def pareto(self, a, size=None, chunks="auto", **kwargs):
|
|
265
|
+
return _wrap_func(self, "pareto", a, size=size, chunks=chunks, **kwargs)
|
|
266
|
+
|
|
267
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
268
|
+
def permutation(self, x):
|
|
269
|
+
from dask_array.slicing._utils import shuffle_slice
|
|
270
|
+
|
|
271
|
+
from ._utils import _shuffle
|
|
272
|
+
|
|
273
|
+
if self._backend_name == "cupy":
|
|
274
|
+
raise NotImplementedError(
|
|
275
|
+
"`Generator.permutation` not supported for cupy-backed "
|
|
276
|
+
"Generator objects. Use the 'numpy' array backend to "
|
|
277
|
+
"call `dask.array.random.default_rng`, or pass in "
|
|
278
|
+
" `numpy.random.PCG64()`."
|
|
279
|
+
)
|
|
280
|
+
|
|
281
|
+
if isinstance(x, numbers.Number):
|
|
282
|
+
x = arange(x, chunks="auto")
|
|
283
|
+
|
|
284
|
+
index = self._backend.arange(len(x))
|
|
285
|
+
_shuffle(self._bit_generator, index)
|
|
286
|
+
return shuffle_slice(x, index)
|
|
287
|
+
|
|
288
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
289
|
+
def poisson(self, lam=1.0, size=None, chunks="auto", **kwargs):
|
|
290
|
+
return _wrap_func(self, "poisson", lam, size=size, chunks=chunks, **kwargs)
|
|
291
|
+
|
|
292
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
293
|
+
def power(self, a, size=None, chunks="auto", **kwargs):
|
|
294
|
+
return _wrap_func(self, "power", a, size=size, chunks=chunks, **kwargs)
|
|
295
|
+
|
|
296
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
297
|
+
def random(self, size=None, dtype=np.float64, out=None, chunks="auto", **kwargs):
|
|
298
|
+
return _wrap_func(self, "random", size=size, dtype=dtype, out=out, chunks=chunks, **kwargs)
|
|
299
|
+
|
|
300
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
301
|
+
def rayleigh(self, scale=1.0, size=None, chunks="auto", **kwargs):
|
|
302
|
+
return _wrap_func(self, "rayleigh", scale, size=size, chunks=chunks, **kwargs)
|
|
303
|
+
|
|
304
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
305
|
+
def standard_cauchy(self, size=None, chunks="auto", **kwargs):
|
|
306
|
+
return _wrap_func(self, "standard_cauchy", size=size, chunks=chunks, **kwargs)
|
|
307
|
+
|
|
308
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
309
|
+
def standard_exponential(self, size=None, chunks="auto", **kwargs):
|
|
310
|
+
return _wrap_func(self, "standard_exponential", size=size, chunks=chunks, **kwargs)
|
|
311
|
+
|
|
312
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
313
|
+
def standard_gamma(self, shape, size=None, chunks="auto", **kwargs):
|
|
314
|
+
return _wrap_func(self, "standard_gamma", shape, size=size, chunks=chunks, **kwargs)
|
|
315
|
+
|
|
316
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
317
|
+
def standard_normal(self, size=None, chunks="auto", **kwargs):
|
|
318
|
+
return _wrap_func(self, "standard_normal", size=size, chunks=chunks, **kwargs)
|
|
319
|
+
|
|
320
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
321
|
+
def standard_t(self, df, size=None, chunks="auto", **kwargs):
|
|
322
|
+
return _wrap_func(self, "standard_t", df, size=size, chunks=chunks, **kwargs)
|
|
323
|
+
|
|
324
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
325
|
+
def triangular(self, left, mode, right, size=None, chunks="auto", **kwargs):
|
|
326
|
+
return _wrap_func(self, "triangular", left, mode, right, size=size, chunks=chunks, **kwargs)
|
|
327
|
+
|
|
328
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
329
|
+
def uniform(self, low=0.0, high=1.0, size=None, chunks="auto", **kwargs):
|
|
330
|
+
return _wrap_func(self, "uniform", low, high, size=size, chunks=chunks, **kwargs)
|
|
331
|
+
|
|
332
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
333
|
+
def vonmises(self, mu, kappa, size=None, chunks="auto", **kwargs):
|
|
334
|
+
return _wrap_func(self, "vonmises", mu, kappa, size=size, chunks=chunks, **kwargs)
|
|
335
|
+
|
|
336
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
337
|
+
def wald(self, mean, scale, size=None, chunks="auto", **kwargs):
|
|
338
|
+
return _wrap_func(self, "wald", mean, scale, size=size, chunks=chunks, **kwargs)
|
|
339
|
+
|
|
340
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
341
|
+
def weibull(self, a, size=None, chunks="auto", **kwargs):
|
|
342
|
+
return _wrap_func(self, "weibull", a, size=size, chunks=chunks, **kwargs)
|
|
343
|
+
|
|
344
|
+
@derived_from(np.random.Generator, skipblocks=1)
|
|
345
|
+
def zipf(self, a, size=None, chunks="auto", **kwargs):
|
|
346
|
+
return _wrap_func(self, "zipf", a, size=size, chunks=chunks, **kwargs)
|
|
347
|
+
|
|
348
|
+
|
|
349
|
+
def default_rng(seed=None):
|
|
350
|
+
"""
|
|
351
|
+
Construct a new Generator with the default BitGenerator (PCG64).
|
|
352
|
+
|
|
353
|
+
Parameters
|
|
354
|
+
----------
|
|
355
|
+
seed : {None, int, array_like[ints], SeedSequence, BitGenerator, Generator}, optional
|
|
356
|
+
A seed to initialize the `BitGenerator`. If None, then fresh,
|
|
357
|
+
unpredictable entropy will be pulled from the OS. If an ``int`` or
|
|
358
|
+
``array_like[ints]`` is passed, then it will be passed to
|
|
359
|
+
`SeedSequence` to derive the initial `BitGenerator` state. One may
|
|
360
|
+
also pass in a `SeedSequence` instance.
|
|
361
|
+
Additionally, when passed a `BitGenerator`, it will be wrapped by
|
|
362
|
+
`Generator`. If passed a `Generator`, it will be returned unaltered.
|
|
363
|
+
|
|
364
|
+
Returns
|
|
365
|
+
-------
|
|
366
|
+
Generator
|
|
367
|
+
The initialized generator object.
|
|
368
|
+
|
|
369
|
+
Notes
|
|
370
|
+
-----
|
|
371
|
+
If ``seed`` is not a `BitGenerator` or a `Generator`, a new
|
|
372
|
+
`BitGenerator` is instantiated. This function does not manage a default
|
|
373
|
+
global instance.
|
|
374
|
+
|
|
375
|
+
Examples
|
|
376
|
+
--------
|
|
377
|
+
``default_rng`` is the recommended constructor for the random number
|
|
378
|
+
class ``Generator``. Here are several ways we can construct a random
|
|
379
|
+
number generator using ``default_rng`` and the ``Generator`` class.
|
|
380
|
+
|
|
381
|
+
Here we use ``default_rng`` to generate a random float:
|
|
382
|
+
|
|
383
|
+
>>> import dask_array as da
|
|
384
|
+
>>> rng = da.random.default_rng(12345)
|
|
385
|
+
>>> print(rng)
|
|
386
|
+
Generator(PCG64)
|
|
387
|
+
>>> rfloat = rng.random().compute()
|
|
388
|
+
>>> rfloat
|
|
389
|
+
array(0.86999885)
|
|
390
|
+
>>> type(rfloat)
|
|
391
|
+
<class 'numpy.ndarray'>
|
|
392
|
+
|
|
393
|
+
Here we use ``default_rng`` to generate 3 random integers between 0
|
|
394
|
+
(inclusive) and 10 (exclusive):
|
|
395
|
+
|
|
396
|
+
>>> import dask_array as da
|
|
397
|
+
>>> rng = da.random.default_rng(12345)
|
|
398
|
+
>>> rints = rng.integers(low=0, high=10, size=3).compute()
|
|
399
|
+
>>> rints
|
|
400
|
+
array([2, 8, 7])
|
|
401
|
+
>>> type(rints[0])
|
|
402
|
+
<class 'numpy.int64'>
|
|
403
|
+
|
|
404
|
+
Here we specify a seed so that we have reproducible results:
|
|
405
|
+
|
|
406
|
+
>>> import dask_array as da
|
|
407
|
+
>>> rng = da.random.default_rng(seed=42)
|
|
408
|
+
>>> print(rng)
|
|
409
|
+
Generator(PCG64)
|
|
410
|
+
>>> arr1 = rng.random((3, 3)).compute()
|
|
411
|
+
>>> arr1
|
|
412
|
+
array([[0.91674416, 0.91098667, 0.8765925 ],
|
|
413
|
+
[0.30931841, 0.95465607, 0.17509458],
|
|
414
|
+
[0.99662814, 0.75203348, 0.15038118]])
|
|
415
|
+
|
|
416
|
+
If we exit and restart our Python interpreter, we'll see that we
|
|
417
|
+
generate the same random numbers again:
|
|
418
|
+
|
|
419
|
+
>>> import dask_array as da
|
|
420
|
+
>>> rng = da.random.default_rng(seed=42)
|
|
421
|
+
>>> arr2 = rng.random((3, 3)).compute()
|
|
422
|
+
>>> arr2
|
|
423
|
+
array([[0.91674416, 0.91098667, 0.8765925 ],
|
|
424
|
+
[0.30931841, 0.95465607, 0.17509458],
|
|
425
|
+
[0.99662814, 0.75203348, 0.15038118]])
|
|
426
|
+
|
|
427
|
+
See Also
|
|
428
|
+
--------
|
|
429
|
+
np.random.default_rng
|
|
430
|
+
"""
|
|
431
|
+
if hasattr(seed, "capsule"):
|
|
432
|
+
# We are passed a BitGenerator, so just wrap it
|
|
433
|
+
return Generator(seed)
|
|
434
|
+
elif isinstance(seed, Generator):
|
|
435
|
+
# Pass through a Generator
|
|
436
|
+
return seed
|
|
437
|
+
elif hasattr(seed, "bit_generator"):
|
|
438
|
+
# a Generator. Just not ours
|
|
439
|
+
return Generator(seed.bit_generator)
|
|
440
|
+
# Otherwise, use the backend-default BitGenerator
|
|
441
|
+
return Generator(array_creation_dispatch.default_bit_generator(seed))
|