From d6460f6396e1a5cb3f356236cf22a1ef1aad3147 Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Fri, 7 Nov 2014 10:11:09 -0600 Subject: [PATCH 01/16] Adds RichDate and RichTimestamp classes with comparison operators, allowing them to be used in DSL expressions. These classes provide initializers which accept string representations of dates or times. They are renamed as Date and Timestamp when the members of an SQLContext are in scope. --- .../spark/sql/catalyst/types/timetypes.scala | 97 +++++++++++++++++++ .../org/apache/spark/sql/SQLContext.scala | 6 ++ .../scala/org/apache/spark/sql/package.scala | 33 +++++++ 3 files changed, 136 insertions(+) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala new file mode 100644 index 000000000000..fcb77e640a8e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala @@ -0,0 +1,97 @@ +/* + * 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.sql.catalyst.expressions + +import java.sql.{Date, Timestamp} +import scala.language.implicitConversions + +/* + * Subclass of java.sql.Date which provides the usual comparison + * operators (as required for catalyst expressions) and which can + * be constructed from a string. + * + * scala> val d1 = Date("2014-02-01") + * d1: Date = 2014-02-01 + * + * scala> val d2 = Date("2014-02-02") + * d2: Date = 2014-02-02 + * + * scala> d1 < d2 + * res1: Boolean = true + */ + +class RichDate(milliseconds: Long) extends Date(milliseconds) { + def <(that: Date): Boolean = this.before(that) + def >(that: Date): Boolean = this.after(that) + def <=(that: Date): Boolean = (this.before(that) || this.equals(that)) + def >=(that: Date): Boolean = (this.after(that) || this.equals(that)) + def ===(that: Date): Boolean = this.equals(that) +} + +object RichDate { + def apply(init: String) = new RichDate(Date.valueOf(init).getTime) +} + +/* + * Analogous subclass of java.sql.Timestamp. + * + * scala> val ts1 = Timestamp("2014-03-04 12:34:56.12") + * ts1: Timestamp = 2014-03-04 12:34:56.12 + * + * scala> val ts2 = Timestamp("2014-03-04 12:34:56.13") + * ts2: Timestamp = 2014-03-04 12:34:56.13 + * + * scala> ts1 < ts2 + * res13: Boolean = true + */ + +class RichTimestamp(milliseconds: Long) extends Timestamp(milliseconds) { + def <(that: Timestamp): Boolean = this.before(that) + def >(that: Timestamp): Boolean = this.after(that) + def <=(that: Timestamp): Boolean = (this.before(that) || this.equals(that)) + def >=(that: Timestamp): Boolean = (this.after(that) || this.equals(that)) + def ===(that: Timestamp): Boolean = this.equals(that) +} + +object RichTimestamp { + def apply(init: String) = new RichTimestamp(Timestamp.valueOf(init).getTime) +} + +/* + * Implicit conversions. + */ + +object TimeConversions { + + implicit def javaDateToRichDate(jdate: Date): RichDate = { + new Date(jdate.getTime) + } + + implicit def javaTimestampToRichTimestamp(jtimestamp: Timestamp): RichTimestamp = { + new Timestamp(jtimestamp.getTime) + } + + implicit def richDateToJavaDate(date: RichDate): Date = { + new Date(date.getTime) + } + + implicit def richTimestampToJavaTimestamp(timestamp: RichTimestamp): Timestamp = { + new Timestamp(timestamp.getTime) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 84eaf401f240..98370dcf8171 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -502,4 +502,10 @@ class SQLContext(@transient val sparkContext: SparkContext) new SchemaRDD(this, LogicalRDD(schema.toAttributes, rowRdd)(self)) } + + /* * + * Map RichDate and RichTimestamp to their expected names in this context. + */ + val Date = org.apache.spark.sql.catalyst.expressions.RichDate + val Timestamp = org.apache.spark.sql.catalyst.expressions.RichTimestamp } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 51dad54f1a3f..a1b3a221978f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -460,4 +460,37 @@ package object sql { */ @DeveloperApi type MetadataBuilder = catalyst.util.MetadataBuilder + + /** + * :: DeveloperApi :: + * + * A Date class which support the standard comparison operators, for + * use in DSL expressions. Implicit conversions to java.sql.Date + * are provided. The class intializer accepts a String, e.g. + * + * {{{ + * val d = RichDate("2014-01-01") + * }}} + * + * @group dataType + */ + @DeveloperApi + val RichDate = catalyst.expressions.RichDate + + /** + * :: DeveloperApi :: + * + * A Timestamp class which support the standard comparison + * operators, for use in DSL expressions. Implicit conversions to + * java.sql.timestamp are provided. The class intializer accepts a + * String, e.g. + * + * {{{ + * val ts = RichTimestamp("2014-01-01 12:34:56.78") + * }}} + * + * @group timeClasses + */ + @DeveloperApi + val RichTimestamp = catalyst.expressions.RichTimestamp } From fa073246a392a1234141bceb61a6a71780fd3b23 Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Fri, 7 Nov 2014 10:17:33 -0600 Subject: [PATCH 02/16] Adds new implicit conversions which allow DSL expressions to start with a literal, e.g. 0 < 'x . These conversions expose a conflict with the scalatest === operator if assert(X === Y) is used when the conversions are in scope. To fix this, several tests are modified, as recommended in the scalatest documentation, by making the change: assert(X === Y) --> assert(convertToEqualizer(X).===(Y)) --- .../spark/sql/catalyst/dsl/package.scala | 38 +++ .../expressions/SpecificMutableRow.scala | 30 ++ .../ExpressionEvaluationSuite.scala | 110 ++++--- .../sql/catalyst/trees/TreeNodeSuite.scala | 25 +- .../apache/spark/sql/CachedTableSuite.scala | 11 +- .../org/apache/spark/sql/DslQuerySuite.scala | 12 +- .../org/apache/spark/sql/JoinSuite.scala | 13 +- .../org/apache/spark/sql/SQLConfSuite.scala | 9 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 11 +- .../spark/sql/UserDefinedTypeSuite.scala | 11 +- .../columnar/PartitionBatchPruningSuite.scala | 11 +- .../spark/sql/execution/PlannerSuite.scala | 17 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 311 +++++++++--------- .../spark/sql/hive/StatisticsSuite.scala | 25 +- .../sql/hive/execution/HiveQuerySuite.scala | 21 +- .../spark/sql/parquet/HiveParquetSuite.scala | 14 +- 16 files changed, 429 insertions(+), 240 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 31dc5a58e68e..84fb594da372 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -148,6 +148,44 @@ package object dsl { def upper(e: Expression) = Upper(e) def lower(e: Expression) = Lower(e) + /* + * Conversions to provide the standard operators in the special case + * where a literal is being combined with a symbol. Without these an + * expression such as 0 < 'x is not recognized. + */ + class LhsLiteral(x: Any) { + val literal = Literal(x) + def + (other: Symbol) = Add(literal, other) + def - (other: Symbol) = Subtract(literal, other) + def * (other: Symbol) = Multiply(literal, other) + def / (other: Symbol) = Divide(literal, other) + def % (other: Symbol) = Remainder(literal, other) + + def && (other: Symbol) = And(literal, other) + def || (other: Symbol) = Or(literal, other) + + def < (other: Symbol) = LessThan(literal, other) + def <= (other: Symbol) = LessThanOrEqual(literal, other) + def > (other: Symbol) = GreaterThan(literal, other) + def >= (other: Symbol) = GreaterThanOrEqual(literal, other) + def === (other: Symbol) = EqualTo(literal, other) + def <=> (other: Symbol) = EqualNullSafe(literal, other) + def !== (other: Symbol) = Not(EqualTo(literal, other)) + } + + implicit def booleanToLhsLiteral(b: Boolean) = new LhsLiteral(b) + implicit def byteToLhsLiteral(b: Byte) = new LhsLiteral(b) + implicit def shortToLhsLiteral(s: Short) = new LhsLiteral(s) + implicit def intToLhsLiteral(i: Int) = new LhsLiteral(i) + implicit def longToLhsLiteral(l: Long) = new LhsLiteral(l) + implicit def floatToLhsLiteral(f: Float) = new LhsLiteral(f) + implicit def doubleToLhsLiteral(d: Double) = new LhsLiteral(d) + implicit def stringToLhsLiteral(s: String) = new LhsLiteral(s) + implicit def bigDecimalToLhsLiteral(d: BigDecimal) = new LhsLiteral(d) + implicit def decimalToLhsLiteral(d: Decimal) = new LhsLiteral(d) + implicit def dateToLhsLiteral(d: Date) = new LhsLiteral(d) + implicit def timestampToLhsLiteral(t: Timestamp) = new LhsLiteral(t) + implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } // TODO more implicit class for literal? implicit class DslString(val s: String) extends ImplicitOperators { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala index 570379c533e1..9f977bf6c2a0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ +import java.sql.{Date, Timestamp} /** * A parent class for mutable container objects that are reused when the values are changed, @@ -169,6 +170,35 @@ final class MutableByte extends MutableValue { newCopy.asInstanceOf[this.type] } } +final class MutableDate extends MutableValue { + var value: Date = new Date(0) + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Date] + } + def copy() = { + val newCopy = new MutableDate + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} + +final class MutableTimestamp extends MutableValue { + var value: Timestamp = new Timestamp(0) + def boxed = if (isNull) null else value + def update(v: Any) = value = { + isNull = false + v.asInstanceOf[Timestamp] + } + def copy() = { + val newCopy = new MutableTimestamp + newCopy.isNull = isNull + newCopy.value = value + newCopy.asInstanceOf[this.type] + } +} final class MutableAny extends MutableValue { var value: Any = _ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 918996f11da2..085f6a6cae70 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -32,6 +32,13 @@ import org.apache.spark.sql.catalyst.types._ /* Implicit conversions */ import org.apache.spark.sql.catalyst.dsl.expressions._ +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + class ExpressionEvaluationSuite extends FunSuite { test("literals") { @@ -318,18 +325,18 @@ class ExpressionEvaluationSuite extends FunSuite { intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} - assert(("abcdef" cast StringType).nullable === false) - assert(("abcdef" cast BinaryType).nullable === false) - assert(("abcdef" cast BooleanType).nullable === false) - assert(("abcdef" cast TimestampType).nullable === true) - assert(("abcdef" cast LongType).nullable === true) - assert(("abcdef" cast IntegerType).nullable === true) - assert(("abcdef" cast ShortType).nullable === true) - assert(("abcdef" cast ByteType).nullable === true) - assert(("abcdef" cast DecimalType.Unlimited).nullable === true) - assert(("abcdef" cast DecimalType(4, 2)).nullable === true) - assert(("abcdef" cast DoubleType).nullable === true) - assert(("abcdef" cast FloatType).nullable === true) + assert(EQ(("abcdef" cast StringType).nullable).===(false)) + assert(EQ(("abcdef" cast BinaryType).nullable).===(false)) + assert(EQ(("abcdef" cast BooleanType).nullable).===(false)) + assert(EQ(("abcdef" cast TimestampType).nullable).===(true)) + assert(EQ(("abcdef" cast LongType).nullable).===(true)) + assert(EQ(("abcdef" cast IntegerType).nullable).===(true)) + assert(EQ(("abcdef" cast ShortType).nullable).===(true)) + assert(EQ(("abcdef" cast ByteType).nullable).===(true)) + assert(EQ(("abcdef" cast DecimalType.Unlimited).nullable).===(true)) + assert(EQ(("abcdef" cast DecimalType(4, 2)).nullable).===(true)) + assert(EQ(("abcdef" cast DoubleType).nullable).===(true)) + assert(EQ(("abcdef" cast FloatType).nullable).===(true)) checkEvaluation(Cast(Literal(null, IntegerType), ShortType), null) } @@ -346,15 +353,15 @@ class ExpressionEvaluationSuite extends FunSuite { // - Values that would overflow the target precision should turn into null // - Because of this, casts to fixed-precision decimals should be nullable - assert(Cast(Literal(123), DecimalType.Unlimited).nullable === false) - assert(Cast(Literal(10.03f), DecimalType.Unlimited).nullable === false) - assert(Cast(Literal(10.03), DecimalType.Unlimited).nullable === false) - assert(Cast(Literal(Decimal(10.03)), DecimalType.Unlimited).nullable === false) + assert(EQ(Cast(Literal(123), DecimalType.Unlimited).nullable).===(false)) + assert(EQ(Cast(Literal(10.03f), DecimalType.Unlimited).nullable).===(false)) + assert(EQ(Cast(Literal(10.03), DecimalType.Unlimited).nullable).===(false)) + assert(EQ(Cast(Literal(Decimal(10.03)), DecimalType.Unlimited).nullable).===(false)) - assert(Cast(Literal(123), DecimalType(2, 1)).nullable === true) - assert(Cast(Literal(10.03f), DecimalType(2, 1)).nullable === true) - assert(Cast(Literal(10.03), DecimalType(2, 1)).nullable === true) - assert(Cast(Literal(Decimal(10.03)), DecimalType(2, 1)).nullable === true) + assert(EQ(Cast(Literal(123), DecimalType(2, 1)).nullable).===(true)) + assert(EQ(Cast(Literal(10.03f), DecimalType(2, 1)).nullable).===(true)) + assert(EQ(Cast(Literal(10.03), DecimalType(2, 1)).nullable).===(true)) + assert(EQ(Cast(Literal(Decimal(10.03)), DecimalType(2, 1)).nullable).===(true)) checkEvaluation(Cast(Literal(123), DecimalType.Unlimited), Decimal(123)) checkEvaluation(Cast(Literal(123), DecimalType(3, 0)), Decimal(123)) @@ -502,26 +509,26 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5, c6)), "c", row) checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5)), null, row) - assert(CaseWhen(Seq(c2, c4, c6)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5, c6)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5)).nullable === true) + assert(EQ(CaseWhen(Seq(c2, c4, c6)).nullable).===(true)) + assert(EQ(CaseWhen(Seq(c2, c4, c3, c5, c6)).nullable).===(true)) + assert(EQ(CaseWhen(Seq(c2, c4, c3, c5)).nullable).===(true)) val c4_notNull = 'a.boolean.notNull.at(3) val c5_notNull = 'a.boolean.notNull.at(4) val c6_notNull = 'a.boolean.notNull.at(5) - assert(CaseWhen(Seq(c2, c4_notNull, c6_notNull)).nullable === false) - assert(CaseWhen(Seq(c2, c4, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c6)).nullable === true) + assert(EQ(CaseWhen(Seq(c2, c4_notNull, c6_notNull)).nullable).===(false)) + assert(EQ(CaseWhen(Seq(c2, c4, c6_notNull)).nullable).===(true)) + assert(EQ(CaseWhen(Seq(c2, c4_notNull, c6)).nullable).===(true)) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6_notNull)).nullable === false) - assert(CaseWhen(Seq(c2, c4, c3, c5_notNull, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5, c6_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6)).nullable === true) + assert(EQ(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6_notNull)).nullable).===(false)) + assert(EQ(CaseWhen(Seq(c2, c4, c3, c5_notNull, c6_notNull)).nullable).===(true)) + assert(EQ(CaseWhen(Seq(c2, c4_notNull, c3, c5, c6_notNull)).nullable).===(true)) + assert(EQ(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6)).nullable).===(true)) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4, c3, c5_notNull)).nullable === true) - assert(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable === true) + assert(EQ(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull)).nullable).===(true)) + assert(EQ(CaseWhen(Seq(c2, c4, c3, c5_notNull)).nullable).===(true)) + assert(EQ(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable).===(true)) } test("complex type") { @@ -561,11 +568,11 @@ class ExpressionEvaluationSuite extends FunSuite { :: StructField("b", StringType, nullable = false) :: Nil ) - assert(GetField(BoundReference(2,typeS, nullable = true), "a").nullable === true) - assert(GetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable === false) + assert(EQ(GetField(BoundReference(2,typeS, nullable = true), "a").nullable).===(true)) + assert(EQ(GetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable).===(false)) - assert(GetField(Literal(null, typeS), "a").nullable === true) - assert(GetField(Literal(null, typeS_notNullable), "a").nullable === true) + assert(EQ(GetField(Literal(null, typeS), "a").nullable).===(true)) + assert(EQ(GetField(Literal(null, typeS_notNullable), "a").nullable).===(true)) checkEvaluation('c.map(typeMap).at(3).getItem("aa"), "bb", row) checkEvaluation('c.array(typeArray.elementType).at(4).getItem(1), "bb", row) @@ -719,10 +726,10 @@ class ExpressionEvaluationSuite extends FunSuite { val s_notNull = 'a.string.notNull.at(0) - assert(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === true) - assert(Substring(s_notNull, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === false) - assert(Substring(s_notNull, Literal(null, IntegerType), Literal(2, IntegerType)).nullable === true) - assert(Substring(s_notNull, Literal(0, IntegerType), Literal(null, IntegerType)).nullable === true) + assert(EQ(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)).nullable).===(true)) + assert(EQ(Substring(s_notNull, Literal(0, IntegerType), Literal(2, IntegerType)).nullable).===(false)) + assert(EQ(Substring(s_notNull, Literal(null, IntegerType), Literal(2, IntegerType)).nullable).===(true)) + assert(EQ(Substring(s_notNull, Literal(0, IntegerType), Literal(null, IntegerType)).nullable).===(true)) checkEvaluation(s.substr(0, 2), "ex", row) checkEvaluation(s.substr(0), "example", row) @@ -774,4 +781,25 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(c1 ^ c2, 3, row) checkEvaluation(~c1, -2, row) } + + test("recognizes literals on the left") { + assert(EQ(-1 + 'x).===(Add(-1, 'x))) + assert(EQ(0 < 'x).===(LessThan(0, 'x))) + assert(EQ(1.5 === 'x).===(EqualTo(1.5, 'x))) + assert(EQ(false !== 'x).===(Not(EqualTo(false, 'x)))) + assert(EQ("a string" >= 'x).===(GreaterThanOrEqual("a string", 'x))) + assert(EQ(RichDate("2014-11-05") > 'date).===(GreaterThan(RichDate("2014-11-05"), 'date))) + assert(EQ(RichTimestamp("2014-11-05 12:34:56.789") < 'now).===( + LessThan(RichTimestamp("2014-11-05 12:34:56.789"), 'now))) + } + + test("comparison operators for RichDate and RichTimestamp") { + assert(EQ(RichDate("2014-11-05") < RichDate("2014-11-06")).===(true)) + assert(EQ(RichDate("2014-11-05") <= RichDate("2013-11-06")).===(false)) + assert(EQ(RichTimestamp("2014-11-05 12:34:56.5432") > RichTimestamp("2014-11-05 00:00:00") + ).===(true)) + assert(EQ(RichTimestamp("2014-11-05 12:34:56") >= RichTimestamp("2014-11-06 00:00:00") + ).===(false)) + } + } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 036fd3fa1d6a..3586e6557aa1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -24,6 +24,13 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{StringType, NullType} +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + case class Dummy(optKey: Option[Expression]) extends Expression { def children = optKey.toSeq def nullable = true @@ -36,21 +43,21 @@ case class Dummy(optKey: Option[Expression]) extends Expression { class TreeNodeSuite extends FunSuite { test("top node changed") { val after = Literal(1) transform { case Literal(1, _) => Literal(2) } - assert(after === Literal(2)) + assert(EQ(after).===(Literal(2))) } test("one child changed") { val before = Add(Literal(1), Literal(2)) val after = before transform { case Literal(2, _) => Literal(1) } - assert(after === Add(Literal(1), Literal(1))) + assert(EQ(after).===(Add(Literal(1), Literal(1)))) } test("no change") { val before = Add(Literal(1), Add(Literal(2), Add(Literal(3), Literal(4)))) val after = before transform { case Literal(5, _) => Literal(1)} - assert(before === after) + assert(EQ(before).===(after)) // Ensure that the objects after are the same objects before the transformation. before.map(identity[Expression]).zip(after.map(identity[Expression])).foreach { case (b, a) => assert(b eq a) @@ -61,7 +68,7 @@ class TreeNodeSuite extends FunSuite { val tree = Add(Literal(1), Add(Literal(2), Add(Literal(3), Literal(4)))) val literals = tree collect {case l: Literal => l} - assert(literals.size === 4) + assert(EQ(literals.size).===(4)) (1 to 4).foreach(i => assert(literals contains Literal(i))) } @@ -74,7 +81,7 @@ class TreeNodeSuite extends FunSuite { case l: Literal => actual.append(l.toString); l } - assert(expected === actual) + assert(EQ(expected).===(actual)) } test("post-order transform") { @@ -86,7 +93,7 @@ class TreeNodeSuite extends FunSuite { case l: Literal => actual.append(l.toString); l } - assert(expected === actual) + assert(EQ(expected).===(actual)) } test("transform works on nodes with Option children") { @@ -95,13 +102,13 @@ class TreeNodeSuite extends FunSuite { val toZero: PartialFunction[Expression, Expression] = { case Literal(_, _) => Literal(0) } var actual = dummy1 transformDown toZero - assert(actual === Dummy(Some(Literal(0)))) + assert(EQ(actual).===(Dummy(Some(Literal(0))))) actual = dummy1 transformUp toZero - assert(actual === Dummy(Some(Literal(0)))) + assert(EQ(actual).===(Dummy(Some(Literal(0))))) actual = dummy2 transform toZero - assert(actual === Dummy(None)) + assert(EQ(actual).===(Dummy(None))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 765fa8277634..bc8efd787b1d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -22,6 +22,13 @@ import org.apache.spark.sql.columnar._ import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.storage.{StorageLevel, RDDBlockId} +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + case class BigData(s: String) class CachedTableSuite extends QueryTest { @@ -74,7 +81,7 @@ class CachedTableSuite extends QueryTest { val data = "*" * 10000 sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).registerTempTable("bigData") table("bigData").persist(StorageLevel.MEMORY_AND_DISK) - assert(table("bigData").count() === 200000L) + assert(EQ(table("bigData").count()).===(200000L)) table("bigData").unpersist(blocking = true) } @@ -228,7 +235,7 @@ class CachedTableSuite extends QueryTest { table("testData").queryExecution.withCachedData.collect { case cached: InMemoryRelation => val actualSizeInBytes = (1 to 100).map(i => INT.defaultSize + i.toString.length + 4).sum - assert(cached.statistics.sizeInBytes === actualSizeInBytes) + assert(EQ(cached.statistics.sizeInBytes).===(actualSizeInBytes)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index e70ad891eea3..34d58a09af85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -24,6 +24,14 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.dsl._ import org.apache.spark.sql.test.TestSQLContext._ +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + + class DslQuerySuite extends QueryTest { import org.apache.spark.sql.TestData._ @@ -171,7 +179,7 @@ class DslQuerySuite extends QueryTest { } test("count") { - assert(testData2.count() === testData2.map(_ => 1).count()) + assert(EQ(testData2.count()).===(testData2.map(_ => 1).count())) } test("null count") { @@ -192,7 +200,7 @@ class DslQuerySuite extends QueryTest { } test("zero count") { - assert(emptyTableData.count() === 0) + assert(EQ(emptyTableData.count()).===(0)) } test("except") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 8b4cf5bac018..3d1f1801778c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -25,6 +25,13 @@ import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftOuter, RightOu import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.test.TestSQLContext._ +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + class JoinSuite extends QueryTest with BeforeAndAfterEach { // Ensures tables are loaded. TestData @@ -34,7 +41,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { val y = testData2.as('y) val join = x.join(y, Inner, Some("x.a".attr === "y.a".attr)).queryExecution.analyzed val planned = planner.HashJoin(join) - assert(planned.size === 1) + assert(EQ(planned.size).===(1)) } def assertJoin(sqlString: String, c: Class[_]): Any = { @@ -50,7 +57,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { case j: BroadcastNestedLoopJoin => j } - assert(operators.size === 1) + assert(EQ(operators.size).===(1)) if (operators(0).getClass() != c) { fail(s"$sqlString expected operator: $c, but got ${operators(0)}\n physical: \n$physical") } @@ -104,7 +111,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { val join = x.join(y, Inner, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)).queryExecution.analyzed val planned = planner.HashJoin(join) - assert(planned.size === 1) + assert(EQ(planned.size).===(1)) } test("inner join where, one match per row") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index 60701f0e154f..df22ec9b1bcd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -24,6 +24,13 @@ import org.apache.spark.sql.test._ /* Implicits */ import TestSQLContext._ +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + class SQLConfSuite extends QueryTest with FunSuiteLike { val testKey = "test.key.0" @@ -38,7 +45,7 @@ class SQLConfSuite extends QueryTest with FunSuiteLike { test("programmatic ways of basic setting and getting") { clear() - assert(getAllConfs.size === 0) + assert(EQ(getAllConfs.size).===(0)) setConf(testKey, testVal) assert(getConf(testKey) == testVal) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index ef9b76b1e251..5d5c2b0b5168 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -22,18 +22,25 @@ import org.apache.spark.sql.test._ /* Implicits */ import TestSQLContext._ +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + case class FunctionResult(f1: String, f2: String) class UDFSuite extends QueryTest { test("Simple UDF") { registerFunction("strLenScala", (_: String).length) - assert(sql("SELECT strLenScala('test')").first().getInt(0) === 4) + assert(EQ(sql("SELECT strLenScala('test')").first().getInt(0)).===(4)) } test("TwoArgument UDF") { registerFunction("strLenScala", (_: String).length + (_:Int)) - assert(sql("SELECT strLenScala('test', 1)").first().getInt(0) === 5) + assert(EQ(sql("SELECT strLenScala('test', 1)").first().getInt(0)).===(5)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 1806a1dd8202..3c1505fd63e4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -24,6 +24,13 @@ import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType import org.apache.spark.sql.catalyst.types.UserDefinedType import org.apache.spark.sql.test.TestSQLContext._ +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { override def equals(other: Any): Boolean = other match { @@ -69,14 +76,14 @@ class UserDefinedTypeSuite extends QueryTest { test("register user type: MyDenseVector for MyLabeledPoint") { val labels: RDD[Double] = pointsRDD.select('label).map { case Row(v: Double) => v } val labelsArrays: Array[Double] = labels.collect() - assert(labelsArrays.size === 2) + assert(EQ(labelsArrays.size).===(2)) assert(labelsArrays.contains(1.0)) assert(labelsArrays.contains(0.0)) val features: RDD[MyDenseVector] = pointsRDD.select('features).map { case Row(v: MyDenseVector) => v } val featuresArrays: Array[MyDenseVector] = features.collect() - assert(featuresArrays.size === 2) + assert(EQ(featuresArrays.size).===(2)) assert(featuresArrays.contains(new MyDenseVector(Array(0.1, 1.0)))) assert(featuresArrays.contains(new MyDenseVector(Array(0.2, 2.0)))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index 9ba3c210171b..a6f31e0e1530 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -22,6 +22,13 @@ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.spark.sql._ import org.apache.spark.sql.test.TestSQLContext._ +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfter { val originalColumnBatchSize = columnBatchSize val originalInMemoryPartitionPruning = inMemoryPartitionPruning @@ -107,8 +114,8 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be case in: InMemoryColumnarTableScan => (in.readPartitions.value, in.readBatches.value) }.head - assert(readBatches === expectedReadBatches, "Wrong number of read batches") - assert(readPartitions === expectedReadPartitions, "Wrong number of read partitions") + assert(EQ(readBatches).===(expectedReadBatches), "Wrong number of read batches") + assert(EQ(readPartitions).===(expectedReadPartitions), "Wrong number of read partitions") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index a5af71acfc79..2c43c110e32b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -27,6 +27,13 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.planner._ +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + class PlannerSuite extends FunSuite { test("unions are collapsed") { val query = testData.unionAll(testData).unionAll(testData).logicalPlan @@ -34,8 +41,8 @@ class PlannerSuite extends FunSuite { val logicalUnions = query collect { case u: logical.Union => u } val physicalUnions = planned collect { case u: execution.Union => u } - assert(logicalUnions.size === 2) - assert(physicalUnions.size === 1) + assert(EQ(logicalUnions.size).===(2)) + assert(EQ(physicalUnions.size).===(1)) } test("count is partially aggregated") { @@ -43,7 +50,7 @@ class PlannerSuite extends FunSuite { val planned = HashAggregation(query).head val aggregations = planned.collect { case n if n.nodeName contains "Aggregate" => n } - assert(aggregations.size === 2) + assert(EQ(aggregations.size).===(2)) } test("count distinct is partially aggregated") { @@ -71,7 +78,7 @@ class PlannerSuite extends FunSuite { val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join } - assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") + assert(EQ(broadcastHashJoins.size).===(1), "Should use broadcast hash join") assert(shuffledHashJoins.isEmpty, "Should not use shuffled hash join") setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, origThreshold.toString) @@ -91,7 +98,7 @@ class PlannerSuite extends FunSuite { val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join } - assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") + assert(EQ(broadcastHashJoins.size).===(1), "Should use broadcast hash join") assert(shuffledHashJoins.isEmpty, "Should not use shuffled hash join") setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, origThreshold.toString) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 3cccafe92d4f..5bede8a05484 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -30,6 +30,13 @@ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.util.Utils +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + case class TestRDDEntry(key: Int, value: String) case class NullReflectData( @@ -172,7 +179,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) var actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + assert(EQ(actualCodec).===(TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil)) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -188,7 +195,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + assert(EQ(actualCodec).===(TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil)) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -204,7 +211,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(actualCodec === "UNCOMPRESSED" :: Nil) + assert(EQ(actualCodec).===("UNCOMPRESSED" :: Nil)) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -220,7 +227,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + assert(EQ(actualCodec).===(TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil)) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -236,7 +243,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + assert(EQ(actualCodec).===(TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil)) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -285,8 +292,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } val result = query.collect() - assert(result.size === 9, "self-join result has incorrect size") - assert(result(0).size === 12, "result row has incorrect size") + assert(EQ(result.size).===(9), "self-join result has incorrect size") + assert(EQ(result(0).size).===(12), "result row has incorrect size") result.zipWithIndex.foreach { case (row, index) => row.zipWithIndex.foreach { case (field, column) => assert(field != null, s"self-join contains null value in row $index field $column") @@ -296,7 +303,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Import of simple Parquet file") { val result = parquetFile(ParquetTestData.testDir.toString).collect() - assert(result.size === 15) + assert(EQ(result.size).===(15)) result.zipWithIndex.foreach { case (row, index) => { val checkBoolean = @@ -304,12 +311,12 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA row(0) == true else row(0) == false - assert(checkBoolean === true, s"boolean field value in line $index did not match") - if (index % 5 == 0) assert(row(1) === 5, s"int field value in line $index did not match") - assert(row(2) === "abc", s"string field value in line $index did not match") - assert(row(3) === (index.toLong << 33), s"long value in line $index did not match") - assert(row(4) === 2.5F, s"float field value in line $index did not match") - assert(row(5) === 4.5D, s"double field value in line $index did not match") + assert(EQ(checkBoolean).===(true), s"boolean field value in line $index did not match") + if (index % 5 == 0) assert(EQ(row(1)).===(5), s"int field value in line $index did not match") + assert(EQ(row(2)).===("abc"), s"string field value in line $index did not match") + assert(EQ(row(3)).===((index.toLong << 33)), s"long value in line $index did not match") + assert(EQ(row(4)).===(2.5F), s"float field value in line $index did not match") + assert(EQ(row(5)).===(4.5D), s"double field value in line $index did not match") } } } @@ -319,11 +326,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA result.zipWithIndex.foreach { case (row, index) => { if (index % 3 == 0) - assert(row(0) === true, s"boolean field value in line $index did not match (every third row)") + assert(EQ(row(0)).===(true), s"boolean field value in line $index did not match (every third row)") else - assert(row(0) === false, s"boolean field value in line $index did not match") - assert(row(1) === (index.toLong << 33), s"long field value in line $index did not match") - assert(row.size === 2, s"number of columns in projection in line $index is incorrect") + assert(EQ(row(0)).===(false), s"boolean field value in line $index did not match") + assert(EQ(row(1)).===((index.toLong << 33)), s"long field value in line $index did not match") + assert(EQ(row.size).===(2), s"number of columns in projection in line $index is incorrect") } } } @@ -381,8 +388,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val rdd_copy = sql("SELECT * FROM tmpx").collect() val rdd_orig = rdd.collect() for(i <- 0 to 99) { - assert(rdd_copy(i).apply(0) === rdd_orig(i).key, s"key error in line $i") - assert(rdd_copy(i).apply(1) === rdd_orig(i).value, s"value error in line $i") + assert(EQ(rdd_copy(i).apply(0)).===(rdd_orig(i).key), s"key error in line $i") + assert(EQ(rdd_copy(i).apply(1)).===(rdd_orig(i).value), s"value error in line $i") } Utils.deleteRecursively(file) } @@ -396,11 +403,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA dest_rdd.registerTempTable("dest") sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect() val rdd_copy1 = sql("SELECT * FROM dest").collect() - assert(rdd_copy1.size === 100) + assert(EQ(rdd_copy1.size).===(100)) sql("INSERT INTO dest SELECT * FROM source") val rdd_copy2 = sql("SELECT * FROM dest").collect().sortBy(_.getInt(0)) - assert(rdd_copy2.size === 200) + assert(EQ(rdd_copy2.size).===(200)) Utils.deleteRecursively(dirname) } @@ -408,7 +415,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA sql("INSERT INTO testsource SELECT * FROM testsource") val double_rdd = sql("SELECT * FROM testsource").collect() assert(double_rdd != null) - assert(double_rdd.size === 30) + assert(EQ(double_rdd.size).===(30)) // let's restore the original test data Utils.deleteRecursively(ParquetTestData.testDir) @@ -425,7 +432,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val readFile = parquetFile(path) val rdd_saved = readFile.collect() - assert(rdd_saved(0) === Seq.fill(5)(null)) + assert(EQ(rdd_saved(0)).===(Seq.fill(5)(null))) Utils.deleteRecursively(file) assert(true) } @@ -440,7 +447,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val readFile = parquetFile(path) val rdd_saved = readFile.collect() - assert(rdd_saved(0) === Seq.fill(5)(null)) + assert(EQ(rdd_saved(0)).===(Seq.fill(5)(null))) Utils.deleteRecursively(file) assert(true) } @@ -478,11 +485,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val attribute2 = new AttributeReference("second", IntegerType, false)() val predicate5 = new GreaterThan(attribute1, attribute2) val badfilter = ParquetFilters.createFilter(predicate5) - assert(badfilter.isDefined === false) + assert(EQ(badfilter.isDefined).===(false)) val predicate6 = And(GreaterThan(attribute1, attribute2), GreaterThan(attribute1, attribute2)) val badfilter2 = ParquetFilters.createFilter(predicate6) - assert(badfilter2.isDefined === false) + assert(EQ(badfilter2.isDefined).===(false)) } test("test filter by predicate pushdown") { @@ -492,21 +499,21 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query1.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result1 = query1.collect() - assert(result1.size === 50) - assert(result1(0)(1) === 100) - assert(result1(49)(1) === 149) + assert(EQ(result1.size).===(50)) + assert(EQ(result1(0)(1)).===(100)) + assert(EQ(result1(49)(1)).===(149)) val query2 = sql(s"SELECT * FROM testfiltersource WHERE $myval > 150 AND $myval <= 200") assert( query2.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result2 = query2.collect() - assert(result2.size === 50) + assert(EQ(result2.size).===(50)) if (myval == "myint" || myval == "mylong") { - assert(result2(0)(1) === 151) - assert(result2(49)(1) === 200) + assert(EQ(result2(0)(1)).===(151)) + assert(EQ(result2(49)(1)).===(200)) } else { - assert(result2(0)(1) === 150) - assert(result2(49)(1) === 199) + assert(EQ(result2(0)(1)).===(150)) + assert(EQ(result2(49)(1)).===(199)) } } for(myval <- Seq("myint", "mylong")) { @@ -515,11 +522,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query3.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result3 = query3.collect() - assert(result3.size === 20) - assert(result3(0)(1) === 0) - assert(result3(9)(1) === 9) - assert(result3(10)(1) === 191) - assert(result3(19)(1) === 200) + assert(EQ(result3.size).===(20)) + assert(EQ(result3(0)(1)).===(0)) + assert(EQ(result3(9)(1)).===(9)) + assert(EQ(result3(10)(1)).===(191)) + assert(EQ(result3(19)(1)).===(200)) } for(myval <- Seq("mydouble", "myfloat")) { val result4 = @@ -534,18 +541,18 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA // currently no way to specify float constants in SqlParser? sql(s"SELECT * FROM testfiltersource WHERE $myval > 190.5 OR $myval < 10").collect() } - assert(result4.size === 20) - assert(result4(0)(1) === 0) - assert(result4(9)(1) === 9) - assert(result4(10)(1) === 191) - assert(result4(19)(1) === 200) + assert(EQ(result4.size).===(20)) + assert(EQ(result4(0)(1)).===(0)) + assert(EQ(result4(9)(1)).===(9)) + assert(EQ(result4(10)(1)).===(191)) + assert(EQ(result4(19)(1)).===(200)) } val query5 = sql(s"SELECT * FROM testfiltersource WHERE myboolean = true AND myint < 40") assert( query5.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val booleanResult = query5.collect() - assert(booleanResult.size === 10) + assert(EQ(booleanResult.size).===(10)) for(i <- 0 until 10) { if (!booleanResult(i).getBoolean(0)) { fail(s"Boolean value in result row $i not true") @@ -559,16 +566,16 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query6.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val stringResult = query6.collect() - assert(stringResult.size === 1) + assert(EQ(stringResult.size).===(1)) assert(stringResult(0).getString(2) == "100", "stringvalue incorrect") - assert(stringResult(0).getInt(1) === 100) + assert(EQ(stringResult(0).getInt(1)).===(100)) val query7 = sql(s"SELECT * FROM testfiltersource WHERE myoptint < 40") assert( query7.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val optResult = query7.collect() - assert(optResult.size === 20) + assert(EQ(optResult.size).===(20)) for(i <- 0 until 20) { if (optResult(i)(7) != i * 2) { fail(s"optional Int value in result row $i should be ${2*4*i}") @@ -580,21 +587,21 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query8.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result8 = query8.collect() - assert(result8.size === 25) - assert(result8(0)(7) === 100) - assert(result8(24)(7) === 148) + assert(EQ(result8.size).===(25)) + assert(EQ(result8(0)(7)).===(100)) + assert(EQ(result8(24)(7)).===(148)) val query9 = sql(s"SELECT * FROM testfiltersource WHERE $myval > 150 AND $myval <= 200") assert( query9.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result9 = query9.collect() - assert(result9.size === 25) + assert(EQ(result9.size).===(25)) if (myval == "myoptint" || myval == "myoptlong") { - assert(result9(0)(7) === 152) - assert(result9(24)(7) === 200) + assert(EQ(result9(0)(7)).===(152)) + assert(EQ(result9(24)(7)).===(200)) } else { - assert(result9(0)(7) === 150) - assert(result9(24)(7) === 198) + assert(EQ(result9(0)(7)).===(150)) + assert(EQ(result9(24)(7)).===(198)) } } val query10 = sql("SELECT * FROM testfiltersource WHERE myoptstring = \"100\"") @@ -602,15 +609,15 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query10.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result10 = query10.collect() - assert(result10.size === 1) + assert(EQ(result10.size).===(1)) assert(result10(0).getString(8) == "100", "stringvalue incorrect") - assert(result10(0).getInt(7) === 100) + assert(EQ(result10(0).getInt(7)).===(100)) val query11 = sql(s"SELECT * FROM testfiltersource WHERE myoptboolean = true AND myoptint < 40") assert( query11.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result11 = query11.collect() - assert(result11.size === 7) + assert(EQ(result11.size).===(7)) for(i <- 0 until 6) { if (!result11(i).getBoolean(6)) { fail(s"optional Boolean value in result row $i not true") @@ -663,7 +670,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("SPARK-1913 regression: columns only referenced by pushed down filters should remain") { val query = sql(s"SELECT mystring FROM testfiltersource WHERE myint < 10") - assert(query.collect().size === 10) + assert(EQ(query.collect().size).===(10)) } test("Importing nested Parquet file (Addressbook)") { @@ -672,32 +679,32 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .toSchemaRDD .collect() assert(result != null) - assert(result.size === 2) + assert(EQ(result.size).===(2)) val first_record = result(0) val second_record = result(1) assert(first_record != null) assert(second_record != null) - assert(first_record.size === 3) - assert(second_record(1) === null) - assert(second_record(2) === null) - assert(second_record(0) === "A. Nonymous") - assert(first_record(0) === "Julien Le Dem") + assert(EQ(first_record.size).===(3)) + assert(EQ(second_record(1)).===(null)) + assert(EQ(second_record(2)).===(null)) + assert(EQ(second_record(0)).===("A. Nonymous")) + assert(EQ(first_record(0)).===("Julien Le Dem")) val first_owner_numbers = first_record(1) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] val first_contacts = first_record(2) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] assert(first_owner_numbers != null) - assert(first_owner_numbers(0) === "555 123 4567") - assert(first_owner_numbers(2) === "XXX XXX XXXX") - assert(first_contacts(0) - .asInstanceOf[CatalystConverter.StructScalaType[_]].size === 2) + assert(EQ(first_owner_numbers(0)).===("555 123 4567")) + assert(EQ(first_owner_numbers(2)).===("XXX XXX XXXX")) + assert(EQ(first_contacts(0) + .asInstanceOf[CatalystConverter.StructScalaType[_]].size).===(2)) val first_contacts_entry_one = first_contacts(0) .asInstanceOf[CatalystConverter.StructScalaType[_]] - assert(first_contacts_entry_one(0) === "Dmitriy Ryaboy") - assert(first_contacts_entry_one(1) === "555 987 6543") + assert(EQ(first_contacts_entry_one(0)).===("Dmitriy Ryaboy")) + assert(EQ(first_contacts_entry_one(1)).===("555 987 6543")) val first_contacts_entry_two = first_contacts(1) .asInstanceOf[CatalystConverter.StructScalaType[_]] - assert(first_contacts_entry_two(0) === "Chris Aniszczyk") + assert(EQ(first_contacts_entry_two(0)).===("Chris Aniszczyk")) } test("Importing nested Parquet file (nested numbers)") { @@ -705,31 +712,31 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .parquetFile(ParquetTestData.testNestedDir2.toString) .toSchemaRDD .collect() - assert(result.size === 1, "number of top-level rows incorrect") - assert(result(0).size === 5, "number of fields in row incorrect") - assert(result(0)(0) === 1) - assert(result(0)(1) === 7) + assert(EQ(result.size).===(1), "number of top-level rows incorrect") + assert(EQ(result(0).size).===(5), "number of fields in row incorrect") + assert(EQ(result(0)(0)).===(1)) + assert(EQ(result(0)(1)).===(7)) val subresult1 = result(0)(2).asInstanceOf[CatalystConverter.ArrayScalaType[_]] - assert(subresult1.size === 3) - assert(subresult1(0) === (1.toLong << 32)) - assert(subresult1(1) === (1.toLong << 33)) - assert(subresult1(2) === (1.toLong << 34)) + assert(EQ(subresult1.size).===(3)) + assert(EQ(subresult1(0)).===((1.toLong << 32))) + assert(EQ(subresult1(1)).===((1.toLong << 33))) + assert(EQ(subresult1(2)).===((1.toLong << 34))) val subresult2 = result(0)(3) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) .asInstanceOf[CatalystConverter.StructScalaType[_]] - assert(subresult2.size === 2) - assert(subresult2(0) === 2.5) - assert(subresult2(1) === false) + assert(EQ(subresult2.size).===(2)) + assert(EQ(subresult2(0)).===(2.5)) + assert(EQ(subresult2(1)).===(false)) val subresult3 = result(0)(4) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] - assert(subresult3.size === 2) - assert(subresult3(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]].size === 2) + assert(EQ(subresult3.size).===(2)) + assert(EQ(subresult3(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]].size).===(2)) val subresult4 = subresult3(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]] - assert(subresult4(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 7) - assert(subresult4(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 8) - assert(subresult3(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]].size === 1) - assert(subresult3(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) - .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 9) + assert(EQ(subresult4(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(7)) + assert(EQ(subresult4(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(8)) + assert(EQ(subresult3(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]].size).===(1)) + assert(EQ(subresult3(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(9)) } test("Simple query on addressbook") { @@ -737,8 +744,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .parquetFile(ParquetTestData.testNestedDir1.toString) .toSchemaRDD val tmp = data.where('owner === "Julien Le Dem").select('owner as 'a, 'contacts as 'c).collect() - assert(tmp.size === 1) - assert(tmp(0)(0) === "Julien Le Dem") + assert(EQ(tmp.size).===(1)) + assert(EQ(tmp(0)(0)).===("Julien Le Dem")) } test("Projection in addressbook") { @@ -746,37 +753,37 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA data.registerTempTable("data") val query = sql("SELECT owner, contacts[1].name FROM data") val tmp = query.collect() - assert(tmp.size === 2) - assert(tmp(0).size === 2) - assert(tmp(0)(0) === "Julien Le Dem") - assert(tmp(0)(1) === "Chris Aniszczyk") - assert(tmp(1)(0) === "A. Nonymous") - assert(tmp(1)(1) === null) + assert(EQ(tmp.size).===(2)) + assert(EQ(tmp(0).size).===(2)) + assert(EQ(tmp(0)(0)).===("Julien Le Dem")) + assert(EQ(tmp(0)(1)).===("Chris Aniszczyk")) + assert(EQ(tmp(1)(0)).===("A. Nonymous")) + assert(EQ(tmp(1)(1)).===(null)) } test("Simple query on nested int data") { val data = parquetFile(ParquetTestData.testNestedDir2.toString).toSchemaRDD data.registerTempTable("data") val result1 = sql("SELECT entries[0].value FROM data").collect() - assert(result1.size === 1) - assert(result1(0).size === 1) - assert(result1(0)(0) === 2.5) + assert(EQ(result1.size).===(1)) + assert(EQ(result1(0).size).===(1)) + assert(EQ(result1(0)(0)).===(2.5)) val result2 = sql("SELECT entries[0] FROM data").collect() - assert(result2.size === 1) + assert(EQ(result2.size).===(1)) val subresult1 = result2(0)(0).asInstanceOf[CatalystConverter.StructScalaType[_]] - assert(subresult1.size === 2) - assert(subresult1(0) === 2.5) - assert(subresult1(1) === false) + assert(EQ(subresult1.size).===(2)) + assert(EQ(subresult1(0)).===(2.5)) + assert(EQ(subresult1(1)).===(false)) val result3 = sql("SELECT outerouter FROM data").collect() val subresult2 = result3(0)(0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] - assert(subresult2(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 7) - assert(subresult2(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 8) - assert(result3(0)(0) + assert(EQ(subresult2(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(7)) + assert(EQ(subresult2(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(8)) + assert(EQ(result3(0)(0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](1) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) - .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 9) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(9)) } test("nested structs") { @@ -784,17 +791,17 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .toSchemaRDD data.registerTempTable("data") val result1 = sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() - assert(result1.size === 1) - assert(result1(0).size === 1) - assert(result1(0)(0) === false) + assert(EQ(result1.size).===(1)) + assert(EQ(result1(0).size).===(1)) + assert(EQ(result1(0)(0)).===(false)) val result2 = sql("SELECT booleanNumberPairs[0].value[1].truth FROM data").collect() - assert(result2.size === 1) - assert(result2(0).size === 1) - assert(result2(0)(0) === true) + assert(EQ(result2.size).===(1)) + assert(EQ(result2(0).size).===(1)) + assert(EQ(result2(0)(0)).===(true)) val result3 = sql("SELECT booleanNumberPairs[1].value[0].truth FROM data").collect() - assert(result3.size === 1) - assert(result3(0).size === 1) - assert(result3(0)(0) === false) + assert(EQ(result3.size).===(1)) + assert(EQ(result3(0).size).===(1)) + assert(EQ(result3(0)(0)).===(false)) } test("simple map") { @@ -803,38 +810,38 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .toSchemaRDD data.registerTempTable("mapTable") val result1 = sql("SELECT data1 FROM mapTable").collect() - assert(result1.size === 1) - assert(result1(0)(0) + assert(EQ(result1.size).===(1)) + assert(EQ(result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, _]] - .getOrElse("key1", 0) === 1) - assert(result1(0)(0) + .getOrElse("key1", 0)).===(1)) + assert(EQ(result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, _]] - .getOrElse("key2", 0) === 2) + .getOrElse("key2", 0)).===(2)) val result2 = sql("""SELECT data1["key1"] FROM mapTable""").collect() - assert(result2(0)(0) === 1) + assert(EQ(result2(0)(0)).===(1)) } test("map with struct values") { val data = parquetFile(ParquetTestData.testNestedDir4.toString).toSchemaRDD data.registerTempTable("mapTable") val result1 = sql("SELECT data2 FROM mapTable").collect() - assert(result1.size === 1) + assert(EQ(result1.size).===(1)) val entry1 = result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] .getOrElse("seven", null) assert(entry1 != null) - assert(entry1(0) === 42) - assert(entry1(1) === "the answer") + assert(EQ(entry1(0)).===(42)) + assert(EQ(entry1(1)).===("the answer")) val entry2 = result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] .getOrElse("eight", null) assert(entry2 != null) - assert(entry2(0) === 49) - assert(entry2(1) === null) + assert(EQ(entry2(0)).===(49)) + assert(EQ(entry2(1)).===(null)) val result2 = sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM mapTable""").collect() - assert(result2.size === 1) - assert(result2(0)(0) === 42.toLong) - assert(result2(0)(1) === "the answer") + assert(EQ(result2.size).===(1)) + assert(EQ(result2(0)(0)).===(42.toLong)) + assert(EQ(result2(0)(1)).===("the answer")) } test("Writing out Addressbook and reading it back in") { @@ -848,12 +855,12 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .toSchemaRDD .registerTempTable("tmpcopy") val tmpdata = sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() - assert(tmpdata.size === 2) - assert(tmpdata(0).size === 2) - assert(tmpdata(0)(0) === "Julien Le Dem") - assert(tmpdata(0)(1) === "Chris Aniszczyk") - assert(tmpdata(1)(0) === "A. Nonymous") - assert(tmpdata(1)(1) === null) + assert(EQ(tmpdata.size).===(2)) + assert(EQ(tmpdata(0).size).===(2)) + assert(EQ(tmpdata(0)(0)).===("Julien Le Dem")) + assert(EQ(tmpdata(0)(1)).===("Chris Aniszczyk")) + assert(EQ(tmpdata(1)(0)).===("A. Nonymous")) + assert(EQ(tmpdata(1)(1)).===(null)) Utils.deleteRecursively(tmpdir) } @@ -866,26 +873,26 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .toSchemaRDD .registerTempTable("tmpmapcopy") val result1 = sql("""SELECT data1["key2"] FROM tmpmapcopy""").collect() - assert(result1.size === 1) - assert(result1(0)(0) === 2) + assert(EQ(result1.size).===(1)) + assert(EQ(result1(0)(0)).===(2)) val result2 = sql("SELECT data2 FROM tmpmapcopy").collect() - assert(result2.size === 1) + assert(EQ(result2.size).===(1)) val entry1 = result2(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] .getOrElse("seven", null) assert(entry1 != null) - assert(entry1(0) === 42) - assert(entry1(1) === "the answer") + assert(EQ(entry1(0)).===(42)) + assert(EQ(entry1(1)).===("the answer")) val entry2 = result2(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] .getOrElse("eight", null) assert(entry2 != null) - assert(entry2(0) === 49) - assert(entry2(1) === null) + assert(EQ(entry2(0)).===(49)) + assert(EQ(entry2(1)).===(null)) val result3 = sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM tmpmapcopy""").collect() - assert(result3.size === 1) - assert(result3(0)(0) === 42.toLong) - assert(result3(0)(1) === "the answer") + assert(EQ(result3.size).===(1)) + assert(EQ(result3(0)(0)).===(42.toLong)) + assert(EQ(result3(0)(1)).===("the answer")) Utils.deleteRecursively(tmpdir) } @@ -894,7 +901,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(tmpdir) createParquetFile[TestRDDEntry](tmpdir.toString()).registerTempTable("tmpemptytable") val result1 = sql("SELECT * FROM tmpemptytable").collect() - assert(result1.size === 0) + assert(EQ(result1.size).===(0)) Utils.deleteRecursively(tmpdir) } @@ -908,7 +915,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA (fromCaseClassString, fromJson).zipped.foreach { (a, b) => assert(a.name == b.name) - assert(a.dataType === b.dataType) + assert(EQ(a.dataType).===(b.dataType)) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index a90fc023e67d..3565b377ef60 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -27,6 +27,13 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + class StatisticsSuite extends QueryTest with BeforeAndAfterAll { TestHive.reset() TestHive.cacheTables = false @@ -39,7 +46,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { case o => o } - assert(operators.size === 1) + assert(EQ(operators.size).===(1)) if (operators(0).getClass() != c) { fail( s"""$analyzeCommand expected command: $c, but got ${operators(0)} @@ -81,11 +88,11 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { // TODO: How does it works? needs to add it back for other hive version. if (HiveShim.version =="0.12.0") { - assert(queryTotalSize("analyzeTable") === defaultSizeInBytes) + assert(EQ(queryTotalSize("analyzeTable")).===(defaultSizeInBytes)) } sql("ANALYZE TABLE analyzeTable COMPUTE STATISTICS noscan") - assert(queryTotalSize("analyzeTable") === BigInt(11624)) + assert(EQ(queryTotalSize("analyzeTable")).===(BigInt(11624))) sql("DROP TABLE analyzeTable").collect() @@ -110,11 +117,11 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { |SELECT * FROM src """.stripMargin).collect() - assert(queryTotalSize("analyzeTable_part") === defaultSizeInBytes) + assert(EQ(queryTotalSize("analyzeTable_part")).===(defaultSizeInBytes)) sql("ANALYZE TABLE analyzeTable_part COMPUTE STATISTICS noscan") - assert(queryTotalSize("analyzeTable_part") === BigInt(17436)) + assert(EQ(queryTotalSize("analyzeTable_part")).===(BigInt(17436))) sql("DROP TABLE analyzeTable_part").collect() @@ -131,7 +138,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { val sizes = rdd.queryExecution.analyzed.collect { case mr: MetastoreRelation => mr.statistics.sizeInBytes } - assert(sizes.size === 1, s"Size wrong for:\n ${rdd.queryExecution}") + assert(EQ(sizes.size).===(1), s"Size wrong for:\n ${rdd.queryExecution}") assert(sizes(0).equals(BigInt(5812)), s"expected exact size 5812 for test table 'src', got: ${sizes(0)}") } @@ -151,14 +158,14 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { val sizes = rdd.queryExecution.analyzed.collect { case r if ct.runtimeClass.isAssignableFrom(r.getClass) => r.statistics.sizeInBytes } - assert(sizes.size === 2 && sizes(0) <= autoBroadcastJoinThreshold + assert(EQ(sizes.size).===(2) && sizes(0) <= autoBroadcastJoinThreshold && sizes(1) <= autoBroadcastJoinThreshold, s"query should contain two relations, each of which has size smaller than autoConvertSize") // Using `sparkPlan` because for relevant patterns in HashJoin to be // matched, other strategies need to be applied. var bhj = rdd.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } - assert(bhj.size === 1, + assert(EQ(bhj.size).===(1), s"actual query plans do not contain broadcast join: ${rdd.queryExecution}") checkAnswer(rdd, expectedAnswer) // check correctness of output @@ -172,7 +179,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") val shj = rdd.queryExecution.sparkPlan.collect { case j: ShuffledHashJoin => j } - assert(shj.size === 1, + assert(EQ(shj.size).===(1), "ShuffledHashJoin should be planned when BroadcastHashJoin is turned off") sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp""") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index b897dff0159f..3e9805dfe806 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -30,6 +30,13 @@ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.{Row, SchemaRDD} +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ +import org.scalatest.Assertions.{convertToEqualizer => EQ} + case class TestData(a: Int, b: String) /** @@ -139,7 +146,7 @@ class HiveQuerySuite extends HiveComparisonTest { test("CREATE TABLE AS runs once") { sql("CREATE TABLE foo AS SELECT 1 FROM src LIMIT 1").collect() - assert(sql("SELECT COUNT(*) FROM foo").collect().head.getLong(0) === 1, + assert(EQ(sql("SELECT COUNT(*) FROM foo").collect().head.getLong(0)).===(1), "Incorrect number of rows in created table") } @@ -161,7 +168,7 @@ class HiveQuerySuite extends HiveComparisonTest { test("Query expressed in SQL") { setConf("spark.sql.dialect", "sql") - assert(sql("SELECT 1").collect() === Array(Seq(1))) + assert(EQ(sql("SELECT 1").collect()).===(Array(Seq(1)))) setConf("spark.sql.dialect", "hiveql") } @@ -365,7 +372,7 @@ class HiveQuerySuite extends HiveComparisonTest { .collect() .toSet - assert(actual === expected) + assert(EQ(actual).===(expected)) } // TODO: adopt this test when Spark SQL has the functionality / framework to report errors. @@ -415,7 +422,7 @@ class HiveQuerySuite extends HiveComparisonTest { .collect() .map(x => Pair(x.getString(0), x.getInt(1))) - assert(results === Array(Pair("foo", 4))) + assert(EQ(results).===(Array(Pair("foo", 4)))) TestHive.reset() } @@ -557,8 +564,8 @@ class HiveQuerySuite extends HiveComparisonTest { sql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10") sql("SELECT * FROM m").collect().zip(sql("SELECT * FROM src LIMIT 10").collect()).map { case (Row(map: Map[_, _]), Row(key: Int, value: String)) => - assert(map.size === 1) - assert(map.head === (key, value)) + assert(EQ(map.size).===(1)) + assert(EQ(map.head).===((key, value))) } } @@ -654,7 +661,7 @@ class HiveQuerySuite extends HiveComparisonTest { sql("CREATE TABLE dp_verify(intcol INT)") sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE dp_verify") - assert(sql("SELECT * FROM dp_verify").collect() === Array(Row(value))) + assert(EQ(sql("SELECT * FROM dp_verify").collect()).===(Array(Row(value)))) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 6f57fe895838..acefbe66dbd9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -29,6 +29,13 @@ import org.apache.spark.util.Utils // Implicits import org.apache.spark.sql.hive.test.TestHive._ +/* + * Note: the DSL conversions collide with the scalatest === operator! + * We can apply the scalatest conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + * (This file already imports convertToEqualizer) + */ + case class Cases(lower: String, UPPER: String) class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { @@ -80,7 +87,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft test("Simple column projection + filter on Parquet table") { val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() - assert(rdd.size === 5, "Filter returned incorrect number of rows") + assert(convertToEqualizer(rdd.size).===(5), "Filter returned incorrect number of rows") assert(rdd.forall(_.getBoolean(0)), "Filter returned incorrect Boolean field value") } @@ -102,7 +109,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() val rddCopy = sql("SELECT * FROM ptable").collect() val rddOrig = sql("SELECT * FROM testsource").collect() - assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of table??") + assert(convertToEqualizer(rddCopy.size).===(rddOrig.size), "INSERT OVERWRITE changed size of table??") compareRDDs(rddOrig, rddCopy, "testsource", ParquetTestData.testSchemaFieldNames) } @@ -111,7 +118,8 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft (rddOne, rddTwo).zipped.foreach { (a,b) => (a,b).zipped.toArray.zipWithIndex.foreach { case ((value_1, value_2), index) => - assert(value_1 === value_2, s"table $tableName row $counter field ${fieldNames(index)} don't match") + assert(convertToEqualizer(value_1).===(value_2), + s"table $tableName row $counter field ${fieldNames(index)} don't match") } counter = counter + 1 } From 492feefaea6531b3374bf3e753e61234b79e1184 Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Fri, 7 Nov 2014 10:38:18 -0600 Subject: [PATCH 03/16] Clarification of one comment. --- .../scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index acefbe66dbd9..0069726426f9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.hive.test.TestHive._ /* * Note: the DSL conversions collide with the scalatest === operator! * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) + * assert(X === Y) --> assert(convertToEqualizer(X).===(Y)) * (This file already imports convertToEqualizer) */ From 53aced3bd6f7dba5d124a7070283e95e05b61914 Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Sat, 8 Nov 2014 13:46:10 -0600 Subject: [PATCH 04/16] Fixed implicit conversions for RichDate and RichTimestamp and added a test which would have detected the problem. --- .../org/apache/spark/sql/catalyst/types/timetypes.scala | 4 ++-- .../catalyst/expressions/ExpressionEvaluationSuite.scala | 9 +++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala index fcb77e640a8e..7be589727e0a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala @@ -79,11 +79,11 @@ object RichTimestamp { object TimeConversions { implicit def javaDateToRichDate(jdate: Date): RichDate = { - new Date(jdate.getTime) + new RichDate(jdate.getTime) } implicit def javaTimestampToRichTimestamp(jtimestamp: Timestamp): RichTimestamp = { - new Timestamp(jtimestamp.getTime) + new RichTimestamp(jtimestamp.getTime) } implicit def richDateToJavaDate(date: RichDate): Date = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 085f6a6cae70..8d3dc302c3b0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} +import TimeConversions._ import scala.collection.immutable.HashSet @@ -802,4 +803,12 @@ class ExpressionEvaluationSuite extends FunSuite { ).===(false)) } + test("implicit conversions for RichDate and RichTimestamp") { + val d1 = RichDate("2014-01-01") + val d2 = javaDateToRichDate(richDateToJavaDate(d1)) + assert(EQ(d1).===(d2)) + val t1 = RichTimestamp("2014-01-01 12:34:56.789") + val t2 = javaTimestampToRichTimestamp(richTimestampToJavaTimestamp(t1)) + assert(EQ(t1).===(t2)) + } } From bd539b6ce8a45e9a6d32b77fe55296a7a960d4bf Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Sat, 8 Nov 2014 15:00:14 -0600 Subject: [PATCH 05/16] Rebased and fixed the assert commands in ParquetQuerySuite.scala. --- .../org/apache/spark/sql/parquet/ParquetQuerySuite.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 5bede8a05484..d4ac64a8c8f9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -632,7 +632,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query12.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result12 = query12.collect() - assert(result12.size === 54) + assert(EQ(result12.size).===(54)) assert(result12(0).getString(2) == "6") assert(result12(4).getString(2) == "50") assert(result12(53).getString(2) == "99") @@ -642,7 +642,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query13.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result13 = query13.collect() - assert(result13.size === 53) + assert(EQ(result13.size).===(53)) assert(result13(0).getString(2) == "6") assert(result13(4).getString(2) == "51") assert(result13(52).getString(2) == "99") @@ -652,7 +652,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query14.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result14 = query14.collect() - assert(result14.size === 148) + assert(EQ(result14.size).===(148)) assert(result14(0).getString(2) == "0") assert(result14(46).getString(2) == "50") assert(result14(147).getString(2) == "200") @@ -662,7 +662,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query15.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result15 = query15.collect() - assert(result15.size === 147) + assert(EQ(result15.size).===(147)) assert(result15(0).getString(2) == "0") assert(result15(46).getString(2) == "100") assert(result15(146).getString(2) == "200") From 353d12ca82eea8b097076adcf5b315641bbb81b9 Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Sun, 9 Nov 2014 10:54:10 -0600 Subject: [PATCH 06/16] Add unapply methods to RichDate and RichTimestamp and make their types available after importing the members of an SQLContext. --- .../spark/sql/catalyst/types/timetypes.scala | 31 +++++++++++-------- .../org/apache/spark/sql/SQLContext.scala | 11 +++++-- 2 files changed, 26 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala index 7be589727e0a..e4a6dc0ae3ed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import scala.language.implicitConversions -/* +/* * * Subclass of java.sql.Date which provides the usual comparison * operators (as required for catalyst expressions) and which can * be constructed from a string. @@ -36,18 +36,20 @@ import scala.language.implicitConversions */ class RichDate(milliseconds: Long) extends Date(milliseconds) { - def <(that: Date): Boolean = this.before(that) - def >(that: Date): Boolean = this.after(that) - def <=(that: Date): Boolean = (this.before(that) || this.equals(that)) - def >=(that: Date): Boolean = (this.after(that) || this.equals(that)) - def ===(that: Date): Boolean = this.equals(that) + def < (that: Date): Boolean = this.before(that) + def > (that: Date): Boolean = this.after(that) + def <= (that: Date): Boolean = (this.before(that) || this.equals(that)) + def >= (that: Date): Boolean = (this.after(that) || this.equals(that)) + def === (that: Date): Boolean = this.equals(that) } object RichDate { def apply(init: String) = new RichDate(Date.valueOf(init).getTime) + + def unapply(richdate: RichDate): Option[Date] = Some(new Date(richdate.getTime)) } -/* +/* * * Analogous subclass of java.sql.Timestamp. * * scala> val ts1 = Timestamp("2014-03-04 12:34:56.12") @@ -61,18 +63,21 @@ object RichDate { */ class RichTimestamp(milliseconds: Long) extends Timestamp(milliseconds) { - def <(that: Timestamp): Boolean = this.before(that) - def >(that: Timestamp): Boolean = this.after(that) - def <=(that: Timestamp): Boolean = (this.before(that) || this.equals(that)) - def >=(that: Timestamp): Boolean = (this.after(that) || this.equals(that)) - def ===(that: Timestamp): Boolean = this.equals(that) + def < (that: Timestamp): Boolean = this.before(that) + def > (that: Timestamp): Boolean = this.after(that) + def <= (that: Timestamp): Boolean = (this.before(that) || this.equals(that)) + def >= (that: Timestamp): Boolean = (this.after(that) || this.equals(that)) + def === (that: Timestamp): Boolean = this.equals(that) } object RichTimestamp { def apply(init: String) = new RichTimestamp(Timestamp.valueOf(init).getTime) + + def unapply(richtimestamp: RichTimestamp): Option[Timestamp] = + Some(new Timestamp(richtimestamp.getTime)) } -/* +/* * * Implicit conversions. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 98370dcf8171..9265660897da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -504,8 +504,13 @@ class SQLContext(@transient val sparkContext: SparkContext) } /* * - * Map RichDate and RichTimestamp to their expected names in this context. + * Make RichDate and RichTimestamp available under the names + * Date and Timestamp when the members of this SQLContext are + * imported. */ - val Date = org.apache.spark.sql.catalyst.expressions.RichDate - val Timestamp = org.apache.spark.sql.catalyst.expressions.RichTimestamp + import org.apache.spark.sql.catalyst.expressions._ + val Date = RichDate + type Date = RichDate + val Timestamp = RichTimestamp + type Timestamp = RichTimestamp } From eff68fe04ac699ad180fd7a96b71d6b14159aba0 Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Sun, 9 Nov 2014 21:28:25 -0600 Subject: [PATCH 07/16] Modified RichTimestamp.toString to use the Hive format. --- .../apache/spark/sql/catalyst/types/timetypes.scala | 13 +++++++++++++ 1 file changed, 13 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala index e4a6dc0ae3ed..61d8793604b6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala @@ -68,6 +68,19 @@ class RichTimestamp(milliseconds: Long) extends Timestamp(milliseconds) { def <= (that: Timestamp): Boolean = (this.before(that) || this.equals(that)) def >= (that: Timestamp): Boolean = (this.after(that) || this.equals(that)) def === (that: Timestamp): Boolean = this.equals(that) + // Follow Hive conventions when converting to a String. + // Copied from the Cast class. + override def toString(): String = { + val jts = new Timestamp(this.getTime) + val timestampString = jts.toString + val formatted = Cast.threadLocalTimestampFormat.get.format(jts) + + if (timestampString.length > 19 && timestampString.substring(19) != ".0") { + formatted + timestampString.substring(19) + } else { + formatted + } + } } object RichTimestamp { From 759e883ed1eab286ef6b437ec8c194bc8cced50b Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Mon, 10 Nov 2014 07:56:16 -0600 Subject: [PATCH 08/16] Rebasing and making comments consistent with ScalaDocs. --- .../spark/sql/catalyst/types/timetypes.scala | 30 +++++++++++-------- 1 file changed, 17 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala index 61d8793604b6..d5dd80eadee0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala @@ -25,12 +25,14 @@ import scala.language.implicitConversions * operators (as required for catalyst expressions) and which can * be constructed from a string. * - * scala> val d1 = Date("2014-02-01") - * d1: Date = 2014-02-01 - * - * scala> val d2 = Date("2014-02-02") - * d2: Date = 2014-02-02 + * {{{ + * scala> val d1 = Date("2014-02-01") + * d1: Date = 2014-02-01 * + * scala> val d2 = Date("2014-02-02") + * d2: Date = 2014-02-02 + * }}} + * * scala> d1 < d2 * res1: Boolean = true */ @@ -49,17 +51,19 @@ object RichDate { def unapply(richdate: RichDate): Option[Date] = Some(new Date(richdate.getTime)) } -/* * +/** * Analogous subclass of java.sql.Timestamp. * - * scala> val ts1 = Timestamp("2014-03-04 12:34:56.12") - * ts1: Timestamp = 2014-03-04 12:34:56.12 + * {{{ + * scala> val ts1 = Timestamp("2014-03-04 12:34:56.12") + * ts1: Timestamp = 2014-03-04 12:34:56.12 * - * scala> val ts2 = Timestamp("2014-03-04 12:34:56.13") - * ts2: Timestamp = 2014-03-04 12:34:56.13 + * scala> val ts2 = Timestamp("2014-03-04 12:34:56.13") + * ts2: Timestamp = 2014-03-04 12:34:56.13 * - * scala> ts1 < ts2 - * res13: Boolean = true + * scala> ts1 < ts2 + * res13: Boolean = true + * }}} */ class RichTimestamp(milliseconds: Long) extends Timestamp(milliseconds) { @@ -90,7 +94,7 @@ object RichTimestamp { Some(new Timestamp(richtimestamp.getTime)) } -/* * +/** * Implicit conversions. */ From 0c86b93e350bb3927f07907b256418036789c038 Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Mon, 10 Nov 2014 09:05:41 -0600 Subject: [PATCH 09/16] Removing all changes to SpecificMutableRow.scala. Mutable Dates and Timestamps are not useful. --- .../expressions/SpecificMutableRow.scala | 30 ------------------- 1 file changed, 30 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala index 9f977bf6c2a0..570379c533e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ -import java.sql.{Date, Timestamp} /** * A parent class for mutable container objects that are reused when the values are changed, @@ -170,35 +169,6 @@ final class MutableByte extends MutableValue { newCopy.asInstanceOf[this.type] } } -final class MutableDate extends MutableValue { - var value: Date = new Date(0) - def boxed = if (isNull) null else value - def update(v: Any) = value = { - isNull = false - v.asInstanceOf[Date] - } - def copy() = { - val newCopy = new MutableDate - newCopy.isNull = isNull - newCopy.value = value - newCopy.asInstanceOf[this.type] - } -} - -final class MutableTimestamp extends MutableValue { - var value: Timestamp = new Timestamp(0) - def boxed = if (isNull) null else value - def update(v: Any) = value = { - isNull = false - v.asInstanceOf[Timestamp] - } - def copy() = { - val newCopy = new MutableTimestamp - newCopy.isNull = isNull - newCopy.value = value - newCopy.asInstanceOf[this.type] - } -} final class MutableAny extends MutableValue { var value: Any = _ From 1414f302fe5587dba5c2dec4f33f7001223584ce Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Mon, 10 Nov 2014 17:31:19 -0600 Subject: [PATCH 10/16] New DSL equals operator -=-. Restored all test files as this does not collide with scalatests === operator. --- .../spark/sql/catalyst/dsl/package.scala | 3 +- .../ExpressionEvaluationSuite.scala | 118 +++---- .../sql/catalyst/trees/TreeNodeSuite.scala | 25 +- .../apache/spark/sql/CachedTableSuite.scala | 11 +- .../org/apache/spark/sql/DslQuerySuite.scala | 12 +- .../org/apache/spark/sql/JoinSuite.scala | 13 +- .../org/apache/spark/sql/SQLConfSuite.scala | 9 +- .../scala/org/apache/spark/sql/UDFSuite.scala | 11 +- .../spark/sql/UserDefinedTypeSuite.scala | 11 +- .../columnar/PartitionBatchPruningSuite.scala | 11 +- .../spark/sql/execution/PlannerSuite.scala | 17 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 319 +++++++++--------- .../spark/sql/hive/StatisticsSuite.scala | 25 +- .../sql/hive/execution/HiveQuerySuite.scala | 21 +- 14 files changed, 262 insertions(+), 344 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 84fb594da372..c9f4daf16b9d 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -84,6 +84,7 @@ package object dsl { def > (other: Expression) = GreaterThan(expr, other) def >= (other: Expression) = GreaterThanOrEqual(expr, other) def === (other: Expression) = EqualTo(expr, other) + def -=- (other: Expression) = EqualTo(expr, other) def <=> (other: Expression) = EqualNullSafe(expr, other) def !== (other: Expression) = Not(EqualTo(expr, other)) @@ -168,7 +169,7 @@ package object dsl { def <= (other: Symbol) = LessThanOrEqual(literal, other) def > (other: Symbol) = GreaterThan(literal, other) def >= (other: Symbol) = GreaterThanOrEqual(literal, other) - def === (other: Symbol) = EqualTo(literal, other) + def -=- (other: Symbol) = EqualTo(literal, other) def <=> (other: Symbol) = EqualNullSafe(literal, other) def !== (other: Symbol) = Not(EqualTo(literal, other)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 8d3dc302c3b0..a1a1e8296964 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} -import TimeConversions._ import scala.collection.immutable.HashSet @@ -33,13 +32,6 @@ import org.apache.spark.sql.catalyst.types._ /* Implicit conversions */ import org.apache.spark.sql.catalyst.dsl.expressions._ -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - class ExpressionEvaluationSuite extends FunSuite { test("literals") { @@ -326,18 +318,18 @@ class ExpressionEvaluationSuite extends FunSuite { intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} - assert(EQ(("abcdef" cast StringType).nullable).===(false)) - assert(EQ(("abcdef" cast BinaryType).nullable).===(false)) - assert(EQ(("abcdef" cast BooleanType).nullable).===(false)) - assert(EQ(("abcdef" cast TimestampType).nullable).===(true)) - assert(EQ(("abcdef" cast LongType).nullable).===(true)) - assert(EQ(("abcdef" cast IntegerType).nullable).===(true)) - assert(EQ(("abcdef" cast ShortType).nullable).===(true)) - assert(EQ(("abcdef" cast ByteType).nullable).===(true)) - assert(EQ(("abcdef" cast DecimalType.Unlimited).nullable).===(true)) - assert(EQ(("abcdef" cast DecimalType(4, 2)).nullable).===(true)) - assert(EQ(("abcdef" cast DoubleType).nullable).===(true)) - assert(EQ(("abcdef" cast FloatType).nullable).===(true)) + assert(("abcdef" cast StringType).nullable === false) + assert(("abcdef" cast BinaryType).nullable === false) + assert(("abcdef" cast BooleanType).nullable === false) + assert(("abcdef" cast TimestampType).nullable === true) + assert(("abcdef" cast LongType).nullable === true) + assert(("abcdef" cast IntegerType).nullable === true) + assert(("abcdef" cast ShortType).nullable === true) + assert(("abcdef" cast ByteType).nullable === true) + assert(("abcdef" cast DecimalType.Unlimited).nullable === true) + assert(("abcdef" cast DecimalType(4, 2)).nullable === true) + assert(("abcdef" cast DoubleType).nullable === true) + assert(("abcdef" cast FloatType).nullable === true) checkEvaluation(Cast(Literal(null, IntegerType), ShortType), null) } @@ -354,15 +346,15 @@ class ExpressionEvaluationSuite extends FunSuite { // - Values that would overflow the target precision should turn into null // - Because of this, casts to fixed-precision decimals should be nullable - assert(EQ(Cast(Literal(123), DecimalType.Unlimited).nullable).===(false)) - assert(EQ(Cast(Literal(10.03f), DecimalType.Unlimited).nullable).===(false)) - assert(EQ(Cast(Literal(10.03), DecimalType.Unlimited).nullable).===(false)) - assert(EQ(Cast(Literal(Decimal(10.03)), DecimalType.Unlimited).nullable).===(false)) + assert(Cast(Literal(123), DecimalType.Unlimited).nullable === false) + assert(Cast(Literal(10.03f), DecimalType.Unlimited).nullable === false) + assert(Cast(Literal(10.03), DecimalType.Unlimited).nullable === false) + assert(Cast(Literal(Decimal(10.03)), DecimalType.Unlimited).nullable === false) - assert(EQ(Cast(Literal(123), DecimalType(2, 1)).nullable).===(true)) - assert(EQ(Cast(Literal(10.03f), DecimalType(2, 1)).nullable).===(true)) - assert(EQ(Cast(Literal(10.03), DecimalType(2, 1)).nullable).===(true)) - assert(EQ(Cast(Literal(Decimal(10.03)), DecimalType(2, 1)).nullable).===(true)) + assert(Cast(Literal(123), DecimalType(2, 1)).nullable === true) + assert(Cast(Literal(10.03f), DecimalType(2, 1)).nullable === true) + assert(Cast(Literal(10.03), DecimalType(2, 1)).nullable === true) + assert(Cast(Literal(Decimal(10.03)), DecimalType(2, 1)).nullable === true) checkEvaluation(Cast(Literal(123), DecimalType.Unlimited), Decimal(123)) checkEvaluation(Cast(Literal(123), DecimalType(3, 0)), Decimal(123)) @@ -510,26 +502,26 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5, c6)), "c", row) checkEvaluation(CaseWhen(Seq(c1, c4, c2, c5)), null, row) - assert(EQ(CaseWhen(Seq(c2, c4, c6)).nullable).===(true)) - assert(EQ(CaseWhen(Seq(c2, c4, c3, c5, c6)).nullable).===(true)) - assert(EQ(CaseWhen(Seq(c2, c4, c3, c5)).nullable).===(true)) + assert(CaseWhen(Seq(c2, c4, c6)).nullable === true) + assert(CaseWhen(Seq(c2, c4, c3, c5, c6)).nullable === true) + assert(CaseWhen(Seq(c2, c4, c3, c5)).nullable === true) val c4_notNull = 'a.boolean.notNull.at(3) val c5_notNull = 'a.boolean.notNull.at(4) val c6_notNull = 'a.boolean.notNull.at(5) - assert(EQ(CaseWhen(Seq(c2, c4_notNull, c6_notNull)).nullable).===(false)) - assert(EQ(CaseWhen(Seq(c2, c4, c6_notNull)).nullable).===(true)) - assert(EQ(CaseWhen(Seq(c2, c4_notNull, c6)).nullable).===(true)) + assert(CaseWhen(Seq(c2, c4_notNull, c6_notNull)).nullable === false) + assert(CaseWhen(Seq(c2, c4, c6_notNull)).nullable === true) + assert(CaseWhen(Seq(c2, c4_notNull, c6)).nullable === true) - assert(EQ(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6_notNull)).nullable).===(false)) - assert(EQ(CaseWhen(Seq(c2, c4, c3, c5_notNull, c6_notNull)).nullable).===(true)) - assert(EQ(CaseWhen(Seq(c2, c4_notNull, c3, c5, c6_notNull)).nullable).===(true)) - assert(EQ(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6)).nullable).===(true)) + assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6_notNull)).nullable === false) + assert(CaseWhen(Seq(c2, c4, c3, c5_notNull, c6_notNull)).nullable === true) + assert(CaseWhen(Seq(c2, c4_notNull, c3, c5, c6_notNull)).nullable === true) + assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull, c6)).nullable === true) - assert(EQ(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull)).nullable).===(true)) - assert(EQ(CaseWhen(Seq(c2, c4, c3, c5_notNull)).nullable).===(true)) - assert(EQ(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable).===(true)) + assert(CaseWhen(Seq(c2, c4_notNull, c3, c5_notNull)).nullable === true) + assert(CaseWhen(Seq(c2, c4, c3, c5_notNull)).nullable === true) + assert(CaseWhen(Seq(c2, c4_notNull, c3, c5)).nullable === true) } test("complex type") { @@ -569,11 +561,11 @@ class ExpressionEvaluationSuite extends FunSuite { :: StructField("b", StringType, nullable = false) :: Nil ) - assert(EQ(GetField(BoundReference(2,typeS, nullable = true), "a").nullable).===(true)) - assert(EQ(GetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable).===(false)) + assert(GetField(BoundReference(2,typeS, nullable = true), "a").nullable === true) + assert(GetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable === false) - assert(EQ(GetField(Literal(null, typeS), "a").nullable).===(true)) - assert(EQ(GetField(Literal(null, typeS_notNullable), "a").nullable).===(true)) + assert(GetField(Literal(null, typeS), "a").nullable === true) + assert(GetField(Literal(null, typeS_notNullable), "a").nullable === true) checkEvaluation('c.map(typeMap).at(3).getItem("aa"), "bb", row) checkEvaluation('c.array(typeArray.elementType).at(4).getItem(1), "bb", row) @@ -727,10 +719,10 @@ class ExpressionEvaluationSuite extends FunSuite { val s_notNull = 'a.string.notNull.at(0) - assert(EQ(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)).nullable).===(true)) - assert(EQ(Substring(s_notNull, Literal(0, IntegerType), Literal(2, IntegerType)).nullable).===(false)) - assert(EQ(Substring(s_notNull, Literal(null, IntegerType), Literal(2, IntegerType)).nullable).===(true)) - assert(EQ(Substring(s_notNull, Literal(0, IntegerType), Literal(null, IntegerType)).nullable).===(true)) + assert(Substring(s, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, Literal(0, IntegerType), Literal(2, IntegerType)).nullable === false) + assert(Substring(s_notNull, Literal(null, IntegerType), Literal(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, Literal(0, IntegerType), Literal(null, IntegerType)).nullable === true) checkEvaluation(s.substr(0, 2), "ex", row) checkEvaluation(s.substr(0), "example", row) @@ -783,32 +775,42 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(~c1, -2, row) } + /* + * Testing the DSL conversions which allow literals on the left hand + * side of an expression. Note: the DSL conversions collide with + * the scalatest === operator. But we can apply the scalatest + * conversion explicitly: + * assert(X === Y) --> assert(EQ(X).===(Y)) + */ + import org.scalatest.Assertions.{convertToEqualizer => EQ} test("recognizes literals on the left") { assert(EQ(-1 + 'x).===(Add(-1, 'x))) assert(EQ(0 < 'x).===(LessThan(0, 'x))) - assert(EQ(1.5 === 'x).===(EqualTo(1.5, 'x))) + assert(EQ(1.5 -=- 'x).===(EqualTo(1.5, 'x))) assert(EQ(false !== 'x).===(Not(EqualTo(false, 'x)))) assert(EQ("a string" >= 'x).===(GreaterThanOrEqual("a string", 'x))) assert(EQ(RichDate("2014-11-05") > 'date).===(GreaterThan(RichDate("2014-11-05"), 'date))) - assert(EQ(RichTimestamp("2014-11-05 12:34:56.789") < 'now).===( - LessThan(RichTimestamp("2014-11-05 12:34:56.789"), 'now))) + assert(EQ(RichTimestamp("2014-11-05 12:34:56.789") < 'now) + .===(LessThan(RichTimestamp("2014-11-05 12:34:56.789"), 'now))) } test("comparison operators for RichDate and RichTimestamp") { assert(EQ(RichDate("2014-11-05") < RichDate("2014-11-06")).===(true)) assert(EQ(RichDate("2014-11-05") <= RichDate("2013-11-06")).===(false)) - assert(EQ(RichTimestamp("2014-11-05 12:34:56.5432") > RichTimestamp("2014-11-05 00:00:00") - ).===(true)) - assert(EQ(RichTimestamp("2014-11-05 12:34:56") >= RichTimestamp("2014-11-06 00:00:00") - ).===(false)) + assert(EQ(RichTimestamp("2014-11-05 12:34:56.5432") > RichTimestamp("2014-11-05 00:00:00")) + .===(true)) + assert(EQ(RichTimestamp("2014-11-05 12:34:56") >= RichTimestamp("2014-11-06 00:00:00")) + .===(false)) } test("implicit conversions for RichDate and RichTimestamp") { + import org.apache.spark.sql.catalyst.expressions.TimeConversions._ val d1 = RichDate("2014-01-01") val d2 = javaDateToRichDate(richDateToJavaDate(d1)) - assert(EQ(d1).===(d2)) + assert(d1 === d2 ) val t1 = RichTimestamp("2014-01-01 12:34:56.789") val t2 = javaTimestampToRichTimestamp(richTimestampToJavaTimestamp(t1)) - assert(EQ(t1).===(t2)) + assert(t1 === t2) } + } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 3586e6557aa1..036fd3fa1d6a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -24,13 +24,6 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{StringType, NullType} -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - case class Dummy(optKey: Option[Expression]) extends Expression { def children = optKey.toSeq def nullable = true @@ -43,21 +36,21 @@ case class Dummy(optKey: Option[Expression]) extends Expression { class TreeNodeSuite extends FunSuite { test("top node changed") { val after = Literal(1) transform { case Literal(1, _) => Literal(2) } - assert(EQ(after).===(Literal(2))) + assert(after === Literal(2)) } test("one child changed") { val before = Add(Literal(1), Literal(2)) val after = before transform { case Literal(2, _) => Literal(1) } - assert(EQ(after).===(Add(Literal(1), Literal(1)))) + assert(after === Add(Literal(1), Literal(1))) } test("no change") { val before = Add(Literal(1), Add(Literal(2), Add(Literal(3), Literal(4)))) val after = before transform { case Literal(5, _) => Literal(1)} - assert(EQ(before).===(after)) + assert(before === after) // Ensure that the objects after are the same objects before the transformation. before.map(identity[Expression]).zip(after.map(identity[Expression])).foreach { case (b, a) => assert(b eq a) @@ -68,7 +61,7 @@ class TreeNodeSuite extends FunSuite { val tree = Add(Literal(1), Add(Literal(2), Add(Literal(3), Literal(4)))) val literals = tree collect {case l: Literal => l} - assert(EQ(literals.size).===(4)) + assert(literals.size === 4) (1 to 4).foreach(i => assert(literals contains Literal(i))) } @@ -81,7 +74,7 @@ class TreeNodeSuite extends FunSuite { case l: Literal => actual.append(l.toString); l } - assert(EQ(expected).===(actual)) + assert(expected === actual) } test("post-order transform") { @@ -93,7 +86,7 @@ class TreeNodeSuite extends FunSuite { case l: Literal => actual.append(l.toString); l } - assert(EQ(expected).===(actual)) + assert(expected === actual) } test("transform works on nodes with Option children") { @@ -102,13 +95,13 @@ class TreeNodeSuite extends FunSuite { val toZero: PartialFunction[Expression, Expression] = { case Literal(_, _) => Literal(0) } var actual = dummy1 transformDown toZero - assert(EQ(actual).===(Dummy(Some(Literal(0))))) + assert(actual === Dummy(Some(Literal(0)))) actual = dummy1 transformUp toZero - assert(EQ(actual).===(Dummy(Some(Literal(0))))) + assert(actual === Dummy(Some(Literal(0)))) actual = dummy2 transform toZero - assert(EQ(actual).===(Dummy(None))) + assert(actual === Dummy(None)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index bc8efd787b1d..765fa8277634 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -22,13 +22,6 @@ import org.apache.spark.sql.columnar._ import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.storage.{StorageLevel, RDDBlockId} -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - case class BigData(s: String) class CachedTableSuite extends QueryTest { @@ -81,7 +74,7 @@ class CachedTableSuite extends QueryTest { val data = "*" * 10000 sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).registerTempTable("bigData") table("bigData").persist(StorageLevel.MEMORY_AND_DISK) - assert(EQ(table("bigData").count()).===(200000L)) + assert(table("bigData").count() === 200000L) table("bigData").unpersist(blocking = true) } @@ -235,7 +228,7 @@ class CachedTableSuite extends QueryTest { table("testData").queryExecution.withCachedData.collect { case cached: InMemoryRelation => val actualSizeInBytes = (1 to 100).map(i => INT.defaultSize + i.toString.length + 4).sum - assert(EQ(cached.statistics.sizeInBytes).===(actualSizeInBytes)) + assert(cached.statistics.sizeInBytes === actualSizeInBytes) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 34d58a09af85..e70ad891eea3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -24,14 +24,6 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.dsl._ import org.apache.spark.sql.test.TestSQLContext._ -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - - class DslQuerySuite extends QueryTest { import org.apache.spark.sql.TestData._ @@ -179,7 +171,7 @@ class DslQuerySuite extends QueryTest { } test("count") { - assert(EQ(testData2.count()).===(testData2.map(_ => 1).count())) + assert(testData2.count() === testData2.map(_ => 1).count()) } test("null count") { @@ -200,7 +192,7 @@ class DslQuerySuite extends QueryTest { } test("zero count") { - assert(EQ(emptyTableData.count()).===(0)) + assert(emptyTableData.count() === 0) } test("except") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 3d1f1801778c..8b4cf5bac018 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -25,13 +25,6 @@ import org.apache.spark.sql.catalyst.plans.{FullOuter, Inner, LeftOuter, RightOu import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.test.TestSQLContext._ -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - class JoinSuite extends QueryTest with BeforeAndAfterEach { // Ensures tables are loaded. TestData @@ -41,7 +34,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { val y = testData2.as('y) val join = x.join(y, Inner, Some("x.a".attr === "y.a".attr)).queryExecution.analyzed val planned = planner.HashJoin(join) - assert(EQ(planned.size).===(1)) + assert(planned.size === 1) } def assertJoin(sqlString: String, c: Class[_]): Any = { @@ -57,7 +50,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { case j: BroadcastNestedLoopJoin => j } - assert(EQ(operators.size).===(1)) + assert(operators.size === 1) if (operators(0).getClass() != c) { fail(s"$sqlString expected operator: $c, but got ${operators(0)}\n physical: \n$physical") } @@ -111,7 +104,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { val join = x.join(y, Inner, Some("x.a".attr === "y.a".attr && "x.b".attr === "y.b".attr)).queryExecution.analyzed val planned = planner.HashJoin(join) - assert(EQ(planned.size).===(1)) + assert(planned.size === 1) } test("inner join where, one match per row") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala index df22ec9b1bcd..60701f0e154f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLConfSuite.scala @@ -24,13 +24,6 @@ import org.apache.spark.sql.test._ /* Implicits */ import TestSQLContext._ -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - class SQLConfSuite extends QueryTest with FunSuiteLike { val testKey = "test.key.0" @@ -45,7 +38,7 @@ class SQLConfSuite extends QueryTest with FunSuiteLike { test("programmatic ways of basic setting and getting") { clear() - assert(EQ(getAllConfs.size).===(0)) + assert(getAllConfs.size === 0) setConf(testKey, testVal) assert(getConf(testKey) == testVal) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 5d5c2b0b5168..ef9b76b1e251 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -22,25 +22,18 @@ import org.apache.spark.sql.test._ /* Implicits */ import TestSQLContext._ -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - case class FunctionResult(f1: String, f2: String) class UDFSuite extends QueryTest { test("Simple UDF") { registerFunction("strLenScala", (_: String).length) - assert(EQ(sql("SELECT strLenScala('test')").first().getInt(0)).===(4)) + assert(sql("SELECT strLenScala('test')").first().getInt(0) === 4) } test("TwoArgument UDF") { registerFunction("strLenScala", (_: String).length + (_:Int)) - assert(EQ(sql("SELECT strLenScala('test', 1)").first().getInt(0)).===(5)) + assert(sql("SELECT strLenScala('test', 1)").first().getInt(0) === 5) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 3c1505fd63e4..1806a1dd8202 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -24,13 +24,6 @@ import org.apache.spark.sql.catalyst.annotation.SQLUserDefinedType import org.apache.spark.sql.catalyst.types.UserDefinedType import org.apache.spark.sql.test.TestSQLContext._ -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { override def equals(other: Any): Boolean = other match { @@ -76,14 +69,14 @@ class UserDefinedTypeSuite extends QueryTest { test("register user type: MyDenseVector for MyLabeledPoint") { val labels: RDD[Double] = pointsRDD.select('label).map { case Row(v: Double) => v } val labelsArrays: Array[Double] = labels.collect() - assert(EQ(labelsArrays.size).===(2)) + assert(labelsArrays.size === 2) assert(labelsArrays.contains(1.0)) assert(labelsArrays.contains(0.0)) val features: RDD[MyDenseVector] = pointsRDD.select('features).map { case Row(v: MyDenseVector) => v } val featuresArrays: Array[MyDenseVector] = features.collect() - assert(EQ(featuresArrays.size).===(2)) + assert(featuresArrays.size === 2) assert(featuresArrays.contains(new MyDenseVector(Array(0.1, 1.0)))) assert(featuresArrays.contains(new MyDenseVector(Array(0.2, 2.0)))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index a6f31e0e1530..9ba3c210171b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -22,13 +22,6 @@ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.spark.sql._ import org.apache.spark.sql.test.TestSQLContext._ -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfter { val originalColumnBatchSize = columnBatchSize val originalInMemoryPartitionPruning = inMemoryPartitionPruning @@ -114,8 +107,8 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be case in: InMemoryColumnarTableScan => (in.readPartitions.value, in.readBatches.value) }.head - assert(EQ(readBatches).===(expectedReadBatches), "Wrong number of read batches") - assert(EQ(readPartitions).===(expectedReadPartitions), "Wrong number of read partitions") + assert(readBatches === expectedReadBatches, "Wrong number of read batches") + assert(readPartitions === expectedReadPartitions, "Wrong number of read partitions") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 2c43c110e32b..a5af71acfc79 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -27,13 +27,6 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.planner._ -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - class PlannerSuite extends FunSuite { test("unions are collapsed") { val query = testData.unionAll(testData).unionAll(testData).logicalPlan @@ -41,8 +34,8 @@ class PlannerSuite extends FunSuite { val logicalUnions = query collect { case u: logical.Union => u } val physicalUnions = planned collect { case u: execution.Union => u } - assert(EQ(logicalUnions.size).===(2)) - assert(EQ(physicalUnions.size).===(1)) + assert(logicalUnions.size === 2) + assert(physicalUnions.size === 1) } test("count is partially aggregated") { @@ -50,7 +43,7 @@ class PlannerSuite extends FunSuite { val planned = HashAggregation(query).head val aggregations = planned.collect { case n if n.nodeName contains "Aggregate" => n } - assert(EQ(aggregations.size).===(2)) + assert(aggregations.size === 2) } test("count distinct is partially aggregated") { @@ -78,7 +71,7 @@ class PlannerSuite extends FunSuite { val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join } - assert(EQ(broadcastHashJoins.size).===(1), "Should use broadcast hash join") + assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") assert(shuffledHashJoins.isEmpty, "Should not use shuffled hash join") setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, origThreshold.toString) @@ -98,7 +91,7 @@ class PlannerSuite extends FunSuite { val broadcastHashJoins = planned.collect { case join: BroadcastHashJoin => join } val shuffledHashJoins = planned.collect { case join: ShuffledHashJoin => join } - assert(EQ(broadcastHashJoins.size).===(1), "Should use broadcast hash join") + assert(broadcastHashJoins.size === 1, "Should use broadcast hash join") assert(shuffledHashJoins.isEmpty, "Should not use shuffled hash join") setConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD, origThreshold.toString) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index d4ac64a8c8f9..3cccafe92d4f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -30,13 +30,6 @@ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.util.Utils -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - case class TestRDDEntry(key: Int, value: String) case class NullReflectData( @@ -179,7 +172,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) var actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(EQ(actualCodec).===(TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil)) + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -195,7 +188,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(EQ(actualCodec).===(TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil)) + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -211,7 +204,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(EQ(actualCodec).===("UNCOMPRESSED" :: Nil)) + assert(actualCodec === "UNCOMPRESSED" :: Nil) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -227,7 +220,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(EQ(actualCodec).===(TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil)) + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -243,7 +236,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA rdd.saveAsParquetFile(path) actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct - assert(EQ(actualCodec).===(TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil)) + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) parquetFile(path).registerTempTable("tmp") checkAnswer( @@ -292,8 +285,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA } val result = query.collect() - assert(EQ(result.size).===(9), "self-join result has incorrect size") - assert(EQ(result(0).size).===(12), "result row has incorrect size") + assert(result.size === 9, "self-join result has incorrect size") + assert(result(0).size === 12, "result row has incorrect size") result.zipWithIndex.foreach { case (row, index) => row.zipWithIndex.foreach { case (field, column) => assert(field != null, s"self-join contains null value in row $index field $column") @@ -303,7 +296,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("Import of simple Parquet file") { val result = parquetFile(ParquetTestData.testDir.toString).collect() - assert(EQ(result.size).===(15)) + assert(result.size === 15) result.zipWithIndex.foreach { case (row, index) => { val checkBoolean = @@ -311,12 +304,12 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA row(0) == true else row(0) == false - assert(EQ(checkBoolean).===(true), s"boolean field value in line $index did not match") - if (index % 5 == 0) assert(EQ(row(1)).===(5), s"int field value in line $index did not match") - assert(EQ(row(2)).===("abc"), s"string field value in line $index did not match") - assert(EQ(row(3)).===((index.toLong << 33)), s"long value in line $index did not match") - assert(EQ(row(4)).===(2.5F), s"float field value in line $index did not match") - assert(EQ(row(5)).===(4.5D), s"double field value in line $index did not match") + assert(checkBoolean === true, s"boolean field value in line $index did not match") + if (index % 5 == 0) assert(row(1) === 5, s"int field value in line $index did not match") + assert(row(2) === "abc", s"string field value in line $index did not match") + assert(row(3) === (index.toLong << 33), s"long value in line $index did not match") + assert(row(4) === 2.5F, s"float field value in line $index did not match") + assert(row(5) === 4.5D, s"double field value in line $index did not match") } } } @@ -326,11 +319,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA result.zipWithIndex.foreach { case (row, index) => { if (index % 3 == 0) - assert(EQ(row(0)).===(true), s"boolean field value in line $index did not match (every third row)") + assert(row(0) === true, s"boolean field value in line $index did not match (every third row)") else - assert(EQ(row(0)).===(false), s"boolean field value in line $index did not match") - assert(EQ(row(1)).===((index.toLong << 33)), s"long field value in line $index did not match") - assert(EQ(row.size).===(2), s"number of columns in projection in line $index is incorrect") + assert(row(0) === false, s"boolean field value in line $index did not match") + assert(row(1) === (index.toLong << 33), s"long field value in line $index did not match") + assert(row.size === 2, s"number of columns in projection in line $index is incorrect") } } } @@ -388,8 +381,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val rdd_copy = sql("SELECT * FROM tmpx").collect() val rdd_orig = rdd.collect() for(i <- 0 to 99) { - assert(EQ(rdd_copy(i).apply(0)).===(rdd_orig(i).key), s"key error in line $i") - assert(EQ(rdd_copy(i).apply(1)).===(rdd_orig(i).value), s"value error in line $i") + assert(rdd_copy(i).apply(0) === rdd_orig(i).key, s"key error in line $i") + assert(rdd_copy(i).apply(1) === rdd_orig(i).value, s"value error in line $i") } Utils.deleteRecursively(file) } @@ -403,11 +396,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA dest_rdd.registerTempTable("dest") sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect() val rdd_copy1 = sql("SELECT * FROM dest").collect() - assert(EQ(rdd_copy1.size).===(100)) + assert(rdd_copy1.size === 100) sql("INSERT INTO dest SELECT * FROM source") val rdd_copy2 = sql("SELECT * FROM dest").collect().sortBy(_.getInt(0)) - assert(EQ(rdd_copy2.size).===(200)) + assert(rdd_copy2.size === 200) Utils.deleteRecursively(dirname) } @@ -415,7 +408,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA sql("INSERT INTO testsource SELECT * FROM testsource") val double_rdd = sql("SELECT * FROM testsource").collect() assert(double_rdd != null) - assert(EQ(double_rdd.size).===(30)) + assert(double_rdd.size === 30) // let's restore the original test data Utils.deleteRecursively(ParquetTestData.testDir) @@ -432,7 +425,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val readFile = parquetFile(path) val rdd_saved = readFile.collect() - assert(EQ(rdd_saved(0)).===(Seq.fill(5)(null))) + assert(rdd_saved(0) === Seq.fill(5)(null)) Utils.deleteRecursively(file) assert(true) } @@ -447,7 +440,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val readFile = parquetFile(path) val rdd_saved = readFile.collect() - assert(EQ(rdd_saved(0)).===(Seq.fill(5)(null))) + assert(rdd_saved(0) === Seq.fill(5)(null)) Utils.deleteRecursively(file) assert(true) } @@ -485,11 +478,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA val attribute2 = new AttributeReference("second", IntegerType, false)() val predicate5 = new GreaterThan(attribute1, attribute2) val badfilter = ParquetFilters.createFilter(predicate5) - assert(EQ(badfilter.isDefined).===(false)) + assert(badfilter.isDefined === false) val predicate6 = And(GreaterThan(attribute1, attribute2), GreaterThan(attribute1, attribute2)) val badfilter2 = ParquetFilters.createFilter(predicate6) - assert(EQ(badfilter2.isDefined).===(false)) + assert(badfilter2.isDefined === false) } test("test filter by predicate pushdown") { @@ -499,21 +492,21 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query1.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result1 = query1.collect() - assert(EQ(result1.size).===(50)) - assert(EQ(result1(0)(1)).===(100)) - assert(EQ(result1(49)(1)).===(149)) + assert(result1.size === 50) + assert(result1(0)(1) === 100) + assert(result1(49)(1) === 149) val query2 = sql(s"SELECT * FROM testfiltersource WHERE $myval > 150 AND $myval <= 200") assert( query2.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result2 = query2.collect() - assert(EQ(result2.size).===(50)) + assert(result2.size === 50) if (myval == "myint" || myval == "mylong") { - assert(EQ(result2(0)(1)).===(151)) - assert(EQ(result2(49)(1)).===(200)) + assert(result2(0)(1) === 151) + assert(result2(49)(1) === 200) } else { - assert(EQ(result2(0)(1)).===(150)) - assert(EQ(result2(49)(1)).===(199)) + assert(result2(0)(1) === 150) + assert(result2(49)(1) === 199) } } for(myval <- Seq("myint", "mylong")) { @@ -522,11 +515,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query3.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result3 = query3.collect() - assert(EQ(result3.size).===(20)) - assert(EQ(result3(0)(1)).===(0)) - assert(EQ(result3(9)(1)).===(9)) - assert(EQ(result3(10)(1)).===(191)) - assert(EQ(result3(19)(1)).===(200)) + assert(result3.size === 20) + assert(result3(0)(1) === 0) + assert(result3(9)(1) === 9) + assert(result3(10)(1) === 191) + assert(result3(19)(1) === 200) } for(myval <- Seq("mydouble", "myfloat")) { val result4 = @@ -541,18 +534,18 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA // currently no way to specify float constants in SqlParser? sql(s"SELECT * FROM testfiltersource WHERE $myval > 190.5 OR $myval < 10").collect() } - assert(EQ(result4.size).===(20)) - assert(EQ(result4(0)(1)).===(0)) - assert(EQ(result4(9)(1)).===(9)) - assert(EQ(result4(10)(1)).===(191)) - assert(EQ(result4(19)(1)).===(200)) + assert(result4.size === 20) + assert(result4(0)(1) === 0) + assert(result4(9)(1) === 9) + assert(result4(10)(1) === 191) + assert(result4(19)(1) === 200) } val query5 = sql(s"SELECT * FROM testfiltersource WHERE myboolean = true AND myint < 40") assert( query5.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val booleanResult = query5.collect() - assert(EQ(booleanResult.size).===(10)) + assert(booleanResult.size === 10) for(i <- 0 until 10) { if (!booleanResult(i).getBoolean(0)) { fail(s"Boolean value in result row $i not true") @@ -566,16 +559,16 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query6.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val stringResult = query6.collect() - assert(EQ(stringResult.size).===(1)) + assert(stringResult.size === 1) assert(stringResult(0).getString(2) == "100", "stringvalue incorrect") - assert(EQ(stringResult(0).getInt(1)).===(100)) + assert(stringResult(0).getInt(1) === 100) val query7 = sql(s"SELECT * FROM testfiltersource WHERE myoptint < 40") assert( query7.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val optResult = query7.collect() - assert(EQ(optResult.size).===(20)) + assert(optResult.size === 20) for(i <- 0 until 20) { if (optResult(i)(7) != i * 2) { fail(s"optional Int value in result row $i should be ${2*4*i}") @@ -587,21 +580,21 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query8.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result8 = query8.collect() - assert(EQ(result8.size).===(25)) - assert(EQ(result8(0)(7)).===(100)) - assert(EQ(result8(24)(7)).===(148)) + assert(result8.size === 25) + assert(result8(0)(7) === 100) + assert(result8(24)(7) === 148) val query9 = sql(s"SELECT * FROM testfiltersource WHERE $myval > 150 AND $myval <= 200") assert( query9.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result9 = query9.collect() - assert(EQ(result9.size).===(25)) + assert(result9.size === 25) if (myval == "myoptint" || myval == "myoptlong") { - assert(EQ(result9(0)(7)).===(152)) - assert(EQ(result9(24)(7)).===(200)) + assert(result9(0)(7) === 152) + assert(result9(24)(7) === 200) } else { - assert(EQ(result9(0)(7)).===(150)) - assert(EQ(result9(24)(7)).===(198)) + assert(result9(0)(7) === 150) + assert(result9(24)(7) === 198) } } val query10 = sql("SELECT * FROM testfiltersource WHERE myoptstring = \"100\"") @@ -609,15 +602,15 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query10.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result10 = query10.collect() - assert(EQ(result10.size).===(1)) + assert(result10.size === 1) assert(result10(0).getString(8) == "100", "stringvalue incorrect") - assert(EQ(result10(0).getInt(7)).===(100)) + assert(result10(0).getInt(7) === 100) val query11 = sql(s"SELECT * FROM testfiltersource WHERE myoptboolean = true AND myoptint < 40") assert( query11.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result11 = query11.collect() - assert(EQ(result11.size).===(7)) + assert(result11.size === 7) for(i <- 0 until 6) { if (!result11(i).getBoolean(6)) { fail(s"optional Boolean value in result row $i not true") @@ -632,7 +625,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query12.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result12 = query12.collect() - assert(EQ(result12.size).===(54)) + assert(result12.size === 54) assert(result12(0).getString(2) == "6") assert(result12(4).getString(2) == "50") assert(result12(53).getString(2) == "99") @@ -642,7 +635,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query13.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result13 = query13.collect() - assert(EQ(result13.size).===(53)) + assert(result13.size === 53) assert(result13(0).getString(2) == "6") assert(result13(4).getString(2) == "51") assert(result13(52).getString(2) == "99") @@ -652,7 +645,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query14.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result14 = query14.collect() - assert(EQ(result14.size).===(148)) + assert(result14.size === 148) assert(result14(0).getString(2) == "0") assert(result14(46).getString(2) == "50") assert(result14(147).getString(2) == "200") @@ -662,7 +655,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA query15.queryExecution.executedPlan(0)(0).isInstanceOf[ParquetTableScan], "Top operator should be ParquetTableScan after pushdown") val result15 = query15.collect() - assert(EQ(result15.size).===(147)) + assert(result15.size === 147) assert(result15(0).getString(2) == "0") assert(result15(46).getString(2) == "100") assert(result15(146).getString(2) == "200") @@ -670,7 +663,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA test("SPARK-1913 regression: columns only referenced by pushed down filters should remain") { val query = sql(s"SELECT mystring FROM testfiltersource WHERE myint < 10") - assert(EQ(query.collect().size).===(10)) + assert(query.collect().size === 10) } test("Importing nested Parquet file (Addressbook)") { @@ -679,32 +672,32 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .toSchemaRDD .collect() assert(result != null) - assert(EQ(result.size).===(2)) + assert(result.size === 2) val first_record = result(0) val second_record = result(1) assert(first_record != null) assert(second_record != null) - assert(EQ(first_record.size).===(3)) - assert(EQ(second_record(1)).===(null)) - assert(EQ(second_record(2)).===(null)) - assert(EQ(second_record(0)).===("A. Nonymous")) - assert(EQ(first_record(0)).===("Julien Le Dem")) + assert(first_record.size === 3) + assert(second_record(1) === null) + assert(second_record(2) === null) + assert(second_record(0) === "A. Nonymous") + assert(first_record(0) === "Julien Le Dem") val first_owner_numbers = first_record(1) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] val first_contacts = first_record(2) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] assert(first_owner_numbers != null) - assert(EQ(first_owner_numbers(0)).===("555 123 4567")) - assert(EQ(first_owner_numbers(2)).===("XXX XXX XXXX")) - assert(EQ(first_contacts(0) - .asInstanceOf[CatalystConverter.StructScalaType[_]].size).===(2)) + assert(first_owner_numbers(0) === "555 123 4567") + assert(first_owner_numbers(2) === "XXX XXX XXXX") + assert(first_contacts(0) + .asInstanceOf[CatalystConverter.StructScalaType[_]].size === 2) val first_contacts_entry_one = first_contacts(0) .asInstanceOf[CatalystConverter.StructScalaType[_]] - assert(EQ(first_contacts_entry_one(0)).===("Dmitriy Ryaboy")) - assert(EQ(first_contacts_entry_one(1)).===("555 987 6543")) + assert(first_contacts_entry_one(0) === "Dmitriy Ryaboy") + assert(first_contacts_entry_one(1) === "555 987 6543") val first_contacts_entry_two = first_contacts(1) .asInstanceOf[CatalystConverter.StructScalaType[_]] - assert(EQ(first_contacts_entry_two(0)).===("Chris Aniszczyk")) + assert(first_contacts_entry_two(0) === "Chris Aniszczyk") } test("Importing nested Parquet file (nested numbers)") { @@ -712,31 +705,31 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .parquetFile(ParquetTestData.testNestedDir2.toString) .toSchemaRDD .collect() - assert(EQ(result.size).===(1), "number of top-level rows incorrect") - assert(EQ(result(0).size).===(5), "number of fields in row incorrect") - assert(EQ(result(0)(0)).===(1)) - assert(EQ(result(0)(1)).===(7)) + assert(result.size === 1, "number of top-level rows incorrect") + assert(result(0).size === 5, "number of fields in row incorrect") + assert(result(0)(0) === 1) + assert(result(0)(1) === 7) val subresult1 = result(0)(2).asInstanceOf[CatalystConverter.ArrayScalaType[_]] - assert(EQ(subresult1.size).===(3)) - assert(EQ(subresult1(0)).===((1.toLong << 32))) - assert(EQ(subresult1(1)).===((1.toLong << 33))) - assert(EQ(subresult1(2)).===((1.toLong << 34))) + assert(subresult1.size === 3) + assert(subresult1(0) === (1.toLong << 32)) + assert(subresult1(1) === (1.toLong << 33)) + assert(subresult1(2) === (1.toLong << 34)) val subresult2 = result(0)(3) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) .asInstanceOf[CatalystConverter.StructScalaType[_]] - assert(EQ(subresult2.size).===(2)) - assert(EQ(subresult2(0)).===(2.5)) - assert(EQ(subresult2(1)).===(false)) + assert(subresult2.size === 2) + assert(subresult2(0) === 2.5) + assert(subresult2(1) === false) val subresult3 = result(0)(4) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] - assert(EQ(subresult3.size).===(2)) - assert(EQ(subresult3(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]].size).===(2)) + assert(subresult3.size === 2) + assert(subresult3(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]].size === 2) val subresult4 = subresult3(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]] - assert(EQ(subresult4(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(7)) - assert(EQ(subresult4(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(8)) - assert(EQ(subresult3(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]].size).===(1)) - assert(EQ(subresult3(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) - .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(9)) + assert(subresult4(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 7) + assert(subresult4(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 8) + assert(subresult3(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]].size === 1) + assert(subresult3(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 9) } test("Simple query on addressbook") { @@ -744,8 +737,8 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .parquetFile(ParquetTestData.testNestedDir1.toString) .toSchemaRDD val tmp = data.where('owner === "Julien Le Dem").select('owner as 'a, 'contacts as 'c).collect() - assert(EQ(tmp.size).===(1)) - assert(EQ(tmp(0)(0)).===("Julien Le Dem")) + assert(tmp.size === 1) + assert(tmp(0)(0) === "Julien Le Dem") } test("Projection in addressbook") { @@ -753,37 +746,37 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA data.registerTempTable("data") val query = sql("SELECT owner, contacts[1].name FROM data") val tmp = query.collect() - assert(EQ(tmp.size).===(2)) - assert(EQ(tmp(0).size).===(2)) - assert(EQ(tmp(0)(0)).===("Julien Le Dem")) - assert(EQ(tmp(0)(1)).===("Chris Aniszczyk")) - assert(EQ(tmp(1)(0)).===("A. Nonymous")) - assert(EQ(tmp(1)(1)).===(null)) + assert(tmp.size === 2) + assert(tmp(0).size === 2) + assert(tmp(0)(0) === "Julien Le Dem") + assert(tmp(0)(1) === "Chris Aniszczyk") + assert(tmp(1)(0) === "A. Nonymous") + assert(tmp(1)(1) === null) } test("Simple query on nested int data") { val data = parquetFile(ParquetTestData.testNestedDir2.toString).toSchemaRDD data.registerTempTable("data") val result1 = sql("SELECT entries[0].value FROM data").collect() - assert(EQ(result1.size).===(1)) - assert(EQ(result1(0).size).===(1)) - assert(EQ(result1(0)(0)).===(2.5)) + assert(result1.size === 1) + assert(result1(0).size === 1) + assert(result1(0)(0) === 2.5) val result2 = sql("SELECT entries[0] FROM data").collect() - assert(EQ(result2.size).===(1)) + assert(result2.size === 1) val subresult1 = result2(0)(0).asInstanceOf[CatalystConverter.StructScalaType[_]] - assert(EQ(subresult1.size).===(2)) - assert(EQ(subresult1(0)).===(2.5)) - assert(EQ(subresult1(1)).===(false)) + assert(subresult1.size === 2) + assert(subresult1(0) === 2.5) + assert(subresult1(1) === false) val result3 = sql("SELECT outerouter FROM data").collect() val subresult2 = result3(0)(0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]] - assert(EQ(subresult2(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(7)) - assert(EQ(subresult2(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(8)) - assert(EQ(result3(0)(0) + assert(subresult2(0).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 7) + assert(subresult2(1).asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 8) + assert(result3(0)(0) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](1) .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) - .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0)).===(9)) + .asInstanceOf[CatalystConverter.ArrayScalaType[_]](0) === 9) } test("nested structs") { @@ -791,17 +784,17 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .toSchemaRDD data.registerTempTable("data") val result1 = sql("SELECT booleanNumberPairs[0].value[0].truth FROM data").collect() - assert(EQ(result1.size).===(1)) - assert(EQ(result1(0).size).===(1)) - assert(EQ(result1(0)(0)).===(false)) + assert(result1.size === 1) + assert(result1(0).size === 1) + assert(result1(0)(0) === false) val result2 = sql("SELECT booleanNumberPairs[0].value[1].truth FROM data").collect() - assert(EQ(result2.size).===(1)) - assert(EQ(result2(0).size).===(1)) - assert(EQ(result2(0)(0)).===(true)) + assert(result2.size === 1) + assert(result2(0).size === 1) + assert(result2(0)(0) === true) val result3 = sql("SELECT booleanNumberPairs[1].value[0].truth FROM data").collect() - assert(EQ(result3.size).===(1)) - assert(EQ(result3(0).size).===(1)) - assert(EQ(result3(0)(0)).===(false)) + assert(result3.size === 1) + assert(result3(0).size === 1) + assert(result3(0)(0) === false) } test("simple map") { @@ -810,38 +803,38 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .toSchemaRDD data.registerTempTable("mapTable") val result1 = sql("SELECT data1 FROM mapTable").collect() - assert(EQ(result1.size).===(1)) - assert(EQ(result1(0)(0) + assert(result1.size === 1) + assert(result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, _]] - .getOrElse("key1", 0)).===(1)) - assert(EQ(result1(0)(0) + .getOrElse("key1", 0) === 1) + assert(result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, _]] - .getOrElse("key2", 0)).===(2)) + .getOrElse("key2", 0) === 2) val result2 = sql("""SELECT data1["key1"] FROM mapTable""").collect() - assert(EQ(result2(0)(0)).===(1)) + assert(result2(0)(0) === 1) } test("map with struct values") { val data = parquetFile(ParquetTestData.testNestedDir4.toString).toSchemaRDD data.registerTempTable("mapTable") val result1 = sql("SELECT data2 FROM mapTable").collect() - assert(EQ(result1.size).===(1)) + assert(result1.size === 1) val entry1 = result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] .getOrElse("seven", null) assert(entry1 != null) - assert(EQ(entry1(0)).===(42)) - assert(EQ(entry1(1)).===("the answer")) + assert(entry1(0) === 42) + assert(entry1(1) === "the answer") val entry2 = result1(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] .getOrElse("eight", null) assert(entry2 != null) - assert(EQ(entry2(0)).===(49)) - assert(EQ(entry2(1)).===(null)) + assert(entry2(0) === 49) + assert(entry2(1) === null) val result2 = sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM mapTable""").collect() - assert(EQ(result2.size).===(1)) - assert(EQ(result2(0)(0)).===(42.toLong)) - assert(EQ(result2(0)(1)).===("the answer")) + assert(result2.size === 1) + assert(result2(0)(0) === 42.toLong) + assert(result2(0)(1) === "the answer") } test("Writing out Addressbook and reading it back in") { @@ -855,12 +848,12 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .toSchemaRDD .registerTempTable("tmpcopy") val tmpdata = sql("SELECT owner, contacts[1].name FROM tmpcopy").collect() - assert(EQ(tmpdata.size).===(2)) - assert(EQ(tmpdata(0).size).===(2)) - assert(EQ(tmpdata(0)(0)).===("Julien Le Dem")) - assert(EQ(tmpdata(0)(1)).===("Chris Aniszczyk")) - assert(EQ(tmpdata(1)(0)).===("A. Nonymous")) - assert(EQ(tmpdata(1)(1)).===(null)) + assert(tmpdata.size === 2) + assert(tmpdata(0).size === 2) + assert(tmpdata(0)(0) === "Julien Le Dem") + assert(tmpdata(0)(1) === "Chris Aniszczyk") + assert(tmpdata(1)(0) === "A. Nonymous") + assert(tmpdata(1)(1) === null) Utils.deleteRecursively(tmpdir) } @@ -873,26 +866,26 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .toSchemaRDD .registerTempTable("tmpmapcopy") val result1 = sql("""SELECT data1["key2"] FROM tmpmapcopy""").collect() - assert(EQ(result1.size).===(1)) - assert(EQ(result1(0)(0)).===(2)) + assert(result1.size === 1) + assert(result1(0)(0) === 2) val result2 = sql("SELECT data2 FROM tmpmapcopy").collect() - assert(EQ(result2.size).===(1)) + assert(result2.size === 1) val entry1 = result2(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] .getOrElse("seven", null) assert(entry1 != null) - assert(EQ(entry1(0)).===(42)) - assert(EQ(entry1(1)).===("the answer")) + assert(entry1(0) === 42) + assert(entry1(1) === "the answer") val entry2 = result2(0)(0) .asInstanceOf[CatalystConverter.MapScalaType[String, CatalystConverter.StructScalaType[_]]] .getOrElse("eight", null) assert(entry2 != null) - assert(EQ(entry2(0)).===(49)) - assert(EQ(entry2(1)).===(null)) + assert(entry2(0) === 49) + assert(entry2(1) === null) val result3 = sql("""SELECT data2["seven"].payload1, data2["seven"].payload2 FROM tmpmapcopy""").collect() - assert(EQ(result3.size).===(1)) - assert(EQ(result3(0)(0)).===(42.toLong)) - assert(EQ(result3(0)(1)).===("the answer")) + assert(result3.size === 1) + assert(result3(0)(0) === 42.toLong) + assert(result3(0)(1) === "the answer") Utils.deleteRecursively(tmpdir) } @@ -901,7 +894,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA Utils.deleteRecursively(tmpdir) createParquetFile[TestRDDEntry](tmpdir.toString()).registerTempTable("tmpemptytable") val result1 = sql("SELECT * FROM tmpemptytable").collect() - assert(EQ(result1.size).===(0)) + assert(result1.size === 0) Utils.deleteRecursively(tmpdir) } @@ -915,7 +908,7 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA (fromCaseClassString, fromJson).zipped.foreach { (a, b) => assert(a.name == b.name) - assert(EQ(a.dataType).===(b.dataType)) + assert(a.dataType === b.dataType) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index 3565b377ef60..a90fc023e67d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -27,13 +27,6 @@ import org.apache.spark.sql.execution.joins.{BroadcastHashJoin, ShuffledHashJoin import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - class StatisticsSuite extends QueryTest with BeforeAndAfterAll { TestHive.reset() TestHive.cacheTables = false @@ -46,7 +39,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { case o => o } - assert(EQ(operators.size).===(1)) + assert(operators.size === 1) if (operators(0).getClass() != c) { fail( s"""$analyzeCommand expected command: $c, but got ${operators(0)} @@ -88,11 +81,11 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { // TODO: How does it works? needs to add it back for other hive version. if (HiveShim.version =="0.12.0") { - assert(EQ(queryTotalSize("analyzeTable")).===(defaultSizeInBytes)) + assert(queryTotalSize("analyzeTable") === defaultSizeInBytes) } sql("ANALYZE TABLE analyzeTable COMPUTE STATISTICS noscan") - assert(EQ(queryTotalSize("analyzeTable")).===(BigInt(11624))) + assert(queryTotalSize("analyzeTable") === BigInt(11624)) sql("DROP TABLE analyzeTable").collect() @@ -117,11 +110,11 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { |SELECT * FROM src """.stripMargin).collect() - assert(EQ(queryTotalSize("analyzeTable_part")).===(defaultSizeInBytes)) + assert(queryTotalSize("analyzeTable_part") === defaultSizeInBytes) sql("ANALYZE TABLE analyzeTable_part COMPUTE STATISTICS noscan") - assert(EQ(queryTotalSize("analyzeTable_part")).===(BigInt(17436))) + assert(queryTotalSize("analyzeTable_part") === BigInt(17436)) sql("DROP TABLE analyzeTable_part").collect() @@ -138,7 +131,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { val sizes = rdd.queryExecution.analyzed.collect { case mr: MetastoreRelation => mr.statistics.sizeInBytes } - assert(EQ(sizes.size).===(1), s"Size wrong for:\n ${rdd.queryExecution}") + assert(sizes.size === 1, s"Size wrong for:\n ${rdd.queryExecution}") assert(sizes(0).equals(BigInt(5812)), s"expected exact size 5812 for test table 'src', got: ${sizes(0)}") } @@ -158,14 +151,14 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { val sizes = rdd.queryExecution.analyzed.collect { case r if ct.runtimeClass.isAssignableFrom(r.getClass) => r.statistics.sizeInBytes } - assert(EQ(sizes.size).===(2) && sizes(0) <= autoBroadcastJoinThreshold + assert(sizes.size === 2 && sizes(0) <= autoBroadcastJoinThreshold && sizes(1) <= autoBroadcastJoinThreshold, s"query should contain two relations, each of which has size smaller than autoConvertSize") // Using `sparkPlan` because for relevant patterns in HashJoin to be // matched, other strategies need to be applied. var bhj = rdd.queryExecution.sparkPlan.collect { case j: BroadcastHashJoin => j } - assert(EQ(bhj.size).===(1), + assert(bhj.size === 1, s"actual query plans do not contain broadcast join: ${rdd.queryExecution}") checkAnswer(rdd, expectedAnswer) // check correctness of output @@ -179,7 +172,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { assert(bhj.isEmpty, "BroadcastHashJoin still planned even though it is switched off") val shj = rdd.queryExecution.sparkPlan.collect { case j: ShuffledHashJoin => j } - assert(EQ(shj.size).===(1), + assert(shj.size === 1, "ShuffledHashJoin should be planned when BroadcastHashJoin is turned off") sql(s"""SET ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD}=$tmp""") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 3e9805dfe806..b897dff0159f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -30,13 +30,6 @@ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.{Row, SchemaRDD} -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ -import org.scalatest.Assertions.{convertToEqualizer => EQ} - case class TestData(a: Int, b: String) /** @@ -146,7 +139,7 @@ class HiveQuerySuite extends HiveComparisonTest { test("CREATE TABLE AS runs once") { sql("CREATE TABLE foo AS SELECT 1 FROM src LIMIT 1").collect() - assert(EQ(sql("SELECT COUNT(*) FROM foo").collect().head.getLong(0)).===(1), + assert(sql("SELECT COUNT(*) FROM foo").collect().head.getLong(0) === 1, "Incorrect number of rows in created table") } @@ -168,7 +161,7 @@ class HiveQuerySuite extends HiveComparisonTest { test("Query expressed in SQL") { setConf("spark.sql.dialect", "sql") - assert(EQ(sql("SELECT 1").collect()).===(Array(Seq(1)))) + assert(sql("SELECT 1").collect() === Array(Seq(1))) setConf("spark.sql.dialect", "hiveql") } @@ -372,7 +365,7 @@ class HiveQuerySuite extends HiveComparisonTest { .collect() .toSet - assert(EQ(actual).===(expected)) + assert(actual === expected) } // TODO: adopt this test when Spark SQL has the functionality / framework to report errors. @@ -422,7 +415,7 @@ class HiveQuerySuite extends HiveComparisonTest { .collect() .map(x => Pair(x.getString(0), x.getInt(1))) - assert(EQ(results).===(Array(Pair("foo", 4)))) + assert(results === Array(Pair("foo", 4))) TestHive.reset() } @@ -564,8 +557,8 @@ class HiveQuerySuite extends HiveComparisonTest { sql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10") sql("SELECT * FROM m").collect().zip(sql("SELECT * FROM src LIMIT 10").collect()).map { case (Row(map: Map[_, _]), Row(key: Int, value: String)) => - assert(EQ(map.size).===(1)) - assert(EQ(map.head).===((key, value))) + assert(map.size === 1) + assert(map.head === (key, value)) } } @@ -661,7 +654,7 @@ class HiveQuerySuite extends HiveComparisonTest { sql("CREATE TABLE dp_verify(intcol INT)") sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE dp_verify") - assert(EQ(sql("SELECT * FROM dp_verify").collect()).===(Array(Row(value)))) + assert(sql("SELECT * FROM dp_verify").collect() === Array(Row(value))) } } From dabcaf262d68565ecae0092e70fe81f0be6307bd Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Mon, 10 Nov 2014 17:44:42 -0600 Subject: [PATCH 11/16] Cleaning up more comments and one more test. --- .../spark/sql/catalyst/types/timetypes.scala | 2 +- .../scala/org/apache/spark/sql/SQLContext.scala | 2 +- .../spark/sql/parquet/HiveParquetSuite.scala | 14 +++----------- 3 files changed, 5 insertions(+), 13 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala index d5dd80eadee0..3fe692f15f32 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import scala.language.implicitConversions -/* * +/** * Subclass of java.sql.Date which provides the usual comparison * operators (as required for catalyst expressions) and which can * be constructed from a string. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 9265660897da..68fcd7b6416c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -503,7 +503,7 @@ class SQLContext(@transient val sparkContext: SparkContext) new SchemaRDD(this, LogicalRDD(schema.toAttributes, rowRdd)(self)) } - /* * + /** * Make RichDate and RichTimestamp available under the names * Date and Timestamp when the members of this SQLContext are * imported. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 0069726426f9..6f57fe895838 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -29,13 +29,6 @@ import org.apache.spark.util.Utils // Implicits import org.apache.spark.sql.hive.test.TestHive._ -/* - * Note: the DSL conversions collide with the scalatest === operator! - * We can apply the scalatest conversion explicitly: - * assert(X === Y) --> assert(convertToEqualizer(X).===(Y)) - * (This file already imports convertToEqualizer) - */ - case class Cases(lower: String, UPPER: String) class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { @@ -87,7 +80,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft test("Simple column projection + filter on Parquet table") { val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() - assert(convertToEqualizer(rdd.size).===(5), "Filter returned incorrect number of rows") + assert(rdd.size === 5, "Filter returned incorrect number of rows") assert(rdd.forall(_.getBoolean(0)), "Filter returned incorrect Boolean field value") } @@ -109,7 +102,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() val rddCopy = sql("SELECT * FROM ptable").collect() val rddOrig = sql("SELECT * FROM testsource").collect() - assert(convertToEqualizer(rddCopy.size).===(rddOrig.size), "INSERT OVERWRITE changed size of table??") + assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of table??") compareRDDs(rddOrig, rddCopy, "testsource", ParquetTestData.testSchemaFieldNames) } @@ -118,8 +111,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft (rddOne, rddTwo).zipped.foreach { (a,b) => (a,b).zipped.toArray.zipWithIndex.foreach { case ((value_1, value_2), index) => - assert(convertToEqualizer(value_1).===(value_2), - s"table $tableName row $counter field ${fieldNames(index)} don't match") + assert(value_1 === value_2, s"table $tableName row $counter field ${fieldNames(index)} don't match") } counter = counter + 1 } From ad2940dad053089182dd65c2d9d0db6c4fedc0cf Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Mon, 10 Nov 2014 22:57:08 -0600 Subject: [PATCH 12/16] Removed LhsLiterals for another PR. Cleaned up. --- .../spark/sql/catalyst/dsl/package.scala | 39 ------------------- .../spark/sql/catalyst/types/timetypes.scala | 19 ++------- .../org/apache/spark/sql/SQLContext.scala | 22 +++++++---- 3 files changed, 17 insertions(+), 63 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index c9f4daf16b9d..31dc5a58e68e 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -84,7 +84,6 @@ package object dsl { def > (other: Expression) = GreaterThan(expr, other) def >= (other: Expression) = GreaterThanOrEqual(expr, other) def === (other: Expression) = EqualTo(expr, other) - def -=- (other: Expression) = EqualTo(expr, other) def <=> (other: Expression) = EqualNullSafe(expr, other) def !== (other: Expression) = Not(EqualTo(expr, other)) @@ -149,44 +148,6 @@ package object dsl { def upper(e: Expression) = Upper(e) def lower(e: Expression) = Lower(e) - /* - * Conversions to provide the standard operators in the special case - * where a literal is being combined with a symbol. Without these an - * expression such as 0 < 'x is not recognized. - */ - class LhsLiteral(x: Any) { - val literal = Literal(x) - def + (other: Symbol) = Add(literal, other) - def - (other: Symbol) = Subtract(literal, other) - def * (other: Symbol) = Multiply(literal, other) - def / (other: Symbol) = Divide(literal, other) - def % (other: Symbol) = Remainder(literal, other) - - def && (other: Symbol) = And(literal, other) - def || (other: Symbol) = Or(literal, other) - - def < (other: Symbol) = LessThan(literal, other) - def <= (other: Symbol) = LessThanOrEqual(literal, other) - def > (other: Symbol) = GreaterThan(literal, other) - def >= (other: Symbol) = GreaterThanOrEqual(literal, other) - def -=- (other: Symbol) = EqualTo(literal, other) - def <=> (other: Symbol) = EqualNullSafe(literal, other) - def !== (other: Symbol) = Not(EqualTo(literal, other)) - } - - implicit def booleanToLhsLiteral(b: Boolean) = new LhsLiteral(b) - implicit def byteToLhsLiteral(b: Byte) = new LhsLiteral(b) - implicit def shortToLhsLiteral(s: Short) = new LhsLiteral(s) - implicit def intToLhsLiteral(i: Int) = new LhsLiteral(i) - implicit def longToLhsLiteral(l: Long) = new LhsLiteral(l) - implicit def floatToLhsLiteral(f: Float) = new LhsLiteral(f) - implicit def doubleToLhsLiteral(d: Double) = new LhsLiteral(d) - implicit def stringToLhsLiteral(s: String) = new LhsLiteral(s) - implicit def bigDecimalToLhsLiteral(d: BigDecimal) = new LhsLiteral(d) - implicit def decimalToLhsLiteral(d: Decimal) = new LhsLiteral(d) - implicit def dateToLhsLiteral(d: Date) = new LhsLiteral(d) - implicit def timestampToLhsLiteral(t: Timestamp) = new LhsLiteral(t) - implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } // TODO more implicit class for literal? implicit class DslString(val s: String) extends ImplicitOperators { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala index 3fe692f15f32..8ec617d226c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala @@ -48,7 +48,7 @@ class RichDate(milliseconds: Long) extends Date(milliseconds) { object RichDate { def apply(init: String) = new RichDate(Date.valueOf(init).getTime) - def unapply(richdate: RichDate): Option[Date] = Some(new Date(richdate.getTime)) + def unapply(date: Any): Option[RichDate] = Some(RichDate(date.toString)) } /** @@ -72,26 +72,13 @@ class RichTimestamp(milliseconds: Long) extends Timestamp(milliseconds) { def <= (that: Timestamp): Boolean = (this.before(that) || this.equals(that)) def >= (that: Timestamp): Boolean = (this.after(that) || this.equals(that)) def === (that: Timestamp): Boolean = this.equals(that) - // Follow Hive conventions when converting to a String. - // Copied from the Cast class. - override def toString(): String = { - val jts = new Timestamp(this.getTime) - val timestampString = jts.toString - val formatted = Cast.threadLocalTimestampFormat.get.format(jts) - - if (timestampString.length > 19 && timestampString.substring(19) != ".0") { - formatted + timestampString.substring(19) - } else { - formatted - } - } } object RichTimestamp { def apply(init: String) = new RichTimestamp(Timestamp.valueOf(init).getTime) - def unapply(richtimestamp: RichTimestamp): Option[Timestamp] = - Some(new Timestamp(richtimestamp.getTime)) + def unapply(timestamp: Any): Option[RichTimestamp] = + Some(RichTimestamp(timestamp.toString)) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 68fcd7b6416c..a19503d440c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -504,13 +504,19 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** - * Make RichDate and RichTimestamp available under the names - * Date and Timestamp when the members of this SQLContext are - * imported. + * In DSL expressions date and time can be used as aliases for RichDate + * and RichTimestamp: + * {{{ + * val res = sqlrdd.where('date > date("2014-01-01")).select('date, 'high, 'close) + * }}} + */ + import org.apache.spark.sql.catalyst.expressions.{RichDate, RichTimestamp} + val date = RichDate + val timestamp = RichTimestamp + + /** + * Row Fields can be extracted asInstanceOf[RichDate] or asInstanceOf[RichTimestamp] */ - import org.apache.spark.sql.catalyst.expressions._ - val Date = RichDate - type Date = RichDate - val Timestamp = RichTimestamp - type Timestamp = RichTimestamp + type RichDate = org.apache.spark.sql.catalyst.expressions.RichDate + type RichTimestamp = org.apache.spark.sql.catalyst.expressions.RichTimestamp } From f917e06db9925dc02bc98df3e25ba60642d007a3 Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Mon, 10 Nov 2014 23:02:22 -0600 Subject: [PATCH 13/16] Removed test which is no longer part of this PR. --- .../ExpressionEvaluationSuite.scala | 19 ------------------- 1 file changed, 19 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index a1a1e8296964..ce9a08f6da37 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -775,25 +775,6 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(~c1, -2, row) } - /* - * Testing the DSL conversions which allow literals on the left hand - * side of an expression. Note: the DSL conversions collide with - * the scalatest === operator. But we can apply the scalatest - * conversion explicitly: - * assert(X === Y) --> assert(EQ(X).===(Y)) - */ - import org.scalatest.Assertions.{convertToEqualizer => EQ} - test("recognizes literals on the left") { - assert(EQ(-1 + 'x).===(Add(-1, 'x))) - assert(EQ(0 < 'x).===(LessThan(0, 'x))) - assert(EQ(1.5 -=- 'x).===(EqualTo(1.5, 'x))) - assert(EQ(false !== 'x).===(Not(EqualTo(false, 'x)))) - assert(EQ("a string" >= 'x).===(GreaterThanOrEqual("a string", 'x))) - assert(EQ(RichDate("2014-11-05") > 'date).===(GreaterThan(RichDate("2014-11-05"), 'date))) - assert(EQ(RichTimestamp("2014-11-05 12:34:56.789") < 'now) - .===(LessThan(RichTimestamp("2014-11-05 12:34:56.789"), 'now))) - } - test("comparison operators for RichDate and RichTimestamp") { assert(EQ(RichDate("2014-11-05") < RichDate("2014-11-06")).===(true)) assert(EQ(RichDate("2014-11-05") <= RichDate("2013-11-06")).===(false)) From 4c32c0416093fedcb70acdcbfbeb112ded4406eb Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Mon, 10 Nov 2014 23:25:55 -0600 Subject: [PATCH 14/16] Restored lost import of scalatest.Assertions.convertToEqualizer. --- .../sql/catalyst/expressions/ExpressionEvaluationSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index ce9a08f6da37..cdd2a0576a0d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -776,6 +776,7 @@ class ExpressionEvaluationSuite extends FunSuite { } test("comparison operators for RichDate and RichTimestamp") { + import org.scalatest.Assertions.{convertToEqualizer => EQ} assert(EQ(RichDate("2014-11-05") < RichDate("2014-11-06")).===(true)) assert(EQ(RichDate("2014-11-05") <= RichDate("2013-11-06")).===(false)) assert(EQ(RichTimestamp("2014-11-05 12:34:56.5432") > RichTimestamp("2014-11-05 00:00:00")) From dd39a8969ce9cbbf219825d1dc0824b594334892 Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Tue, 6 Jan 2015 13:17:35 -0600 Subject: [PATCH 15/16] Update timetypes.scala --- .../apache/spark/sql/catalyst/types/timetypes.scala | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala index 8ec617d226c3..9c92bd6d5044 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/timetypes.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} +import java.text.SimpleDateFormat import scala.language.implicitConversions /** @@ -43,6 +44,12 @@ class RichDate(milliseconds: Long) extends Date(milliseconds) { def <= (that: Date): Boolean = (this.before(that) || this.equals(that)) def >= (that: Date): Boolean = (this.after(that) || this.equals(that)) def === (that: Date): Boolean = this.equals(that) + def compare(that: Date): Int = this.getTime.compare(that.getTime) + def format(format: String): String = { + val sdf = new SimpleDateFormat(format) + val d = new Date(this.getTime) + sdf.format(d) + } } object RichDate { @@ -72,6 +79,11 @@ class RichTimestamp(milliseconds: Long) extends Timestamp(milliseconds) { def <= (that: Timestamp): Boolean = (this.before(that) || this.equals(that)) def >= (that: Timestamp): Boolean = (this.after(that) || this.equals(that)) def === (that: Timestamp): Boolean = this.equals(that) + def format(format: String): String = { + val sdf = new SimpleDateFormat(format) + val ts = new Timestamp(this.getTime) + sdf.format(ts) + } } object RichTimestamp { From 31f0d7ec2c6062fdf3db49d3b516c46c6ca2250f Mon Sep 17 00:00:00 2001 From: Marc Culler Date: Tue, 6 Jan 2015 14:09:41 -0600 Subject: [PATCH 16/16] Update ExpressionEvaluationSuite.scala --- .../catalyst/expressions/ExpressionEvaluationSuite.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 4067d987bd56..12ae30ed5115 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -830,6 +830,13 @@ class ExpressionEvaluationSuite extends FunSuite { .===(false)) } + test("format methods for RichDate and RichTimestamp") { + val s1:String = RichDate("2014-11-22").format("MMMM d yyyy") + val s2:String = RichTimestamp("2014-11-22 12:34:56").format("MMMM d HH:mm") + assert(s1 == "November 22 2014") + assert(s2 == "November 22 12:34") + } + test("implicit conversions for RichDate and RichTimestamp") { import org.apache.spark.sql.catalyst.expressions.TimeConversions._ val d1 = RichDate("2014-01-01")