Return-Path: X-Original-To: apmail-spark-commits-archive@minotaur.apache.org Delivered-To: apmail-spark-commits-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id E3EEC182C5 for ; Mon, 3 Aug 2015 00:54:33 +0000 (UTC) Received: (qmail 92931 invoked by uid 500); 3 Aug 2015 00:54:33 -0000 Delivered-To: apmail-spark-commits-archive@spark.apache.org Received: (qmail 92897 invoked by uid 500); 3 Aug 2015 00:54:33 -0000 Mailing-List: contact commits-help@spark.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Delivered-To: mailing list commits@spark.apache.org Received: (qmail 92888 invoked by uid 99); 3 Aug 2015 00:54:33 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 03 Aug 2015 00:54:33 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 7E54BDFCB4; Mon, 3 Aug 2015 00:54:33 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: rxin@apache.org To: commits@spark.apache.org Message-Id: <5f4a53bbca164845b9d2975d2ed1b6f6@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: spark git commit: [SPARK-9543][SQL] Add randomized testing for UnsafeKVExternalSorter. Date: Mon, 3 Aug 2015 00:54:33 +0000 (UTC) Repository: spark Updated Branches: refs/heads/master 0722f4331 -> 9d03ad910 [SPARK-9543][SQL] Add randomized testing for UnsafeKVExternalSorter. The detailed approach is documented in UnsafeKVExternalSorterSuite.testKVSorter(), working as follows: 1. Create input by generating data randomly based on the given key/value schema (which is also randomly drawn from a list of candidate types) 2. Run UnsafeKVExternalSorter on the generated data 3. Collect the output from the sorter, and make sure the keys are sorted in ascending order 4. Sort the input by both key and value, and sort the sorter output also by both key and value. Compare the sorted input and sorted output together to make sure all the key/values match. 5. Check memory allocation to make sure there is no memory leak. There is also a spill flag. When set to true, the sorter will spill probabilistically roughly every 100 records. Author: Reynold Xin Closes #7873 from rxin/kvsorter-randomized-test and squashes the following commits: a08c251 [Reynold Xin] Resource cleanup. 0488b5c [Reynold Xin] [SPARK-9543][SQL] Add randomized testing for UnsafeKVExternalSorter. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9d03ad91 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9d03ad91 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9d03ad91 Branch: refs/heads/master Commit: 9d03ad910b853c1e0dcb946b05d48ab36a1ac533 Parents: 0722f43 Author: Reynold Xin Authored: Sun Aug 2 17:54:30 2015 -0700 Committer: Reynold Xin Committed: Sun Aug 2 17:54:30 2015 -0700 ---------------------------------------------------------------------- .../apache/spark/sql/RandomDataGenerator.scala | 6 + .../UnsafeFixedWidthAggregationMapSuite.scala | 18 +- .../execution/UnsafeKVExternalSorterSuite.scala | 222 +++++++++---------- .../unsafe/types/CalendarIntervalSuite.java | 149 +++++++++++++ .../spark/unsafe/types/IntervalSuite.java | 149 ------------- 5 files changed, 274 insertions(+), 270 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/spark/blob/9d03ad91/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala ---------------------------------------------------------------------- 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 ea1fd23..11e0c12 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 @@ -24,6 +24,7 @@ import java.math.MathContext import scala.util.Random import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval /** * Random data generators for Spark SQL DataTypes. These generators do not generate uniformly random @@ -106,6 +107,11 @@ object RandomDataGenerator { case BooleanType => Some(() => rand.nextBoolean()) case DateType => Some(() => new java.sql.Date(rand.nextInt())) case TimestampType => Some(() => new java.sql.Timestamp(rand.nextLong())) + case CalendarIntervalType => Some(() => { + val months = rand.nextInt(1000) + val ns = rand.nextLong() + new CalendarInterval(months, ns) + }) case DecimalType.Fixed(precision, scale) => Some( () => BigDecimal.apply( rand.nextLong() % math.pow(10, precision).toLong, http://git-wip-us.apache.org/repos/asf/spark/blob/9d03ad91/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala index 4c94b33..7c591f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMapSuite.scala @@ -56,11 +56,21 @@ class UnsafeFixedWidthAggregationMapSuite extends SparkFunSuite with Matchers { assert(leakedShuffleMemory === 0) taskMemoryManager = null } + TaskContext.unset() } test(name) { taskMemoryManager = new TaskMemoryManager(new ExecutorMemoryManager(MemoryAllocator.HEAP)) shuffleMemoryManager = new TestShuffleMemoryManager + + TaskContext.setTaskContext(new TaskContextImpl( + stageId = 0, + partitionId = 0, + taskAttemptId = Random.nextInt(10000), + attemptNumber = 0, + taskMemoryManager = taskMemoryManager, + metricsSystem = null)) + try { f } catch { @@ -163,14 +173,6 @@ class UnsafeFixedWidthAggregationMapSuite extends SparkFunSuite with Matchers { // Calling this make sure we have block manager and everything else setup. TestSQLContext - TaskContext.setTaskContext(new TaskContextImpl( - stageId = 0, - partitionId = 0, - taskAttemptId = 0, - attemptNumber = 0, - taskMemoryManager = taskMemoryManager, - metricsSystem = null)) - // Memory consumption in the beginning of the task. val initialMemoryConsumption = shuffleMemoryManager.getMemoryConsumptionForThisTask() http://git-wip-us.apache.org/repos/asf/spark/blob/9d03ad91/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala ---------------------------------------------------------------------- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala index 5d214d7..0282b25 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeKVExternalSorterSuite.scala @@ -19,140 +19,136 @@ package org.apache.spark.sql.execution import scala.util.Random -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{RowOrdering, UnsafeProjection} +import org.apache.spark._ +import org.apache.spark.sql.RandomDataGenerator +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.expressions.{UnsafeRow, RowOrdering, UnsafeProjection} import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.types._ import org.apache.spark.unsafe.memory.{ExecutorMemoryManager, MemoryAllocator, TaskMemoryManager} -import org.apache.spark.unsafe.types.UTF8String -import org.apache.spark._ +/** + * Test suite for [[UnsafeKVExternalSorter]], with randomly generated test data. + */ class UnsafeKVExternalSorterSuite extends SparkFunSuite { - test("sorting string key and int int value") { - - // Calling this make sure we have block manager and everything else setup. - TestSQLContext + private val keyTypes = Seq(IntegerType, FloatType, DoubleType, StringType) + private val valueTypes = Seq(IntegerType, FloatType, DoubleType, StringType) - val taskMemMgr = new TaskMemoryManager(new ExecutorMemoryManager(MemoryAllocator.HEAP)) - val shuffleMemMgr = new TestShuffleMemoryManager + testKVSorter(new StructType, new StructType, spill = true) + testKVSorter(new StructType().add("c1", IntegerType), new StructType, spill = true) + testKVSorter(new StructType, new StructType().add("c1", IntegerType), spill = true) - TaskContext.setTaskContext(new TaskContextImpl( - stageId = 0, - partitionId = 0, - taskAttemptId = 0, - attemptNumber = 0, - taskMemoryManager = taskMemMgr, - metricsSystem = null)) - - val keySchema = new StructType().add("a", StringType) - val valueSchema = new StructType().add("b", IntegerType).add("c", IntegerType) - val sorter = new UnsafeKVExternalSorter( - keySchema, valueSchema, SparkEnv.get.blockManager, shuffleMemMgr, - 16 * 1024) - - val keyConverter = UnsafeProjection.create(keySchema) - val valueConverter = UnsafeProjection.create(valueSchema) + private val rand = new Random(42) + for (i <- 0 until 6) { + val keySchema = RandomDataGenerator.randomSchema(rand.nextInt(10) + 1, keyTypes) + val valueSchema = RandomDataGenerator.randomSchema(rand.nextInt(10) + 1, valueTypes) + testKVSorter(keySchema, valueSchema, spill = i > 3) + } - val rand = new Random(42) - val data = null +: Seq.fill[String](10) { - Seq.fill(rand.nextInt(100))(rand.nextPrintableChar()).mkString - } + /** + * Create a test case using randomly generated data for the given key and value schema. + * + * The approach works as follows: + * + * - Create input by randomly generating data based on the given schema + * - Run [[UnsafeKVExternalSorter]] on the generated data + * - Collect the output from the sorter, and make sure the keys are sorted in ascending order + * - Sort the input by both key and value, and sort the sorter output also by both key and value. + * Compare the sorted input and sorted output together to make sure all the key/values match. + * + * If spill is set to true, the sorter will spill probabilistically roughly every 100 records. + */ + private def testKVSorter(keySchema: StructType, valueSchema: StructType, spill: Boolean): Unit = { + + val keySchemaStr = keySchema.map(_.dataType.simpleString).mkString("[", ",", "]") + val valueSchemaStr = valueSchema.map(_.dataType.simpleString).mkString("[", ",", "]") + + test(s"kv sorting key schema $keySchemaStr and value schema $valueSchemaStr") { + // Calling this make sure we have block manager and everything else setup. + TestSQLContext + + val taskMemMgr = new TaskMemoryManager(new ExecutorMemoryManager(MemoryAllocator.HEAP)) + val shuffleMemMgr = new TestShuffleMemoryManager + TaskContext.setTaskContext(new TaskContextImpl( + stageId = 0, + partitionId = 0, + taskAttemptId = 98456, + attemptNumber = 0, + taskMemoryManager = taskMemMgr, + metricsSystem = null)) + + // Create the data converters + val kExternalConverter = CatalystTypeConverters.createToCatalystConverter(keySchema) + val vExternalConverter = CatalystTypeConverters.createToCatalystConverter(valueSchema) + val kConverter = UnsafeProjection.create(keySchema) + val vConverter = UnsafeProjection.create(valueSchema) + + val keyDataGen = RandomDataGenerator.forType(keySchema, nullable = false).get + val valueDataGen = RandomDataGenerator.forType(valueSchema, nullable = false).get + + val input = Seq.fill(1024) { + val k = kConverter(kExternalConverter.apply(keyDataGen.apply()).asInstanceOf[InternalRow]) + val v = vConverter(vExternalConverter.apply(valueDataGen.apply()).asInstanceOf[InternalRow]) + (k.asInstanceOf[InternalRow].copy(), v.asInstanceOf[InternalRow].copy()) + } - val inputRows = data.map { str => - keyConverter.apply(InternalRow(UTF8String.fromString(str))).copy() - } + val sorter = new UnsafeKVExternalSorter( + keySchema, valueSchema, SparkEnv.get.blockManager, shuffleMemMgr, 16 * 1024 * 1024) - var i = 0 - data.foreach { str => - if (str != null) { - val k = InternalRow(UTF8String.fromString(str)) - val v = InternalRow(str.length, str.length + 1) - sorter.insertKV(keyConverter.apply(k), valueConverter.apply(v)) - } else { - val k = InternalRow(UTF8String.fromString(str)) - val v = InternalRow(-1, -2) - sorter.insertKV(keyConverter.apply(k), valueConverter.apply(v)) + // Insert generated keys and values into the sorter + input.foreach { case (k, v) => + sorter.insertKV(k.asInstanceOf[UnsafeRow], v.asInstanceOf[UnsafeRow]) + // 1% chance we will spill + if (rand.nextDouble() < 0.01 && spill) { + shuffleMemMgr.markAsOutOfMemory() + sorter.closeCurrentPage() + } } - if ((i % 100) == 0) { - shuffleMemMgr.markAsOutOfMemory() - sorter.closeCurrentPage() + // Collect the sorted output + val out = new scala.collection.mutable.ArrayBuffer[(InternalRow, InternalRow)] + val iter = sorter.sortedIterator() + while (iter.next()) { + out += Tuple2(iter.getKey.copy(), iter.getValue.copy()) } - i += 1 - } - val out = new scala.collection.mutable.ArrayBuffer[InternalRow] - val iter = sorter.sortedIterator() - while (iter.next()) { - if (iter.getKey.getUTF8String(0) == null) { - withClue(s"for null key") { - assert(-1 === iter.getValue.getInt(0)) - assert(-2 === iter.getValue.getInt(1)) - } - } else { - val key = iter.getKey.getString(0) - withClue(s"for key $key") { - assert(key.length === iter.getValue.getInt(0)) - assert(key.length + 1 === iter.getValue.getInt(1)) + val keyOrdering = RowOrdering.forSchema(keySchema.map(_.dataType)) + val valueOrdering = RowOrdering.forSchema(valueSchema.map(_.dataType)) + val kvOrdering = new Ordering[(InternalRow, InternalRow)] { + override def compare(x: (InternalRow, InternalRow), y: (InternalRow, InternalRow)): Int = { + keyOrdering.compare(x._1, y._1) match { + case 0 => valueOrdering.compare(x._2, y._2) + case cmp => cmp + } } } - out += iter.getKey.copy() - } - assert(out === inputRows.sorted(RowOrdering.forSchema(keySchema.map(_.dataType)))) - } - - test("sorting arbitrary string data") { - - // Calling this make sure we have block manager and everything else setup. - TestSQLContext - - val taskMemMgr = new TaskMemoryManager(new ExecutorMemoryManager(MemoryAllocator.HEAP)) - val shuffleMemMgr = new TestShuffleMemoryManager - - TaskContext.setTaskContext(new TaskContextImpl( - stageId = 0, - partitionId = 0, - taskAttemptId = 0, - attemptNumber = 0, - taskMemoryManager = taskMemMgr, - metricsSystem = null)) - - val keySchema = new StructType().add("a", StringType) - val valueSchema = new StructType().add("b", IntegerType) - val sorter = new UnsafeKVExternalSorter( - keySchema, valueSchema, SparkEnv.get.blockManager, shuffleMemMgr, - 16 * 1024) - - val keyConverter = UnsafeProjection.create(keySchema) - val valueConverter = UnsafeProjection.create(valueSchema) - - val rand = new Random(42) - val data = Seq.fill(512) { - Seq.fill(rand.nextInt(100))(rand.nextPrintableChar()).mkString - } + // Testing to make sure output from the sorter is sorted by key + var prevK: InternalRow = null + out.zipWithIndex.foreach { case ((k, v), i) => + if (prevK != null) { + assert(keyOrdering.compare(prevK, k) <= 0, + s""" + |key is not in sorted order: + |previous key: $prevK + |current key : $k + """.stripMargin) + } + prevK = k + } - var i = 0 - data.foreach { str => - val k = InternalRow(UTF8String.fromString(str)) - val v = InternalRow(str.length) - sorter.insertKV(keyConverter.apply(k), valueConverter.apply(v)) + // Testing to make sure the key/value in output matches input + assert(out.sorted(kvOrdering) === input.sorted(kvOrdering)) - if ((i % 100) == 0) { - shuffleMemMgr.markAsOutOfMemory() - sorter.closeCurrentPage() + // Make sure there is no memory leak + val leakedUnsafeMemory: Long = taskMemMgr.cleanUpAllAllocatedMemory + if (shuffleMemMgr != null) { + val leakedShuffleMemory: Long = shuffleMemMgr.getMemoryConsumptionForThisTask() + assert(0L === leakedShuffleMemory) } - i += 1 + assert(0 === leakedUnsafeMemory) + TaskContext.unset() } - - val out = new scala.collection.mutable.ArrayBuffer[String] - val iter = sorter.sortedIterator() - while (iter.next()) { - assert(iter.getKey.getString(0).length === iter.getValue.getInt(0)) - out += iter.getKey.getString(0) - } - - assert(out === data.sorted) } } http://git-wip-us.apache.org/repos/asf/spark/blob/9d03ad91/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java ---------------------------------------------------------------------- diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java new file mode 100644 index 0000000..6274b92 --- /dev/null +++ b/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -0,0 +1,149 @@ +/* +* 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.unsafe.types; + +import org.junit.Test; + +import static junit.framework.Assert.*; +import static org.apache.spark.unsafe.types.CalendarInterval.*; + +public class CalendarIntervalSuite { + + @Test + public void equalsTest() { + CalendarInterval i1 = new CalendarInterval(3, 123); + CalendarInterval i2 = new CalendarInterval(3, 321); + CalendarInterval i3 = new CalendarInterval(1, 123); + CalendarInterval i4 = new CalendarInterval(3, 123); + + assertNotSame(i1, i2); + assertNotSame(i1, i3); + assertNotSame(i2, i3); + assertEquals(i1, i4); + } + + @Test + public void toStringTest() { + CalendarInterval i; + + i = new CalendarInterval(34, 0); + assertEquals(i.toString(), "interval 2 years 10 months"); + + i = new CalendarInterval(-34, 0); + assertEquals(i.toString(), "interval -2 years -10 months"); + + i = new CalendarInterval(0, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); + assertEquals(i.toString(), "interval 3 weeks 13 hours 123 microseconds"); + + i = new CalendarInterval(0, -3 * MICROS_PER_WEEK - 13 * MICROS_PER_HOUR - 123); + assertEquals(i.toString(), "interval -3 weeks -13 hours -123 microseconds"); + + i = new CalendarInterval(34, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); + assertEquals(i.toString(), "interval 2 years 10 months 3 weeks 13 hours 123 microseconds"); + } + + @Test + public void fromStringTest() { + testSingleUnit("year", 3, 36, 0); + testSingleUnit("month", 3, 3, 0); + testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); + testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); + testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); + testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); + testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); + testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); + testSingleUnit("microsecond", 3, 0, 3); + + String input; + + input = "interval -5 years 23 month"; + CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); + assertEquals(CalendarInterval.fromString(input), result); + + input = "interval -5 years 23 month "; + assertEquals(CalendarInterval.fromString(input), result); + + input = " interval -5 years 23 month "; + assertEquals(CalendarInterval.fromString(input), result); + + // Error cases + input = "interval 3month 1 hour"; + assertEquals(CalendarInterval.fromString(input), null); + + input = "interval 3 moth 1 hour"; + assertEquals(CalendarInterval.fromString(input), null); + + input = "interval"; + assertEquals(CalendarInterval.fromString(input), null); + + input = "int"; + assertEquals(CalendarInterval.fromString(input), null); + + input = ""; + assertEquals(CalendarInterval.fromString(input), null); + + input = null; + assertEquals(CalendarInterval.fromString(input), null); + } + + @Test + public void addTest() { + String input = "interval 3 month 1 hour"; + String input2 = "interval 2 month 100 hour"; + + CalendarInterval interval = CalendarInterval.fromString(input); + CalendarInterval interval2 = CalendarInterval.fromString(input2); + + assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); + + input = "interval -10 month -81 hour"; + input2 = "interval 75 month 200 hour"; + + interval = CalendarInterval.fromString(input); + interval2 = CalendarInterval.fromString(input2); + + assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); + } + + @Test + public void subtractTest() { + String input = "interval 3 month 1 hour"; + String input2 = "interval 2 month 100 hour"; + + CalendarInterval interval = CalendarInterval.fromString(input); + CalendarInterval interval2 = CalendarInterval.fromString(input2); + + assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); + + input = "interval -10 month -81 hour"; + input2 = "interval 75 month 200 hour"; + + interval = CalendarInterval.fromString(input); + interval2 = CalendarInterval.fromString(input2); + + assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); + } + + private void testSingleUnit(String unit, int number, int months, long microseconds) { + String input1 = "interval " + number + " " + unit; + String input2 = "interval " + number + " " + unit + "s"; + CalendarInterval result = new CalendarInterval(months, microseconds); + assertEquals(CalendarInterval.fromString(input1), result); + assertEquals(CalendarInterval.fromString(input2), result); + } +} http://git-wip-us.apache.org/repos/asf/spark/blob/9d03ad91/unsafe/src/test/java/org/apache/spark/unsafe/types/IntervalSuite.java ---------------------------------------------------------------------- diff --git a/unsafe/src/test/java/org/apache/spark/unsafe/types/IntervalSuite.java b/unsafe/src/test/java/org/apache/spark/unsafe/types/IntervalSuite.java deleted file mode 100644 index e6733a7..0000000 --- a/unsafe/src/test/java/org/apache/spark/unsafe/types/IntervalSuite.java +++ /dev/null @@ -1,149 +0,0 @@ -/* -* 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.unsafe.types; - -import org.junit.Test; - -import static junit.framework.Assert.*; -import static org.apache.spark.unsafe.types.CalendarInterval.*; - -public class IntervalSuite { - - @Test - public void equalsTest() { - CalendarInterval i1 = new CalendarInterval(3, 123); - CalendarInterval i2 = new CalendarInterval(3, 321); - CalendarInterval i3 = new CalendarInterval(1, 123); - CalendarInterval i4 = new CalendarInterval(3, 123); - - assertNotSame(i1, i2); - assertNotSame(i1, i3); - assertNotSame(i2, i3); - assertEquals(i1, i4); - } - - @Test - public void toStringTest() { - CalendarInterval i; - - i = new CalendarInterval(34, 0); - assertEquals(i.toString(), "interval 2 years 10 months"); - - i = new CalendarInterval(-34, 0); - assertEquals(i.toString(), "interval -2 years -10 months"); - - i = new CalendarInterval(0, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals(i.toString(), "interval 3 weeks 13 hours 123 microseconds"); - - i = new CalendarInterval(0, -3 * MICROS_PER_WEEK - 13 * MICROS_PER_HOUR - 123); - assertEquals(i.toString(), "interval -3 weeks -13 hours -123 microseconds"); - - i = new CalendarInterval(34, 3 * MICROS_PER_WEEK + 13 * MICROS_PER_HOUR + 123); - assertEquals(i.toString(), "interval 2 years 10 months 3 weeks 13 hours 123 microseconds"); - } - - @Test - public void fromStringTest() { - testSingleUnit("year", 3, 36, 0); - testSingleUnit("month", 3, 3, 0); - testSingleUnit("week", 3, 0, 3 * MICROS_PER_WEEK); - testSingleUnit("day", 3, 0, 3 * MICROS_PER_DAY); - testSingleUnit("hour", 3, 0, 3 * MICROS_PER_HOUR); - testSingleUnit("minute", 3, 0, 3 * MICROS_PER_MINUTE); - testSingleUnit("second", 3, 0, 3 * MICROS_PER_SECOND); - testSingleUnit("millisecond", 3, 0, 3 * MICROS_PER_MILLI); - testSingleUnit("microsecond", 3, 0, 3); - - String input; - - input = "interval -5 years 23 month"; - CalendarInterval result = new CalendarInterval(-5 * 12 + 23, 0); - assertEquals(CalendarInterval.fromString(input), result); - - input = "interval -5 years 23 month "; - assertEquals(CalendarInterval.fromString(input), result); - - input = " interval -5 years 23 month "; - assertEquals(CalendarInterval.fromString(input), result); - - // Error cases - input = "interval 3month 1 hour"; - assertEquals(CalendarInterval.fromString(input), null); - - input = "interval 3 moth 1 hour"; - assertEquals(CalendarInterval.fromString(input), null); - - input = "interval"; - assertEquals(CalendarInterval.fromString(input), null); - - input = "int"; - assertEquals(CalendarInterval.fromString(input), null); - - input = ""; - assertEquals(CalendarInterval.fromString(input), null); - - input = null; - assertEquals(CalendarInterval.fromString(input), null); - } - - @Test - public void addTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = CalendarInterval.fromString(input); - CalendarInterval interval2 = CalendarInterval.fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(5, 101 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = CalendarInterval.fromString(input); - interval2 = CalendarInterval.fromString(input2); - - assertEquals(interval.add(interval2), new CalendarInterval(65, 119 * MICROS_PER_HOUR)); - } - - @Test - public void subtractTest() { - String input = "interval 3 month 1 hour"; - String input2 = "interval 2 month 100 hour"; - - CalendarInterval interval = CalendarInterval.fromString(input); - CalendarInterval interval2 = CalendarInterval.fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(1, -99 * MICROS_PER_HOUR)); - - input = "interval -10 month -81 hour"; - input2 = "interval 75 month 200 hour"; - - interval = CalendarInterval.fromString(input); - interval2 = CalendarInterval.fromString(input2); - - assertEquals(interval.subtract(interval2), new CalendarInterval(-85, -281 * MICROS_PER_HOUR)); - } - - private void testSingleUnit(String unit, int number, int months, long microseconds) { - String input1 = "interval " + number + " " + unit; - String input2 = "interval " + number + " " + unit + "s"; - CalendarInterval result = new CalendarInterval(months, microseconds); - assertEquals(CalendarInterval.fromString(input1), result); - assertEquals(CalendarInterval.fromString(input2), result); - } -} --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscribe@spark.apache.org For additional commands, e-mail: commits-help@spark.apache.org