Skip to content

Commit

Permalink
Join side aliases should be optional (#862)
Browse files Browse the repository at this point in the history
* Join side aliases should be optional

Signed-off-by: Lantao Jin <[email protected]>

* address comments

Signed-off-by: Lantao Jin <[email protected]>

* typo

Signed-off-by: Lantao Jin <[email protected]>

---------

Signed-off-by: Lantao Jin <[email protected]>
  • Loading branch information
LantaoJin authored Nov 6, 2024
1 parent 502f0ec commit cfd41a3
Show file tree
Hide file tree
Showing 15 changed files with 755 additions and 103 deletions.
6 changes: 5 additions & 1 deletion docs/ppl-lang/PPL-Example-Commands.md
Original file line number Diff line number Diff line change
Expand Up @@ -306,7 +306,11 @@ source = table | where ispresent(a) |
- `source = table1 | left semi join left = l right = r on l.a = r.a table2`
- `source = table1 | left anti join left = l right = r on l.a = r.a table2`
- `source = table1 | join left = l right = r [ source = table2 | where d > 10 | head 5 ]`

- `source = table1 | inner join on table1.a = table2.a table2 | fields table1.a, table2.a, table1.b, table1.c` (directly refer table name)
- `source = table1 | inner join on a = c table2 | fields a, b, c, d` (ignore side aliases as long as no ambiguous)
- `source = table1 as t1 | join left = l right = r on l.a = r.a table2 as t2 | fields l.a, r.a` (side alias overrides table alias)
- `source = table1 as t1 | join left = l right = r on l.a = r.a table2 as t2 | fields t1.a, t2.a` (error, side alias overrides table alias)
- `source = table1 | join left = l right = r on l.a = r.a [ source = table2 ] as s | fields l.a, s.a` (error, side alias overrides subquery alias)

#### **Lookup**
[See additional command details](ppl-lookup-command.md)
Expand Down
23 changes: 12 additions & 11 deletions docs/ppl-lang/ppl-join-command.md
Original file line number Diff line number Diff line change
Expand Up @@ -65,8 +65,8 @@ WHERE t1.serviceName = `order`
SEARCH source=<left-table>
| <other piped command>
| [joinType] JOIN
leftAlias
rightAlias
[leftAlias]
[rightAlias]
[joinHints]
ON joinCriteria
<right-table>
Expand All @@ -79,12 +79,12 @@ SEARCH source=<left-table>

**leftAlias**
- Syntax: `left = <leftAlias>`
- Required
- Optional
- Description: The subquery alias to use with the left join side, to avoid ambiguous naming.

**rightAlias**
- Syntax: `right = <rightAlias>`
- Required
- Optional
- Description: The subquery alias to use with the right join side, to avoid ambiguous naming.

**joinHints**
Expand Down Expand Up @@ -138,11 +138,11 @@ Rewritten by PPL Join query:
```sql
SEARCH source=customer
| FIELDS c_custkey
| LEFT OUTER JOIN left = c, right = o
ON c.c_custkey = o.o_custkey AND o_comment NOT LIKE '%unusual%packages%'
| LEFT OUTER JOIN
ON c_custkey = o_custkey AND o_comment NOT LIKE '%unusual%packages%'
orders
| STATS count(o_orderkey) AS c_count BY c.c_custkey
| STATS count(1) AS custdist BY c_count
| STATS count(o_orderkey) AS c_count BY c_custkey
| STATS count() AS custdist BY c_count
| SORT - custdist, - c_count
```
_- **Limitation: sub-searches is unsupported in join right side**_
Expand All @@ -151,14 +151,15 @@ If sub-searches is supported, above ppl query could be rewritten as:
```sql
SEARCH source=customer
| FIELDS c_custkey
| LEFT OUTER JOIN left = c, right = o ON c.c_custkey = o.o_custkey
| LEFT OUTER JOIN
ON c_custkey = o_custkey
[
SEARCH source=orders
| WHERE o_comment NOT LIKE '%unusual%packages%'
| FIELDS o_orderkey, o_custkey
]
| STATS count(o_orderkey) AS c_count BY c.c_custkey
| STATS count(1) AS custdist BY c_count
| STATS count(o_orderkey) AS c_count BY c_custkey
| STATS count() AS custdist BY c_count
| SORT - custdist, - c_count
```

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -597,4 +597,68 @@ class FlintSparkPPLBasicITSuite
| """.stripMargin))
assert(ex.getMessage().contains("Invalid table name"))
}

test("Search multiple tables - translated into union call with fields") {
val frame = sql(s"""
| source = $t1, $t2
| """.stripMargin)
assertSameRows(
Seq(
Row("Hello", 30, "New York", "USA", 2023, 4),
Row("Hello", 30, "New York", "USA", 2023, 4),
Row("Jake", 70, "California", "USA", 2023, 4),
Row("Jake", 70, "California", "USA", 2023, 4),
Row("Jane", 20, "Quebec", "Canada", 2023, 4),
Row("Jane", 20, "Quebec", "Canada", 2023, 4),
Row("John", 25, "Ontario", "Canada", 2023, 4),
Row("John", 25, "Ontario", "Canada", 2023, 4)),
frame)

val logicalPlan: LogicalPlan = frame.queryExecution.logical
val table1 = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test1"))
val table2 = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test2"))

val allFields1 = UnresolvedStar(None)
val allFields2 = UnresolvedStar(None)

val projectedTable1 = Project(Seq(allFields1), table1)
val projectedTable2 = Project(Seq(allFields2), table2)

val expectedPlan =
Union(Seq(projectedTable1, projectedTable2), byName = true, allowMissingCol = true)

comparePlans(expectedPlan, logicalPlan, checkAnalysis = false)
}

test("Search multiple tables - with table alias") {
val frame = sql(s"""
| source = $t1, $t2 as t | where t.country = "USA"
| """.stripMargin)
assertSameRows(
Seq(
Row("Hello", 30, "New York", "USA", 2023, 4),
Row("Hello", 30, "New York", "USA", 2023, 4),
Row("Jake", 70, "California", "USA", 2023, 4),
Row("Jake", 70, "California", "USA", 2023, 4)),
frame)

val logicalPlan: LogicalPlan = frame.queryExecution.logical
val table1 = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test1"))
val table2 = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test2"))

val plan1 = Filter(
EqualTo(UnresolvedAttribute("t.country"), Literal("USA")),
SubqueryAlias("t", table1))
val plan2 = Filter(
EqualTo(UnresolvedAttribute("t.country"), Literal("USA")),
SubqueryAlias("t", table2))

val projectedTable1 = Project(Seq(UnresolvedStar(None)), plan1)
val projectedTable2 = Project(Seq(UnresolvedStar(None)), plan2)

val expectedPlan =
Union(Seq(projectedTable1, projectedTable2), byName = true, allowMissingCol = true)

comparePlans(expectedPlan, logicalPlan, checkAnalysis = false)
}
}
Loading

0 comments on commit cfd41a3

Please sign in to comment.