forked from dask-contrib/dask-sql
-
Notifications
You must be signed in to change notification settings - Fork 0
Expand file tree
/
Copy pathjoin.py
More file actions
362 lines (309 loc) · 14.3 KB
/
join.py
File metadata and controls
362 lines (309 loc) · 14.3 KB
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
import logging
import operator
import warnings
from functools import reduce
from typing import TYPE_CHECKING, List, Tuple
import dask.dataframe as dd
from dask import config as dask_config
from dask.base import tokenize
from dask.highlevelgraph import HighLevelGraph
from dask_sql._compat import BROADCAST_JOIN_SUPPORT_WORKING
from dask_sql.datacontainer import ColumnContainer, DataContainer
from dask_sql.physical.rel.base import BaseRelPlugin
from dask_sql.physical.rel.logical.filter import filter_or_scalar
from dask_sql.physical.rex import RexConverter
from dask_sql.utils import is_cudf_type
if TYPE_CHECKING:
import dask_sql
from dask_planner.rust import Expression, LogicalPlan
logger = logging.getLogger(__name__)
class DaskJoinPlugin(BaseRelPlugin):
"""
A DaskJoin is used when (surprise) joining two tables.
SQL allows for quite complicated joins with difficult conditions.
dask/pandas only knows about equijoins on a specific column.
We use a trick, which is also used in e.g. blazingSQL:
we split the join condition into two parts:
* everything which is an equijoin
* the rest
The first part is then used for the dask merging,
whereas the second part is just applied as a filter afterwards.
This will make joining more time-consuming that is needs to be
but so far, it is the only solution...
"""
class_name = "Join"
JOIN_TYPE_MAPPING = {
"INNER": "inner",
"LEFT": "left",
"RIGHT": "right",
"FULL": "outer",
"LEFTSEMI": "leftsemi",
"LEFTANTI": "leftanti",
}
def convert(self, rel: "LogicalPlan", context: "dask_sql.Context") -> DataContainer:
# Joining is a bit more complicated, so lets do it in steps:
join = rel.join()
# 1. We now have two inputs (from left and right), so we fetch them both
dc_lhs, dc_rhs = self.assert_inputs(rel, 2, context)
cc_lhs = dc_lhs.column_container
cc_rhs = dc_rhs.column_container
# 2. dask's merge will do some smart things with columns, which have the same name
# on lhs an rhs (which also includes reordering).
# However, that will confuse our column numbering in SQL.
# So we make our life easier by converting the column names into unique names
# We will convert back in the end
cc_lhs_renamed = cc_lhs.make_unique("lhs")
cc_rhs_renamed = cc_rhs.make_unique("rhs")
dc_lhs_renamed = DataContainer(dc_lhs.df, cc_lhs_renamed)
dc_rhs_renamed = DataContainer(dc_rhs.df, cc_rhs_renamed)
df_lhs_renamed = dc_lhs_renamed.assign()
df_rhs_renamed = dc_rhs_renamed.assign()
join_type = join.getJoinType()
join_type = self.JOIN_TYPE_MAPPING[str(join_type)]
# TODO: update with correct implementation of leftsemi
if join_type == "leftsemi" and not is_cudf_type(df_lhs_renamed):
join_type = "inner"
# 3. The join condition can have two forms, that we can understand
# (a) a = b
# (b) X AND Y AND a = b AND Z ... (can also be multiple a = b)
# The first case is very simple and we do not need any additional filter
# In the second case we do a merge on all the a = b,
# and then apply a filter using the other expressions.
# In all other cases, we need to do a full table cross join and filter afterwards.
# As this is probably non-sense for large tables, but there is no other
# known solution so far.
join_condition = join.getCondition()
lhs_on, rhs_on, filter_condition = None, None, None
# A user can write certain queries that really should be `cross join` queries
# that will still enter this portion of the logic. IF the join_condition is
# None that means there are no conditions to join on. This means a cross join.
# By not entering this body during that condition we ensure that later on in
# processing we perform a cross join.
if join_condition is not None:
lhs_on, rhs_on, filter_condition = self._split_join_condition(
join_condition
)
# lhs_on and rhs_on are the indices of the columns to merge on.
# The given column indices are for the full, merged table which consists
# of lhs and rhs put side-by-side (in this order)
# We therefore need to normalize the rhs indices relative to the rhs table.
rhs_on = [index - len(df_lhs_renamed.columns) for index in rhs_on]
# 4. dask can only merge on the same column names.
# We therefore create new columns on purpose, which have a distinct name.
assert len(lhs_on) == len(rhs_on)
if lhs_on:
# 5. Now we can finally merge on these columns
# The resulting dataframe will contain all (renamed) columns from the lhs and rhs
# plus the added columns
df = self._join_on_columns(
df_lhs_renamed,
df_rhs_renamed,
lhs_on,
rhs_on,
join_type,
)
else:
# 5. We are in the complex join case
# where we have no column to merge on
# This means we have no other chance than to merge
# everything with everything...
# TODO: we should implement a shortcut
# for filter conditions that are always false
def merge_single_partitions(lhs_partition, rhs_partition):
# Do a cross join with the two partitions
# TODO: it would be nice to apply the filter already here
# problem: this would mean we need to ship the rex to the
# workers (as this is executed on the workers),
# which is definitely not possible (java dependency, JVM start...)
lhs_partition = lhs_partition.assign(common=1)
rhs_partition = rhs_partition.assign(common=1)
return lhs_partition.merge(rhs_partition, on="common").drop(
columns="common"
)
# Iterate nested over all partitions from lhs and rhs and merge them
name = "cross-join-" + tokenize(df_lhs_renamed, df_rhs_renamed)
dsk = {
(name, i * df_rhs_renamed.npartitions + j): (
merge_single_partitions,
(df_lhs_renamed._name, i),
(df_rhs_renamed._name, j),
)
for i in range(df_lhs_renamed.npartitions)
for j in range(df_rhs_renamed.npartitions)
}
graph = HighLevelGraph.from_collections(
name, dsk, dependencies=[df_lhs_renamed, df_rhs_renamed]
)
meta = dd.dispatch.concat(
[df_lhs_renamed._meta_nonempty, df_rhs_renamed._meta_nonempty], axis=1
)
# TODO: Do we know the divisions in any way here?
divisions = [None] * (len(dsk) + 1)
df = dd.DataFrame(graph, name, meta=meta, divisions=divisions)
warnings.warn(
"Need to do a cross-join, which is typically very resource heavy",
ResourceWarning,
)
# 6. So the next step is to make sure
# we have the correct column order (and to remove the temporary join columns)
if join_type in ("leftsemi", "leftanti"):
correct_column_order = list(df_lhs_renamed.columns)
else:
correct_column_order = list(df_lhs_renamed.columns) + list(
df_rhs_renamed.columns
)
cc = ColumnContainer(df.columns).limit_to(correct_column_order)
# and to rename them like the rel specifies
row_type = rel.getRowType()
field_specifications = [str(f) for f in row_type.getFieldNames()]
if join_type in ("leftsemi", "leftanti"):
field_specifications = field_specifications[: len(cc.columns)]
cc = cc.rename(
{
from_col: to_col
for from_col, to_col in zip(cc.columns, field_specifications)
}
)
cc = self.fix_column_to_row_type(cc, row_type, join_type)
dc = DataContainer(df, cc)
# 7. Last but not least we apply any filters by and-chaining together the filters
if filter_condition:
# This line is a bit of code duplication with RexCallPlugin - but I guess it is worth to keep it separate
filter_condition = reduce(
operator.and_,
[
RexConverter.convert(rel, rex, dc, context=context)
for rex in filter_condition
],
)
logger.debug(f"Additionally applying filter {filter_condition}")
df = filter_or_scalar(df, filter_condition)
dc = DataContainer(df, cc)
dc = self.fix_dtype_to_row_type(dc, rel.getRowType(), join_type)
# # Rename underlying DataFrame column names back to their original values before returning
# df = dc.assign()
# dc = DataContainer(df, ColumnContainer(cc.columns))
return dc
def _join_on_columns(
self,
df_lhs_renamed: dd.DataFrame,
df_rhs_renamed: dd.DataFrame,
lhs_on: List[str],
rhs_on: List[str],
join_type: str,
) -> dd.DataFrame:
# SQL compatibility: when joining on columns that
# contain NULLs, pandas will actually happily
# keep those NULLs. That is however not compatible with
# SQL, so we get rid of them here
if join_type in ["inner", "right"]:
df_lhs_filter = reduce(
operator.and_,
[~df_lhs_renamed.iloc[:, index].isna() for index in lhs_on],
)
df_lhs_renamed = df_lhs_renamed[df_lhs_filter]
if join_type in ["inner", "left", "leftanti", "leftsemi"]:
df_rhs_filter = reduce(
operator.and_,
[~df_rhs_renamed.iloc[:, index].isna() for index in rhs_on],
)
df_rhs_renamed = df_rhs_renamed[df_rhs_filter]
lhs_columns_to_add = {
f"common_{i}": df_lhs_renamed["lhs_" + str(index)]
for i, index in enumerate(lhs_on)
}
rhs_columns_to_add = {
f"common_{i}": df_rhs_renamed.iloc[:, index]
for i, index in enumerate(rhs_on)
}
df_lhs_with_tmp = df_lhs_renamed.assign(**lhs_columns_to_add)
df_rhs_with_tmp = df_rhs_renamed.assign(**rhs_columns_to_add)
added_columns = list(lhs_columns_to_add.keys())
broadcast = dask_config.get("sql.join.broadcast")
if not BROADCAST_JOIN_SUPPORT_WORKING and (
isinstance(broadcast, float) or broadcast
):
warnings.warn(
"Broadcast Joins may not work as expected with dask<2023.1.1"
"For more information refer to https://github.com/dask/dask/issues/9851"
" and https://github.com/dask/dask/issues/9870"
)
if join_type == "leftanti" and not is_cudf_type(df_lhs_with_tmp):
df = df_lhs_with_tmp.merge(
df_rhs_with_tmp,
on=added_columns,
how="left",
broadcast=broadcast,
indicator=True,
).drop(columns=added_columns)
df = df[df["_merge"] == "left_only"].drop(
columns=["_merge"] + list(df_rhs_with_tmp.columns), errors="ignore"
)
else:
df = df_lhs_with_tmp.merge(
df_rhs_with_tmp,
on=added_columns,
how=join_type,
broadcast=broadcast,
).drop(columns=added_columns)
return df
def _split_join_condition(
self, join_condition: "Expression"
) -> Tuple[List[str], List[str], List["Expression"]]:
if str(join_condition.getRexType()) in ["RexType.Literal", "RexType.Reference"]:
return [], [], [join_condition]
elif not str(join_condition.getRexType()) == "RexType.Call":
raise NotImplementedError("Can not understand join condition.")
lhs_on = []
rhs_on = []
filter_condition = []
try:
lhs_on, rhs_on, filter_condition_part = self._extract_lhs_rhs(
join_condition
)
filter_condition.extend(filter_condition_part)
except AssertionError:
filter_condition.append(join_condition)
if lhs_on and rhs_on:
return lhs_on, rhs_on, filter_condition
return [], [], [join_condition]
def _extract_lhs_rhs(self, rex):
assert str(rex.getRexType()) == "RexType.Call"
operator_name = str(rex.getOperatorName())
assert operator_name in ["=", "AND"]
operands = rex.getOperands()
assert len(operands) == 2
if operator_name == "=":
operand_lhs = operands[0]
operand_rhs = operands[1]
if (
str(operand_lhs.getRexType()) == "RexType.Reference"
and str(operand_rhs.getRexType()) == "RexType.Reference"
):
lhs_index = operand_lhs.getIndex()
rhs_index = operand_rhs.getIndex()
# The rhs table always comes after the lhs
# table. Therefore we have a very simple
# way of checking, which index comes from which
# input
if lhs_index > rhs_index:
lhs_index, rhs_index = rhs_index, lhs_index
return [lhs_index], [rhs_index], []
raise AssertionError(
"Invalid join condition"
) # pragma: no cover. Do not how how it could be triggered.
else:
lhs_indices = []
rhs_indices = []
filter_conditions = []
for operand in operands:
try:
lhs_index, rhs_index, filter_condition = self._extract_lhs_rhs(
operand
)
filter_conditions.extend(filter_condition)
lhs_indices.extend(lhs_index)
rhs_indices.extend(rhs_index)
except AssertionError:
filter_conditions.append(operand)
return lhs_indices, rhs_indices, filter_conditions