You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by rx...@apache.org on 2013/11/18 03:42:29 UTC

[3/5] git commit: Add PrimitiveVectorSuite and fix bug in resize()

Add PrimitiveVectorSuite and fix bug in resize()


Project: http://git-wip-us.apache.org/repos/asf/incubator-spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-spark/commit/85763f49
Tree: http://git-wip-us.apache.org/repos/asf/incubator-spark/tree/85763f49
Diff: http://git-wip-us.apache.org/repos/asf/incubator-spark/diff/85763f49

Branch: refs/heads/master
Commit: 85763f4942afc095595dc32c853d077bdbf49644
Parents: 16a2286
Author: Aaron Davidson <aa...@databricks.com>
Authored: Sun Nov 17 17:59:18 2013 -0800
Committer: Aaron Davidson <aa...@databricks.com>
Committed: Sun Nov 17 18:16:51 2013 -0800

----------------------------------------------------------------------
 .../spark/util/collection/PrimitiveVector.scala |   3 +
 .../util/collection/PrimitiveVectorSuite.scala  | 117 +++++++++++++++++++
 2 files changed, 120 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/85763f49/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
index b4fcc92..20554f0 100644
--- a/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
+++ b/core/src/main/scala/org/apache/spark/util/collection/PrimitiveVector.scala
@@ -59,6 +59,9 @@ class PrimitiveVector[@specialized(Long, Int, Double) V: ClassManifest](initialS
     val newArray = new Array[V](newLength)
     _array.copyToArray(newArray)
     _array = newArray
+    if (newLength < _numElements) {
+      _numElements = newLength
+    }
     this
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-spark/blob/85763f49/core/src/test/scala/org/apache/spark/util/collection/PrimitiveVectorSuite.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/org/apache/spark/util/collection/PrimitiveVectorSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveVectorSuite.scala
new file mode 100644
index 0000000..970dade
--- /dev/null
+++ b/core/src/test/scala/org/apache/spark/util/collection/PrimitiveVectorSuite.scala
@@ -0,0 +1,117 @@
+/*
+ * 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.util.collection
+
+import org.scalatest.FunSuite
+
+import org.apache.spark.util.SizeEstimator
+
+class PrimitiveVectorSuite extends FunSuite {
+
+  test("primitive value") {
+    val vector = new PrimitiveVector[Int]
+
+    for (i <- 0 until 1000) {
+      vector += i
+      assert(vector(i) === i)
+    }
+
+    assert(vector.size === 1000)
+    assert(vector.size == vector.length)
+    intercept[IllegalArgumentException] {
+      vector(1000)
+    }
+
+    for (i <- 0 until 1000) {
+      assert(vector(i) == i)
+    }
+  }
+
+  test("non-primitive value") {
+    val vector = new PrimitiveVector[String]
+
+    for (i <- 0 until 1000) {
+      vector += i.toString
+      assert(vector(i) === i.toString)
+    }
+
+    assert(vector.size === 1000)
+    assert(vector.size == vector.length)
+    intercept[IllegalArgumentException] {
+      vector(1000)
+    }
+
+    for (i <- 0 until 1000) {
+      assert(vector(i) == i.toString)
+    }
+  }
+
+  test("ideal growth") {
+    val vector = new PrimitiveVector[Long](initialSize = 1)
+    vector += 1
+    for (i <- 1 until 1024) {
+      vector += i
+      assert(vector.size === i + 1)
+      assert(vector.capacity === Integer.highestOneBit(i) * 2)
+    }
+    assert(vector.capacity === 1024)
+    vector += 1024
+    assert(vector.capacity === 2048)
+  }
+
+  test("ideal size") {
+    val vector = new PrimitiveVector[Long](8192)
+    for (i <- 0 until 8192) {
+      vector += i
+    }
+    assert(vector.size === 8192)
+    assert(vector.capacity === 8192)
+    val actualSize = SizeEstimator.estimate(vector)
+    val expectedSize = 8192 * 8
+    // Make sure we are not allocating a significant amount of memory beyond our expected.
+    // Due to specialization wonkiness, we need to ensure we don't have 2 copies of the array.
+    assert(actualSize < expectedSize * 1.1)
+  }
+
+  test("resizing") {
+    val vector = new PrimitiveVector[Long]
+    for (i <- 0 until 4097) {
+      vector += i
+    }
+    assert(vector.size === 4097)
+    assert(vector.capacity === 8192)
+    vector.trim()
+    assert(vector.size === 4097)
+    assert(vector.capacity === 4097)
+    vector.resize(5000)
+    assert(vector.size === 4097)
+    assert(vector.capacity === 5000)
+    vector.resize(4000)
+    assert(vector.size === 4000)
+    assert(vector.capacity === 4000)
+    vector.resize(5000)
+    assert(vector.size === 4000)
+    assert(vector.capacity === 5000)
+    for (i <- 0 until 4000) {
+      assert(vector(i) == i)
+    }
+    intercept[IllegalArgumentException] {
+      vector(4000)
+    }
+  }
+}