Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
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
Prev Previous commit
Next Next commit
Initial attempt at using ScalaCheck.
  • Loading branch information
JoshRosen committed Jul 2, 2015
commit 0c209051777620b62a6cab4b18673dd145ca91c8
Original file line number Diff line number Diff line change
Expand Up @@ -17,12 +17,10 @@

package org.apache.spark.sql

import java.lang.Double.longBitsToDouble
import java.lang.Float.intBitsToFloat

import scala.util.Random
import java.sql.Timestamp

import org.apache.spark.sql.types._
import org.scalacheck.{Arbitrary, Gen}

/**
* Random data generators for Spark SQL DataTypes. These generators do not generate uniformly random
Expand All @@ -31,39 +29,6 @@ import org.apache.spark.sql.types._
*/
object RandomDataGenerator {

/**
* The conditional probability of a non-null value being drawn from a set of "interesting" values
* instead of being chosen uniformly at random.
*/
private val PROBABILITY_OF_INTERESTING_VALUE: Float = 0.5f

/**
* The probability of the generated value being null
*/
private val PROBABILITY_OF_NULL: Float = 0.1f

private val MAX_STR_LEN: Int = 1024
private val MAX_ARR_SIZE: Int = 128
private val MAX_MAP_SIZE: Int = 128

/**
* Helper function for constructing a biased random number generator which returns "interesting"
* values with a higher probability.
*/
private def randomNumeric[T](
rand: Random,
uniformRand: Random => T,
interestingValues: Seq[T]): Some[() => T] = {
val f = () => {
if (rand.nextFloat() <= PROBABILITY_OF_INTERESTING_VALUE) {
interestingValues(rand.nextInt(interestingValues.length))
} else {
uniformRand(rand)
}
}
Some(f)
}

/**
* Returns a function which generates random values for the given [[DataType]], or `None` if no
* random data generator is defined for that data type. The generated values will use an external
Expand All @@ -73,82 +38,58 @@ object RandomDataGenerator {
*
* @param dataType the type to generate values for
* @param nullable whether null values should be generated
* @param seed an optional seed for the random number generator
* @return a function which can be called to generate random values.
* @return a ScalaCheck [[Gen]] which can be used to produce random values.
*/
def forType(
dataType: DataType,
nullable: Boolean = true,
seed: Option[Long] = None): Option[() => Any] = {
val rand = new Random()
seed.foreach(rand.setSeed)

val valueGenerator: Option[() => Any] = dataType match {
case StringType => Some(() => rand.nextString(rand.nextInt(MAX_STR_LEN)))
case BinaryType => Some(() => {
val arr = new Array[Byte](rand.nextInt(MAX_STR_LEN))
rand.nextBytes(arr)
arr
})
case BooleanType => Some(() => rand.nextBoolean())
case DateType => Some(() => new java.sql.Date(rand.nextInt(Int.MaxValue)))
case DoubleType => randomNumeric[Double](
rand, r => longBitsToDouble(r.nextLong()), Seq(Double.MinValue, Double.MinPositiveValue,
Double.MaxValue, Double.PositiveInfinity, Double.NegativeInfinity, Double.NaN, 0.0))
case FloatType => randomNumeric[Float](
rand, r => intBitsToFloat(r.nextInt()), Seq(Float.MinValue, Float.MinPositiveValue,
Float.MaxValue, Float.PositiveInfinity, Float.NegativeInfinity, Float.NaN, 0.0f))
case ByteType => randomNumeric[Byte](
rand, _.nextInt().toByte, Seq(Byte.MinValue, Byte.MaxValue, 0.toByte))
case IntegerType => randomNumeric[Int](
rand, _.nextInt(), Seq(Int.MinValue, Int.MaxValue, 0))
case LongType => randomNumeric[Long](
rand, _.nextLong(), Seq(Long.MinValue, Long.MaxValue, 0L))
case ShortType => randomNumeric[Short](
rand, _.nextInt().toShort, Seq(Short.MinValue, Short.MaxValue, 0.toShort))
case NullType => Some(() => null)
nullable: Boolean = true): Option[Gen[Any]] = {
val valueGenerator: Option[Gen[Any]] = dataType match {
case StringType => Some(Arbitrary.arbitrary[String])
case BinaryType => Some(Gen.listOf(Arbitrary.arbitrary[Byte]).map(_.toArray))
Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Note that I mapped a list into an Array instead of using Arbitrary.arbitrary[Array[Byte]] because the latter seems to run into a possible ScalaCheck NPE bug when we wrap the resulting generator to sometimes return nulls. I've filed typelevel/scalacheck#177 to try to investigate this upstream. AFAIK the only downside to the workaround here is that we lose some of the failing test case minimization benefits that ScalaCheck provides.

case BooleanType => Some(Arbitrary.arbitrary[Boolean])
case DateType => Some(Arbitrary.arbitrary[Int].suchThat(_ >= 0).map(new java.sql.Date(_)))
case DoubleType => Some(Arbitrary.arbitrary[Double])
case FloatType => Some(Arbitrary.arbitrary[Float])
case ByteType => Some(Arbitrary.arbitrary[Byte])
case IntegerType => Some(Arbitrary.arbitrary[Int])
case LongType => Some(Arbitrary.arbitrary[Long])
case ShortType => Some(Arbitrary.arbitrary[Short])
case NullType => Some(Gen.const[Any](null))
case TimestampType => Some(Arbitrary.arbitrary[Long].suchThat(_ >= 0).map(new Timestamp(_)))
case DecimalType.Unlimited => Some(Arbitrary.arbitrary[BigDecimal])
case ArrayType(elementType, containsNull) => {
forType(elementType, nullable = containsNull, seed = Some(rand.nextLong())).map {
elementGenerator => () => Array.fill(rand.nextInt(MAX_ARR_SIZE))(elementGenerator())
forType(elementType, nullable = containsNull).map { elementGen =>
Gen.listOf(elementGen).map(_.toArray)
}
}
case MapType(keyType, valueType, valueContainsNull) => {
for (
keyGenerator <- forType(keyType, nullable = false, seed = Some(rand.nextLong()));
valueGenerator <-
forType(valueType, nullable = valueContainsNull, seed = Some(rand.nextLong()))
keyGenerator <- forType(keyType, nullable = false);
valueGenerator <- forType(valueType, nullable = valueContainsNull)
// Scala's BigDecimal.hashCode can lead to OutOfMemoryError on Scala 2.10 (see SI-6173)
// and Spark can hit NumberFormatException errors converting certain BigDecimals
// (SPARK-8802). For these reasons, we don't support generation of maps with decimal keys.
if !keyType.isInstanceOf[DecimalType]
) yield {
() => {
Seq.fill(rand.nextInt(MAX_MAP_SIZE))((keyGenerator(), valueGenerator())).toMap
}
Gen.listOf(Gen.zip(keyGenerator, valueGenerator)).map(_.toMap)
}
}
case StructType(fields) => {
val maybeFieldGenerators: Seq[Option[() => Any]] = fields.map { field =>
forType(field.dataType, nullable = field.nullable, seed = Some(rand.nextLong()))
val maybeFieldGenerators: Seq[Option[Gen[Any]]] = fields.map { field =>
forType(field.dataType, nullable = field.nullable)
}
if (maybeFieldGenerators.forall(_.isDefined)) {
val fieldGenerators: Seq[() => Any] = maybeFieldGenerators.map(_.get)
Some(() => Row.fromSeq(fieldGenerators.map(_.apply())))
Some(Gen.sequence[Seq[Any], Any](maybeFieldGenerators.flatten).map(vs => Row.fromSeq(vs)))
} else {
None
}
}
case unsupportedType => None
}
// Handle nullability by wrapping the non-null value generator:
valueGenerator.map { valueGenerator =>
if (nullable) {
() => {
if (rand.nextFloat() <= PROBABILITY_OF_NULL) {
null
} else {
valueGenerator()
}
}
} else {
valueGenerator
}
if (nullable) {
valueGenerator.map(Gen.oneOf(_, Gen.const[Any](null)))
} else {
valueGenerator
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -17,46 +17,68 @@

package org.apache.spark.sql

import org.scalacheck.Prop.{exists, forAll, secure}
import org.scalatest.prop.Checkers

import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.catalyst.CatalystTypeConverters
import org.apache.spark.sql.types._

/**
* Tests of [[RandomDataGenerator]].
*/
class RandomDataGeneratorSuite extends SparkFunSuite {
class RandomDataGeneratorSuite extends SparkFunSuite with Checkers {

/**
* Tests random data generation for the given type by using it to generate random values then
* converting those values into their Catalyst equivalents using CatalystTypeConverters.
*/
def testRandomDataGeneration(dataType: DataType, nullable: Boolean = true): Unit = {
val toCatalyst = CatalystTypeConverters.createToCatalystConverter(dataType)
RandomDataGenerator.forType(dataType, nullable, Some(42L)).foreach { generator =>
for (_ <- 1 to 10) {
val generatedValue = generator()
val convertedValue = toCatalyst(generatedValue)
if (!nullable) {
assert(convertedValue !== null)
}
}
val generator = RandomDataGenerator.forType(dataType, nullable).getOrElse {
fail(s"Random data generator was not defined for $dataType")
}
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Should we throw an exception if no generator is defined for the given DataType?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Good idea; this uncovered the fact that I forgot to implement a generator for Timestamp.


if (nullable) {
check(exists(generator) { _ == null })
}
if (!nullable) {
check(forAll(generator) { _ != null })
}
check(secure(forAll(generator) { v => { toCatalyst(v); true } }))
}

// Basic types:

(DataTypeTestUtils.atomicTypes ++ DataTypeTestUtils.atomicArrayTypes).foreach { dataType =>
test(s"$dataType") {
for (
dataType <- DataTypeTestUtils.atomicTypes;
nullable <- Seq(true, false)
if !dataType.isInstanceOf[DecimalType] ||
dataType.asInstanceOf[DecimalType].precisionInfo.isEmpty
) {
test(s"$dataType (nullable=$nullable)") {
testRandomDataGeneration(dataType)
}
}

// Complex types:
for (
arrayType <- DataTypeTestUtils.atomicArrayTypes
if RandomDataGenerator.forType(arrayType.elementType, arrayType.containsNull).isDefined
) {
test(s"$arrayType") {
testRandomDataGeneration(arrayType)
}
}

val atomicTypesWithDataGenerators =
DataTypeTestUtils.atomicTypes.filter(RandomDataGenerator.forType(_).isDefined)

// Complex types:
for (
keyType <- DataTypeTestUtils.atomicTypes;
valueType <- DataTypeTestUtils.atomicTypes
keyType <- atomicTypesWithDataGenerators;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Probably use for { ... } for multi-line for-comprehension and thus we can remove the trailing ;.

valueType <- atomicTypesWithDataGenerators
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Value type of a map can be any type.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Yeah, I'm going to come back and fix this.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

It might be worth deferring this to a followup patch; I think that it should work but don't want to necessarily test all maps-of-maps or maps-of-arrays here.

// Scala's BigDecimal.hashCode can lead to OutOfMemoryError on Scala 2.10 (see SI-6173) and
// Spark can hit NumberFormatException errors when converting certain BigDecimals (SPARK-8802).
// For these reasons, we don't support generation of maps with decimal keys.
if !keyType.isInstanceOf[DecimalType]
) {
val mapType = MapType(keyType, valueType)
test(s"$mapType") {
Expand All @@ -65,8 +87,8 @@ class RandomDataGeneratorSuite extends SparkFunSuite {
}

for (
colOneType <- DataTypeTestUtils.atomicTypes;
colTwoType <- DataTypeTestUtils.atomicTypes
colOneType <- atomicTypesWithDataGenerators;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Remove the trailing ;

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We need the semicolons when we're doing a for comprehension over multiple inputs.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Oh I didn't notice you were using (). You can omit the ; if you use {} instead.

Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

hm to me it is less clear to drop the ; here, although i don't have a strong preference

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I don't have a super strong preference either.

colTwoType <- atomicTypesWithDataGenerators
) {
val structType = StructType(StructField("a", colOneType) :: StructField("b", colTwoType) :: Nil)
test(s"$structType") {
Expand Down