Skip to content

[SPARK-30258][TESTS] Eliminate warnings of deprecated Spark APIs in tests #26885

New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Closed
wants to merge 8 commits into from
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,9 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession {
test("roundtrip in to_avro and from_avro - int and string") {
val df = spark.range(10).select('id, 'id.cast("string").as("str"))

val avroDF = df.select(to_avro('id).as("a"), to_avro('str).as("b"))
val avroDF = df.select(
functions.to_avro('id).as("a"),
functions.to_avro('str).as("b"))
val avroTypeLong = s"""
|{
| "type": "int",
Expand All @@ -51,12 +53,14 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession {
| "name": "str"
|}
""".stripMargin
checkAnswer(avroDF.select(from_avro('a, avroTypeLong), from_avro('b, avroTypeStr)), df)
checkAnswer(avroDF.select(
functions.from_avro('a, avroTypeLong),
functions.from_avro('b, avroTypeStr)), df)
}

test("roundtrip in to_avro and from_avro - struct") {
val df = spark.range(10).select(struct('id, 'id.cast("string").as("str")).as("struct"))
val avroStructDF = df.select(to_avro('struct).as("avro"))
val avroStructDF = df.select(functions.to_avro('struct).as("avro"))
val avroTypeStruct = s"""
|{
| "type": "record",
Expand All @@ -67,13 +71,14 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession {
| ]
|}
""".stripMargin
checkAnswer(avroStructDF.select(from_avro('avro, avroTypeStruct)), df)
checkAnswer(avroStructDF.select(
functions.from_avro('avro, avroTypeStruct)), df)
}

test("handle invalid input in from_avro") {
val count = 10
val df = spark.range(count).select(struct('id, 'id.as("id2")).as("struct"))
val avroStructDF = df.select(to_avro('struct).as("avro"))
val avroStructDF = df.select(functions.to_avro('struct).as("avro"))
val avroTypeStruct = s"""
|{
| "type": "record",
Expand All @@ -87,15 +92,15 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession {

intercept[SparkException] {
avroStructDF.select(
org.apache.spark.sql.avro.functions.from_avro(
functions.from_avro(
'avro, avroTypeStruct, Map("mode" -> "FAILFAST").asJava)).collect()
}

// For PERMISSIVE mode, the result should be row of null columns.
val expected = (0 until count).map(_ => Row(Row(null, null)))
checkAnswer(
avroStructDF.select(
org.apache.spark.sql.avro.functions.from_avro(
functions.from_avro(
'avro, avroTypeStruct, Map("mode" -> "PERMISSIVE").asJava)),
expected)
}
Expand All @@ -115,8 +120,8 @@ class AvroFunctionsSuite extends QueryTest with SharedSparkSession {
| }, "null" ]
|}, "null" ]
""".stripMargin
val readBackOne = dfOne.select(to_avro($"array").as("avro"))
.select(from_avro($"avro", avroTypeArrStruct).as("array"))
val readBackOne = dfOne.select(functions.to_avro($"array").as("avro"))
.select(functions.from_avro($"avro", avroTypeArrStruct).as("array"))
checkAnswer(dfOne, readBackOne)
}

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,123 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with
* this work for additional information regarding copyright ownership.
* The ASF licenses this file to You under the Apache License, Version 2.0
* (the "License"); you may not use this file except in compliance with
* the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/

package org.apache.spark.sql.avro

import java.io.ByteArrayOutputStream

import org.apache.avro.Schema
import org.apache.avro.generic.{GenericDatumWriter, GenericRecord, GenericRecordBuilder}
import org.apache.avro.io.EncoderFactory

import org.apache.spark.sql.{QueryTest, Row}
import org.apache.spark.sql.execution.LocalTableScanExec
import org.apache.spark.sql.functions.{col, struct}
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.test.SharedSparkSession

@deprecated("This test suite will be removed.", "3.0.0")
class DeprecatedAvroFunctionsSuite extends QueryTest with SharedSparkSession {
import testImplicits._

test("roundtrip in to_avro and from_avro - int and string") {
val df = spark.range(10).select('id, 'id.cast("string").as("str"))

val avroDF = df.select(to_avro('id).as("a"), to_avro('str).as("b"))
val avroTypeLong = s"""
|{
| "type": "int",
| "name": "id"
|}
""".stripMargin
val avroTypeStr = s"""
|{
| "type": "string",
| "name": "str"
|}
""".stripMargin
checkAnswer(avroDF.select(from_avro('a, avroTypeLong), from_avro('b, avroTypeStr)), df)
}

test("roundtrip in to_avro and from_avro - struct") {
val df = spark.range(10).select(struct('id, 'id.cast("string").as("str")).as("struct"))
val avroStructDF = df.select(to_avro('struct).as("avro"))
val avroTypeStruct = s"""
|{
| "type": "record",
| "name": "struct",
| "fields": [
| {"name": "col1", "type": "long"},
| {"name": "col2", "type": "string"}
| ]
|}
""".stripMargin
checkAnswer(avroStructDF.select(from_avro('avro, avroTypeStruct)), df)
}

test("roundtrip in to_avro and from_avro - array with null") {
val dfOne = Seq(Tuple1(Tuple1(1) :: Nil), Tuple1(null :: Nil)).toDF("array")
val avroTypeArrStruct = s"""
|[ {
| "type" : "array",
| "items" : [ {
| "type" : "record",
| "name" : "x",
| "fields" : [ {
| "name" : "y",
| "type" : "int"
| } ]
| }, "null" ]
|}, "null" ]
""".stripMargin
val readBackOne = dfOne.select(to_avro($"array").as("avro"))
.select(from_avro($"avro", avroTypeArrStruct).as("array"))
checkAnswer(dfOne, readBackOne)
}

test("SPARK-27798: from_avro produces same value when converted to local relation") {
val simpleSchema =
"""
|{
| "type": "record",
| "name" : "Payload",
| "fields" : [ {"name" : "message", "type" : "string" } ]
|}
""".stripMargin

def generateBinary(message: String, avroSchema: String): Array[Byte] = {
val schema = new Schema.Parser().parse(avroSchema)
val out = new ByteArrayOutputStream()
val writer = new GenericDatumWriter[GenericRecord](schema)
val encoder = EncoderFactory.get().binaryEncoder(out, null)
val rootRecord = new GenericRecordBuilder(schema).set("message", message).build()
writer.write(rootRecord, encoder)
encoder.flush()
out.toByteArray
}

// This bug is hit when the rule `ConvertToLocalRelation` is run. But the rule was excluded
// in `SharedSparkSession`.
withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "") {
val df = Seq("one", "two", "three", "four").map(generateBinary(_, simpleSchema))
.toDF()
.withColumn("value", from_avro(col("value"), simpleSchema))

assert(df.queryExecution.executedPlan.isInstanceOf[LocalTableScanExec])
assert(df.collect().map(_.get(0)) === Seq(Row("one"), Row("two"), Row("three"), Row("four")))
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -19,12 +19,10 @@ package org.apache.spark.sql

import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
import org.apache.spark.sql.expressions.Aggregator
import org.apache.spark.sql.expressions.scalalang.typed
import org.apache.spark.sql.functions._
import org.apache.spark.sql.test.SharedSparkSession
import org.apache.spark.sql.types.{BooleanType, IntegerType, StringType, StructType}


object ComplexResultAgg extends Aggregator[(String, Int), (Long, Long), (Long, Long)] {
override def zero: (Long, Long) = (0, 0)
override def reduce(countAndSum: (Long, Long), input: (String, Int)): (Long, Long) = {
Expand Down Expand Up @@ -226,25 +224,6 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession {

private implicit val ordering = Ordering.by((c: AggData) => c.a -> c.b)

test("typed aggregation: TypedAggregator") {
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()

checkDataset(
ds.groupByKey(_._1).agg(typed.sum(_._2)),
("a", 30.0), ("b", 3.0), ("c", 1.0))
}

test("typed aggregation: TypedAggregator, expr, expr") {
val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS()

checkDataset(
ds.groupByKey(_._1).agg(
typed.sum(_._2),
expr("sum(_2)").as[Long],
count("*")),
("a", 30.0, 30L, 2L), ("b", 3.0, 3L, 2L), ("c", 1.0, 1L, 1L))
}

test("typed aggregation: complex result type") {
val ds = Seq("a" -> 1, "a" -> 3, "b" -> 3).toDS()

Expand All @@ -255,17 +234,6 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession {
("a", 2.0, (2L, 4L)), ("b", 3.0, (1L, 3L)))
}

test("typed aggregation: in project list") {
val ds = Seq(1, 3, 2, 5).toDS()

checkDataset(
ds.select(typed.sum((i: Int) => i)),
11.0)
checkDataset(
ds.select(typed.sum((i: Int) => i), typed.sum((i: Int) => i * 2)),
11.0 -> 22.0)
}

test("typed aggregation: class input") {
val ds = Seq(AggData(1, "one"), AggData(2, "two")).toDS()

Expand Down Expand Up @@ -315,14 +283,6 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession {
("one", 1), ("two", 1))
}

test("typed aggregate: avg, count, sum") {
val ds = Seq("a" -> 1, "a" -> 3, "b" -> 3).toDS()
checkDataset(
ds.groupByKey(_._1).agg(
typed.avg(_._2), typed.count(_._2), typed.sum(_._2), typed.sumLong(_._2)),
("a", 2.0, 2L, 4.0, 4L), ("b", 3.0, 1L, 3.0, 3L))
}

test("generic typed sum") {
val ds = Seq("a" -> 1, "a" -> 3, "b" -> 3).toDS()
checkDataset(
Expand Down Expand Up @@ -366,18 +326,6 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession {
checkAnswer(df2.agg(RowAgg.toColumn as "b").select("b"), Row(6) :: Nil)
}

test("spark-15114 shorter system generated alias names") {
val ds = Seq(1, 3, 2, 5).toDS()
assert(ds.select(typed.sum((i: Int) => i)).columns.head === "TypedSumDouble(int)")
val ds2 = ds.select(typed.sum((i: Int) => i), typed.avg((i: Int) => i))
assert(ds2.columns.head === "TypedSumDouble(int)")
assert(ds2.columns.last === "TypedAverage(int)")
val df = Seq(1 -> "a", 2 -> "b", 3 -> "b").toDF("i", "j")
assert(df.groupBy($"j").agg(RowAgg.toColumn).columns.last ==
"RowAgg(org.apache.spark.sql.Row)")
assert(df.groupBy($"j").agg(RowAgg.toColumn as "agg1").columns.last == "agg1")
}

test("SPARK-15814 Aggregator can return null result") {
val ds = Seq(AggData(1, "one"), AggData(2, "two")).toDS()
checkDatasetUnorderly(
Expand All @@ -390,15 +338,6 @@ class DatasetAggregatorSuite extends QueryTest with SharedSparkSession {
checkDataset(ds.select(MapTypeBufferAgg.toColumn), 1)
}

test("SPARK-15204 improve nullability inference for Aggregator") {
val ds1 = Seq(1, 3, 2, 5).toDS()
assert(ds1.select(typed.sum((i: Int) => i)).schema.head.nullable === false)
val ds2 = Seq(AggData(1, "a"), AggData(2, "a")).toDS()
assert(ds2.select(SeqAgg.toColumn).schema.head.nullable)
val ds3 = sql("SELECT 'Some String' AS b, 1279869254 AS a").as[AggData]
assert(ds3.select(NameAgg.toColumn).schema.head.nullable)
}

test("SPARK-18147: very complex aggregator result type") {
val df = Seq(1 -> "a", 2 -> "b", 2 -> "c").toDF("i", "j")

Expand Down
Loading