dask: svd_compressed() fails for complex input

What happened: Complex singular values and vectors of svd_compressed() are different from standard np.linalg.svd.

What you expected to happen: Complex singular values and vectors of svd_compressed() are similar (within some uncertainties) to standard np.linalg.svd.

Minimal Complete Verifiable Example:

import numpy as np
import xarray as xr
import dask.array as da
from matplotlib.pyplot import imshow

data = xr.tutorial.open_dataset(
    'air_temperature',
    chunks={'lat': 25, 'lon': 25, 'time': -1}
)
temp = data.air
temp = temp.stack(x=('lat', 'lon')).compute()
temp -= temp.mean('time')
# artificial complexification of data
temp = temp + (1j * 0.1 * temp**2)
kernel = np.dot(temp.conj().T, temp) / temp.shape[0]

# standard SVD
u, s, vt = np.linalg.svd(kernel, full_matrices=False)
# dask SVD 
dask_kernel = da.from_array(kernel)
k = 100
dsvd = da.linalg.svd_compressed(dask_kernel, k)
u2, s2, vt2 = (x.compute() for x in dsvd)

np.allclose(s[:100], s2)  # False
np.allclose(vt[:100], vt2)  # False

# visual check: look at first singular vector
v = vt.reshape((temp.shape[1],) + data.air.shape[1:])
v2 = vt2.reshape((k,) + data.air.shape[1:])

# imaginary parts of first singular vector are clearly different
imshow(v[0,:,:].imag, vmin=-1e-3, vmax=1e-3)
imshow(-v2[0,:,:].imag, vmin=-1e-3, vmax=1e-3)

Anything else we need to know?:

  1. For real input, both methods yield the same results.
  2. When running svd_compressed() a warning is thrown complaining about complex being casted to real dtypes:
/home/nrieger/anaconda3/envs/xmca/lib/python3.7/site-packages/dask/array/utils.py:108: ComplexWarning: Casting complex values to real discards the imaginary part
  meta = meta.astype(dtype)

Environment:

  • Dask version: 2021.04.0
  • Python version: 3.7.10
  • Operating System: Ubuntu 20.04
  • Install method: conda

About this issue

  • Original URL
  • State: open
  • Created 3 years ago
  • Comments: 17 (12 by maintainers)

Most upvoted comments

I think I have found the bugs in svd_compressed(). The svd calculation for complex matrix is $A=U\Sigma V^\dagger$, where $U$ and $V^\dagger$ are complex matrices. The " ${}^\dagger$ " means “Hermitian transpose”, which on the case of real matrix reduces to a transpose matrix. While calculating complex matrix svd, we should use Hermitian transpose but not transpose matrix. At the source code for dask.array.linalg, I found some bugs, and I will figure out the bugs by comments. First, at the “compression_matrix()” function,

    if iterator == "power":
        for i in range(n_power_iter):
            if compute:
                mat_h = mat_h.persist()
                wait(mat_h)
            tmp = data.T.dot(mat_h) #<-should be Hermitian transpose da.conj(data.T) on the complex case
            if compute:
                tmp = tmp.persist()
                wait(tmp)
            mat_h = data.dot(tmp)
        q, _ = tsqr(mat_h)
    else:
        q, _ = tsqr(mat_h)
        for i in range(n_power_iter):
            if compute:
                q = q.persist()
                wait(q)
            q, _ = tsqr(data.T.dot(q)) #<-should be Hermitian transpose da.conj(data.T) on the complex case
            if compute:
                q = q.persist()
                wait(q)
            q, _ = tsqr(data.dot(q))

    return q.T #<-should be Hermitian transpose da.conj(q.T) on the complex case

Then, at the “svd_compressed()” function,

    a_compressed = comp.dot(a)
    v, s, u = tsqr(a_compressed.T, compute_svd=True)
    u = comp.T.dot(u.T) #<-should be  da.conj(comp.T).dot(u.T) on the complex case
    v = v.T
    u = u[:, :k]
    s = s[:k]
    v = v[:k, :]
    if coerce_signs:
        u, v = svd_flip(u, v)
    return u, s, v

Here is my code of randomized svd by the use of power iteration

randomized_svd.py
def rsvd(A:da.Array, k:int, n_oversamples=10, n_power_iter=0, seed=da.random.RandomState(seed=1234,RandomState=np.random.RandomState)):
    """
    A: MxN dask array\\
    k:rank\\
    n_oversamples+k=l << min{M,N} must be satisfied\\
    seed:default is numpy.random.RandomState(seed=1234)
    """
    
    M, N = (da.shape(A)[0], da.shape(A)[1])

    l = k + n_oversamples
    Omega = seed.normal(size=(N, l))
    Q = __power_iteration__(A, Omega, n_power_iter)
    del Omega

    if A.dtype == "complex16" or A.dtype == "complex32" or A.dtype == "complex64" \
        or A.dtype == "complex128" or  A.dtype == "complex256" or A.dtype == "complex512":
        B = da.conj(da.transpose(Q)) @ A
    else:
        B = da.transpose(Q) @ A

    u_tilde, s, vh = da.linalg.svd(B)
    del B
    u = Q @ u_tilde
    del u_tilde
    
    return u[:,:k], s[:k], vh[:k,:]

def __power_iteration__(A, Omega, n_power_iter:int):
    
    Y = A @ Omega
    if A.dtype == "complex16" or A.dtype == "complex32" or A.dtype == "complex64" \
        or A.dtype == "complex128" or  A.dtype == "complex256" or A.dtype == "complex512":
        for q in range(n_power_iter):
            Y = A @ (da.conj(da.transpose(A)) @ Y)
    else:
        for q in range(n_power_iter):
            Y = A @ (da.transpose(A) @ Y)
    Q, _ = da.linalg.tsqr(Y)

    return Q

Here is the test code, which compares to the full svd “cupy.linalg.svd” and “dask.array.linalg.svd_compressed”

test.py
import sys
import time
import numpy as np
from opt_einsum import contract

from dask.distributed import Client
from dask_cuda import LocalCUDACluster
from dask_cuda.initialize import initialize
from dask.utils import parse_bytes
from dask.distributed import performance_report
from dask.distributed import wait
from dask.distributed import get_task_stream

import cupy
import rmm
import cudf
import dask.array as da


def setup_rmm_pool(client):
    client.run(
        cudf.set_allocator,
        pool=False,
        #initial_pool_size= parse_bytes("1GB"),
        allocator="default"
    )
    client.run(
        cupy.cuda.set_allocator,
        #rmm.rmm_cupy_allocator,
        rmm.mr.set_current_device_resource(rmm.mr.ManagedMemoryResource())
    )

if __name__ == "__main__":

    initialize(create_cuda_context=True)
    
    cluster = LocalCUDACluster(local_directory="./tmp/",
                                memory_limit=None)
       
    client = Client(cluster)
    setup_rmm_pool(client)

    nprs = np.random.RandomState(seed=1234)
    rs = da.random.RandomState(seed=1234,RandomState=cupy.random.RandomState)

    SIZE = 10000
    k = 32

    b = nprs.rand(SIZE) + 1j * nprs.rand(SIZE)
    b = cupy.asarray(b, dtype=cupy.complex128)
    a = contract("i,j->ij",b,b) * 10
    a = cupy.exp(1.2*a)

    #full svd
    t0 = time.time()
    u,s,vh = cupy.linalg.svd(a)
    t1=time.time()
    del u,vh
    print("full svd time: {:.2f} s".format(t1-t0))
    print(s[:k])

    a = da.from_array(a, chunks=(5000,5000))

    #rsvd
    from randomized_svd import rsvd
    t0 = time.time()
    u,s,vh = rsvd(a, k=k, seed=rs)
    u,s,vh = da.compute(u,s,vh) 
    t1 = time.time()
    print("rsvd time: {:.2f} s".format(t1-t0))
    print(s)

    u  = da.from_array(u,chunks=(SIZE,k))
    vh = da.from_array(vh,chunks=(k,SIZE))
    b = da.einsum("ij,j,jk->ik",u,s,vh)
    err = a - b
    tr = da.trace(err).compute()
    print("trace:{:19.12e}{:+19.12e}".format(tr.real,tr.imag))
    materr = da.linalg.norm(err).compute()
    materr = float(materr / da.max(da.abs(a)))
    print("err:{:18.12e}".format(materr))


    #dask.array.linalg.svd_compress
    t0 = time.time()
    u,s,vh = da.linalg.svd_compressed(a, k=k, seed=rs)
    u,s,vh = da.compute(u,s,vh) 
    t1 = time.time()
    print("da.linalg.svd_compress time: {:.2f} s".format(t1-t0))
    print(s)

    u  = da.from_array(u,chunks=(SIZE,k))
    vh = da.from_array(vh,chunks=(k,SIZE))
    b = da.einsum("ij,j,jk->ik",u,s,vh)
    err = a - b
    tr = da.trace(err).compute()
    print("trace:{:19.12e}{:+19.12e}".format(tr.real,tr.imag))
    materr = da.linalg.norm(err).compute()
    materr = float(materr / da.max(da.abs(a)))
    print("err:{:18.12e}".format(materr))

    sys.exit(0)

The results are:

full svd by using cupy.linalg.svd()
full svd time: 86.01 s
[2.58296767e+07 1.05558179e+07 3.38844060e+06 9.88898912e+05
 2.59627169e+05 6.74255883e+04 1.89209066e+04 5.94213692e+03
 2.20050461e+03 8.92638939e+02 3.50820653e+02 1.26915979e+02
 4.22639368e+01 1.30184354e+01 3.72038540e+00 9.90751139e-01
 2.47328075e-01 5.81214650e-02 1.29043303e-02 2.71591129e-03
 5.43455898e-04 1.03658501e-04 1.88871552e-05 3.29431051e-06
 5.51039526e-07 8.85671784e-08 1.37849358e-08 6.26629286e-09
 5.37837639e-09 5.31998482e-09 5.17301750e-09 5.07648171e-09]
ramdomized svd by using my function "rsvd()"
rsvd time: 4.72 s
[2.58296767e+07 1.05558179e+07 3.38844060e+06 9.88898912e+05
 2.59627169e+05 6.74255883e+04 1.89209066e+04 5.94213692e+03
 2.20050461e+03 8.92638939e+02 3.50820653e+02 1.26915979e+02
 4.22639368e+01 1.30184354e+01 3.72038540e+00 9.90751139e-01
 2.47328075e-01 5.81214650e-02 1.29043303e-02 2.71591127e-03
 5.43455901e-04 1.03658517e-04 1.88872026e-05 3.29426812e-06
 5.50886046e-07 8.78125741e-08 1.03905719e-08 2.97395585e-09
 2.87520604e-09 2.74762063e-09 2.67777793e-09 2.61763907e-09]
trace:-3.837440116878e-09-2.231108248151e-09
err:4.611527374705e-13
ramdomized svd by using dask.array.linalg.svd_compressed()
da.linalg.svd_compress time: 3.34 s
[1.22753454e+07 3.51378639e+06 1.41504285e+06 2.65017653e+05
 6.59603978e+04 2.11401367e+04 5.50606266e+03 1.51125671e+03
 6.00777209e+02 2.37258157e+02 7.97001757e+01 2.92310333e+01
 8.82265661e+00 2.17949256e+00 5.76741777e-01 1.48938971e-01
 3.01250371e-02 7.56122991e-03 1.67292376e-03 3.35416451e-04
 7.11668163e-05 1.36569683e-05 2.32605468e-06 3.61926695e-07
 5.14757978e-08 6.07435046e-09 2.95743600e-09 2.66944135e-09
 2.56615446e-09 2.42686096e-09 2.35138792e-09 2.28905663e-09]
trace: 1.585523226585e+06+4.090091981828e+06
err:2.223140103552e+02

I calculated the trace and error of matrix $M-M'$, in which $M$ is the original matrix, and $$M_{ik} \simeq M^\prime_{ik}=\sum_{k=1}^{32} U_{ij} \sigma_{j} V^\dagger_{jk}.$$ The error is defined by $\text{err}=||M-M^\prime||/||M||$. The results show that my code yields a reasonable answer because the trace and error are quite small, and the large singular values are the same as full svd. By the way, the skinny matrix svd and tsqr seem to be currect because I used them in my code. But I’m not sure if there are the same bugs (at where should use Hermitian transpose, but have used tranpose matrix for complex matrix) in the code.

Just checked, the TSQR gives same results for complex matrices as numpy QR. I am not sure whether the sampling for complex matrices should also be a complex matrix.

@eric-czech: The SVD is unique (up to signs) for square, non-square, and complex matrices as long as no singular value pairs (two or more singular values with the same value exist). See Trefethen, L. N. & Bau III, D. Numerical linear algebra, vol. 50 (Siam, 1997).

I am not sure whether randomized methods such as svd_compressed provide unique solutions, as they severely depend on the sampling of the matrix. In my experience they are not unique.