Skip to content

Commit 881c5c8

Browse files
jliworkgatorsmile
authored andcommitted
[SPARK-22548][SQL] Incorrect nested AND expression pushed down to JDBC data source
## What changes were proposed in this pull request? Let’s say I have a nested AND expression shown below and p2 can not be pushed down, (p1 AND p2) OR p3 In current Spark code, during data source filter translation, (p1 AND p2) is returned as p1 only and p2 is simply lost. This issue occurs with JDBC data source and is similar to [SPARK-12218](#10362) for Parquet. When we have AND nested below another expression, we should either push both legs or nothing. Note that: - The current Spark code will always split conjunctive predicate before it determines if a predicate can be pushed down or not - If I have (p1 AND p2) AND p3, it will be split into p1, p2, p3. There won't be nested AND expression. - The current Spark code logic for OR is OK. It either pushes both legs or nothing. The same translation method is also called by Data Source V2. ## How was this patch tested? Added new unit test cases to JDBCSuite gatorsmile Author: Jia Li <[email protected]> Closes #19776 from jliwork/spark-22548.
1 parent ac10171 commit 881c5c8

File tree

3 files changed

+250
-4
lines changed

3 files changed

+250
-4
lines changed

sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala

Lines changed: 13 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -497,7 +497,19 @@ object DataSourceStrategy {
497497
Some(sources.IsNotNull(a.name))
498498

499499
case expressions.And(left, right) =>
500-
(translateFilter(left) ++ translateFilter(right)).reduceOption(sources.And)
500+
// See SPARK-12218 for detailed discussion
501+
// It is not safe to just convert one side if we do not understand the
502+
// other side. Here is an example used to explain the reason.
503+
// Let's say we have (a = 2 AND trim(b) = 'blah') OR (c > 0)
504+
// and we do not understand how to convert trim(b) = 'blah'.
505+
// If we only convert a = 2, we will end up with
506+
// (a = 2) OR (c > 0), which will generate wrong results.
507+
// Pushing one leg of AND down is only safe to do at the top level.
508+
// You can see ParquetFilters' createFilter for more details.
509+
for {
510+
leftFilter <- translateFilter(left)
511+
rightFilter <- translateFilter(right)
512+
} yield sources.And(leftFilter, rightFilter)
501513

502514
case expressions.Or(left, right) =>
503515
for {
Lines changed: 231 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,231 @@
1+
/*
2+
* Licensed to the Apache Software Foundation (ASF) under one or more
3+
* contributor license agreements. See the NOTICE file distributed with
4+
* this work for additional information regarding copyright ownership.
5+
* The ASF licenses this file to You under the Apache License, Version 2.0
6+
* (the "License"); you may not use this file except in compliance with
7+
* the License. You may obtain a copy of the License at
8+
*
9+
* http://www.apache.org/licenses/LICENSE-2.0
10+
*
11+
* Unless required by applicable law or agreed to in writing, software
12+
* distributed under the License is distributed on an "AS IS" BASIS,
13+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
14+
* See the License for the specific language governing permissions and
15+
* limitations under the License.
16+
*/
17+
18+
package org.apache.spark.sql.execution.datasources
19+
20+
import org.apache.spark.sql.catalyst.dsl.expressions._
21+
import org.apache.spark.sql.catalyst.expressions._
22+
import org.apache.spark.sql.catalyst.plans.PlanTest
23+
import org.apache.spark.sql.sources
24+
import org.apache.spark.sql.test.SharedSQLContext
25+
26+
class DataSourceStrategySuite extends PlanTest with SharedSQLContext {
27+
28+
test("translate simple expression") {
29+
val attrInt = 'cint.int
30+
val attrStr = 'cstr.string
31+
32+
testTranslateFilter(EqualTo(attrInt, 1), Some(sources.EqualTo("cint", 1)))
33+
testTranslateFilter(EqualTo(1, attrInt), Some(sources.EqualTo("cint", 1)))
34+
35+
testTranslateFilter(EqualNullSafe(attrStr, Literal(null)),
36+
Some(sources.EqualNullSafe("cstr", null)))
37+
testTranslateFilter(EqualNullSafe(Literal(null), attrStr),
38+
Some(sources.EqualNullSafe("cstr", null)))
39+
40+
testTranslateFilter(GreaterThan(attrInt, 1), Some(sources.GreaterThan("cint", 1)))
41+
testTranslateFilter(GreaterThan(1, attrInt), Some(sources.LessThan("cint", 1)))
42+
43+
testTranslateFilter(LessThan(attrInt, 1), Some(sources.LessThan("cint", 1)))
44+
testTranslateFilter(LessThan(1, attrInt), Some(sources.GreaterThan("cint", 1)))
45+
46+
testTranslateFilter(GreaterThanOrEqual(attrInt, 1), Some(sources.GreaterThanOrEqual("cint", 1)))
47+
testTranslateFilter(GreaterThanOrEqual(1, attrInt), Some(sources.LessThanOrEqual("cint", 1)))
48+
49+
testTranslateFilter(LessThanOrEqual(attrInt, 1), Some(sources.LessThanOrEqual("cint", 1)))
50+
testTranslateFilter(LessThanOrEqual(1, attrInt), Some(sources.GreaterThanOrEqual("cint", 1)))
51+
52+
testTranslateFilter(InSet(attrInt, Set(1, 2, 3)), Some(sources.In("cint", Array(1, 2, 3))))
53+
54+
testTranslateFilter(In(attrInt, Seq(1, 2, 3)), Some(sources.In("cint", Array(1, 2, 3))))
55+
56+
testTranslateFilter(IsNull(attrInt), Some(sources.IsNull("cint")))
57+
testTranslateFilter(IsNotNull(attrInt), Some(sources.IsNotNull("cint")))
58+
59+
// cint > 1 AND cint < 10
60+
testTranslateFilter(And(
61+
GreaterThan(attrInt, 1),
62+
LessThan(attrInt, 10)),
63+
Some(sources.And(
64+
sources.GreaterThan("cint", 1),
65+
sources.LessThan("cint", 10))))
66+
67+
// cint >= 8 OR cint <= 2
68+
testTranslateFilter(Or(
69+
GreaterThanOrEqual(attrInt, 8),
70+
LessThanOrEqual(attrInt, 2)),
71+
Some(sources.Or(
72+
sources.GreaterThanOrEqual("cint", 8),
73+
sources.LessThanOrEqual("cint", 2))))
74+
75+
testTranslateFilter(Not(GreaterThanOrEqual(attrInt, 8)),
76+
Some(sources.Not(sources.GreaterThanOrEqual("cint", 8))))
77+
78+
testTranslateFilter(StartsWith(attrStr, "a"), Some(sources.StringStartsWith("cstr", "a")))
79+
80+
testTranslateFilter(EndsWith(attrStr, "a"), Some(sources.StringEndsWith("cstr", "a")))
81+
82+
testTranslateFilter(Contains(attrStr, "a"), Some(sources.StringContains("cstr", "a")))
83+
}
84+
85+
test("translate complex expression") {
86+
val attrInt = 'cint.int
87+
88+
// ABS(cint) - 2 <= 1
89+
testTranslateFilter(LessThanOrEqual(
90+
// Expressions are not supported
91+
// Functions such as 'Abs' are not supported
92+
Subtract(Abs(attrInt), 2), 1), None)
93+
94+
// (cin1 > 1 AND cint < 10) OR (cint > 50 AND cint > 100)
95+
testTranslateFilter(Or(
96+
And(
97+
GreaterThan(attrInt, 1),
98+
LessThan(attrInt, 10)
99+
),
100+
And(
101+
GreaterThan(attrInt, 50),
102+
LessThan(attrInt, 100))),
103+
Some(sources.Or(
104+
sources.And(
105+
sources.GreaterThan("cint", 1),
106+
sources.LessThan("cint", 10)),
107+
sources.And(
108+
sources.GreaterThan("cint", 50),
109+
sources.LessThan("cint", 100)))))
110+
111+
// SPARK-22548 Incorrect nested AND expression pushed down to JDBC data source
112+
// (cint > 1 AND ABS(cint) < 10) OR (cint < 50 AND cint > 100)
113+
testTranslateFilter(Or(
114+
And(
115+
GreaterThan(attrInt, 1),
116+
// Functions such as 'Abs' are not supported
117+
LessThan(Abs(attrInt), 10)
118+
),
119+
And(
120+
GreaterThan(attrInt, 50),
121+
LessThan(attrInt, 100))), None)
122+
123+
// NOT ((cint <= 1 OR ABS(cint) >= 10) AND (cint <= 50 OR cint >= 100))
124+
testTranslateFilter(Not(And(
125+
Or(
126+
LessThanOrEqual(attrInt, 1),
127+
// Functions such as 'Abs' are not supported
128+
GreaterThanOrEqual(Abs(attrInt), 10)
129+
),
130+
Or(
131+
LessThanOrEqual(attrInt, 50),
132+
GreaterThanOrEqual(attrInt, 100)))), None)
133+
134+
// (cint = 1 OR cint = 10) OR (cint > 0 OR cint < -10)
135+
testTranslateFilter(Or(
136+
Or(
137+
EqualTo(attrInt, 1),
138+
EqualTo(attrInt, 10)
139+
),
140+
Or(
141+
GreaterThan(attrInt, 0),
142+
LessThan(attrInt, -10))),
143+
Some(sources.Or(
144+
sources.Or(
145+
sources.EqualTo("cint", 1),
146+
sources.EqualTo("cint", 10)),
147+
sources.Or(
148+
sources.GreaterThan("cint", 0),
149+
sources.LessThan("cint", -10)))))
150+
151+
// (cint = 1 OR ABS(cint) = 10) OR (cint > 0 OR cint < -10)
152+
testTranslateFilter(Or(
153+
Or(
154+
EqualTo(attrInt, 1),
155+
// Functions such as 'Abs' are not supported
156+
EqualTo(Abs(attrInt), 10)
157+
),
158+
Or(
159+
GreaterThan(attrInt, 0),
160+
LessThan(attrInt, -10))), None)
161+
162+
// In end-to-end testing, conjunctive predicate should has been split
163+
// before reaching DataSourceStrategy.translateFilter.
164+
// This is for UT purpose to test each [[case]].
165+
// (cint > 1 AND cint < 10) AND (cint = 6 AND cint IS NOT NULL)
166+
testTranslateFilter(And(
167+
And(
168+
GreaterThan(attrInt, 1),
169+
LessThan(attrInt, 10)
170+
),
171+
And(
172+
EqualTo(attrInt, 6),
173+
IsNotNull(attrInt))),
174+
Some(sources.And(
175+
sources.And(
176+
sources.GreaterThan("cint", 1),
177+
sources.LessThan("cint", 10)),
178+
sources.And(
179+
sources.EqualTo("cint", 6),
180+
sources.IsNotNull("cint")))))
181+
182+
// (cint > 1 AND cint < 10) AND (ABS(cint) = 6 AND cint IS NOT NULL)
183+
testTranslateFilter(And(
184+
And(
185+
GreaterThan(attrInt, 1),
186+
LessThan(attrInt, 10)
187+
),
188+
And(
189+
// Functions such as 'Abs' are not supported
190+
EqualTo(Abs(attrInt), 6),
191+
IsNotNull(attrInt))), None)
192+
193+
// (cint > 1 OR cint < 10) AND (cint = 6 OR cint IS NOT NULL)
194+
testTranslateFilter(And(
195+
Or(
196+
GreaterThan(attrInt, 1),
197+
LessThan(attrInt, 10)
198+
),
199+
Or(
200+
EqualTo(attrInt, 6),
201+
IsNotNull(attrInt))),
202+
Some(sources.And(
203+
sources.Or(
204+
sources.GreaterThan("cint", 1),
205+
sources.LessThan("cint", 10)),
206+
sources.Or(
207+
sources.EqualTo("cint", 6),
208+
sources.IsNotNull("cint")))))
209+
210+
// (cint > 1 OR cint < 10) AND (cint = 6 OR cint IS NOT NULL)
211+
testTranslateFilter(And(
212+
Or(
213+
GreaterThan(attrInt, 1),
214+
LessThan(attrInt, 10)
215+
),
216+
Or(
217+
// Functions such as 'Abs' are not supported
218+
EqualTo(Abs(attrInt), 6),
219+
IsNotNull(attrInt))), None)
220+
}
221+
222+
/**
223+
* Translate the given Catalyst [[Expression]] into data source [[sources.Filter]]
224+
* then verify against the given [[sources.Filter]].
225+
*/
226+
def testTranslateFilter(catalystFilter: Expression, result: Option[sources.Filter]): Unit = {
227+
assertResult(result) {
228+
DataSourceStrategy.translateFilter(catalystFilter)
229+
}
230+
}
231+
}

sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -294,10 +294,13 @@ class JDBCSuite extends SparkFunSuite
294294

295295
// This is a test to reflect discussion in SPARK-12218.
296296
// The older versions of spark have this kind of bugs in parquet data source.
297-
val df1 = sql("SELECT * FROM foobar WHERE NOT (THEID != 2 AND NAME != 'mary')")
298-
val df2 = sql("SELECT * FROM foobar WHERE NOT (THEID != 2) OR NOT (NAME != 'mary')")
297+
val df1 = sql("SELECT * FROM foobar WHERE NOT (THEID != 2) OR NOT (NAME != 'mary')")
299298
assert(df1.collect.toSet === Set(Row("mary", 2)))
300-
assert(df2.collect.toSet === Set(Row("mary", 2)))
299+
300+
// SPARK-22548: Incorrect nested AND expression pushed down to JDBC data source
301+
val df2 = sql("SELECT * FROM foobar " +
302+
"WHERE (THEID > 0 AND TRIM(NAME) = 'mary') OR (NAME = 'fred')")
303+
assert(df2.collect.toSet === Set(Row("fred", 1), Row("mary", 2)))
301304

302305
def checkNotPushdown(df: DataFrame): DataFrame = {
303306
val parentPlan = df.queryExecution.executedPlan

0 commit comments

Comments
 (0)