|
| 1 | +/* |
| 2 | + * Copyright OpenSearch Contributors |
| 3 | + * SPDX-License-Identifier: Apache-2.0 |
| 4 | + */ |
| 5 | + |
| 6 | +package org.opensearch.flint.spark.ppl |
| 7 | + |
| 8 | +import org.opensearch.flint.spark.ppl.PlaneUtils.plan |
| 9 | +import org.opensearch.sql.ppl.{CatalystPlanContext, CatalystQueryPlanVisitor} |
| 10 | +import org.opensearch.sql.ppl.utils.DataTypeTransformer.seq |
| 11 | +import org.scalatest.matchers.should.Matchers |
| 12 | + |
| 13 | +import org.apache.spark.SparkFunSuite |
| 14 | +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar} |
| 15 | +import org.apache.spark.sql.catalyst.expressions.{Alias, Literal, NamedExpression} |
| 16 | +import org.apache.spark.sql.catalyst.plans.PlanTest |
| 17 | +import org.apache.spark.sql.catalyst.plans.logical.{DataFrameDropColumns, Project} |
| 18 | + |
| 19 | +class PPLLogicalPlanRenameCommandTranslatorTestSuite |
| 20 | + extends SparkFunSuite |
| 21 | + with PlanTest |
| 22 | + with LogicalPlanTestUtils |
| 23 | + with Matchers { |
| 24 | + |
| 25 | + private val planTransformer = new CatalystQueryPlanVisitor() |
| 26 | + private val pplParser = new PPLSyntaxParser() |
| 27 | + |
| 28 | + test("test fillnull with one null replacement value and one column") { |
| 29 | + val context = new CatalystPlanContext |
| 30 | + val logPlan = |
| 31 | + planTransformer.visit( |
| 32 | + plan( |
| 33 | + pplParser, |
| 34 | + "source=relation | fillnull value = 'null replacement value' column_name"), |
| 35 | + context) |
| 36 | + |
| 37 | + val relation = UnresolvedRelation(Seq("relation")) |
| 38 | + |
| 39 | + val renameProjectList: Seq[NamedExpression] = |
| 40 | + Seq( |
| 41 | + UnresolvedStar(None), |
| 42 | + Alias( |
| 43 | + UnresolvedFunction( |
| 44 | + "coalesce", |
| 45 | + Seq(UnresolvedAttribute("column_name"), Literal("null replacement value")), |
| 46 | + isDistinct = false), |
| 47 | + "column_name")()) |
| 48 | + val renameProject = Project(renameProjectList, relation) |
| 49 | + |
| 50 | + val dropSourceColumn = |
| 51 | + DataFrameDropColumns(Seq(UnresolvedAttribute("column_name")), renameProject) |
| 52 | + |
| 53 | + val expectedPlan = Project(seq(UnresolvedStar(None)), dropSourceColumn) |
| 54 | + comparePlans(expectedPlan, logPlan, checkAnalysis = false) |
| 55 | + } |
| 56 | + |
| 57 | + test("test fillnull with one null replacement value and multiple column") { |
| 58 | + val context = new CatalystPlanContext |
| 59 | + val logPlan = |
| 60 | + planTransformer.visit( |
| 61 | + plan( |
| 62 | + pplParser, |
| 63 | + "source=relation | fillnull value = 'another null replacement value' column_name_one, column_name_two, column_name_three"), |
| 64 | + context) |
| 65 | + |
| 66 | + val relation = UnresolvedRelation(Seq("relation")) |
| 67 | + |
| 68 | + val renameProjectList: Seq[NamedExpression] = Seq( |
| 69 | + UnresolvedStar(None), |
| 70 | + Alias( |
| 71 | + UnresolvedFunction( |
| 72 | + "coalesce", |
| 73 | + Seq(UnresolvedAttribute("column_name_one"), Literal("another null replacement value")), |
| 74 | + isDistinct = false), |
| 75 | + "column_name_one")(), |
| 76 | + Alias( |
| 77 | + UnresolvedFunction( |
| 78 | + "coalesce", |
| 79 | + Seq(UnresolvedAttribute("column_name_two"), Literal("another null replacement value")), |
| 80 | + isDistinct = false), |
| 81 | + "column_name_two")(), |
| 82 | + Alias( |
| 83 | + UnresolvedFunction( |
| 84 | + "coalesce", |
| 85 | + Seq( |
| 86 | + UnresolvedAttribute("column_name_three"), |
| 87 | + Literal("another null replacement value")), |
| 88 | + isDistinct = false), |
| 89 | + "column_name_three")()) |
| 90 | + val renameProject = Project(renameProjectList, relation) |
| 91 | + |
| 92 | + val dropSourceColumn = DataFrameDropColumns( |
| 93 | + Seq( |
| 94 | + UnresolvedAttribute("column_name_one"), |
| 95 | + UnresolvedAttribute("column_name_two"), |
| 96 | + UnresolvedAttribute("column_name_three")), |
| 97 | + renameProject) |
| 98 | + |
| 99 | + val expectedPlan = Project(seq(UnresolvedStar(None)), dropSourceColumn) |
| 100 | + comparePlans(expectedPlan, logPlan, checkAnalysis = false) |
| 101 | + } |
| 102 | + |
| 103 | + test("test fillnull with possibly various null replacement value and one column") { |
| 104 | + val context = new CatalystPlanContext |
| 105 | + val logPlan = |
| 106 | + planTransformer.visit( |
| 107 | + plan(pplParser, "source=relation | fillnull fields column_name='null replacement value'"), |
| 108 | + context) |
| 109 | + |
| 110 | + val relation = UnresolvedRelation(Seq("relation")) |
| 111 | + |
| 112 | + val renameProjectList: Seq[NamedExpression] = |
| 113 | + Seq( |
| 114 | + UnresolvedStar(None), |
| 115 | + Alias( |
| 116 | + UnresolvedFunction( |
| 117 | + "coalesce", |
| 118 | + Seq(UnresolvedAttribute("column_name"), Literal("null replacement value")), |
| 119 | + isDistinct = false), |
| 120 | + "column_name")()) |
| 121 | + val renameProject = Project(renameProjectList, relation) |
| 122 | + |
| 123 | + val dropSourceColumn = |
| 124 | + DataFrameDropColumns(Seq(UnresolvedAttribute("column_name")), renameProject) |
| 125 | + |
| 126 | + val expectedPlan = Project(seq(UnresolvedStar(None)), dropSourceColumn) |
| 127 | + comparePlans(expectedPlan, logPlan, checkAnalysis = false) |
| 128 | + } |
| 129 | + |
| 130 | + test("test fillnull with possibly various null replacement value and three columns") { |
| 131 | + val context = new CatalystPlanContext |
| 132 | + val logPlan = |
| 133 | + planTransformer.visit( |
| 134 | + plan( |
| 135 | + pplParser, |
| 136 | + "source=relation | fillnull fields column_name_1='null replacement value 1', column_name_2='null replacement value 2', column_name_3='null replacement value 3'"), |
| 137 | + context) |
| 138 | + |
| 139 | + val relation = UnresolvedRelation(Seq("relation")) |
| 140 | + |
| 141 | + val renameProjectList: Seq[NamedExpression] = Seq( |
| 142 | + UnresolvedStar(None), |
| 143 | + Alias( |
| 144 | + UnresolvedFunction( |
| 145 | + "coalesce", |
| 146 | + Seq(UnresolvedAttribute("column_name_1"), Literal("null replacement value 1")), |
| 147 | + isDistinct = false), |
| 148 | + "column_name_1")(), |
| 149 | + Alias( |
| 150 | + UnresolvedFunction( |
| 151 | + "coalesce", |
| 152 | + Seq(UnresolvedAttribute("column_name_2"), Literal("null replacement value 2")), |
| 153 | + isDistinct = false), |
| 154 | + "column_name_2")(), |
| 155 | + Alias( |
| 156 | + UnresolvedFunction( |
| 157 | + "coalesce", |
| 158 | + Seq(UnresolvedAttribute("column_name_3"), Literal("null replacement value 3")), |
| 159 | + isDistinct = false), |
| 160 | + "column_name_3")()) |
| 161 | + val renameProject = Project(renameProjectList, relation) |
| 162 | + |
| 163 | + val dropSourceColumn = DataFrameDropColumns( |
| 164 | + Seq( |
| 165 | + UnresolvedAttribute("column_name_1"), |
| 166 | + UnresolvedAttribute("column_name_2"), |
| 167 | + UnresolvedAttribute("column_name_3")), |
| 168 | + renameProject) |
| 169 | + |
| 170 | + val expectedPlan = Project(seq(UnresolvedStar(None)), dropSourceColumn) |
| 171 | + comparePlans(expectedPlan, logPlan, checkAnalysis = false) |
| 172 | + } |
| 173 | +} |
0 commit comments