-
-
Notifications
You must be signed in to change notification settings - Fork 1.8k
Description
Apparently dask.dataframe.merge fails under some circumstances if the common columns have different data types while being otherwise identical. E.g. merge(df1[id1, id2, value1], df2[id1, id2, value2], on=['id1', 'id2'], how='left') would result in a table where some or all values of value2 are NaN if df1.id1.dtype is float64 and df2.id1.dtype is int64. The same merge operation would succeed if all corresponding id columns were either floats or ints. This behavior depends on the number of chunks in the dataframes.
Would it make sense to throw a warning if merge columns have different dtypes?
Example
In the example below the function test_dtype_merge produces some sample dataframes and performs a merge operation for different dtypes and npartitions. The merge has failed if any of the value columns from the right-hand table are nan. This is the case if the dtypes are different and the right table has npartitions > 1 (#NaN: 3 in the printed results below).
import dask
import numpy as np
import pandas as pd
import dask.dataframe as dd
list_dtypes = [(np.float64, np.float64, np.float64, np.float64),
(np.float64, np.int64, np.float64, np.int64),
(np.float64, np.float64, np.float64, np.int64),
(np.float64, np.int64, np.float64, np.float64)]
def test_dtype_merge(npartitions_left, npartitions_right):
print('npartitions_left=%s, npartitions_right=%s'%(npartitions_left,
npartitions_right))
df = pd.DataFrame({'id_1': [0,1] * 3, 'id_2': [0] * 3 + [1] * 3,
'id_3': [0, 0, 1, 1, 2, 2], 'val_1': np.random.random(6)})
df_join = df[['id_1', 'id_2']].drop_duplicates()
df_join['val_2'] = np.random.random(len(df_join))
df = dd.from_pandas(df, npartitions=npartitions_left)
df_join = dd.from_pandas(df_join, npartitions=npartitions_right)
on_cols = ['id_1', 'id_2']
for t1, t2, t3, t4 in list_dtypes:
df_join.id_1 = df_join.id_1.astype(t1)
df_join.id_2 = df_join.id_2.astype(t2)
df.id_1 = df.id_1.astype(t3)
df.id_2 = df.id_2.astype(t4)
df_result = dd.merge(df, df_join, on=on_cols,
how='left').compute()
print('#NaN: %d'%df_result.val_2.isna().sum(), t1, t2, t3, t4)In [3]: test_dtype_merge(1, 2)
...:
npartitions_left=1, npartitions_right=2
#NaN: 0 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 3 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 3 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.float64'>
In [4]: test_dtype_merge(1, 1)
npartitions_left=1, npartitions_right=1
#NaN: 0 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.float64'>
In [5]: test_dtype_merge(2, 1)
npartitions_left=2, npartitions_right=1
#NaN: 0 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.float64'>
In [6]: test_dtype_merge(2, 2)
npartitions_left=2, npartitions_right=2
#NaN: 0 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 3 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 3 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.float64'>
In [7]: test_dtype_merge(3, 1)
npartitions_left=3, npartitions_right=1
#NaN: 0 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.float64'>
In [8]: test_dtype_merge(1, 3)
npartitions_left=1, npartitions_right=3
#NaN: 0 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'>
#NaN: 0 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 3 <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.float64'> <class 'numpy.int64'>
#NaN: 3 <class 'numpy.float64'> <class 'numpy.int64'> <class 'numpy.float64'> <class 'numpy.float64'>