summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
-rw-r--r--src/library/scala/collection/mutable/HashTable.scala116
-rw-r--r--src/library/scala/collection/package.scala13
-rw-r--r--src/library/scala/collection/parallel/ParIterableLike.scala9
-rw-r--r--src/library/scala/collection/parallel/mutable/ParHashMap.scala92
-rw-r--r--src/library/scala/collection/parallel/package.scala2
-rw-r--r--test/files/scalacheck/parallel-collections/IntOperators.scala105
-rw-r--r--test/files/scalacheck/parallel-collections/ParallelHashMapCheck.scala24
-rw-r--r--test/files/scalacheck/parallel-collections/ParallelIterableCheck.scala529
-rw-r--r--test/files/scalacheck/parallel-collections/ParallelMapCheck1.scala14
-rw-r--r--test/files/scalacheck/parallel-collections/pc.scala12
10 files changed, 537 insertions, 379 deletions
diff --git a/src/library/scala/collection/mutable/HashTable.scala b/src/library/scala/collection/mutable/HashTable.scala
index 3d31c3860e..c233bd2719 100644
--- a/src/library/scala/collection/mutable/HashTable.scala
+++ b/src/library/scala/collection/mutable/HashTable.scala
@@ -31,7 +31,7 @@ package mutable
*
* @tparam A type of the elements contained in this hash table.
*/
-trait HashTable[A, Entry >: Null <: HashEntry[A, Entry]] {
+trait HashTable[A, Entry >: Null <: HashEntry[A, Entry]] extends HashTable.HashUtils[A] {
import HashTable._
@transient protected var _loadFactor = defaultLoadFactor
@@ -218,10 +218,7 @@ trait HashTable[A, Entry >: Null <: HashEntry[A, Entry]] {
}
@transient protected var sizemap: Array[Int] = null
- protected final def sizeMapBucketBitSize = 5
- // so that:
- protected final def sizeMapBucketSize = 1 << sizeMapBucketBitSize
- protected final def totalSizeMapBuckets = if (sizeMapBucketSize < table.length) 1 else table.length / sizeMapBucketSize
+ private[collection] final def totalSizeMapBuckets = if (sizeMapBucketSize < table.length) 1 else table.length / sizeMapBucketSize
/*
* The following three sizeMap* functions (Add, Remove, Reset)
@@ -298,39 +295,6 @@ trait HashTable[A, Entry >: Null <: HashEntry[A, Entry]] {
protected def elemEquals(key1: A, key2: A): Boolean = (key1 == key2)
- protected def elemHashCode(key: A) = if (key == null) 0 else key.##
-
- protected final def improve(hcode: Int) = {
- /* Murmur hash
- * m = 0x5bd1e995
- * r = 24
- * note: h = seed = 0 in mmix
- * mmix(h,k) = k *= m; k ^= k >> r; k *= m; h *= m; h ^= k; */
- var k = hcode * 0x5bd1e995
- k ^= k >> 24
- k *= 0x5bd1e995
- k
-
- /* Jenkins hash
- * for range 0-10000, output has the msb set to zero */
- // var h = hcode + (hcode << 12)
- // h ^= (h >> 22)
- // h += (h << 4)
- // h ^= (h >> 9)
- // h += (h << 10)
- // h ^= (h >> 2)
- // h += (h << 7)
- // h ^= (h >> 12)
- // h
-
- /* OLD VERSION
- * since 2003 */
- // var h: Int = hcode + ~(hcode << 9)
- // h = h ^ (h >>> 14)
- // h = h + (h << 4)
- // h ^ (h >>> 10)
- }
-
// Note:
// we take the most significant bits of the hashcode, not the lower ones
// this is of crucial importance when populating the table in parallel
@@ -380,6 +344,68 @@ private[collection] object HashTable {
private[collection] final def capacity(expectedSize: Int) = if (expectedSize == 0) 1 else powerOfTwo(expectedSize)
+ trait HashUtils[KeyType] {
+ protected final def sizeMapBucketBitSize = 5
+ // so that:
+ protected final def sizeMapBucketSize = 1 << sizeMapBucketBitSize
+
+ protected def elemHashCode(key: KeyType) = if (key == null) 0 else key.##
+
+ protected final def improve(hcode: Int) = {
+ /* Murmur hash
+ * m = 0x5bd1e995
+ * r = 24
+ * note: h = seed = 0 in mmix
+ * mmix(h,k) = k *= m; k ^= k >> r; k *= m; h *= m; h ^= k; */
+ // var k = hcode * 0x5bd1e995
+ // k ^= k >> 24
+ // k *= 0x5bd1e995
+ // k
+
+ /* Another fast multiplicative hash
+ * by Phil Bagwell
+ *
+ * Comment:
+ * Multiplication doesn't affect all the bits in the same way, so we want to
+ * multiply twice, "once from each side".
+ * It would be ideal to reverse all the bits after the first multiplication,
+ * however, this is more costly. We therefore restrict ourselves only to
+ * reversing the bytes before final multiplication. This yields a slightly
+ * worse entropy in the lower 8 bits, but that can be improved by adding:
+ *
+ * `i ^= i >> 6`
+ *
+ * For performance reasons, we avoid this improvement.
+ * */
+ var i = hcode * 0x9e3775cd
+ i = java.lang.Integer.reverseBytes(i)
+ i * 0x9e3775cd
+ // a slower alternative for byte reversal:
+ // i = (i << 16) | (i >> 16)
+ // i = ((i >> 8) & 0x00ff00ff) | ((i << 8) & 0xff00ff00)
+
+ /* Jenkins hash
+ * for range 0-10000, output has the msb set to zero */
+ // var h = hcode + (hcode << 12)
+ // h ^= (h >> 22)
+ // h += (h << 4)
+ // h ^= (h >> 9)
+ // h += (h << 10)
+ // h ^= (h >> 2)
+ // h += (h << 7)
+ // h ^= (h >> 12)
+ // h
+
+ /* OLD VERSION
+ * quick, but bad for sequence 0-10000 - little enthropy in higher bits
+ * since 2003 */
+ // var h: Int = hcode + ~(hcode << 9)
+ // h = h ^ (h >>> 14)
+ // h = h + (h << 4)
+ // h ^ (h >>> 10)
+ }
+ }
+
/**
* Returns a power of two >= `target`.
*/
@@ -400,6 +426,18 @@ private[collection] object HashTable {
val tableSize: Int,
val threshold: Int,
val sizemap: Array[Int]
- )
+ ) {
+ import collection.DebugUtils._
+ private[collection] def debugInformation = buildString {
+ append =>
+ append("Hash table contents")
+ append("-------------------")
+ append("Table: [" + arrayString(table, 0, table.length) + "]")
+ append("Table size: " + tableSize)
+ append("Load factor: " + loadFactor)
+ append("Threshold: " + threshold)
+ append("Sizemap: [" + arrayString(sizemap, 0, sizemap.length) + "]")
+ }
+ }
}
diff --git a/src/library/scala/collection/package.scala b/src/library/scala/collection/package.scala
index 19d65b73e2..f1eb50c5e0 100644
--- a/src/library/scala/collection/package.scala
+++ b/src/library/scala/collection/package.scala
@@ -76,4 +76,17 @@ package object collection {
new CanBuildFrom[From, T, To] { // TODO: could we just return b instead?
def apply(from: From) = b.apply() ; def apply() = b.apply()
}
+
+ private[collection] object DebugUtils {
+ /* debug utils */
+ def buildString(closure: (String => Unit) => Unit): String = {
+ var output = ""
+ def appendln(s: String) = output += s + "\n"
+ closure(appendln)
+ output
+ }
+
+ def arrayString[T](array: Array[T], from: Int, until: Int) = array.slice(from, until).map(x => if (x != null) x.toString else "n/a").mkString(" | ")
+ }
+
}
diff --git a/src/library/scala/collection/parallel/ParIterableLike.scala b/src/library/scala/collection/parallel/ParIterableLike.scala
index 348068c78c..681fe3570e 100644
--- a/src/library/scala/collection/parallel/ParIterableLike.scala
+++ b/src/library/scala/collection/parallel/ParIterableLike.scala
@@ -275,7 +275,6 @@ self =>
* if this $coll is empty.
*/
def reduce[U >: T](op: (U, U) => U): U = {
- println("------------------------------------------------")
executeAndWaitResult(new Reduce(op, parallelIterator) mapResult { _.get })
}
@@ -758,7 +757,7 @@ self =>
protected[this] class Reduce[U >: T](op: (U, U) => U, protected[this] val pit: ParIterableIterator[T]) extends Accessor[Option[U], Reduce[U]] {
var result: Option[U] = None
def leaf(prevr: Option[Option[U]]) = {
- // pit.printDebugInformation
+ // pit.debugInformation
// val rem = pit.remaining
// val lst = pit.toList
// val pa = mutable.ParArray(lst: _*)
@@ -1227,6 +1226,12 @@ self =>
override def merge(that: FromArray[S, A, That]) = result = result combine that.result
}
+ /* debug information */
+
+ private[parallel] def debugInformation = "Parallel collection: " + this.getClass
+
+ private[parallel] def brokenInvariants = Seq[String]()
+
}
diff --git a/src/library/scala/collection/parallel/mutable/ParHashMap.scala b/src/library/scala/collection/parallel/mutable/ParHashMap.scala
index c5b404e092..66c491086f 100644
--- a/src/library/scala/collection/parallel/mutable/ParHashMap.scala
+++ b/src/library/scala/collection/parallel/mutable/ParHashMap.scala
@@ -75,6 +75,33 @@ self =>
new ParHashMapIterator(idxFrom, idxUntil, totalSz, es) with SCPI
}
+ private[parallel] override def brokenInvariants = {
+ // bucket by bucket, count elements
+ val buckets = for (i <- 0 until (table.length / sizeMapBucketSize)) yield checkBucket(i)
+
+ // check if each element is in the position corresponding to its key
+ val elems = for (i <- 0 until table.length) yield checkEntry(i)
+
+ buckets.flatMap(x => x) ++ elems.flatMap(x => x)
+ }
+
+ private def checkBucket(i: Int) = {
+ def count(e: HashEntry[K, DefaultEntry[K, V]]): Int = if (e eq null) 0 else 1 + count(e.next)
+ val expected = sizemap(i)
+ val found = ((i * sizeMapBucketSize) until ((i + 1) * sizeMapBucketSize)).foldLeft(0) {
+ (acc, c) => acc + count(table(c))
+ }
+ if (found != expected) List("Found " + found + " elements, while sizemap showed " + expected)
+ else Nil
+ }
+
+ private def checkEntry(i: Int) = {
+ def check(e: HashEntry[K, DefaultEntry[K, V]]): List[String] = if (e eq null) Nil else
+ if (index(elemHashCode(e.key)) == i) check(e.next)
+ else ("Element " + e.key + " at " + i + " with " + elemHashCode(e.key) + " maps to " + index(elemHashCode(e.key))) :: check(e.next)
+ check(table(i))
+ }
+
}
@@ -88,14 +115,17 @@ object ParHashMap extends ParMapFactory[ParHashMap] {
private[mutable] abstract class ParHashMapCombiner[K, V](private val tableLoadFactor: Int)
-extends collection.parallel.BucketCombiner[(K, V), ParHashMap[K, V], DefaultEntry[K, V], ParHashMapCombiner[K, V]](ParHashMapCombiner.numblocks) {
+extends collection.parallel.BucketCombiner[(K, V), ParHashMap[K, V], DefaultEntry[K, V], ParHashMapCombiner[K, V]](ParHashMapCombiner.numblocks)
+ with collection.mutable.HashTable.HashUtils[K]
+{
self: EnvironmentPassingCombiner[(K, V), ParHashMap[K, V]] =>
private var mask = ParHashMapCombiner.discriminantmask
+ private var nonmasklen = ParHashMapCombiner.nonmasklength
def +=(elem: (K, V)) = {
sz += 1
- val hc = elem._1.##
- val pos = hc & mask
+ val hc = improve(elemHashCode(elem._1))
+ val pos = (hc >>> nonmasklen)
if (lasts(pos) eq null) {
// initialize bucket
heads(pos) = new Unrolled[DefaultEntry[K, V]]
@@ -106,14 +136,28 @@ self: EnvironmentPassingCombiner[(K, V), ParHashMap[K, V]] =>
this
}
- def result: ParHashMap[K, V] = {
+ def result: ParHashMap[K, V] = if (size >= (ParHashMapCombiner.numblocks * sizeMapBucketSize)) { // 1024
// construct table
val table = new AddingHashTable(size, tableLoadFactor)
-
val insertcount = executeAndWaitResult(new FillBlocks(heads, table, 0, ParHashMapCombiner.numblocks))
-
+ table.setSize(insertcount)
// TODO compare insertcount and size to see if compression is needed
-
+ val c = table.hashTableContents
+ new ParHashMap(c)
+ } else {
+ // construct a normal table and fill it sequentially
+ val table = new HashTable[K, DefaultEntry[K, V]] {
+ def insertEntry(e: DefaultEntry[K, V]) = if (super.findEntry(e.key) eq null) super.addEntry(e)
+ sizeMapInit(table.length)
+ }
+ var i = 0
+ while (i < ParHashMapCombiner.numblocks) {
+ if (heads(i) ne null) {
+ for (elem <- heads(i)) table.insertEntry(elem)
+ }
+ i += 1
+ }
+ // TODO compression
val c = table.hashTableContents
new ParHashMap(c)
}
@@ -123,17 +167,20 @@ self: EnvironmentPassingCombiner[(K, V), ParHashMap[K, V]] =>
*
* Entries are added using the `insertEntry` method. This method checks whether the element
* exists and updates the size map. It returns false if the key was already in the table,
- * and true if the key was successfully inserted.
+ * and true if the key was successfully inserted. It does not update the number of elements
+ * in the table.
*/
- class AddingHashTable(numelems: Int, lf: Int) extends HashTable[K, DefaultEntry[K, V]] {
+ private[ParHashMapCombiner] class AddingHashTable(numelems: Int, lf: Int) extends HashTable[K, DefaultEntry[K, V]] {
import HashTable._
_loadFactor = lf
table = new Array[HashEntry[K, DefaultEntry[K, V]]](capacity(sizeForThreshold(_loadFactor, numelems)))
tableSize = 0
threshold = newThreshold(_loadFactor, table.length)
sizeMapInit(table.length)
- def insertEntry(e: DefaultEntry[K, V]) = {
+ def setSize(sz: Int) = tableSize = sz
+ def insertEntry(block: Int, e: DefaultEntry[K, V]) = {
var h = index(elemHashCode(e.key))
+ // assertCorrectBlock(h, block)
var olde = table(h).asInstanceOf[DefaultEntry[K, V]]
// check if key already exists
@@ -149,11 +196,17 @@ self: EnvironmentPassingCombiner[(K, V), ParHashMap[K, V]] =>
if (h != -1) {
e.next = olde
table(h) = e
- tableSize = tableSize + 1
nnSizeMapAdd(h)
true
} else false
}
+ private def assertCorrectBlock(h: Int, block: Int) {
+ val blocksize = table.length / (1 << ParHashMapCombiner.discriminantbits)
+ if (!(h >= block * blocksize && h < (block + 1) * blocksize)) {
+ println("trying to put " + h + " into block no.: " + block + ", range: [" + block * blocksize + ", " + (block + 1) * blocksize + ">")
+ assert(h >= block * blocksize && h < (block + 1) * blocksize)
+ }
+ }
}
/* tasks */
@@ -166,11 +219,11 @@ self: EnvironmentPassingCombiner[(K, V), ParHashMap[K, V]] =>
val until = offset + howmany
result = 0
while (i < until) {
- result += fillBlock(buckets(i))
+ result += fillBlock(i, buckets(i))
i += 1
}
}
- private def fillBlock(elems: Unrolled[DefaultEntry[K, V]]) = {
+ private def fillBlock(block: Int, elems: Unrolled[DefaultEntry[K, V]]) = {
var insertcount = 0
var unrolled = elems
var i = 0
@@ -180,7 +233,8 @@ self: EnvironmentPassingCombiner[(K, V), ParHashMap[K, V]] =>
val chunksz = unrolled.size
while (i < chunksz) {
val elem = chunkarr(i)
- if (t.insertEntry(elem)) insertcount += 1
+ // assertCorrectBlock(block, elem.key)
+ if (t.insertEntry(block, elem)) insertcount += 1
i += 1
}
i = 0
@@ -188,6 +242,13 @@ self: EnvironmentPassingCombiner[(K, V), ParHashMap[K, V]] =>
}
insertcount
}
+ private def assertCorrectBlock(block: Int, k: K) {
+ val hc = improve(elemHashCode(k))
+ if ((hc >>> nonmasklen) != block) {
+ println(hc + " goes to " + (hc >>> nonmasklen) + ", while expected block is " + block)
+ assert((hc >>> nonmasklen) == block)
+ }
+ }
def split = {
val fp = howmany / 2
List(new FillBlocks(buckets, table, offset, fp), new FillBlocks(buckets, table, offset + fp, howmany - fp))
@@ -204,7 +265,8 @@ self: EnvironmentPassingCombiner[(K, V), ParHashMap[K, V]] =>
private[mutable] object ParHashMapCombiner {
private[mutable] val discriminantbits = 5
private[mutable] val numblocks = 1 << discriminantbits
- private[mutable] val discriminantmask = ((1 << discriminantbits) - 1) << (32 - discriminantbits)
+ private[mutable] val discriminantmask = ((1 << discriminantbits) - 1);
+ private[mutable] val nonmasklength = 32 - discriminantbits
def apply[K, V] = new ParHashMapCombiner[K, V](HashTable.defaultLoadFactor) with EnvironmentPassingCombiner[(K, V), ParHashMap[K, V]]
}
diff --git a/src/library/scala/collection/parallel/package.scala b/src/library/scala/collection/parallel/package.scala
index a694aeba17..1a3b35c853 100644
--- a/src/library/scala/collection/parallel/package.scala
+++ b/src/library/scala/collection/parallel/package.scala
@@ -219,6 +219,8 @@ package object parallel {
} else this
}
+
+
}
diff --git a/test/files/scalacheck/parallel-collections/IntOperators.scala b/test/files/scalacheck/parallel-collections/IntOperators.scala
index 2b7e0191a2..8d214b614f 100644
--- a/test/files/scalacheck/parallel-collections/IntOperators.scala
+++ b/test/files/scalacheck/parallel-collections/IntOperators.scala
@@ -10,82 +10,85 @@ trait IntOperators extends Operators[Int] {
def forallPredicates = List(_ >= 0, _ < 0, _ % 2 == 0, _ != 55, _ != 505, _ != 5005)
def existsPredicates = List(_ >= 0, _ < 0, _ % 2 == 0, _ == 55, _ == 505, _ == 5005)
def findPredicates = List(_ >= 0, _ % 2 == 0, _ < 0, _ == 50, _ == 500, _ == 5000)
- def mapFunctions = List(-_)
+ def mapFunctions = List(-_, math.abs(_), _ % 2, _ % 3, _ % 4, _ % 150, _ % 500)
def partialMapFunctions = List({case x => -x}, { case 0 => -1; case x if x > 0 => x + 1}, {case x if x % 3 == 0 => x / 3})
def flatMapFunctions = List(
- (n: Int) => if (n < 0) List() else if (n % 2 == 0) List(1, 2, 3) else List(4, 5, 6),
- (n: Int) => List[Int](),
- (n: Int) => if (n == 0) List(1, 2, 3, 4, 5) else if (n < 0) List(1, 2, 3) else List()
- )
+ (n: Int) => if (n < 0) List() else if (n % 2 == 0) List(1, 2, 3) else List(4, 5, 6),
+ (n: Int) => List[Int](),
+ (n: Int) => if (n == 0) List(1, 2, 3, 4, 5) else if (n < 0) List(1, 2, 3) else List()
+ )
def filterPredicates = List(
- _ % 2 == 0, _ % 3 == 0, n => n > 50 && n < 100, _ >= 0, _ < 0, _ == 99,
- _ > 500, _ > 5000, _ > 50000)
+ _ % 2 == 0, _ % 3 == 0, _ % 4 != 0, _ % 17 != 0, n => n > 50 && n < 100, _ >= 0, _ < 0, _ == 99,
+ _ > 500, _ > 5000, _ > 50000, _ < 500, _ < 50, _ < -50, _ < -5e5,
+ x => true, x => false, x => x % 53 == 0 && x % 17 == 0
+ )
def filterNotPredicates = filterPredicates
def partitionPredicates = filterPredicates
def takeWhilePredicates = List(
- _ != 50, _ != 500, _ != 5000, _ != 50000, _ % 2 == 0, _ % 3 == 1, _ % 47 != 0,
- _ < 100, _ < 1000, _ < 10000, _ < 0,
- _ < -100, _ < -1000, _ > -200, _ > -50,
- n => -90 < n && n < -10,
- n => 50 < n && n < 550,
- n => 5000 < n && n < 7500)
+ _ != 50, _ != 500, _ != 5000, _ != 50000, _ % 2 == 0, _ % 3 == 1, _ % 47 != 0,
+ _ < 100, _ < 1000, _ < 10000, _ < 0,
+ _ < -100, _ < -1000, _ > -200, _ > -50,
+ n => -90 < n && n < -10,
+ n => 50 < n && n < 550,
+ n => 5000 < n && n < 7500
+ )
def dropWhilePredicates = takeWhilePredicates
def spanPredicates = takeWhilePredicates
def foldArguments = List(
- (0, _ + _),
- (1, _ * _),
- (Int.MinValue, math.max(_, _))
- )
+ (0, _ + _),
+ (1, _ * _),
+ (Int.MinValue, math.max(_, _))
+ )
def addAllTraversables = List(
- List[Int](),
- List(1),
- List(1, 2),
- List(1, 2, 3, 4, 5, 6, 7, 8, 9, 10),
- Array.fill(1000)(1).toSeq
- )
+ List[Int](),
+ List(1),
+ List(1, 2),
+ List(1, 2, 3, 4, 5, 6, 7, 8, 9, 10),
+ Array.fill(1000)(1).toSeq
+ )
def newArray(sz: Int) = new Array[Int](sz)
}
trait IntSeqOperators extends IntOperators with SeqOperators[Int] {
def segmentLengthPredicates = List(
- _ % 2 == 0, _ > 0, _ >= 0, _ < 0, _ <= 0, _ > -5000, _ > 5000, _ % 541 != 0, _ < -50, _ > 500,
- n => -90 < n && n < -10, n => 500 < n && n < 1500
+ _ % 2 == 0, _ > 0, _ >= 0, _ < 0, _ <= 0, _ > -5000, _ > 5000, _ % 541 != 0, _ < -50, _ > 500,
+ n => -90 < n && n < -10, n => 500 < n && n < 1500
)
def indexWherePredicates = List(
- _ % 2 == 0, _ % 11 == 0, _ % 123 == 0, _ % 901 == 0,
- _ > 0, _ >= 0, _ < 0, _ <= 0,
- _ > 50, _ > 500, _ > 5000,
- _ < -10, _ < -100, _ < -1000,
- n => n > 50 && n < 100,
- n => n * n > 1000000 && n % 111 == 0
+ _ % 2 == 0, _ % 11 == 0, _ % 123 == 0, _ % 901 == 0,
+ _ > 0, _ >= 0, _ < 0, _ <= 0,
+ _ > 50, _ > 500, _ > 5000,
+ _ < -10, _ < -100, _ < -1000,
+ n => n > 50 && n < 100,
+ n => n * n > 1000000 && n % 111 == 0
)
def lastIndexWherePredicates = List(
- _ % 2 == 0, _ % 17 == 0, _ % 314 == 0, _ % 1017 == 0,
- _ > 0, _ >= 0, _ < 0, _ <= 0,
- _ > 50, _ > 500, _ > 5000,
- _ < -20, _ < -200, _ < -2000,
- _ == 0,
- n => n > -40 && n < 40,
- n => n > -80 && n < -10,
- n => n > 110 && n < 150
+ _ % 2 == 0, _ % 17 == 0, _ % 314 == 0, _ % 1017 == 0,
+ _ > 0, _ >= 0, _ < 0, _ <= 0,
+ _ > 50, _ > 500, _ > 5000,
+ _ < -20, _ < -200, _ < -2000,
+ _ == 0,
+ n => n > -40 && n < 40,
+ n => n > -80 && n < -10,
+ n => n > 110 && n < 150
)
def reverseMapFunctions = List(-_, n => n * n, _ + 1)
def sameElementsSeqs = List(
- List[Int](),
- List(1),
- List(1, 2, 3, 4, 5, 6, 7, 8, 9),
- Array.fill(150)(1).toSeq,
- Array.fill(1000)(1).toSeq
+ List[Int](),
+ List(1),
+ List(1, 2, 3, 4, 5, 6, 7, 8, 9),
+ Array.fill(150)(1).toSeq,
+ Array.fill(1000)(1).toSeq
)
def startEndSeqs = List(
- Nil,
- List(1),
- List(1, 2, 3, 4, 5),
- List(0, 1, 2, 3, 4, 5),
- List(4, 5, 6, 7, 8, 9, 10),
- List(4, 5, 6, 7, 8, 9, 0),
- List(-4, -3, -2, -1)
+ Nil,
+ List(1),
+ List(1, 2, 3, 4, 5),
+ List(0, 1, 2, 3, 4, 5),
+ List(4, 5, 6, 7, 8, 9, 10),
+ List(4, 5, 6, 7, 8, 9, 0),
+ List(-4, -3, -2, -1)
)
}
diff --git a/test/files/scalacheck/parallel-collections/ParallelHashMapCheck.scala b/test/files/scalacheck/parallel-collections/ParallelHashMapCheck.scala
index 5278d28f0f..d53c0ba9d6 100644
--- a/test/files/scalacheck/parallel-collections/ParallelHashMapCheck.scala
+++ b/test/files/scalacheck/parallel-collections/ParallelHashMapCheck.scala
@@ -58,10 +58,32 @@ with PairValues[Int, Int]
override def printDataStructureDebugInfo(ds: AnyRef) = ds match {
case pm: ParHashMap[k, v] =>
- println("Mutable parallel hash map")
+ println("Mutable parallel hash map\n" + pm.hashTableContents.debugInformation)
case _ =>
println("could not match data structure type: " + ds.getClass)
}
+
+ override def checkDataStructureInvariants(orig: Traversable[(Int, Int)], ds: AnyRef) = ds match {
+ case pm: ParHashMap[k, v] =>
+ val invs = pm.brokenInvariants
+
+ val containsall = (for ((k, v) <- orig) yield {
+ if (pm.asInstanceOf[ParHashMap[Int, Int]].get(k) == Some(v)) true
+ else {
+ println("Does not contain original element: " + (k, v))
+ false
+ }
+ }).foldLeft(true)(_ && _)
+
+
+ if (invs.isEmpty) containsall
+ else {
+ println("Invariants broken:\n" + invs.mkString("\n"))
+ false
+ }
+ case _ => true
+ }
+
}
diff --git a/test/files/scalacheck/parallel-collections/ParallelIterableCheck.scala b/test/files/scalacheck/parallel-collections/ParallelIterableCheck.scala
index 52edf9e641..0acdb2b0a7 100644
--- a/test/files/scalacheck/parallel-collections/ParallelIterableCheck.scala
+++ b/test/files/scalacheck/parallel-collections/ParallelIterableCheck.scala
@@ -29,9 +29,17 @@ abstract class ParallelIterableCheck[T](collName: String) extends Properties(col
sz =>
ofSize(vals, sz)
),
- for (sz <- choose(1000, 2000)) yield ofSize(vals, sz)
+ for (sz <- choose(1000, 2000)) yield ofSize(vals, sz),
+ for (sz <- choose(4000, 4001)) yield ofSize(vals, sz),
+ for (sz <- choose(10000, 10001)) yield ofSize(vals, sz)
)
+ // used to check if constructed collection is valid
+ def checkDataStructureInvariants(orig: Traversable[T], cf: AnyRef) = {
+ // can be overriden in subclasses
+ true
+ }
+
def printDataStructureDebugInfo(cf: AnyRef) {
// can be overridden in subclasses
}
@@ -63,6 +71,17 @@ abstract class ParallelIterableCheck[T](collName: String) extends Properties(col
(inst, fromTraversable(inst), modif)
}
+ def areEqual(t1: Traversable[T], t2: Traversable[T]) = if (hasStrictOrder) {
+ t1 == t2 && t2 == t1
+ } else (t1, t2) match { // it is slightly delicate what `equal` means if the order is not strict
+ case (m1: Map[_, _], m2: Map[_, _]) => m1 == m2 && m2 == m1
+ case (i1: Iterable[_], i2: Iterable[_]) =>
+ val i1s = i1.toSet
+ val i2s = i2.toSet
+ i1s == i2s && i2s == i1s
+ case _ => t1 == t2 && t2 == t1
+ }
+
property("reductions must be equal for assoc. operators") = forAll(collectionPairs) { case (t, coll) =>
if (t.size != 0) {
val results = for ((op, ind) <- reduceOperators.zipWithIndex) yield {
@@ -81,263 +100,257 @@ abstract class ParallelIterableCheck[T](collName: String) extends Properties(col
} else "has size 0" |: true
}
- // property("counts must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // val results = for ((pred, ind) <- countPredicates.zipWithIndex) yield {
- // val tc = t.count(pred)
- // val cc = coll.count(pred)
- // if (tc != cc) {
- // println("from: " + t)
- // println("and: " + coll.toList)
- // println(tc)
- // println(cc)
- // }
- // ("op index: " + ind) |: tc == cc
- // }
- // results.reduceLeft(_ && _)
- // }
-
- // property("forall must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // val results = for ((pred, ind) <- forallPredicates.zipWithIndex)
- // yield ("op index: " + ind) |: t.forall(pred) == coll.forall(pred)
- // results.reduceLeft(_ && _)
- // }
-
- // property("exists must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // val results = for ((pred, ind) <- existsPredicates.zipWithIndex)
- // yield ("op index: " + ind) |: t.exists(pred) == coll.exists(pred)
- // results.reduceLeft(_ && _)
- // }
-
- // property("both must find or not find an element") = forAll(collectionPairs) { case (t, coll) =>
- // val results = for ((pred, ind) <- findPredicates.zipWithIndex) yield {
- // val ft = t.find(pred)
- // val fcoll = coll.find(pred)
- // ("op index: " + ind) |: ((ft == None && fcoll == None) || (ft != None && fcoll != None))
- // }
- // results.reduceLeft(_ && _)
- // }
-
- // def areEqual(t1: Traversable[T], t2: Traversable[T]) = if (hasStrictOrder) {
- // t1 == t2
- // } else (t1, t2) match { // it is slightly delicate what `equal` means if the order is not strict
- // case (m1: Map[_, _], m2: Map[_, _]) => m1 == m2
- // case (i1: Iterable[_], i2: Iterable[_]) => i1.toSet == i2.toSet
- // case _ => t1 == t2
- // }
-
- // property("mappings must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // val results = for ((f, ind) <- mapFunctions.zipWithIndex) yield {
- // val ms = t.map(f)
- // val mp = coll.map(f)
- // if (!areEqual(ms, mp)) {
- // println(t)
- // println(coll)
- // println("mapped to: ")
- // println(ms)
- // println(mp)
- // }
- // ("op index: " + ind) |: areEqual(ms, mp)
- // }
- // results.reduceLeft(_ && _)
- // }
-
- // property("collects must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // val results = for ((f, ind) <- partialMapFunctions.zipWithIndex) yield {
- // val ps = t.collect(f)
- // val pp = coll.collect(f)
- // if (!areEqual(ps, pp)) {
- // println(t)
- // println(coll)
- // println("collected to: ")
- // println(ps)
- // println(pp)
- // }
- // ("op index: " + ind) |: areEqual(ps, pp)
- // }
- // results.reduceLeft(_ && _)
- // }
-
- // property("flatMaps must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // (for ((f, ind) <- flatMapFunctions.zipWithIndex)
- // yield ("op index: " + ind) |: areEqual(t.flatMap(f), coll.flatMap(f))).reduceLeft(_ && _)
- // }
-
- // property("filters must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // (for ((p, ind) <- filterPredicates.zipWithIndex) yield {
- // val tf = t.filter(p)
- // val cf = coll.filter(p)
- // if (tf != cf || cf != tf) {
- // println(t)
- // println(coll)
- // println("filtered to:")
- // println(cf)
- // println(tf)
- // println("tf == cf - " + (tf == cf))
- // println("cf == tf - " + (cf == tf))
- // printDataStructureDebugInfo(cf)
- // }
- // ("op index: " + ind) |: tf == cf && cf == tf
- // }).reduceLeft(_ && _)
- // }
-
- // property("filterNots must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // (for ((p, ind) <- filterNotPredicates.zipWithIndex)
- // yield ("op index: " + ind) |: t.filterNot(p) == coll.filterNot(p)).reduceLeft(_ && _)
- // }
-
- // if (!isCheckingViews) property("partitions must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // (for ((p, ind) <- partitionPredicates.zipWithIndex) yield {
- // val tpart = t.partition(p)
- // val cpart = coll.partition(p)
- // if (tpart != cpart) {
- // println("from: " + t)
- // println("and: " + coll)
- // println(cpart)
- // println(tpart)
- // }
- // ("op index: " + ind) |: tpart == cpart
- // }).reduceLeft(_ && _)
- // }
-
- // if (hasStrictOrder) property("takes must be equal") = forAll(collectionPairsWithLengths) { case (t, coll, n) =>
- // ("take " + n + " elements") |: t.take(n) == coll.take(n)
- // }
-
- // if (hasStrictOrder) property("drops must be equal") = forAll(collectionPairsWithLengths) { case (t, coll, n) =>
- // ("drop " + n + " elements") |: t.drop(n) == coll.drop(n)
- // }
-
- // if (hasStrictOrder) property("slices must be equal") = forAll(collectionPairsWith2Indices)
- // { case (t, coll, fr, slicelength) =>
- // val from = if (fr < 0) 0 else fr
- // val until = if (from + slicelength > t.size) t.size else from + slicelength
- // val tsl = t.slice(from, until)
- // val collsl = coll.slice(from, until)
- // if (tsl != collsl) {
- // println("---------------------- " + from + ", " + until)
- // println("from: " + t)
- // println("and: " + coll)
- // println(tsl)
- // println(collsl)
- // println("as list: " + collsl.toList)
- // println(collsl.iterator.hasNext)
- // println(collsl.iterator.next)
- // println(collsl.iterator.hasNext)
- // println(collsl.iterator.next)
- // println(collsl.iterator.hasNext)
- // println(collsl.iterator.next)
- // println(collsl.iterator.hasNext)
- // }
- // ("slice from " + from + " until " + until) |: tsl == collsl
- // }
-
- // if (hasStrictOrder) property("splits must be equal") = forAll(collectionPairsWithLengths) { case (t, coll, n) =>
- // val tspl = t.splitAt(n)
- // val cspl = coll.splitAt(n)
- // if (tspl != cspl) {
- // println("at: " + n)
- // println("from: " + t)
- // println("and: " + coll)
- // println(tspl)
- // println(cspl)
- // }
- // ("splitAt " + n) |: tspl == cspl
- // }
-
- // if (hasStrictOrder) property("takeWhiles must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // (for ((pred, ind) <- takeWhilePredicates.zipWithIndex) yield {
- // val tt = t.takeWhile(pred)
- // val ct = coll.takeWhile(pred)
- // if (tt != ct) {
- // println("from: " + t)
- // println("and: " + coll)
- // println("taking while...")
- // println(tt)
- // println(ct)
- // }
- // ("operator " + ind) |: tt == ct
- // }).reduceLeft(_ && _)
- // }
-
- // if (hasStrictOrder) property("spans must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // (for ((pred, ind) <- spanPredicates.zipWithIndex) yield {
- // val tsp = t.span(pred)
- // val csp = coll.span(pred)
- // if (tsp != csp) {
- // println("from: " + t)
- // println("and: " + coll)
- // println("span with predicate " + ind)
- // println(tsp)
- // println(csp)
- // println("---------------------------------")
- // println(coll.span(pred))
- // println("---------------------------------")
- // }
- // ("operator " + ind) |: tsp == csp
- // }).reduceLeft(_ && _)
- // }
-
- // if (hasStrictOrder) property("dropWhiles must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // (for ((pred, ind) <- dropWhilePredicates.zipWithIndex) yield {
- // ("operator " + ind) |: t.dropWhile(pred) == coll.dropWhile(pred)
- // }).reduceLeft(_ && _)
- // }
-
- // property("folds must be equal for assoc. operators") = forAll(collectionPairs) { case (t, coll) =>
- // (for (((first, op), ind) <- foldArguments.zipWithIndex) yield {
- // val tres = t.foldLeft(first)(op)
- // val cres = coll.fold(first)(op)
- // if (cres != tres) {
- // println("from: " + t)
- // println("and: " + coll)
- // println("folds are: ")
- // println(tres)
- // println(cres)
- // }
- // ("operator " + ind) |: tres == cres
- // }).reduceLeft(_ && _)
- // }
-
- // property("++s must be equal") = forAll(collectionTriplets) { case (t, coll, colltoadd) =>
- // val toadd = colltoadd
- // val tr = t ++ toadd.iterator
- // val cr = coll ++ toadd.iterator
- // if (!areEqual(tr, cr)) {
- // println("from: " + t)
- // println("and: " + coll.iterator.toList)
- // println("adding: " + toadd)
- // println(tr.toList)
- // println(cr.iterator.toList)
- // }
- // ("adding " |: areEqual(tr, cr)) &&
- // (for ((trav, ind) <- (addAllTraversables).zipWithIndex) yield {
- // val tadded = t ++ trav
- // val cadded = coll ++ collection.parallel.mutable.ParArray(trav.toSeq: _*)
- // if (!areEqual(tadded, cadded)) {
- // println("----------------------")
- // println("from: " + t)
- // println("and: " + coll)
- // println("adding: " + trav)
- // println(tadded)
- // println(cadded)
- // }
- // ("traversable " + ind) |: areEqual(tadded, cadded)
- // }).reduceLeft(_ && _)
- // }
-
- // if (hasStrictOrder) property("copies to array must be equal") = forAll(collectionPairs) { case (t, coll) =>
- // val tarr = newArray(t.size)
- // val collarr = newArray(coll.size)
- // t.copyToArray(tarr, 0, t.size)
- // coll.copyToArray(collarr, 0, coll.size)
- // if (tarr.toSeq != collarr.toSeq) {
- // println("from: " + t)
- // println("and: " + coll)
- // println(tarr.toSeq)
- // println(collarr.toSeq)
- // }
- // tarr.toSeq == collarr.toSeq
- // }
+ property("counts must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ val results = for ((pred, ind) <- countPredicates.zipWithIndex) yield {
+ val tc = t.count(pred)
+ val cc = coll.count(pred)
+ if (tc != cc) {
+ println("from: " + t)
+ println("and: " + coll.toList)
+ println(tc)
+ println(cc)
+ }
+ ("op index: " + ind) |: tc == cc
+ }
+ results.reduceLeft(_ && _)
+ }
+
+ property("forall must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ val results = for ((pred, ind) <- forallPredicates.zipWithIndex)
+ yield ("op index: " + ind) |: t.forall(pred) == coll.forall(pred)
+ results.reduceLeft(_ && _)
+ }
+
+ property("exists must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ val results = for ((pred, ind) <- existsPredicates.zipWithIndex)
+ yield ("op index: " + ind) |: t.exists(pred) == coll.exists(pred)
+ results.reduceLeft(_ && _)
+ }
+
+ property("both must find or not find an element") = forAll(collectionPairs) { case (t, coll) =>
+ val results = for ((pred, ind) <- findPredicates.zipWithIndex) yield {
+ val ft = t.find(pred)
+ val fcoll = coll.find(pred)
+ ("op index: " + ind) |: ((ft == None && fcoll == None) || (ft != None && fcoll != None))
+ }
+ results.reduceLeft(_ && _)
+ }
+
+ property("mappings must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ val results = for ((f, ind) <- mapFunctions.zipWithIndex) yield {
+ val ms = t.map(f)
+ val mp = coll.map(f)
+ if (!areEqual(ms, mp) || !checkDataStructureInvariants(ms, mp)) {
+ println(t)
+ println(coll)
+ println("mapped to: ")
+ println(ms)
+ println(mp)
+ println("valid: " + !checkDataStructureInvariants(ms, mp))
+ }
+ ("op index: " + ind) |: (areEqual(ms, mp) && checkDataStructureInvariants(ms, mp))
+ }
+ results.reduceLeft(_ && _)
+ }
+
+ property("collects must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ val results = for ((f, ind) <- partialMapFunctions.zipWithIndex) yield {
+ val ps = t.collect(f)
+ val pp = coll.collect(f)
+ if (!areEqual(ps, pp)) {
+ println(t)
+ println(coll)
+ println("collected to: ")
+ println(ps)
+ println(pp)
+ }
+ ("op index: " + ind) |: areEqual(ps, pp)
+ }
+ results.reduceLeft(_ && _)
+ }
+
+ property("flatMaps must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ (for ((f, ind) <- flatMapFunctions.zipWithIndex)
+ yield ("op index: " + ind) |: areEqual(t.flatMap(f), coll.flatMap(f))).reduceLeft(_ && _)
+ }
+
+ property("filters must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ (for ((p, ind) <- filterPredicates.zipWithIndex) yield {
+ val tf = t.filter(p)
+ val cf = coll.filter(p)
+ if (tf != cf || cf != tf || !checkDataStructureInvariants(tf, cf)) {
+ println(t)
+ println(coll)
+ println("filtered to:")
+ println(cf)
+ println(tf)
+ println("tf == cf - " + (tf == cf))
+ println("cf == tf - " + (cf == tf))
+ printDataStructureDebugInfo(cf)
+ println("valid: " + checkDataStructureInvariants(tf, cf))
+ }
+ ("op index: " + ind) |: tf == cf && cf == tf && checkDataStructureInvariants(tf, cf)
+ }).reduceLeft(_ && _)
+ }
+
+ property("filterNots must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ (for ((p, ind) <- filterNotPredicates.zipWithIndex)
+ yield ("op index: " + ind) |: t.filterNot(p) == coll.filterNot(p)).reduceLeft(_ && _)
+ }
+
+ if (!isCheckingViews) property("partitions must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ (for ((p, ind) <- partitionPredicates.zipWithIndex) yield {
+ val tpart = t.partition(p)
+ val cpart = coll.partition(p)
+ if (tpart != cpart) {
+ println("from: " + t)
+ println("and: " + coll)
+ println(cpart)
+ println(tpart)
+ }
+ ("op index: " + ind) |: tpart == cpart
+ }).reduceLeft(_ && _)
+ }
+
+ if (hasStrictOrder) property("takes must be equal") = forAll(collectionPairsWithLengths) { case (t, coll, n) =>
+ ("take " + n + " elements") |: t.take(n) == coll.take(n)
+ }
+
+ if (hasStrictOrder) property("drops must be equal") = forAll(collectionPairsWithLengths) { case (t, coll, n) =>
+ ("drop " + n + " elements") |: t.drop(n) == coll.drop(n)
+ }
+
+ if (hasStrictOrder) property("slices must be equal") = forAll(collectionPairsWith2Indices)
+ { case (t, coll, fr, slicelength) =>
+ val from = if (fr < 0) 0 else fr
+ val until = if (from + slicelength > t.size) t.size else from + slicelength
+ val tsl = t.slice(from, until)
+ val collsl = coll.slice(from, until)
+ if (tsl != collsl) {
+ println("---------------------- " + from + ", " + until)
+ println("from: " + t)
+ println("and: " + coll)
+ println(tsl)
+ println(collsl)
+ println("as list: " + collsl.toList)
+ println(collsl.iterator.hasNext)
+ println(collsl.iterator.next)
+ println(collsl.iterator.hasNext)
+ println(collsl.iterator.next)
+ println(collsl.iterator.hasNext)
+ println(collsl.iterator.next)
+ println(collsl.iterator.hasNext)
+ }
+ ("slice from " + from + " until " + until) |: tsl == collsl
+ }
+
+ if (hasStrictOrder) property("splits must be equal") = forAll(collectionPairsWithLengths) { case (t, coll, n) =>
+ val tspl = t.splitAt(n)
+ val cspl = coll.splitAt(n)
+ if (tspl != cspl) {
+ println("at: " + n)
+ println("from: " + t)
+ println("and: " + coll)
+ println(tspl)
+ println(cspl)
+ }
+ ("splitAt " + n) |: tspl == cspl
+ }
+
+ if (hasStrictOrder) property("takeWhiles must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ (for ((pred, ind) <- takeWhilePredicates.zipWithIndex) yield {
+ val tt = t.takeWhile(pred)
+ val ct = coll.takeWhile(pred)
+ if (tt != ct) {
+ println("from: " + t)
+ println("and: " + coll)
+ println("taking while...")
+ println(tt)
+ println(ct)
+ }
+ ("operator " + ind) |: tt == ct
+ }).reduceLeft(_ && _)
+ }
+
+ if (hasStrictOrder) property("spans must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ (for ((pred, ind) <- spanPredicates.zipWithIndex) yield {
+ val tsp = t.span(pred)
+ val csp = coll.span(pred)
+ if (tsp != csp) {
+ println("from: " + t)
+ println("and: " + coll)
+ println("span with predicate " + ind)
+ println(tsp)
+ println(csp)
+ println("---------------------------------")
+ println(coll.span(pred))
+ println("---------------------------------")
+ }
+ ("operator " + ind) |: tsp == csp
+ }).reduceLeft(_ && _)
+ }
+
+ if (hasStrictOrder) property("dropWhiles must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ (for ((pred, ind) <- dropWhilePredicates.zipWithIndex) yield {
+ ("operator " + ind) |: t.dropWhile(pred) == coll.dropWhile(pred)
+ }).reduceLeft(_ && _)
+ }
+
+ property("folds must be equal for assoc. operators") = forAll(collectionPairs) { case (t, coll) =>
+ (for (((first, op), ind) <- foldArguments.zipWithIndex) yield {
+ val tres = t.foldLeft(first)(op)
+ val cres = coll.fold(first)(op)
+ if (cres != tres) {
+ println("from: " + t)
+ println("and: " + coll)
+ println("folds are: ")
+ println(tres)
+ println(cres)
+ }
+ ("operator " + ind) |: tres == cres
+ }).reduceLeft(_ && _)
+ }
+
+ property("++s must be equal") = forAll(collectionTriplets) { case (t, coll, colltoadd) =>
+ val toadd = colltoadd
+ val tr = t ++ toadd.iterator
+ val cr = coll ++ toadd.iterator
+ if (!areEqual(tr, cr)) {
+ println("from: " + t)
+ println("and: " + coll.iterator.toList)
+ println("adding: " + toadd)
+ println(tr.toList)
+ println(cr.iterator.toList)
+ }
+ ("adding " |: areEqual(tr, cr)) &&
+ (for ((trav, ind) <- (addAllTraversables).zipWithIndex) yield {
+ val tadded = t ++ trav
+ val cadded = coll ++ collection.parallel.mutable.ParArray(trav.toSeq: _*)
+ if (!areEqual(tadded, cadded)) {
+ println("----------------------")
+ println("from: " + t)
+ println("and: " + coll)
+ println("adding: " + trav)
+ println(tadded)
+ println(cadded)
+ }
+ ("traversable " + ind) |: areEqual(tadded, cadded)
+ }).reduceLeft(_ && _)
+ }
+
+ if (hasStrictOrder) property("copies to array must be equal") = forAll(collectionPairs) { case (t, coll) =>
+ val tarr = newArray(t.size)
+ val collarr = newArray(coll.size)
+ t.copyToArray(tarr, 0, t.size)
+ coll.copyToArray(collarr, 0, coll.size)
+ if (tarr.toSeq != collarr.toSeq) {
+ println("from: " + t)
+ println("and: " + coll)
+ println(tarr.toSeq)
+ println(collarr.toSeq)
+ }
+ tarr.toSeq == collarr.toSeq
+ }
}
diff --git a/test/files/scalacheck/parallel-collections/ParallelMapCheck1.scala b/test/files/scalacheck/parallel-collections/ParallelMapCheck1.scala
index c4f241c02e..6b30f61b57 100644
--- a/test/files/scalacheck/parallel-collections/ParallelMapCheck1.scala
+++ b/test/files/scalacheck/parallel-collections/ParallelMapCheck1.scala
@@ -17,13 +17,13 @@ import scala.collection.parallel._
abstract class ParallelMapCheck[K, V](collname: String) extends ParallelIterableCheck[(K, V)](collname) {
type CollType <: ParMap[K, V] with Sequentializable[(K, V), Map[K, V]]
- // property("gets iterated keys") = forAll(collectionPairs) {
- // case (t, coll) =>
- // val containsT = for ((k, v) <- t) yield (coll.get(k) == Some(v))
- // val containsSelf = for ((k, v) <- coll) yield (coll.get(k) == Some(v))
- // ("Par contains elements of seq map" |: containsT.forall(_ == true)) &&
- // ("Par contains elements of itself" |: containsSelf.forall(_ == true))
- // }
+ property("gets iterated keys") = forAll(collectionPairs) {
+ case (t, coll) =>
+ val containsT = for ((k, v) <- t) yield (coll.get(k) == Some(v))
+ val containsSelf = for ((k, v) <- coll) yield (coll.get(k) == Some(v))
+ ("Par contains elements of seq map" |: containsT.forall(_ == true)) &&
+ ("Par contains elements of itself" |: containsSelf.forall(_ == true))
+ }
}
diff --git a/test/files/scalacheck/parallel-collections/pc.scala b/test/files/scalacheck/parallel-collections/pc.scala
index fe406a9408..aba8efef6a 100644
--- a/test/files/scalacheck/parallel-collections/pc.scala
+++ b/test/files/scalacheck/parallel-collections/pc.scala
@@ -11,16 +11,16 @@ class ParCollProperties extends Properties("Parallel collections") {
/* Collections */
// parallel arrays
- //include(mutable.IntParallelArrayCheck)
+ include(mutable.IntParallelArrayCheck)
// parallel ranges
- //include(immutable.ParallelRangeCheck)
+ include(immutable.ParallelRangeCheck)
// parallel immutable hash maps (tries)
- //include(immutable.IntIntParallelHashMapCheck)
+ include(immutable.IntIntParallelHashMapCheck)
// parallel immutable hash sets (tries)
- //include(immutable.IntParallelHashSetCheck)
+ include(immutable.IntParallelHashSetCheck)
// parallel mutable hash maps (tables)
include(mutable.IntIntParallelHashMapCheck)
@@ -44,8 +44,8 @@ object Test {
testCallback = new ConsoleReporter(0),
workers = 1,
minSize = 0,
- maxSize = 250,
- minSuccessfulTests = 250
+ maxSize = 4000,
+ minSuccessfulTests = 100
),
pc
)