|
| 1 | +/* |
| 2 | + * Copyright OpenSearch Contributors |
| 3 | + * SPDX-License-Identifier: Apache-2.0 |
| 4 | + */ |
| 5 | +package org.opensearch.flint.spark.ppl |
| 6 | + |
| 7 | +import org.opensearch.sql.ppl.utils.DataTypeTransformer.seq |
| 8 | + |
| 9 | +import org.apache.spark.sql.{AnalysisException, QueryTest, Row} |
| 10 | +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, UnresolvedFunction, UnresolvedRelation, UnresolvedStar} |
| 11 | +import org.apache.spark.sql.catalyst.expressions.{Alias, Ascending, Expression, Literal, SortOrder} |
| 12 | +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, DataFrameDropColumns, LogicalPlan, Project, Sort} |
| 13 | +import org.apache.spark.sql.streaming.StreamTest |
| 14 | + |
| 15 | +class FlintSparkPPLFillnullITSuite |
| 16 | + extends QueryTest |
| 17 | + with LogicalPlanTestUtils |
| 18 | + with FlintPPLSuite |
| 19 | + with StreamTest { |
| 20 | + |
| 21 | + private val testTable = "spark_catalog.default.flint_ppl_test" |
| 22 | + |
| 23 | + override def beforeAll(): Unit = { |
| 24 | + super.beforeAll() |
| 25 | + |
| 26 | + // Create test table |
| 27 | + createNullableTableHttpLog(testTable) |
| 28 | + } |
| 29 | + |
| 30 | + protected override def afterEach(): Unit = { |
| 31 | + super.afterEach() |
| 32 | + // Stop all streaming jobs if any |
| 33 | + spark.streams.active.foreach { job => |
| 34 | + job.stop() |
| 35 | + job.awaitTermination() |
| 36 | + } |
| 37 | + } |
| 38 | + |
| 39 | + test("test fillnull with one null replacement value and one column") { |
| 40 | + val frame = sql(s""" |
| 41 | + | source = $testTable | fillnull with 0 in status_code |
| 42 | + | """.stripMargin) |
| 43 | + |
| 44 | + assert(frame.columns.sameElements(Array("id", "request_path", "timestamp", "status_code"))) |
| 45 | + val results: Array[Row] = frame.collect() |
| 46 | + val expectedResults: Array[Row] = |
| 47 | + Array( |
| 48 | + Row(1, "/home", null, 200), |
| 49 | + Row(2, "/about", "2023-10-01 10:05:00", 0), |
| 50 | + Row(3, "/contact", "2023-10-01 10:10:00", 0), |
| 51 | + Row(4, null, "2023-10-01 10:15:00", 301), |
| 52 | + Row(5, null, "2023-10-01 10:20:00", 200), |
| 53 | + Row(6, "/home", null, 403)) |
| 54 | + // Compare the results |
| 55 | + implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, Int](_.getAs[Int](0)) |
| 56 | + assert(results.sorted.sameElements(expectedResults.sorted)) |
| 57 | + |
| 58 | + // Retrieve the logical plan |
| 59 | + val logicalPlan: LogicalPlan = frame.queryExecution.logical |
| 60 | + val expectedPlan = fillNullExpectedPlan(Seq(("status_code", Literal(0)))) |
| 61 | + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) |
| 62 | + } |
| 63 | + |
| 64 | + test("test fillnull with various null replacement values and one column") { |
| 65 | + val frame = sql(s""" |
| 66 | + | source = $testTable | fillnull using status_code=101 |
| 67 | + | """.stripMargin) |
| 68 | + |
| 69 | + assert(frame.columns.sameElements(Array("id", "request_path", "timestamp", "status_code"))) |
| 70 | + val results: Array[Row] = frame.collect() |
| 71 | + val expectedResults: Array[Row] = |
| 72 | + Array( |
| 73 | + Row(1, "/home", null, 200), |
| 74 | + Row(2, "/about", "2023-10-01 10:05:00", 101), |
| 75 | + Row(3, "/contact", "2023-10-01 10:10:00", 101), |
| 76 | + Row(4, null, "2023-10-01 10:15:00", 301), |
| 77 | + Row(5, null, "2023-10-01 10:20:00", 200), |
| 78 | + Row(6, "/home", null, 403)) |
| 79 | + // Compare the results |
| 80 | + implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, Int](_.getAs[Int](0)) |
| 81 | + assert(results.sorted.sameElements(expectedResults.sorted)) |
| 82 | + |
| 83 | + // Retrieve the logical plan |
| 84 | + val logicalPlan: LogicalPlan = frame.queryExecution.logical |
| 85 | + val expectedPlan = fillNullExpectedPlan(Seq(("status_code", Literal(101)))) |
| 86 | + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) |
| 87 | + } |
| 88 | + |
| 89 | + test("test fillnull with one null replacement value and two columns") { |
| 90 | + val frame = sql(s""" |
| 91 | + | source = $testTable | fillnull with concat('??', '?') in request_path, timestamp | fields id, request_path, timestamp |
| 92 | + | """.stripMargin) |
| 93 | + |
| 94 | + assert(frame.columns.sameElements(Array("id", "request_path", "timestamp"))) |
| 95 | + val results: Array[Row] = frame.collect() |
| 96 | + val expectedResults: Array[Row] = |
| 97 | + Array( |
| 98 | + Row(1, "/home", "???"), |
| 99 | + Row(2, "/about", "2023-10-01 10:05:00"), |
| 100 | + Row(3, "/contact", "2023-10-01 10:10:00"), |
| 101 | + Row(4, "???", "2023-10-01 10:15:00"), |
| 102 | + Row(5, "???", "2023-10-01 10:20:00"), |
| 103 | + Row(6, "/home", "???")) |
| 104 | + // Compare the results |
| 105 | + implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, Int](_.getAs[Int](0)) |
| 106 | + assert(results.sorted.sameElements(expectedResults.sorted)) |
| 107 | + |
| 108 | + // Retrieve the logical plan |
| 109 | + val logicalPlan: LogicalPlan = frame.queryExecution.logical |
| 110 | + val fillNullPlan = fillNullExpectedPlan( |
| 111 | + Seq( |
| 112 | + ( |
| 113 | + "request_path", |
| 114 | + UnresolvedFunction("concat", Seq(Literal("??"), Literal("?")), isDistinct = false)), |
| 115 | + ( |
| 116 | + "timestamp", |
| 117 | + UnresolvedFunction("concat", Seq(Literal("??"), Literal("?")), isDistinct = false))), |
| 118 | + addDefaultProject = false) |
| 119 | + val expectedPlan = Project( |
| 120 | + Seq( |
| 121 | + UnresolvedAttribute("id"), |
| 122 | + UnresolvedAttribute("request_path"), |
| 123 | + UnresolvedAttribute("timestamp")), |
| 124 | + fillNullPlan) |
| 125 | + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) |
| 126 | + } |
| 127 | + |
| 128 | + test("test fillnull with various null replacement values and two columns") { |
| 129 | + val frame = sql(s""" |
| 130 | + | source = $testTable | fillnull using request_path=upper('/not_found'), timestamp='*' | fields id, request_path, timestamp |
| 131 | + | """.stripMargin) |
| 132 | + |
| 133 | + assert(frame.columns.sameElements(Array("id", "request_path", "timestamp"))) |
| 134 | + val results: Array[Row] = frame.collect() |
| 135 | + val expectedResults: Array[Row] = |
| 136 | + Array( |
| 137 | + Row(1, "/home", "*"), |
| 138 | + Row(2, "/about", "2023-10-01 10:05:00"), |
| 139 | + Row(3, "/contact", "2023-10-01 10:10:00"), |
| 140 | + Row(4, "/NOT_FOUND", "2023-10-01 10:15:00"), |
| 141 | + Row(5, "/NOT_FOUND", "2023-10-01 10:20:00"), |
| 142 | + Row(6, "/home", "*")) |
| 143 | + // Compare the results |
| 144 | + implicit val rowOrdering: Ordering[Row] = Ordering.by[Row, Int](_.getAs[Int](0)) |
| 145 | + assert(results.sorted.sameElements(expectedResults.sorted)) |
| 146 | + |
| 147 | + // Retrieve the logical plan |
| 148 | + val logicalPlan: LogicalPlan = frame.queryExecution.logical |
| 149 | + val fillNullPlan = fillNullExpectedPlan( |
| 150 | + Seq( |
| 151 | + ( |
| 152 | + "request_path", |
| 153 | + UnresolvedFunction("upper", Seq(Literal("/not_found")), isDistinct = false)), |
| 154 | + ("timestamp", Literal("*"))), |
| 155 | + addDefaultProject = false) |
| 156 | + val expectedPlan = Project( |
| 157 | + Seq( |
| 158 | + UnresolvedAttribute("id"), |
| 159 | + UnresolvedAttribute("request_path"), |
| 160 | + UnresolvedAttribute("timestamp")), |
| 161 | + fillNullPlan) |
| 162 | + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) |
| 163 | + } |
| 164 | + |
| 165 | + test("test fillnull with one null replacement value and stats and sort command") { |
| 166 | + val frame = sql(s""" |
| 167 | + | source = $testTable | fillnull with 500 in status_code |
| 168 | + | | stats count(status_code) by status_code, request_path |
| 169 | + | | sort request_path, status_code |
| 170 | + | """.stripMargin) |
| 171 | + |
| 172 | + assert(frame.columns.sameElements(Array("count(status_code)", "status_code", "request_path"))) |
| 173 | + val results: Array[Row] = frame.collect() |
| 174 | + val expectedResults: Array[Row] = |
| 175 | + Array( |
| 176 | + Row(1, 200, null), |
| 177 | + Row(1, 301, null), |
| 178 | + Row(1, 500, "/about"), |
| 179 | + Row(1, 500, "/contact"), |
| 180 | + Row(1, 200, "/home"), |
| 181 | + Row(1, 403, "/home")) |
| 182 | + // Compare the results |
| 183 | + assert(results.sameElements(expectedResults)) |
| 184 | + |
| 185 | + // Retrieve the logical plan |
| 186 | + val logicalPlan: LogicalPlan = frame.queryExecution.logical |
| 187 | + val fillNullPlan = |
| 188 | + fillNullExpectedPlan(Seq(("status_code", Literal(500))), addDefaultProject = false) |
| 189 | + val aggregateExpressions = |
| 190 | + Seq( |
| 191 | + Alias( |
| 192 | + UnresolvedFunction( |
| 193 | + Seq("COUNT"), |
| 194 | + Seq(UnresolvedAttribute("status_code")), |
| 195 | + isDistinct = false), |
| 196 | + "count(status_code)")(), |
| 197 | + Alias(UnresolvedAttribute("status_code"), "status_code")(), |
| 198 | + Alias(UnresolvedAttribute("request_path"), "request_path")()) |
| 199 | + val aggregatePlan = Aggregate( |
| 200 | + Seq( |
| 201 | + Alias(UnresolvedAttribute("status_code"), "status_code")(), |
| 202 | + Alias(UnresolvedAttribute("request_path"), "request_path")()), |
| 203 | + aggregateExpressions, |
| 204 | + fillNullPlan) |
| 205 | + val sortPlan = Sort( |
| 206 | + Seq( |
| 207 | + SortOrder(UnresolvedAttribute("request_path"), Ascending), |
| 208 | + SortOrder(UnresolvedAttribute("status_code"), Ascending)), |
| 209 | + global = true, |
| 210 | + aggregatePlan) |
| 211 | + val expectedPlan = Project(seq(UnresolvedStar(None)), sortPlan) |
| 212 | + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) |
| 213 | + } |
| 214 | + |
| 215 | + test("test fillnull with various null replacement value and stats and sort command") { |
| 216 | + val frame = sql(s""" |
| 217 | + | source = $testTable | fillnull using status_code = 500, request_path = '/home' |
| 218 | + | | stats count(status_code) by status_code, request_path |
| 219 | + | | sort request_path, status_code |
| 220 | + | """.stripMargin) |
| 221 | + |
| 222 | + assert(frame.columns.sameElements(Array("count(status_code)", "status_code", "request_path"))) |
| 223 | + val results: Array[Row] = frame.collect() |
| 224 | + val expectedResults: Array[Row] = |
| 225 | + Array( |
| 226 | + Row(1, 500, "/about"), |
| 227 | + Row(1, 500, "/contact"), |
| 228 | + Row(2, 200, "/home"), |
| 229 | + Row(1, 301, "/home"), |
| 230 | + Row(1, 403, "/home")) |
| 231 | + // Compare the results |
| 232 | + assert(results.sameElements(expectedResults)) |
| 233 | + |
| 234 | + val logicalPlan: LogicalPlan = frame.queryExecution.logical |
| 235 | + val fillNullPlan = fillNullExpectedPlan( |
| 236 | + Seq(("status_code", Literal(500)), ("request_path", Literal("/home"))), |
| 237 | + addDefaultProject = false) |
| 238 | + val aggregateExpressions = |
| 239 | + Seq( |
| 240 | + Alias( |
| 241 | + UnresolvedFunction( |
| 242 | + Seq("COUNT"), |
| 243 | + Seq(UnresolvedAttribute("status_code")), |
| 244 | + isDistinct = false), |
| 245 | + "count(status_code)")(), |
| 246 | + Alias(UnresolvedAttribute("status_code"), "status_code")(), |
| 247 | + Alias(UnresolvedAttribute("request_path"), "request_path")()) |
| 248 | + val aggregatePlan = Aggregate( |
| 249 | + Seq( |
| 250 | + Alias(UnresolvedAttribute("status_code"), "status_code")(), |
| 251 | + Alias(UnresolvedAttribute("request_path"), "request_path")()), |
| 252 | + aggregateExpressions, |
| 253 | + fillNullPlan) |
| 254 | + val sortPlan = Sort( |
| 255 | + Seq( |
| 256 | + SortOrder(UnresolvedAttribute("request_path"), Ascending), |
| 257 | + SortOrder(UnresolvedAttribute("status_code"), Ascending)), |
| 258 | + global = true, |
| 259 | + aggregatePlan) |
| 260 | + val expectedPlan = Project(seq(UnresolvedStar(None)), sortPlan) |
| 261 | + comparePlans(logicalPlan, expectedPlan, checkAnalysis = false) |
| 262 | + } |
| 263 | + |
| 264 | + test("test fillnull with one null replacement value and missing columns") { |
| 265 | + val ex = intercept[AnalysisException](sql(s""" |
| 266 | + | source = $testTable | fillnull with '!!!' in |
| 267 | + | """.stripMargin)) |
| 268 | + |
| 269 | + assert(ex.getMessage().contains("Syntax error ")) |
| 270 | + } |
| 271 | + |
| 272 | + test("test fillnull with various null replacement values and missing columns") { |
| 273 | + val ex = intercept[AnalysisException](sql(s""" |
| 274 | + | source = $testTable | fillnull using |
| 275 | + | """.stripMargin)) |
| 276 | + |
| 277 | + assert(ex.getMessage().contains("Syntax error ")) |
| 278 | + } |
| 279 | + |
| 280 | + private def fillNullExpectedPlan( |
| 281 | + nullReplacements: Seq[(String, Expression)], |
| 282 | + addDefaultProject: Boolean = true): LogicalPlan = { |
| 283 | + val table = UnresolvedRelation(Seq("spark_catalog", "default", "flint_ppl_test")) |
| 284 | + val renameProjectList = UnresolvedStar(None) +: nullReplacements.map { |
| 285 | + case (nullableColumn, nullReplacement) => |
| 286 | + Alias( |
| 287 | + UnresolvedFunction( |
| 288 | + "coalesce", |
| 289 | + Seq(UnresolvedAttribute(nullableColumn), nullReplacement), |
| 290 | + isDistinct = false), |
| 291 | + nullableColumn)() |
| 292 | + } |
| 293 | + val renameProject = Project(renameProjectList, table) |
| 294 | + val droppedColumns = |
| 295 | + nullReplacements.map(_._1).map(columnName => UnresolvedAttribute(columnName)) |
| 296 | + val dropSourceColumn = DataFrameDropColumns(droppedColumns, renameProject) |
| 297 | + if (addDefaultProject) { |
| 298 | + Project(seq(UnresolvedStar(None)), dropSourceColumn) |
| 299 | + } else { |
| 300 | + dropSourceColumn |
| 301 | + } |
| 302 | + } |
| 303 | +} |
0 commit comments