[BUG][GPU Error Bug] "SELECT ((<string>)!=(<column>)) FROM <tables>" brings Error
What happened:
"SELECT ((<string>)!=(<column>)) FROM <tables>" brings error, when using GPU.
However it is able to output result, when using CPU.
What you expected to happen:
It will not bring error, when using GPU.
Minimal Complete Verifiable Example:
import pandas as pd
import dask.dataframe as dd
from dask_sql import Context
c = Context()
df0 = pd.DataFrame({
'c0': [6464.0000]
})
t0 = dd.from_pandas(df0, npartitions=1)
c.create_table('t0', t0, gpu=False)
c.create_table('t0_gpu', t0, gpu=True)
df1 = pd.DataFrame({
'c0': [''],
})
t1 = dd.from_pandas(df1, npartitions=1)
c.create_table('t1', t1, gpu=False)
c.create_table('t1_gpu', t1, gpu=True)
df3 = pd.DataFrame({
'c0': [0.6918491708906861],
})
t3 = dd.from_pandas(df3, npartitions=1)
c.create_table('t3', t3, gpu=False)
c.create_table('t3_gpu', t3, gpu=True)
print('CPU Result:')
result1= c.sql("SELECT (('A')!=(t1.c0)) FROM t0, t1, t3").compute()
print(result1)
print('GPU Result:')
result2= c.sql("SELECT (('A')!=(t1_gpu.c0)) FROM t0_gpu, t1_gpu, t3_gpu").compute()
print(result2)
Result:
INFO:numba.cuda.cudadrv.driver:init
CPU Result:
Utf8("A") != t1.c0
0 True
GPU Result:
Traceback (most recent call last):
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask/dataframe/utils.py", line 193, in raise_on_meta_error
yield
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask/dataframe/core.py", line 6470, in elemwise
meta = partial_by_order(*parts, function=op, other=other)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask/utils.py", line 1327, in partial_by_order
return function(*args2, **kwargs)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/mixins/mixin_factory.py", line 11, in wrapper
return method(self, *args1, *args2, **kwargs1, **kwargs2)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/indexed_frame.py", line 3350, in _binaryop
ColumnAccessor(type(self)._colwise_binop(operands, op)),
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/nvtx/nvtx.py", line 101, in inner
result = func(*args, **kwargs)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/frame.py", line 1750, in _colwise_binop
else getattr(operator, fn)(left_column, right_column)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/mixins/mixin_factory.py", line 11, in wrapper
return method(self, *args1, *args2, **kwargs1, **kwargs2)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/column/numerical.py", line 219, in _binaryop
if (other := self._wrap_binop_normalization(other)) is NotImplemented:
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/column/column.py", line 607, in _wrap_binop_normalization
return self.normalize_binop_value(other)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/column/numerical.py", line 312, in normalize_binop_value
common_dtype = np.result_type(self.dtype, other)
File "<__array_function__ internals>", line 200, in result_type
TypeError: data type 'A' not understood
The above exception was the direct cause of the following exception:
Traceback (most recent call last):
File "/tmp/bug.py", line 36, in <module>
result2= c.sql("SELECT (('A')!=(t1_gpu.c0)) FROM t0_gpu, t1_gpu, t3_gpu").compute()
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask_sql/context.py", line 513, in sql
return self._compute_table_from_rel(rel, return_futures)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask_sql/context.py", line 839, in _compute_table_from_rel
dc = RelConverter.convert(rel, context=self)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask_sql/physical/rel/convert.py", line 61, in convert
df = plugin_instance.convert(rel, context=context)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask_sql/physical/rel/logical/project.py", line 57, in convert
new_columns[random_name] = RexConverter.convert(
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask_sql/physical/rex/convert.py", line 74, in convert
df = plugin_instance.convert(rel, rex, dc, context=context)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask_sql/physical/rex/core/call.py", line 1129, in convert
return operation(*operands, **kwargs)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask_sql/physical/rex/core/call.py", line 77, in __call__
return self.f(*operands, **kwargs)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask_sql/physical/rex/core/call.py", line 140, in reduce
return reduce(partial(self.operation, **kwargs), operands)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask/dataframe/core.py", line 1926, in <lambda>
return lambda self, other: elemwise(op, self, other)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask/dataframe/core.py", line 6469, in elemwise
with raise_on_meta_error(funcname(op)):
File "/opt/conda/envs/rapids/lib/python3.10/contextlib.py", line 153, in __exit__
self.gen.throw(typ, value, traceback)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask/dataframe/utils.py", line 214, in raise_on_meta_error
raise ValueError(msg) from e
ValueError: Metadata inference failed in `ne`.
Original error is below:
------------------------
TypeError("data type 'A' not understood")
Traceback:
---------
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask/dataframe/utils.py", line 193, in raise_on_meta_error
yield
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask/dataframe/core.py", line 6470, in elemwise
meta = partial_by_order(*parts, function=op, other=other)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/dask/utils.py", line 1327, in partial_by_order
return function(*args2, **kwargs)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/mixins/mixin_factory.py", line 11, in wrapper
return method(self, *args1, *args2, **kwargs1, **kwargs2)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/indexed_frame.py", line 3350, in _binaryop
ColumnAccessor(type(self)._colwise_binop(operands, op)),
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/nvtx/nvtx.py", line 101, in inner
result = func(*args, **kwargs)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/frame.py", line 1750, in _colwise_binop
else getattr(operator, fn)(left_column, right_column)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/mixins/mixin_factory.py", line 11, in wrapper
return method(self, *args1, *args2, **kwargs1, **kwargs2)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/column/numerical.py", line 219, in _binaryop
if (other := self._wrap_binop_normalization(other)) is NotImplemented:
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/column/column.py", line 607, in _wrap_binop_normalization
return self.normalize_binop_value(other)
File "/opt/conda/envs/rapids/lib/python3.10/site-packages/cudf/core/column/numerical.py", line 312, in normalize_binop_value
common_dtype = np.result_type(self.dtype, other)
File "<__array_function__ internals>", line 200, in result_type
Anything else we need to know?:
Environment:
- dask-sql version: 2023.6.0
- Python version: Python 3.10.11
- Operating System: Ubuntu22.04
- Install method (conda, pip, source): Docker deploy by https://hub.docker.com/layers/rapidsai/rapidsai-dev/23.06-cuda11.8-devel-ubuntu22.04-py3.10/images/sha256-cfbb61fdf7227b090a435a2e758114f3f1c31872ed8dbd96e5e564bb5fd184a7?context=explore
I have also encountered the same situation. Can the developer reply?
It looks like the underlying issue here is that dask-cuDF is failing because we're trying to do a binop between a float column and a string scalar:
import cudf
import dask_cudf
s = cudf.Series(["A"])
ds = dask_cudf.from_cudf(s, npartitions=1)
ds != "A"
On dask-sql's end, it seems like the specific combination of cross joins planned when selecting from all 3 tables in your reproducer is making it such that t1_gpu.c0 is registered a float column rather than the string/object column I would've expected - will dig into this deeper to get a sense of what's happening
Okay seems like the issue here is handling of duplicate column names on GPU, it looks like the cross join itself fails with 3 tables:
import pandas as pd
import dask.dataframe as dd
from dask_sql import Context
c = Context()
c.create_table('df1', pd.DataFrame({"a": [1]}), gpu=True)
c.create_table('df2', pd.DataFrame({"a": [2]}), gpu=True)
c.create_table('df3', pd.DataFrame({"a": [3]}), gpu=True)
query = "SELECT * FROM df1, df2, df3"
explain = c.explain(query)
res = c.sql(query) # AssertionError
There's a good chance that this could be related to #1133 and potentially resolved with #1134, I can look into reviving that PR and giving it a try here
EDIT:
Tried #1134 without much luck, looks like the cross join issue is independent of CPU/GPU as I get errors with the above block even when gpu=False, think it just ends up failing on GPU for your particular reproducer because dask-cuDF doesn't support float column / string scalar binops whereas Dask CPU does
Trying out your reproducer with #1250, I'm now able to get things passing:
CPU Result:
Utf8("A") != t1.c0
0 True
GPU Result:
Utf8("A") != t1_gpu.c0
0 True
Would you mind giving this branch a look on your end?
The bug came up at dask-sql version: 2023.6.0. After my verification, the bug was fixed at https://github.com/dask-contrib/dask-sql/pull/1250 . Thanks for your work.