Github user rxin commented on a diff in the pull request: https://github.com/apache/spark/pull/21056#discussion_r181530121 --- Diff: sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonSuite.scala --- @@ -2128,38 +2128,60 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } } - test("SPARK-23849: schema inferring touches less data if samplingRation < 1.0") { - val predefinedSample = Set[Int](2, 8, 15, 27, 30, 34, 35, 37, 44, 46, + val sampledTestData = (row: Row) => { + val value = row.getLong(0) + val predefinedSample = Set[Long](2, 8, 15, 27, 30, 34, 35, 37, 44, 46, 57, 62, 68, 72) - withTempPath { path => - val writer = Files.newBufferedWriter(Paths.get(path.getAbsolutePath), - StandardCharsets.UTF_8, StandardOpenOption.CREATE_NEW) - for (i <- 0 until 100) { - if (predefinedSample.contains(i)) { - writer.write(s"""{"f1":${i.toString}}""" + "\n") - } else { - writer.write(s"""{"f1":${(i.toDouble + 0.1).toString}}""" + "\n") - } - } - writer.close() + if (predefinedSample.contains(value)) { + s"""{"f1":${value.toString}}""" + } else { + s"""{"f1":${(value.toDouble + 0.1).toString}}""" + } + } - val ds = spark.read.option("samplingRatio", 0.1).json(path.getCanonicalPath) + test("SPARK-23849: schema inferring touches less data if samplingRatio < 1.0") { + // Set default values for the DataSource parameters to make sure + // that whole test file is mapped to only one partition. This will guarantee + // reliable sampling of the input file. + withSQLConf( + "spark.sql.files.maxPartitionBytes" -> (128 * 1024 * 1024).toString, + "spark.sql.files.openCostInBytes" -> (4 * 1024 * 1024).toString + )(withTempPath { path => + val rdd = spark.sqlContext.range(0, 100, 1, 1).map(sampledTestData) + rdd.write.text(path.getAbsolutePath) + + val ds = spark.read + .option("inferSchema", true) + .option("samplingRatio", 0.1) + .json(path.getCanonicalPath) assert(ds.schema == new StructType().add("f1", LongType)) - } + }) } - test("SPARK-23849: usage of samplingRation while parsing of dataset of strings") { - val dstr = spark.sparkContext.parallelize(0 until 100, 1).map { i => - val predefinedSample = Set[Int](2, 8, 15, 27, 30, 34, 35, 37, 44, 46, - 57, 62, 68, 72) - if (predefinedSample.contains(i)) { - s"""{"f1":${i.toString}}""" + "\n" - } else { - s"""{"f1":${(i.toDouble + 0.1).toString}}""" + "\n" - } - }.toDS() - val ds = spark.read.option("samplingRatio", 0.1).json(dstr) + test("SPARK-23849: usage of samplingRatio while parsing a dataset of strings") { + val rdd = spark.sqlContext.range(0, 100, 1, 1).map(sampledTestData) + val ds = spark.read + .option("inferSchema", true) + .option("samplingRatio", 0.1) + .json(rdd) assert(ds.schema == new StructType().add("f1", LongType)) } + + test("SPARK-23849: samplingRatio is out of the range (0, 1.0]") { + val dstr = spark.sparkContext.parallelize(0 until 100, 1).map(_.toString).toDS() --- End diff -- can you just use spark.range?
--- --------------------------------------------------------------------- To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org