diff --git a/LICENSE b/LICENSE index 2a41ea294e461..f9e412cade345 100644 --- a/LICENSE +++ b/LICENSE @@ -922,7 +922,7 @@ The following components are provided under a BSD-style license. See project lin (BSD-like) Scala Compiler (org.scala-lang:scala-reflect:2.10.4 - http://www.scala-lang.org/) (BSD-like) Scala Library (org.scala-lang:scala-library:2.10.4 - http://www.scala-lang.org/) (BSD-like) Scalap (org.scala-lang:scalap:2.10.4 - http://www.scala-lang.org/) - (BSD-style) scalacheck (org.scalacheck:scalacheck_2.10:1.12.4 - http://www.scalacheck.org) + (BSD-style) scalacheck (org.scalacheck:scalacheck_2.10:1.10.0 - http://www.scalacheck.org) (BSD-style) spire (org.spire-math:spire_2.10:0.7.1 - http://spire-math.org) (BSD-style) spire-macros (org.spire-math:spire-macros_2.10:0.7.1 - http://spire-math.org) (New BSD License) Kryo (com.esotericsoftware.kryo:kryo:2.21 - http://code.google.com/p/kryo/) diff --git a/pom.xml b/pom.xml index 6a41aa4f65101..211da9ee74a3f 100644 --- a/pom.xml +++ b/pom.xml @@ -689,7 +689,7 @@ org.scalacheck scalacheck_${scala.binary.version} - 1.12.4 + 1.11.3 test diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala index ccf171543f8e4..13aad467fa578 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala @@ -17,17 +17,54 @@ package org.apache.spark.sql -import java.sql.Timestamp +import java.lang.Double.longBitsToDouble +import java.lang.Float.intBitsToFloat +import java.math.MathContext -import org.scalacheck.{Arbitrary, Gen} +import scala.util.Random import org.apache.spark.sql.types._ /** - * ScalaCheck random data generators for Spark SQL DataTypes. + * Random data generators for Spark SQL DataTypes. These generators do not generate uniformly random + * values; instead, they're biased to return "interesting" values (such as maximum / minimum values) + * with higher probability. */ 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 @@ -37,58 +74,85 @@ object RandomDataGenerator { * * @param dataType the type to generate values for * @param nullable whether null values should be generated - * @return a ScalaCheck [[Gen]] which can be used to produce random values. + * @param seed an optional seed for the random number generator + * @return a function which can be called to generate random values. */ def forType( dataType: DataType, - 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)) - 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]) + 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())) + case TimestampType => Some(() => new java.sql.Timestamp(rand.nextLong())) + case DecimalType.Unlimited => Some( + () => BigDecimal.apply(rand.nextLong, rand.nextInt, MathContext.UNLIMITED)) + 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) case ArrayType(elementType, containsNull) => { - forType(elementType, nullable = containsNull).map { elementGen => - Gen.listOf(elementGen).map(_.toArray) + forType(elementType, nullable = containsNull, seed = Some(rand.nextLong())).map { + elementGenerator => () => Array.fill(rand.nextInt(MAX_ARR_SIZE))(elementGenerator()) } } case MapType(keyType, valueType, valueContainsNull) => { for ( - 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] + keyGenerator <- forType(keyType, nullable = false, seed = Some(rand.nextLong())); + valueGenerator <- + forType(valueType, nullable = valueContainsNull, seed = Some(rand.nextLong())) ) yield { - Gen.listOf(Gen.zip(keyGenerator, valueGenerator)).map(_.toMap) + () => { + Seq.fill(rand.nextInt(MAX_MAP_SIZE))((keyGenerator(), valueGenerator())).toMap + } } } case StructType(fields) => { - val maybeFieldGenerators: Seq[Option[Gen[Any]]] = fields.map { field => - forType(field.dataType, nullable = field.nullable) + val maybeFieldGenerators: Seq[Option[() => Any]] = fields.map { field => + forType(field.dataType, nullable = field.nullable, seed = Some(rand.nextLong())) } if (maybeFieldGenerators.forall(_.isDefined)) { - Some(Gen.sequence[Seq[Any], Any](maybeFieldGenerators.flatten).map(vs => Row.fromSeq(vs))) + val fieldGenerators: Seq[() => Any] = maybeFieldGenerators.map(_.get) + Some(() => Row.fromSeq(fieldGenerators.map(_.apply()))) } else { None } } case unsupportedType => None } - if (nullable) { - valueGenerator.map(Gen.oneOf(_, Gen.const[Any](null))) - } else { - valueGenerator + // 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 + } } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala index dc07a732cdb1c..dbba93dba668e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala @@ -17,9 +17,6 @@ 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._ @@ -27,7 +24,7 @@ import org.apache.spark.sql.types._ /** * Tests of [[RandomDataGenerator]]. */ -class RandomDataGeneratorSuite extends SparkFunSuite with Checkers { +class RandomDataGeneratorSuite extends SparkFunSuite { /** * Tests random data generation for the given type by using it to generate random values then @@ -39,12 +36,14 @@ class RandomDataGeneratorSuite extends SparkFunSuite with Checkers { fail(s"Random data generator was not defined for $dataType") } if (nullable) { - check(exists(generator) { _ == null }) + assert(Iterator.fill(100)(generator()).contains(null)) + } else { + assert(Iterator.fill(100)(generator()).forall(_ != null)) } - if (!nullable) { - check(forAll(generator) { _ != null }) + for (_ <- 1 to 10) { + val generatedValue = generator() + toCatalyst(generatedValue) } - check(secure(forAll(generator) { v => { toCatalyst(v); true } })) } // Basic types: