From d986fb9a378416248768828e6e6c7405697f9a5a Mon Sep 17 00:00:00 2001 From: Rene Treffer Date: Sat, 13 Jun 2015 11:58:22 -0700 Subject: [PATCH 01/13] [SPARK-7897] Improbe type for jdbc/"unsigned bigint" The original fix uses DecimalType.Unlimited, which is harder to handle afterwards. There is no scale and most data should fit into a long, thus DecimalType(20,0) should be better. Author: Rene Treffer Closes #6789 from rtreffer/spark-7897-unsigned-bigint-as-decimal and squashes the following commits: 2006613 [Rene Treffer] Fix type for "unsigned bigint" jdbc loading. --- sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 667fc70cff956..226b143923df6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -55,7 +55,7 @@ private[sql] object JDBCRDD extends Logging { val answer = sqlType match { // scalastyle:off case java.sql.Types.ARRAY => null - case java.sql.Types.BIGINT => if (signed) { LongType } else { DecimalType.Unlimited } + case java.sql.Types.BIGINT => if (signed) { LongType } else { DecimalType(20,0) } case java.sql.Types.BINARY => BinaryType case java.sql.Types.BIT => BooleanType // @see JdbcDialect for quirks case java.sql.Types.BLOB => BinaryType From ce1041c38f92449ca14894551c358c875672afe6 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 13 Jun 2015 16:13:26 -0700 Subject: [PATCH 02/13] [SPARK-8346] [SQL] Use InternalRow instread of catalyst.InternalRow cc rxin marmbrus Author: Davies Liu Closes #6802 from davies/cleanup_internalrow and squashes the following commits: 769d2aa [Davies Liu] remove not needed cast 4acbbe4 [Davies Liu] catalyst.Internal -> InternalRow --- .../sql/catalyst/analysis/unresolved.scala | 10 +- .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../catalyst/expressions/ExtractValue.scala | 10 +- .../sql/catalyst/expressions/Projection.scala | 105 +++++++++--------- .../sql/catalyst/expressions/ScalaUdf.scala | 48 ++++---- .../sql/catalyst/expressions/aggregates.scala | 68 ++++++------ .../sql/catalyst/expressions/arithmetic.scala | 12 +- .../expressions/codegen/CodeGenerator.scala | 6 +- .../codegen/GenerateMutableProjection.scala | 3 +- .../codegen/GenerateOrdering.scala | 10 +- .../codegen/GeneratePredicate.scala | 8 +- .../catalyst/expressions/complexTypes.scala | 4 +- .../catalyst/expressions/conditionals.scala | 6 +- .../expressions/decimalFunctions.scala | 4 +- .../sql/catalyst/expressions/generators.scala | 14 +-- .../sql/catalyst/expressions/literals.scala | 6 +- .../spark/sql/catalyst/expressions/math.scala | 8 +- .../expressions/namedExpressions.scala | 6 +- .../catalyst/expressions/nullFunctions.scala | 13 +-- .../sql/catalyst/expressions/package.scala | 6 +- .../org/apache/spark/sql/DataFrame.scala | 6 +- .../sql/execution/joins/HashOuterJoin.scala | 2 +- 22 files changed, 176 insertions(+), 183 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 5de188d418924..c9d91425788a8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -68,7 +68,7 @@ case class UnresolvedAttribute(nameParts: Seq[String]) override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute.quoted(newName) // Unresolved attributes are transient at compile time and don't get evaluated during execution. - override def eval(input: catalyst.InternalRow = null): Any = + override def eval(input: InternalRow = null): Any = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") override def toString: String = s"'$name" @@ -86,7 +86,7 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E override lazy val resolved = false // Unresolved functions are transient at compile time and don't get evaluated during execution. - override def eval(input: catalyst.InternalRow = null): Any = + override def eval(input: InternalRow = null): Any = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") override def toString: String = s"'$name(${children.mkString(",")})" @@ -108,7 +108,7 @@ trait Star extends NamedExpression with trees.LeafNode[Expression] { override lazy val resolved = false // Star gets expanded at runtime so we never evaluate a Star. - override def eval(input: catalyst.InternalRow = null): Any = + override def eval(input: InternalRow = null): Any = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") def expand(input: Seq[Attribute], resolver: Resolver): Seq[NamedExpression] @@ -167,7 +167,7 @@ case class MultiAlias(child: Expression, names: Seq[String]) override lazy val resolved = false - override def eval(input: catalyst.InternalRow = null): Any = + override def eval(input: InternalRow = null): Any = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") override def toString: String = s"$child AS $names" @@ -201,7 +201,7 @@ case class UnresolvedExtractValue(child: Expression, extraction: Expression) override def nullable: Boolean = throw new UnresolvedException(this, "nullable") override lazy val resolved = false - override def eval(input: catalyst.InternalRow = null): Any = + override def eval(input: InternalRow = null): Any = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") override def toString: String = s"$child[$extraction]" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index afbf30af332d8..05a04bdff9b3e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -394,7 +394,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } // TODO: Could be faster? val newRow = new GenericMutableRow(from.fields.size) - buildCast[catalyst.InternalRow](_, row => { + buildCast[InternalRow](_, row => { var i = 0 while (i < row.length) { val v = row(i) @@ -426,7 +426,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w private[this] lazy val cast: Any => Any = cast(child.dataType, dataType) - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val evaluated = child.eval(input) if (evaluated == null) null else cast(evaluated) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala index 16f3ccc3d6b88..4aaabff15b6ee 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExtractValue.scala @@ -105,8 +105,8 @@ case class GetStructField(child: Expression, field: StructField, ordinal: Int) override def foldable: Boolean = child.foldable override def toString: String = s"$child.${field.name}" - override def eval(input: catalyst.InternalRow): Any = { - val baseValue = child.eval(input).asInstanceOf[catalyst.InternalRow] + override def eval(input: InternalRow): Any = { + val baseValue = child.eval(input).asInstanceOf[InternalRow] if (baseValue == null) null else baseValue(ordinal) } } @@ -125,8 +125,8 @@ case class GetArrayStructFields( override def foldable: Boolean = child.foldable override def toString: String = s"$child.${field.name}" - override def eval(input: catalyst.InternalRow): Any = { - val baseValue = child.eval(input).asInstanceOf[Seq[catalyst.InternalRow]] + override def eval(input: InternalRow): Any = { + val baseValue = child.eval(input).asInstanceOf[Seq[InternalRow]] if (baseValue == null) null else { baseValue.map { row => if (row == null) null else row(ordinal) @@ -146,7 +146,7 @@ abstract class ExtractValueWithOrdinal extends ExtractValue { override def toString: String = s"$child[$ordinal]" override def children: Seq[Expression] = child :: ordinal :: Nil - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val value = child.eval(input) if (value == null) { null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index d6806f78ab3fd..d5967438ccb5a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -17,9 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst - - /** * A [[Projection]] that is calculated by calling the `eval` of each of the specified expressions. * @param expressions a sequence of expressions that determine the value of each column of the @@ -32,7 +29,7 @@ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { // null check is required for when Kryo invokes the no-arg constructor. protected val exprArray = if (expressions != null) expressions.toArray else null - def apply(input: catalyst.InternalRow): catalyst.InternalRow = { + def apply(input: InternalRow): InternalRow = { val outputArray = new Array[Any](exprArray.length) var i = 0 while (i < exprArray.length) { @@ -57,14 +54,14 @@ case class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mu private[this] val exprArray = expressions.toArray private[this] var mutableRow: MutableRow = new GenericMutableRow(exprArray.size) - def currentValue: catalyst.InternalRow = mutableRow + def currentValue: InternalRow = mutableRow override def target(row: MutableRow): MutableProjection = { mutableRow = row this } - override def apply(input: catalyst.InternalRow): catalyst.InternalRow = { + override def apply(input: InternalRow): InternalRow = { var i = 0 while (i < exprArray.length) { mutableRow(i) = exprArray(i).eval(input) @@ -78,31 +75,31 @@ case class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mu * A mutable wrapper that makes two rows appear as a single concatenated row. Designed to * be instantiated once per thread and reused. */ -class JoinedRow extends catalyst.InternalRow { - private[this] var row1: catalyst.InternalRow = _ - private[this] var row2: catalyst.InternalRow = _ +class JoinedRow extends InternalRow { + private[this] var row1: InternalRow = _ + private[this] var row2: InternalRow = _ - def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = { + def this(left: InternalRow, right: InternalRow) = { this() row1 = left row2 = right } /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ - def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = { + def apply(r1: InternalRow, r2: InternalRow): InternalRow = { row1 = r1 row2 = r2 this } /** Updates this JoinedRow by updating its left base row. Returns itself. */ - def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = { + def withLeft(newLeft: InternalRow): InternalRow = { row1 = newLeft this } /** Updates this JoinedRow by updating its right base row. Returns itself. */ - def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = { + def withRight(newRight: InternalRow): InternalRow = { row2 = newRight this } @@ -144,7 +141,7 @@ class JoinedRow extends catalyst.InternalRow { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy(): catalyst.InternalRow = { + override def copy(): InternalRow = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -178,31 +175,31 @@ class JoinedRow extends catalyst.InternalRow { * Row will be referenced, increasing the opportunity for the JIT to play tricks. This sounds * crazy but in benchmarks it had noticeable effects. */ -class JoinedRow2 extends catalyst.InternalRow { - private[this] var row1: catalyst.InternalRow = _ - private[this] var row2: catalyst.InternalRow = _ +class JoinedRow2 extends InternalRow { + private[this] var row1: InternalRow = _ + private[this] var row2: InternalRow = _ - def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = { + def this(left: InternalRow, right: InternalRow) = { this() row1 = left row2 = right } /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ - def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = { + def apply(r1: InternalRow, r2: InternalRow): InternalRow = { row1 = r1 row2 = r2 this } /** Updates this JoinedRow by updating its left base row. Returns itself. */ - def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = { + def withLeft(newLeft: InternalRow): InternalRow = { row1 = newLeft this } /** Updates this JoinedRow by updating its right base row. Returns itself. */ - def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = { + def withRight(newRight: InternalRow): InternalRow = { row2 = newRight this } @@ -244,7 +241,7 @@ class JoinedRow2 extends catalyst.InternalRow { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy(): catalyst.InternalRow = { + override def copy(): InternalRow = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -272,31 +269,31 @@ class JoinedRow2 extends catalyst.InternalRow { /** * JIT HACK: Replace with macros */ -class JoinedRow3 extends catalyst.InternalRow { - private[this] var row1: catalyst.InternalRow = _ - private[this] var row2: catalyst.InternalRow = _ +class JoinedRow3 extends InternalRow { + private[this] var row1: InternalRow = _ + private[this] var row2: InternalRow = _ - def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = { + def this(left: InternalRow, right: InternalRow) = { this() row1 = left row2 = right } /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ - def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = { + def apply(r1: InternalRow, r2: InternalRow): InternalRow = { row1 = r1 row2 = r2 this } /** Updates this JoinedRow by updating its left base row. Returns itself. */ - def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = { + def withLeft(newLeft: InternalRow): InternalRow = { row1 = newLeft this } /** Updates this JoinedRow by updating its right base row. Returns itself. */ - def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = { + def withRight(newRight: InternalRow): InternalRow = { row2 = newRight this } @@ -338,7 +335,7 @@ class JoinedRow3 extends catalyst.InternalRow { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy(): catalyst.InternalRow = { + override def copy(): InternalRow = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -366,31 +363,31 @@ class JoinedRow3 extends catalyst.InternalRow { /** * JIT HACK: Replace with macros */ -class JoinedRow4 extends catalyst.InternalRow { - private[this] var row1: catalyst.InternalRow = _ - private[this] var row2: catalyst.InternalRow = _ +class JoinedRow4 extends InternalRow { + private[this] var row1: InternalRow = _ + private[this] var row2: InternalRow = _ - def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = { + def this(left: InternalRow, right: InternalRow) = { this() row1 = left row2 = right } /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ - def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = { + def apply(r1: InternalRow, r2: InternalRow): InternalRow = { row1 = r1 row2 = r2 this } /** Updates this JoinedRow by updating its left base row. Returns itself. */ - def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = { + def withLeft(newLeft: InternalRow): InternalRow = { row1 = newLeft this } /** Updates this JoinedRow by updating its right base row. Returns itself. */ - def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = { + def withRight(newRight: InternalRow): InternalRow = { row2 = newRight this } @@ -432,7 +429,7 @@ class JoinedRow4 extends catalyst.InternalRow { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy(): catalyst.InternalRow = { + override def copy(): InternalRow = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -460,31 +457,31 @@ class JoinedRow4 extends catalyst.InternalRow { /** * JIT HACK: Replace with macros */ -class JoinedRow5 extends catalyst.InternalRow { - private[this] var row1: catalyst.InternalRow = _ - private[this] var row2: catalyst.InternalRow = _ +class JoinedRow5 extends InternalRow { + private[this] var row1: InternalRow = _ + private[this] var row2: InternalRow = _ - def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = { + def this(left: InternalRow, right: InternalRow) = { this() row1 = left row2 = right } /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ - def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = { + def apply(r1: InternalRow, r2: InternalRow): InternalRow = { row1 = r1 row2 = r2 this } /** Updates this JoinedRow by updating its left base row. Returns itself. */ - def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = { + def withLeft(newLeft: InternalRow): InternalRow = { row1 = newLeft this } /** Updates this JoinedRow by updating its right base row. Returns itself. */ - def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = { + def withRight(newRight: InternalRow): InternalRow = { row2 = newRight this } @@ -526,7 +523,7 @@ class JoinedRow5 extends catalyst.InternalRow { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy(): catalyst.InternalRow = { + override def copy(): InternalRow = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 @@ -554,31 +551,31 @@ class JoinedRow5 extends catalyst.InternalRow { /** * JIT HACK: Replace with macros */ -class JoinedRow6 extends catalyst.InternalRow { - private[this] var row1: catalyst.InternalRow = _ - private[this] var row2: catalyst.InternalRow = _ +class JoinedRow6 extends InternalRow { + private[this] var row1: InternalRow = _ + private[this] var row2: InternalRow = _ - def this(left: catalyst.InternalRow, right: catalyst.InternalRow) = { + def this(left: InternalRow, right: InternalRow) = { this() row1 = left row2 = right } /** Updates this JoinedRow to used point at two new base rows. Returns itself. */ - def apply(r1: catalyst.InternalRow, r2: catalyst.InternalRow): catalyst.InternalRow = { + def apply(r1: InternalRow, r2: InternalRow): InternalRow = { row1 = r1 row2 = r2 this } /** Updates this JoinedRow by updating its left base row. Returns itself. */ - def withLeft(newLeft: catalyst.InternalRow): catalyst.InternalRow = { + def withLeft(newLeft: InternalRow): InternalRow = { row1 = newLeft this } /** Updates this JoinedRow by updating its right base row. Returns itself. */ - def withRight(newRight: catalyst.InternalRow): catalyst.InternalRow = { + def withRight(newRight: InternalRow): InternalRow = { row2 = newRight this } @@ -620,7 +617,7 @@ class JoinedRow6 extends catalyst.InternalRow { override def getAs[T](i: Int): T = if (i < row1.length) row1.getAs[T](i) else row2.getAs[T](i - row1.length) - override def copy(): catalyst.InternalRow = { + override def copy(): InternalRow = { val totalSize = row1.length + row2.length val copiedValues = new Array[Any](totalSize) var i = 0 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 40f235fc19536..b3ce698c5552d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -58,7 +58,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi private[this] val f = children.size match { case 0 => val func = function.asInstanceOf[() => Any] - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func() } @@ -66,7 +66,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val func = function.asInstanceOf[(Any) => Any] val child0 = children(0) lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input))) } @@ -77,7 +77,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child1 = children(1) lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input))) @@ -91,7 +91,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -108,7 +108,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -128,7 +128,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -151,7 +151,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -177,7 +177,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -206,7 +206,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -238,7 +238,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -273,7 +273,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -311,7 +311,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -352,7 +352,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -396,7 +396,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -443,7 +443,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -493,7 +493,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -546,7 +546,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -602,7 +602,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -661,7 +661,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -723,7 +723,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -788,7 +788,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -856,7 +856,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -927,7 +927,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) lazy val converter21 = CatalystTypeConverters.createToScalaConverter(child21.dataType) - (input: catalyst.InternalRow) => { + (input: InternalRow) => { func( converter0(child0.eval(input)), converter1(child1.eval(input)), @@ -956,6 +956,6 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi // scalastyle:on private[this] val converter = CatalystTypeConverters.createToCatalystConverter(dataType) - override def eval(input: catalyst.InternalRow): Any = converter(f(input)) + override def eval(input: InternalRow): Any = converter(f(input)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index f9e8150a689c1..00d2e499c5890 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -38,7 +38,7 @@ abstract class AggregateExpression extends Expression { * [[AggregateExpression.eval]] should never be invoked because [[AggregateExpression]]'s are * replaced with a physical aggregate operator at runtime. */ - override def eval(input: catalyst.InternalRow = null): Any = + override def eval(input: InternalRow = null): Any = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } @@ -81,7 +81,7 @@ abstract class AggregateFunction override def nullable: Boolean = base.nullable override def dataType: DataType = base.dataType - def update(input: catalyst.InternalRow): Unit + def update(input: InternalRow): Unit // Do we really need this? override def newInstance(): AggregateFunction = { @@ -109,7 +109,7 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr val currentMin: MutableLiteral = MutableLiteral(null, expr.dataType) val cmp = GreaterThan(currentMin, expr) - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { if (currentMin.value == null) { currentMin.value = expr.eval(input) } else if (cmp.eval(input) == true) { @@ -117,7 +117,7 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr } } - override def eval(input: catalyst.InternalRow): Any = currentMin.value + override def eval(input: InternalRow): Any = currentMin.value } case class Max(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -140,7 +140,7 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr val currentMax: MutableLiteral = MutableLiteral(null, expr.dataType) val cmp = LessThan(currentMax, expr) - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { if (currentMax.value == null) { currentMax.value = expr.eval(input) } else if (cmp.eval(input) == true) { @@ -148,7 +148,7 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr } } - override def eval(input: catalyst.InternalRow): Any = currentMax.value + override def eval(input: InternalRow): Any = currentMax.value } case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -206,14 +206,14 @@ case class CollectHashSetFunction( @transient val distinctValue = new InterpretedProjection(expr) - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { val evaluatedExpr = distinctValue(input) if (!evaluatedExpr.anyNull) { seen.add(evaluatedExpr) } } - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { seen } } @@ -239,7 +239,7 @@ case class CombineSetsAndCountFunction( val seen = new OpenHashSet[Any]() - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { val inputSetEval = inputSet.eval(input).asInstanceOf[OpenHashSet[Any]] val inputIterator = inputSetEval.iterator while (inputIterator.hasNext) { @@ -247,7 +247,7 @@ case class CombineSetsAndCountFunction( } } - override def eval(input: catalyst.InternalRow): Any = seen.size.toLong + override def eval(input: InternalRow): Any = seen.size.toLong } /** The data type of ApproxCountDistinctPartition since its output is a HyperLogLog object. */ @@ -454,7 +454,7 @@ case class CombineSetsAndSumFunction( val seen = new OpenHashSet[Any]() - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { val inputSetEval = inputSet.eval(input).asInstanceOf[OpenHashSet[Any]] val inputIterator = inputSetEval.iterator while (inputIterator.hasNext) { @@ -462,8 +462,8 @@ case class CombineSetsAndSumFunction( } } - override def eval(input: catalyst.InternalRow): Any = { - val casted = seen.asInstanceOf[OpenHashSet[catalyst.InternalRow]] + override def eval(input: InternalRow): Any = { + val casted = seen.asInstanceOf[OpenHashSet[InternalRow]] if (casted.size == 0) { null } else { @@ -525,7 +525,7 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) private def addFunction(value: Any) = Add(sum, Cast(Literal.create(value, expr.dataType), calcType)) - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { if (count == 0L) { null } else { @@ -542,7 +542,7 @@ case class AverageFunction(expr: Expression, base: AggregateExpression) } } - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { val evaluatedExpr = expr.eval(input) if (evaluatedExpr != null) { count += 1 @@ -556,14 +556,14 @@ case class CountFunction(expr: Expression, base: AggregateExpression) extends Ag var count: Long = _ - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { val evaluatedExpr = expr.eval(input) if (evaluatedExpr != null) { count += 1L } } - override def eval(input: catalyst.InternalRow): Any = count + override def eval(input: InternalRow): Any = count } case class ApproxCountDistinctPartitionFunction( @@ -575,14 +575,14 @@ case class ApproxCountDistinctPartitionFunction( private val hyperLogLog = new HyperLogLog(relativeSD) - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { val evaluatedExpr = expr.eval(input) if (evaluatedExpr != null) { hyperLogLog.offer(evaluatedExpr) } } - override def eval(input: catalyst.InternalRow): Any = hyperLogLog + override def eval(input: InternalRow): Any = hyperLogLog } case class ApproxCountDistinctMergeFunction( @@ -594,12 +594,12 @@ case class ApproxCountDistinctMergeFunction( private val hyperLogLog = new HyperLogLog(relativeSD) - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { val evaluatedExpr = expr.eval(input) hyperLogLog.addAll(evaluatedExpr.asInstanceOf[HyperLogLog]) } - override def eval(input: catalyst.InternalRow): Any = hyperLogLog.cardinality() + override def eval(input: InternalRow): Any = hyperLogLog.cardinality() } case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -620,11 +620,11 @@ case class SumFunction(expr: Expression, base: AggregateExpression) extends Aggr private val addFunction = Coalesce(Seq(Add(Coalesce(Seq(sum, zero)), Cast(expr, calcType)), sum, zero)) - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { sum.update(addFunction, input) } - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { expr.dataType match { case DecimalType.Fixed(_, _) => Cast(sum, dataType).eval(null) @@ -653,7 +653,7 @@ case class CombineSumFunction(expr: Expression, base: AggregateExpression) private val addFunction = Coalesce(Seq(Add(Coalesce(Seq(sum, zero)), Cast(expr, calcType)), sum, zero)) - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { val result = expr.eval(input) // partial sum result can be null only when no input rows present if(result != null) { @@ -661,7 +661,7 @@ case class CombineSumFunction(expr: Expression, base: AggregateExpression) } } - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { expr.dataType match { case DecimalType.Fixed(_, _) => Cast(sum, dataType).eval(null) @@ -677,14 +677,14 @@ case class SumDistinctFunction(expr: Expression, base: AggregateExpression) private val seen = new scala.collection.mutable.HashSet[Any]() - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { val evaluatedExpr = expr.eval(input) if (evaluatedExpr != null) { seen += evaluatedExpr } } - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { if (seen.size == 0) { null } else { @@ -708,14 +708,14 @@ case class CountDistinctFunction( @transient val distinctValue = new InterpretedProjection(expr) - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { val evaluatedExpr = distinctValue(input) if (!evaluatedExpr.anyNull) { seen.add(evaluatedExpr) } } - override def eval(input: catalyst.InternalRow): Any = seen.size.toLong + override def eval(input: InternalRow): Any = seen.size.toLong } case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -723,13 +723,13 @@ case class FirstFunction(expr: Expression, base: AggregateExpression) extends Ag var result: Any = null - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { if (result == null) { result = expr.eval(input) } } - override def eval(input: catalyst.InternalRow): Any = result + override def eval(input: InternalRow): Any = result } case class LastFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -737,11 +737,11 @@ case class LastFunction(expr: Expression, base: AggregateExpression) extends Agg var result: Any = null - override def update(input: catalyst.InternalRow): Unit = { + override def update(input: InternalRow): Unit = { result = input } - override def eval(input: catalyst.InternalRow): Any = { - if (result != null) expr.eval(result.asInstanceOf[catalyst.InternalRow]) else null + override def eval(input: InternalRow): Any = { + if (result != null) expr.eval(result.asInstanceOf[InternalRow]) else null } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 0ba2ff75aac5c..18ddac1b598e6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -30,7 +30,7 @@ abstract class UnaryArithmetic extends UnaryExpression { override def nullable: Boolean = child.nullable override def dataType: DataType = child.dataType - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val evalE = child.eval(input) if (evalE == null) { null @@ -125,7 +125,7 @@ abstract class BinaryArithmetic extends BinaryExpression { protected def checkTypesInternal(t: DataType): TypeCheckResult - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val evalE1 = left.eval(input) if(evalE1 == null) { null @@ -220,7 +220,7 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic case it: IntegralType => it.integral.asInstanceOf[Integral[Any]].quot } - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val evalE2 = right.eval(input) if (evalE2 == null || evalE2 == 0) { null @@ -280,7 +280,7 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet case i: FractionalType => i.asIntegral.asInstanceOf[Integral[Any]] } - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val evalE2 = right.eval(input) if (evalE2 == null || evalE2 == 0) { null @@ -331,7 +331,7 @@ case class MaxOf(left: Expression, right: Expression) extends BinaryArithmetic { private lazy val ordering = TypeUtils.getOrdering(dataType) - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val evalE1 = left.eval(input) val evalE2 = right.eval(input) if (evalE1 == null) { @@ -385,7 +385,7 @@ case class MinOf(left: Expression, right: Expression) extends BinaryArithmetic { private lazy val ordering = TypeUtils.getOrdering(dataType) - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val evalE1 = left.eval(input) val evalE2 = right.eval(input) if (evalE1 == null) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 244a06638f61f..54f06aaa10484 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -35,7 +35,7 @@ class IntegerHashSet extends org.apache.spark.util.collection.OpenHashSet[Int] class LongHashSet extends org.apache.spark.util.collection.OpenHashSet[Long] /** - * Java source for evaluating an [[Expression]] given a [[catalyst.InternalRow]] of input. + * Java source for evaluating an [[Expression]] given a [[InternalRow]] of input. * * @param code The sequence of statements required to evaluate the expression. * @param isNull A term that holds a boolean value representing whether the expression evaluated @@ -184,13 +184,13 @@ class CodeGenContext { } /** - * List of data types that have special accessors and setters in [[catalyst.InternalRow]]. + * List of data types that have special accessors and setters in [[InternalRow]]. */ val nativeTypes = Seq(IntegerType, BooleanType, LongType, DoubleType, FloatType, ShortType, ByteType) /** - * Returns true if the data type has a special accessor and setter in [[catalyst.InternalRow]]. + * Returns true if the data type has a special accessor and setter in [[InternalRow]]. */ def isNativeType(dt: DataType): Boolean = nativeTypes.contains(dt) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index 35cb954c54308..573a9ea0a5471 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.expressions.codegen -import org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.expressions._ // MutableProjection is not accessible in Java @@ -25,7 +24,7 @@ abstract class BaseMutableProjection extends MutableProjection {} /** * Generates byte code that produces a [[MutableRow]] object that can update itself based on a new - * input [[catalyst.InternalRow]] for a fixed set of [[Expression Expressions]]. + * input [[InternalRow]] for a fixed set of [[Expression Expressions]]. */ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], () => MutableProjection] { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index db5d570aeb6d4..3e9ee60f33037 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -19,15 +19,15 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.Logging import org.apache.spark.annotation.Private -import org.apache.spark.sql.{catalyst, Row} +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions._ /** * Inherits some default implementation for Java from `Ordering[Row]` */ @Private -class BaseOrdering extends Ordering[catalyst.InternalRow] { - def compare(a: catalyst.InternalRow, b: catalyst.InternalRow): Int = { +class BaseOrdering extends Ordering[InternalRow] { + def compare(a: InternalRow, b: InternalRow): Int = { throw new UnsupportedOperationException } } @@ -37,7 +37,7 @@ class BaseOrdering extends Ordering[catalyst.InternalRow] { * [[Expression Expressions]]. */ object GenerateOrdering - extends CodeGenerator[Seq[SortOrder], Ordering[catalyst.InternalRow]] with Logging { + extends CodeGenerator[Seq[SortOrder], Ordering[InternalRow]] with Logging { import scala.reflect.runtime.universe._ protected def canonicalize(in: Seq[SortOrder]): Seq[SortOrder] = @@ -46,7 +46,7 @@ object GenerateOrdering protected def bind(in: Seq[SortOrder], inputSchema: Seq[Attribute]): Seq[SortOrder] = in.map(BindReferences.bindReference(_, inputSchema)) - protected def create(ordering: Seq[SortOrder]): Ordering[catalyst.InternalRow] = { + protected def create(ordering: Seq[SortOrder]): Ordering[InternalRow] = { val a = newTermName("a") val b = newTermName("b") val ctx = newCodeGenContext() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala index 9e191dc2e9422..dad4364bdd94a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratePredicate.scala @@ -24,20 +24,20 @@ import org.apache.spark.sql.catalyst.expressions._ * Interface for generated predicate */ abstract class Predicate { - def eval(r: catalyst.InternalRow): Boolean + def eval(r: InternalRow): Boolean } /** * Generates bytecode that evaluates a boolean [[Expression]] on a given input [[InternalRow]]. */ -object GeneratePredicate extends CodeGenerator[Expression, (catalyst.InternalRow) => Boolean] { +object GeneratePredicate extends CodeGenerator[Expression, (InternalRow) => Boolean] { protected def canonicalize(in: Expression): Expression = ExpressionCanonicalizer.execute(in) protected def bind(in: Expression, inputSchema: Seq[Attribute]): Expression = BindReferences.bindReference(in, inputSchema) - protected def create(predicate: Expression): ((catalyst.InternalRow) => Boolean) = { + protected def create(predicate: Expression): ((InternalRow) => Boolean) = { val ctx = newCodeGenContext() val eval = predicate.gen(ctx) val code = s""" @@ -66,6 +66,6 @@ object GeneratePredicate extends CodeGenerator[Expression, (catalyst.InternalRow // fetch the only one method `generate(Expression[])` val m = c.getDeclaredMethods()(0) val p = m.invoke(c.newInstance(), ctx.references.toArray).asInstanceOf[Predicate] - (r: catalyst.InternalRow) => p.eval(r) + (r: InternalRow) => p.eval(r) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index a6913cc03ca20..1aaf9b309efc3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -42,7 +42,7 @@ case class CreateArray(children: Seq[Expression]) extends Expression { override def nullable: Boolean = false - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { children.map(_.eval(input)) } @@ -70,7 +70,7 @@ case class CreateStruct(children: Seq[NamedExpression]) extends Expression { override def nullable: Boolean = false - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { InternalRow(children.map(_.eval(input)): _*) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala index a119c313007c8..1d7393d3d91f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionals.scala @@ -43,7 +43,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi override def dataType: DataType = trueValue.dataType - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { if (true == predicate.eval(input)) { trueValue.eval(input) } else { @@ -138,7 +138,7 @@ case class CaseWhen(branches: Seq[Expression]) extends CaseWhenLike { } /** Written in imperative fashion for performance considerations. */ - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val len = branchesArr.length var i = 0 // If all branches fail and an elseVal is not provided, the whole statement @@ -230,7 +230,7 @@ case class CaseKeyWhen(key: Expression, branches: Seq[Expression]) extends CaseW } /** Written in imperative fashion for performance considerations. */ - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val evaluatedKey = key.eval(input) val len = branchesArr.length var i = 0 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala index de8b66bc3bcbd..2bc893af02641 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalFunctions.scala @@ -29,7 +29,7 @@ case class UnscaledValue(child: Expression) extends UnaryExpression { override def nullable: Boolean = child.nullable override def toString: String = s"UnscaledValue($child)" - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val childResult = child.eval(input) if (childResult == null) { null @@ -51,7 +51,7 @@ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends Un override def nullable: Boolean = child.nullable override def toString: String = s"MakeDecimal($child,$precision,$scale)" - override def eval(input: catalyst.InternalRow): Decimal = { + override def eval(input: InternalRow): Decimal = { val childResult = child.eval(input) if (childResult == null) { null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index a80c255a296af..f30cb42d12b83 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -54,13 +54,13 @@ abstract class Generator extends Expression { def elementTypes: Seq[(DataType, Boolean)] /** Should be implemented by child classes to perform specific Generators. */ - override def eval(input: catalyst.InternalRow): TraversableOnce[catalyst.InternalRow] + override def eval(input: InternalRow): TraversableOnce[InternalRow] /** * Notifies that there are no more rows to process, clean up code, and additional * rows can be made here. */ - def terminate(): TraversableOnce[catalyst.InternalRow] = Nil + def terminate(): TraversableOnce[InternalRow] = Nil } /** @@ -68,22 +68,22 @@ abstract class Generator extends Expression { */ case class UserDefinedGenerator( elementTypes: Seq[(DataType, Boolean)], - function: catalyst.InternalRow => TraversableOnce[catalyst.InternalRow], + function: InternalRow => TraversableOnce[InternalRow], children: Seq[Expression]) extends Generator { @transient private[this] var inputRow: InterpretedProjection = _ - @transient private[this] var convertToScala: (catalyst.InternalRow) => catalyst.InternalRow = _ + @transient private[this] var convertToScala: (InternalRow) => InternalRow = _ private def initializeConverters(): Unit = { inputRow = new InterpretedProjection(children) convertToScala = { val inputSchema = StructType(children.map(e => StructField(e.simpleString, e.dataType, true))) CatalystTypeConverters.createToScalaConverter(inputSchema) - }.asInstanceOf[(catalyst.InternalRow => catalyst.InternalRow)] + }.asInstanceOf[(InternalRow => InternalRow)] } - override def eval(input: catalyst.InternalRow): TraversableOnce[catalyst.InternalRow] = { + override def eval(input: InternalRow): TraversableOnce[InternalRow] = { if (inputRow == null) { initializeConverters() } @@ -109,7 +109,7 @@ case class Explode(child: Expression) case MapType(kt, vt, valueContainsNull) => (kt, false) :: (vt, valueContainsNull) :: Nil } - override def eval(input: catalyst.InternalRow): TraversableOnce[catalyst.InternalRow] = { + override def eval(input: InternalRow): TraversableOnce[InternalRow] = { child.dataType match { case ArrayType(_, _) => val inputArray = child.eval(input).asInstanceOf[Seq[Any]] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index d8fff2b84d585..6c86a47ba200c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -88,7 +88,7 @@ case class Literal protected (value: Any, dataType: DataType) extends LeafExpres case _ => false } - override def eval(input: catalyst.InternalRow): Any = value + override def eval(input: InternalRow): Any = value override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { // change the isNull and primitive to consts, to inline them @@ -143,9 +143,9 @@ case class Literal protected (value: Any, dataType: DataType) extends LeafExpres case class MutableLiteral(var value: Any, dataType: DataType, nullable: Boolean = true) extends LeafExpression { - def update(expression: Expression, input: catalyst.InternalRow): Unit = { + def update(expression: Expression, input: InternalRow): Unit = { value = expression.eval(input) } - override def eval(input: catalyst.InternalRow): Any = value + override def eval(input: InternalRow): Any = value } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala index 6f90d607ddbcc..42c596b5b31ab 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/math.scala @@ -35,7 +35,7 @@ abstract class LeafMathExpression(c: Double, name: String) override def nullable: Boolean = false override def toString: String = s"$name()" - override def eval(input: catalyst.InternalRow): Any = c + override def eval(input: InternalRow): Any = c override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { s""" @@ -61,7 +61,7 @@ abstract class UnaryMathExpression(f: Double => Double, name: String) override def nullable: Boolean = true override def toString: String = s"$name($child)" - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val evalE = child.eval(input) if (evalE == null) { null @@ -104,7 +104,7 @@ abstract class BinaryMathExpression(f: (Double, Double) => Double, name: String) override def dataType: DataType = DoubleType - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val evalE1 = left.eval(input) if (evalE1 == null) { null @@ -216,7 +216,7 @@ case class ToRadians(child: Expression) extends UnaryMathExpression(math.toRadia case class Atan2(left: Expression, right: Expression) extends BinaryMathExpression(math.atan2, "ATAN2") { - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { val evalE1 = left.eval(input) if (evalE1 == null) { null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 20505129e96c3..f22c8a7f6a374 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -115,7 +115,7 @@ case class Alias(child: Expression, name: String)( // Alias(Generator, xx) need to be transformed into Generate(generator, ...) override lazy val resolved = childrenResolved && !child.isInstanceOf[Generator] - override def eval(input: catalyst.InternalRow): Any = child.eval(input) + override def eval(input: InternalRow): Any = child.eval(input) override def gen(ctx: CodeGenContext): GeneratedExpressionCode = child.gen(ctx) @@ -231,7 +231,7 @@ case class AttributeReference( } // Unresolved attributes are transient at compile time and don't get evaluated during execution. - override def eval(input: catalyst.InternalRow = null): Any = + override def eval(input: InternalRow = null): Any = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") override def toString: String = s"$name#${exprId.id}$typeSuffix" @@ -253,7 +253,7 @@ case class PrettyAttribute(name: String) extends Attribute with trees.LeafNode[E override def withName(newName: String): Attribute = throw new UnsupportedOperationException override def qualifiers: Seq[String] = throw new UnsupportedOperationException override def exprId: ExprId = throw new UnsupportedOperationException - override def eval(input: catalyst.InternalRow): Any = throw new UnsupportedOperationException + override def eval(input: InternalRow): Any = throw new UnsupportedOperationException override def nullable: Boolean = throw new UnsupportedOperationException override def dataType: DataType = NullType } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index 292d626f019f4..0d06589a795b1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -17,10 +17,9 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst -import org.apache.spark.sql.catalyst.expressions.codegen.{GeneratedExpressionCode, CodeGenContext} -import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode} +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.types.DataType case class Coalesce(children: Seq[Expression]) extends Expression { @@ -44,7 +43,7 @@ case class Coalesce(children: Seq[Expression]) extends Expression { this, s"Coalesce cannot have children of different types. $childTypes") } - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { var i = 0 var result: Any = null val childIterator = children.iterator @@ -78,7 +77,7 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr override def foldable: Boolean = child.foldable override def nullable: Boolean = false - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { child.eval(input) == null } @@ -97,7 +96,7 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E override def nullable: Boolean = false override def toString: String = s"IS NOT NULL $child" - override def eval(input: catalyst.InternalRow): Any = { + override def eval(input: InternalRow): Any = { child.eval(input) != null } @@ -119,7 +118,7 @@ case class AtLeastNNonNulls(n: Int, children: Seq[Expression]) extends Predicate private[this] val childrenArray = children.toArray - override def eval(input: catalyst.InternalRow): Boolean = { + override def eval(input: InternalRow): Boolean = { var numNonNulls = 0 var i = 0 while (i < childrenArray.length && numNonNulls < n) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala index c2e57b4715a79..d24d74e7b82ad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst -import org.apache.spark.sql.catalyst - /** * A set of classes that can be used to represent trees of relational expressions. A key goal of * the expression library is to hide the details of naming and scoping from developers who want to @@ -51,9 +49,9 @@ import org.apache.spark.sql.catalyst */ package object expressions { - type InternalRow = catalyst.InternalRow + type InternalRow = org.apache.spark.sql.catalyst.InternalRow - val InternalRow = catalyst.InternalRow + val InternalRow = org.apache.spark.sql.catalyst.InternalRow /** * Converts a [[InternalRow]] to another Row given a sequence of expression that define each diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index f1acdfeea5793..9ca168881c5b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1060,7 +1060,7 @@ class DataFrame private[sql]( def rowFunction(row: Row): TraversableOnce[InternalRow] = { f(row(0).asInstanceOf[A]).map(o => - catalyst.InternalRow(CatalystTypeConverters.convertToCatalyst(o, dataType))) + InternalRow(CatalystTypeConverters.convertToCatalyst(o, dataType))) } val generator = UserDefinedGenerator(elementTypes, rowFunction, apply(inputColumn).expr :: Nil) @@ -1232,11 +1232,11 @@ class DataFrame private[sql]( // Pivot the data so each summary is one row row.grouped(outputCols.size).toSeq.zip(statistics).map { case (aggregation, (statistic, _)) => - catalyst.InternalRow(statistic :: aggregation.toList: _*) + InternalRow(statistic :: aggregation.toList: _*) } } else { // If there are no output columns, just output a single column that contains the stats. - statistics.map { case (name, _) => catalyst.InternalRow(name) } + statistics.map { case (name, _) => InternalRow(name) } } // All columns are string type diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala index 19aef9978e732..bce0e8d70a57b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashOuterJoin.scala @@ -90,7 +90,7 @@ case class HashOuterJoin( case r if boundCondition(joinedRow.withRight(r)) => joinedRow.copy() } if (temp.size == 0) { - joinedRow.withRight(rightNullRow).copy.asInstanceOf[InternalRow] :: Nil + joinedRow.withRight(rightNullRow).copy :: Nil } else { temp } From af31335adce13e1452ce1990496c9bfac9778b5c Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 13 Jun 2015 16:14:24 -0700 Subject: [PATCH 03/13] [SPARK-8319] [CORE] [SQL] Update logic related to key orderings in shuffle dependencies This patch updates two pieces of logic that are related to handling of keyOrderings in ShuffleDependencies: - The Tungsten ShuffleManager falls back to regular SortShuffleManager whenever the shuffle dependency specifies a key ordering, but technically we only need to fall back when an aggregator is also specified. This patch updates the fallback logic to reflect this so that the Tungsten optimizations can apply to more workloads. - The SQL Exchange operator performs defensive copying of shuffle inputs when a key ordering is specified, but this is unnecessary. The copying was added to guard against cases where ExternalSorter would buffer non-serialized records in memory. When ExternalSorter is configured without an aggregator, it uses the following logic to determine whether to buffer records in a serialized or deserialized format: ```scala private val useSerializedPairBuffer = ordering.isEmpty && conf.getBoolean("spark.shuffle.sort.serializeMapOutputs", true) && ser.supportsRelocationOfSerializedObjects ``` The `newOrdering.isDefined` branch in `ExternalSorter.needToCopyObjectsBeforeShuffle`, removed by this patch, is not necessary: - It was checked even if we weren't using sort-based shuffle, but this was unnecessary because only SortShuffleManager performs map-side sorting. - Map-side sorting during shuffle writing is only performed for shuffles that perform map-side aggregation as part of the shuffle (to see this, look at how SortShuffleWriter constructs ExternalSorter). Since SQL never pushes aggregation into Spark's shuffle, we can guarantee that both the aggregator and ordering will be empty and Spark SQL always uses serializers that support relocation, so sort-shuffle will use the serialized pair buffer unless the user has explicitly disabled it via the SparkConf feature-flag. Therefore, I think my optimization in Exchange should be safe. Author: Josh Rosen Closes #6773 from JoshRosen/SPARK-8319 and squashes the following commits: 7a14129 [Josh Rosen] Revise comments; add handler to guard against future ShuffleManager implementations 07bb2c9 [Josh Rosen] Update comment to clarify circumstances under which shuffle operates on serialized records 269089a [Josh Rosen] Avoid unnecessary copy in SQL Exchange 34e526e [Josh Rosen] Enable Tungsten shuffle for non-agg shuffles w/ key orderings --- .../shuffle/unsafe/UnsafeShuffleManager.scala | 3 --- .../unsafe/UnsafeShuffleManagerSuite.scala | 19 ++++++++++--------- .../apache/spark/sql/execution/Exchange.scala | 19 +++++++++++-------- 3 files changed, 21 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala index f2bfef376d3ca..df7bbd64247dd 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManager.scala @@ -56,9 +56,6 @@ private[spark] object UnsafeShuffleManager extends Logging { } else if (dependency.aggregator.isDefined) { log.debug(s"Can't use UnsafeShuffle for shuffle $shufId because an aggregator is defined") false - } else if (dependency.keyOrdering.isDefined) { - log.debug(s"Can't use UnsafeShuffle for shuffle $shufId because a key ordering is defined") - false } else if (dependency.partitioner.numPartitions > MAX_SHUFFLE_OUTPUT_PARTITIONS) { log.debug(s"Can't use UnsafeShuffle for shuffle $shufId because it has more than " + s"$MAX_SHUFFLE_OUTPUT_PARTITIONS partitions") diff --git a/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManagerSuite.scala index a73e94e05575e..6727934d8c7ca 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/unsafe/UnsafeShuffleManagerSuite.scala @@ -76,6 +76,15 @@ class UnsafeShuffleManagerSuite extends SparkFunSuite with Matchers { mapSideCombine = false ))) + // Shuffles with key orderings are supported as long as no aggregator is specified + assert(canUseUnsafeShuffle(shuffleDep( + partitioner = new HashPartitioner(2), + serializer = kryo, + keyOrdering = Some(mock(classOf[Ordering[Any]])), + aggregator = None, + mapSideCombine = false + ))) + } test("unsupported shuffle dependencies") { @@ -100,14 +109,7 @@ class UnsafeShuffleManagerSuite extends SparkFunSuite with Matchers { mapSideCombine = false ))) - // We do not support shuffles that perform any kind of aggregation or sorting of keys - assert(!canUseUnsafeShuffle(shuffleDep( - partitioner = new HashPartitioner(2), - serializer = kryo, - keyOrdering = Some(mock(classOf[Ordering[Any]])), - aggregator = None, - mapSideCombine = false - ))) + // We do not support shuffles that perform aggregation assert(!canUseUnsafeShuffle(shuffleDep( partitioner = new HashPartitioner(2), serializer = kryo, @@ -115,7 +117,6 @@ class UnsafeShuffleManagerSuite extends SparkFunSuite with Matchers { aggregator = Some(mock(classOf[Aggregator[Any, Any, Any]])), mapSideCombine = false ))) - // We do not support shuffles that perform any kind of aggregation or sorting of keys assert(!canUseUnsafeShuffle(shuffleDep( partitioner = new HashPartitioner(2), serializer = kryo, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index c9a188309a4d8..edc64a03335d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.shuffle.unsafe.UnsafeShuffleManager import org.apache.spark.sql.SQLContext @@ -81,11 +82,7 @@ case class Exchange( shuffleManager.isInstanceOf[UnsafeShuffleManager] val bypassMergeThreshold = conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) val serializeMapOutputs = conf.getBoolean("spark.shuffle.sort.serializeMapOutputs", true) - if (newOrdering.nonEmpty) { - // If a new ordering is required, then records will be sorted with Spark's `ExternalSorter`, - // which requires a defensive copy. - true - } else if (sortBasedShuffleOn) { + if (sortBasedShuffleOn) { val bypassIsSupported = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] if (bypassIsSupported && partitioner.numPartitions <= bypassMergeThreshold) { // If we're using the original SortShuffleManager and the number of output partitions is @@ -96,8 +93,11 @@ case class Exchange( } else if (serializeMapOutputs && serializer.supportsRelocationOfSerializedObjects) { // SPARK-4550 extended sort-based shuffle to serialize individual records prior to sorting // them. This optimization is guarded by a feature-flag and is only applied in cases where - // shuffle dependency does not specify an ordering and the record serializer has certain - // properties. If this optimization is enabled, we can safely avoid the copy. + // shuffle dependency does not specify an aggregator or ordering and the record serializer + // has certain properties. If this optimization is enabled, we can safely avoid the copy. + // + // Exchange never configures its ShuffledRDDs with aggregators or key orderings, so we only + // need to check whether the optimization is enabled and supported by our serializer. // // This optimization also applies to UnsafeShuffleManager (added in SPARK-7081). false @@ -108,9 +108,12 @@ case class Exchange( // both cases, we must copy. true } - } else { + } else if (shuffleManager.isInstanceOf[HashShuffleManager]) { // We're using hash-based shuffle, so we don't need to copy. false + } else { + // Catch-all case to safely handle any future ShuffleManager implementations. + true } } From ddec45279ed1061f4c05fd0760309a53581d03f5 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 13 Jun 2015 16:39:52 -0700 Subject: [PATCH 04/13] [SPARK-8052] [SQL] Use java.math.BigDecimal for casting String to Decimal instead of using toDouble JIRA: https://issues.apache.org/jira/browse/SPARK-8052 Author: Liang-Chi Hsieh Closes #6645 from viirya/cast_string_integraltype and squashes the following commits: e19c6a3 [Liang-Chi Hsieh] For comment. c3e472a [Liang-Chi Hsieh] Add test. 7ced9b0 [Liang-Chi Hsieh] Use java.math.BigDecimal for casting String to Decimal instead of using toDouble. --- .../org/apache/spark/sql/catalyst/expressions/Cast.scala | 3 ++- .../org/apache/spark/sql/hive/execution/SQLQuerySuite.scala | 4 ++++ 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 05a04bdff9b3e..b20086bcc48b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.expressions +import java.math.{BigDecimal => JavaBigDecimal} import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} @@ -320,7 +321,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w private[this] def castToDecimal(from: DataType, target: DecimalType): Any => Any = from match { case StringType => buildCast[UTF8String](_, s => try { - changePrecision(Decimal(s.toString.toDouble), target) + changePrecision(Decimal(new JavaBigDecimal(s.toString)), target) } catch { case _: NumberFormatException => null }) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index c8e5e246322df..f8908760cc897 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -875,6 +875,10 @@ class SQLQuerySuite extends QueryTest { } } + test("Cast STRING to BIGINT") { + checkAnswer(sql("SELECT CAST('775983671874188101' as BIGINT)"), Row(775983671874188101L)) + } + // `Math.exp(1.0)` has different result for different jdk version, so not use createQueryTest test("udf_java_method") { checkAnswer(sql( From a138953391975886c88bfe81d4ce6b6dd189cd32 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 13 Jun 2015 17:10:13 -0700 Subject: [PATCH 05/13] [SPARK-8347][SQL] Add unit tests for abs. Also addressed code review feedback from #6754 Author: Reynold Xin Closes #6803 from rxin/abs and squashes the following commits: d07beba [Reynold Xin] [SPARK-8347] Add unit tests for abs. --- .../catalyst/analysis/HiveTypeCoercion.scala | 6 +++--- .../expressions/ArithmeticExpressionSuite.scala | 17 ++++++++++++++++- .../spark/sql/ColumnExpressionSuite.scala | 17 ----------------- .../apache/spark/sql/MathExpressionsSuite.scala | 12 ++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 12 ------------ 5 files changed, 31 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 6ed192360dd62..e7bf7cc1f1313 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -672,13 +672,13 @@ trait HiveTypeCoercion { findTightestCommonTypeToString(left.dataType, right.dataType).map { widestType => val newLeft = if (left.dataType == widestType) left else Cast(left, widestType) val newRight = if (right.dataType == widestType) right else Cast(right, widestType) - i.makeCopy(Array(pred, newLeft, newRight)) + If(pred, newLeft, newRight) }.getOrElse(i) // If there is no applicable conversion, leave expression unchanged. // Convert If(null literal, _, _) into boolean type. // In the optimizer, we should short-circuit this directly into false value. - case i @ If(pred, left, right) if pred.dataType == NullType => - i.makeCopy(Array(Literal.create(null, BooleanType), left, right)) + case If(pred, left, right) if pred.dataType == NullType => + If(Literal.create(null, BooleanType), left, right) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index e1afa81a7a82f..5ff1bca260b24 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -21,7 +21,7 @@ import org.scalatest.Matchers._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.types.{DoubleType, IntegerType} +import org.apache.spark.sql.types.{Decimal, DoubleType, IntegerType} class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -75,6 +75,21 @@ class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkDoubleEvaluation(c3 % c2, (1.1 +- 0.001), row) } + test("Abs") { + def testAbs(convert: (Int) => Any): Unit = { + checkEvaluation(Abs(Literal(convert(0))), convert(0)) + checkEvaluation(Abs(Literal(convert(1))), convert(1)) + checkEvaluation(Abs(Literal(convert(-1))), convert(1)) + } + testAbs(_.toByte) + testAbs(_.toShort) + testAbs(identity) + testAbs(_.toLong) + testAbs(_.toFloat) + testAbs(_.toDouble) + testAbs(Decimal(_)) + } + test("Divide") { checkEvaluation(Divide(Literal(2), Literal(1)), 2) checkEvaluation(Divide(Literal(1.0), Literal(2.0)), 0.5) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index efcdae5bce031..5a08578e7ba4b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -369,23 +369,6 @@ class ColumnExpressionSuite extends QueryTest { ) } - test("abs") { - checkAnswer( - testData.select(abs('key)).orderBy('key.asc), - (1 to 100).map(n => Row(n)) - ) - - checkAnswer( - negativeData.select(abs('key)).orderBy('key.desc), - (1 to 100).map(n => Row(n)) - ) - - checkAnswer( - testData.select(abs(lit(null))), - (1 to 100).map(_ => Row(null)) - ) - } - test("upper") { checkAnswer( lowerCaseData.select(upper('l)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala index 6561c3b2322c7..faa1d1193b509 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/MathExpressionsSuite.scala @@ -236,6 +236,18 @@ class MathExpressionsSuite extends QueryTest { testOneToOneNonNegativeMathFunction(log1p, math.log1p) } + test("abs") { + val input = + Seq[(java.lang.Double, java.lang.Double)]((null, null), (0.0, 0.0), (1.5, 1.5), (-2.5, 2.5)) + checkAnswer( + input.toDF("key", "value").select(abs($"key").alias("a")).sort("a"), + input.map(pair => Row(pair._2))) + + checkAnswer( + input.toDF("key", "value").selectExpr("abs(key) a").sort("a"), + input.map(pair => Row(pair._2))) + } + test("log2") { val df = Seq((1, 2)).toDF("a", "b") checkAnswer( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 6898d584414ba..d1520b757e57b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -178,18 +178,6 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll with SQLTestUtils { Seq(Row("1"), Row("2"))) } - test("SPARK-3176 Added Parser of SQL ABS()") { - checkAnswer( - sql("SELECT ABS(-1.3)"), - Row(1.3)) - checkAnswer( - sql("SELECT ABS(0.0)"), - Row(0.0)) - checkAnswer( - sql("SELECT ABS(2.5)"), - Row(2.5)) - } - test("aggregation with codegen") { val originalValue = sqlContext.conf.codegenEnabled sqlContext.setConf(SQLConf.CODEGEN_ENABLED, "true") From 2d71ba4c8a629deab672869ac8e8b6a4b3aec479 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 13 Jun 2015 18:22:17 -0700 Subject: [PATCH 06/13] [SPARK-8349] [SQL] Use expression constructors (rather than apply) in FunctionRegistry Author: Reynold Xin Closes #6806 from rxin/gs and squashes the following commits: ed1aebb [Reynold Xin] Fixed style. c7fc3e6 [Reynold Xin] [SPARK-8349][SQL] Use expression constructors (rather than apply) in FunctionRegistry --- .../catalyst/analysis/FunctionRegistry.scala | 18 +++++++----------- .../sql/catalyst/expressions/Expression.scala | 3 +-- .../sql/catalyst/expressions/random.scala | 12 ++++-------- .../expressions/stringOperations.scala | 11 +++++------ .../spark/sql/catalyst/trees/TreeNode.scala | 10 +++++----- 5 files changed, 22 insertions(+), 32 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index 45bcbf73fae98..04e306da23e4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -158,27 +158,23 @@ object FunctionRegistry { /** See usage above. */ private def expression[T <: Expression](name: String) (implicit tag: ClassTag[T]): (String, FunctionBuilder) = { - // Use the companion class to find apply methods. - val objectClass = Class.forName(tag.runtimeClass.getName + "$") - val companionObj = objectClass.getDeclaredField("MODULE$").get(null) - - // See if we can find an apply that accepts Seq[Expression] - val varargApply = Try(objectClass.getDeclaredMethod("apply", classOf[Seq[_]])).toOption + // See if we can find a constructor that accepts Seq[Expression] + val varargCtor = Try(tag.runtimeClass.getDeclaredConstructor(classOf[Seq[_]])).toOption val builder = (expressions: Seq[Expression]) => { - if (varargApply.isDefined) { + if (varargCtor.isDefined) { // If there is an apply method that accepts Seq[Expression], use that one. - varargApply.get.invoke(companionObj, expressions).asInstanceOf[Expression] + varargCtor.get.newInstance(expressions).asInstanceOf[Expression] } else { - // Otherwise, find an apply method that matches the number of arguments, and use that. + // Otherwise, find an ctor method that matches the number of arguments, and use that. val params = Seq.fill(expressions.size)(classOf[Expression]) - val f = Try(objectClass.getDeclaredMethod("apply", params : _*)) match { + val f = Try(tag.runtimeClass.getDeclaredConstructor(params : _*)) match { case Success(e) => e case Failure(e) => throw new AnalysisException(s"Invalid number of arguments for function $name") } - f.invoke(companionObj, expressions : _*).asInstanceOf[Expression] + f.newInstance(expressions : _*).asInstanceOf[Expression] } } (name, builder) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index 61de34bfa4b3b..7427ca76b54d7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -27,8 +27,7 @@ import org.apache.spark.sql.types._ /** * If an expression wants to be exposed in the function registry (so users can call it with * "name(arguments...)", the concrete implementation must be a case class whose constructor - * arguments are all Expressions types. In addition, if it needs to support more than one - * constructor, define those constructors explicitly as apply methods in the companion object. + * arguments are all Expressions types. * * See [[Substring]] for an example. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala index 7e8033307ea4e..cc34467391b96 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala @@ -49,12 +49,10 @@ abstract class RDG(seed: Long) extends LeafExpression with Serializable { /** Generate a random column with i.i.d. uniformly distributed values in [0, 1). */ case class Rand(seed: Long) extends RDG(seed) { override def eval(input: InternalRow): Double = rng.nextDouble() -} -object Rand { - def apply(): Rand = apply(Utils.random.nextLong()) + def this() = this(Utils.random.nextLong()) - def apply(seed: Expression): Rand = apply(seed match { + def this(seed: Expression) = this(seed match { case IntegerLiteral(s) => s case _ => throw new AnalysisException("Input argument to rand must be an integer literal.") }) @@ -63,12 +61,10 @@ object Rand { /** Generate a random column with i.i.d. gaussian random distribution. */ case class Randn(seed: Long) extends RDG(seed) { override def eval(input: InternalRow): Double = rng.nextGaussian() -} -object Randn { - def apply(): Randn = apply(Utils.random.nextLong()) + def this() = this(Utils.random.nextLong()) - def apply(seed: Expression): Randn = apply(seed match { + def this(seed: Expression) = this(seed match { case IntegerLiteral(s) => s case _ => throw new AnalysisException("Input argument to rand must be an integer literal.") }) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index 8ca8d22bc4697..315c63e63c635 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import java.util.regex.Pattern import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.expressions.Substring import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -225,6 +226,10 @@ case class EndsWith(left: Expression, right: Expression) case class Substring(str: Expression, pos: Expression, len: Expression) extends Expression with ExpectsInputTypes { + def this(str: Expression, pos: Expression) = { + this(str, pos, Literal(Integer.MAX_VALUE)) + } + override def foldable: Boolean = str.foldable && pos.foldable && len.foldable override def nullable: Boolean = str.nullable || pos.nullable || len.nullable @@ -290,12 +295,6 @@ case class Substring(str: Expression, pos: Expression, len: Expression) } } -object Substring { - def apply(str: Expression, pos: Expression): Substring = { - apply(str, pos, Literal(Integer.MAX_VALUE)) - } -} - /** * A function that return the length of the given string expression. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index 36d005d0e1684..5964e3dc3d77e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -344,11 +344,11 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { * @param newArgs the new product arguments. */ def makeCopy(newArgs: Array[AnyRef]): this.type = attachTree(this, "makeCopy") { - val defaultCtor = - getClass.getConstructors - .find(_.getParameterTypes.size != 0) - .headOption - .getOrElse(sys.error(s"No valid constructor for $nodeName")) + val ctors = getClass.getConstructors.filter(_.getParameterTypes.size != 0) + if (ctors.isEmpty) { + sys.error(s"No valid constructor for $nodeName") + } + val defaultCtor = ctors.maxBy(_.getParameterTypes.size) try { CurrentOrigin.withOrigin(origin) { From 35d1267cf8e918032c92a206b22bb301bf0c806e Mon Sep 17 00:00:00 2001 From: Mike Dusenberry Date: Sat, 13 Jun 2015 21:22:46 -0700 Subject: [PATCH 07/13] [Spark-8343] [Streaming] [Docs] Improve Spark Streaming Guides. This improves the Spark Streaming Guides by fixing broken links, rewording confusing sections, fixing typos, adding missing words, etc. Author: Mike Dusenberry Closes #6801 from dusenberrymw/SPARK-8343_Improve_Spark_Streaming_Guides_MERGED and squashes the following commits: 6688090 [Mike Dusenberry] Improvements to the Spark Streaming Custom Receiver Guide, including slight rewording of confusing sections, and fixing typos & missing words. 436fbd8 [Mike Dusenberry] Bunch of improvements to the Spark Streaming Guide, including fixing broken links, slight rewording of confusing sections, fixing typos & missing words, etc. --- docs/streaming-custom-receivers.md | 26 ++- docs/streaming-programming-guide.md | 247 ++++++++++++++-------------- 2 files changed, 133 insertions(+), 140 deletions(-) diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 6a2048121f8bf..a75587a92adc7 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -4,7 +4,7 @@ title: Spark Streaming Custom Receivers --- Spark Streaming can receive streaming data from any arbitrary data source beyond -the one's for which it has in-built support (that is, beyond Flume, Kafka, Kinesis, files, sockets, etc.). +the ones for which it has built-in support (that is, beyond Flume, Kafka, Kinesis, files, sockets, etc.). This requires the developer to implement a *receiver* that is customized for receiving data from the concerned data source. This guide walks through the process of implementing a custom receiver and using it in a Spark Streaming application. Note that custom receivers can be implemented @@ -21,15 +21,15 @@ A custom receiver must extend this abstract class by implementing two methods - `onStop()`: Things to do to stop receiving data. Both `onStart()` and `onStop()` must not block indefinitely. Typically, `onStart()` would start the threads -that responsible for receiving the data and `onStop()` would ensure that the receiving by those threads +that are responsible for receiving the data, and `onStop()` would ensure that these threads receiving the data are stopped. The receiving threads can also use `isStopped()`, a `Receiver` method, to check whether they should stop receiving data. Once the data is received, that data can be stored inside Spark by calling `store(data)`, which is a method provided by the Receiver class. -There are number of flavours of `store()` which allow you store the received data -record-at-a-time or as whole collection of objects / serialized bytes. Note that the flavour of -`store()` used to implemented a receiver affects its reliability and fault-tolerance semantics. +There are a number of flavors of `store()` which allow one to store the received data +record-at-a-time or as whole collection of objects / serialized bytes. Note that the flavor of +`store()` used to implement a receiver affects its reliability and fault-tolerance semantics. This is discussed [later](#receiver-reliability) in more detail. Any exception in the receiving threads should be caught and handled properly to avoid silent @@ -60,7 +60,7 @@ class CustomReceiver(host: String, port: Int) def onStop() { // There is nothing much to do as the thread calling receive() - // is designed to stop by itself isStopped() returns false + // is designed to stop by itself if isStopped() returns false } /** Create a socket connection and receive data until receiver is stopped */ @@ -123,7 +123,7 @@ public class JavaCustomReceiver extends Receiver { public void onStop() { // There is nothing much to do as the thread calling receive() - // is designed to stop by itself isStopped() returns false + // is designed to stop by itself if isStopped() returns false } /** Create a socket connection and receive data until receiver is stopped */ @@ -167,7 +167,7 @@ public class JavaCustomReceiver extends Receiver { The custom receiver can be used in a Spark Streaming application by using `streamingContext.receiverStream()`. This will create -input DStream using data received by the instance of custom receiver, as shown below +an input DStream using data received by the instance of custom receiver, as shown below:
@@ -206,22 +206,20 @@ there are two kinds of receivers based on their reliability and fault-tolerance and stored in Spark reliably (that is, replicated successfully). Usually, implementing this receiver involves careful consideration of the semantics of source acknowledgements. -1. *Unreliable Receiver* - These are receivers for unreliable sources that do not support - acknowledging. Even for reliable sources, one may implement an unreliable receiver that - do not go into the complexity of acknowledging correctly. +1. *Unreliable Receiver* - An *unreliable receiver* does *not* send acknowledgement to a source. This can be used for sources that do not support acknowledgement, or even for reliable sources when one does not want or need to go into the complexity of acknowledgement. To implement a *reliable receiver*, you have to use `store(multiple-records)` to store data. -This flavour of `store` is a blocking call which returns only after all the given records have +This flavor of `store` is a blocking call which returns only after all the given records have been stored inside Spark. If the receiver's configured storage level uses replication (enabled by default), then this call returns after replication has completed. Thus it ensures that the data is reliably stored, and the receiver can now acknowledge the -source appropriately. This ensures that no data is caused when the receiver fails in the middle +source appropriately. This ensures that no data is lost when the receiver fails in the middle of replicating data -- the buffered data will not be acknowledged and hence will be later resent by the source. An *unreliable receiver* does not have to implement any of this logic. It can simply receive records from the source and insert them one-at-a-time using `store(single-record)`. While it does -not get the reliability guarantees of `store(multiple-records)`, it has the following advantages. +not get the reliability guarantees of `store(multiple-records)`, it has the following advantages: - The system takes care of chunking that data into appropriate sized blocks (look for block interval in the [Spark Streaming Programming Guide](streaming-programming-guide.html)). diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 836f0473597d8..1eb3b30332e4f 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -11,7 +11,7 @@ description: Spark Streaming programming guide and tutorial for Spark SPARK_VERS # Overview Spark Streaming is an extension of the core Spark API that enables scalable, high-throughput, fault-tolerant stream processing of live data streams. Data can be ingested from many sources -like Kafka, Flume, Twitter, ZeroMQ, Kinesis or TCP sockets can be processed using complex +like Kafka, Flume, Twitter, ZeroMQ, Kinesis, or TCP sockets, and can be processed using complex algorithms expressed with high-level functions like `map`, `reduce`, `join` and `window`. Finally, processed data can be pushed out to filesystems, databases, and live dashboards. In fact, you can apply Spark's @@ -52,7 +52,7 @@ different languages. **Note:** Python API for Spark Streaming has been introduced in Spark 1.2. It has all the DStream transformations and almost all the output operations available in Scala and Java interfaces. -However, it has only support for basic sources like text files and text data over sockets. +However, it only has support for basic sources like text files and text data over sockets. APIs for additional sources, like Kafka and Flume, will be available in the future. Further information about available features in the Python API are mentioned throughout this document; look out for the tag @@ -69,10 +69,10 @@ do is as follows.
-First, we import the names of the Spark Streaming classes, and some implicit -conversions from StreamingContext into our environment, to add useful methods to +First, we import the names of the Spark Streaming classes and some implicit +conversions from StreamingContext into our environment in order to add useful methods to other classes we need (like DStream). [StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) is the -main entry point for all streaming functionality. We create a local StreamingContext with two execution threads, and batch interval of 1 second. +main entry point for all streaming functionality. We create a local StreamingContext with two execution threads, and a batch interval of 1 second. {% highlight scala %} import org.apache.spark._ @@ -96,7 +96,7 @@ val lines = ssc.socketTextStream("localhost", 9999) This `lines` DStream represents the stream of data that will be received from the data server. Each record in this DStream is a line of text. Next, we want to split the lines by -space into words. +space characters into words. {% highlight scala %} // Split each line into words @@ -463,7 +463,7 @@ receive it there. However, for local testing and unit tests, you can pass "local in-process (detects the number of cores in the local system). Note that this internally creates a [SparkContext](api/scala/index.html#org.apache.spark.SparkContext) (starting point of all Spark functionality) which can be accessed as `ssc.sparkContext`. The batch interval must be set based on the latency requirements of your application -and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) +and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-interval) section for more details. A `StreamingContext` object can also be created from an existing `SparkContext` object. @@ -498,7 +498,7 @@ receive it there. However, for local testing and unit tests, you can pass "local in-process. Note that this internally creates a [JavaSparkContext](api/java/index.html?org/apache/spark/api/java/JavaSparkContext.html) (starting point of all Spark functionality) which can be accessed as `ssc.sparkContext`. The batch interval must be set based on the latency requirements of your application -and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) +and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-interval) section for more details. A `JavaStreamingContext` object can also be created from an existing `JavaSparkContext`. @@ -531,7 +531,7 @@ receive it there. However, for local testing and unit tests, you can pass "local in-process (detects the number of cores in the local system). The batch interval must be set based on the latency requirements of your application -and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-size) +and available cluster resources. See the [Performance Tuning](#setting-the-right-batch-interval) section for more details.
@@ -549,7 +549,7 @@ After a context is defined, you have to do the following. - Once a context has been started, no new streaming computations can be set up or added to it. - Once a context has been stopped, it cannot be restarted. - Only one StreamingContext can be active in a JVM at the same time. -- stop() on StreamingContext also stops the SparkContext. To stop only the StreamingContext, set optional parameter of `stop()` called `stopSparkContext` to false. +- stop() on StreamingContext also stops the SparkContext. To stop only the StreamingContext, set the optional parameter of `stop()` called `stopSparkContext` to false. - A SparkContext can be re-used to create multiple StreamingContexts, as long as the previous StreamingContext is stopped (without stopping the SparkContext) before the next StreamingContext is created. *** @@ -583,7 +583,7 @@ the `flatMap` operation is applied on each RDD in the `lines` DStream to generat These underlying RDD transformations are computed by the Spark engine. The DStream operations -hide most of these details and provide the developer with higher-level API for convenience. +hide most of these details and provide the developer with a higher-level API for convenience. These operations are discussed in detail in later sections. *** @@ -600,7 +600,7 @@ data from a source and stores it in Spark's memory for processing. Spark Streaming provides two categories of built-in streaming sources. - *Basic sources*: Sources directly available in the StreamingContext API. - Example: file systems, socket connections, and Akka actors. + Examples: file systems, socket connections, and Akka actors. - *Advanced sources*: Sources like Kafka, Flume, Kinesis, Twitter, etc. are available through extra utility classes. These require linking against extra dependencies as discussed in the [linking](#linking) section. @@ -610,11 +610,11 @@ We are going to discuss some of the sources present in each category later in th Note that, if you want to receive multiple streams of data in parallel in your streaming application, you can create multiple input DStreams (discussed further in the [Performance Tuning](#level-of-parallelism-in-data-receiving) section). This will -create multiple receivers which will simultaneously receive multiple data streams. But note that -Spark worker/executor as a long-running task, hence it occupies one of the cores allocated to the -Spark Streaming application. Hence, it is important to remember that Spark Streaming application +create multiple receivers which will simultaneously receive multiple data streams. But note that a +Spark worker/executor is a long-running task, hence it occupies one of the cores allocated to the +Spark Streaming application. Therefore, it is important to remember that a Spark Streaming application needs to be allocated enough cores (or threads, if running locally) to process the received data, -as well as, to run the receiver(s). +as well as to run the receiver(s). ##### Points to remember {:.no_toc} @@ -623,13 +623,13 @@ as well as, to run the receiver(s). Either of these means that only one thread will be used for running tasks locally. If you are using a input DStream based on a receiver (e.g. sockets, Kafka, Flume, etc.), then the single thread will be used to run the receiver, leaving no thread for processing the received data. Hence, when - running locally, always use "local[*n*]" as the master URL where *n* > number of receivers to run - (see [Spark Properties](configuration.html#spark-properties.html) for information on how to set + running locally, always use "local[*n*]" as the master URL, where *n* > number of receivers to run + (see [Spark Properties](configuration.html#spark-properties) for information on how to set the master). - Extending the logic to running on a cluster, the number of cores allocated to the Spark Streaming - application must be more than the number of receivers. Otherwise the system will receive data, but - not be able to process them. + application must be more than the number of receivers. Otherwise the system will receive data, but + not be able to process it. ### Basic Sources {:.no_toc} @@ -639,7 +639,7 @@ which creates a DStream from text data received over a TCP socket connection. Besides sockets, the StreamingContext API provides methods for creating DStreams from files and Akka actors as input sources. -- **File Streams:** For reading data from files on any file system compatible with the HDFS API (that is, HDFS, S3, NFS, etc.), a DStream can be created as +- **File Streams:** For reading data from files on any file system compatible with the HDFS API (that is, HDFS, S3, NFS, etc.), a DStream can be created as:
@@ -687,9 +687,9 @@ out of these sources, *only* Kafka is available in the Python API. We will add m This category of sources require interfacing with external non-Spark libraries, some of them with complex dependencies (e.g., Kafka and Flume). Hence, to minimize issues related to version conflicts -of dependencies, the functionality to create DStreams from these sources have been moved to separate -libraries, that can be [linked](#linking) to explicitly when necessary. For example, if you want to -create a DStream using data from Twitter's stream of tweets, you have to do the following. +of dependencies, the functionality to create DStreams from these sources has been moved to separate +libraries that can be [linked](#linking) to explicitly when necessary. For example, if you want to +create a DStream using data from Twitter's stream of tweets, you have to do the following: 1. *Linking*: Add the artifact `spark-streaming-twitter_{{site.SCALA_BINARY_VERSION}}` to the SBT/Maven project dependencies. @@ -719,7 +719,7 @@ TwitterUtils.createStream(jssc); Note that these advanced sources are not available in the Spark shell, hence applications based on these advanced sources cannot be tested in the shell. If you really want to use them in the Spark shell you will have to download the corresponding Maven artifact's JAR along with its dependencies -and it in the classpath. +and add it to the classpath. Some of these advanced sources are as follows. @@ -743,7 +743,7 @@ Some of these advanced sources are as follows. Python API This is not yet supported in Python. -Input DStreams can also be created out of custom data sources. All you have to do is implement an +Input DStreams can also be created out of custom data sources. All you have to do is implement a user-defined **receiver** (see next section to understand what that is) that can receive data from the custom sources and push it into Spark. See the [Custom Receiver Guide](streaming-custom-receivers.html) for details. @@ -753,14 +753,12 @@ Guide](streaming-custom-receivers.html) for details. There can be two kinds of data sources based on their *reliability*. Sources (like Kafka and Flume) allow the transferred data to be acknowledged. If the system receiving -data from these *reliable* sources acknowledge the received data correctly, it can be ensured -that no data gets lost due to any kind of failure. This leads to two kinds of receivers. +data from these *reliable* sources acknowledges the received data correctly, it can be ensured +that no data will be lost due to any kind of failure. This leads to two kinds of receivers: -1. *Reliable Receiver* - A *reliable receiver* correctly acknowledges a reliable - source that the data has been received and stored in Spark with replication. -1. *Unreliable Receiver* - These are receivers for sources that do not support acknowledging. Even - for reliable sources, one may implement an unreliable receiver that do not go into the complexity - of acknowledging correctly. +1. *Reliable Receiver* - A *reliable receiver* correctly sends acknowledgment to a reliable + source when the data has been received and stored in Spark with replication. +1. *Unreliable Receiver* - An *unreliable receiver* does *not* send acknowledgment to a source. This can be used for sources that do not support acknowledgment, or even for reliable sources when one does not want or need to go into the complexity of acknowledgment. The details of how to write a reliable receiver are discussed in the [Custom Receiver Guide](streaming-custom-receivers.html). @@ -828,7 +826,7 @@ Some of the common ones are as follows. cogroup(otherStream, [numTasks]) - When called on DStream of (K, V) and (K, W) pairs, return a new DStream of + When called on a DStream of (K, V) and (K, W) pairs, return a new DStream of (K, Seq[V], Seq[W]) tuples. @@ -852,13 +850,13 @@ A few of these transformations are worth discussing in more detail. The `updateStateByKey` operation allows you to maintain arbitrary state while continuously updating it with new information. To use this, you will have to do two steps. -1. Define the state - The state can be of arbitrary data type. +1. Define the state - The state can be an arbitrary data type. 1. Define the state update function - Specify with a function how to update the state using the -previous state and the new values from input stream. +previous state and the new values from an input stream. Let's illustrate this with an example. Say you want to maintain a running count of each word seen in a text data stream. Here, the running count is the state and it is an integer. We -define the update function as +define the update function as:
@@ -947,7 +945,7 @@ operation that is not exposed in the DStream API. For example, the functionality of joining every batch in a data stream with another dataset is not directly exposed in the DStream API. However, you can easily use `transform` to do this. This enables very powerful possibilities. For example, -if you want to do real-time data cleaning by joining the input data stream with precomputed +one can do real-time data cleaning by joining the input data stream with precomputed spam information (maybe generated with Spark as well) and then filtering based on it.
@@ -998,7 +996,7 @@ etc. can be changed between batches. #### Window Operations {:.no_toc} Spark Streaming also provides *windowed computations*, which allow you to apply -transformations over a sliding window of data. This following figure illustrates this sliding +transformations over a sliding window of data. The following figure illustrates this sliding window.

@@ -1010,11 +1008,11 @@ window. As shown in the figure, every time the window *slides* over a source DStream, the source RDDs that fall within the window are combined and operated upon to produce the -RDDs of the windowed DStream. In this specific case, the operation is applied over last 3 time +RDDs of the windowed DStream. In this specific case, the operation is applied over the last 3 time units of data, and slides by 2 time units. This shows that any window operation needs to specify two parameters. - * window length - The duration of the window (3 in the figure) + * window length - The duration of the window (3 in the figure). * sliding interval - The interval at which the window operation is performed (2 in the figure). @@ -1022,7 +1020,7 @@ These two parameters must be multiples of the batch interval of the source DStre figure). Let's illustrate the window operations with an example. Say, you want to extend the -[earlier example](#a-quick-example) by generating word counts over last 30 seconds of data, +[earlier example](#a-quick-example) by generating word counts over the last 30 seconds of data, every 10 seconds. To do this, we have to apply the `reduceByKey` operation on the `pairs` DStream of `(word, 1)` pairs over the last 30 seconds of data. This is done using the operation `reduceByKeyAndWindow`. @@ -1097,13 +1095,13 @@ said two parameters - windowLength and slideInterval. reduceByKeyAndWindow(func, invFunc, windowLength, slideInterval, [numTasks]) - A more efficient version of the above reduceByKeyAndWindow() where the reduce + A more efficient version of the above reduceByKeyAndWindow() where the reduce value of each window is calculated incrementally using the reduce values of the previous window. - This is done by reducing the new data that enter the sliding window, and "inverse reducing" the - old data that leave the window. An example would be that of "adding" and "subtracting" counts - of keys as the window slides. However, it is applicable to only "invertible reduce functions", + This is done by reducing the new data that enters the sliding window, and "inverse reducing" the + old data that leaves the window. An example would be that of "adding" and "subtracting" counts + of keys as the window slides. However, it is applicable only to "invertible reduce functions", that is, those reduce functions which have a corresponding "inverse reduce" function (taken as - parameter invFunc. Like in reduceByKeyAndWindow, the number of reduce tasks + parameter invFunc). Like in reduceByKeyAndWindow, the number of reduce tasks is configurable through an optional argument. Note that [checkpointing](#checkpointing) must be enabled for using this operation. @@ -1225,7 +1223,7 @@ For the Python API, see [DStream](api/python/pyspark.streaming.html#pyspark.stre *** ## Output Operations on DStreams -Output operations allow DStream's data to be pushed out external systems like a database or a file systems. +Output operations allow DStream's data to be pushed out to external systems like a database or a file systems. Since the output operations actually allow the transformed data to be consumed by external systems, they trigger the actual execution of all the DStream transformations (similar to actions for RDDs). Currently, the following output operations are defined: @@ -1234,7 +1232,7 @@ Currently, the following output operations are defined: Output OperationMeaning print() - Prints first ten elements of every batch of data in a DStream on the driver node running + Prints the first ten elements of every batch of data in a DStream on the driver node running the streaming application. This is useful for development and debugging.
Python API This is called @@ -1243,12 +1241,12 @@ Currently, the following output operations are defined: saveAsTextFiles(prefix, [suffix]) - Save this DStream's contents as a text files. The file name at each batch interval is + Save this DStream's contents as text files. The file name at each batch interval is generated based on prefix and suffix: "prefix-TIME_IN_MS[.suffix]". saveAsObjectFiles(prefix, [suffix]) - Save this DStream's contents as a SequenceFile of serialized Java objects. The file + Save this DStream's contents as SequenceFiles of serialized Java objects. The file name at each batch interval is generated based on prefix and suffix: "prefix-TIME_IN_MS[.suffix]".
@@ -1258,7 +1256,7 @@ Currently, the following output operations are defined: saveAsHadoopFiles(prefix, [suffix]) - Save this DStream's contents as a Hadoop file. The file name at each batch interval is + Save this DStream's contents as Hadoop files. The file name at each batch interval is generated based on prefix and suffix: "prefix-TIME_IN_MS[.suffix]".
Python API This is not available in @@ -1268,7 +1266,7 @@ Currently, the following output operations are defined: foreachRDD(func) The most generic output operator that applies a function, func, to each RDD generated from - the stream. This function should push the data in each RDD to a external system, like saving the RDD to + the stream. This function should push the data in each RDD to an external system, such as saving the RDD to files, or writing it over the network to a database. Note that the function func is executed in the driver process running the streaming application, and will usually have RDD actions in it that will force the computation of the streaming RDDs. @@ -1278,14 +1276,14 @@ Currently, the following output operations are defined: ### Design Patterns for using foreachRDD {:.no_toc} -`dstream.foreachRDD` is a powerful primitive that allows data to sent out to external systems. +`dstream.foreachRDD` is a powerful primitive that allows data to be sent out to external systems. However, it is important to understand how to use this primitive correctly and efficiently. Some of the common mistakes to avoid are as follows. Often writing data to external system requires creating a connection object (e.g. TCP connection to a remote server) and using it to send data to a remote system. For this purpose, a developer may inadvertently try creating a connection object at -the Spark driver, but try to use it in a Spark worker to save records in the RDDs. +the Spark driver, and then try to use it in a Spark worker to save records in the RDDs. For example (in Scala),

@@ -1347,7 +1345,7 @@ dstream.foreachRDD(lambda rdd: rdd.foreach(sendRecord)) Typically, creating a connection object has time and resource overheads. Therefore, creating and destroying a connection object for each record can incur unnecessarily high overheads and can significantly reduce the overall throughput of the system. A better solution is to use -`rdd.foreachPartition` - create a single connection object and send all the records in a RDD +`rdd.foreachPartition` - create a single connection object and send all the records in a RDD partition using that connection.
@@ -1551,7 +1549,7 @@ See the full [source code]({{site.SPARK_GITHUB_URL}}/blob/master/examples/src/ma
-You can also run SQL queries on tables defined on streaming data from a different thread (that is, asynchronous to the running StreamingContext). Just make sure that you set the StreamingContext to remember sufficient amount of streaming data such that query can run. Otherwise the StreamingContext, which is unaware of the any asynchronous SQL queries, will delete off old streaming data before the query can complete. For example, if you want to query the last batch, but your query can take 5 minutes to run, then call `streamingContext.remember(Minutes(5))` (in Scala, or equivalent in other languages). +You can also run SQL queries on tables defined on streaming data from a different thread (that is, asynchronous to the running StreamingContext). Just make sure that you set the StreamingContext to remember a sufficient amount of streaming data such that the query can run. Otherwise the StreamingContext, which is unaware of the any asynchronous SQL queries, will delete off old streaming data before the query can complete. For example, if you want to query the last batch, but your query can take 5 minutes to run, then call `streamingContext.remember(Minutes(5))` (in Scala, or equivalent in other languages). See the [DataFrames and SQL](sql-programming-guide.html) guide to learn more about DataFrames. @@ -1564,7 +1562,7 @@ You can also easily use machine learning algorithms provided by [MLlib](mllib-gu ## Caching / Persistence Similar to RDDs, DStreams also allow developers to persist the stream's data in memory. That is, -using `persist()` method on a DStream will automatically persist every RDD of that DStream in +using the `persist()` method on a DStream will automatically persist every RDD of that DStream in memory. This is useful if the data in the DStream will be computed multiple times (e.g., multiple operations on the same data). For window-based operations like `reduceByWindow` and `reduceByKeyAndWindow` and state-based operations like `updateStateByKey`, this is implicitly true. @@ -1576,28 +1574,27 @@ default persistence level is set to replicate the data to two nodes for fault-to Note that, unlike RDDs, the default persistence level of DStreams keeps the data serialized in memory. This is further discussed in the [Performance Tuning](#memory-tuning) section. More -information on different persistence levels can be found in -[Spark Programming Guide](programming-guide.html#rdd-persistence). +information on different persistence levels can be found in the [Spark Programming Guide](programming-guide.html#rdd-persistence). *** ## Checkpointing A streaming application must operate 24/7 and hence must be resilient to failures unrelated to the application logic (e.g., system failures, JVM crashes, etc.). For this to be possible, -Spark Streaming needs to *checkpoints* enough information to a fault- +Spark Streaming needs to *checkpoint* enough information to a fault- tolerant storage system such that it can recover from failures. There are two types of data that are checkpointed. - *Metadata checkpointing* - Saving of the information defining the streaming computation to fault-tolerant storage like HDFS. This is used to recover from failure of the node running the driver of the streaming application (discussed in detail later). Metadata includes: - + *Configuration* - The configuration that were used to create the streaming application. + + *Configuration* - The configuration that was used to create the streaming application. + *DStream operations* - The set of DStream operations that define the streaming application. + *Incomplete batches* - Batches whose jobs are queued but have not completed yet. - *Data checkpointing* - Saving of the generated RDDs to reliable storage. This is necessary in some *stateful* transformations that combine data across multiple batches. In such - transformations, the generated RDDs depends on RDDs of previous batches, which causes the length - of the dependency chain to keep increasing with time. To avoid such unbounded increase in recovery + transformations, the generated RDDs depend on RDDs of previous batches, which causes the length + of the dependency chain to keep increasing with time. To avoid such unbounded increases in recovery time (proportional to dependency chain), intermediate RDDs of stateful transformations are periodically *checkpointed* to reliable storage (e.g. HDFS) to cut off the dependency chains. @@ -1611,10 +1608,10 @@ transformations are used. Checkpointing must be enabled for applications with any of the following requirements: - *Usage of stateful transformations* - If either `updateStateByKey` or `reduceByKeyAndWindow` (with - inverse function) is used in the application, then the checkpoint directory must be provided for - allowing periodic RDD checkpointing. + inverse function) is used in the application, then the checkpoint directory must be provided to + allow for periodic RDD checkpointing. - *Recovering from failures of the driver running the application* - Metadata checkpoints are used - for to recover with progress information. + to recover with progress information. Note that simple streaming applications without the aforementioned stateful transformations can be run without enabling checkpointing. The recovery from driver failures will also be partial in @@ -1629,7 +1626,7 @@ Checkpointing can be enabled by setting a directory in a fault-tolerant, reliable file system (e.g., HDFS, S3, etc.) to which the checkpoint information will be saved. This is done by using `streamingContext.checkpoint(checkpointDirectory)`. This will allow you to use the aforementioned stateful transformations. Additionally, -if you want make the application recover from driver failures, you should rewrite your +if you want to make the application recover from driver failures, you should rewrite your streaming application to have the following behavior. + When the program is being started for the first time, it will create a new StreamingContext, @@ -1750,18 +1747,17 @@ You can also explicitly create a `StreamingContext` from the checkpoint data and In addition to using `getOrCreate` one also needs to ensure that the driver process gets restarted automatically on failure. This can only be done by the deployment infrastructure that is used to run the application. This is further discussed in the -[Deployment](#deploying-applications.html) section. +[Deployment](#deploying-applications) section. Note that checkpointing of RDDs incurs the cost of saving to reliable storage. This may cause an increase in the processing time of those batches where RDDs get checkpointed. Hence, the interval of checkpointing needs to be set carefully. At small batch sizes (say 1 second), checkpointing every batch may significantly reduce operation throughput. Conversely, checkpointing too infrequently -causes the lineage and task sizes to grow which may have detrimental effects. For stateful +causes the lineage and task sizes to grow, which may have detrimental effects. For stateful transformations that require RDD checkpointing, the default interval is a multiple of the batch interval that is at least 10 seconds. It can be set by using -`dstream.checkpoint(checkpointInterval)`. Typically, a checkpoint interval of 5 - 10 times of -sliding interval of a DStream is good setting to try. +`dstream.checkpoint(checkpointInterval)`. Typically, a checkpoint interval of 5 - 10 sliding intervals of a DStream is a good setting to try. *** @@ -1834,17 +1830,17 @@ To run a Spark Streaming applications, you need to have the following. {:.no_toc} If a running Spark Streaming application needs to be upgraded with new -application code, then there are two possible mechanism. +application code, then there are two possible mechanisms. - The upgraded Spark Streaming application is started and run in parallel to the existing application. -Once the new one (receiving the same data as the old one) has been warmed up and ready +Once the new one (receiving the same data as the old one) has been warmed up and is ready for prime time, the old one be can be brought down. Note that this can be done for data sources that support sending the data to two destinations (i.e., the earlier and upgraded applications). - The existing application is shutdown gracefully (see [`StreamingContext.stop(...)`](api/scala/index.html#org.apache.spark.streaming.StreamingContext) or [`JavaStreamingContext.stop(...)`](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html) -for graceful shutdown options) which ensure data that have been received is completely +for graceful shutdown options) which ensure data that has been received is completely processed before shutdown. Then the upgraded application can be started, which will start processing from the same point where the earlier application left off. Note that this can be done only with input sources that support source-side buffering @@ -1879,10 +1875,10 @@ The following two metrics in web UI are particularly important: to finish. If the batch processing time is consistently more than the batch interval and/or the queueing -delay keeps increasing, then it indicates the system is -not able to process the batches as fast they are being generated and falling behind. +delay keeps increasing, then it indicates that the system is +not able to process the batches as fast they are being generated and is falling behind. In that case, consider -[reducing](#reducing-the-processing-time-of-each-batch) the batch processing time. +[reducing](#reducing-the-batch-processing-times) the batch processing time. The progress of a Spark Streaming program can also be monitored using the [StreamingListener](api/scala/index.html#org.apache.spark.streaming.scheduler.StreamingListener) interface, @@ -1893,8 +1889,8 @@ and it is likely to be improved upon (i.e., more information reported) in the fu *************************************************************************************************** # Performance Tuning -Getting the best performance of a Spark Streaming application on a cluster requires a bit of -tuning. This section explains a number of the parameters and configurations that can tuned to +Getting the best performance out of a Spark Streaming application on a cluster requires a bit of +tuning. This section explains a number of the parameters and configurations that can be tuned to improve the performance of you application. At a high level, you need to consider two things: 1. Reducing the processing time of each batch of data by efficiently using cluster resources. @@ -1904,12 +1900,12 @@ improve the performance of you application. At a high level, you need to conside ## Reducing the Batch Processing Times There are a number of optimizations that can be done in Spark to minimize the processing time of -each batch. These have been discussed in detail in [Tuning Guide](tuning.html). This section +each batch. These have been discussed in detail in the [Tuning Guide](tuning.html). This section highlights some of the most important ones. ### Level of Parallelism in Data Receiving {:.no_toc} -Receiving data over the network (like Kafka, Flume, socket, etc.) requires the data to deserialized +Receiving data over the network (like Kafka, Flume, socket, etc.) requires the data to be deserialized and stored in Spark. If the data receiving becomes a bottleneck in the system, then consider parallelizing the data receiving. Note that each input DStream creates a single receiver (running on a worker machine) that receives a single stream of data. @@ -1917,7 +1913,7 @@ Receiving multiple data streams can therefore be achieved by creating multiple i and configuring them to receive different partitions of the data stream from the source(s). For example, a single Kafka input DStream receiving two topics of data can be split into two Kafka input streams, each receiving only one topic. This would run two receivers, -allowing data to be received in parallel, and increasing overall throughput. These multiple +allowing data to be received in parallel, thus increasing overall throughput. These multiple DStreams can be unioned together to create a single DStream. Then the transformations that were being applied on a single input DStream can be applied on the unified stream. This is done as follows. @@ -1947,10 +1943,10 @@ Another parameter that should be considered is the receiver's blocking interval, which is determined by the [configuration parameter](configuration.html#spark-streaming) `spark.streaming.blockInterval`. For most receivers, the received data is coalesced together into blocks of data before storing inside Spark's memory. The number of blocks in each batch -determines the number of tasks that will be used to process those +determines the number of tasks that will be used to process the received data in a map-like transformation. The number of tasks per receiver per batch will be approximately (batch interval / block interval). For example, block interval of 200 ms will -create 10 tasks per 2 second batches. Too low the number of tasks (that is, less than the number +create 10 tasks per 2 second batches. If the number of tasks is too low (that is, less than the number of cores per machine), then it will be inefficient as all available cores will not be used to process the data. To increase the number of tasks for a given batch interval, reduce the block interval. However, the recommended minimum value of block interval is about 50 ms, @@ -1958,7 +1954,7 @@ below which the task launching overheads may be a problem. An alternative to receiving data with multiple input streams / receivers is to explicitly repartition the input data stream (using `inputStream.repartition()`). -This distributes the received batches of data across specified number of machines in the cluster +This distributes the received batches of data across the specified number of machines in the cluster before further processing. ### Level of Parallelism in Data Processing @@ -1966,7 +1962,7 @@ before further processing. Cluster resources can be under-utilized if the number of parallel tasks used in any stage of the computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is controlled by -the`spark.default.parallelism` [configuration property](configuration.html#spark-properties). You +the `spark.default.parallelism` [configuration property](configuration.html#spark-properties). You can pass the level of parallelism as an argument (see [`PairDStreamFunctions`](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions) documentation), or set the `spark.default.parallelism` @@ -1974,20 +1970,20 @@ documentation), or set the `spark.default.parallelism` ### Data Serialization {:.no_toc} -The overheads of data serialization can be reduce by tuning the serialization formats. In case of streaming, there are two types of data that are being serialized. +The overheads of data serialization can be reduced by tuning the serialization formats. In the case of streaming, there are two types of data that are being serialized. -* **Input data**: By default, the input data received through Receivers is stored in the executors' memory with [StorageLevel.MEMORY_AND_DISK_SER_2](api/scala/index.html#org.apache.spark.storage.StorageLevel$). That is, the data is serialized into bytes to reduce GC overheads, and replicated for tolerating executor failures. Also, the data is kept first in memory, and spilled over to disk only if the memory is unsufficient to hold all the input data necessary for the streaming computation. This serialization obviously has overheads -- the receiver must deserialize the received data and re-serialize it using Spark's serialization format. +* **Input data**: By default, the input data received through Receivers is stored in the executors' memory with [StorageLevel.MEMORY_AND_DISK_SER_2](api/scala/index.html#org.apache.spark.storage.StorageLevel$). That is, the data is serialized into bytes to reduce GC overheads, and replicated for tolerating executor failures. Also, the data is kept first in memory, and spilled over to disk only if the memory is insufficient to hold all of the input data necessary for the streaming computation. This serialization obviously has overheads -- the receiver must deserialize the received data and re-serialize it using Spark's serialization format. -* **Persisted RDDs generated by Streaming Operations**: RDDs generated by streaming computations may be persisted in memory. For example, window operation persist data in memory as they would be processed multiple times. However, unlike Spark, by default RDDs are persisted with [StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) (i.e. serialized) to minimize GC overheads. +* **Persisted RDDs generated by Streaming Operations**: RDDs generated by streaming computations may be persisted in memory. For example, window operations persist data in memory as they would be processed multiple times. However, unlike the Spark Core default of [StorageLevel.MEMORY_ONLY](api/scala/index.html#org.apache.spark.storage.StorageLevel$), persisted RDDs generated by streaming computations are persisted with [StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) (i.e. serialized) by default to minimize GC overheads. -In both cases, using Kryo serialization can reduce both CPU and memory overheads. See the [Spark Tuning Guide](tuning.html#data-serialization)) for more details. Consider registering custom classes, and disabling object reference tracking for Kryo (see Kryo-related configurations in the [Configuration Guide](configuration.html#compression-and-serialization)). +In both cases, using Kryo serialization can reduce both CPU and memory overheads. See the [Spark Tuning Guide](tuning.html#data-serialization) for more details. For Kryo, consider registering custom classes, and disabling object reference tracking (see Kryo-related configurations in the [Configuration Guide](configuration.html#compression-and-serialization)). -In specific cases where the amount of data that needs to be retained for the streaming application is not large, it may be feasible to persist data (both types) as deserialized objects without incurring excessive GC overheads. For example, if you are using batch intervals of few seconds and no window operations, then you can try disabling serialization in persisted data by explicitly setting the storage level accordingly. This would reduce the CPU overheads due to serialization, potentially improving performance without too much GC overheads. +In specific cases where the amount of data that needs to be retained for the streaming application is not large, it may be feasible to persist data (both types) as deserialized objects without incurring excessive GC overheads. For example, if you are using batch intervals of a few seconds and no window operations, then you can try disabling serialization in persisted data by explicitly setting the storage level accordingly. This would reduce the CPU overheads due to serialization, potentially improving performance without too much GC overheads. ### Task Launching Overheads {:.no_toc} If the number of tasks launched per second is high (say, 50 or more per second), then the overhead -of sending out tasks to the slaves maybe significant and will make it hard to achieve sub-second +of sending out tasks to the slaves may be significant and will make it hard to achieve sub-second latencies. The overhead can be reduced by the following changes: * **Task Serialization**: Using Kryo serialization for serializing tasks can reduce the task @@ -2006,7 +2002,7 @@ thus allowing sub-second batch size to be viable. For a Spark Streaming application running on a cluster to be stable, the system should be able to process data as fast as it is being received. In other words, batches of data should be processed as fast as they are being generated. Whether this is true for an application can be found by -[monitoring](#monitoring) the processing times in the streaming web UI, where the batch +[monitoring](#monitoring-applications) the processing times in the streaming web UI, where the batch processing time should be less than the batch interval. Depending on the nature of the streaming @@ -2019,35 +2015,35 @@ production can be sustained. A good approach to figure out the right batch size for your application is to test it with a conservative batch interval (say, 5-10 seconds) and a low data rate. To verify whether the system -is able to keep up with data rate, you can check the value of the end-to-end delay experienced +is able to keep up with the data rate, you can check the value of the end-to-end delay experienced by each processed batch (either look for "Total delay" in Spark driver log4j logs, or use the [StreamingListener](api/scala/index.html#org.apache.spark.streaming.scheduler.StreamingListener) interface). If the delay is maintained to be comparable to the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it therefore unstable. Once you have an idea of a stable configuration, you can try increasing the -data rate and/or reducing the batch size. Note that momentary increase in the delay due to -temporary data rate increases maybe fine as long as the delay reduces back to a low value +data rate and/or reducing the batch size. Note that a momentary increase in the delay due to +temporary data rate increases may be fine as long as the delay reduces back to a low value (i.e., less than batch size). *** ## Memory Tuning -Tuning the memory usage and GC behavior of Spark applications have been discussed in great detail +Tuning the memory usage and GC behavior of Spark applications has been discussed in great detail in the [Tuning Guide](tuning.html#memory-tuning). It is strongly recommended that you read that. In this section, we discuss a few tuning parameters specifically in the context of Spark Streaming applications. -The amount of cluster memory required by a Spark Streaming application depends heavily on the type of transformations used. For example, if you want to use a window operation on last 10 minutes of data, then your cluster should have sufficient memory to hold 10 minutes of worth of data in memory. Or if you want to use `updateStateByKey` with a large number of keys, then the necessary memory will be high. On the contrary, if you want to do a simple map-filter-store operation, then necessary memory will be low. +The amount of cluster memory required by a Spark Streaming application depends heavily on the type of transformations used. For example, if you want to use a window operation on the last 10 minutes of data, then your cluster should have sufficient memory to hold 10 minutes worth of data in memory. Or if you want to use `updateStateByKey` with a large number of keys, then the necessary memory will be high. On the contrary, if you want to do a simple map-filter-store operation, then the necessary memory will be low. -In general, since the data received through receivers are stored with StorageLevel.MEMORY_AND_DISK_SER_2, the data that does not fit in memory will spill over to the disk. This may reduce the performance of the streaming application, and hence it is advised to provide sufficient memory as required by your streaming application. Its best to try and see the memory usage on a small scale and estimate accordingly. +In general, since the data received through receivers is stored with StorageLevel.MEMORY_AND_DISK_SER_2, the data that does not fit in memory will spill over to the disk. This may reduce the performance of the streaming application, and hence it is advised to provide sufficient memory as required by your streaming application. Its best to try and see the memory usage on a small scale and estimate accordingly. -Another aspect of memory tuning is garbage collection. For a streaming application that require low latency, it is undesirable to have large pauses caused by JVM Garbage Collection. +Another aspect of memory tuning is garbage collection. For a streaming application that requires low latency, it is undesirable to have large pauses caused by JVM Garbage Collection. -There are a few parameters that can help you tune the memory usage and GC overheads. +There are a few parameters that can help you tune the memory usage and GC overheads: -* **Persistence Level of DStreams**: As mentioned earlier in the [Data Serialization](#data-serialization) section, the input data and RDDs are by default persisted as serialized bytes. This reduces both, the memory usage and GC overheads, compared to deserialized persistence. Enabling Kryo serialization further reduces serialized sizes and memory usage. Further reduction in memory usage can be achieved with compression (see the Spark configuration `spark.rdd.compress`), at the cost of CPU time. +* **Persistence Level of DStreams**: As mentioned earlier in the [Data Serialization](#data-serialization) section, the input data and RDDs are by default persisted as serialized bytes. This reduces both the memory usage and GC overheads, compared to deserialized persistence. Enabling Kryo serialization further reduces serialized sizes and memory usage. Further reduction in memory usage can be achieved with compression (see the Spark configuration `spark.rdd.compress`), at the cost of CPU time. -* **Clearing old data**: By default, all input data and persisted RDDs generated by DStream transformations are automatically cleared. Spark Streaming decides when to clear the data based on the transformations that are used. For example, if you are using window operation of 10 minutes, then Spark Streaming will keep around last 10 minutes of data, and actively throw away older data. -Data can be retained for longer duration (e.g. interactively querying older data) by setting `streamingContext.remember`. +* **Clearing old data**: By default, all input data and persisted RDDs generated by DStream transformations are automatically cleared. Spark Streaming decides when to clear the data based on the transformations that are used. For example, if you are using a window operation of 10 minutes, then Spark Streaming will keep around the last 10 minutes of data, and actively throw away older data. +Data can be retained for a longer duration (e.g. interactively querying older data) by setting `streamingContext.remember`. * **CMS Garbage Collector**: Use of the concurrent mark-and-sweep GC is strongly recommended for keeping GC-related pauses consistently low. Even though concurrent GC is known to reduce the overall processing throughput of the system, its use is still recommended to achieve more @@ -2077,18 +2073,18 @@ re-computed from the original fault-tolerant dataset using the lineage of operat 1. Assuming that all of the RDD transformations are deterministic, the data in the final transformed RDD will always be the same irrespective of failures in the Spark cluster. -Spark operates on data on fault-tolerant file systems like HDFS or S3. Hence, +Spark operates on data in fault-tolerant file systems like HDFS or S3. Hence, all of the RDDs generated from the fault-tolerant data are also fault-tolerant. However, this is not the case for Spark Streaming as the data in most cases is received over the network (except when `fileStream` is used). To achieve the same fault-tolerance properties for all of the generated RDDs, the received data is replicated among multiple Spark executors in worker nodes in the cluster (default replication factor is 2). This leads to two kinds of data in the -system that needs to recovered in the event of failures: +system that need to recovered in the event of failures: 1. *Data received and replicated* - This data survives failure of a single worker node as a copy - of it exists on one of the nodes. + of it exists on one of the other nodes. 1. *Data received but buffered for replication* - Since this is not replicated, - the only way to recover that data is to get it again from the source. + the only way to recover this data is to get it again from the source. Furthermore, there are two kinds of failures that we should be concerned about: @@ -2115,13 +2111,13 @@ In any stream processing system, broadly speaking, there are three steps in proc 1. *Receiving the data*: The data is received from sources using Receivers or otherwise. -1. *Transforming the data*: The data received data is transformed using DStream and RDD transformations. +1. *Transforming the data*: The received data is transformed using DStream and RDD transformations. 1. *Pushing out the data*: The final transformed data is pushed out to external systems like file systems, databases, dashboards, etc. -If a streaming application has to achieve end-to-end exactly-once guarantees, then each step has to provide exactly-once guarantee. That is, each record must be received exactly once, transformed exactly once, and pushed to downstream systems exactly once. Let's understand the semantics of these steps in the context of Spark Streaming. +If a streaming application has to achieve end-to-end exactly-once guarantees, then each step has to provide an exactly-once guarantee. That is, each record must be received exactly once, transformed exactly once, and pushed to downstream systems exactly once. Let's understand the semantics of these steps in the context of Spark Streaming. -1. *Receiving the data*: Different input sources provided different guarantees. This is discussed in detail in the next subsection. +1. *Receiving the data*: Different input sources provide different guarantees. This is discussed in detail in the next subsection. 1. *Transforming the data*: All data that has been received will be processed _exactly once_, thanks to the guarantees that RDDs provide. Even if there are failures, as long as the received input data is accessible, the final transformed RDDs will always have the same contents. @@ -2133,9 +2129,9 @@ Different input sources provide different guarantees, ranging from _at-least onc ### With Files {:.no_toc} -If all of the input data is already present in a fault-tolerant files system like -HDFS, Spark Streaming can always recover from any failure and process all the data. This gives -*exactly-once* semantics, that all the data will be processed exactly once no matter what fails. +If all of the input data is already present in a fault-tolerant file system like +HDFS, Spark Streaming can always recover from any failure and process all of the data. This gives +*exactly-once* semantics, meaning all of the data will be processed exactly once no matter what fails. ### With Receiver-based Sources {:.no_toc} @@ -2144,21 +2140,21 @@ scenario and the type of receiver. As we discussed [earlier](#receiver-reliability), there are two types of receivers: 1. *Reliable Receiver* - These receivers acknowledge reliable sources only after ensuring that - the received data has been replicated. If such a receiver fails, - the buffered (unreplicated) data does not get acknowledged to the source. If the receiver is - restarted, the source will resend the data, and therefore no data will be lost due to the failure. -1. *Unreliable Receiver* - Such receivers can lose data when they fail due to worker - or driver failures. + the received data has been replicated. If such a receiver fails, the source will not receive + acknowledgment for the buffered (unreplicated) data. Therefore, if the receiver is + restarted, the source will resend the data, and no data will be lost due to the failure. +1. *Unreliable Receiver* - Such receivers do *not* send acknowledgment and therefore *can* lose + data when they fail due to worker or driver failures. Depending on what type of receivers are used we achieve the following semantics. If a worker node fails, then there is no data loss with reliable receivers. With unreliable receivers, data received but not replicated can get lost. If the driver node fails, -then besides these losses, all the past data that was received and replicated in memory will be +then besides these losses, all of the past data that was received and replicated in memory will be lost. This will affect the results of the stateful transformations. To avoid this loss of past received data, Spark 1.2 introduced _write -ahead logs_ which saves the received data to fault-tolerant storage. With the [write ahead logs -enabled](#deploying-applications) and reliable receivers, there is zero data loss. In terms of semantics, it provides at-least once guarantee. +ahead logs_ which save the received data to fault-tolerant storage. With the [write ahead logs +enabled](#deploying-applications) and reliable receivers, there is zero data loss. In terms of semantics, it provides an at-least once guarantee. The following table summarizes the semantics under failures: @@ -2219,7 +2215,7 @@ additional effort may be necessary to achieve exactly-once semantics. There are - *Transactional updates*: All updates are made transactionally so that updates are made exactly once atomically. One way to do this would be the following. - Use the batch time (available in `foreachRDD`) and the partition index of the RDD to create an identifier. This identifier uniquely identifies a blob data in the streaming application. - - Update external system with this blob transactionally (that is, exactly once, atomically) using the identifier. That is, if the identifier is not already committed, commit the partition data and the identifier atomically. Else if this was already committed, skip the update. + - Update external system with this blob transactionally (that is, exactly once, atomically) using the identifier. That is, if the identifier is not already committed, commit the partition data and the identifier atomically. Else, if this was already committed, skip the update. dstream.foreachRDD { (rdd, time) => rdd.foreachPartition { partitionIterator => @@ -2229,7 +2225,6 @@ additional effort may be necessary to achieve exactly-once semantics. There are } } - *************************************************************************************************** *************************************************************************************************** @@ -2303,7 +2298,7 @@ package and renamed for better clarity. - Java docs * [JavaStreamingContext](api/java/index.html?org/apache/spark/streaming/api/java/JavaStreamingContext.html), [JavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaDStream.html) and - [PairJavaDStream](api/java/index.html?org/apache/spark/streaming/api/java/PairJavaDStream.html) + [JavaPairDStream](api/java/index.html?org/apache/spark/streaming/api/java/JavaPairDStream.html) * [KafkaUtils](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html), [FlumeUtils](api/java/index.html?org/apache/spark/streaming/flume/FlumeUtils.html), [KinesisUtils](api/java/index.html?org/apache/spark/streaming/kinesis/KinesisUtils.html) From cb7ada1196b5e13f0d4bb1988dc707a72003b8bc Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 13 Jun 2015 22:42:28 -0700 Subject: [PATCH 08/13] [SPARK-8342][SQL] Fix Decimal setOrNull JIRA: https://issues.apache.org/jira/browse/SPARK-8342 Author: Liang-Chi Hsieh Closes #6797 from viirya/fix_decimal and squashes the following commits: 8a447b1 [Liang-Chi Hsieh] Add unit test. d67a5ea [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into fix_decimal ab6d8af [Liang-Chi Hsieh] Fix setOrNull. --- .../src/main/scala/org/apache/spark/sql/types/Decimal.scala | 2 +- .../org/apache/spark/sql/types/decimal/DecimalSuite.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index eb3c58c37f308..a85af9e04aedb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -86,7 +86,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { if (precision < 19) { return null // Requested precision is too low to represent this value } - this.decimalVal = BigDecimal(longVal) + this.decimalVal = BigDecimal(unscaled) this.longVal = 0L } else { val p = POW_10(math.min(precision, MAX_LONG_DIGITS)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala index 28b373e258311..4c0365cf1b6f9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/decimal/DecimalSuite.scala @@ -156,4 +156,10 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { assert(Decimal(-100) % Decimal(3) === Decimal(-1)) assert(Decimal(100) % Decimal(0) === null) } + + test("set/setOrNull") { + assert(new Decimal().set(10L, 10, 0).toUnscaledLong === 10L) + assert(new Decimal().set(100L, 10, 0).toUnscaledLong === 100L) + assert(Decimal(Long.MaxValue, 100, 0).toUnscaledLong === Long.MaxValue) + } } From ea7fd2ff6454e8d819a39bf49901074e49b5714e Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sun, 14 Jun 2015 09:34:35 -0700 Subject: [PATCH 09/13] [SPARK-8354] [SQL] Fix off-by-factor-of-8 error when allocating scratch space in UnsafeFixedWidthAggregationMap UnsafeFixedWidthAggregationMap contains an off-by-factor-of-8 error when allocating row conversion scratch space: we take a size requirement, measured in bytes, then allocate a long array of that size. This means that we end up allocating 8x too much conversion space. This patch fixes this by allocating a `byte[]` array instead. This doesn't impose any new limitations on the maximum sizes of UnsafeRows, since UnsafeRowConverter already used integers when calculating the size requirements for rows. Author: Josh Rosen Closes #6809 from JoshRosen/sql-bytes-vs-words-fix and squashes the following commits: 6520339 [Josh Rosen] Updates to reflect fact that UnsafeRow max size is constrained by max byte[] size --- .../UnsafeFixedWidthAggregationMap.java | 30 +++++++++---------- .../expressions/UnsafeRowConverter.scala | 2 +- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java index b23e0efc83332..f7849ebebc573 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeFixedWidthAggregationMap.java @@ -39,7 +39,7 @@ public final class UnsafeFixedWidthAggregationMap { * An empty aggregation buffer, encoded in UnsafeRow format. When inserting a new key into the * map, we copy this buffer and use it as the value. */ - private final long[] emptyAggregationBuffer; + private final byte[] emptyAggregationBuffer; private final StructType aggregationBufferSchema; @@ -63,10 +63,10 @@ public final class UnsafeFixedWidthAggregationMap { /** * Scratch space that is used when encoding grouping keys into UnsafeRow format. * - * By default, this is a 1MB array, but it will grow as necessary in case larger keys are + * By default, this is a 8 kb array, but it will grow as necessary in case larger keys are * encountered. */ - private long[] groupingKeyConversionScratchSpace = new long[1024 / 8]; + private byte[] groupingKeyConversionScratchSpace = new byte[1024 * 8]; private final boolean enablePerfMetrics; @@ -123,13 +123,13 @@ public UnsafeFixedWidthAggregationMap( } /** - * Convert a Java object row into an UnsafeRow, allocating it into a new long array. + * Convert a Java object row into an UnsafeRow, allocating it into a new byte array. */ - private static long[] convertToUnsafeRow(InternalRow javaRow, StructType schema) { + private static byte[] convertToUnsafeRow(InternalRow javaRow, StructType schema) { final UnsafeRowConverter converter = new UnsafeRowConverter(schema); - final long[] unsafeRow = new long[converter.getSizeRequirement(javaRow)]; - final long writtenLength = - converter.writeRow(javaRow, unsafeRow, PlatformDependent.LONG_ARRAY_OFFSET); + final byte[] unsafeRow = new byte[converter.getSizeRequirement(javaRow)]; + final int writtenLength = + converter.writeRow(javaRow, unsafeRow, PlatformDependent.BYTE_ARRAY_OFFSET); assert (writtenLength == unsafeRow.length): "Size requirement calculation was wrong!"; return unsafeRow; } @@ -143,34 +143,34 @@ public UnsafeRow getAggregationBuffer(InternalRow groupingKey) { // Make sure that the buffer is large enough to hold the key. If it's not, grow it: if (groupingKeySize > groupingKeyConversionScratchSpace.length) { // This new array will be initially zero, so there's no need to zero it out here - groupingKeyConversionScratchSpace = new long[groupingKeySize]; + groupingKeyConversionScratchSpace = new byte[groupingKeySize]; } else { // Zero out the buffer that's used to hold the current row. This is necessary in order // to ensure that rows hash properly, since garbage data from the previous row could // otherwise end up as padding in this row. As a performance optimization, we only zero out // the portion of the buffer that we'll actually write to. - Arrays.fill(groupingKeyConversionScratchSpace, 0, groupingKeySize, 0); + Arrays.fill(groupingKeyConversionScratchSpace, 0, groupingKeySize, (byte) 0); } - final long actualGroupingKeySize = groupingKeyToUnsafeRowConverter.writeRow( + final int actualGroupingKeySize = groupingKeyToUnsafeRowConverter.writeRow( groupingKey, groupingKeyConversionScratchSpace, - PlatformDependent.LONG_ARRAY_OFFSET); + PlatformDependent.BYTE_ARRAY_OFFSET); assert (groupingKeySize == actualGroupingKeySize) : "Size requirement calculation was wrong!"; // Probe our map using the serialized key final BytesToBytesMap.Location loc = map.lookup( groupingKeyConversionScratchSpace, - PlatformDependent.LONG_ARRAY_OFFSET, + PlatformDependent.BYTE_ARRAY_OFFSET, groupingKeySize); if (!loc.isDefined()) { // This is the first time that we've seen this grouping key, so we'll insert a copy of the // empty aggregation buffer into the map: loc.putNewKey( groupingKeyConversionScratchSpace, - PlatformDependent.LONG_ARRAY_OFFSET, + PlatformDependent.BYTE_ARRAY_OFFSET, groupingKeySize, emptyAggregationBuffer, - PlatformDependent.LONG_ARRAY_OFFSET, + PlatformDependent.BYTE_ARRAY_OFFSET, emptyAggregationBuffer.length ); } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala index d771e454b5170..5c92f41c639fa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverter.scala @@ -68,7 +68,7 @@ class UnsafeRowConverter(fieldTypes: Array[DataType]) { * @param baseOffset the base offset of the destination address * @return the number of bytes written. This should be equal to `getSizeRequirement(row)`. */ - def writeRow(row: InternalRow, baseObject: Object, baseOffset: Long): Long = { + def writeRow(row: InternalRow, baseObject: Object, baseOffset: Long): Int = { unsafeRow.pointTo(baseObject, baseOffset, writers.length, null) var fieldNumber = 0 var appendCursor: Int = fixedLengthSize From 9073a426e444e4bc6efa8608e54e0a986f38a270 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 14 Jun 2015 11:21:42 -0700 Subject: [PATCH 10/13] [SPARK-8358] [SQL] Wait for child resolution when resolving generators Author: Michael Armbrust Closes #6811 from marmbrus/aliasExplodeStar and squashes the following commits: fbd2065 [Michael Armbrust] more style 806a373 [Michael Armbrust] fix style 7cbb530 [Michael Armbrust] [SPARK-8358][SQL] Wait for child resolution when resolving generatorsa --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 6 ++++-- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 8 ++++++++ 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index cbd8def4f1d3c..4b7fef7126989 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -563,7 +563,9 @@ class Analyzer( private object AliasedGenerator { def unapply(e: Expression): Option[(Generator, Seq[String])] = e match { case Alias(g: Generator, name) - if g.elementTypes.size > 1 && java.util.regex.Pattern.matches("_c[0-9]+", name) => { + if g.resolved && + g.elementTypes.size > 1 && + java.util.regex.Pattern.matches("_c[0-9]+", name) => { // Assume the default name given by parser is "_c[0-9]+", // TODO in long term, move the naming logic from Parser to Analyzer. // In projection, Parser gave default name for TGF as does for normal UDF, @@ -572,7 +574,7 @@ class Analyzer( // Let's simply ignore the default given name for this case. Some((g, Nil)) } - case Alias(g: Generator, name) if g.elementTypes.size > 1 => + case Alias(g: Generator, name) if g.resolved && g.elementTypes.size > 1 => // If not given the default names, and the TGF with multiple output columns failAnalysis( s"""Expect multiple names given for ${g.getClass.getName}, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 84835c0db765d..fa98e23e3d147 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -134,6 +134,14 @@ class DataFrameSuite extends QueryTest { ) } + test("explode alias and star") { + val df = Seq((Array("a"), 1)).toDF("a", "b") + + checkAnswer( + df.select(explode($"a").as("a"), $"*"), + Row("a", Seq("a"), 1) :: Nil) + } + test("selectExpr") { checkAnswer( testData.selectExpr("abs(key)", "value"), From 53c16b92a537c392a7c3ebc3ef24c1e86cb1a7a4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 14 Jun 2015 11:23:23 -0700 Subject: [PATCH 11/13] [SPARK-8362] [SQL] Add unit tests for +, -, *, /, % Added unit tests for all supported data types for: - Add - Subtract - Multiply - Divide - UnaryMinus - Remainder Fixed bugs caught by the unit tests. Author: Reynold Xin Closes #6813 from rxin/SPARK-8362 and squashes the following commits: fb3fe62 [Reynold Xin] Added Remainder. 3b266ba [Reynold Xin] [SPARK-8362] Add unit tests for +, -, *, /. --- .../sql/catalyst/expressions/arithmetic.scala | 31 ++-- .../ArithmeticExpressionSuite.scala | 173 +++++++++--------- 2 files changed, 99 insertions(+), 105 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 18ddac1b598e6..9d1e96572a26d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.{CodeGenContext, GeneratedExpressionCode} import org.apache.spark.sql.catalyst.util.TypeUtils @@ -52,8 +51,8 @@ case class UnaryMinus(child: Expression) extends UnaryArithmetic { private lazy val numeric = TypeUtils.getNumeric(dataType) override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = dataType match { - case dt: DecimalType => defineCodeGen(ctx, ev, c => s"c.unary_$$minus()") - case dt: NumericType => defineCodeGen(ctx, ev, c => s"-($c)") + case dt: DecimalType => defineCodeGen(ctx, ev, c => s"$c.unary_$$minus()") + case dt: NumericType => defineCodeGen(ctx, ev, c => s"(${ctx.javaType(dt)})(-($c))") } protected override def evalInternal(evalE: Any) = numeric.negate(evalE) @@ -144,8 +143,8 @@ abstract class BinaryArithmetic extends BinaryExpression { defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1.$decimalMethod($eval2)") // byte and short are casted into int when add, minus, times or divide case ByteType | ShortType => - defineCodeGen(ctx, ev, (eval1, eval2) => - s"(${ctx.javaType(dataType)})($eval1 $symbol $eval2)") + defineCodeGen(ctx, ev, + (eval1, eval2) => s"(${ctx.javaType(dataType)})($eval1 $symbol $eval2)") case _ => defineCodeGen(ctx, ev, (eval1, eval2) => s"$eval1 $symbol $eval2") } @@ -205,7 +204,7 @@ case class Multiply(left: Expression, right: Expression) extends BinaryArithmeti case class Divide(left: Expression, right: Expression) extends BinaryArithmetic { override def symbol: String = "/" - override def decimalMethod: String = "$divide" + override def decimalMethod: String = "$div" override def nullable: Boolean = true @@ -245,11 +244,8 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic } else { s"${eval2.primitive} == 0" } - val method = if (left.dataType.isInstanceOf[DecimalType]) { - s".$decimalMethod" - } else { - s"$symbol" - } + val method = if (left.dataType.isInstanceOf[DecimalType]) s".$decimalMethod" else s" $symbol " + val javaType = ctx.javaType(left.dataType) eval1.code + eval2.code + s""" boolean ${ev.isNull} = false; @@ -257,7 +253,7 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic if (${eval1.isNull} || ${eval2.isNull} || $test) { ${ev.isNull} = true; } else { - ${ev.primitive} = ${eval1.primitive}$method(${eval2.primitive}); + ${ev.primitive} = ($javaType) (${eval1.primitive}$method(${eval2.primitive})); } """ } @@ -265,7 +261,7 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { override def symbol: String = "%" - override def decimalMethod: String = "reminder" + override def decimalMethod: String = "remainder" override def nullable: Boolean = true @@ -305,11 +301,8 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet } else { s"${eval2.primitive} == 0" } - val method = if (left.dataType.isInstanceOf[DecimalType]) { - s".$decimalMethod" - } else { - s"$symbol" - } + val method = if (left.dataType.isInstanceOf[DecimalType]) s".$decimalMethod" else s" $symbol " + val javaType = ctx.javaType(left.dataType) eval1.code + eval2.code + s""" boolean ${ev.isNull} = false; @@ -317,7 +310,7 @@ case class Remainder(left: Expression, right: Expression) extends BinaryArithmet if (${eval1.isNull} || ${eval2.isNull} || $test) { ${ev.isNull} = true; } else { - ${ev.primitive} = ${eval1.primitive}$method(${eval2.primitive}); + ${ev.primitive} = ($javaType) (${eval1.primitive}$method(${eval2.primitive})); } """ } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala index 5ff1bca260b24..3f4843259e80b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ArithmeticExpressionSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import org.scalatest.Matchers._ - import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.types.{Decimal, DoubleType, IntegerType} @@ -26,100 +24,103 @@ import org.apache.spark.sql.types.{Decimal, DoubleType, IntegerType} class ArithmeticExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { - test("arithmetic") { - val row = create_row(1, 2, 3, null) - val c1 = 'a.int.at(0) - val c2 = 'a.int.at(1) - val c3 = 'a.int.at(2) - val c4 = 'a.int.at(3) - - checkEvaluation(UnaryMinus(c1), -1, row) - checkEvaluation(UnaryMinus(Literal.create(100, IntegerType)), -100) - - checkEvaluation(Add(c1, c4), null, row) - checkEvaluation(Add(c1, c2), 3, row) - checkEvaluation(Add(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation(Add(Literal.create(null, IntegerType), c2), null, row) - checkEvaluation( - Add(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) - - checkEvaluation(-c1, -1, row) - checkEvaluation(c1 + c2, 3, row) - checkEvaluation(c1 - c2, -1, row) - checkEvaluation(c1 * c2, 2, row) - checkEvaluation(c1 / c2, 0, row) - checkEvaluation(c1 % c2, 1, row) + /** + * Runs through the testFunc for all numeric data types. + * + * @param testFunc a test function that accepts a conversion function to convert an integer + * into another data type. + */ + private def testNumericDataTypes(testFunc: (Int => Any) => Unit): Unit = { + testFunc(_.toByte) + testFunc(_.toShort) + testFunc(identity) + testFunc(_.toLong) + testFunc(_.toFloat) + testFunc(_.toDouble) + testFunc(Decimal(_)) } - test("fractional arithmetic") { - val row = create_row(1.1, 2.0, 3.1, null) - val c1 = 'a.double.at(0) - val c2 = 'a.double.at(1) - val c3 = 'a.double.at(2) - val c4 = 'a.double.at(3) - - checkEvaluation(UnaryMinus(c1), -1.1, row) - checkEvaluation(UnaryMinus(Literal.create(100.0, DoubleType)), -100.0) - checkEvaluation(Add(c1, c4), null, row) - checkEvaluation(Add(c1, c2), 3.1, row) - checkEvaluation(Add(c1, Literal.create(null, DoubleType)), null, row) - checkEvaluation(Add(Literal.create(null, DoubleType), c2), null, row) - checkEvaluation( - Add(Literal.create(null, DoubleType), Literal.create(null, DoubleType)), null, row) - - checkEvaluation(-c1, -1.1, row) - checkEvaluation(c1 + c2, 3.1, row) - checkDoubleEvaluation(c1 - c2, (-0.9 +- 0.001), row) - checkDoubleEvaluation(c1 * c2, (2.2 +- 0.001), row) - checkDoubleEvaluation(c1 / c2, (0.55 +- 0.001), row) - checkDoubleEvaluation(c3 % c2, (1.1 +- 0.001), row) + test("+ (Add)") { + testNumericDataTypes { convert => + val left = Literal(convert(1)) + val right = Literal(convert(2)) + checkEvaluation(Add(left, right), convert(3)) + checkEvaluation(Add(Literal.create(null, left.dataType), right), null) + checkEvaluation(Add(left, Literal.create(null, right.dataType)), null) + } } - test("Abs") { - def testAbs(convert: (Int) => Any): Unit = { - checkEvaluation(Abs(Literal(convert(0))), convert(0)) - checkEvaluation(Abs(Literal(convert(1))), convert(1)) - checkEvaluation(Abs(Literal(convert(-1))), convert(1)) + test("- (UnaryMinus)") { + testNumericDataTypes { convert => + val input = Literal(convert(1)) + val dataType = input.dataType + checkEvaluation(UnaryMinus(input), convert(-1)) + checkEvaluation(UnaryMinus(Literal.create(null, dataType)), null) } - testAbs(_.toByte) - testAbs(_.toShort) - testAbs(identity) - testAbs(_.toLong) - testAbs(_.toFloat) - testAbs(_.toDouble) - testAbs(Decimal(_)) } - test("Divide") { - checkEvaluation(Divide(Literal(2), Literal(1)), 2) - checkEvaluation(Divide(Literal(1.0), Literal(2.0)), 0.5) + test("- (Minus)") { + testNumericDataTypes { convert => + val left = Literal(convert(1)) + val right = Literal(convert(2)) + checkEvaluation(Subtract(left, right), convert(-1)) + checkEvaluation(Subtract(Literal.create(null, left.dataType), right), null) + checkEvaluation(Subtract(left, Literal.create(null, right.dataType)), null) + } + } + + test("* (Multiply)") { + testNumericDataTypes { convert => + val left = Literal(convert(1)) + val right = Literal(convert(2)) + checkEvaluation(Multiply(left, right), convert(2)) + checkEvaluation(Multiply(Literal.create(null, left.dataType), right), null) + checkEvaluation(Multiply(left, Literal.create(null, right.dataType)), null) + } + } + + test("/ (Divide) basic") { + testNumericDataTypes { convert => + val left = Literal(convert(2)) + val right = Literal(convert(1)) + val dataType = left.dataType + checkEvaluation(Divide(left, right), convert(2)) + checkEvaluation(Divide(Literal.create(null, dataType), right), null) + checkEvaluation(Divide(left, Literal.create(null, right.dataType)), null) + checkEvaluation(Divide(left, Literal(convert(0))), null) // divide by zero + } + } + + test("/ (Divide) for integral type") { + checkEvaluation(Divide(Literal(1.toByte), Literal(2.toByte)), 0.toByte) + checkEvaluation(Divide(Literal(1.toShort), Literal(2.toShort)), 0.toShort) checkEvaluation(Divide(Literal(1), Literal(2)), 0) - checkEvaluation(Divide(Literal(1), Literal(0)), null) - checkEvaluation(Divide(Literal(1.0), Literal(0.0)), null) - checkEvaluation(Divide(Literal(0.0), Literal(0.0)), null) - checkEvaluation(Divide(Literal(0), Literal.create(null, IntegerType)), null) - checkEvaluation(Divide(Literal(1), Literal.create(null, IntegerType)), null) - checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(0)), null) - checkEvaluation(Divide(Literal.create(null, DoubleType), Literal(0.0)), null) - checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(1)), null) - checkEvaluation(Divide(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), - null) + checkEvaluation(Divide(Literal(1.toLong), Literal(2.toLong)), 0.toLong) } - test("Remainder") { - checkEvaluation(Remainder(Literal(2), Literal(1)), 0) - checkEvaluation(Remainder(Literal(1.0), Literal(2.0)), 1.0) - checkEvaluation(Remainder(Literal(1), Literal(2)), 1) - checkEvaluation(Remainder(Literal(1), Literal(0)), null) - checkEvaluation(Remainder(Literal(1.0), Literal(0.0)), null) - checkEvaluation(Remainder(Literal(0.0), Literal(0.0)), null) - checkEvaluation(Remainder(Literal(0), Literal.create(null, IntegerType)), null) - checkEvaluation(Remainder(Literal(1), Literal.create(null, IntegerType)), null) - checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(0)), null) - checkEvaluation(Remainder(Literal.create(null, DoubleType), Literal(0.0)), null) - checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(1)), null) - checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), - null) + test("/ (Divide) for floating point") { + checkEvaluation(Divide(Literal(1.0f), Literal(2.0f)), 0.5f) + checkEvaluation(Divide(Literal(1.0), Literal(2.0)), 0.5) + checkEvaluation(Divide(Literal(Decimal(1.0)), Literal(Decimal(2.0))), Decimal(0.5)) + } + + test("% (Remainder)") { + testNumericDataTypes { convert => + val left = Literal(convert(1)) + val right = Literal(convert(2)) + checkEvaluation(Remainder(left, right), convert(1)) + checkEvaluation(Remainder(Literal.create(null, left.dataType), right), null) + checkEvaluation(Remainder(left, Literal.create(null, right.dataType)), null) + checkEvaluation(Remainder(left, Literal(convert(0))), null) // mod by 0 + } + } + + test("Abs") { + testNumericDataTypes { convert => + checkEvaluation(Abs(Literal(convert(0))), convert(0)) + checkEvaluation(Abs(Literal(convert(1))), convert(1)) + checkEvaluation(Abs(Literal(convert(-1))), convert(1)) + } } test("MaxOf") { From f3f2a4397da164f0ddfa5d60bf441099296c4346 Mon Sep 17 00:00:00 2001 From: Peter Hoffmann Date: Sun, 14 Jun 2015 11:41:16 -0700 Subject: [PATCH 12/13] fix read/write mixup Author: Peter Hoffmann Closes #6815 from hoffmann/patch-1 and squashes the following commits: 2abb6da [Peter Hoffmann] fix read/write mixup --- docs/sql-programming-guide.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index c5ab074e4439f..7fed1bf8829f5 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -995,11 +995,11 @@ List teenagerNames = teenagers.javaRDD().map(new Function() schemaPeople # The DataFrame from the previous example. # DataFrames can be saved as Parquet files, maintaining the schema information. -schemaPeople.read.parquet("people.parquet") +schemaPeople.write.parquet("people.parquet") # Read in the Parquet file created above. Parquet files are self-describing so the schema is preserved. # The result of loading a parquet file is also a DataFrame. -parquetFile = sqlContext.write.parquet("people.parquet") +parquetFile = sqlContext.read.parquet("people.parquet") # Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerTempTable("parquetFile"); From 4eb48ed1dadee80d78ada5d15884dd348c46ad27 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Sun, 14 Jun 2015 11:49:16 -0700 Subject: [PATCH 13/13] [SPARK-8065] [SQL] Add support for Hive 0.14 metastores This change has two parts. The first one gets rid of "ReflectionMagic". That worked well for the differences between 0.12 and 0.13, but breaks in 0.14, since some of the APIs that need to be used have primitive types. I could not figure out a way to make that class work with primitive types. So instead I wrote some shims (I can already hear the collective sigh) that find the appropriate methods via reflection. This should be faster since the method instances are cached, and the code is not much uglier than before, with the advantage that all the ugliness is local to one file (instead of multiple switch statements on the version being used scattered in ClientWrapper). The second part is simple: add code to handle Hive 0.14. A few new methods had to be added to the new shims. Author: Marcelo Vanzin Closes #6627 from vanzin/SPARK-8065 and squashes the following commits: 3fa4270 [Marcelo Vanzin] Indentation style. 4b8a3d4 [Marcelo Vanzin] Fix dep exclusion. be3d0cc [Marcelo Vanzin] Merge branch 'master' into SPARK-8065 ca3fb1e [Marcelo Vanzin] Merge branch 'master' into SPARK-8065 b43f13e [Marcelo Vanzin] Since exclusions seem to work, clean up some of the code. 73bd161 [Marcelo Vanzin] Botched merge. d2ddf01 [Marcelo Vanzin] Comment about excluded dep. 0c929d1 [Marcelo Vanzin] Merge branch 'master' into SPARK-8065 2c3c02e [Marcelo Vanzin] Try to fix tests by adding support for exclusions. 0a03470 [Marcelo Vanzin] Try to fix tests by upgrading calcite dependency. 13b2dfa [Marcelo Vanzin] Fix NPE. 6439d88 [Marcelo Vanzin] Minor style thing. 69b017b [Marcelo Vanzin] Style. a21cad8 [Marcelo Vanzin] Part II: Add shims / version for Hive 0.14. ae98c87 [Marcelo Vanzin] PART I: Get rid of reflection magic. --- .../org/apache/spark/deploy/SparkSubmit.scala | 33 +- .../spark/deploy/SparkSubmitUtilsSuite.scala | 16 +- .../spark/sql/hive/client/ClientWrapper.scala | 86 ++--- .../spark/sql/hive/client/HiveShim.scala | 349 ++++++++++++++++++ .../hive/client/IsolatedClientLoader.scala | 39 +- .../sql/hive/client/ReflectionMagic.scala | 208 ----------- .../spark/sql/hive/client/package.scala | 24 +- .../spark/sql/hive/client/VersionsSuite.scala | 2 +- 8 files changed, 444 insertions(+), 313 deletions(-) create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala delete mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index b8978e25a02d2..cfcc6d355801e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -832,11 +832,7 @@ private[spark] object SparkSubmitUtils { ivyConfName: String, md: DefaultModuleDescriptor): Unit = { // Add scala exclusion rule - val scalaArtifacts = new ArtifactId(new ModuleId("*", "scala-library"), "*", "*", "*") - val scalaDependencyExcludeRule = - new DefaultExcludeRule(scalaArtifacts, ivySettings.getMatcher("glob"), null) - scalaDependencyExcludeRule.addConfiguration(ivyConfName) - md.addExcludeRule(scalaDependencyExcludeRule) + md.addExcludeRule(createExclusion("*:scala-library:*", ivySettings, ivyConfName)) // We need to specify each component explicitly, otherwise we miss spark-streaming-kafka and // other spark-streaming utility components. Underscore is there to differentiate between @@ -845,13 +841,8 @@ private[spark] object SparkSubmitUtils { "sql_", "streaming_", "yarn_", "network-common_", "network-shuffle_", "network-yarn_") components.foreach { comp => - val sparkArtifacts = - new ArtifactId(new ModuleId("org.apache.spark", s"spark-$comp*"), "*", "*", "*") - val sparkDependencyExcludeRule = - new DefaultExcludeRule(sparkArtifacts, ivySettings.getMatcher("glob"), null) - sparkDependencyExcludeRule.addConfiguration(ivyConfName) - - md.addExcludeRule(sparkDependencyExcludeRule) + md.addExcludeRule(createExclusion(s"org.apache.spark:spark-$comp*:*", ivySettings, + ivyConfName)) } } @@ -864,6 +855,7 @@ private[spark] object SparkSubmitUtils { * @param coordinates Comma-delimited string of maven coordinates * @param remoteRepos Comma-delimited string of remote repositories other than maven central * @param ivyPath The path to the local ivy repository + * @param exclusions Exclusions to apply when resolving transitive dependencies * @return The comma-delimited path to the jars of the given maven artifacts including their * transitive dependencies */ @@ -871,6 +863,7 @@ private[spark] object SparkSubmitUtils { coordinates: String, remoteRepos: Option[String], ivyPath: Option[String], + exclusions: Seq[String] = Nil, isTest: Boolean = false): String = { if (coordinates == null || coordinates.trim.isEmpty) { "" @@ -928,6 +921,10 @@ private[spark] object SparkSubmitUtils { // add all supplied maven artifacts as dependencies addDependenciesToIvy(md, artifacts, ivyConfName) + exclusions.foreach { e => + md.addExcludeRule(createExclusion(e + ":*", ivySettings, ivyConfName)) + } + // resolve dependencies val rr: ResolveReport = ivy.resolve(md, resolveOptions) if (rr.hasError) { @@ -944,6 +941,18 @@ private[spark] object SparkSubmitUtils { } } } + + private def createExclusion( + coords: String, + ivySettings: IvySettings, + ivyConfName: String): ExcludeRule = { + val c = extractMavenCoordinates(coords)(0) + val id = new ArtifactId(new ModuleId(c.groupId, c.artifactId), "*", "*", "*") + val rule = new DefaultExcludeRule(id, ivySettings.getMatcher("glob"), null) + rule.addConfiguration(ivyConfName) + rule + } + } /** diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index 07d261cc428c4..3a8da9fb9ea17 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -106,7 +106,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { IvyTestUtils.withRepository(main, None, None) { repo => // end to end val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, Option(repo), - Option(tempIvyPath), true) + Option(tempIvyPath), isTest = true) assert(jarPath.indexOf(tempIvyPath) >= 0, "should use non-default ivy path") } } @@ -115,21 +115,23 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { val main = new MavenCoordinate("my.awesome.lib", "mylib", "0.1") // Local M2 repository IvyTestUtils.withRepository(main, None, Some(SparkSubmitUtils.m2Path)) { repo => - val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, None, None, true) + val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, None, None, + isTest = true) assert(jarPath.indexOf("mylib") >= 0, "should find artifact") } // Local Ivy Repository val settings = new IvySettings val ivyLocal = new File(settings.getDefaultIvyUserDir, "local" + File.separator) IvyTestUtils.withRepository(main, None, Some(ivyLocal), true) { repo => - val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, None, None, true) + val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, None, None, + isTest = true) assert(jarPath.indexOf("mylib") >= 0, "should find artifact") } // Local ivy repository with modified home val dummyIvyLocal = new File(tempIvyPath, "local" + File.separator) IvyTestUtils.withRepository(main, None, Some(dummyIvyLocal), true) { repo => val jarPath = SparkSubmitUtils.resolveMavenCoordinates(main.toString, None, - Some(tempIvyPath), true) + Some(tempIvyPath), isTest = true) assert(jarPath.indexOf("mylib") >= 0, "should find artifact") assert(jarPath.indexOf(tempIvyPath) >= 0, "should be in new ivy path") } @@ -137,7 +139,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { test("dependency not found throws RuntimeException") { intercept[RuntimeException] { - SparkSubmitUtils.resolveMavenCoordinates("a:b:c", None, None, true) + SparkSubmitUtils.resolveMavenCoordinates("a:b:c", None, None, isTest = true) } } @@ -149,12 +151,12 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { components.map(comp => s"org.apache.spark:spark-${comp}2.10:1.2.0").mkString(",") + ",org.apache.spark:spark-core_fake:1.2.0" - val path = SparkSubmitUtils.resolveMavenCoordinates(coordinates, None, None, true) + val path = SparkSubmitUtils.resolveMavenCoordinates(coordinates, None, None, isTest = true) assert(path === "", "should return empty path") val main = MavenCoordinate("org.apache.spark", "spark-streaming-kafka-assembly_2.10", "1.2.0") IvyTestUtils.withRepository(main, None, None) { repo => val files = SparkSubmitUtils.resolveMavenCoordinates(coordinates + "," + main.toString, - Some(repo), None, true) + Some(repo), None, isTest = true) assert(files.indexOf(main.artifactId) >= 0, "Did not return artifact") } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala index 99aa0f1ded3f8..0fcba65ca6129 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala @@ -27,7 +27,7 @@ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.metastore.api.Database import org.apache.hadoop.hive.conf.HiveConf -import org.apache.hadoop.hive.metastore.TableType +import org.apache.hadoop.hive.metastore.{TableType => HTableType} import org.apache.hadoop.hive.metastore.api import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata @@ -59,8 +59,7 @@ private[hive] class ClientWrapper( version: HiveVersion, config: Map[String, String]) extends ClientInterface - with Logging - with ReflectionMagic { + with Logging { // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. private val outputBuffer = new java.io.OutputStream { @@ -90,6 +89,12 @@ private[hive] class ClientWrapper( } } + private val shim = version match { + case hive.v12 => new Shim_v0_12() + case hive.v13 => new Shim_v0_13() + case hive.v14 => new Shim_v0_14() + } + val state = { val original = Thread.currentThread().getContextClassLoader Thread.currentThread().setContextClassLoader(getClass.getClassLoader) @@ -128,14 +133,7 @@ private[hive] class ClientWrapper( val original = Thread.currentThread().getContextClassLoader Thread.currentThread().setContextClassLoader(getClass.getClassLoader) Hive.set(client) - version match { - case hive.v12 => - classOf[SessionState] - .callStatic[SessionState, SessionState]("start", state) - case hive.v13 => - classOf[SessionState] - .callStatic[SessionState, SessionState]("setCurrentSessionState", state) - } + shim.setCurrentSessionState(state) val ret = try f finally { Thread.currentThread().setContextClassLoader(original) } @@ -193,15 +191,12 @@ private[hive] class ClientWrapper( properties = h.getParameters.toMap, serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.toMap, tableType = h.getTableType match { - case TableType.MANAGED_TABLE => ManagedTable - case TableType.EXTERNAL_TABLE => ExternalTable - case TableType.VIRTUAL_VIEW => VirtualView - case TableType.INDEX_TABLE => IndexTable - }, - location = version match { - case hive.v12 => Option(h.call[URI]("getDataLocation")).map(_.toString) - case hive.v13 => Option(h.call[Path]("getDataLocation")).map(_.toString) + case HTableType.MANAGED_TABLE => ManagedTable + case HTableType.EXTERNAL_TABLE => ExternalTable + case HTableType.VIRTUAL_VIEW => VirtualView + case HTableType.INDEX_TABLE => IndexTable }, + location = shim.getDataLocation(h), inputFormat = Option(h.getInputFormatClass).map(_.getName), outputFormat = Option(h.getOutputFormatClass).map(_.getName), serde = Option(h.getSerializationLib), @@ -231,14 +226,7 @@ private[hive] class ClientWrapper( // set create time qlTable.setCreateTime((System.currentTimeMillis() / 1000).asInstanceOf[Int]) - version match { - case hive.v12 => - table.location.map(new URI(_)).foreach(u => qlTable.call[URI, Unit]("setDataLocation", u)) - case hive.v13 => - table.location - .map(new org.apache.hadoop.fs.Path(_)) - .foreach(qlTable.call[Path, Unit]("setDataLocation", _)) - } + table.location.foreach { loc => shim.setDataLocation(qlTable, loc) } table.inputFormat.map(toInputFormat).foreach(qlTable.setInputFormatClass) table.outputFormat.map(toOutputFormat).foreach(qlTable.setOutputFormatClass) table.serde.foreach(qlTable.setSerializationLib) @@ -279,13 +267,7 @@ private[hive] class ClientWrapper( override def getAllPartitions(hTable: HiveTable): Seq[HivePartition] = withHiveState { val qlTable = toQlTable(hTable) - val qlPartitions = version match { - case hive.v12 => - client.call[metadata.Table, JSet[metadata.Partition]]("getAllPartitionsForPruner", qlTable) - case hive.v13 => - client.call[metadata.Table, JSet[metadata.Partition]]("getAllPartitionsOf", qlTable) - } - qlPartitions.toSeq.map(toHivePartition) + shim.getAllPartitions(client, qlTable).map(toHivePartition) } override def listTables(dbName: String): Seq[String] = withHiveState { @@ -315,15 +297,7 @@ private[hive] class ClientWrapper( val tokens: Array[String] = cmd_trimmed.split("\\s+") // The remainder of the command. val cmd_1: String = cmd_trimmed.substring(tokens(0).length()).trim() - val proc: CommandProcessor = version match { - case hive.v12 => - classOf[CommandProcessorFactory] - .callStatic[String, HiveConf, CommandProcessor]("get", tokens(0), conf) - case hive.v13 => - classOf[CommandProcessorFactory] - .callStatic[Array[String], HiveConf, CommandProcessor]("get", Array(tokens(0)), conf) - } - + val proc = shim.getCommandProcessor(tokens(0), conf) proc match { case driver: Driver => val response: CommandProcessorResponse = driver.run(cmd) @@ -334,21 +308,7 @@ private[hive] class ClientWrapper( } driver.setMaxRows(maxRows) - val results = version match { - case hive.v12 => - val res = new JArrayList[String] - driver.call[JArrayList[String], Boolean]("getResults", res) - res.toSeq - case hive.v13 => - val res = new JArrayList[Object] - driver.call[JList[Object], Boolean]("getResults", res) - res.map { r => - r match { - case s: String => s - case a: Array[Object] => a(0).asInstanceOf[String] - } - } - } + val results = shim.getDriverResults(driver) driver.close() results @@ -382,8 +342,8 @@ private[hive] class ClientWrapper( holdDDLTime: Boolean, inheritTableSpecs: Boolean, isSkewedStoreAsSubdir: Boolean): Unit = withHiveState { - - client.loadPartition( + shim.loadPartition( + client, new Path(loadPath), // TODO: Use URI tableName, partSpec, @@ -398,7 +358,8 @@ private[hive] class ClientWrapper( tableName: String, replace: Boolean, holdDDLTime: Boolean): Unit = withHiveState { - client.loadTable( + shim.loadTable( + client, new Path(loadPath), tableName, replace, @@ -413,7 +374,8 @@ private[hive] class ClientWrapper( numDP: Int, holdDDLTime: Boolean, listBucketingEnabled: Boolean): Unit = withHiveState { - client.loadDynamicPartitions( + shim.loadDynamicPartitions( + client, new Path(loadPath), tableName, partSpec, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala new file mode 100644 index 0000000000000..40c167926c8d6 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveShim.scala @@ -0,0 +1,349 @@ +/* + * 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.hive.client + +import java.lang.{Boolean => JBoolean, Integer => JInteger} +import java.lang.reflect.{Method, Modifier} +import java.net.URI +import java.util.{ArrayList => JArrayList, List => JList, Map => JMap, Set => JSet} + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.Driver +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} +import org.apache.hadoop.hive.ql.processors.{CommandProcessor, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.session.SessionState + +/** + * A shim that defines the interface between ClientWrapper and the underlying Hive library used to + * talk to the metastore. Each Hive version has its own implementation of this class, defining + * version-specific version of needed functions. + * + * The guideline for writing shims is: + * - always extend from the previous version unless really not possible + * - initialize methods in lazy vals, both for quicker access for multiple invocations, and to + * avoid runtime errors due to the above guideline. + */ +private[client] sealed abstract class Shim { + + def setCurrentSessionState(state: SessionState): Unit + + /** + * This shim is necessary because the return type is different on different versions of Hive. + * All parameters are the same, though. + */ + def getDataLocation(table: Table): Option[String] + + def setDataLocation(table: Table, loc: String): Unit + + def getAllPartitions(hive: Hive, table: Table): Seq[Partition] + + def getCommandProcessor(token: String, conf: HiveConf): CommandProcessor + + def getDriverResults(driver: Driver): Seq[String] + + def loadPartition( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + holdDDLTime: Boolean, + inheritTableSpecs: Boolean, + isSkewedStoreAsSubdir: Boolean): Unit + + def loadTable( + hive: Hive, + loadPath: Path, + tableName: String, + replace: Boolean, + holdDDLTime: Boolean): Unit + + def loadDynamicPartitions( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + numDP: Int, + holdDDLTime: Boolean, + listBucketingEnabled: Boolean): Unit + + protected def findStaticMethod(klass: Class[_], name: String, args: Class[_]*): Method = { + val method = findMethod(klass, name, args: _*) + require(Modifier.isStatic(method.getModifiers()), + s"Method $name of class $klass is not static.") + method + } + + protected def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { + klass.getMethod(name, args: _*) + } + +} + +private[client] class Shim_v0_12 extends Shim { + + private lazy val startMethod = + findStaticMethod( + classOf[SessionState], + "start", + classOf[SessionState]) + private lazy val getDataLocationMethod = findMethod(classOf[Table], "getDataLocation") + private lazy val setDataLocationMethod = + findMethod( + classOf[Table], + "setDataLocation", + classOf[URI]) + private lazy val getAllPartitionsMethod = + findMethod( + classOf[Hive], + "getAllPartitionsForPruner", + classOf[Table]) + private lazy val getCommandProcessorMethod = + findStaticMethod( + classOf[CommandProcessorFactory], + "get", + classOf[String], + classOf[HiveConf]) + private lazy val getDriverResultsMethod = + findMethod( + classOf[Driver], + "getResults", + classOf[JArrayList[String]]) + private lazy val loadPartitionMethod = + findMethod( + classOf[Hive], + "loadPartition", + classOf[Path], + classOf[String], + classOf[JMap[String, String]], + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE) + private lazy val loadTableMethod = + findMethod( + classOf[Hive], + "loadTable", + classOf[Path], + classOf[String], + JBoolean.TYPE, + JBoolean.TYPE) + private lazy val loadDynamicPartitionsMethod = + findMethod( + classOf[Hive], + "loadDynamicPartitions", + classOf[Path], + classOf[String], + classOf[JMap[String, String]], + JBoolean.TYPE, + JInteger.TYPE, + JBoolean.TYPE, + JBoolean.TYPE) + + override def setCurrentSessionState(state: SessionState): Unit = startMethod.invoke(null, state) + + override def getDataLocation(table: Table): Option[String] = + Option(getDataLocationMethod.invoke(table)).map(_.toString()) + + override def setDataLocation(table: Table, loc: String): Unit = + setDataLocationMethod.invoke(table, new URI(loc)) + + override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = + getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].toSeq + + override def getCommandProcessor(token: String, conf: HiveConf): CommandProcessor = + getCommandProcessorMethod.invoke(null, token, conf).asInstanceOf[CommandProcessor] + + override def getDriverResults(driver: Driver): Seq[String] = { + val res = new JArrayList[String]() + getDriverResultsMethod.invoke(driver, res) + res.toSeq + } + + override def loadPartition( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + holdDDLTime: Boolean, + inheritTableSpecs: Boolean, + isSkewedStoreAsSubdir: Boolean): Unit = { + loadPartitionMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, + holdDDLTime: JBoolean, inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean) + } + + override def loadTable( + hive: Hive, + loadPath: Path, + tableName: String, + replace: Boolean, + holdDDLTime: Boolean): Unit = { + loadTableMethod.invoke(hive, loadPath, tableName, replace: JBoolean, holdDDLTime: JBoolean) + } + + override def loadDynamicPartitions( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + numDP: Int, + holdDDLTime: Boolean, + listBucketingEnabled: Boolean): Unit = { + loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, + numDP: JInteger, holdDDLTime: JBoolean, listBucketingEnabled: JBoolean) + } + +} + +private[client] class Shim_v0_13 extends Shim_v0_12 { + + private lazy val setCurrentSessionStateMethod = + findStaticMethod( + classOf[SessionState], + "setCurrentSessionState", + classOf[SessionState]) + private lazy val setDataLocationMethod = + findMethod( + classOf[Table], + "setDataLocation", + classOf[Path]) + private lazy val getAllPartitionsMethod = + findMethod( + classOf[Hive], + "getAllPartitionsOf", + classOf[Table]) + private lazy val getCommandProcessorMethod = + findStaticMethod( + classOf[CommandProcessorFactory], + "get", + classOf[Array[String]], + classOf[HiveConf]) + private lazy val getDriverResultsMethod = + findMethod( + classOf[Driver], + "getResults", + classOf[JList[Object]]) + + override def setCurrentSessionState(state: SessionState): Unit = + setCurrentSessionStateMethod.invoke(null, state) + + override def setDataLocation(table: Table, loc: String): Unit = + setDataLocationMethod.invoke(table, new Path(loc)) + + override def getAllPartitions(hive: Hive, table: Table): Seq[Partition] = + getAllPartitionsMethod.invoke(hive, table).asInstanceOf[JSet[Partition]].toSeq + + override def getCommandProcessor(token: String, conf: HiveConf): CommandProcessor = + getCommandProcessorMethod.invoke(null, Array(token), conf).asInstanceOf[CommandProcessor] + + override def getDriverResults(driver: Driver): Seq[String] = { + val res = new JArrayList[Object]() + getDriverResultsMethod.invoke(driver, res) + res.map { r => + r match { + case s: String => s + case a: Array[Object] => a(0).asInstanceOf[String] + } + } + } + +} + +private[client] class Shim_v0_14 extends Shim_v0_13 { + + private lazy val loadPartitionMethod = + findMethod( + classOf[Hive], + "loadPartition", + classOf[Path], + classOf[String], + classOf[JMap[String, String]], + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE) + private lazy val loadTableMethod = + findMethod( + classOf[Hive], + "loadTable", + classOf[Path], + classOf[String], + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE) + private lazy val loadDynamicPartitionsMethod = + findMethod( + classOf[Hive], + "loadDynamicPartitions", + classOf[Path], + classOf[String], + classOf[JMap[String, String]], + JBoolean.TYPE, + JInteger.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE) + + override def loadPartition( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + holdDDLTime: Boolean, + inheritTableSpecs: Boolean, + isSkewedStoreAsSubdir: Boolean): Unit = { + loadPartitionMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, + holdDDLTime: JBoolean, inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean, + JBoolean.TRUE, JBoolean.FALSE) + } + + override def loadTable( + hive: Hive, + loadPath: Path, + tableName: String, + replace: Boolean, + holdDDLTime: Boolean): Unit = { + loadTableMethod.invoke(hive, loadPath, tableName, replace: JBoolean, holdDDLTime: JBoolean, + JBoolean.TRUE, JBoolean.FALSE, JBoolean.FALSE) + } + + override def loadDynamicPartitions( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + numDP: Int, + holdDDLTime: Boolean, + listBucketingEnabled: Boolean): Unit = { + loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, replace: JBoolean, + numDP: JInteger, holdDDLTime: JBoolean, listBucketingEnabled: JBoolean, JBoolean.FALSE) + } + +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 16851fdd71a98..69cfc5c3c3216 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.client import java.io.File +import java.lang.reflect.InvocationTargetException import java.net.{URL, URLClassLoader} import java.util @@ -28,6 +29,7 @@ import org.apache.commons.io.{FileUtils, IOUtils} import org.apache.spark.Logging import org.apache.spark.deploy.SparkSubmitUtils +import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.util.quietly import org.apache.spark.sql.hive.HiveContext @@ -48,29 +50,27 @@ private[hive] object IsolatedClientLoader { def hiveVersion(version: String): HiveVersion = version match { case "12" | "0.12" | "0.12.0" => hive.v12 case "13" | "0.13" | "0.13.0" | "0.13.1" => hive.v13 + case "14" | "0.14" | "0.14.0" => hive.v14 } private def downloadVersion(version: HiveVersion): Seq[URL] = { - val hiveArtifacts = - (Seq("hive-metastore", "hive-exec", "hive-common", "hive-serde") ++ - (if (version.hasBuiltinsJar) "hive-builtins" :: Nil else Nil)) - .map(a => s"org.apache.hive:$a:${version.fullVersion}") :+ - "com.google.guava:guava:14.0.1" :+ - "org.apache.hadoop:hadoop-client:2.4.0" + val hiveArtifacts = version.extraDeps ++ + Seq("hive-metastore", "hive-exec", "hive-common", "hive-serde") + .map(a => s"org.apache.hive:$a:${version.fullVersion}") ++ + Seq("com.google.guava:guava:14.0.1", + "org.apache.hadoop:hadoop-client:2.4.0") val classpath = quietly { SparkSubmitUtils.resolveMavenCoordinates( hiveArtifacts.mkString(","), Some("http://www.datanucleus.org/downloads/maven2"), - None) + None, + exclusions = version.exclusions) } val allFiles = classpath.split(",").map(new File(_)).toSet // TODO: Remove copy logic. - val tempDir = File.createTempFile("hive", "v" + version.toString) - tempDir.delete() - tempDir.mkdir() - + val tempDir = Utils.createTempDir(namePrefix = s"hive-${version}") allFiles.foreach(f => FileUtils.copyFileToDirectory(f, tempDir)) tempDir.listFiles().map(_.toURL) } @@ -129,7 +129,7 @@ private[hive] class IsolatedClientLoader( /** True if `name` refers to a spark class that must see specific version of Hive. */ protected def isBarrierClass(name: String): Boolean = name.startsWith(classOf[ClientWrapper].getName) || - name.startsWith(classOf[ReflectionMagic].getName) || + name.startsWith(classOf[Shim].getName) || barrierPrefixes.exists(name.startsWith) protected def classToPath(name: String): String = @@ -170,11 +170,16 @@ private[hive] class IsolatedClientLoader( .newInstance(version, config) .asInstanceOf[ClientInterface] } catch { - case ReflectionException(cnf: NoClassDefFoundError) => - throw new ClassNotFoundException( - s"$cnf when creating Hive client using classpath: ${execJars.mkString(", ")}\n" + - "Please make sure that jars for your version of hive and hadoop are included in the " + - s"paths passed to ${HiveContext.HIVE_METASTORE_JARS}.") + case e: InvocationTargetException => + if (e.getCause().isInstanceOf[NoClassDefFoundError]) { + val cnf = e.getCause().asInstanceOf[NoClassDefFoundError] + throw new ClassNotFoundException( + s"$cnf when creating Hive client using classpath: ${execJars.mkString(", ")}\n" + + "Please make sure that jars for your version of hive and hadoop are included in the " + + s"paths passed to ${HiveContext.HIVE_METASTORE_JARS}.") + } else { + throw e + } } finally { Thread.currentThread.setContextClassLoader(baseClassLoader) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala deleted file mode 100644 index 4d053ae42c2ea..0000000000000 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ReflectionMagic.scala +++ /dev/null @@ -1,208 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.hive.client - -import scala.reflect._ - -/** Unwraps reflection exceptions. */ -private[client] object ReflectionException { - def unapply(a: Throwable): Option[Throwable] = a match { - case ite: java.lang.reflect.InvocationTargetException => Option(ite.getCause) - case _ => None - } -} - -/** - * Provides implicit functions on any object for calling methods reflectively. - */ -private[client] trait ReflectionMagic { - /** code for InstanceMagic - println( - (1 to 22).map { n => - def repeat(str: String => String) = (1 to n).map(i => str(i.toString)).mkString(", ") - val types = repeat(n => s"A$n <: AnyRef : ClassTag") - val inArgs = repeat(n => s"a$n: A$n") - val erasure = repeat(n => s"classTag[A$n].erasure") - val outArgs = repeat(n => s"a$n") - s"""|def call[$types, R](name: String, $inArgs): R = { - | clazz.getMethod(name, $erasure).invoke(a, $outArgs).asInstanceOf[R] - |}""".stripMargin - }.mkString("\n") - ) - */ - - // scalastyle:off - protected implicit class InstanceMagic(a: Any) { - private val clazz = a.getClass - - def call[R](name: String): R = { - clazz.getMethod(name).invoke(a).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, R](name: String, a1: A1): R = { - clazz.getMethod(name, classTag[A1].erasure).invoke(a, a1).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure).invoke(a, a1, a2).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure).invoke(a, a1, a2, a3).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure).invoke(a, a1, a2, a3, a4).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure).invoke(a, a1, a2, a3, a4, a5).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure).invoke(a, a1, a2, a3, a4, a5, a6).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21).asInstanceOf[R] - } - def call[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, A22 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21, a22: A22): R = { - clazz.getMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure, classTag[A22].erasure).invoke(a, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21, a22).asInstanceOf[R] - } - } - - /** code for StaticMagic - println( - (1 to 22).map { n => - def repeat(str: String => String) = (1 to n).map(i => str(i.toString)).mkString(", ") - val types = repeat(n => s"A$n <: AnyRef : ClassTag") - val inArgs = repeat(n => s"a$n: A$n") - val erasure = repeat(n => s"classTag[A$n].erasure") - val outArgs = repeat(n => s"a$n") - s"""|def callStatic[$types, R](name: String, $inArgs): R = { - | c.getDeclaredMethod(name, $erasure).invoke(c, $outArgs).asInstanceOf[R] - |}""".stripMargin - }.mkString("\n") - ) - */ - - protected implicit class StaticMagic(c: Class[_]) { - def callStatic[A1 <: AnyRef : ClassTag, R](name: String, a1: A1): R = { - c.getDeclaredMethod(name, classTag[A1].erasure).invoke(c, a1).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure).invoke(c, a1, a2).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure).invoke(c, a1, a2, a3).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure).invoke(c, a1, a2, a3, a4).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure).invoke(c, a1, a2, a3, a4, a5).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure).invoke(c, a1, a2, a3, a4, a5, a6).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21).asInstanceOf[R] - } - def callStatic[A1 <: AnyRef : ClassTag, A2 <: AnyRef : ClassTag, A3 <: AnyRef : ClassTag, A4 <: AnyRef : ClassTag, A5 <: AnyRef : ClassTag, A6 <: AnyRef : ClassTag, A7 <: AnyRef : ClassTag, A8 <: AnyRef : ClassTag, A9 <: AnyRef : ClassTag, A10 <: AnyRef : ClassTag, A11 <: AnyRef : ClassTag, A12 <: AnyRef : ClassTag, A13 <: AnyRef : ClassTag, A14 <: AnyRef : ClassTag, A15 <: AnyRef : ClassTag, A16 <: AnyRef : ClassTag, A17 <: AnyRef : ClassTag, A18 <: AnyRef : ClassTag, A19 <: AnyRef : ClassTag, A20 <: AnyRef : ClassTag, A21 <: AnyRef : ClassTag, A22 <: AnyRef : ClassTag, R](name: String, a1: A1, a2: A2, a3: A3, a4: A4, a5: A5, a6: A6, a7: A7, a8: A8, a9: A9, a10: A10, a11: A11, a12: A12, a13: A13, a14: A14, a15: A15, a16: A16, a17: A17, a18: A18, a19: A19, a20: A20, a21: A21, a22: A22): R = { - c.getDeclaredMethod(name, classTag[A1].erasure, classTag[A2].erasure, classTag[A3].erasure, classTag[A4].erasure, classTag[A5].erasure, classTag[A6].erasure, classTag[A7].erasure, classTag[A8].erasure, classTag[A9].erasure, classTag[A10].erasure, classTag[A11].erasure, classTag[A12].erasure, classTag[A13].erasure, classTag[A14].erasure, classTag[A15].erasure, classTag[A16].erasure, classTag[A17].erasure, classTag[A18].erasure, classTag[A19].erasure, classTag[A20].erasure, classTag[A21].erasure, classTag[A22].erasure).invoke(c, a1, a2, a3, a4, a5, a6, a7, a8, a9, a10, a11, a12, a13, a14, a15, a16, a17, a18, a19, a20, a21, a22).asInstanceOf[R] - } - } - // scalastyle:on -} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala index 410d9881ac214..27a3d8f5896cc 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala @@ -19,15 +19,27 @@ package org.apache.spark.sql.hive /** Support for interacting with different versions of the HiveMetastoreClient */ package object client { - private[client] abstract class HiveVersion(val fullVersion: String, val hasBuiltinsJar: Boolean) + private[client] abstract class HiveVersion( + val fullVersion: String, + val extraDeps: Seq[String] = Nil, + val exclusions: Seq[String] = Nil) // scalastyle:off private[client] object hive { - case object v10 extends HiveVersion("0.10.0", true) - case object v11 extends HiveVersion("0.11.0", false) - case object v12 extends HiveVersion("0.12.0", false) - case object v13 extends HiveVersion("0.13.1", false) + case object v12 extends HiveVersion("0.12.0") + case object v13 extends HiveVersion("0.13.1") + + // Hive 0.14 depends on calcite 0.9.2-incubating-SNAPSHOT which does not exist in + // maven central anymore, so override those with a version that exists. + // + // org.pentaho:pentaho-aggdesigner-algorithm is also nowhere to be found, so exclude + // it explicitly. If it's needed by the metastore client, users will have to dig it + // out of somewhere and use configuration to point Spark at the correct jars. + case object v14 extends HiveVersion("0.14.0", + Seq("org.apache.calcite:calcite-core:1.3.0-incubating", + "org.apache.calcite:calcite-avatica:1.3.0-incubating"), + Seq("org.pentaho:pentaho-aggdesigner-algorithm")) } // scalastyle:on -} \ No newline at end of file +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala index 7eb4842726665..9a571650b6e25 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala @@ -72,7 +72,7 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(getNestedMessages(e) contains "Unknown column 'A0.OWNER_NAME' in 'field list'") } - private val versions = Seq("12", "13") + private val versions = Seq("12", "13", "14") private var client: ClientInterface = null