@@ -36,7 +36,7 @@ class OptimizeWindowFunctionsSuite extends PlanTest {
3636 val b = testRelation.output(1 )
3737 val c = testRelation.output(2 )
3838
39- test(" replace first(col) by nth_value(col, 1) " ) {
39+ test(" replace first by nth_value if frame is UNBOUNDED PRECEDING AND CURRENT ROW " ) {
4040 val inputPlan = testRelation.select(
4141 WindowExpression (
4242 First (a, false ).toAggregateExpression(),
@@ -52,7 +52,34 @@ class OptimizeWindowFunctionsSuite extends PlanTest {
5252 assert(optimized == correctAnswer)
5353 }
5454
55- test(" can't replace first(col) by nth_value(col, 1) if the window frame type is range" ) {
55+ test(" replace first by nth_value if frame is UNBOUNDED PRECEDING AND UNBOUNDED FOLLOWING" ) {
56+ val inputPlan = testRelation.select(
57+ WindowExpression (
58+ First (a, false ).toAggregateExpression(),
59+ WindowSpecDefinition (b :: Nil , c.asc :: Nil ,
60+ SpecifiedWindowFrame (RowFrame , UnboundedPreceding , UnboundedFollowing ))))
61+ val correctAnswer = testRelation.select(
62+ WindowExpression (
63+ NthValue (a, Literal (1 ), false ),
64+ WindowSpecDefinition (b :: Nil , c.asc :: Nil ,
65+ SpecifiedWindowFrame (RowFrame , UnboundedPreceding , UnboundedFollowing ))))
66+
67+ val optimized = Optimize .execute(inputPlan)
68+ assert(optimized == correctAnswer)
69+ }
70+
71+ test(" can't replace first by nth_value if frame is not suitable" ) {
72+ val inputPlan = testRelation.select(
73+ WindowExpression (
74+ First (a, false ).toAggregateExpression(),
75+ WindowSpecDefinition (b :: Nil , c.asc :: Nil ,
76+ SpecifiedWindowFrame (RowFrame , Literal (1 ), CurrentRow ))))
77+
78+ val optimized = Optimize .execute(inputPlan)
79+ assert(optimized == inputPlan)
80+ }
81+
82+ test(" can't replace first by nth_value if the window frame type is range" ) {
5683 val inputPlan = testRelation.select(
5784 WindowExpression (
5885 First (a, false ).toAggregateExpression(),
@@ -63,7 +90,7 @@ class OptimizeWindowFunctionsSuite extends PlanTest {
6390 assert(optimized == inputPlan)
6491 }
6592
66- test(" can't replace first(col) by nth_value(col, 1) if the window frame isn't ordered" ) {
93+ test(" can't replace first by nth_value if the window frame isn't ordered" ) {
6794 val inputPlan = testRelation.select(
6895 WindowExpression (
6996 First (a, false ).toAggregateExpression(),
0 commit comments