From 6b3574e68704d58ba41efe0ea4fe928cc166afcd Mon Sep 17 00:00:00 2001 From: Tarek Auel Date: Sat, 4 Jul 2015 01:10:52 -0700 Subject: [PATCH 01/73] [SPARK-8270][SQL] levenshtein distance Jira: https://issues.apache.org/jira/browse/SPARK-8270 Info: I can not build the latest master, it stucks during the build process: `[INFO] Dependency-reduced POM written at: /Users/tarek/test/spark/bagel/dependency-reduced-pom.xml` Author: Tarek Auel Closes #7214 from tarekauel/SPARK-8270 and squashes the following commits: ab348b9 [Tarek Auel] Merge branch 'master' into SPARK-8270 a2ad318 [Tarek Auel] [SPARK-8270] changed order of fields d91b12c [Tarek Auel] [SPARK-8270] python fix adbd075 [Tarek Auel] [SPARK-8270] fixed typo 23185c9 [Tarek Auel] [SPARK-8270] levenshtein distance --- python/pyspark/sql/functions.py | 14 ++++++++ .../catalyst/analysis/FunctionRegistry.scala | 1 + .../expressions/stringOperations.scala | 32 +++++++++++++++++++ .../expressions/StringFunctionsSuite.scala | 9 ++++++ .../org/apache/spark/sql/functions.scala | 23 ++++++++++--- .../spark/sql/DataFrameFunctionsSuite.scala | 6 ++++ 6 files changed, 81 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 69e563ef36e87..49dd0332afe74 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -323,6 +323,20 @@ def explode(col): return Column(jc) +@ignore_unicode_prefix +@since(1.5) +def levenshtein(left, right): + """Computes the Levenshtein distance of the two given strings. + + >>> df0 = sqlContext.createDataFrame([('kitten', 'sitting',)], ['l', 'r']) + >>> df0.select(levenshtein('l', 'r').alias('d')).collect() + [Row(d=3)] + """ + sc = SparkContext._active_spark_context + jc = sc._jvm.functions.levenshtein(_to_java_column(left), _to_java_column(right)) + return Column(jc) + + @ignore_unicode_prefix @since(1.5) def md5(col): 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 e249b58927cc4..92a50e7092317 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 @@ -163,6 +163,7 @@ object FunctionRegistry { expression[Lower]("lcase"), expression[Lower]("lower"), expression[StringLength]("length"), + expression[Levenshtein]("levenshtein"), expression[Substring]("substr"), expression[Substring]("substring"), expression[UnBase64]("unbase64"), 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 154ac3508c0c5..6de40629ff27e 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.util.regex.Pattern +import org.apache.commons.lang3.StringUtils import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.types._ @@ -299,6 +300,37 @@ case class StringLength(child: Expression) extends UnaryExpression with ExpectsI override def prettyName: String = "length" } +/** + * A function that return the Levenshtein distance between the two given strings. + */ +case class Levenshtein(left: Expression, right: Expression) extends BinaryExpression + with ExpectsInputTypes { + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType, StringType) + + override def dataType: DataType = IntegerType + + override def eval(input: InternalRow): Any = { + val leftValue = left.eval(input) + if (leftValue == null) { + null + } else { + val rightValue = right.eval(input) + if(rightValue == null) { + null + } else { + StringUtils.getLevenshteinDistance(leftValue.toString, rightValue.toString) + } + } + } + + override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { + val stringUtils = classOf[StringUtils].getName + nullSafeCodeGen(ctx, ev, (res, left, right) => + s"$res = $stringUtils.getLevenshteinDistance($left.toString(), $right.toString());") + } +} + /** * Returns the numeric value of the first character of str. */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringFunctionsSuite.scala index 468df20442d38..1efbe1a245e83 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/StringFunctionsSuite.scala @@ -274,4 +274,13 @@ class StringFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Decode(Literal.create(null, BinaryType), Literal("utf-8")), null) checkEvaluation(Decode(b, Literal.create(null, StringType)), null, create_row(null)) } + + test("Levenshtein distance") { + checkEvaluation(Levenshtein(Literal.create(null, StringType), Literal("")), null) + checkEvaluation(Levenshtein(Literal(""), Literal.create(null, StringType)), null) + checkEvaluation(Levenshtein(Literal(""), Literal("")), 0) + checkEvaluation(Levenshtein(Literal("abc"), Literal("abc")), 0) + checkEvaluation(Levenshtein(Literal("kitten"), Literal("sitting")), 3) + checkEvaluation(Levenshtein(Literal("frog"), Literal("fog")), 1) + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index b63c6ee8aba4b..e4109da08e0a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1580,21 +1580,36 @@ object functions { ////////////////////////////////////////////////////////////////////////////////////////////// /** - * Computes the length of a given string value - * + * Computes the length of a given string value. + * * @group string_funcs * @since 1.5.0 */ def strlen(e: Column): Column = StringLength(e.expr) /** - * Computes the length of a given string column - * + * Computes the length of a given string column. + * * @group string_funcs * @since 1.5.0 */ def strlen(columnName: String): Column = strlen(Column(columnName)) + /** + * Computes the Levenshtein distance of the two given strings. + * @group string_funcs + * @since 1.5.0 + */ + def levenshtein(l: Column, r: Column): Column = Levenshtein(l.expr, r.expr) + + /** + * Computes the Levenshtein distance of the two given strings. + * @group string_funcs + * @since 1.5.0 + */ + def levenshtein(leftColumnName: String, rightColumnName: String): Column = + levenshtein(Column(leftColumnName), Column(rightColumnName)) + /** * Computes the numeric value of the first character of the specified string value. * diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index bd9fa400e5b34..bc455a922d154 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -226,6 +226,12 @@ class DataFrameFunctionsSuite extends QueryTest { }) } + test("Levenshtein distance") { + val df = Seq(("kitten", "sitting"), ("frog", "fog")).toDF("l", "r") + checkAnswer(df.select(levenshtein("l", "r")), Seq(Row(3), Row(1))) + checkAnswer(df.selectExpr("levenshtein(l, r)"), Seq(Row(3), Row(1))) + } + test("string ascii function") { val df = Seq(("abc", "")).toDF("a", "b") checkAnswer( From 48f7aed686afde70a6f0802c6cb37b0cad0509f1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 4 Jul 2015 01:11:35 -0700 Subject: [PATCH 02/73] Fixed minor style issue with the previous merge. --- sql/core/src/main/scala/org/apache/spark/sql/functions.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index e4109da08e0a4..abcfc0b65020c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1581,7 +1581,7 @@ object functions { /** * Computes the length of a given string value. - * + * * @group string_funcs * @since 1.5.0 */ @@ -1589,7 +1589,7 @@ object functions { /** * Computes the length of a given string column. - * + * * @group string_funcs * @since 1.5.0 */ From 347cab85cd924ffd326f3d1367b3b156ee08052d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 4 Jul 2015 11:55:04 -0700 Subject: [PATCH 03/73] [SQL] More unit tests for implicit type cast & add simpleString to AbstractDataType. Author: Reynold Xin Closes #7221 from rxin/implicit-cast-tests and squashes the following commits: 64b13bd [Reynold Xin] Fixed a bug .. 489b732 [Reynold Xin] [SQL] More unit tests for implicit type cast & add simpleString to AbstractDataType. --- .../sql/catalyst/analysis/CheckAnalysis.scala | 6 ++--- .../spark/sql/types/AbstractDataType.scala | 7 ++++++ .../apache/spark/sql/types/ArrayType.scala | 2 ++ .../apache/spark/sql/types/DecimalType.scala | 2 ++ .../org/apache/spark/sql/types/MapType.scala | 2 ++ .../apache/spark/sql/types/StructType.scala | 2 ++ .../analysis/HiveTypeCoercionSuite.scala | 25 ++++++++++++++++++- 7 files changed, 42 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index 583338da57117..476ac2b7cb474 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -40,7 +40,7 @@ trait CheckAnalysis { def containsMultipleGenerators(exprs: Seq[Expression]): Boolean = { exprs.flatMap(_.collect { case e: Generator => true - }).length >= 1 + }).nonEmpty } def checkAnalysis(plan: LogicalPlan): Unit = { @@ -85,12 +85,12 @@ trait CheckAnalysis { case Aggregate(groupingExprs, aggregateExprs, child) => def checkValidAggregateExpression(expr: Expression): Unit = expr match { case _: AggregateExpression => // OK - case e: Attribute if groupingExprs.find(_ semanticEquals e).isEmpty => + case e: Attribute if !groupingExprs.exists(_.semanticEquals(e)) => failAnalysis( s"expression '${e.prettyString}' is neither present in the group by, " + s"nor is it an aggregate function. " + "Add to group by or wrap in first() if you don't care which value you get.") - case e if groupingExprs.find(_ semanticEquals e).isDefined => // OK + case e if groupingExprs.exists(_.semanticEquals(e)) => // OK case e if e.references.isEmpty => // OK case e => e.children.foreach(checkValidAggregateExpression) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index e5dc99fb625d8..ffefb0e7837e9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -37,6 +37,9 @@ private[sql] abstract class AbstractDataType { * Returns true if this data type is a parent of the `childCandidate`. */ private[sql] def isParentOf(childCandidate: DataType): Boolean + + /** Readable string representation for the type. */ + private[sql] def simpleString: String } @@ -56,6 +59,10 @@ private[sql] class TypeCollection(private val types: Seq[DataType]) extends Abst private[sql] override def defaultConcreteType: DataType = types.head private[sql] override def isParentOf(childCandidate: DataType): Boolean = false + + private[sql] override def simpleString: String = { + types.map(_.simpleString).mkString("(", " or ", ")") + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala index 8ea6cb14c360e..43413ec761e6b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -31,6 +31,8 @@ object ArrayType extends AbstractDataType { private[sql] override def isParentOf(childCandidate: DataType): Boolean = { childCandidate.isInstanceOf[ArrayType] } + + private[sql] override def simpleString: String = "array" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index 434fc037aad4f..127b16ff85bed 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -90,6 +90,8 @@ object DecimalType extends AbstractDataType { childCandidate.isInstanceOf[DecimalType] } + private[sql] override def simpleString: String = "decimal" + val Unlimited: DecimalType = DecimalType(None) private[sql] object Fixed { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala index 2b25617ec6655..868dea13d971e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala @@ -75,6 +75,8 @@ object MapType extends AbstractDataType { childCandidate.isInstanceOf[MapType] } + private[sql] override def simpleString: String = "map" + /** * Construct a [[MapType]] object with the given key type and value type. * The `valueContainsNull` is true. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 7e77b77e73940..3b17566d54d9b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -309,6 +309,8 @@ object StructType extends AbstractDataType { childCandidate.isInstanceOf[StructType] } + private[sql] override def simpleString: String = "struct" + def apply(fields: Seq[StructField]): StructType = StructType(fields.toArray) def apply(fields: java.util.List[StructField]): StructType = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index 60e727c6c7d4d..67d05ab536b7f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.types._ class HiveTypeCoercionSuite extends PlanTest { - test("implicit type cast") { + test("eligible implicit type cast") { def shouldCast(from: DataType, to: AbstractDataType, expected: DataType): Unit = { val got = HiveTypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to) assert(got.map(_.dataType) == Option(expected), @@ -68,6 +68,29 @@ class HiveTypeCoercionSuite extends PlanTest { shouldCast(IntegerType, TypeCollection(BinaryType, IntegerType), IntegerType) shouldCast(BinaryType, TypeCollection(BinaryType, IntegerType), BinaryType) shouldCast(BinaryType, TypeCollection(IntegerType, BinaryType), BinaryType) + + shouldCast(IntegerType, TypeCollection(StringType, BinaryType), StringType) + shouldCast(IntegerType, TypeCollection(BinaryType, StringType), StringType) + } + + test("ineligible implicit type cast") { + def shouldNotCast(from: DataType, to: AbstractDataType): Unit = { + val got = HiveTypeCoercion.ImplicitTypeCasts.implicitCast(Literal.create(null, from), to) + assert(got.isEmpty, s"Should not be able to cast $from to $to, but got $got") + } + + shouldNotCast(IntegerType, DateType) + shouldNotCast(IntegerType, TimestampType) + shouldNotCast(LongType, DateType) + shouldNotCast(LongType, TimestampType) + shouldNotCast(DecimalType.Unlimited, DateType) + shouldNotCast(DecimalType.Unlimited, TimestampType) + + shouldNotCast(IntegerType, TypeCollection(DateType, TimestampType)) + + shouldNotCast(IntegerType, ArrayType) + shouldNotCast(IntegerType, MapType) + shouldNotCast(IntegerType, StructType) } test("tightest common bound for types") { From c991ef5abbb501933b2a68eea1987cf8d88794a5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 4 Jul 2015 11:55:20 -0700 Subject: [PATCH 04/73] [SPARK-8822][SQL] clean up type checking in math.scala. Author: Reynold Xin Closes #7220 from rxin/SPARK-8822 and squashes the following commits: 0cda076 [Reynold Xin] Test cases. 22d0463 [Reynold Xin] Fixed type precedence. beb2a97 [Reynold Xin] [SPARK-8822][SQL] clean up type checking in math.scala. --- .../spark/sql/catalyst/expressions/math.scala | 260 +++++++----------- .../expressions/MathFunctionsSuite.scala | 31 ++- 2 files changed, 123 insertions(+), 168 deletions(-) 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 0fc320fb08876..45b7e4d3405c8 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 @@ -17,10 +17,8 @@ package org.apache.spark.sql.catalyst.expressions -import java.lang.{Long => JLong} -import java.util.Arrays +import java.{lang => jl} -import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -206,7 +204,7 @@ case class Factorial(child: Expression) extends UnaryExpression with ExpectsInpu if (evalE == null) { null } else { - val input = evalE.asInstanceOf[Integer] + val input = evalE.asInstanceOf[jl.Integer] if (input > 20 || input < 0) { null } else { @@ -290,7 +288,7 @@ case class Bin(child: Expression) if (evalE == null) { null } else { - UTF8String.fromString(JLong.toBinaryString(evalE.asInstanceOf[Long])) + UTF8String.fromString(jl.Long.toBinaryString(evalE.asInstanceOf[Long])) } } @@ -300,27 +298,18 @@ case class Bin(child: Expression) } } - /** * If the argument is an INT or binary, hex returns the number as a STRING in hexadecimal format. * Otherwise if the number is a STRING, it converts each character into its hex representation * and returns the resulting STRING. Negative numbers would be treated as two's complement. */ -case class Hex(child: Expression) extends UnaryExpression with Serializable { +case class Hex(child: Expression) extends UnaryExpression with ExpectsInputTypes { + // TODO: Create code-gen version. - override def dataType: DataType = StringType + override def inputTypes: Seq[AbstractDataType] = + Seq(TypeCollection(LongType, StringType, BinaryType)) - override def checkInputDataTypes(): TypeCheckResult = { - if (child.dataType.isInstanceOf[StringType] - || child.dataType.isInstanceOf[IntegerType] - || child.dataType.isInstanceOf[LongType] - || child.dataType.isInstanceOf[BinaryType] - || child.dataType == NullType) { - TypeCheckResult.TypeCheckSuccess - } else { - TypeCheckResult.TypeCheckFailure(s"hex doesn't accepts ${child.dataType} type") - } - } + override def dataType: DataType = StringType override def eval(input: InternalRow): Any = { val num = child.eval(input) @@ -329,7 +318,6 @@ case class Hex(child: Expression) extends UnaryExpression with Serializable { } else { child.dataType match { case LongType => hex(num.asInstanceOf[Long]) - case IntegerType => hex(num.asInstanceOf[Integer].toLong) case BinaryType => hex(num.asInstanceOf[Array[Byte]]) case StringType => hex(num.asInstanceOf[UTF8String]) } @@ -371,7 +359,55 @@ case class Hex(child: Expression) extends UnaryExpression with Serializable { Character.toUpperCase(Character.forDigit((numBuf & 0xF).toInt, 16)).toByte numBuf >>>= 4 } while (numBuf != 0) - UTF8String.fromBytes(Arrays.copyOfRange(value, value.length - len, value.length)) + UTF8String.fromBytes(java.util.Arrays.copyOfRange(value, value.length - len, value.length)) + } +} + + +/** + * Performs the inverse operation of HEX. + * Resulting characters are returned as a byte array. + */ +case class UnHex(child: Expression) extends UnaryExpression with ExpectsInputTypes { + // TODO: Create code-gen version. + + override def inputTypes: Seq[AbstractDataType] = Seq(StringType) + + override def dataType: DataType = BinaryType + + override def eval(input: InternalRow): Any = { + val num = child.eval(input) + if (num == null) { + null + } else { + unhex(num.asInstanceOf[UTF8String].getBytes) + } + } + + private val unhexDigits = { + val array = Array.fill[Byte](128)(-1) + (0 to 9).foreach(i => array('0' + i) = i.toByte) + (0 to 5).foreach(i => array('A' + i) = (i + 10).toByte) + (0 to 5).foreach(i => array('a' + i) = (i + 10).toByte) + array + } + + private def unhex(inputBytes: Array[Byte]): Array[Byte] = { + var bytes = inputBytes + if ((bytes.length & 0x01) != 0) { + bytes = '0'.toByte +: bytes + } + val out = new Array[Byte](bytes.length >> 1) + // two characters form the hex value. + var i = 0 + while (i < bytes.length) { + val first = unhexDigits(bytes(i)) + val second = unhexDigits(bytes(i + 1)) + if (first == -1 || second == -1) { return null} + out(i / 2) = (((first << 4) | second) & 0xFF).toByte + i += 2 + } + out } } @@ -423,22 +459,19 @@ case class Pow(left: Expression, right: Expression) } } -case class ShiftLeft(left: Expression, right: Expression) extends BinaryExpression { - override def checkInputDataTypes(): TypeCheckResult = { - (left.dataType, right.dataType) match { - case (NullType, _) | (_, NullType) => return TypeCheckResult.TypeCheckSuccess - case (_, IntegerType) => left.dataType match { - case LongType | IntegerType | ShortType | ByteType => - return TypeCheckResult.TypeCheckSuccess - case _ => // failed - } - case _ => // failed - } - TypeCheckResult.TypeCheckFailure( - s"ShiftLeft expects long, integer, short or byte value as first argument and an " + - s"integer value as second argument, not (${left.dataType}, ${right.dataType})") - } +/** + * Bitwise unsigned left shift. + * @param left the base number to shift. + * @param right number of bits to left shift. + */ +case class ShiftLeft(left: Expression, right: Expression) + extends BinaryExpression with ExpectsInputTypes { + + override def inputTypes: Seq[AbstractDataType] = + Seq(TypeCollection(IntegerType, LongType), IntegerType) + + override def dataType: DataType = left.dataType override def eval(input: InternalRow): Any = { val valueLeft = left.eval(input) @@ -446,10 +479,8 @@ case class ShiftLeft(left: Expression, right: Expression) extends BinaryExpressi val valueRight = right.eval(input) if (valueRight != null) { valueLeft match { - case l: Long => l << valueRight.asInstanceOf[Integer] - case i: Integer => i << valueRight.asInstanceOf[Integer] - case s: Short => s << valueRight.asInstanceOf[Integer] - case b: Byte => b << valueRight.asInstanceOf[Integer] + case l: jl.Long => l << valueRight.asInstanceOf[jl.Integer] + case i: jl.Integer => i << valueRight.asInstanceOf[jl.Integer] } } else { null @@ -459,35 +490,24 @@ case class ShiftLeft(left: Expression, right: Expression) extends BinaryExpressi } } - override def dataType: DataType = { - left.dataType match { - case LongType => LongType - case IntegerType | ShortType | ByteType => IntegerType - case _ => NullType - } - } - override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { nullSafeCodeGen(ctx, ev, (result, left, right) => s"$result = $left << $right;") } } -case class ShiftRight(left: Expression, right: Expression) extends BinaryExpression { - override def checkInputDataTypes(): TypeCheckResult = { - (left.dataType, right.dataType) match { - case (NullType, _) | (_, NullType) => return TypeCheckResult.TypeCheckSuccess - case (_, IntegerType) => left.dataType match { - case LongType | IntegerType | ShortType | ByteType => - return TypeCheckResult.TypeCheckSuccess - case _ => // failed - } - case _ => // failed - } - TypeCheckResult.TypeCheckFailure( - s"ShiftRight expects long, integer, short or byte value as first argument and an " + - s"integer value as second argument, not (${left.dataType}, ${right.dataType})") - } +/** + * Bitwise unsigned left shift. + * @param left the base number to shift. + * @param right number of bits to left shift. + */ +case class ShiftRight(left: Expression, right: Expression) + extends BinaryExpression with ExpectsInputTypes { + + override def inputTypes: Seq[AbstractDataType] = + Seq(TypeCollection(IntegerType, LongType), IntegerType) + + override def dataType: DataType = left.dataType override def eval(input: InternalRow): Any = { val valueLeft = left.eval(input) @@ -495,10 +515,8 @@ case class ShiftRight(left: Expression, right: Expression) extends BinaryExpress val valueRight = right.eval(input) if (valueRight != null) { valueLeft match { - case l: Long => l >> valueRight.asInstanceOf[Integer] - case i: Integer => i >> valueRight.asInstanceOf[Integer] - case s: Short => s >> valueRight.asInstanceOf[Integer] - case b: Byte => b >> valueRight.asInstanceOf[Integer] + case l: jl.Long => l >> valueRight.asInstanceOf[jl.Integer] + case i: jl.Integer => i >> valueRight.asInstanceOf[jl.Integer] } } else { null @@ -508,35 +526,24 @@ case class ShiftRight(left: Expression, right: Expression) extends BinaryExpress } } - override def dataType: DataType = { - left.dataType match { - case LongType => LongType - case IntegerType | ShortType | ByteType => IntegerType - case _ => NullType - } - } - override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { nullSafeCodeGen(ctx, ev, (result, left, right) => s"$result = $left >> $right;") } } -case class ShiftRightUnsigned(left: Expression, right: Expression) extends BinaryExpression { - override def checkInputDataTypes(): TypeCheckResult = { - (left.dataType, right.dataType) match { - case (NullType, _) | (_, NullType) => return TypeCheckResult.TypeCheckSuccess - case (_, IntegerType) => left.dataType match { - case LongType | IntegerType | ShortType | ByteType => - return TypeCheckResult.TypeCheckSuccess - case _ => // failed - } - case _ => // failed - } - TypeCheckResult.TypeCheckFailure( - s"ShiftRightUnsigned expects long, integer, short or byte value as first argument and an " + - s"integer value as second argument, not (${left.dataType}, ${right.dataType})") - } +/** + * Bitwise unsigned right shift, for integer and long data type. + * @param left the base number. + * @param right the number of bits to right shift. + */ +case class ShiftRightUnsigned(left: Expression, right: Expression) + extends BinaryExpression with ExpectsInputTypes { + + override def inputTypes: Seq[AbstractDataType] = + Seq(TypeCollection(IntegerType, LongType), IntegerType) + + override def dataType: DataType = left.dataType override def eval(input: InternalRow): Any = { val valueLeft = left.eval(input) @@ -544,10 +551,8 @@ case class ShiftRightUnsigned(left: Expression, right: Expression) extends Binar val valueRight = right.eval(input) if (valueRight != null) { valueLeft match { - case l: Long => l >>> valueRight.asInstanceOf[Integer] - case i: Integer => i >>> valueRight.asInstanceOf[Integer] - case s: Short => s >>> valueRight.asInstanceOf[Integer] - case b: Byte => b >>> valueRight.asInstanceOf[Integer] + case l: jl.Long => l >>> valueRight.asInstanceOf[jl.Integer] + case i: jl.Integer => i >>> valueRight.asInstanceOf[jl.Integer] } } else { null @@ -557,74 +562,21 @@ case class ShiftRightUnsigned(left: Expression, right: Expression) extends Binar } } - override def dataType: DataType = { - left.dataType match { - case LongType => LongType - case IntegerType | ShortType | ByteType => IntegerType - case _ => NullType - } - } - override protected def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): String = { nullSafeCodeGen(ctx, ev, (result, left, right) => s"$result = $left >>> $right;") } } -/** - * Performs the inverse operation of HEX. - * Resulting characters are returned as a byte array. - */ -case class UnHex(child: Expression) extends UnaryExpression with Serializable { - - override def dataType: DataType = BinaryType - - override def checkInputDataTypes(): TypeCheckResult = { - if (child.dataType.isInstanceOf[StringType] || child.dataType == NullType) { - TypeCheckResult.TypeCheckSuccess - } else { - TypeCheckResult.TypeCheckFailure(s"unHex accepts String type, not ${child.dataType}") - } - } - - override def eval(input: InternalRow): Any = { - val num = child.eval(input) - if (num == null) { - null - } else { - unhex(num.asInstanceOf[UTF8String].getBytes) - } - } - - private val unhexDigits = { - val array = Array.fill[Byte](128)(-1) - (0 to 9).foreach(i => array('0' + i) = i.toByte) - (0 to 5).foreach(i => array('A' + i) = (i + 10).toByte) - (0 to 5).foreach(i => array('a' + i) = (i + 10).toByte) - array - } - - private def unhex(inputBytes: Array[Byte]): Array[Byte] = { - var bytes = inputBytes - if ((bytes.length & 0x01) != 0) { - bytes = '0'.toByte +: bytes - } - val out = new Array[Byte](bytes.length >> 1) - // two characters form the hex value. - var i = 0 - while (i < bytes.length) { - val first = unhexDigits(bytes(i)) - val second = unhexDigits(bytes(i + 1)) - if (first == -1 || second == -1) { return null} - out(i / 2) = (((first << 4) | second) & 0xFF).toByte - i += 2 - } - out - } -} case class Hypot(left: Expression, right: Expression) extends BinaryMathExpression(math.hypot, "HYPOT") + +/** + * Computes the logarithm of a number. + * @param left the logarithm base, default to e. + * @param right the number to compute the logarithm of. + */ case class Logarithm(left: Expression, right: Expression) extends BinaryMathExpression((c1, c2) => math.log(c2) / math.log(c1), "LOG") { @@ -642,7 +594,7 @@ case class Logarithm(left: Expression, right: Expression) defineCodeGen(ctx, ev, (c1, c2) => s"java.lang.Math.log($c2) / java.lang.Math.log($c1)") } logCode + s""" - if (Double.valueOf(${ev.primitive}).isNaN()) { + if (Double.isNaN(${ev.primitive})) { ${ev.isNull} = true; } """ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala index 20839c83d4fd0..03d8400cf356b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala @@ -161,11 +161,10 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("factorial") { - val dataLong = (0 to 20) - dataLong.foreach { value => + (0 to 20).foreach { value => checkEvaluation(Factorial(Literal(value)), LongMath.factorial(value), EmptyRow) } - checkEvaluation((Literal.create(null, IntegerType)), null, create_row(null)) + checkEvaluation(Literal.create(null, IntegerType), null, create_row(null)) checkEvaluation(Factorial(Literal(20)), 2432902008176640000L, EmptyRow) checkEvaluation(Factorial(Literal(21)), null, EmptyRow) } @@ -244,10 +243,8 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( ShiftLeft(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null) checkEvaluation(ShiftLeft(Literal(21), Literal(1)), 42) - checkEvaluation(ShiftLeft(Literal(21.toByte), Literal(1)), 42) - checkEvaluation(ShiftLeft(Literal(21.toShort), Literal(1)), 42) - checkEvaluation(ShiftLeft(Literal(21.toLong), Literal(1)), 42.toLong) + checkEvaluation(ShiftLeft(Literal(21.toLong), Literal(1)), 42.toLong) checkEvaluation(ShiftLeft(Literal(-21.toLong), Literal(1)), -42.toLong) } @@ -257,10 +254,8 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( ShiftRight(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null) checkEvaluation(ShiftRight(Literal(42), Literal(1)), 21) - checkEvaluation(ShiftRight(Literal(42.toByte), Literal(1)), 21) - checkEvaluation(ShiftRight(Literal(42.toShort), Literal(1)), 21) - checkEvaluation(ShiftRight(Literal(42.toLong), Literal(1)), 21.toLong) + checkEvaluation(ShiftRight(Literal(42.toLong), Literal(1)), 21.toLong) checkEvaluation(ShiftRight(Literal(-42.toLong), Literal(1)), -21.toLong) } @@ -270,16 +265,12 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( ShiftRight(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null) checkEvaluation(ShiftRightUnsigned(Literal(42), Literal(1)), 21) - checkEvaluation(ShiftRightUnsigned(Literal(42.toByte), Literal(1)), 21) - checkEvaluation(ShiftRightUnsigned(Literal(42.toShort), Literal(1)), 21) - checkEvaluation(ShiftRightUnsigned(Literal(42.toLong), Literal(1)), 21.toLong) + checkEvaluation(ShiftRightUnsigned(Literal(42.toLong), Literal(1)), 21.toLong) checkEvaluation(ShiftRightUnsigned(Literal(-42.toLong), Literal(1)), 9223372036854775787L) } test("hex") { - checkEvaluation(Hex(Literal(28)), "1C") - checkEvaluation(Hex(Literal(-28)), "FFFFFFFFFFFFFFE4") checkEvaluation(Hex(Literal(100800200404L)), "177828FED4") checkEvaluation(Hex(Literal(-100800200404L)), "FFFFFFE887D7012C") checkEvaluation(Hex(Literal("helloHex")), "68656C6C6F486578") @@ -313,6 +304,8 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Logarithm(Literal(v2), Literal(v1)), f(v2 + 0.0, v1 + 0.0), EmptyRow) checkEvaluation(new Logarithm(Literal(v1)), f(math.E, v1 + 0.0), EmptyRow) } + + // null input should yield null output checkEvaluation( Logarithm(Literal.create(null, DoubleType), Literal(1.0)), null, @@ -321,5 +314,15 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { Logarithm(Literal(1.0), Literal.create(null, DoubleType)), null, create_row(null)) + + // negative input should yield null output + checkEvaluation( + Logarithm(Literal(-1.0), Literal(1.0)), + null, + create_row(null)) + checkEvaluation( + Logarithm(Literal(1.0), Literal(-1.0)), + null, + create_row(null)) } } From 2b820f2a4bf9b154762e7516a5b0485322799da9 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 4 Jul 2015 22:52:50 -0700 Subject: [PATCH 05/73] [MINOR] [SQL] Minor fix for CatalystSchemaConverter ping liancheng Author: Liang-Chi Hsieh Closes #7224 from viirya/few_fix_catalystschema and squashes the following commits: d994330 [Liang-Chi Hsieh] Minor fix for CatalystSchemaConverter. --- .../main/scala/org/apache/spark/sql/SQLConf.scala | 2 +- .../spark/sql/parquet/CatalystSchemaConverter.scala | 12 ++++++------ 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 2c258b6ee399c..6005d35f015a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -273,7 +273,7 @@ private[spark] object SQLConf { val PARQUET_FOLLOW_PARQUET_FORMAT_SPEC = booleanConf( key = "spark.sql.parquet.followParquetFormatSpec", defaultValue = Some(false), - doc = "Wether to stick to Parquet format specification when converting Parquet schema to " + + doc = "Whether to stick to Parquet format specification when converting Parquet schema to " + "Spark SQL schema and vice versa. Sticks to the specification if set to true; falls back " + "to compatible mode if set to false.", isPublic = false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala index 2be7c64612cd2..4ab274ec17a02 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala @@ -142,7 +142,7 @@ private[parquet] class CatalystSchemaConverter( DecimalType(precision, scale) } - field.getPrimitiveTypeName match { + typeName match { case BOOLEAN => BooleanType case FLOAT => FloatType @@ -150,7 +150,7 @@ private[parquet] class CatalystSchemaConverter( case DOUBLE => DoubleType case INT32 => - field.getOriginalType match { + originalType match { case INT_8 => ByteType case INT_16 => ShortType case INT_32 | null => IntegerType @@ -161,7 +161,7 @@ private[parquet] class CatalystSchemaConverter( } case INT64 => - field.getOriginalType match { + originalType match { case INT_64 | null => LongType case DECIMAL => makeDecimalType(maxPrecisionForBytes(8)) case TIMESTAMP_MILLIS => typeNotImplemented() @@ -176,7 +176,7 @@ private[parquet] class CatalystSchemaConverter( TimestampType case BINARY => - field.getOriginalType match { + originalType match { case UTF8 | ENUM => StringType case null if assumeBinaryIsString => StringType case null => BinaryType @@ -185,7 +185,7 @@ private[parquet] class CatalystSchemaConverter( } case FIXED_LEN_BYTE_ARRAY => - field.getOriginalType match { + originalType match { case DECIMAL => makeDecimalType(maxPrecisionForBytes(field.getTypeLength)) case INTERVAL => typeNotImplemented() case _ => illegalType() @@ -261,7 +261,7 @@ private[parquet] class CatalystSchemaConverter( // Here we implement Parquet LIST backwards-compatibility rules. // See: https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules // scalastyle:on - private def isElementType(repeatedType: Type, parentName: String) = { + private def isElementType(repeatedType: Type, parentName: String): Boolean = { { // For legacy 2-level list types with primitive element type, e.g.: // From f9c448dce8139e85ac564daa0f7e0325e778cffe Mon Sep 17 00:00:00 2001 From: Joshi Date: Sun, 5 Jul 2015 12:58:03 -0700 Subject: [PATCH 06/73] [SPARK-7137] [ML] Update SchemaUtils checkInputColumn to print more info if needed Author: Joshi Author: Rekha Joshi Closes #5992 from rekhajoshm/fix/SPARK-7137 and squashes the following commits: 8c42b57 [Joshi] update checkInputColumn to print more info if needed 33ddd2e [Joshi] update checkInputColumn to print more info if needed acf3e17 [Joshi] update checkInputColumn to print more info if needed 8993c0e [Joshi] SPARK-7137: Add checkInputColumn back to Params and print more info e3677c9 [Rekha Joshi] Merge pull request #1 from apache/master --- .../scala/org/apache/spark/ml/util/SchemaUtils.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala index 7cd53c6d7ef79..76f651488aef9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala @@ -32,10 +32,15 @@ private[spark] object SchemaUtils { * @param colName column name * @param dataType required column data type */ - def checkColumnType(schema: StructType, colName: String, dataType: DataType): Unit = { + def checkColumnType( + schema: StructType, + colName: String, + dataType: DataType, + msg: String = ""): Unit = { val actualDataType = schema(colName).dataType + val message = if (msg != null && msg.trim.length > 0) " " + msg else "" require(actualDataType.equals(dataType), - s"Column $colName must be of type $dataType but was actually $actualDataType.") + s"Column $colName must be of type $dataType but was actually $actualDataType.$message") } /** From a0cb111b22cb093e86b0daeecb3dcc41d095df40 Mon Sep 17 00:00:00 2001 From: Yu ISHIKAWA Date: Sun, 5 Jul 2015 20:50:02 -0700 Subject: [PATCH 07/73] [SPARK-8549] [SPARKR] Fix the line length of SparkR [[SPARK-8549] Fix the line length of SparkR - ASF JIRA](https://issues.apache.org/jira/browse/SPARK-8549) Author: Yu ISHIKAWA Closes #7204 from yu-iskw/SPARK-8549 and squashes the following commits: 6fb131a [Yu ISHIKAWA] Fix the typo 1737598 [Yu ISHIKAWA] [SPARK-8549][SparkR] Fix the line length of SparkR --- R/pkg/R/generics.R | 3 ++- R/pkg/R/pairRDD.R | 12 ++++++------ R/pkg/R/sparkR.R | 9 ++++++--- R/pkg/R/utils.R | 31 +++++++++++++++++------------- R/pkg/inst/tests/test_includeJAR.R | 4 ++-- R/pkg/inst/tests/test_rdd.R | 12 ++++++++---- R/pkg/inst/tests/test_sparkSQL.R | 11 +++++++++-- 7 files changed, 51 insertions(+), 31 deletions(-) diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 79055b7f18558..fad9d71158c51 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -20,7 +20,8 @@ # @rdname aggregateRDD # @seealso reduce # @export -setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) +setGeneric("aggregateRDD", + function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) # @rdname cache-methods # @export diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 7f902ba8e683e..0f1179e0aa51a 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -560,8 +560,8 @@ setMethod("join", # Left outer join two RDDs # # @description -# \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of the form list(K, V). -# The key types of the two RDDs should be the same. +# \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of +# the form list(K, V). The key types of the two RDDs should be the same. # # @param x An RDD to be joined. Should be an RDD where each element is # list(K, V). @@ -597,8 +597,8 @@ setMethod("leftOuterJoin", # Right outer join two RDDs # # @description -# \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of the form list(K, V). -# The key types of the two RDDs should be the same. +# \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of +# the form list(K, V). The key types of the two RDDs should be the same. # # @param x An RDD to be joined. Should be an RDD where each element is # list(K, V). @@ -634,8 +634,8 @@ setMethod("rightOuterJoin", # Full outer join two RDDs # # @description -# \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). -# The key types of the two RDDs should be the same. +# \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of +# the form list(K, V). The key types of the two RDDs should be the same. # # @param x An RDD to be joined. Should be an RDD where each element is # list(K, V). diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 86233e01db365..048eb8ed541e4 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -105,7 +105,8 @@ sparkR.init <- function( sparkPackages = "") { if (exists(".sparkRjsc", envir = .sparkREnv)) { - cat("Re-using existing Spark Context. Please stop SparkR with sparkR.stop() or restart R to create a new Spark Context\n") + cat(paste("Re-using existing Spark Context.", + "Please stop SparkR with sparkR.stop() or restart R to create a new Spark Context\n")) return(get(".sparkRjsc", envir = .sparkREnv)) } @@ -180,14 +181,16 @@ sparkR.init <- function( sparkExecutorEnvMap <- new.env() if (!any(names(sparkExecutorEnv) == "LD_LIBRARY_PATH")) { - sparkExecutorEnvMap[["LD_LIBRARY_PATH"]] <- paste0("$LD_LIBRARY_PATH:",Sys.getenv("LD_LIBRARY_PATH")) + sparkExecutorEnvMap[["LD_LIBRARY_PATH"]] <- + paste0("$LD_LIBRARY_PATH:",Sys.getenv("LD_LIBRARY_PATH")) } for (varname in names(sparkExecutorEnv)) { sparkExecutorEnvMap[[varname]] <- sparkExecutorEnv[[varname]] } nonEmptyJars <- Filter(function(x) { x != "" }, jars) - localJarPaths <- sapply(nonEmptyJars, function(j) { utils::URLencode(paste("file:", uriSep, j, sep = "")) }) + localJarPaths <- sapply(nonEmptyJars, + function(j) { utils::URLencode(paste("file:", uriSep, j, sep = "")) }) # Set the start time to identify jobjs # Seconds resolution is good enough for this purpose, so use ints diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R index 13cec0f712fb4..ea629a64f7158 100644 --- a/R/pkg/R/utils.R +++ b/R/pkg/R/utils.R @@ -334,18 +334,21 @@ getStorageLevel <- function(newLevel = c("DISK_ONLY", "MEMORY_ONLY_SER_2", "OFF_HEAP")) { match.arg(newLevel) + storageLevelClass <- "org.apache.spark.storage.StorageLevel" storageLevel <- switch(newLevel, - "DISK_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY"), - "DISK_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY_2"), - "MEMORY_AND_DISK" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK"), - "MEMORY_AND_DISK_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_2"), - "MEMORY_AND_DISK_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER"), - "MEMORY_AND_DISK_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER_2"), - "MEMORY_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY"), - "MEMORY_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_2"), - "MEMORY_ONLY_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER"), - "MEMORY_ONLY_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER_2"), - "OFF_HEAP" = callJStatic("org.apache.spark.storage.StorageLevel", "OFF_HEAP")) + "DISK_ONLY" = callJStatic(storageLevelClass, "DISK_ONLY"), + "DISK_ONLY_2" = callJStatic(storageLevelClass, "DISK_ONLY_2"), + "MEMORY_AND_DISK" = callJStatic(storageLevelClass, "MEMORY_AND_DISK"), + "MEMORY_AND_DISK_2" = callJStatic(storageLevelClass, "MEMORY_AND_DISK_2"), + "MEMORY_AND_DISK_SER" = callJStatic(storageLevelClass, + "MEMORY_AND_DISK_SER"), + "MEMORY_AND_DISK_SER_2" = callJStatic(storageLevelClass, + "MEMORY_AND_DISK_SER_2"), + "MEMORY_ONLY" = callJStatic(storageLevelClass, "MEMORY_ONLY"), + "MEMORY_ONLY_2" = callJStatic(storageLevelClass, "MEMORY_ONLY_2"), + "MEMORY_ONLY_SER" = callJStatic(storageLevelClass, "MEMORY_ONLY_SER"), + "MEMORY_ONLY_SER_2" = callJStatic(storageLevelClass, "MEMORY_ONLY_SER_2"), + "OFF_HEAP" = callJStatic(storageLevelClass, "OFF_HEAP")) } # Utility function for functions where an argument needs to be integer but we want to allow @@ -545,9 +548,11 @@ mergePartitions <- function(rdd, zip) { lengthOfKeys <- part[[len - lengthOfValues]] stopifnot(len == lengthOfKeys + lengthOfValues) - # For zip operation, check if corresponding partitions of both RDDs have the same number of elements. + # For zip operation, check if corresponding partitions + # of both RDDs have the same number of elements. if (zip && lengthOfKeys != lengthOfValues) { - stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") + stop(paste("Can only zip RDDs with same number of elements", + "in each pair of corresponding partitions.")) } if (lengthOfKeys > 1) { diff --git a/R/pkg/inst/tests/test_includeJAR.R b/R/pkg/inst/tests/test_includeJAR.R index 844d86f3cc97f..cc1faeabffe30 100644 --- a/R/pkg/inst/tests/test_includeJAR.R +++ b/R/pkg/inst/tests/test_includeJAR.R @@ -18,8 +18,8 @@ context("include an external JAR in SparkContext") runScript <- function() { sparkHome <- Sys.getenv("SPARK_HOME") - jarPath <- paste("--jars", - shQuote(file.path(sparkHome, "R/lib/SparkR/test_support/sparktestjar_2.10-1.0.jar"))) + sparkTestJarPath <- "R/lib/SparkR/test_support/sparktestjar_2.10-1.0.jar" + jarPath <- paste("--jars", shQuote(file.path(sparkHome, sparkTestJarPath))) scriptPath <- file.path(sparkHome, "R/lib/SparkR/tests/jarTest.R") submitPath <- file.path(sparkHome, "bin/spark-submit") res <- system2(command = submitPath, diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index fc3c01d837de4..b79692873cec3 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -669,13 +669,15 @@ test_that("fullOuterJoin() on pairwise RDDs", { rdd1 <- parallelize(sc, list(list(1,2), list(1,3), list(3,3))) rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4)), list(3, list(3, NULL))) + expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), + list(2, list(NULL, 4)), list(3, list(3, NULL))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) rdd1 <- parallelize(sc, list(list("a",2), list("a",3), list("c", 1))) rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) - expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1)), list("c", list(1, NULL))) + expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), + list("a", list(3, 1)), list("c", list(1, NULL))) expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) @@ -683,13 +685,15 @@ test_that("fullOuterJoin() on pairwise RDDs", { rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list(1, list(1, NULL)), list(2, list(2, NULL)), list(3, list(NULL, 3)), list(4, list(NULL, 4))))) + sortKeyValueList(list(list(1, list(1, NULL)), list(2, list(2, NULL)), + list(3, list(NULL, 3)), list(4, list(NULL, 4))))) rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) expect_equal(sortKeyValueList(actual), - sortKeyValueList(list(list("a", list(1, NULL)), list("b", list(2, NULL)), list("d", list(NULL, 4)), list("c", list(NULL, 3))))) + sortKeyValueList(list(list("a", list(1, NULL)), list("b", list(2, NULL)), + list("d", list(NULL, 4)), list("c", list(NULL, 3))))) }) test_that("sortByKey() on pairwise RDDs", { diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R index 0e4235ea8b4b3..b0ea38854304e 100644 --- a/R/pkg/inst/tests/test_sparkSQL.R +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -391,7 +391,7 @@ test_that("collect() and take() on a DataFrame return the same number of rows an expect_equal(ncol(collect(df)), ncol(take(df, 10))) }) -test_that("multiple pipeline transformations starting with a DataFrame result in an RDD with the correct values", { +test_that("multiple pipeline transformations result in an RDD with the correct values", { df <- jsonFile(sqlContext, jsonPath) first <- lapply(df, function(row) { row$age <- row$age + 5 @@ -756,7 +756,14 @@ test_that("toJSON() returns an RDD of the correct values", { test_that("showDF()", { df <- jsonFile(sqlContext, jsonPath) s <- capture.output(showDF(df)) - expect_output(s , "+----+-------+\n| age| name|\n+----+-------+\n|null|Michael|\n| 30| Andy|\n| 19| Justin|\n+----+-------+\n") + expected <- paste("+----+-------+\n", + "| age| name|\n", + "+----+-------+\n", + "|null|Michael|\n", + "| 30| Andy|\n", + "| 19| Justin|\n", + "+----+-------+\n", sep="") + expect_output(s , expected) }) test_that("isLocal()", { From 6d0411b4f3a202cfb53f638ee5fd49072b42d3a6 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Sun, 5 Jul 2015 21:50:52 -0700 Subject: [PATCH 08/73] [SQL][Minor] Update the DataFrame API for encode/decode This is a the follow up of #6843. Author: Cheng Hao Closes #7230 from chenghao-intel/str_funcs2_followup and squashes the following commits: 52cc553 [Cheng Hao] update the code as comment --- .../expressions/stringOperations.scala | 21 ++++++++++--------- .../org/apache/spark/sql/functions.scala | 14 +++++++------ .../spark/sql/DataFrameFunctionsSuite.scala | 8 +++++-- 3 files changed, 25 insertions(+), 18 deletions(-) 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 6de40629ff27e..1a14a7a449342 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 @@ -392,12 +392,13 @@ case class UnBase64(child: Expression) extends UnaryExpression with ExpectsInput /** * Decodes the first argument into a String using the provided character set * (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16'). - * If either argument is null, the result will also be null. (As of Hive 0.12.0.). + * If either argument is null, the result will also be null. */ -case class Decode(bin: Expression, charset: Expression) extends Expression with ExpectsInputTypes { - override def children: Seq[Expression] = bin :: charset :: Nil - override def foldable: Boolean = bin.foldable && charset.foldable - override def nullable: Boolean = bin.nullable || charset.nullable +case class Decode(bin: Expression, charset: Expression) + extends BinaryExpression with ExpectsInputTypes { + + override def left: Expression = bin + override def right: Expression = charset override def dataType: DataType = StringType override def inputTypes: Seq[DataType] = Seq(BinaryType, StringType) @@ -420,13 +421,13 @@ case class Decode(bin: Expression, charset: Expression) extends Expression with /** * Encodes the first argument into a BINARY using the provided character set * (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16'). - * If either argument is null, the result will also be null. (As of Hive 0.12.0.) + * If either argument is null, the result will also be null. */ case class Encode(value: Expression, charset: Expression) - extends Expression with ExpectsInputTypes { - override def children: Seq[Expression] = value :: charset :: Nil - override def foldable: Boolean = value.foldable && charset.foldable - override def nullable: Boolean = value.nullable || charset.nullable + extends BinaryExpression with ExpectsInputTypes { + + override def left: Expression = value + override def right: Expression = charset override def dataType: DataType = BinaryType override def inputTypes: Seq[DataType] = Seq(StringType, StringType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index abcfc0b65020c..f80291776f335 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1666,18 +1666,19 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def encode(value: Column, charset: Column): Column = Encode(value.expr, charset.expr) + def encode(value: Column, charset: String): Column = Encode(value.expr, lit(charset).expr) /** * Computes the first argument into a binary from a string using the provided character set * (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16'). * If either argument is null, the result will also be null. + * NOTE: charset represents the string value of the character set, not the column name. * * @group string_funcs * @since 1.5.0 */ - def encode(columnName: String, charsetColumnName: String): Column = - encode(Column(columnName), Column(charsetColumnName)) + def encode(columnName: String, charset: String): Column = + encode(Column(columnName), charset) /** * Computes the first argument into a string from a binary using the provided character set @@ -1687,18 +1688,19 @@ object functions { * @group string_funcs * @since 1.5.0 */ - def decode(value: Column, charset: Column): Column = Decode(value.expr, charset.expr) + def decode(value: Column, charset: String): Column = Decode(value.expr, lit(charset).expr) /** * Computes the first argument into a string from a binary using the provided character set * (one of 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16'). * If either argument is null, the result will also be null. + * NOTE: charset represents the string value of the character set, not the column name. * * @group string_funcs * @since 1.5.0 */ - def decode(columnName: String, charsetColumnName: String): Column = - decode(Column(columnName), Column(charsetColumnName)) + def decode(columnName: String, charset: String): Column = + decode(Column(columnName), charset) ////////////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala index bc455a922d154..afba28515e032 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala @@ -261,11 +261,15 @@ class DataFrameFunctionsSuite extends QueryTest { // non ascii characters are not allowed in the code, so we disable the scalastyle here. val df = Seq(("大千世界", "utf-8", bytes)).toDF("a", "b", "c") checkAnswer( - df.select(encode($"a", $"b"), encode("a", "b"), decode($"c", $"b"), decode("c", "b")), + df.select( + encode($"a", "utf-8"), + encode("a", "utf-8"), + decode($"c", "utf-8"), + decode("c", "utf-8")), Row(bytes, bytes, "大千世界", "大千世界")) checkAnswer( - df.selectExpr("encode(a, b)", "decode(c, b)"), + df.selectExpr("encode(a, 'utf-8')", "decode(c, 'utf-8')"), Row(bytes, "大千世界")) // scalastyle:on } From 86768b7b3b0c2964e744bc491bc20a1d3140ce93 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 5 Jul 2015 23:54:25 -0700 Subject: [PATCH 09/73] [SPARK-8831][SQL] Support AbstractDataType in TypeCollection. Otherwise it is impossible to declare an expression supporting DecimalType. Author: Reynold Xin Closes #7232 from rxin/typecollection-adt and squashes the following commits: 934d3d1 [Reynold Xin] [SPARK-8831][SQL] Support AbstractDataType in TypeCollection. --- .../spark/sql/catalyst/analysis/HiveTypeCoercion.scala | 2 -- .../org/apache/spark/sql/types/AbstractDataType.scala | 10 ++++++---- .../sql/catalyst/analysis/HiveTypeCoercionSuite.scala | 6 ++++++ 3 files changed, 12 insertions(+), 6 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 84acc0e7e90ec..5367b7f3308ee 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 @@ -708,8 +708,6 @@ object HiveTypeCoercion { case (NullType, target) => Cast(e, target.defaultConcreteType) // Implicit cast among numeric types - // If input is decimal, and we expect a decimal type, just use the input. - case (_: DecimalType, DecimalType) => e // If input is a numeric type but not decimal, and we expect a decimal type, // cast the input to unlimited precision decimal. case (_: NumericType, DecimalType) if !inType.isInstanceOf[DecimalType] => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index ffefb0e7837e9..fb1b47e946214 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -53,10 +53,12 @@ private[sql] abstract class AbstractDataType { * * This means that we prefer StringType over BinaryType if it is possible to cast to StringType. */ -private[sql] class TypeCollection(private val types: Seq[DataType]) extends AbstractDataType { +private[sql] class TypeCollection(private val types: Seq[AbstractDataType]) + extends AbstractDataType { + require(types.nonEmpty, s"TypeCollection ($types) cannot be empty") - private[sql] override def defaultConcreteType: DataType = types.head + private[sql] override def defaultConcreteType: DataType = types.head.defaultConcreteType private[sql] override def isParentOf(childCandidate: DataType): Boolean = false @@ -68,9 +70,9 @@ private[sql] class TypeCollection(private val types: Seq[DataType]) extends Abst private[sql] object TypeCollection { - def apply(types: DataType*): TypeCollection = new TypeCollection(types) + def apply(types: AbstractDataType*): TypeCollection = new TypeCollection(types) - def unapply(typ: AbstractDataType): Option[Seq[DataType]] = typ match { + def unapply(typ: AbstractDataType): Option[Seq[AbstractDataType]] = typ match { case typ: TypeCollection => Some(typ.types) case _ => None } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index 67d05ab536b7f..b56426617789e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -71,6 +71,12 @@ class HiveTypeCoercionSuite extends PlanTest { shouldCast(IntegerType, TypeCollection(StringType, BinaryType), StringType) shouldCast(IntegerType, TypeCollection(BinaryType, StringType), StringType) + + shouldCast( + DecimalType.Unlimited, TypeCollection(IntegerType, DecimalType), DecimalType.Unlimited) + shouldCast(DecimalType(10, 2), TypeCollection(IntegerType, DecimalType), DecimalType(10, 2)) + shouldCast(DecimalType(10, 2), TypeCollection(DecimalType, IntegerType), DecimalType(10, 2)) + shouldCast(IntegerType, TypeCollection(DecimalType(10, 2), StringType), DecimalType(10, 2)) } test("ineligible implicit type cast") { From 39e4e7e4d89077a637c4cad3a986e0e3447d1ae7 Mon Sep 17 00:00:00 2001 From: Steve Lindemann Date: Mon, 6 Jul 2015 10:17:05 -0700 Subject: [PATCH 10/73] [SPARK-8841] [SQL] Fix partition pruning percentage log message When pruning partitions for a query plan, a message is logged indicating what how many partitions were selected based on predicate criteria, and what percent were pruned. The current release erroneously uses `1 - total/selected` to compute this quantity, leading to nonsense messages like "pruned -1000% partitions". The fix is simple and obvious. Author: Steve Lindemann Closes #7227 from srlindemann/master and squashes the following commits: c788061 [Steve Lindemann] fix percentPruned log message --- .../scala/org/apache/spark/sql/sources/DataSourceStrategy.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index ce16e050c56ed..66f7ba90140b5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -65,7 +65,7 @@ private[sql] object DataSourceStrategy extends Strategy with Logging { logInfo { val total = t.partitionSpec.partitions.length val selected = selectedPartitions.length - val percentPruned = (1 - total.toDouble / selected.toDouble) * 100 + val percentPruned = (1 - selected.toDouble / total.toDouble) * 100 s"Selected $selected partitions out of $total, pruned $percentPruned% partitions." } From 293225e0cd9318ad368dde30ac6a17725d33ebb6 Mon Sep 17 00:00:00 2001 From: "Daniel Emaasit (PhD Student)" Date: Mon, 6 Jul 2015 10:36:02 -0700 Subject: [PATCH 11/73] [SPARK-8124] [SPARKR] Created more examples on SparkR DataFrames Here are more examples on SparkR DataFrames including creating a Spark Contect and a SQL context, loading data and simple data manipulation. Author: Daniel Emaasit (PhD Student) Closes #6668 from Emaasit/dan-dev and squashes the following commits: 3a97867 [Daniel Emaasit (PhD Student)] Used fewer rows for createDataFrame f7227f9 [Daniel Emaasit (PhD Student)] Using command line arguments a550f70 [Daniel Emaasit (PhD Student)] Used base R functions 33f9882 [Daniel Emaasit (PhD Student)] Renamed file b6603e3 [Daniel Emaasit (PhD Student)] changed "Describe" function to "describe" 90565dd [Daniel Emaasit (PhD Student)] Deleted the getting-started file b95a103 [Daniel Emaasit (PhD Student)] Deleted this file cc55cd8 [Daniel Emaasit (PhD Student)] combined all the code into one .R file c6933af [Daniel Emaasit (PhD Student)] changed variable name to SQLContext 8e0fe14 [Daniel Emaasit (PhD Student)] provided two options for creating DataFrames 2653573 [Daniel Emaasit (PhD Student)] Updates to a comment and variable name 275b787 [Daniel Emaasit (PhD Student)] Added the Apache License at the top of the file 2e8f724 [Daniel Emaasit (PhD Student)] Added the Apache License at the top of the file 486f44e [Daniel Emaasit (PhD Student)] Added the Apache License at the file d705112 [Daniel Emaasit (PhD Student)] Created more examples on SparkR DataFrames --- examples/src/main/r/data-manipulation.R | 107 ++++++++++++++++++++++++ 1 file changed, 107 insertions(+) create mode 100644 examples/src/main/r/data-manipulation.R diff --git a/examples/src/main/r/data-manipulation.R b/examples/src/main/r/data-manipulation.R new file mode 100644 index 0000000000000..aa2336e300a91 --- /dev/null +++ b/examples/src/main/r/data-manipulation.R @@ -0,0 +1,107 @@ +# +# 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. +# + +# For this example, we shall use the "flights" dataset +# The dataset consists of every flight departing Houston in 2011. +# The data set is made up of 227,496 rows x 14 columns. + +# To run this example use +# ./bin/sparkR --packages com.databricks:spark-csv_2.10:1.0.3 +# examples/src/main/r/data-manipulation.R + +# Load SparkR library into your R session +library(SparkR) + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 1) { + print("Usage: data-manipulation.R % + summarize(avg(flightsDF$dep_delay), avg(flightsDF$arr_delay)) -> dailyDelayDF + + # Print the computed data frame + head(dailyDelayDF) +} + +# Stop the SparkContext now +sparkR.stop() From 0e194645f42be0d6ac9b5a712f8fc1798418736d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 6 Jul 2015 13:26:46 -0700 Subject: [PATCH 12/73] [SPARK-8837][SPARK-7114][SQL] support using keyword in column name Author: Wenchen Fan Closes #7237 from cloud-fan/parser and squashes the following commits: e7b49bb [Wenchen Fan] support using keyword in column name --- .../apache/spark/sql/catalyst/SqlParser.scala | 28 ++++++++++++------- .../org/apache/spark/sql/SQLQuerySuite.scala | 9 ++++++ 2 files changed, 27 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index 8d02fbf4f92c4..e8e9b9802e94b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -287,15 +287,18 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { throw new AnalysisException(s"invalid function approximate($floatLit) $udfName") } } - | CASE ~> expression.? ~ rep1(WHEN ~> expression ~ (THEN ~> expression)) ~ - (ELSE ~> expression).? <~ END ^^ { - case casePart ~ altPart ~ elsePart => - val branches = altPart.flatMap { case whenExpr ~ thenExpr => - Seq(whenExpr, thenExpr) - } ++ elsePart - casePart.map(CaseKeyWhen(_, branches)).getOrElse(CaseWhen(branches)) - } - ) + | CASE ~> whenThenElse ^^ CaseWhen + | CASE ~> expression ~ whenThenElse ^^ + { case keyPart ~ branches => CaseKeyWhen(keyPart, branches) } + ) + + protected lazy val whenThenElse: Parser[List[Expression]] = + rep1(WHEN ~> expression ~ (THEN ~> expression)) ~ (ELSE ~> expression).? <~ END ^^ { + case altPart ~ elsePart => + altPart.flatMap { case whenExpr ~ thenExpr => + Seq(whenExpr, thenExpr) + } ++ elsePart + } protected lazy val cast: Parser[Expression] = CAST ~ "(" ~> expression ~ (AS ~> dataType) <~ ")" ^^ { @@ -354,6 +357,11 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { protected lazy val signedPrimary: Parser[Expression] = sign ~ primary ^^ { case s ~ e => if (s == "-") UnaryMinus(e) else e} + protected lazy val attributeName: Parser[String] = acceptMatch("attribute name", { + case lexical.Identifier(str) => str + case lexical.Keyword(str) if !lexical.delimiters.contains(str) => str + }) + protected lazy val primary: PackratParser[Expression] = ( literal | expression ~ ("[" ~> expression <~ "]") ^^ @@ -364,9 +372,9 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { | "(" ~> expression <~ ")" | function | dotExpressionHeader - | ident ^^ {case i => UnresolvedAttribute.quoted(i)} | signedPrimary | "~" ~> expression ^^ BitwiseNot + | attributeName ^^ UnresolvedAttribute.quoted ) protected lazy val dotExpressionHeader: Parser[Expression] = 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 cc6af1ccc1cce..12ad019e8b473 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 @@ -1458,4 +1458,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll with SQLTestUtils { checkAnswer(sql("SELECT * FROM t ORDER BY NULL"), Seq(Row(1, 2), Row(1, 2))) } } + + test("SPARK-8837: use keyword in column name") { + withTempTable("t") { + val df = Seq(1 -> "a").toDF("count", "sort") + checkAnswer(df.filter("count > 0"), Row(1, "a")) + df.registerTempTable("t") + checkAnswer(sql("select count, sort from t"), Row(1, "a")) + } + } } From 57c72fcce75907c08a1ae53a0d85447176fc3c69 Mon Sep 17 00:00:00 2001 From: Dirceu Semighini Filho Date: Mon, 6 Jul 2015 13:28:07 -0700 Subject: [PATCH 13/73] Small update in the readme file Just change the attribute from -PsparkR to -Psparkr Author: Dirceu Semighini Filho Closes #7242 from dirceusemighini/patch-1 and squashes the following commits: fad5991 [Dirceu Semighini Filho] Small update in the readme file --- R/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/R/README.md b/R/README.md index d7d65b4f0eca5..005f56da1670c 100644 --- a/R/README.md +++ b/R/README.md @@ -6,7 +6,7 @@ SparkR is an R package that provides a light-weight frontend to use Spark from R #### Build Spark -Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-PsparkR` profile to build the R package. For example to use the default Hadoop versions you can run +Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run ``` build/mvn -DskipTests -Psparkr package ``` From 37e4d92142a6309e2df7d36883e0c7892c3d792d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 6 Jul 2015 13:31:31 -0700 Subject: [PATCH 14/73] [SPARK-8784] [SQL] Add Python API for hex and unhex Add Python API for hex/unhex, also cleanup Hex/Unhex Author: Davies Liu Closes #7223 from davies/hex and squashes the following commits: 6f1249d [Davies Liu] no explicit rule to cast string into binary 711a6ed [Davies Liu] fix test f9fe5a3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into hex f032fbb [Davies Liu] Merge branch 'hex' of github.com:davies/spark into hex 49e325f [Davies Liu] Merge branch 'master' of github.com:apache/spark into hex b31fc9a [Davies Liu] Update math.scala 25156b7 [Davies Liu] address comments and fix test c3af78c [Davies Liu] address commments 1a24082 [Davies Liu] Add Python API for hex and unhex --- python/pyspark/sql/functions.py | 28 +++++++ .../catalyst/analysis/FunctionRegistry.scala | 2 +- .../spark/sql/catalyst/expressions/math.scala | 83 ++++++++++--------- .../expressions/MathFunctionsSuite.scala | 25 ++++-- .../org/apache/spark/sql/functions.scala | 2 +- 5 files changed, 93 insertions(+), 47 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 49dd0332afe74..dca39fa833435 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -395,6 +395,34 @@ def randn(seed=None): return Column(jc) +@ignore_unicode_prefix +@since(1.5) +def hex(col): + """Computes hex value of the given column, which could be StringType, + BinaryType, IntegerType or LongType. + + >>> sqlContext.createDataFrame([('ABC', 3)], ['a', 'b']).select(hex('a'), hex('b')).collect() + [Row(hex(a)=u'414243', hex(b)=u'3')] + """ + sc = SparkContext._active_spark_context + jc = sc._jvm.functions.hex(_to_java_column(col)) + return Column(jc) + + +@ignore_unicode_prefix +@since(1.5) +def unhex(col): + """Inverse of hex. Interprets each pair of characters as a hexadecimal number + and converts to the byte representation of number. + + >>> sqlContext.createDataFrame([('414243',)], ['a']).select(unhex('a')).collect() + [Row(unhex(a)=bytearray(b'ABC'))] + """ + sc = SparkContext._active_spark_context + jc = sc._jvm.functions.unhex(_to_java_column(col)) + return Column(jc) + + @ignore_unicode_prefix @since(1.5) def sha1(col): 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 92a50e7092317..fef276353022c 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 @@ -168,7 +168,7 @@ object FunctionRegistry { expression[Substring]("substring"), expression[UnBase64]("unbase64"), expression[Upper]("ucase"), - expression[UnHex]("unhex"), + expression[Unhex]("unhex"), expression[Upper]("upper"), // datetime functions 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 45b7e4d3405c8..92500453980f6 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 @@ -298,6 +298,21 @@ case class Bin(child: Expression) } } +object Hex { + val hexDigits = Array[Char]( + '0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F' + ).map(_.toByte) + + // lookup table to translate '0' -> 0 ... 'F'/'f' -> 15 + val unhexDigits = { + val array = Array.fill[Byte](128)(-1) + (0 to 9).foreach(i => array('0' + i) = i.toByte) + (0 to 5).foreach(i => array('A' + i) = (i + 10).toByte) + (0 to 5).foreach(i => array('a' + i) = (i + 10).toByte) + array + } +} + /** * If the argument is an INT or binary, hex returns the number as a STRING in hexadecimal format. * Otherwise if the number is a STRING, it converts each character into its hex representation @@ -307,7 +322,7 @@ case class Hex(child: Expression) extends UnaryExpression with ExpectsInputTypes // TODO: Create code-gen version. override def inputTypes: Seq[AbstractDataType] = - Seq(TypeCollection(LongType, StringType, BinaryType)) + Seq(TypeCollection(LongType, BinaryType, StringType)) override def dataType: DataType = StringType @@ -319,30 +334,18 @@ case class Hex(child: Expression) extends UnaryExpression with ExpectsInputTypes child.dataType match { case LongType => hex(num.asInstanceOf[Long]) case BinaryType => hex(num.asInstanceOf[Array[Byte]]) - case StringType => hex(num.asInstanceOf[UTF8String]) + case StringType => hex(num.asInstanceOf[UTF8String].getBytes) } } } - /** - * Converts every character in s to two hex digits. - */ - private def hex(str: UTF8String): UTF8String = { - hex(str.getBytes) - } - - private def hex(bytes: Array[Byte]): UTF8String = { - doHex(bytes, bytes.length) - } - - private def doHex(bytes: Array[Byte], length: Int): UTF8String = { + private[this] def hex(bytes: Array[Byte]): UTF8String = { + val length = bytes.length val value = new Array[Byte](length * 2) var i = 0 while (i < length) { - value(i * 2) = Character.toUpperCase(Character.forDigit( - (bytes(i) & 0xF0) >>> 4, 16)).toByte - value(i * 2 + 1) = Character.toUpperCase(Character.forDigit( - bytes(i) & 0x0F, 16)).toByte + value(i * 2) = Hex.hexDigits((bytes(i) & 0xF0) >> 4) + value(i * 2 + 1) = Hex.hexDigits(bytes(i) & 0x0F) i += 1 } UTF8String.fromBytes(value) @@ -355,24 +358,23 @@ case class Hex(child: Expression) extends UnaryExpression with ExpectsInputTypes var len = 0 do { len += 1 - value(value.length - len) = - Character.toUpperCase(Character.forDigit((numBuf & 0xF).toInt, 16)).toByte + value(value.length - len) = Hex.hexDigits((numBuf & 0xF).toInt) numBuf >>>= 4 } while (numBuf != 0) UTF8String.fromBytes(java.util.Arrays.copyOfRange(value, value.length - len, value.length)) } } - /** * Performs the inverse operation of HEX. * Resulting characters are returned as a byte array. */ -case class UnHex(child: Expression) extends UnaryExpression with ExpectsInputTypes { +case class Unhex(child: Expression) extends UnaryExpression with ExpectsInputTypes { // TODO: Create code-gen version. override def inputTypes: Seq[AbstractDataType] = Seq(StringType) + override def nullable: Boolean = true override def dataType: DataType = BinaryType override def eval(input: InternalRow): Any = { @@ -384,26 +386,31 @@ case class UnHex(child: Expression) extends UnaryExpression with ExpectsInputTyp } } - private val unhexDigits = { - val array = Array.fill[Byte](128)(-1) - (0 to 9).foreach(i => array('0' + i) = i.toByte) - (0 to 5).foreach(i => array('A' + i) = (i + 10).toByte) - (0 to 5).foreach(i => array('a' + i) = (i + 10).toByte) - array - } - - private def unhex(inputBytes: Array[Byte]): Array[Byte] = { - var bytes = inputBytes + private[this] def unhex(bytes: Array[Byte]): Array[Byte] = { + val out = new Array[Byte]((bytes.length + 1) >> 1) + var i = 0 if ((bytes.length & 0x01) != 0) { - bytes = '0'.toByte +: bytes + // padding with '0' + if (bytes(0) < 0) { + return null + } + val v = Hex.unhexDigits(bytes(0)) + if (v == -1) { + return null + } + out(0) = v + i += 1 } - val out = new Array[Byte](bytes.length >> 1) // two characters form the hex value. - var i = 0 while (i < bytes.length) { - val first = unhexDigits(bytes(i)) - val second = unhexDigits(bytes(i + 1)) - if (first == -1 || second == -1) { return null} + if (bytes(i) < 0 || bytes(i + 1) < 0) { + return null + } + val first = Hex.unhexDigits(bytes(i)) + val second = Hex.unhexDigits(bytes(i + 1)) + if (first == -1 || second == -1) { + return null + } out(i / 2) = (((first << 4) | second) & 0xFF).toByte i += 2 } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala index 03d8400cf356b..7ca9e30b2bcd5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathFunctionsSuite.scala @@ -21,8 +21,7 @@ import com.google.common.math.LongMath import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.types.{DataType, LongType} -import org.apache.spark.sql.types.{IntegerType, DoubleType} +import org.apache.spark.sql.types._ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -271,20 +270,32 @@ class MathFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("hex") { + checkEvaluation(Hex(Literal.create(null, LongType)), null) + checkEvaluation(Hex(Literal(28L)), "1C") + checkEvaluation(Hex(Literal(-28L)), "FFFFFFFFFFFFFFE4") checkEvaluation(Hex(Literal(100800200404L)), "177828FED4") checkEvaluation(Hex(Literal(-100800200404L)), "FFFFFFE887D7012C") - checkEvaluation(Hex(Literal("helloHex")), "68656C6C6F486578") + checkEvaluation(Hex(Literal.create(null, BinaryType)), null) checkEvaluation(Hex(Literal("helloHex".getBytes())), "68656C6C6F486578") // scalastyle:off // Turn off scala style for non-ascii chars - checkEvaluation(Hex(Literal("三重的")), "E4B889E9878DE79A84") + checkEvaluation(Hex(Literal("三重的".getBytes("UTF8"))), "E4B889E9878DE79A84") // scalastyle:on } test("unhex") { - checkEvaluation(UnHex(Literal("737472696E67")), "string".getBytes) - checkEvaluation(UnHex(Literal("")), new Array[Byte](0)) - checkEvaluation(UnHex(Literal("0")), Array[Byte](0)) + checkEvaluation(Unhex(Literal.create(null, StringType)), null) + checkEvaluation(Unhex(Literal("737472696E67")), "string".getBytes) + checkEvaluation(Unhex(Literal("")), new Array[Byte](0)) + checkEvaluation(Unhex(Literal("F")), Array[Byte](15)) + checkEvaluation(Unhex(Literal("ff")), Array[Byte](-1)) + checkEvaluation(Unhex(Literal("GG")), null) + // scalastyle:off + // Turn off scala style for non-ascii chars + checkEvaluation(Unhex(Literal("E4B889E9878DE79A84")), "三重的".getBytes("UTF-8")) + checkEvaluation(Unhex(Literal("三重的")), null) + + // scalastyle:on } test("hypot") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index f80291776f335..4da9ffc495e17 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -1095,7 +1095,7 @@ object functions { * @group math_funcs * @since 1.5.0 */ - def unhex(column: Column): Column = UnHex(column.expr) + def unhex(column: Column): Column = Unhex(column.expr) /** * Inverse of hex. Interprets each pair of characters as a hexadecimal number From 2471c0bf7f463bb144b44a2e51c0f363e71e099d Mon Sep 17 00:00:00 2001 From: kai Date: Mon, 6 Jul 2015 14:33:30 -0700 Subject: [PATCH 15/73] [SPARK-4485] [SQL] 1) Add broadcast hash outer join, (2) Fix SparkPlanTest This pull request (1) extracts common functions used by hash outer joins and put it in interface HashOuterJoin (2) adds ShuffledHashOuterJoin and BroadcastHashOuterJoin (3) adds test cases for shuffled and broadcast hash outer join (3) makes SparkPlanTest to support binary or more complex operators, and fixes bugs in plan composition in SparkPlanTest Author: kai Closes #7162 from kai-zeng/outer and squashes the following commits: 3742359 [kai] Fix not-serializable exception for code-generated keys in broadcasted relations 14e4bf8 [kai] Use CanBroadcast in broadcast outer join planning dc5127e [kai] code style fixes b5a4efa [kai] (1) Add broadcast hash outer join, (2) Fix SparkPlanTest --- .../spark/sql/execution/SparkStrategies.scala | 12 +- .../joins/BroadcastHashOuterJoin.scala | 121 ++++++++++++++++++ .../sql/execution/joins/HashOuterJoin.scala | 95 ++++---------- .../joins/ShuffledHashOuterJoin.scala | 85 ++++++++++++ .../org/apache/spark/sql/JoinSuite.scala | 40 +++++- .../spark/sql/execution/SparkPlanTest.scala | 99 +++++++++++--- .../sql/execution/joins/OuterJoinSuite.scala | 88 +++++++++++++ 7 files changed, 441 insertions(+), 99 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 5daf86d817586..32044989044a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -117,8 +117,18 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { leftKeys, rightKeys, buildSide, planLater(left), planLater(right)) condition.map(Filter(_, hashJoin)).getOrElse(hashJoin) :: Nil + case ExtractEquiJoinKeys( + LeftOuter, leftKeys, rightKeys, condition, left, CanBroadcast(right)) => + joins.BroadcastHashOuterJoin( + leftKeys, rightKeys, LeftOuter, condition, planLater(left), planLater(right)) :: Nil + + case ExtractEquiJoinKeys( + RightOuter, leftKeys, rightKeys, condition, CanBroadcast(left), right) => + joins.BroadcastHashOuterJoin( + leftKeys, rightKeys, RightOuter, condition, planLater(left), planLater(right)) :: Nil + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) => - joins.HashOuterJoin( + joins.ShuffledHashOuterJoin( leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)) :: Nil case _ => Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala new file mode 100644 index 0000000000000..5da04c78744d9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala @@ -0,0 +1,121 @@ +/* + * 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.execution.joins + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, RightOuter} +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.util.ThreadUtils + +import scala.collection.JavaConversions._ +import scala.concurrent._ +import scala.concurrent.duration._ + +/** + * :: DeveloperApi :: + * Performs a outer hash join for two child relations. When the output RDD of this operator is + * being constructed, a Spark job is asynchronously started to calculate the values for the + * broadcasted relation. This data is then placed in a Spark broadcast variable. The streamed + * relation is not shuffled. + */ +@DeveloperApi +case class BroadcastHashOuterJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode with HashOuterJoin { + + val timeout = { + val timeoutValue = sqlContext.conf.broadcastTimeout + if (timeoutValue < 0) { + Duration.Inf + } else { + timeoutValue.seconds + } + } + + override def requiredChildDistribution: Seq[Distribution] = + UnspecifiedDistribution :: UnspecifiedDistribution :: Nil + + private[this] lazy val (buildPlan, streamedPlan) = joinType match { + case RightOuter => (left, right) + case LeftOuter => (right, left) + case x => + throw new IllegalArgumentException( + s"BroadcastHashOuterJoin should not take $x as the JoinType") + } + + private[this] lazy val (buildKeys, streamedKeys) = joinType match { + case RightOuter => (leftKeys, rightKeys) + case LeftOuter => (rightKeys, leftKeys) + case x => + throw new IllegalArgumentException( + s"BroadcastHashOuterJoin should not take $x as the JoinType") + } + + @transient + private val broadcastFuture = future { + // Note that we use .execute().collect() because we don't want to convert data to Scala types + val input: Array[InternalRow] = buildPlan.execute().map(_.copy()).collect() + // buildHashTable uses code-generated rows as keys, which are not serializable + val hashed = + buildHashTable(input.iterator, new InterpretedProjection(buildKeys, buildPlan.output)) + sparkContext.broadcast(hashed) + }(BroadcastHashOuterJoin.broadcastHashOuterJoinExecutionContext) + + override def doExecute(): RDD[InternalRow] = { + val broadcastRelation = Await.result(broadcastFuture, timeout) + + streamedPlan.execute().mapPartitions { streamedIter => + val joinedRow = new JoinedRow() + val hashTable = broadcastRelation.value + val keyGenerator = newProjection(streamedKeys, streamedPlan.output) + + joinType match { + case LeftOuter => + streamedIter.flatMap(currentRow => { + val rowKey = keyGenerator(currentRow) + joinedRow.withLeft(currentRow) + leftOuterIterator(rowKey, joinedRow, hashTable.getOrElse(rowKey, EMPTY_LIST)) + }) + + case RightOuter => + streamedIter.flatMap(currentRow => { + val rowKey = keyGenerator(currentRow) + joinedRow.withRight(currentRow) + rightOuterIterator(rowKey, hashTable.getOrElse(rowKey, EMPTY_LIST), joinedRow) + }) + + case x => + throw new IllegalArgumentException( + s"BroadcastHashOuterJoin should not take $x as the JoinType") + } + } + } +} + +object BroadcastHashOuterJoin { + + private val broadcastHashOuterJoinExecutionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("broadcast-hash-outer-join", 128)) +} 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 e41538ec1fc1a..886b5fa0c5103 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 @@ -19,32 +19,25 @@ package org.apache.spark.sql.execution.joins import java.util.{HashMap => JavaHashMap} -import org.apache.spark.rdd.RDD - -import scala.collection.JavaConversions._ - import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} -import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.util.collection.CompactBuffer -/** - * :: DeveloperApi :: - * Performs a hash based outer join for two child relations by shuffling the data using - * the join keys. This operator requires loading the associated partition in both side into memory. - */ @DeveloperApi -case class HashOuterJoin( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], - joinType: JoinType, - condition: Option[Expression], - left: SparkPlan, - right: SparkPlan) extends BinaryNode { - - override def outputPartitioning: Partitioning = joinType match { +trait HashOuterJoin { + self: SparkPlan => + + val leftKeys: Seq[Expression] + val rightKeys: Seq[Expression] + val joinType: JoinType + val condition: Option[Expression] + val left: SparkPlan + val right: SparkPlan + +override def outputPartitioning: Partitioning = joinType match { case LeftOuter => left.outputPartitioning case RightOuter => right.outputPartitioning case FullOuter => UnknownPartitioning(left.outputPartitioning.numPartitions) @@ -52,9 +45,6 @@ case class HashOuterJoin( throw new IllegalArgumentException(s"HashOuterJoin should not take $x as the JoinType") } - override def requiredChildDistribution: Seq[ClusteredDistribution] = - ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil - override def output: Seq[Attribute] = { joinType match { case LeftOuter => @@ -68,8 +58,8 @@ case class HashOuterJoin( } } - @transient private[this] lazy val DUMMY_LIST = Seq[InternalRow](null) - @transient private[this] lazy val EMPTY_LIST = Seq.empty[InternalRow] + @transient private[this] lazy val DUMMY_LIST = CompactBuffer[InternalRow](null) + @transient protected[this] lazy val EMPTY_LIST = CompactBuffer[InternalRow]() @transient private[this] lazy val leftNullRow = new GenericInternalRow(left.output.length) @transient private[this] lazy val rightNullRow = new GenericInternalRow(right.output.length) @@ -80,7 +70,7 @@ case class HashOuterJoin( // TODO we need to rewrite all of the iterators with our own implementation instead of the Scala // iterator for performance purpose. - private[this] def leftOuterIterator( + protected[this] def leftOuterIterator( key: InternalRow, joinedRow: JoinedRow, rightIter: Iterable[InternalRow]): Iterator[InternalRow] = { @@ -89,7 +79,7 @@ case class HashOuterJoin( val temp = rightIter.collect { case r if boundCondition(joinedRow.withRight(r)) => joinedRow.copy() } - if (temp.size == 0) { + if (temp.isEmpty) { joinedRow.withRight(rightNullRow).copy :: Nil } else { temp @@ -101,18 +91,17 @@ case class HashOuterJoin( ret.iterator } - private[this] def rightOuterIterator( + protected[this] def rightOuterIterator( key: InternalRow, leftIter: Iterable[InternalRow], joinedRow: JoinedRow): Iterator[InternalRow] = { - val ret: Iterable[InternalRow] = { if (!key.anyNull) { val temp = leftIter.collect { case l if boundCondition(joinedRow.withLeft(l)) => - joinedRow.copy + joinedRow.copy() } - if (temp.size == 0) { + if (temp.isEmpty) { joinedRow.withLeft(leftNullRow).copy :: Nil } else { temp @@ -124,10 +113,9 @@ case class HashOuterJoin( ret.iterator } - private[this] def fullOuterIterator( + protected[this] def fullOuterIterator( key: InternalRow, leftIter: Iterable[InternalRow], rightIter: Iterable[InternalRow], joinedRow: JoinedRow): Iterator[InternalRow] = { - if (!key.anyNull) { // Store the positions of records in right, if one of its associated row satisfy // the join condition. @@ -171,7 +159,7 @@ case class HashOuterJoin( } } - private[this] def buildHashTable( + protected[this] def buildHashTable( iter: Iterator[InternalRow], keyGenerator: Projection): JavaHashMap[InternalRow, CompactBuffer[InternalRow]] = { val hashTable = new JavaHashMap[InternalRow, CompactBuffer[InternalRow]]() @@ -190,43 +178,4 @@ case class HashOuterJoin( hashTable } - - protected override def doExecute(): RDD[InternalRow] = { - val joinedRow = new JoinedRow() - left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => - // TODO this probably can be replaced by external sort (sort merged join?) - - joinType match { - case LeftOuter => - val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) - val keyGenerator = newProjection(leftKeys, left.output) - leftIter.flatMap( currentRow => { - val rowKey = keyGenerator(currentRow) - joinedRow.withLeft(currentRow) - leftOuterIterator(rowKey, joinedRow, rightHashTable.getOrElse(rowKey, EMPTY_LIST)) - }) - - case RightOuter => - val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) - val keyGenerator = newProjection(rightKeys, right.output) - rightIter.flatMap ( currentRow => { - val rowKey = keyGenerator(currentRow) - joinedRow.withRight(currentRow) - rightOuterIterator(rowKey, leftHashTable.getOrElse(rowKey, EMPTY_LIST), joinedRow) - }) - - case FullOuter => - val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) - val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) - (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => - fullOuterIterator(key, - leftHashTable.getOrElse(key, EMPTY_LIST), - rightHashTable.getOrElse(key, EMPTY_LIST), joinedRow) - } - - case x => - throw new IllegalArgumentException(s"HashOuterJoin should not take $x as the JoinType") - } - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala new file mode 100644 index 0000000000000..cfc9c14aaa363 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala @@ -0,0 +1,85 @@ +/* + * 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.execution.joins + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{Distribution, ClusteredDistribution} +import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, LeftOuter, RightOuter} +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} + +import scala.collection.JavaConversions._ + +/** + * :: DeveloperApi :: + * Performs a hash based outer join for two child relations by shuffling the data using + * the join keys. This operator requires loading the associated partition in both side into memory. + */ +@DeveloperApi +case class ShuffledHashOuterJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + joinType: JoinType, + condition: Option[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode with HashOuterJoin { + + override def requiredChildDistribution: Seq[Distribution] = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + protected override def doExecute(): RDD[InternalRow] = { + val joinedRow = new JoinedRow() + left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) => + // TODO this probably can be replaced by external sort (sort merged join?) + joinType match { + case LeftOuter => + val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) + val keyGenerator = newProjection(leftKeys, left.output) + leftIter.flatMap( currentRow => { + val rowKey = keyGenerator(currentRow) + joinedRow.withLeft(currentRow) + leftOuterIterator(rowKey, joinedRow, rightHashTable.getOrElse(rowKey, EMPTY_LIST)) + }) + + case RightOuter => + val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) + val keyGenerator = newProjection(rightKeys, right.output) + rightIter.flatMap ( currentRow => { + val rowKey = keyGenerator(currentRow) + joinedRow.withRight(currentRow) + rightOuterIterator(rowKey, leftHashTable.getOrElse(rowKey, EMPTY_LIST), joinedRow) + }) + + case FullOuter => + val leftHashTable = buildHashTable(leftIter, newProjection(leftKeys, left.output)) + val rightHashTable = buildHashTable(rightIter, newProjection(rightKeys, right.output)) + (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap { key => + fullOuterIterator(key, + leftHashTable.getOrElse(key, EMPTY_LIST), + rightHashTable.getOrElse(key, EMPTY_LIST), + joinedRow) + } + + case x => + throw new IllegalArgumentException( + s"ShuffledHashOuterJoin should not take $x as the JoinType") + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index 20390a5544304..8953889d1fae9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -45,9 +45,10 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { val physical = df.queryExecution.sparkPlan val operators = physical.collect { case j: ShuffledHashJoin => j - case j: HashOuterJoin => j + case j: ShuffledHashOuterJoin => j case j: LeftSemiJoinHash => j case j: BroadcastHashJoin => j + case j: BroadcastHashOuterJoin => j case j: LeftSemiJoinBNL => j case j: CartesianProduct => j case j: BroadcastNestedLoopJoin => j @@ -81,12 +82,13 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[ShuffledHashJoin]), ("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[ShuffledHashJoin]), ("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[ShuffledHashJoin]), - ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[HashOuterJoin]), + ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[ShuffledHashOuterJoin]), ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", - classOf[HashOuterJoin]), + classOf[ShuffledHashOuterJoin]), ("SELECT * FROM testData right join testData2 ON key = a and key = 2", - classOf[HashOuterJoin]), - ("SELECT * FROM testData full outer join testData2 ON key = a", classOf[HashOuterJoin]), + classOf[ShuffledHashOuterJoin]), + ("SELECT * FROM testData full outer join testData2 ON key = a", + classOf[ShuffledHashOuterJoin]), ("SELECT * FROM testData left JOIN testData2 ON (key * a != key + a)", classOf[BroadcastNestedLoopJoin]), ("SELECT * FROM testData right JOIN testData2 ON (key * a != key + a)", @@ -133,6 +135,34 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { ctx.sql("UNCACHE TABLE testData") } + test("broadcasted hash outer join operator selection") { + ctx.cacheManager.clearCache() + ctx.sql("CACHE TABLE testData") + + val SORTMERGEJOIN_ENABLED: Boolean = ctx.conf.sortMergeJoinEnabled + Seq( + ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[ShuffledHashOuterJoin]), + ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", + classOf[BroadcastHashOuterJoin]), + ("SELECT * FROM testData right join testData2 ON key = a and key = 2", + classOf[BroadcastHashOuterJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + try { + ctx.conf.setConf(SQLConf.SORTMERGE_JOIN, true) + Seq( + ("SELECT * FROM testData LEFT JOIN testData2 ON key = a", classOf[ShuffledHashOuterJoin]), + ("SELECT * FROM testData RIGHT JOIN testData2 ON key = a where key = 2", + classOf[BroadcastHashOuterJoin]), + ("SELECT * FROM testData right join testData2 ON key = a and key = 2", + classOf[BroadcastHashOuterJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + } finally { + ctx.conf.setConf(SQLConf.SORTMERGE_JOIN, SORTMERGEJOIN_ENABLED) + } + + ctx.sql("UNCACHE TABLE testData") + } + test("multiple-key equi-join is hash-join") { val x = testData2.as("x") val y = testData2.as("y") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala index 13f3be8ca28d6..108b1122f7bff 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanTest.scala @@ -54,6 +54,37 @@ class SparkPlanTest extends SparkFunSuite { input: DataFrame, planFunction: SparkPlan => SparkPlan, expectedAnswer: Seq[Row]): Unit = { + checkAnswer(input :: Nil, (plans: Seq[SparkPlan]) => planFunction(plans.head), expectedAnswer) + } + + /** + * Runs the plan and makes sure the answer matches the expected result. + * @param left the left input data to be used. + * @param right the right input data to be used. + * @param planFunction a function which accepts the input SparkPlan and uses it to instantiate + * the physical operator that's being tested. + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + */ + protected def checkAnswer( + left: DataFrame, + right: DataFrame, + planFunction: (SparkPlan, SparkPlan) => SparkPlan, + expectedAnswer: Seq[Row]): Unit = { + checkAnswer(left :: right :: Nil, + (plans: Seq[SparkPlan]) => planFunction(plans(0), plans(1)), expectedAnswer) + } + + /** + * Runs the plan and makes sure the answer matches the expected result. + * @param input the input data to be used. + * @param planFunction a function which accepts the input SparkPlan and uses it to instantiate + * the physical operator that's being tested. + * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. + */ + protected def checkAnswer( + input: Seq[DataFrame], + planFunction: Seq[SparkPlan] => SparkPlan, + expectedAnswer: Seq[Row]): Unit = { SparkPlanTest.checkAnswer(input, planFunction, expectedAnswer) match { case Some(errorMessage) => fail(errorMessage) case None => @@ -72,11 +103,41 @@ class SparkPlanTest extends SparkFunSuite { planFunction: SparkPlan => SparkPlan, expectedAnswer: Seq[A]): Unit = { val expectedRows = expectedAnswer.map(Row.fromTuple) - SparkPlanTest.checkAnswer(input, planFunction, expectedRows) match { - case Some(errorMessage) => fail(errorMessage) - case None => - } + checkAnswer(input, planFunction, expectedRows) + } + + /** + * Runs the plan and makes sure the answer matches the expected result. + * @param left the left input data to be used. + * @param right the right input data to be used. + * @param planFunction a function which accepts the input SparkPlan and uses it to instantiate + * the physical operator that's being tested. + * @param expectedAnswer the expected result in a [[Seq]] of [[Product]]s. + */ + protected def checkAnswer[A <: Product : TypeTag]( + left: DataFrame, + right: DataFrame, + planFunction: (SparkPlan, SparkPlan) => SparkPlan, + expectedAnswer: Seq[A]): Unit = { + val expectedRows = expectedAnswer.map(Row.fromTuple) + checkAnswer(left, right, planFunction, expectedRows) + } + + /** + * Runs the plan and makes sure the answer matches the expected result. + * @param input the input data to be used. + * @param planFunction a function which accepts the input SparkPlan and uses it to instantiate + * the physical operator that's being tested. + * @param expectedAnswer the expected result in a [[Seq]] of [[Product]]s. + */ + protected def checkAnswer[A <: Product : TypeTag]( + input: Seq[DataFrame], + planFunction: Seq[SparkPlan] => SparkPlan, + expectedAnswer: Seq[A]): Unit = { + val expectedRows = expectedAnswer.map(Row.fromTuple) + checkAnswer(input, planFunction, expectedRows) } + } /** @@ -92,27 +153,25 @@ object SparkPlanTest { * @param expectedAnswer the expected result in a [[Seq]] of [[Row]]s. */ def checkAnswer( - input: DataFrame, - planFunction: SparkPlan => SparkPlan, + input: Seq[DataFrame], + planFunction: Seq[SparkPlan] => SparkPlan, expectedAnswer: Seq[Row]): Option[String] = { - val outputPlan = planFunction(input.queryExecution.sparkPlan) + val outputPlan = planFunction(input.map(_.queryExecution.sparkPlan)) // A very simple resolver to make writing tests easier. In contrast to the real resolver // this is always case sensitive and does not try to handle scoping or complex type resolution. - val resolvedPlan = outputPlan transform { - case plan: SparkPlan => - val inputMap = plan.children.flatMap(_.output).zipWithIndex.map { - case (a, i) => - (a.name, BoundReference(i, a.dataType, a.nullable)) - }.toMap - - plan.transformExpressions { - case UnresolvedAttribute(Seq(u)) => - inputMap.getOrElse(u, - sys.error(s"Invalid Test: Cannot resolve $u given input $inputMap")) - } - } + val resolvedPlan = TestSQLContext.prepareForExecution.execute( + outputPlan transform { + case plan: SparkPlan => + val inputMap = plan.children.flatMap(_.output).map(a => (a.name, a)).toMap + plan.transformExpressions { + case UnresolvedAttribute(Seq(u)) => + inputMap.getOrElse(u, + sys.error(s"Invalid Test: Cannot resolve $u given input $inputMap")) + } + } + ) def prepareAnswer(answer: Seq[Row]): Seq[Row] = { // Converts data to types that we can do equality comparison using Scala collections. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala new file mode 100644 index 0000000000000..5707d2fb300ae --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala @@ -0,0 +1,88 @@ +/* + * 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.execution.joins + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Expression, LessThan} +import org.apache.spark.sql.catalyst.plans.{FullOuter, LeftOuter, RightOuter} +import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest} + +class OuterJoinSuite extends SparkPlanTest { + + val left = Seq( + (1, 2.0), + (2, 1.0), + (3, 3.0) + ).toDF("a", "b") + + val right = Seq( + (2, 3.0), + (3, 2.0), + (4, 1.0) + ).toDF("c", "d") + + val leftKeys: List[Expression] = 'a :: Nil + val rightKeys: List[Expression] = 'c :: Nil + val condition = Some(LessThan('b, 'd)) + + test("shuffled hash outer join") { + checkAnswer(left, right, (left: SparkPlan, right: SparkPlan) => + ShuffledHashOuterJoin(leftKeys, rightKeys, LeftOuter, condition, left, right), + Seq( + (1, 2.0, null, null), + (2, 1.0, 2, 3.0), + (3, 3.0, null, null) + )) + + checkAnswer(left, right, (left: SparkPlan, right: SparkPlan) => + ShuffledHashOuterJoin(leftKeys, rightKeys, RightOuter, condition, left, right), + Seq( + (2, 1.0, 2, 3.0), + (null, null, 3, 2.0), + (null, null, 4, 1.0) + )) + + checkAnswer(left, right, (left: SparkPlan, right: SparkPlan) => + ShuffledHashOuterJoin(leftKeys, rightKeys, FullOuter, condition, left, right), + Seq( + (1, 2.0, null, null), + (2, 1.0, 2, 3.0), + (3, 3.0, null, null), + (null, null, 3, 2.0), + (null, null, 4, 1.0) + )) + } + + test("broadcast hash outer join") { + checkAnswer(left, right, (left: SparkPlan, right: SparkPlan) => + BroadcastHashOuterJoin(leftKeys, rightKeys, LeftOuter, condition, left, right), + Seq( + (1, 2.0, null, null), + (2, 1.0, 2, 3.0), + (3, 3.0, null, null) + )) + + checkAnswer(left, right, (left: SparkPlan, right: SparkPlan) => + BroadcastHashOuterJoin(leftKeys, rightKeys, RightOuter, condition, left, right), + Seq( + (2, 1.0, 2, 3.0), + (null, null, 3, 2.0), + (null, null, 4, 1.0) + )) + } +} From 132e7fca129be8f00ba429a51bcef60abb2eed6d Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 6 Jul 2015 15:54:43 -0700 Subject: [PATCH 16/73] [MINOR] [SQL] remove unused code in Exchange Author: Daoyuan Wang Closes #7234 from adrian-wang/exchangeclean and squashes the following commits: b093ec9 [Daoyuan Wang] remove unused code --- .../org/apache/spark/sql/execution/Exchange.scala | 14 -------------- 1 file changed, 14 deletions(-) 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 edc64a03335d6..e054c1d144e34 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 @@ -117,20 +117,6 @@ case class Exchange( } } - private val keyOrdering = { - if (newOrdering.nonEmpty) { - val key = newPartitioning.keyExpressions - val boundOrdering = newOrdering.map { o => - val ordinal = key.indexOf(o.child) - if (ordinal == -1) sys.error(s"Invalid ordering on $o requested for $newPartitioning") - o.copy(child = BoundReference(ordinal, o.child.dataType, o.child.nullable)) - } - new RowOrdering(boundOrdering) - } else { - null // Ordering will not be used - } - } - @transient private lazy val sparkConf = child.sqlContext.sparkContext.getConf private def getSerializer( From 9ff203346ca4decf2999e33bfb8c400ec75313e6 Mon Sep 17 00:00:00 2001 From: Wisely Chen Date: Mon, 6 Jul 2015 16:04:01 -0700 Subject: [PATCH 17/73] [SPARK-8656] [WEBUI] Fix the webUI and JSON API number is not synced Spark standalone master web UI show "Alive Workers" total core, total used cores and "Alive workers" total memory, memory used. But the JSON API page "http://MASTERURL:8088/json" shows "ALL workers" core, memory number. This webUI data is not sync with the JSON API. The proper way is to sync the number with webUI and JSON API. Author: Wisely Chen Closes #7038 from thegiive/SPARK-8656 and squashes the following commits: 9e54bf0 [Wisely Chen] Change variable name to camel case 2c8ea89 [Wisely Chen] Change some styling and add local variable 431d2b0 [Wisely Chen] Worker List should contain DEAD node also 8b3b8e8 [Wisely Chen] [SPARK-8656] Fix the webUI and JSON API number is not synced --- .../scala/org/apache/spark/deploy/JsonProtocol.scala | 9 +++++---- .../org/apache/spark/deploy/master/WorkerInfo.scala | 2 ++ 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 2954f932b4f41..ccffb36652988 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -76,12 +76,13 @@ private[deploy] object JsonProtocol { } def writeMasterState(obj: MasterStateResponse): JObject = { + val aliveWorkers = obj.workers.filter(_.isAlive()) ("url" -> obj.uri) ~ ("workers" -> obj.workers.toList.map(writeWorkerInfo)) ~ - ("cores" -> obj.workers.map(_.cores).sum) ~ - ("coresused" -> obj.workers.map(_.coresUsed).sum) ~ - ("memory" -> obj.workers.map(_.memory).sum) ~ - ("memoryused" -> obj.workers.map(_.memoryUsed).sum) ~ + ("cores" -> aliveWorkers.map(_.cores).sum) ~ + ("coresused" -> aliveWorkers.map(_.coresUsed).sum) ~ + ("memory" -> aliveWorkers.map(_.memory).sum) ~ + ("memoryused" -> aliveWorkers.map(_.memoryUsed).sum) ~ ("activeapps" -> obj.activeApps.toList.map(writeApplicationInfo)) ~ ("completedapps" -> obj.completedApps.toList.map(writeApplicationInfo)) ~ ("activedrivers" -> obj.activeDrivers.toList.map(writeDriverInfo)) ~ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala index 471811037e5e2..f751966605206 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/WorkerInfo.scala @@ -105,4 +105,6 @@ private[spark] class WorkerInfo( def setState(state: WorkerState.Value): Unit = { this.state = state } + + def isAlive(): Boolean = this.state == WorkerState.ALIVE } From 1165b17d24cdf1dbebb2faca14308dfe5c2a652c Mon Sep 17 00:00:00 2001 From: Ankur Chauhan Date: Mon, 6 Jul 2015 16:04:57 -0700 Subject: [PATCH 18/73] [SPARK-6707] [CORE] [MESOS] Mesos Scheduler should allow the user to specify constraints based on slave attributes Currently, the mesos scheduler only looks at the 'cpu' and 'mem' resources when trying to determine the usablility of a resource offer from a mesos slave node. It may be preferable for the user to be able to ensure that the spark jobs are only started on a certain set of nodes (based on attributes). For example, If the user sets a property, let's say `spark.mesos.constraints` is set to `tachyon=true;us-east-1=false`, then the resource offers will be checked to see if they meet both these constraints and only then will be accepted to start new executors. Author: Ankur Chauhan Closes #5563 from ankurcha/mesos_attribs and squashes the following commits: 902535b [Ankur Chauhan] Fix line length d83801c [Ankur Chauhan] Update code as per code review comments 8b73f2d [Ankur Chauhan] Fix imports c3523e7 [Ankur Chauhan] Added docs 1a24d0b [Ankur Chauhan] Expand scope of attributes matching to include all data types 482fd71 [Ankur Chauhan] Update access modifier to private[this] for offer constraints 5ccc32d [Ankur Chauhan] Fix nit pick whitespace 1bce782 [Ankur Chauhan] Fix nit pick whitespace c0cbc75 [Ankur Chauhan] Use offer id value for debug message 7fee0ea [Ankur Chauhan] Add debug statements fc7eb5b [Ankur Chauhan] Fix import codestyle 00be252 [Ankur Chauhan] Style changes as per code review comments 662535f [Ankur Chauhan] Incorporate code review comments + use SparkFunSuite fdc0937 [Ankur Chauhan] Decline offers that did not meet criteria 67b58a0 [Ankur Chauhan] Add documentation for spark.mesos.constraints 63f53f4 [Ankur Chauhan] Update codestyle - uniform style for config values 02031e4 [Ankur Chauhan] Fix scalastyle warnings in tests c09ed84 [Ankur Chauhan] Fixed the access modifier on offerConstraints val to private[mesos] 0c64df6 [Ankur Chauhan] Rename overhead fractions to memory_*, fix spacing 8cc1e8f [Ankur Chauhan] Make exception message more explicit about the source of the error addedba [Ankur Chauhan] Added test case for malformed constraint string ec9d9a6 [Ankur Chauhan] Add tests for parse constraint string 72fe88a [Ankur Chauhan] Fix up tests + remove redundant method override, combine utility class into new mesos scheduler util trait 92b47fd [Ankur Chauhan] Add attributes based constraints support to MesosScheduler --- .../mesos/CoarseMesosSchedulerBackend.scala | 43 +++-- .../scheduler/cluster/mesos/MemoryUtils.scala | 31 ---- .../cluster/mesos/MesosClusterScheduler.scala | 1 + .../cluster/mesos/MesosSchedulerBackend.scala | 62 ++++--- .../cluster/mesos/MesosSchedulerUtils.scala | 153 +++++++++++++++++- .../cluster/mesos/MemoryUtilsSuite.scala | 46 ------ .../mesos/MesosSchedulerBackendSuite.scala | 6 +- .../mesos/MesosSchedulerUtilsSuite.scala | 140 ++++++++++++++++ docs/running-on-mesos.md | 22 +++ 9 files changed, 376 insertions(+), 128 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala delete mode 100644 core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtilsSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 6b8edca5aa485..b68f8c7685eba 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -18,18 +18,18 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File -import java.util.{Collections, List => JList} +import java.util.{List => JList} import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, HashSet} -import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} import org.apache.mesos.{Scheduler => MScheduler, _} +import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} +import org.apache.spark.{SparkContext, SparkEnv, SparkException, TaskState} import org.apache.spark.rpc.RpcAddress import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils -import org.apache.spark.{SparkContext, SparkEnv, SparkException, TaskState} /** * A SchedulerBackend that runs tasks on Mesos, but uses "coarse-grained" tasks, where it holds @@ -66,6 +66,10 @@ private[spark] class CoarseMesosSchedulerBackend( val extraCoresPerSlave = conf.getInt("spark.mesos.extra.cores", 0) + // Offer constraints + private val slaveOfferConstraints = + parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) + var nextMesosTaskId = 0 @volatile var appId: String = _ @@ -170,13 +174,16 @@ private[spark] class CoarseMesosSchedulerBackend( override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { synchronized { val filters = Filters.newBuilder().setRefuseSeconds(5).build() - for (offer <- offers) { + val offerAttributes = toAttributeMap(offer.getAttributesList) + val meetsConstraints = matchesAttributeRequirements(slaveOfferConstraints, offerAttributes) val slaveId = offer.getSlaveId.toString val mem = getResource(offer.getResourcesList, "mem") val cpus = getResource(offer.getResourcesList, "cpus").toInt - if (totalCoresAcquired < maxCores && - mem >= MemoryUtils.calculateTotalMemory(sc) && + val id = offer.getId.getValue + if (meetsConstraints && + totalCoresAcquired < maxCores && + mem >= calculateTotalMemory(sc) && cpus >= 1 && failuresBySlaveId.getOrElse(slaveId, 0) < MAX_SLAVE_FAILURES && !slaveIdsWithExecutors.contains(slaveId)) { @@ -193,33 +200,25 @@ private[spark] class CoarseMesosSchedulerBackend( .setCommand(createCommand(offer, cpusToUse + extraCoresPerSlave)) .setName("Task " + taskId) .addResources(createResource("cpus", cpusToUse)) - .addResources(createResource("mem", - MemoryUtils.calculateTotalMemory(sc))) + .addResources(createResource("mem", calculateTotalMemory(sc))) sc.conf.getOption("spark.mesos.executor.docker.image").foreach { image => MesosSchedulerBackendUtil - .setupContainerBuilderDockerInfo(image, sc.conf, task.getContainerBuilder()) + .setupContainerBuilderDockerInfo(image, sc.conf, task.getContainerBuilder) } - d.launchTasks( - Collections.singleton(offer.getId), Collections.singletonList(task.build()), filters) + // accept the offer and launch the task + logDebug(s"Accepting offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus") + d.launchTasks(List(offer.getId), List(task.build()), filters) } else { - // Filter it out - d.launchTasks( - Collections.singleton(offer.getId), Collections.emptyList[MesosTaskInfo](), filters) + // Decline the offer + logDebug(s"Declining offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus") + d.declineOffer(offer.getId) } } } } - /** Build a Mesos resource protobuf object */ - private def createResource(resourceName: String, quantity: Double): Protos.Resource = { - Resource.newBuilder() - .setName(resourceName) - .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(quantity).build()) - .build() - } override def statusUpdate(d: SchedulerDriver, status: TaskStatus) { val taskId = status.getTaskId.getValue.toInt diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala deleted file mode 100644 index 8df4f3b554c41..0000000000000 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtils.scala +++ /dev/null @@ -1,31 +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.scheduler.cluster.mesos - -import org.apache.spark.SparkContext - -private[spark] object MemoryUtils { - // These defaults copied from YARN - val OVERHEAD_FRACTION = 0.10 - val OVERHEAD_MINIMUM = 384 - - def calculateTotalMemory(sc: SparkContext): Int = { - sc.conf.getInt("spark.mesos.executor.memoryOverhead", - math.max(OVERHEAD_FRACTION * sc.executorMemory, OVERHEAD_MINIMUM).toInt) + sc.executorMemory - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 1067a7f1caf4c..d3a20f822176e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -29,6 +29,7 @@ import org.apache.mesos.Protos.Environment.Variable import org.apache.mesos.Protos.TaskStatus.Reason import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} import org.apache.mesos.{Scheduler, SchedulerDriver} + import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} import org.apache.spark.metrics.MetricsSystem diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 49de85ef48ada..d72e2af456e15 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -23,14 +23,14 @@ import java.util.{ArrayList => JArrayList, Collections, List => JList} import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, HashSet} +import org.apache.mesos.{Scheduler => MScheduler, _} import org.apache.mesos.Protos.{ExecutorInfo => MesosExecutorInfo, TaskInfo => MesosTaskInfo, _} import org.apache.mesos.protobuf.ByteString -import org.apache.mesos.{Scheduler => MScheduler, _} +import org.apache.spark.{SparkContext, SparkException, TaskState} import org.apache.spark.executor.MesosExecutorBackend import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils -import org.apache.spark.{SparkContext, SparkException, TaskState} /** * A SchedulerBackend for running fine-grained tasks on Mesos. Each Spark task is mapped to a @@ -59,6 +59,10 @@ private[spark] class MesosSchedulerBackend( private[mesos] val mesosExecutorCores = sc.conf.getDouble("spark.mesos.mesosExecutor.cores", 1) + // Offer constraints + private[this] val slaveOfferConstraints = + parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) + @volatile var appId: String = _ override def start() { @@ -71,8 +75,8 @@ private[spark] class MesosSchedulerBackend( val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home") .orElse(sc.getSparkHome()) // Fall back to driver Spark home for backward compatibility .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") - } + throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + } val environment = Environment.newBuilder() sc.conf.getOption("spark.executor.extraClassPath").foreach { cp => environment.addVariables( @@ -115,14 +119,14 @@ private[spark] class MesosSchedulerBackend( .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Value.Scalar.newBuilder() - .setValue(mesosExecutorCores).build()) + .setValue(mesosExecutorCores).build()) .build() val memory = Resource.newBuilder() .setName("mem") .setType(Value.Type.SCALAR) .setScalar( Value.Scalar.newBuilder() - .setValue(MemoryUtils.calculateTotalMemory(sc)).build()) + .setValue(calculateTotalMemory(sc)).build()) .build() val executorInfo = MesosExecutorInfo.newBuilder() .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) @@ -191,13 +195,31 @@ private[spark] class MesosSchedulerBackend( val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue - (mem >= MemoryUtils.calculateTotalMemory(sc) && - // need at least 1 for executor, 1 for task - cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK)) || - (slaveIdsWithExecutors.contains(slaveId) && - cpus >= scheduler.CPUS_PER_TASK) + val offerAttributes = toAttributeMap(o.getAttributesList) + + // check if all constraints are satisfield + // 1. Attribute constraints + // 2. Memory requirements + // 3. CPU requirements - need at least 1 for executor, 1 for task + val meetsConstraints = matchesAttributeRequirements(slaveOfferConstraints, offerAttributes) + val meetsMemoryRequirements = mem >= calculateTotalMemory(sc) + val meetsCPURequirements = cpus >= (mesosExecutorCores + scheduler.CPUS_PER_TASK) + + val meetsRequirements = + (meetsConstraints && meetsMemoryRequirements && meetsCPURequirements) || + (slaveIdsWithExecutors.contains(slaveId) && cpus >= scheduler.CPUS_PER_TASK) + + // add some debug messaging + val debugstr = if (meetsRequirements) "Accepting" else "Declining" + val id = o.getId.getValue + logDebug(s"$debugstr offer: $id with attributes: $offerAttributes mem: $mem cpu: $cpus") + + meetsRequirements } + // Decline offers we ruled out immediately + unUsableOffers.foreach(o => d.declineOffer(o.getId)) + val workerOffers = usableOffers.map { o => val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt @@ -223,15 +245,15 @@ private[spark] class MesosSchedulerBackend( val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) acceptedOffers .foreach { offer => - offer.foreach { taskDesc => - val slaveId = taskDesc.executorId - slaveIdsWithExecutors += slaveId - slavesIdsOfAcceptedOffers += slaveId - taskIdToSlaveId(taskDesc.taskId) = slaveId - mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) - .add(createMesosTask(taskDesc, slaveId)) - } + offer.foreach { taskDesc => + val slaveId = taskDesc.executorId + slaveIdsWithExecutors += slaveId + slavesIdsOfAcceptedOffers += slaveId + taskIdToSlaveId(taskDesc.taskId) = slaveId + mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) + .add(createMesosTask(taskDesc, slaveId)) } + } // Reply to the offers val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? @@ -251,8 +273,6 @@ private[spark] class MesosSchedulerBackend( d.declineOffer(o.getId) } - // Decline offers we ruled out immediately - unUsableOffers.foreach(o => d.declineOffer(o.getId)) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index d11228f3d016a..d8a8c848bb4d1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -17,14 +17,17 @@ package org.apache.spark.scheduler.cluster.mesos -import java.util.List +import java.util.{List => JList} import java.util.concurrent.CountDownLatch import scala.collection.JavaConversions._ +import scala.util.control.NonFatal -import org.apache.mesos.Protos.{FrameworkInfo, Resource, Status} -import org.apache.mesos.{MesosSchedulerDriver, Scheduler} -import org.apache.spark.Logging +import com.google.common.base.Splitter +import org.apache.mesos.{MesosSchedulerDriver, Protos, Scheduler} +import org.apache.mesos.Protos._ +import org.apache.mesos.protobuf.GeneratedMessage +import org.apache.spark.{Logging, SparkContext} import org.apache.spark.util.Utils /** @@ -86,10 +89,150 @@ private[mesos] trait MesosSchedulerUtils extends Logging { /** * Get the amount of resources for the specified type from the resource list */ - protected def getResource(res: List[Resource], name: String): Double = { + protected def getResource(res: JList[Resource], name: String): Double = { for (r <- res if r.getName == name) { return r.getScalar.getValue } 0.0 } + + /** Helper method to get the key,value-set pair for a Mesos Attribute protobuf */ + protected def getAttribute(attr: Attribute): (String, Set[String]) = { + (attr.getName, attr.getText.getValue.split(',').toSet) + } + + + /** Build a Mesos resource protobuf object */ + protected def createResource(resourceName: String, quantity: Double): Protos.Resource = { + Resource.newBuilder() + .setName(resourceName) + .setType(Value.Type.SCALAR) + .setScalar(Value.Scalar.newBuilder().setValue(quantity).build()) + .build() + } + + /** + * Converts the attributes from the resource offer into a Map of name -> Attribute Value + * The attribute values are the mesos attribute types and they are + * @param offerAttributes + * @return + */ + protected def toAttributeMap(offerAttributes: JList[Attribute]): Map[String, GeneratedMessage] = { + offerAttributes.map(attr => { + val attrValue = attr.getType match { + case Value.Type.SCALAR => attr.getScalar + case Value.Type.RANGES => attr.getRanges + case Value.Type.SET => attr.getSet + case Value.Type.TEXT => attr.getText + } + (attr.getName, attrValue) + }).toMap + } + + + /** + * Match the requirements (if any) to the offer attributes. + * if attribute requirements are not specified - return true + * else if attribute is defined and no values are given, simple attribute presence is performed + * else if attribute name and value is specified, subset match is performed on slave attributes + */ + def matchesAttributeRequirements( + slaveOfferConstraints: Map[String, Set[String]], + offerAttributes: Map[String, GeneratedMessage]): Boolean = { + slaveOfferConstraints.forall { + // offer has the required attribute and subsumes the required values for that attribute + case (name, requiredValues) => + offerAttributes.get(name) match { + case None => false + case Some(_) if requiredValues.isEmpty => true // empty value matches presence + case Some(scalarValue: Value.Scalar) => + // check if provided values is less than equal to the offered values + requiredValues.map(_.toDouble).exists(_ <= scalarValue.getValue) + case Some(rangeValue: Value.Range) => + val offerRange = rangeValue.getBegin to rangeValue.getEnd + // Check if there is some required value that is between the ranges specified + // Note: We only support the ability to specify discrete values, in the future + // we may expand it to subsume ranges specified with a XX..YY value or something + // similar to that. + requiredValues.map(_.toLong).exists(offerRange.contains(_)) + case Some(offeredValue: Value.Set) => + // check if the specified required values is a subset of offered set + requiredValues.subsetOf(offeredValue.getItemList.toSet) + case Some(textValue: Value.Text) => + // check if the specified value is equal, if multiple values are specified + // we succeed if any of them match. + requiredValues.contains(textValue.getValue) + } + } + } + + /** + * Parses the attributes constraints provided to spark and build a matching data struct: + * Map[, Set[values-to-match]] + * The constraints are specified as ';' separated key-value pairs where keys and values + * are separated by ':'. The ':' implies equality (for singular values) and "is one of" for + * multiple values (comma separated). For example: + * {{{ + * parseConstraintString("tachyon:true;zone:us-east-1a,us-east-1b") + * // would result in + * + * Map( + * "tachyon" -> Set("true"), + * "zone": -> Set("us-east-1a", "us-east-1b") + * ) + * }}} + * + * Mesos documentation: http://mesos.apache.org/documentation/attributes-resources/ + * https://github.com/apache/mesos/blob/master/src/common/values.cpp + * https://github.com/apache/mesos/blob/master/src/common/attributes.cpp + * + * @param constraintsVal constaints string consisting of ';' separated key-value pairs (separated + * by ':') + * @return Map of constraints to match resources offers. + */ + def parseConstraintString(constraintsVal: String): Map[String, Set[String]] = { + /* + Based on mesos docs: + attributes : attribute ( ";" attribute )* + attribute : labelString ":" ( labelString | "," )+ + labelString : [a-zA-Z0-9_/.-] + */ + val splitter = Splitter.on(';').trimResults().withKeyValueSeparator(':') + // kv splitter + if (constraintsVal.isEmpty) { + Map() + } else { + try { + Map() ++ mapAsScalaMap(splitter.split(constraintsVal)).map { + case (k, v) => + if (v == null || v.isEmpty) { + (k, Set[String]()) + } else { + (k, v.split(',').toSet) + } + } + } catch { + case NonFatal(e) => + throw new IllegalArgumentException(s"Bad constraint string: $constraintsVal", e) + } + } + } + + // These defaults copied from YARN + private val MEMORY_OVERHEAD_FRACTION = 0.10 + private val MEMORY_OVERHEAD_MINIMUM = 384 + + /** + * Return the amount of memory to allocate to each executor, taking into account + * container overheads. + * @param sc SparkContext to use to get `spark.mesos.executor.memoryOverhead` value + * @return memory requirement as (0.1 * ) or MEMORY_OVERHEAD_MINIMUM + * (whichever is larger) + */ + def calculateTotalMemory(sc: SparkContext): Int = { + sc.conf.getInt("spark.mesos.executor.memoryOverhead", + math.max(MEMORY_OVERHEAD_FRACTION * sc.executorMemory, MEMORY_OVERHEAD_MINIMUM).toInt) + + sc.executorMemory + } + } diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtilsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtilsSuite.scala deleted file mode 100644 index e72285d03d3ee..0000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MemoryUtilsSuite.scala +++ /dev/null @@ -1,46 +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.scheduler.cluster.mesos - -import org.mockito.Mockito._ -import org.scalatest.mock.MockitoSugar - -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} - -class MemoryUtilsSuite extends SparkFunSuite with MockitoSugar { - test("MesosMemoryUtils should always override memoryOverhead when it's set") { - val sparkConf = new SparkConf - - val sc = mock[SparkContext] - when(sc.conf).thenReturn(sparkConf) - - // 384 > sc.executorMemory * 0.1 => 512 + 384 = 896 - when(sc.executorMemory).thenReturn(512) - assert(MemoryUtils.calculateTotalMemory(sc) === 896) - - // 384 < sc.executorMemory * 0.1 => 4096 + (4096 * 0.1) = 4505.6 - when(sc.executorMemory).thenReturn(4096) - assert(MemoryUtils.calculateTotalMemory(sc) === 4505) - - // set memoryOverhead - sparkConf.set("spark.mesos.executor.memoryOverhead", "100") - assert(MemoryUtils.calculateTotalMemory(sc) === 4196) - sparkConf.set("spark.mesos.executor.memoryOverhead", "400") - assert(MemoryUtils.calculateTotalMemory(sc) === 4496) - } -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index 68df46a41ddc8..d01837fe78957 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -149,7 +149,9 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi when(sc.conf).thenReturn(new SparkConf) when(sc.listenerBus).thenReturn(listenerBus) - val minMem = MemoryUtils.calculateTotalMemory(sc).toInt + val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") + + val minMem = backend.calculateTotalMemory(sc) val minCpu = 4 val mesosOffers = new java.util.ArrayList[Offer] @@ -157,8 +159,6 @@ class MesosSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext wi mesosOffers.add(createOffer(2, minMem - 1, minCpu)) mesosOffers.add(createOffer(3, minMem, minCpu)) - val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") - val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) expectedWorkerOffers.append(new WorkerOffer( mesosOffers.get(0).getSlaveId.getValue, diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala new file mode 100644 index 0000000000000..b354914b6ffd0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala @@ -0,0 +1,140 @@ +/* + * 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.scheduler.cluster.mesos + +import org.apache.mesos.Protos.Value +import org.mockito.Mockito._ +import org.scalatest._ +import org.scalatest.mock.MockitoSugar +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} + +class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoSugar { + + // scalastyle:off structural.type + // this is the documented way of generating fixtures in scalatest + def fixture: Object {val sc: SparkContext; val sparkConf: SparkConf} = new { + val sparkConf = new SparkConf + val sc = mock[SparkContext] + when(sc.conf).thenReturn(sparkConf) + } + val utils = new MesosSchedulerUtils { } + // scalastyle:on structural.type + + test("use at-least minimum overhead") { + val f = fixture + when(f.sc.executorMemory).thenReturn(512) + utils.calculateTotalMemory(f.sc) shouldBe 896 + } + + test("use overhead if it is greater than minimum value") { + val f = fixture + when(f.sc.executorMemory).thenReturn(4096) + utils.calculateTotalMemory(f.sc) shouldBe 4505 + } + + test("use spark.mesos.executor.memoryOverhead (if set)") { + val f = fixture + when(f.sc.executorMemory).thenReturn(1024) + f.sparkConf.set("spark.mesos.executor.memoryOverhead", "512") + utils.calculateTotalMemory(f.sc) shouldBe 1536 + } + + test("parse a non-empty constraint string correctly") { + val expectedMap = Map( + "tachyon" -> Set("true"), + "zone" -> Set("us-east-1a", "us-east-1b") + ) + utils.parseConstraintString("tachyon:true;zone:us-east-1a,us-east-1b") should be (expectedMap) + } + + test("parse an empty constraint string correctly") { + utils.parseConstraintString("") shouldBe Map() + } + + test("throw an exception when the input is malformed") { + an[IllegalArgumentException] should be thrownBy + utils.parseConstraintString("tachyon;zone:us-east") + } + + test("empty values for attributes' constraints matches all values") { + val constraintsStr = "tachyon:" + val parsedConstraints = utils.parseConstraintString(constraintsStr) + + parsedConstraints shouldBe Map("tachyon" -> Set()) + + val zoneSet = Value.Set.newBuilder().addItem("us-east-1a").addItem("us-east-1b").build() + val noTachyonOffer = Map("zone" -> zoneSet) + val tachyonTrueOffer = Map("tachyon" -> Value.Text.newBuilder().setValue("true").build()) + val tachyonFalseOffer = Map("tachyon" -> Value.Text.newBuilder().setValue("false").build()) + + utils.matchesAttributeRequirements(parsedConstraints, noTachyonOffer) shouldBe false + utils.matchesAttributeRequirements(parsedConstraints, tachyonTrueOffer) shouldBe true + utils.matchesAttributeRequirements(parsedConstraints, tachyonFalseOffer) shouldBe true + } + + test("subset match is performed for set attributes") { + val supersetConstraint = Map( + "tachyon" -> Value.Text.newBuilder().setValue("true").build(), + "zone" -> Value.Set.newBuilder() + .addItem("us-east-1a") + .addItem("us-east-1b") + .addItem("us-east-1c") + .build()) + + val zoneConstraintStr = "tachyon:;zone:us-east-1a,us-east-1c" + val parsedConstraints = utils.parseConstraintString(zoneConstraintStr) + + utils.matchesAttributeRequirements(parsedConstraints, supersetConstraint) shouldBe true + } + + test("less than equal match is performed on scalar attributes") { + val offerAttribs = Map("gpus" -> Value.Scalar.newBuilder().setValue(3).build()) + + val ltConstraint = utils.parseConstraintString("gpus:2") + val eqConstraint = utils.parseConstraintString("gpus:3") + val gtConstraint = utils.parseConstraintString("gpus:4") + + utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe true + utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true + utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false + } + + test("contains match is performed for range attributes") { + val offerAttribs = Map("ports" -> Value.Range.newBuilder().setBegin(7000).setEnd(8000).build()) + val ltConstraint = utils.parseConstraintString("ports:6000") + val eqConstraint = utils.parseConstraintString("ports:7500") + val gtConstraint = utils.parseConstraintString("ports:8002") + val multiConstraint = utils.parseConstraintString("ports:5000,7500,8300") + + utils.matchesAttributeRequirements(ltConstraint, offerAttribs) shouldBe false + utils.matchesAttributeRequirements(eqConstraint, offerAttribs) shouldBe true + utils.matchesAttributeRequirements(gtConstraint, offerAttribs) shouldBe false + utils.matchesAttributeRequirements(multiConstraint, offerAttribs) shouldBe true + } + + test("equality match is performed for text attributes") { + val offerAttribs = Map("tachyon" -> Value.Text.newBuilder().setValue("true").build()) + + val trueConstraint = utils.parseConstraintString("tachyon:true") + val falseConstraint = utils.parseConstraintString("tachyon:false") + + utils.matchesAttributeRequirements(trueConstraint, offerAttribs) shouldBe true + utils.matchesAttributeRequirements(falseConstraint, offerAttribs) shouldBe false + } + +} diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 5f1d6daeb27f0..1f915d8ea1d73 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -184,6 +184,14 @@ acquire. By default, it will acquire *all* cores in the cluster (that get offere only makes sense if you run just one application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example). +You may also make use of `spark.mesos.constraints` to set attribute based constraints on mesos resource offers. By default, all resource offers will be accepted. + +{% highlight scala %} +conf.set("spark.mesos.constraints", "tachyon=true;us-east-1=false") +{% endhighlight %} + +For example, Let's say `spark.mesos.constraints` is set to `tachyon=true;us-east-1=false`, then the resource offers will be checked to see if they meet both these constraints and only then will be accepted to start new executors. + # Mesos Docker Support Spark can make use of a Mesos Docker containerizer by setting the property `spark.mesos.executor.docker.image` @@ -298,6 +306,20 @@ See the [configuration page](configuration.html) for information on Spark config the final overhead will be this value. + + spark.mesos.constraints + Attribute based constraints to be matched against when accepting resource offers. + + Attribute based constraints on mesos resource offers. By default, all resource offers will be accepted. Refer to Mesos Attributes & Resources for more information on attributes. +
    +
  • Scalar constraints are matched with "less than equal" semantics i.e. value in the constraint must be less than or equal to the value in the resource offer.
  • +
  • Range constraints are matched with "contains" semantics i.e. value in the constraint must be within the resource offer's value.
  • +
  • Set constraints are matched with "subset of" semantics i.e. value in the constraint must be a subset of the resource offer's value.
  • +
  • Text constraints are metched with "equality" semantics i.e. value in the constraint must be exactly equal to the resource offer's value.
  • +
  • In case there is no value present as a part of the constraint any offer with the corresponding attribute will be accepted (without value check).
  • +
+ + # Troubleshooting and Debugging From 96c5eeec3970e8b1ebc6ddf5c97a7acc47f539dc Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 6 Jul 2015 16:11:22 -0700 Subject: [PATCH 19/73] Revert "[SPARK-7212] [MLLIB] Add sequence learning flag" This reverts commit 25f574eb9a3cb9b93b7d9194a8ec16e00ce2c036. After speaking to some users and developers, we realized that FP-growth doesn't meet the requirement for frequent sequence mining. PrefixSpan (SPARK-6487) would be the correct algorithm for it. feynmanliang Author: Xiangrui Meng Closes #7240 from mengxr/SPARK-7212.revert and squashes the following commits: 2b3d66b [Xiangrui Meng] Revert "[SPARK-7212] [MLLIB] Add sequence learning flag" --- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 38 +++----------- .../spark/mllib/fpm/FPGrowthSuite.scala | 52 +------------------ python/pyspark/mllib/fpm.py | 4 +- 3 files changed, 12 insertions(+), 82 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index abac08022ea47..efa8459d3cdba 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -36,7 +36,7 @@ import org.apache.spark.storage.StorageLevel * :: Experimental :: * * Model trained by [[FPGrowth]], which holds frequent itemsets. - * @param freqItemsets frequent itemsets, which is an RDD of [[FreqItemset]] + * @param freqItemsets frequent itemset, which is an RDD of [[FreqItemset]] * @tparam Item item type */ @Experimental @@ -62,14 +62,13 @@ class FPGrowthModel[Item: ClassTag](val freqItemsets: RDD[FreqItemset[Item]]) ex @Experimental class FPGrowth private ( private var minSupport: Double, - private var numPartitions: Int, - private var ordered: Boolean) extends Logging with Serializable { + private var numPartitions: Int) extends Logging with Serializable { /** * Constructs a default instance with default parameters {minSupport: `0.3`, numPartitions: same - * as the input data, ordered: `false`}. + * as the input data}. */ - def this() = this(0.3, -1, false) + def this() = this(0.3, -1) /** * Sets the minimal support level (default: `0.3`). @@ -87,15 +86,6 @@ class FPGrowth private ( this } - /** - * Indicates whether to mine itemsets (unordered) or sequences (ordered) (default: false, mine - * itemsets). - */ - def setOrdered(ordered: Boolean): this.type = { - this.ordered = ordered - this - } - /** * Computes an FP-Growth model that contains frequent itemsets. * @param data input data set, each element contains a transaction @@ -165,7 +155,7 @@ class FPGrowth private ( .flatMap { case (part, tree) => tree.extract(minCount, x => partitioner.getPartition(x) == part) }.map { case (ranks, count) => - new FreqItemset(ranks.map(i => freqItems(i)).reverse.toArray, count, ordered) + new FreqItemset(ranks.map(i => freqItems(i)).toArray, count) } } @@ -181,12 +171,9 @@ class FPGrowth private ( itemToRank: Map[Item, Int], partitioner: Partitioner): mutable.Map[Int, Array[Int]] = { val output = mutable.Map.empty[Int, Array[Int]] - // Filter the basket by frequent items pattern + // Filter the basket by frequent items pattern and sort their ranks. val filtered = transaction.flatMap(itemToRank.get) - if (!this.ordered) { - ju.Arrays.sort(filtered) - } - // Generate conditional transactions + ju.Arrays.sort(filtered) val n = filtered.length var i = n - 1 while (i >= 0) { @@ -211,18 +198,9 @@ object FPGrowth { * Frequent itemset. * @param items items in this itemset. Java users should call [[FreqItemset#javaItems]] instead. * @param freq frequency - * @param ordered indicates if items represents an itemset (false) or sequence (true) * @tparam Item item type */ - class FreqItemset[Item](val items: Array[Item], val freq: Long, val ordered: Boolean) - extends Serializable { - - /** - * Auxillary constructor, assumes unordered by default. - */ - def this(items: Array[Item], freq: Long) { - this(items, freq, false) - } + class FreqItemset[Item](val items: Array[Item], val freq: Long) extends Serializable { /** * Returns items in a Java List. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala index 1a8a1e79f2810..66ae3543ecc4e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { - test("FP-Growth frequent itemsets using String type") { + test("FP-Growth using String type") { val transactions = Seq( "r z h k p", "z y x w v u t s", @@ -38,14 +38,12 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { val model6 = fpg .setMinSupport(0.9) .setNumPartitions(1) - .setOrdered(false) .run(rdd) assert(model6.freqItemsets.count() === 0) val model3 = fpg .setMinSupport(0.5) .setNumPartitions(2) - .setOrdered(false) .run(rdd) val freqItemsets3 = model3.freqItemsets.collect().map { itemset => (itemset.items.toSet, itemset.freq) @@ -63,59 +61,17 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { val model2 = fpg .setMinSupport(0.3) .setNumPartitions(4) - .setOrdered(false) .run(rdd) assert(model2.freqItemsets.count() === 54) val model1 = fpg .setMinSupport(0.1) .setNumPartitions(8) - .setOrdered(false) .run(rdd) assert(model1.freqItemsets.count() === 625) } - test("FP-Growth frequent sequences using String type"){ - val transactions = Seq( - "r z h k p", - "z y x w v u t s", - "s x o n r", - "x z y m t s q e", - "z", - "x z y r q t p") - .map(_.split(" ")) - val rdd = sc.parallelize(transactions, 2).cache() - - val fpg = new FPGrowth() - - val model1 = fpg - .setMinSupport(0.5) - .setNumPartitions(2) - .setOrdered(true) - .run(rdd) - - /* - Use the following R code to verify association rules using arulesSequences package. - - data = read_baskets("path", info = c("sequenceID","eventID","SIZE")) - freqItemSeq = cspade(data, parameter = list(support = 0.5)) - resSeq = as(freqItemSeq, "data.frame") - resSeq$support = resSeq$support * length(transactions) - names(resSeq)[names(resSeq) == "support"] = "freq" - resSeq - */ - val expected = Set( - (Seq("r"), 3L), (Seq("s"), 3L), (Seq("t"), 3L), (Seq("x"), 4L), (Seq("y"), 3L), - (Seq("z"), 5L), (Seq("z", "y"), 3L), (Seq("x", "t"), 3L), (Seq("y", "t"), 3L), - (Seq("z", "t"), 3L), (Seq("z", "y", "t"), 3L) - ) - val freqItemseqs1 = model1.freqItemsets.collect().map { itemset => - (itemset.items.toSeq, itemset.freq) - }.toSet - assert(freqItemseqs1 == expected) - } - - test("FP-Growth frequent itemsets using Int type") { + test("FP-Growth using Int type") { val transactions = Seq( "1 2 3", "1 2 3 4", @@ -132,14 +88,12 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { val model6 = fpg .setMinSupport(0.9) .setNumPartitions(1) - .setOrdered(false) .run(rdd) assert(model6.freqItemsets.count() === 0) val model3 = fpg .setMinSupport(0.5) .setNumPartitions(2) - .setOrdered(false) .run(rdd) assert(model3.freqItemsets.first().items.getClass === Array(1).getClass, "frequent itemsets should use primitive arrays") @@ -155,14 +109,12 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { val model2 = fpg .setMinSupport(0.3) .setNumPartitions(4) - .setOrdered(false) .run(rdd) assert(model2.freqItemsets.count() === 15) val model1 = fpg .setMinSupport(0.1) .setNumPartitions(8) - .setOrdered(false) .run(rdd) assert(model1.freqItemsets.count() === 65) } diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py index b7f00d60069e6..bdc4a132b1b18 100644 --- a/python/pyspark/mllib/fpm.py +++ b/python/pyspark/mllib/fpm.py @@ -39,8 +39,8 @@ class FPGrowthModel(JavaModelWrapper): >>> data = [["a", "b", "c"], ["a", "b", "d", "e"], ["a", "c", "e"], ["a", "c", "f"]] >>> rdd = sc.parallelize(data, 2) >>> model = FPGrowth.train(rdd, 0.6, 2) - >>> sorted(model.freqItemsets().collect(), key=lambda x: x.items) - [FreqItemset(items=[u'a'], freq=4), FreqItemset(items=[u'a', u'c'], freq=3), ... + >>> sorted(model.freqItemsets().collect()) + [FreqItemset(items=[u'a'], freq=4), FreqItemset(items=[u'c'], freq=3), ... """ def freqItemsets(self): From 0effe180f4c2cf37af1012b33b43912bdecaf756 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Mon, 6 Jul 2015 16:15:12 -0700 Subject: [PATCH 20/73] [SPARK-8765] [MLLIB] Fix PySpark PowerIterationClustering test issue PySpark PowerIterationClustering test failure due to bad demo data. If the data is small, PowerIterationClustering will behavior indeterministic. Author: Yanbo Liang Closes #7177 from yanboliang/spark-8765 and squashes the following commits: 392ae54 [Yanbo Liang] fix model.assignments output 5ec3f1e [Yanbo Liang] fix PySpark PowerIterationClustering test issue --- python/pyspark/mllib/clustering.py | 16 ++++++++++++++-- 1 file changed, 14 insertions(+), 2 deletions(-) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index a3eab635282f6..ed4d78a2c6788 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -282,18 +282,30 @@ class PowerIterationClusteringModel(JavaModelWrapper, JavaSaveable, JavaLoader): Model produced by [[PowerIterationClustering]]. - >>> data = [(0, 1, 1.0), (0, 2, 1.0), (1, 3, 1.0), (2, 3, 1.0), - ... (0, 3, 1.0), (1, 2, 1.0), (0, 4, 0.1)] + >>> data = [(0, 1, 1.0), (0, 2, 1.0), (0, 3, 1.0), (1, 2, 1.0), (1, 3, 1.0), + ... (2, 3, 1.0), (3, 4, 0.1), (4, 5, 1.0), (4, 15, 1.0), (5, 6, 1.0), + ... (6, 7, 1.0), (7, 8, 1.0), (8, 9, 1.0), (9, 10, 1.0), (10, 11, 1.0), + ... (11, 12, 1.0), (12, 13, 1.0), (13, 14, 1.0), (14, 15, 1.0)] >>> rdd = sc.parallelize(data, 2) >>> model = PowerIterationClustering.train(rdd, 2, 100) >>> model.k 2 + >>> result = sorted(model.assignments().collect(), key=lambda x: x.id) + >>> result[0].cluster == result[1].cluster == result[2].cluster == result[3].cluster + True + >>> result[4].cluster == result[5].cluster == result[6].cluster == result[7].cluster + True >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> model.save(sc, path) >>> sameModel = PowerIterationClusteringModel.load(sc, path) >>> sameModel.k 2 + >>> result = sorted(model.assignments().collect(), key=lambda x: x.id) + >>> result[0].cluster == result[1].cluster == result[2].cluster == result[3].cluster + True + >>> result[4].cluster == result[5].cluster == result[6].cluster == result[7].cluster + True >>> from shutil import rmtree >>> try: ... rmtree(path) From 7b467cc9348fa910e445ad08914a72f8ed4fc249 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Mon, 6 Jul 2015 16:26:31 -0700 Subject: [PATCH 21/73] [SPARK-8588] [SQL] Regression test This PR adds regression test for https://issues.apache.org/jira/browse/SPARK-8588 (fixed by https://github.com/apache/spark/commit/457d07eaa023b44b75344110508f629925eb6247). Author: Yin Huai This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #7103 from yhuai/SPARK-8588-test and squashes the following commits: eb5f418 [Yin Huai] Add a query test. c61a173 [Yin Huai] Regression test for SPARK-8588. --- .../analysis/HiveTypeCoercionSuite.scala | 21 +++++++++++++++++++ .../sql/hive/execution/SQLQuerySuite.scala | 16 ++++++++++++++ 2 files changed, 37 insertions(+) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index b56426617789e..93db33d44eb25 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -271,4 +271,25 @@ class HiveTypeCoercionSuite extends PlanTest { Literal(true) ) } + + /** + * There are rules that need to not fire before child expressions get resolved. + * We use this test to make sure those rules do not fire early. + */ + test("make sure rules do not fire early") { + // InConversion + val inConversion = HiveTypeCoercion.InConversion + ruleTest(inConversion, + In(UnresolvedAttribute("a"), Seq(Literal(1))), + In(UnresolvedAttribute("a"), Seq(Literal(1))) + ) + ruleTest(inConversion, + In(Literal("test"), Seq(UnresolvedAttribute("a"), Literal(1))), + In(Literal("test"), Seq(UnresolvedAttribute("a"), Literal(1))) + ) + ruleTest(inConversion, + In(Literal("a"), Seq(Literal(1), Literal("b"))), + In(Literal("a"), Seq(Cast(Literal(1), StringType), Cast(Literal("b"), StringType))) + ) + } } 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 6d645393a6da1..bf9f2ecd51793 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 @@ -990,5 +990,21 @@ class SQLQuerySuite extends QueryTest { Timestamp.valueOf("1969-12-31 16:00:00"), String.valueOf("1969-12-31 16:00:00"), Timestamp.valueOf("1970-01-01 00:00:00"))) + + } + + test("SPARK-8588 HiveTypeCoercion.inConversion fires too early") { + val df = + TestHive.createDataFrame(Seq((1, "2014-01-01"), (2, "2015-01-01"), (3, "2016-01-01"))) + df.toDF("id", "date").registerTempTable("test_SPARK8588") + checkAnswer( + TestHive.sql( + """ + |select id, concat(year(date)) + |from test_SPARK8588 where concat(year(date), ' year') in ('2015 year', '2014 year') + """.stripMargin), + Row(1, "2014") :: Row(2, "2015") :: Nil + ) + TestHive.dropTempTable("test_SPARK8588") } } From 09a06418debc25da0191d98798f7c5016d39be91 Mon Sep 17 00:00:00 2001 From: animesh Date: Mon, 6 Jul 2015 16:39:49 -0700 Subject: [PATCH 22/73] [SPARK-8072] [SQL] Better AnalysisException for writing DataFrame with identically named columns Adding a function checkConstraints which will check for the constraints to be applied on the dataframe / dataframe schema. Function called before storing the dataframe to an external storage. Function added in the corresponding datasource API. cc rxin marmbrus Author: animesh This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #7013 from animeshbaranawal/8072 and squashes the following commits: f70dd0e [animesh] Change IO exception to Analysis Exception fd45e1b [animesh] 8072: Fix Style Issues a8a964f [animesh] 8072: Improving on previous commits 3cc4d2c [animesh] Fix Style Issues 1a89115 [animesh] Fix Style Issues 98b4399 [animesh] 8072 : Moved the exception handling to ResolvedDataSource specific to parquet format 7c3d928 [animesh] 8072: Adding check to DataFrameWriter.scala --- .../apache/spark/sql/json/JSONRelation.scala | 31 +++++++++++++++++++ .../apache/spark/sql/parquet/newParquet.scala | 19 +++++++++++- .../org/apache/spark/sql/DataFrameSuite.scala | 24 ++++++++++++++ 3 files changed, 73 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index 69bf13e1e5a6a..2361d3bf52d2b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -22,6 +22,7 @@ import java.io.IOException import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType @@ -37,6 +38,17 @@ private[sql] class DefaultSource parameters.getOrElse("path", sys.error("'path' must be specified for json data.")) } + /** Constraints to be imposed on dataframe to be stored. */ + private def checkConstraints(data: DataFrame): Unit = { + if (data.schema.fieldNames.length != data.schema.fieldNames.distinct.length) { + val duplicateColumns = data.schema.fieldNames.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => "\"" + x + "\"" + }.mkString(", ") + throw new AnalysisException(s"Duplicate column(s) : $duplicateColumns found, " + + s"cannot save to JSON format") + } + } + /** Returns a new base relation with the parameters. */ override def createRelation( sqlContext: SQLContext, @@ -63,6 +75,10 @@ private[sql] class DefaultSource mode: SaveMode, parameters: Map[String, String], data: DataFrame): BaseRelation = { + // check if dataframe satisfies the constraints + // before moving forward + checkConstraints(data) + val path = checkPath(parameters) val filesystemPath = new Path(path) val fs = filesystemPath.getFileSystem(sqlContext.sparkContext.hadoopConfiguration) @@ -130,6 +146,17 @@ private[sql] class JSONRelation( samplingRatio, userSpecifiedSchema)(sqlContext) + /** Constraints to be imposed on dataframe to be stored. */ + private def checkConstraints(data: DataFrame): Unit = { + if (data.schema.fieldNames.length != data.schema.fieldNames.distinct.length) { + val duplicateColumns = data.schema.fieldNames.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => "\"" + x + "\"" + }.mkString(", ") + throw new AnalysisException(s"Duplicate column(s) : $duplicateColumns found, " + + s"cannot save to JSON format") + } + } + private val useJacksonStreamingAPI: Boolean = sqlContext.conf.useJacksonStreamingAPI override val needConversion: Boolean = false @@ -178,6 +205,10 @@ private[sql] class JSONRelation( } override def insert(data: DataFrame, overwrite: Boolean): Unit = { + // check if dataframe satisfies constraints + // before moving forward + checkConstraints(data) + val filesystemPath = path match { case Some(p) => new Path(p) case None => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 5ac3e9a44e6fe..6bc69c6ad0847 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -164,7 +164,24 @@ private[sql] class ParquetRelation2( } } - override def dataSchema: StructType = maybeDataSchema.getOrElse(metadataCache.dataSchema) + /** Constraints on schema of dataframe to be stored. */ + private def checkConstraints(schema: StructType): Unit = { + if (schema.fieldNames.length != schema.fieldNames.distinct.length) { + val duplicateColumns = schema.fieldNames.groupBy(identity).collect { + case (x, ys) if ys.length > 1 => "\"" + x + "\"" + }.mkString(", ") + throw new AnalysisException(s"Duplicate column(s) : $duplicateColumns found, " + + s"cannot save to parquet format") + } + } + + override def dataSchema: StructType = { + val schema = maybeDataSchema.getOrElse(metadataCache.dataSchema) + // check if schema satisfies the constraints + // before moving forward + checkConstraints(schema) + schema + } override private[sql] def refresh(): Unit = { super.refresh() 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 afb1cf5f8d1cb..f592a9934d0e6 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 @@ -737,4 +737,28 @@ class DataFrameSuite extends QueryTest { df.col("") df.col("t.``") } + + test("SPARK-8072: Better Exception for Duplicate Columns") { + // only one duplicate column present + val e = intercept[org.apache.spark.sql.AnalysisException] { + val df1 = Seq((1, 2, 3), (2, 3, 4), (3, 4, 5)).toDF("column1", "column2", "column1") + .write.format("parquet").save("temp") + } + assert(e.getMessage.contains("Duplicate column(s)")) + assert(e.getMessage.contains("parquet")) + assert(e.getMessage.contains("column1")) + assert(!e.getMessage.contains("column2")) + + // multiple duplicate columns present + val f = intercept[org.apache.spark.sql.AnalysisException] { + val df2 = Seq((1, 2, 3, 4, 5), (2, 3, 4, 5, 6), (3, 4, 5, 6, 7)) + .toDF("column1", "column2", "column3", "column1", "column3") + .write.format("json").save("temp") + } + assert(f.getMessage.contains("Duplicate column(s)")) + assert(f.getMessage.contains("JSON")) + assert(f.getMessage.contains("column1")) + assert(f.getMessage.contains("column3")) + assert(!f.getMessage.contains("column2")) + } } From d4d6d31db5cc5c69ac369f754b7489f444c9ba2f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 6 Jul 2015 17:16:44 -0700 Subject: [PATCH 23/73] [SPARK-8463][SQL] Use DriverRegistry to load jdbc driver at writing path JIRA: https://issues.apache.org/jira/browse/SPARK-8463 Currently, at the reading path, `DriverRegistry` is used to load needed jdbc driver at executors. However, at the writing path, we also need `DriverRegistry` to load jdbc driver. Author: Liang-Chi Hsieh Closes #6900 from viirya/jdbc_write_driver and squashes the following commits: 16cd04b [Liang-Chi Hsieh] Use DriverRegistry to load jdbc driver at writing path. --- .../main/scala/org/apache/spark/sql/jdbc/jdbc.scala | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala index dd8aaf6474895..f7ea852fe7f58 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala @@ -58,13 +58,12 @@ package object jdbc { * are used. */ def savePartition( - url: String, + getConnection: () => Connection, table: String, iterator: Iterator[Row], rddSchema: StructType, - nullTypes: Array[Int], - properties: Properties): Iterator[Byte] = { - val conn = DriverManager.getConnection(url, properties) + nullTypes: Array[Int]): Iterator[Byte] = { + val conn = getConnection() var committed = false try { conn.setAutoCommit(false) // Everything in the same db transaction. @@ -185,8 +184,10 @@ package object jdbc { } val rddSchema = df.schema + val driver: String = DriverRegistry.getDriverClassName(url) + val getConnection: () => Connection = JDBCRDD.getConnector(driver, url, properties) df.foreachPartition { iterator => - JDBCWriteDetails.savePartition(url, table, iterator, rddSchema, nullTypes, properties) + JDBCWriteDetails.savePartition(getConnection, table, iterator, rddSchema, nullTypes) } } From 9eae5fa642317dd11fc783d832d4cbb7e62db471 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 6 Jul 2015 19:22:30 -0700 Subject: [PATCH 24/73] [SPARK-8819] Fix build for maven 3.3.x This is a workaround for MSHADE-148, which leads to an infinite loop when building Spark with maven 3.3.x. This was originally caused by #6441, which added a bunch of test dependencies on the spark-core test module. Recently, it was revealed by #7193. This patch adds a `-Prelease` profile. If present, it will set `createDependencyReducedPom` to true. The consequences are: - If you are releasing Spark with this profile, you are fine as long as you use maven 3.2.x or before. - If you are releasing Spark without this profile, you will run into SPARK-8781. - If you are not releasing Spark but you are using this profile, you may run into SPARK-8819. - If you are not releasing Spark and you did not include this profile, you are fine. This is all documented in `pom.xml` and tested locally with both versions of maven. Author: Andrew Or Closes #7219 from andrewor14/fix-maven-build and squashes the following commits: 1d37e87 [Andrew Or] Merge branch 'master' of github.com:apache/spark into fix-maven-build 3574ae4 [Andrew Or] Review comments f39199c [Andrew Or] Create a -Prelease profile that flags `createDependencyReducedPom` --- dev/create-release/create-release.sh | 4 ++-- pom.xml | 24 ++++++++++++++++++++++++ 2 files changed, 26 insertions(+), 2 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 54274a83f6d66..cfe2cd4752b3f 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -118,13 +118,13 @@ if [[ ! "$@" =~ --skip-publish ]]; then rm -rf $SPARK_REPO - build/mvn -DskipTests -Pyarn -Phive \ + build/mvn -DskipTests -Pyarn -Phive -Prelease-profile\ -Phive-thriftserver -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install ./dev/change-version-to-2.11.sh - build/mvn -DskipTests -Pyarn -Phive \ + build/mvn -DskipTests -Pyarn -Phive -Prelease-profile\ -Dscala-2.11 -Phadoop-2.2 -Pspark-ganglia-lgpl -Pkinesis-asl \ clean install diff --git a/pom.xml b/pom.xml index ffa96128a3d61..fbcc9152765cf 100644 --- a/pom.xml +++ b/pom.xml @@ -161,6 +161,8 @@ 2.4.4 1.1.1.7 1.1.2 + + false ${java.home} @@ -1440,6 +1442,8 @@ 2.3 false + + ${create.dependency.reduced.pom} @@ -1826,6 +1830,26 @@ + + + release-profile + + + true + + + - release-profile + release false @@ -179,6 +180,8 @@ compile compile compile + test + test + + twttr-repo + Twttr Repository + http://maven.twttr.com + + true + + + false + + spark-1.4-staging @@ -1101,6 +1116,24 @@ ${parquet.version} ${parquet.deps.scope} + + org.apache.parquet + parquet-avro + ${parquet.version} + ${parquet.test.deps.scope} + + + org.apache.parquet + parquet-thrift + ${parquet.version} + ${parquet.test.deps.scope} + + + org.apache.thrift + libthrift + ${thrift.version} + ${thrift.test.deps.scope} + org.apache.flume flume-ng-core diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 41e19fd9cc11e..7346d804632bc 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -62,21 +62,8 @@ object MimaExcludes { "org.apache.spark.ml.classification.LogisticCostFun.this"), // SQL execution is considered private. excludePackage("org.apache.spark.sql.execution"), - // NanoTime and CatalystTimestampConverter is only used inside catalyst, - // not needed anymore - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.timestamp.NanoTime"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.timestamp.NanoTime$"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystTimestampConverter"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.CatalystTimestampConverter$"), - // SPARK-6777 Implements backwards compatibility rules in CatalystSchemaConverter - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetTypeInfo"), - ProblemFilters.exclude[MissingClassProblem]( - "org.apache.spark.sql.parquet.ParquetTypeInfo$") + // Parquet support is considered private. + excludePackage("org.apache.spark.sql.parquet") ) ++ Seq( // SPARK-8479 Add numNonzeros and numActives to Matrix. ProblemFilters.exclude[MissingMethodProblem]( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index 7d00047d08d74..a4c2da8e05f5d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -17,11 +17,12 @@ package org.apache.spark.sql.types +import scala.util.Try import scala.util.parsing.combinator.RegexParsers -import org.json4s._ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ +import org.json4s._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.DeveloperApi @@ -82,6 +83,9 @@ abstract class DataType extends AbstractDataType { object DataType { + private[sql] def fromString(raw: String): DataType = { + Try(DataType.fromJson(raw)).getOrElse(DataType.fromCaseClassString(raw)) + } def fromJson(json: String): DataType = parseDataType(parse(json)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 3b17566d54d9b..e2d3f53f7d978 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -311,6 +311,11 @@ object StructType extends AbstractDataType { private[sql] override def simpleString: String = "struct" + private[sql] def fromString(raw: String): StructType = DataType.fromString(raw) match { + case t: StructType => t + case _ => throw new RuntimeException(s"Failed parsing StructType: $raw") + } + def apply(fields: Seq[StructField]): StructType = StructType(fields.toArray) def apply(fields: java.util.List[StructField]): StructType = { diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 8fc16928adbd9..f90099f22d4bd 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -101,9 +101,45 @@ 9.3-1102-jdbc41 test + + org.apache.parquet + parquet-avro + test + + + org.apache.parquet + parquet-thrift + test + + + org.apache.thrift + libthrift + test + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + org.codehaus.mojo + build-helper-maven-plugin + + + add-scala-test-sources + generate-test-sources + + add-test-source + + + + src/test/scala + src/test/gen-java + + + + + + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala new file mode 100644 index 0000000000000..0c3d8fdab6bd2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystRowConverter.scala @@ -0,0 +1,434 @@ +/* + * 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.parquet + +import java.nio.ByteOrder + +import scala.collection.JavaConversions._ +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.parquet.column.Dictionary +import org.apache.parquet.io.api.{Binary, Converter, GroupConverter, PrimitiveConverter} +import org.apache.parquet.schema.Type.Repetition +import org.apache.parquet.schema.{GroupType, PrimitiveType, Type} + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +/** + * A [[ParentContainerUpdater]] is used by a Parquet converter to set converted values to some + * corresponding parent container. For example, a converter for a `StructType` field may set + * converted values to a [[MutableRow]]; or a converter for array elements may append converted + * values to an [[ArrayBuffer]]. + */ +private[parquet] trait ParentContainerUpdater { + def set(value: Any): Unit = () + def setBoolean(value: Boolean): Unit = set(value) + def setByte(value: Byte): Unit = set(value) + def setShort(value: Short): Unit = set(value) + def setInt(value: Int): Unit = set(value) + def setLong(value: Long): Unit = set(value) + def setFloat(value: Float): Unit = set(value) + def setDouble(value: Double): Unit = set(value) +} + +/** A no-op updater used for root converter (who doesn't have a parent). */ +private[parquet] object NoopUpdater extends ParentContainerUpdater + +/** + * A [[CatalystRowConverter]] is used to convert Parquet "structs" into Spark SQL [[Row]]s. Since + * any Parquet record is also a struct, this converter can also be used as root converter. + * + * When used as a root converter, [[NoopUpdater]] should be used since root converters don't have + * any "parent" container. + * + * @param parquetType Parquet schema of Parquet records + * @param catalystType Spark SQL schema that corresponds to the Parquet record type + * @param updater An updater which propagates converted field values to the parent container + */ +private[parquet] class CatalystRowConverter( + parquetType: GroupType, + catalystType: StructType, + updater: ParentContainerUpdater) + extends GroupConverter { + + /** + * Updater used together with field converters within a [[CatalystRowConverter]]. It propagates + * converted filed values to the `ordinal`-th cell in `currentRow`. + */ + private final class RowUpdater(row: MutableRow, ordinal: Int) extends ParentContainerUpdater { + override def set(value: Any): Unit = row(ordinal) = value + override def setBoolean(value: Boolean): Unit = row.setBoolean(ordinal, value) + override def setByte(value: Byte): Unit = row.setByte(ordinal, value) + override def setShort(value: Short): Unit = row.setShort(ordinal, value) + override def setInt(value: Int): Unit = row.setInt(ordinal, value) + override def setLong(value: Long): Unit = row.setLong(ordinal, value) + override def setDouble(value: Double): Unit = row.setDouble(ordinal, value) + override def setFloat(value: Float): Unit = row.setFloat(ordinal, value) + } + + /** + * Represents the converted row object once an entire Parquet record is converted. + * + * @todo Uses [[UnsafeRow]] for better performance. + */ + val currentRow = new SpecificMutableRow(catalystType.map(_.dataType)) + + // Converters for each field. + private val fieldConverters: Array[Converter] = { + parquetType.getFields.zip(catalystType).zipWithIndex.map { + case ((parquetFieldType, catalystField), ordinal) => + // Converted field value should be set to the `ordinal`-th cell of `currentRow` + newConverter(parquetFieldType, catalystField.dataType, new RowUpdater(currentRow, ordinal)) + }.toArray + } + + override def getConverter(fieldIndex: Int): Converter = fieldConverters(fieldIndex) + + override def end(): Unit = updater.set(currentRow) + + override def start(): Unit = { + var i = 0 + while (i < currentRow.length) { + currentRow.setNullAt(i) + i += 1 + } + } + + /** + * Creates a converter for the given Parquet type `parquetType` and Spark SQL data type + * `catalystType`. Converted values are handled by `updater`. + */ + private def newConverter( + parquetType: Type, + catalystType: DataType, + updater: ParentContainerUpdater): Converter = { + + catalystType match { + case BooleanType | IntegerType | LongType | FloatType | DoubleType | BinaryType => + new CatalystPrimitiveConverter(updater) + + case ByteType => + new PrimitiveConverter { + override def addInt(value: Int): Unit = + updater.setByte(value.asInstanceOf[ByteType#InternalType]) + } + + case ShortType => + new PrimitiveConverter { + override def addInt(value: Int): Unit = + updater.setShort(value.asInstanceOf[ShortType#InternalType]) + } + + case t: DecimalType => + new CatalystDecimalConverter(t, updater) + + case StringType => + new CatalystStringConverter(updater) + + case TimestampType => + // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. + new PrimitiveConverter { + // Converts nanosecond timestamps stored as INT96 + override def addBinary(value: Binary): Unit = { + assert( + value.length() == 12, + "Timestamps (with nanoseconds) are expected to be stored in 12-byte long binaries, " + + s"but got a ${value.length()}-byte binary.") + + val buf = value.toByteBuffer.order(ByteOrder.LITTLE_ENDIAN) + val timeOfDayNanos = buf.getLong + val julianDay = buf.getInt + updater.setLong(DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos)) + } + } + + case DateType => + new PrimitiveConverter { + override def addInt(value: Int): Unit = { + // DateType is not specialized in `SpecificMutableRow`, have to box it here. + updater.set(value.asInstanceOf[DateType#InternalType]) + } + } + + case t: ArrayType => + new CatalystArrayConverter(parquetType.asGroupType(), t, updater) + + case t: MapType => + new CatalystMapConverter(parquetType.asGroupType(), t, updater) + + case t: StructType => + new CatalystRowConverter(parquetType.asGroupType(), t, new ParentContainerUpdater { + override def set(value: Any): Unit = updater.set(value.asInstanceOf[Row].copy()) + }) + + case t: UserDefinedType[_] => + val catalystTypeForUDT = t.sqlType + val nullable = parquetType.isRepetition(Repetition.OPTIONAL) + val field = StructField("udt", catalystTypeForUDT, nullable) + val parquetTypeForUDT = new CatalystSchemaConverter().convertField(field) + newConverter(parquetTypeForUDT, catalystTypeForUDT, updater) + + case _ => + throw new RuntimeException( + s"Unable to create Parquet converter for data type ${catalystType.json}") + } + } + + /** + * Parquet converter for Parquet primitive types. Note that not all Spark SQL atomic types + * are handled by this converter. Parquet primitive types are only a subset of those of Spark + * SQL. For example, BYTE, SHORT, and INT in Spark SQL are all covered by INT32 in Parquet. + */ + private final class CatalystPrimitiveConverter(updater: ParentContainerUpdater) + extends PrimitiveConverter { + + override def addBoolean(value: Boolean): Unit = updater.setBoolean(value) + override def addInt(value: Int): Unit = updater.setInt(value) + override def addLong(value: Long): Unit = updater.setLong(value) + override def addFloat(value: Float): Unit = updater.setFloat(value) + override def addDouble(value: Double): Unit = updater.setDouble(value) + override def addBinary(value: Binary): Unit = updater.set(value.getBytes) + } + + /** + * Parquet converter for strings. A dictionary is used to minimize string decoding cost. + */ + private final class CatalystStringConverter(updater: ParentContainerUpdater) + extends PrimitiveConverter { + + private var expandedDictionary: Array[UTF8String] = null + + override def hasDictionarySupport: Boolean = true + + override def setDictionary(dictionary: Dictionary): Unit = { + this.expandedDictionary = Array.tabulate(dictionary.getMaxId + 1) { i => + UTF8String.fromBytes(dictionary.decodeToBinary(i).getBytes) + } + } + + override def addValueFromDictionary(dictionaryId: Int): Unit = { + updater.set(expandedDictionary(dictionaryId)) + } + + override def addBinary(value: Binary): Unit = { + updater.set(UTF8String.fromBytes(value.getBytes)) + } + } + + /** + * Parquet converter for fixed-precision decimals. + */ + private final class CatalystDecimalConverter( + decimalType: DecimalType, + updater: ParentContainerUpdater) + extends PrimitiveConverter { + + // Converts decimals stored as INT32 + override def addInt(value: Int): Unit = { + addLong(value: Long) + } + + // Converts decimals stored as INT64 + override def addLong(value: Long): Unit = { + updater.set(Decimal(value, decimalType.precision, decimalType.scale)) + } + + // Converts decimals stored as either FIXED_LENGTH_BYTE_ARRAY or BINARY + override def addBinary(value: Binary): Unit = { + updater.set(toDecimal(value)) + } + + private def toDecimal(value: Binary): Decimal = { + val precision = decimalType.precision + val scale = decimalType.scale + val bytes = value.getBytes + + var unscaled = 0L + var i = 0 + + while (i < bytes.length) { + unscaled = (unscaled << 8) | (bytes(i) & 0xff) + i += 1 + } + + val bits = 8 * bytes.length + unscaled = (unscaled << (64 - bits)) >> (64 - bits) + Decimal(unscaled, precision, scale) + } + } + + /** + * Parquet converter for arrays. Spark SQL arrays are represented as Parquet lists. Standard + * Parquet lists are represented as a 3-level group annotated by `LIST`: + * {{{ + * group (LIST) { <-- parquetSchema points here + * repeated group list { + * element; + * } + * } + * }}} + * The `parquetSchema` constructor argument points to the outermost group. + * + * However, before this representation is standardized, some Parquet libraries/tools also use some + * non-standard formats to represent list-like structures. Backwards-compatibility rules for + * handling these cases are described in Parquet format spec. + * + * @see https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#lists + */ + private final class CatalystArrayConverter( + parquetSchema: GroupType, + catalystSchema: ArrayType, + updater: ParentContainerUpdater) + extends GroupConverter { + + private var currentArray: ArrayBuffer[Any] = _ + + private val elementConverter: Converter = { + val repeatedType = parquetSchema.getType(0) + val elementType = catalystSchema.elementType + + if (isElementType(repeatedType, elementType)) { + newConverter(repeatedType, elementType, new ParentContainerUpdater { + override def set(value: Any): Unit = currentArray += value + }) + } else { + new ElementConverter(repeatedType.asGroupType().getType(0), elementType) + } + } + + override def getConverter(fieldIndex: Int): Converter = elementConverter + + override def end(): Unit = updater.set(currentArray) + + // NOTE: We can't reuse the mutable `ArrayBuffer` here and must instantiate a new buffer for the + // next value. `Row.copy()` only copies row cells, it doesn't do deep copy to objects stored + // in row cells. + override def start(): Unit = currentArray = ArrayBuffer.empty[Any] + + // scalastyle:off + /** + * Returns whether the given type is the element type of a list or is a syntactic group with + * one field that is the element type. This is determined by checking whether the type can be + * a syntactic group and by checking whether a potential syntactic group matches the expected + * schema. + * {{{ + * group (LIST) { + * repeated group list { <-- repeatedType points here + * element; + * } + * } + * }}} + * In short, here we handle Parquet list backwards-compatibility rules on the read path. This + * method is based on `AvroIndexedRecordConverter.isElementType`. + * + * @see https://github.com/apache/parquet-format/blob/master/LogicalTypes.md#backward-compatibility-rules + */ + // scalastyle:on + private def isElementType(parquetRepeatedType: Type, catalystElementType: DataType): Boolean = { + (parquetRepeatedType, catalystElementType) match { + case (t: PrimitiveType, _) => true + case (t: GroupType, _) if t.getFieldCount > 1 => true + case (t: GroupType, StructType(Array(f))) if f.name == t.getFieldName(0) => true + case _ => false + } + } + + /** Array element converter */ + private final class ElementConverter(parquetType: Type, catalystType: DataType) + extends GroupConverter { + + private var currentElement: Any = _ + + private val converter = newConverter(parquetType, catalystType, new ParentContainerUpdater { + override def set(value: Any): Unit = currentElement = value + }) + + override def getConverter(fieldIndex: Int): Converter = converter + + override def end(): Unit = currentArray += currentElement + + override def start(): Unit = currentElement = null + } + } + + /** Parquet converter for maps */ + private final class CatalystMapConverter( + parquetType: GroupType, + catalystType: MapType, + updater: ParentContainerUpdater) + extends GroupConverter { + + private var currentMap: mutable.Map[Any, Any] = _ + + private val keyValueConverter = { + val repeatedType = parquetType.getType(0).asGroupType() + new KeyValueConverter( + repeatedType.getType(0), + repeatedType.getType(1), + catalystType.keyType, + catalystType.valueType) + } + + override def getConverter(fieldIndex: Int): Converter = keyValueConverter + + override def end(): Unit = updater.set(currentMap) + + // NOTE: We can't reuse the mutable Map here and must instantiate a new `Map` for the next + // value. `Row.copy()` only copies row cells, it doesn't do deep copy to objects stored in row + // cells. + override def start(): Unit = currentMap = mutable.Map.empty[Any, Any] + + /** Parquet converter for key-value pairs within the map. */ + private final class KeyValueConverter( + parquetKeyType: Type, + parquetValueType: Type, + catalystKeyType: DataType, + catalystValueType: DataType) + extends GroupConverter { + + private var currentKey: Any = _ + + private var currentValue: Any = _ + + private val converters = Array( + // Converter for keys + newConverter(parquetKeyType, catalystKeyType, new ParentContainerUpdater { + override def set(value: Any): Unit = currentKey = value + }), + + // Converter for values + newConverter(parquetValueType, catalystValueType, new ParentContainerUpdater { + override def set(value: Any): Unit = currentValue = value + })) + + override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) + + override def end(): Unit = currentMap(currentKey) = currentValue + + override def start(): Unit = { + currentKey = null + currentValue = null + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala index 4ab274ec17a02..de3a72d8146c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/CatalystSchemaConverter.scala @@ -358,9 +358,24 @@ private[parquet] class CatalystSchemaConverter( case DateType => Types.primitive(INT32, repetition).as(DATE).named(field.name) - // NOTE: !! This timestamp type is not specified in Parquet format spec !! - // However, Impala and older versions of Spark SQL use INT96 to store timestamps with - // nanosecond precision (not TIME_MILLIS or TIMESTAMP_MILLIS described in the spec). + // NOTE: Spark SQL TimestampType is NOT a well defined type in Parquet format spec. + // + // As stated in PARQUET-323, Parquet `INT96` was originally introduced to represent nanosecond + // timestamp in Impala for some historical reasons, it's not recommended to be used for any + // other types and will probably be deprecated in future Parquet format spec. That's the + // reason why Parquet format spec only defines `TIMESTAMP_MILLIS` and `TIMESTAMP_MICROS` which + // are both logical types annotating `INT64`. + // + // Originally, Spark SQL uses the same nanosecond timestamp type as Impala and Hive. Starting + // from Spark 1.5.0, we resort to a timestamp type with 100 ns precision so that we can store + // a timestamp into a `Long`. This design decision is subject to change though, for example, + // we may resort to microsecond precision in the future. + // + // For Parquet, we plan to write all `TimestampType` value as `TIMESTAMP_MICROS`, but it's + // currently not implemented yet because parquet-mr 1.7.0 (the version we're currently using) + // hasn't implemented `TIMESTAMP_MICROS` yet. + // + // TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that. case TimestampType => Types.primitive(INT96, repetition).named(field.name) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 86a77bf965daa..be0a2029d233b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -17,61 +17,15 @@ package org.apache.spark.sql.parquet -import java.nio.ByteOrder - -import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap} - -import org.apache.parquet.Preconditions -import org.apache.parquet.column.Dictionary -import org.apache.parquet.io.api.{Binary, Converter, GroupConverter, PrimitiveConverter} -import org.apache.parquet.schema.MessageType - import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.parquet.CatalystConverter.FieldType -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String - -/** - * Collection of converters of Parquet types (group and primitive types) that - * model arrays and maps. The conversions are partly based on the AvroParquet - * converters that are part of Parquet in order to be able to process these - * types. - * - * There are several types of converters: - *
    - *
  • [[org.apache.spark.sql.parquet.CatalystPrimitiveConverter]] for primitive - * (numeric, boolean and String) types
  • - *
  • [[org.apache.spark.sql.parquet.CatalystNativeArrayConverter]] for arrays - * of native JVM element types; note: currently null values are not supported!
  • - *
  • [[org.apache.spark.sql.parquet.CatalystArrayConverter]] for arrays of - * arbitrary element types (including nested element types); note: currently - * null values are not supported!
  • - *
  • [[org.apache.spark.sql.parquet.CatalystStructConverter]] for structs
  • - *
  • [[org.apache.spark.sql.parquet.CatalystMapConverter]] for maps; note: - * currently null values are not supported!
  • - *
  • [[org.apache.spark.sql.parquet.CatalystPrimitiveRowConverter]] for rows - * of only primitive element types
  • - *
  • [[org.apache.spark.sql.parquet.CatalystGroupConverter]] for other nested - * records, including the top-level row record
  • - *
- */ private[sql] object CatalystConverter { - // The type internally used for fields - type FieldType = StructField - // This is mostly Parquet convention (see, e.g., `ConversionPatterns`). // Note that "array" for the array elements is chosen by ParquetAvro. // Using a different value will result in Parquet silently dropping columns. val ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME = "bag" val ARRAY_ELEMENTS_SCHEMA_NAME = "array" - // SPARK-4520: Thrift generated parquet files have different array element - // schema names than avro. Thrift parquet uses array_schema_name + "_tuple" - // as opposed to "array" used by default. For more information, check - // TestThriftSchemaConverter.java in parquet.thrift. - val THRIFT_ARRAY_ELEMENTS_SCHEMA_NAME_SUFFIX = "_tuple" + val MAP_KEY_SCHEMA_NAME = "key" val MAP_VALUE_SCHEMA_NAME = "value" val MAP_SCHEMA_NAME = "map" @@ -80,787 +34,4 @@ private[sql] object CatalystConverter { type ArrayScalaType[T] = Seq[T] type StructScalaType[T] = InternalRow type MapScalaType[K, V] = Map[K, V] - - protected[parquet] def createConverter( - field: FieldType, - fieldIndex: Int, - parent: CatalystConverter): Converter = { - val fieldType: DataType = field.dataType - fieldType match { - case udt: UserDefinedType[_] => { - createConverter(field.copy(dataType = udt.sqlType), fieldIndex, parent) - } - // For native JVM types we use a converter with native arrays - case ArrayType(elementType: AtomicType, false) => { - new CatalystNativeArrayConverter(elementType, fieldIndex, parent) - } - // This is for other types of arrays, including those with nested fields - case ArrayType(elementType: DataType, false) => { - new CatalystArrayConverter(elementType, fieldIndex, parent) - } - case ArrayType(elementType: DataType, true) => { - new CatalystArrayContainsNullConverter(elementType, fieldIndex, parent) - } - case StructType(fields: Array[StructField]) => { - new CatalystStructConverter(fields, fieldIndex, parent) - } - case MapType(keyType: DataType, valueType: DataType, valueContainsNull: Boolean) => { - new CatalystMapConverter( - Array( - new FieldType(MAP_KEY_SCHEMA_NAME, keyType, false), - new FieldType(MAP_VALUE_SCHEMA_NAME, valueType, valueContainsNull)), - fieldIndex, - parent) - } - // Strings, Shorts and Bytes do not have a corresponding type in Parquet - // so we need to treat them separately - case StringType => - new CatalystPrimitiveStringConverter(parent, fieldIndex) - case ShortType => { - new CatalystPrimitiveConverter(parent, fieldIndex) { - override def addInt(value: Int): Unit = - parent.updateShort(fieldIndex, value.asInstanceOf[ShortType.InternalType]) - } - } - case ByteType => { - new CatalystPrimitiveConverter(parent, fieldIndex) { - override def addInt(value: Int): Unit = - parent.updateByte(fieldIndex, value.asInstanceOf[ByteType.InternalType]) - } - } - case DateType => { - new CatalystPrimitiveConverter(parent, fieldIndex) { - override def addInt(value: Int): Unit = - parent.updateDate(fieldIndex, value.asInstanceOf[DateType.InternalType]) - } - } - case d: DecimalType => { - new CatalystPrimitiveConverter(parent, fieldIndex) { - override def addBinary(value: Binary): Unit = - parent.updateDecimal(fieldIndex, value, d) - } - } - case TimestampType => { - new CatalystPrimitiveConverter(parent, fieldIndex) { - override def addBinary(value: Binary): Unit = - parent.updateTimestamp(fieldIndex, value) - } - } - // All other primitive types use the default converter - case ctype: DataType if ParquetTypesConverter.isPrimitiveType(ctype) => { - // note: need the type tag here! - new CatalystPrimitiveConverter(parent, fieldIndex) - } - case _ => throw new RuntimeException( - s"unable to convert datatype ${field.dataType.toString} in CatalystConverter") - } - } - - protected[parquet] def createRootConverter( - parquetSchema: MessageType, - attributes: Seq[Attribute]): CatalystConverter = { - // For non-nested types we use the optimized Row converter - if (attributes.forall(a => ParquetTypesConverter.isPrimitiveType(a.dataType))) { - new CatalystPrimitiveRowConverter(attributes.toArray) - } else { - new CatalystGroupConverter(attributes.toArray) - } - } -} - -private[parquet] abstract class CatalystConverter extends GroupConverter { - /** - * The number of fields this group has - */ - protected[parquet] val size: Int - - /** - * The index of this converter in the parent - */ - protected[parquet] val index: Int - - /** - * The parent converter - */ - protected[parquet] val parent: CatalystConverter - - /** - * Called by child converters to update their value in its parent (this). - * Note that if possible the more specific update methods below should be used - * to avoid auto-boxing of native JVM types. - * - * @param fieldIndex - * @param value - */ - protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit - - protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateDate(fieldIndex: Int, value: Int): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateShort(fieldIndex: Int, value: Short): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateByte(fieldIndex: Int, value: Byte): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = - updateField(fieldIndex, value) - - protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = - updateField(fieldIndex, value.getBytes) - - protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = - updateField(fieldIndex, UTF8String.fromBytes(value)) - - protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = - updateField(fieldIndex, readTimestamp(value)) - - protected[parquet] def updateDecimal(fieldIndex: Int, value: Binary, ctype: DecimalType): Unit = - updateField(fieldIndex, readDecimal(new Decimal(), value, ctype)) - - protected[parquet] def isRootConverter: Boolean = parent == null - - protected[parquet] def clearBuffer(): Unit - - /** - * Should only be called in the root (group) converter! - * - * @return - */ - def getCurrentRecord: InternalRow = throw new UnsupportedOperationException - - /** - * Read a decimal value from a Parquet Binary into "dest". Only supports decimals that fit in - * a long (i.e. precision <= 18) - * - * Returned value is needed by CatalystConverter, which doesn't reuse the Decimal object. - */ - protected[parquet] def readDecimal(dest: Decimal, value: Binary, ctype: DecimalType): Decimal = { - val precision = ctype.precisionInfo.get.precision - val scale = ctype.precisionInfo.get.scale - val bytes = value.getBytes - require(bytes.length <= 16, "Decimal field too large to read") - var unscaled = 0L - var i = 0 - while (i < bytes.length) { - unscaled = (unscaled << 8) | (bytes(i) & 0xFF) - i += 1 - } - // Make sure unscaled has the right sign, by sign-extending the first bit - val numBits = 8 * bytes.length - unscaled = (unscaled << (64 - numBits)) >> (64 - numBits) - dest.set(unscaled, precision, scale) - } - - /** - * Read a Timestamp value from a Parquet Int96Value - */ - protected[parquet] def readTimestamp(value: Binary): Long = { - Preconditions.checkArgument(value.length() == 12, "Must be 12 bytes") - val buf = value.toByteBuffer - buf.order(ByteOrder.LITTLE_ENDIAN) - val timeOfDayNanos = buf.getLong - val julianDay = buf.getInt - DateTimeUtils.fromJulianDay(julianDay, timeOfDayNanos) - } -} - -/** - * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record - * to a [[org.apache.spark.sql.catalyst.expressions.InternalRow]] object. - * - * @param schema The corresponding Catalyst schema in the form of a list of attributes. - */ -private[parquet] class CatalystGroupConverter( - protected[parquet] val schema: Array[FieldType], - protected[parquet] val index: Int, - protected[parquet] val parent: CatalystConverter, - protected[parquet] var current: ArrayBuffer[Any], - protected[parquet] var buffer: ArrayBuffer[InternalRow]) - extends CatalystConverter { - - def this(schema: Array[FieldType], index: Int, parent: CatalystConverter) = - this( - schema, - index, - parent, - current = null, - buffer = new ArrayBuffer[InternalRow]( - CatalystArrayConverter.INITIAL_ARRAY_SIZE)) - - /** - * This constructor is used for the root converter only! - */ - def this(attributes: Array[Attribute]) = - this(attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), 0, null) - - protected [parquet] val converters: Array[Converter] = - schema.zipWithIndex.map { - case (field, idx) => CatalystConverter.createConverter(field, idx, this) - }.toArray - - override val size = schema.size - - override def getCurrentRecord: InternalRow = { - assert(isRootConverter, "getCurrentRecord should only be called in root group converter!") - // TODO: use iterators if possible - // Note: this will ever only be called in the root converter when the record has been - // fully processed. Therefore it will be difficult to use mutable rows instead, since - // any non-root converter never would be sure when it would be safe to re-use the buffer. - new GenericInternalRow(current.toArray) - } - - override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) - - // for child converters to update upstream values - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - current.update(fieldIndex, value) - } - - override protected[parquet] def clearBuffer(): Unit = buffer.clear() - - override def start(): Unit = { - current = ArrayBuffer.fill(size)(null) - converters.foreach { converter => - if (!converter.isPrimitive) { - converter.asInstanceOf[CatalystConverter].clearBuffer() - } - } - } - - override def end(): Unit = { - if (!isRootConverter) { - assert(current != null) // there should be no empty groups - buffer.append(new GenericInternalRow(current.toArray)) - parent.updateField(index, new GenericInternalRow(buffer.toArray.asInstanceOf[Array[Any]])) - } - } -} - -/** - * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record - * to a [[org.apache.spark.sql.catalyst.expressions.InternalRow]] object. Note that his - * converter is optimized for rows of primitive types (non-nested records). - */ -private[parquet] class CatalystPrimitiveRowConverter( - protected[parquet] val schema: Array[FieldType], - protected[parquet] var current: MutableRow) - extends CatalystConverter { - - // This constructor is used for the root converter only - def this(attributes: Array[Attribute]) = - this( - attributes.map(a => new FieldType(a.name, a.dataType, a.nullable)), - new SpecificMutableRow(attributes.map(_.dataType))) - - protected [parquet] val converters: Array[Converter] = - schema.zipWithIndex.map { - case (field, idx) => CatalystConverter.createConverter(field, idx, this) - }.toArray - - override val size = schema.size - - override val index = 0 - - override val parent = null - - // Should be only called in root group converter! - override def getCurrentRecord: InternalRow = current - - override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) - - // for child converters to update upstream values - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - throw new UnsupportedOperationException // child converters should use the - // specific update methods below - } - - override protected[parquet] def clearBuffer(): Unit = {} - - override def start(): Unit = { - var i = 0 - while (i < size) { - current.setNullAt(i) - i = i + 1 - } - } - - override def end(): Unit = {} - - // Overridden here to avoid auto-boxing for primitive types - override protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = - current.setBoolean(fieldIndex, value) - - override protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = - current.setInt(fieldIndex, value) - - override protected[parquet] def updateDate(fieldIndex: Int, value: Int): Unit = - current.setInt(fieldIndex, value) - - override protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = - current.setLong(fieldIndex, value) - - override protected[parquet] def updateShort(fieldIndex: Int, value: Short): Unit = - current.setShort(fieldIndex, value) - - override protected[parquet] def updateByte(fieldIndex: Int, value: Byte): Unit = - current.setByte(fieldIndex, value) - - override protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = - current.setDouble(fieldIndex, value) - - override protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = - current.setFloat(fieldIndex, value) - - override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = - current.update(fieldIndex, value.getBytes) - - override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = - current.update(fieldIndex, UTF8String.fromBytes(value)) - - override protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = - current.setLong(fieldIndex, readTimestamp(value)) - - override protected[parquet] def updateDecimal( - fieldIndex: Int, value: Binary, ctype: DecimalType): Unit = { - var decimal = current(fieldIndex).asInstanceOf[Decimal] - if (decimal == null) { - decimal = new Decimal - current(fieldIndex) = decimal - } - readDecimal(decimal, value, ctype) - } -} - -/** - * A `parquet.io.api.PrimitiveConverter` that converts Parquet types to Catalyst types. - * - * @param parent The parent group converter. - * @param fieldIndex The index inside the record. - */ -private[parquet] class CatalystPrimitiveConverter( - parent: CatalystConverter, - fieldIndex: Int) extends PrimitiveConverter { - override def addBinary(value: Binary): Unit = - parent.updateBinary(fieldIndex, value) - - override def addBoolean(value: Boolean): Unit = - parent.updateBoolean(fieldIndex, value) - - override def addDouble(value: Double): Unit = - parent.updateDouble(fieldIndex, value) - - override def addFloat(value: Float): Unit = - parent.updateFloat(fieldIndex, value) - - override def addInt(value: Int): Unit = - parent.updateInt(fieldIndex, value) - - override def addLong(value: Long): Unit = - parent.updateLong(fieldIndex, value) -} - -/** - * A `parquet.io.api.PrimitiveConverter` that converts Parquet Binary to Catalyst String. - * Supports dictionaries to reduce Binary to String conversion overhead. - * - * Follows pattern in Parquet of using dictionaries, where supported, for String conversion. - * - * @param parent The parent group converter. - * @param fieldIndex The index inside the record. - */ -private[parquet] class CatalystPrimitiveStringConverter(parent: CatalystConverter, fieldIndex: Int) - extends CatalystPrimitiveConverter(parent, fieldIndex) { - - private[this] var dict: Array[Array[Byte]] = null - - override def hasDictionarySupport: Boolean = true - - override def setDictionary(dictionary: Dictionary): Unit = - dict = Array.tabulate(dictionary.getMaxId + 1) { dictionary.decodeToBinary(_).getBytes } - - override def addValueFromDictionary(dictionaryId: Int): Unit = - parent.updateString(fieldIndex, dict(dictionaryId)) - - override def addBinary(value: Binary): Unit = - parent.updateString(fieldIndex, value.getBytes) -} - -private[parquet] object CatalystArrayConverter { - val INITIAL_ARRAY_SIZE = 20 -} - -/** - * A `parquet.io.api.GroupConverter` that converts a single-element groups that - * match the characteristics of an array (see - * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.types.ArrayType]]. - * - * @param elementType The type of the array elements (complex or primitive) - * @param index The position of this (array) field inside its parent converter - * @param parent The parent converter - * @param buffer A data buffer - */ -private[parquet] class CatalystArrayConverter( - val elementType: DataType, - val index: Int, - protected[parquet] val parent: CatalystConverter, - protected[parquet] var buffer: Buffer[Any]) - extends CatalystConverter { - - def this(elementType: DataType, index: Int, parent: CatalystConverter) = - this( - elementType, - index, - parent, - new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE)) - - protected[parquet] val converter: Converter = CatalystConverter.createConverter( - new CatalystConverter.FieldType( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - elementType, - false), - fieldIndex = 0, - parent = this) - - override def getConverter(fieldIndex: Int): Converter = converter - - // arrays have only one (repeated) field, which is its elements - override val size = 1 - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - // fieldIndex is ignored (assumed to be zero but not checked) - if (value == null) { - throw new IllegalArgumentException("Null values inside Parquet arrays are not supported!") - } - buffer += value - } - - override protected[parquet] def clearBuffer(): Unit = { - buffer.clear() - } - - override def start(): Unit = { - if (!converter.isPrimitive) { - converter.asInstanceOf[CatalystConverter].clearBuffer() - } - } - - override def end(): Unit = { - assert(parent != null) - // here we need to make sure to use ArrayScalaType - parent.updateField(index, buffer.toArray.toSeq) - clearBuffer() - } -} - -/** - * A `parquet.io.api.GroupConverter` that converts a single-element groups that - * match the characteristics of an array (see - * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.types.ArrayType]]. - * - * @param elementType The type of the array elements (native) - * @param index The position of this (array) field inside its parent converter - * @param parent The parent converter - * @param capacity The (initial) capacity of the buffer - */ -private[parquet] class CatalystNativeArrayConverter( - val elementType: AtomicType, - val index: Int, - protected[parquet] val parent: CatalystConverter, - protected[parquet] var capacity: Int = CatalystArrayConverter.INITIAL_ARRAY_SIZE) - extends CatalystConverter { - - type NativeType = elementType.InternalType - - private var buffer: Array[NativeType] = elementType.classTag.newArray(capacity) - - private var elements: Int = 0 - - protected[parquet] val converter: Converter = CatalystConverter.createConverter( - new CatalystConverter.FieldType( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - elementType, - false), - fieldIndex = 0, - parent = this) - - override def getConverter(fieldIndex: Int): Converter = converter - - // arrays have only one (repeated) field, which is its elements - override val size = 1 - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = - throw new UnsupportedOperationException - - // Overridden here to avoid auto-boxing for primitive types - override protected[parquet] def updateBoolean(fieldIndex: Int, value: Boolean): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateInt(fieldIndex: Int, value: Int): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateShort(fieldIndex: Int, value: Short): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateByte(fieldIndex: Int, value: Byte): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateLong(fieldIndex: Int, value: Long): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateDouble(fieldIndex: Int, value: Double): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateFloat(fieldIndex: Int, value: Float): Unit = { - checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = { - checkGrowBuffer() - buffer(elements) = value.getBytes.asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = { - checkGrowBuffer() - buffer(elements) = UTF8String.fromBytes(value).asInstanceOf[NativeType] - elements += 1 - } - - override protected[parquet] def clearBuffer(): Unit = { - elements = 0 - } - - override def start(): Unit = {} - - override def end(): Unit = { - assert(parent != null) - // here we need to make sure to use ArrayScalaType - parent.updateField( - index, - buffer.slice(0, elements).toSeq) - clearBuffer() - } - - private def checkGrowBuffer(): Unit = { - if (elements >= capacity) { - val newCapacity = 2 * capacity - val tmp: Array[NativeType] = elementType.classTag.newArray(newCapacity) - Array.copy(buffer, 0, tmp, 0, capacity) - buffer = tmp - capacity = newCapacity - } - } -} - -/** - * A `parquet.io.api.GroupConverter` that converts a single-element groups that - * match the characteristics of an array contains null (see - * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.types.ArrayType]]. - * - * @param elementType The type of the array elements (complex or primitive) - * @param index The position of this (array) field inside its parent converter - * @param parent The parent converter - * @param buffer A data buffer - */ -private[parquet] class CatalystArrayContainsNullConverter( - val elementType: DataType, - val index: Int, - protected[parquet] val parent: CatalystConverter, - protected[parquet] var buffer: Buffer[Any]) - extends CatalystConverter { - - def this(elementType: DataType, index: Int, parent: CatalystConverter) = - this( - elementType, - index, - parent, - new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE)) - - protected[parquet] val converter: Converter = new CatalystConverter { - - private var current: Any = null - - val converter = CatalystConverter.createConverter( - new CatalystConverter.FieldType( - CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, - elementType, - false), - fieldIndex = 0, - parent = this) - - override def getConverter(fieldIndex: Int): Converter = converter - - override def end(): Unit = parent.updateField(index, current) - - override def start(): Unit = { - current = null - } - - override protected[parquet] val size: Int = 1 - override protected[parquet] val index: Int = 0 - override protected[parquet] val parent = CatalystArrayContainsNullConverter.this - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - current = value - } - - override protected[parquet] def clearBuffer(): Unit = {} - } - - override def getConverter(fieldIndex: Int): Converter = converter - - // arrays have only one (repeated) field, which is its elements - override val size = 1 - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - buffer += value - } - - override protected[parquet] def clearBuffer(): Unit = { - buffer.clear() - } - - override def start(): Unit = {} - - override def end(): Unit = { - assert(parent != null) - // here we need to make sure to use ArrayScalaType - parent.updateField(index, buffer.toArray.toSeq) - clearBuffer() - } -} - -/** - * This converter is for multi-element groups of primitive or complex types - * that have repetition level optional or required (so struct fields). - * - * @param schema The corresponding Catalyst schema in the form of a list of - * attributes. - * @param index - * @param parent - */ -private[parquet] class CatalystStructConverter( - override protected[parquet] val schema: Array[FieldType], - override protected[parquet] val index: Int, - override protected[parquet] val parent: CatalystConverter) - extends CatalystGroupConverter(schema, index, parent) { - - override protected[parquet] def clearBuffer(): Unit = {} - - // TODO: think about reusing the buffer - override def end(): Unit = { - assert(!isRootConverter) - // here we need to make sure to use StructScalaType - // Note: we need to actually make a copy of the array since we - // may be in a nested field - parent.updateField(index, new GenericInternalRow(current.toArray)) - } -} - -/** - * A `parquet.io.api.GroupConverter` that converts two-element groups that - * match the characteristics of a map (see - * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an - * [[org.apache.spark.sql.types.MapType]]. - * - * @param schema - * @param index - * @param parent - */ -private[parquet] class CatalystMapConverter( - protected[parquet] val schema: Array[FieldType], - override protected[parquet] val index: Int, - override protected[parquet] val parent: CatalystConverter) - extends CatalystConverter { - - private val map = new HashMap[Any, Any]() - - private val keyValueConverter = new CatalystConverter { - private var currentKey: Any = null - private var currentValue: Any = null - val keyConverter = CatalystConverter.createConverter(schema(0), 0, this) - val valueConverter = CatalystConverter.createConverter(schema(1), 1, this) - - override def getConverter(fieldIndex: Int): Converter = { - if (fieldIndex == 0) keyConverter else valueConverter - } - - override def end(): Unit = CatalystMapConverter.this.map += currentKey -> currentValue - - override def start(): Unit = { - currentKey = null - currentValue = null - } - - override protected[parquet] val size: Int = 2 - override protected[parquet] val index: Int = 0 - override protected[parquet] val parent: CatalystConverter = CatalystMapConverter.this - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { - fieldIndex match { - case 0 => - currentKey = value - case 1 => - currentValue = value - case _ => - new RuntimePermission(s"trying to update Map with fieldIndex $fieldIndex") - } - } - - override protected[parquet] def clearBuffer(): Unit = {} - } - - override protected[parquet] val size: Int = 1 - - override protected[parquet] def clearBuffer(): Unit = {} - - override def start(): Unit = { - map.clear() - } - - override def end(): Unit = { - // here we need to make sure to use MapScalaType - parent.updateField(index, map.toMap) - } - - override def getConverter(fieldIndex: Int): Converter = keyValueConverter - - override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = - throw new UnsupportedOperationException } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 8402cd756140d..e8851ddb68026 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -17,14 +17,17 @@ package org.apache.spark.sql.parquet -import java.nio.{ByteOrder, ByteBuffer} +import java.nio.{ByteBuffer, ByteOrder} +import java.util import java.util.{HashMap => JHashMap} +import scala.collection.JavaConversions._ + import org.apache.hadoop.conf.Configuration import org.apache.parquet.column.ParquetProperties import org.apache.parquet.hadoop.ParquetOutputFormat import org.apache.parquet.hadoop.api.ReadSupport.ReadContext -import org.apache.parquet.hadoop.api.{ReadSupport, WriteSupport} +import org.apache.parquet.hadoop.api.{InitContext, ReadSupport, WriteSupport} import org.apache.parquet.io.api._ import org.apache.parquet.schema.MessageType @@ -36,87 +39,133 @@ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String /** - * A `parquet.io.api.RecordMaterializer` for Rows. + * A [[RecordMaterializer]] for Catalyst rows. * - *@param root The root group converter for the record. + * @param parquetSchema Parquet schema of the records to be read + * @param catalystSchema Catalyst schema of the rows to be constructed */ -private[parquet] class RowRecordMaterializer(root: CatalystConverter) +private[parquet] class RowRecordMaterializer(parquetSchema: MessageType, catalystSchema: StructType) extends RecordMaterializer[InternalRow] { - def this(parquetSchema: MessageType, attributes: Seq[Attribute]) = - this(CatalystConverter.createRootConverter(parquetSchema, attributes)) + private val rootConverter = new CatalystRowConverter(parquetSchema, catalystSchema, NoopUpdater) - override def getCurrentRecord: InternalRow = root.getCurrentRecord + override def getCurrentRecord: InternalRow = rootConverter.currentRow - override def getRootConverter: GroupConverter = root.asInstanceOf[GroupConverter] + override def getRootConverter: GroupConverter = rootConverter } -/** - * A `parquet.hadoop.api.ReadSupport` for Row objects. - */ private[parquet] class RowReadSupport extends ReadSupport[InternalRow] with Logging { - override def prepareForRead( conf: Configuration, - stringMap: java.util.Map[String, String], + keyValueMetaData: util.Map[String, String], fileSchema: MessageType, readContext: ReadContext): RecordMaterializer[InternalRow] = { - log.debug(s"preparing for read with Parquet file schema $fileSchema") - // Note: this very much imitates AvroParquet - val parquetSchema = readContext.getRequestedSchema - var schema: Seq[Attribute] = null - - if (readContext.getReadSupportMetadata != null) { - // first try to find the read schema inside the metadata (can result from projections) - if ( - readContext - .getReadSupportMetadata - .get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA) != null) { - schema = ParquetTypesConverter.convertFromString( - readContext.getReadSupportMetadata.get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA)) - } else { - // if unavailable, try the schema that was read originally from the file or provided - // during the creation of the Parquet relation - if (readContext.getReadSupportMetadata.get(RowReadSupport.SPARK_METADATA_KEY) != null) { - schema = ParquetTypesConverter.convertFromString( - readContext.getReadSupportMetadata.get(RowReadSupport.SPARK_METADATA_KEY)) - } + log.debug(s"Preparing for read Parquet file with message type: $fileSchema") + + val toCatalyst = new CatalystSchemaConverter(conf) + val parquetRequestedSchema = readContext.getRequestedSchema + + val catalystRequestedSchema = + Option(readContext.getReadSupportMetadata).map(_.toMap).flatMap { metadata => + metadata + // First tries to read requested schema, which may result from projections + .get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA) + // If not available, tries to read Catalyst schema from file metadata. It's only + // available if the target file is written by Spark SQL. + .orElse(metadata.get(RowReadSupport.SPARK_METADATA_KEY)) + }.map(StructType.fromString).getOrElse { + logDebug("Catalyst schema not available, falling back to Parquet schema") + toCatalyst.convert(parquetRequestedSchema) } - } - // if both unavailable, fall back to deducing the schema from the given Parquet schema - // TODO: Why it can be null? - if (schema == null) { - log.debug("falling back to Parquet read schema") - schema = ParquetTypesConverter.convertToAttributes(parquetSchema, false, true) - } - log.debug(s"list of attributes that will be read: $schema") - new RowRecordMaterializer(parquetSchema, schema) + + logDebug(s"Catalyst schema used to read Parquet files: $catalystRequestedSchema") + new RowRecordMaterializer(parquetRequestedSchema, catalystRequestedSchema) } - override def init( - configuration: Configuration, - keyValueMetaData: java.util.Map[String, String], - fileSchema: MessageType): ReadContext = { - var parquetSchema = fileSchema - val metadata = new JHashMap[String, String]() - val requestedAttributes = RowReadSupport.getRequestedSchema(configuration) - - if (requestedAttributes != null) { - // If the parquet file is thrift derived, there is a good chance that - // it will have the thrift class in metadata. - val isThriftDerived = keyValueMetaData.keySet().contains("thrift.class") - parquetSchema = ParquetTypesConverter.convertFromAttributes(requestedAttributes) - metadata.put( - RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, - ParquetTypesConverter.convertToString(requestedAttributes)) - } + override def init(context: InitContext): ReadContext = { + val conf = context.getConfiguration + + // If the target file was written by Spark SQL, we should be able to find a serialized Catalyst + // schema of this file from its the metadata. + val maybeRowSchema = Option(conf.get(RowWriteSupport.SPARK_ROW_SCHEMA)) + + // Optional schema of requested columns, in the form of a string serialized from a Catalyst + // `StructType` containing all requested columns. + val maybeRequestedSchema = Option(conf.get(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA)) + + // Below we construct a Parquet schema containing all requested columns. This schema tells + // Parquet which columns to read. + // + // If `maybeRequestedSchema` is defined, we assemble an equivalent Parquet schema. Otherwise, + // we have to fallback to the full file schema which contains all columns in the file. + // Obviously this may waste IO bandwidth since it may read more columns than requested. + // + // Two things to note: + // + // 1. It's possible that some requested columns don't exist in the target Parquet file. For + // example, in the case of schema merging, the globally merged schema may contain extra + // columns gathered from other Parquet files. These columns will be simply filled with nulls + // when actually reading the target Parquet file. + // + // 2. When `maybeRequestedSchema` is available, we can't simply convert the Catalyst schema to + // Parquet schema using `CatalystSchemaConverter`, because the mapping is not unique due to + // non-standard behaviors of some Parquet libraries/tools. For example, a Parquet file + // containing a single integer array field `f1` may have the following legacy 2-level + // structure: + // + // message root { + // optional group f1 (LIST) { + // required INT32 element; + // } + // } + // + // while `CatalystSchemaConverter` may generate a standard 3-level structure: + // + // message root { + // optional group f1 (LIST) { + // repeated group list { + // required INT32 element; + // } + // } + // } + // + // Apparently, we can't use the 2nd schema to read the target Parquet file as they have + // different physical structures. + val parquetRequestedSchema = + maybeRequestedSchema.fold(context.getFileSchema) { schemaString => + val toParquet = new CatalystSchemaConverter(conf) + val fileSchema = context.getFileSchema.asGroupType() + val fileFieldNames = fileSchema.getFields.map(_.getName).toSet + + StructType + // Deserializes the Catalyst schema of requested columns + .fromString(schemaString) + .map { field => + if (fileFieldNames.contains(field.name)) { + // If the field exists in the target Parquet file, extracts the field type from the + // full file schema and makes a single-field Parquet schema + new MessageType("root", fileSchema.getType(field.name)) + } else { + // Otherwise, just resorts to `CatalystSchemaConverter` + toParquet.convert(StructType(Array(field))) + } + } + // Merges all single-field Parquet schemas to form a complete schema for all requested + // columns. Note that it's possible that no columns are requested at all (e.g., count + // some partition column of a partitioned Parquet table). That's why `fold` is used here + // and always fallback to an empty Parquet schema. + .fold(new MessageType("root")) { + _ union _ + } + } - val origAttributesStr: String = configuration.get(RowWriteSupport.SPARK_ROW_SCHEMA) - if (origAttributesStr != null) { - metadata.put(RowReadSupport.SPARK_METADATA_KEY, origAttributesStr) - } + val metadata = + Map.empty[String, String] ++ + maybeRequestedSchema.map(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA -> _) ++ + maybeRowSchema.map(RowWriteSupport.SPARK_ROW_SCHEMA -> _) - new ReadSupport.ReadContext(parquetSchema, metadata) + logInfo(s"Going to read Parquet file with these requested columns: $parquetRequestedSchema") + new ReadContext(parquetRequestedSchema, metadata) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index ce456e7fbe17e..01dd6f471bd7c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -259,6 +259,10 @@ private[sql] class ParquetRelation2( broadcastedConf: Broadcast[SerializableConfiguration]): RDD[Row] = { val useMetadataCache = sqlContext.getConf(SQLConf.PARQUET_CACHE_METADATA) val parquetFilterPushDown = sqlContext.conf.parquetFilterPushDown + val assumeBinaryIsString = sqlContext.conf.isParquetBinaryAsString + val assumeInt96IsTimestamp = sqlContext.conf.isParquetINT96AsTimestamp + val followParquetFormatSpec = sqlContext.conf.followParquetFormatSpec + // Create the function to set variable Parquet confs at both driver and executor side. val initLocalJobFuncOpt = ParquetRelation2.initializeLocalJobFunc( @@ -266,7 +270,11 @@ private[sql] class ParquetRelation2( filters, dataSchema, useMetadataCache, - parquetFilterPushDown) _ + parquetFilterPushDown, + assumeBinaryIsString, + assumeInt96IsTimestamp, + followParquetFormatSpec) _ + // Create the function to set input paths at the driver side. val setInputPaths = ParquetRelation2.initializeDriverSideJobFunc(inputFiles) _ @@ -471,9 +479,12 @@ private[sql] object ParquetRelation2 extends Logging { filters: Array[Filter], dataSchema: StructType, useMetadataCache: Boolean, - parquetFilterPushDown: Boolean)(job: Job): Unit = { + parquetFilterPushDown: Boolean, + assumeBinaryIsString: Boolean, + assumeInt96IsTimestamp: Boolean, + followParquetFormatSpec: Boolean)(job: Job): Unit = { val conf = job.getConfiguration - conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[RowReadSupport].getName()) + conf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[RowReadSupport].getName) // Try to push down filters when filter push-down is enabled. if (parquetFilterPushDown) { @@ -497,6 +508,11 @@ private[sql] object ParquetRelation2 extends Logging { // Tell FilteringParquetRowInputFormat whether it's okay to cache Parquet and FS metadata conf.setBoolean(SQLConf.PARQUET_CACHE_METADATA.key, useMetadataCache) + + // Sets flags for Parquet schema conversion + conf.setBoolean(SQLConf.PARQUET_BINARY_AS_STRING.key, assumeBinaryIsString) + conf.setBoolean(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, assumeInt96IsTimestamp) + conf.setBoolean(SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC.key, followParquetFormatSpec) } /** This closure sets input paths at the driver side. */ diff --git a/sql/core/src/test/README.md b/sql/core/src/test/README.md new file mode 100644 index 0000000000000..3dd9861b4896d --- /dev/null +++ b/sql/core/src/test/README.md @@ -0,0 +1,33 @@ +# Notes for Parquet compatibility tests + +The following directories and files are used for Parquet compatibility tests: + +``` +. +├── README.md # This file +├── avro +│   ├── parquet-compat.avdl # Testing Avro IDL +│   └── parquet-compat.avpr # !! NO TOUCH !! Protocol file generated from parquet-compat.avdl +├── gen-java # !! NO TOUCH !! Generated Java code +├── scripts +│   └── gen-code.sh # Script used to generate Java code for Thrift and Avro +└── thrift + └── parquet-compat.thrift # Testing Thrift schema +``` + +Generated Java code are used in the following test suites: + +- `org.apache.spark.sql.parquet.ParquetAvroCompatibilitySuite` +- `org.apache.spark.sql.parquet.ParquetThriftCompatibilitySuite` + +To avoid code generation during build time, Java code generated from testing Thrift schema and Avro IDL are also checked in. + +When updating the testing Thrift schema and Avro IDL, please run `gen-code.sh` to update all the generated Java code. + +## Prerequisites + +Please ensure `avro-tools` and `thrift` are installed. You may install these two on Mac OS X via: + +```bash +$ brew install thrift avro-tools +``` diff --git a/sql/core/src/test/avro/parquet-compat.avdl b/sql/core/src/test/avro/parquet-compat.avdl new file mode 100644 index 0000000000000..24729f6143e6c --- /dev/null +++ b/sql/core/src/test/avro/parquet-compat.avdl @@ -0,0 +1,47 @@ +/* + * 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. + */ + +// This is a test protocol for testing parquet-avro compatibility. +@namespace("org.apache.spark.sql.parquet.test.avro") +protocol CompatibilityTest { + record Nested { + array nested_ints_column; + string nested_string_column; + } + + record ParquetAvroCompat { + boolean bool_column; + int int_column; + long long_column; + float float_column; + double double_column; + bytes binary_column; + string string_column; + + union { null, boolean } maybe_bool_column; + union { null, int } maybe_int_column; + union { null, long } maybe_long_column; + union { null, float } maybe_float_column; + union { null, double } maybe_double_column; + union { null, bytes } maybe_binary_column; + union { null, string } maybe_string_column; + + array strings_column; + map string_to_int_column; + map> complex_column; + } +} diff --git a/sql/core/src/test/avro/parquet-compat.avpr b/sql/core/src/test/avro/parquet-compat.avpr new file mode 100644 index 0000000000000..a83b7c990dd2e --- /dev/null +++ b/sql/core/src/test/avro/parquet-compat.avpr @@ -0,0 +1,86 @@ +{ + "protocol" : "CompatibilityTest", + "namespace" : "org.apache.spark.sql.parquet.test.avro", + "types" : [ { + "type" : "record", + "name" : "Nested", + "fields" : [ { + "name" : "nested_ints_column", + "type" : { + "type" : "array", + "items" : "int" + } + }, { + "name" : "nested_string_column", + "type" : "string" + } ] + }, { + "type" : "record", + "name" : "ParquetAvroCompat", + "fields" : [ { + "name" : "bool_column", + "type" : "boolean" + }, { + "name" : "int_column", + "type" : "int" + }, { + "name" : "long_column", + "type" : "long" + }, { + "name" : "float_column", + "type" : "float" + }, { + "name" : "double_column", + "type" : "double" + }, { + "name" : "binary_column", + "type" : "bytes" + }, { + "name" : "string_column", + "type" : "string" + }, { + "name" : "maybe_bool_column", + "type" : [ "null", "boolean" ] + }, { + "name" : "maybe_int_column", + "type" : [ "null", "int" ] + }, { + "name" : "maybe_long_column", + "type" : [ "null", "long" ] + }, { + "name" : "maybe_float_column", + "type" : [ "null", "float" ] + }, { + "name" : "maybe_double_column", + "type" : [ "null", "double" ] + }, { + "name" : "maybe_binary_column", + "type" : [ "null", "bytes" ] + }, { + "name" : "maybe_string_column", + "type" : [ "null", "string" ] + }, { + "name" : "strings_column", + "type" : { + "type" : "array", + "items" : "string" + } + }, { + "name" : "string_to_int_column", + "type" : { + "type" : "map", + "values" : "int" + } + }, { + "name" : "complex_column", + "type" : { + "type" : "map", + "values" : { + "type" : "array", + "items" : "Nested" + } + } + } ] + } ], + "messages" : { } +} \ No newline at end of file diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/CompatibilityTest.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/CompatibilityTest.java new file mode 100644 index 0000000000000..daec65a5bbe57 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/CompatibilityTest.java @@ -0,0 +1,17 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.spark.sql.parquet.test.avro; + +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public interface CompatibilityTest { + public static final org.apache.avro.Protocol PROTOCOL = org.apache.avro.Protocol.parse("{\"protocol\":\"CompatibilityTest\",\"namespace\":\"org.apache.spark.sql.parquet.test.avro\",\"types\":[{\"type\":\"record\",\"name\":\"Nested\",\"fields\":[{\"name\":\"nested_ints_column\",\"type\":{\"type\":\"array\",\"items\":\"int\"}},{\"name\":\"nested_string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]},{\"type\":\"record\",\"name\":\"ParquetAvroCompat\",\"fields\":[{\"name\":\"bool_column\",\"type\":\"boolean\"},{\"name\":\"int_column\",\"type\":\"int\"},{\"name\":\"long_column\",\"type\":\"long\"},{\"name\":\"float_column\",\"type\":\"float\"},{\"name\":\"double_column\",\"type\":\"double\"},{\"name\":\"binary_column\",\"type\":\"bytes\"},{\"name\":\"string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"maybe_bool_column\",\"type\":[\"null\",\"boolean\"]},{\"name\":\"maybe_int_column\",\"type\":[\"null\",\"int\"]},{\"name\":\"maybe_long_column\",\"type\":[\"null\",\"long\"]},{\"name\":\"maybe_float_column\",\"type\":[\"null\",\"float\"]},{\"name\":\"maybe_double_column\",\"type\":[\"null\",\"double\"]},{\"name\":\"maybe_binary_column\",\"type\":[\"null\",\"bytes\"]},{\"name\":\"maybe_string_column\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}]},{\"name\":\"strings_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}},{\"name\":\"string_to_int_column\",\"type\":{\"type\":\"map\",\"values\":\"int\",\"avro.java.string\":\"String\"}},{\"name\":\"complex_column\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"array\",\"items\":\"Nested\"},\"avro.java.string\":\"String\"}}]}],\"messages\":{}}"); + + @SuppressWarnings("all") + public interface Callback extends CompatibilityTest { + public static final org.apache.avro.Protocol PROTOCOL = org.apache.spark.sql.parquet.test.avro.CompatibilityTest.PROTOCOL; + } +} \ No newline at end of file diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/Nested.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/Nested.java new file mode 100644 index 0000000000000..051f1ee903863 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/Nested.java @@ -0,0 +1,196 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.spark.sql.parquet.test.avro; +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public class Nested extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"Nested\",\"namespace\":\"org.apache.spark.sql.parquet.test.avro\",\"fields\":[{\"name\":\"nested_ints_column\",\"type\":{\"type\":\"array\",\"items\":\"int\"}},{\"name\":\"nested_string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + @Deprecated public java.util.List nested_ints_column; + @Deprecated public java.lang.String nested_string_column; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public Nested() {} + + /** + * All-args constructor. + */ + public Nested(java.util.List nested_ints_column, java.lang.String nested_string_column) { + this.nested_ints_column = nested_ints_column; + this.nested_string_column = nested_string_column; + } + + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return nested_ints_column; + case 1: return nested_string_column; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: nested_ints_column = (java.util.List)value$; break; + case 1: nested_string_column = (java.lang.String)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'nested_ints_column' field. + */ + public java.util.List getNestedIntsColumn() { + return nested_ints_column; + } + + /** + * Sets the value of the 'nested_ints_column' field. + * @param value the value to set. + */ + public void setNestedIntsColumn(java.util.List value) { + this.nested_ints_column = value; + } + + /** + * Gets the value of the 'nested_string_column' field. + */ + public java.lang.String getNestedStringColumn() { + return nested_string_column; + } + + /** + * Sets the value of the 'nested_string_column' field. + * @param value the value to set. + */ + public void setNestedStringColumn(java.lang.String value) { + this.nested_string_column = value; + } + + /** Creates a new Nested RecordBuilder */ + public static org.apache.spark.sql.parquet.test.avro.Nested.Builder newBuilder() { + return new org.apache.spark.sql.parquet.test.avro.Nested.Builder(); + } + + /** Creates a new Nested RecordBuilder by copying an existing Builder */ + public static org.apache.spark.sql.parquet.test.avro.Nested.Builder newBuilder(org.apache.spark.sql.parquet.test.avro.Nested.Builder other) { + return new org.apache.spark.sql.parquet.test.avro.Nested.Builder(other); + } + + /** Creates a new Nested RecordBuilder by copying an existing Nested instance */ + public static org.apache.spark.sql.parquet.test.avro.Nested.Builder newBuilder(org.apache.spark.sql.parquet.test.avro.Nested other) { + return new org.apache.spark.sql.parquet.test.avro.Nested.Builder(other); + } + + /** + * RecordBuilder for Nested instances. + */ + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private java.util.List nested_ints_column; + private java.lang.String nested_string_column; + + /** Creates a new Builder */ + private Builder() { + super(org.apache.spark.sql.parquet.test.avro.Nested.SCHEMA$); + } + + /** Creates a Builder by copying an existing Builder */ + private Builder(org.apache.spark.sql.parquet.test.avro.Nested.Builder other) { + super(other); + if (isValidValue(fields()[0], other.nested_ints_column)) { + this.nested_ints_column = data().deepCopy(fields()[0].schema(), other.nested_ints_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.nested_string_column)) { + this.nested_string_column = data().deepCopy(fields()[1].schema(), other.nested_string_column); + fieldSetFlags()[1] = true; + } + } + + /** Creates a Builder by copying an existing Nested instance */ + private Builder(org.apache.spark.sql.parquet.test.avro.Nested other) { + super(org.apache.spark.sql.parquet.test.avro.Nested.SCHEMA$); + if (isValidValue(fields()[0], other.nested_ints_column)) { + this.nested_ints_column = data().deepCopy(fields()[0].schema(), other.nested_ints_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.nested_string_column)) { + this.nested_string_column = data().deepCopy(fields()[1].schema(), other.nested_string_column); + fieldSetFlags()[1] = true; + } + } + + /** Gets the value of the 'nested_ints_column' field */ + public java.util.List getNestedIntsColumn() { + return nested_ints_column; + } + + /** Sets the value of the 'nested_ints_column' field */ + public org.apache.spark.sql.parquet.test.avro.Nested.Builder setNestedIntsColumn(java.util.List value) { + validate(fields()[0], value); + this.nested_ints_column = value; + fieldSetFlags()[0] = true; + return this; + } + + /** Checks whether the 'nested_ints_column' field has been set */ + public boolean hasNestedIntsColumn() { + return fieldSetFlags()[0]; + } + + /** Clears the value of the 'nested_ints_column' field */ + public org.apache.spark.sql.parquet.test.avro.Nested.Builder clearNestedIntsColumn() { + nested_ints_column = null; + fieldSetFlags()[0] = false; + return this; + } + + /** Gets the value of the 'nested_string_column' field */ + public java.lang.String getNestedStringColumn() { + return nested_string_column; + } + + /** Sets the value of the 'nested_string_column' field */ + public org.apache.spark.sql.parquet.test.avro.Nested.Builder setNestedStringColumn(java.lang.String value) { + validate(fields()[1], value); + this.nested_string_column = value; + fieldSetFlags()[1] = true; + return this; + } + + /** Checks whether the 'nested_string_column' field has been set */ + public boolean hasNestedStringColumn() { + return fieldSetFlags()[1]; + } + + /** Clears the value of the 'nested_string_column' field */ + public org.apache.spark.sql.parquet.test.avro.Nested.Builder clearNestedStringColumn() { + nested_string_column = null; + fieldSetFlags()[1] = false; + return this; + } + + @Override + public Nested build() { + try { + Nested record = new Nested(); + record.nested_ints_column = fieldSetFlags()[0] ? this.nested_ints_column : (java.util.List) defaultValue(fields()[0]); + record.nested_string_column = fieldSetFlags()[1] ? this.nested_string_column : (java.lang.String) defaultValue(fields()[1]); + return record; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } +} diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/ParquetAvroCompat.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/ParquetAvroCompat.java new file mode 100644 index 0000000000000..354c9d73cca31 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/avro/ParquetAvroCompat.java @@ -0,0 +1,1001 @@ +/** + * Autogenerated by Avro + * + * DO NOT EDIT DIRECTLY + */ +package org.apache.spark.sql.parquet.test.avro; +@SuppressWarnings("all") +@org.apache.avro.specific.AvroGenerated +public class ParquetAvroCompat extends org.apache.avro.specific.SpecificRecordBase implements org.apache.avro.specific.SpecificRecord { + public static final org.apache.avro.Schema SCHEMA$ = new org.apache.avro.Schema.Parser().parse("{\"type\":\"record\",\"name\":\"ParquetAvroCompat\",\"namespace\":\"org.apache.spark.sql.parquet.test.avro\",\"fields\":[{\"name\":\"bool_column\",\"type\":\"boolean\"},{\"name\":\"int_column\",\"type\":\"int\"},{\"name\":\"long_column\",\"type\":\"long\"},{\"name\":\"float_column\",\"type\":\"float\"},{\"name\":\"double_column\",\"type\":\"double\"},{\"name\":\"binary_column\",\"type\":\"bytes\"},{\"name\":\"string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}},{\"name\":\"maybe_bool_column\",\"type\":[\"null\",\"boolean\"]},{\"name\":\"maybe_int_column\",\"type\":[\"null\",\"int\"]},{\"name\":\"maybe_long_column\",\"type\":[\"null\",\"long\"]},{\"name\":\"maybe_float_column\",\"type\":[\"null\",\"float\"]},{\"name\":\"maybe_double_column\",\"type\":[\"null\",\"double\"]},{\"name\":\"maybe_binary_column\",\"type\":[\"null\",\"bytes\"]},{\"name\":\"maybe_string_column\",\"type\":[\"null\",{\"type\":\"string\",\"avro.java.string\":\"String\"}]},{\"name\":\"strings_column\",\"type\":{\"type\":\"array\",\"items\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}},{\"name\":\"string_to_int_column\",\"type\":{\"type\":\"map\",\"values\":\"int\",\"avro.java.string\":\"String\"}},{\"name\":\"complex_column\",\"type\":{\"type\":\"map\",\"values\":{\"type\":\"array\",\"items\":{\"type\":\"record\",\"name\":\"Nested\",\"fields\":[{\"name\":\"nested_ints_column\",\"type\":{\"type\":\"array\",\"items\":\"int\"}},{\"name\":\"nested_string_column\",\"type\":{\"type\":\"string\",\"avro.java.string\":\"String\"}}]}},\"avro.java.string\":\"String\"}}]}"); + public static org.apache.avro.Schema getClassSchema() { return SCHEMA$; } + @Deprecated public boolean bool_column; + @Deprecated public int int_column; + @Deprecated public long long_column; + @Deprecated public float float_column; + @Deprecated public double double_column; + @Deprecated public java.nio.ByteBuffer binary_column; + @Deprecated public java.lang.String string_column; + @Deprecated public java.lang.Boolean maybe_bool_column; + @Deprecated public java.lang.Integer maybe_int_column; + @Deprecated public java.lang.Long maybe_long_column; + @Deprecated public java.lang.Float maybe_float_column; + @Deprecated public java.lang.Double maybe_double_column; + @Deprecated public java.nio.ByteBuffer maybe_binary_column; + @Deprecated public java.lang.String maybe_string_column; + @Deprecated public java.util.List strings_column; + @Deprecated public java.util.Map string_to_int_column; + @Deprecated public java.util.Map> complex_column; + + /** + * Default constructor. Note that this does not initialize fields + * to their default values from the schema. If that is desired then + * one should use newBuilder(). + */ + public ParquetAvroCompat() {} + + /** + * All-args constructor. + */ + public ParquetAvroCompat(java.lang.Boolean bool_column, java.lang.Integer int_column, java.lang.Long long_column, java.lang.Float float_column, java.lang.Double double_column, java.nio.ByteBuffer binary_column, java.lang.String string_column, java.lang.Boolean maybe_bool_column, java.lang.Integer maybe_int_column, java.lang.Long maybe_long_column, java.lang.Float maybe_float_column, java.lang.Double maybe_double_column, java.nio.ByteBuffer maybe_binary_column, java.lang.String maybe_string_column, java.util.List strings_column, java.util.Map string_to_int_column, java.util.Map> complex_column) { + this.bool_column = bool_column; + this.int_column = int_column; + this.long_column = long_column; + this.float_column = float_column; + this.double_column = double_column; + this.binary_column = binary_column; + this.string_column = string_column; + this.maybe_bool_column = maybe_bool_column; + this.maybe_int_column = maybe_int_column; + this.maybe_long_column = maybe_long_column; + this.maybe_float_column = maybe_float_column; + this.maybe_double_column = maybe_double_column; + this.maybe_binary_column = maybe_binary_column; + this.maybe_string_column = maybe_string_column; + this.strings_column = strings_column; + this.string_to_int_column = string_to_int_column; + this.complex_column = complex_column; + } + + public org.apache.avro.Schema getSchema() { return SCHEMA$; } + // Used by DatumWriter. Applications should not call. + public java.lang.Object get(int field$) { + switch (field$) { + case 0: return bool_column; + case 1: return int_column; + case 2: return long_column; + case 3: return float_column; + case 4: return double_column; + case 5: return binary_column; + case 6: return string_column; + case 7: return maybe_bool_column; + case 8: return maybe_int_column; + case 9: return maybe_long_column; + case 10: return maybe_float_column; + case 11: return maybe_double_column; + case 12: return maybe_binary_column; + case 13: return maybe_string_column; + case 14: return strings_column; + case 15: return string_to_int_column; + case 16: return complex_column; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + // Used by DatumReader. Applications should not call. + @SuppressWarnings(value="unchecked") + public void put(int field$, java.lang.Object value$) { + switch (field$) { + case 0: bool_column = (java.lang.Boolean)value$; break; + case 1: int_column = (java.lang.Integer)value$; break; + case 2: long_column = (java.lang.Long)value$; break; + case 3: float_column = (java.lang.Float)value$; break; + case 4: double_column = (java.lang.Double)value$; break; + case 5: binary_column = (java.nio.ByteBuffer)value$; break; + case 6: string_column = (java.lang.String)value$; break; + case 7: maybe_bool_column = (java.lang.Boolean)value$; break; + case 8: maybe_int_column = (java.lang.Integer)value$; break; + case 9: maybe_long_column = (java.lang.Long)value$; break; + case 10: maybe_float_column = (java.lang.Float)value$; break; + case 11: maybe_double_column = (java.lang.Double)value$; break; + case 12: maybe_binary_column = (java.nio.ByteBuffer)value$; break; + case 13: maybe_string_column = (java.lang.String)value$; break; + case 14: strings_column = (java.util.List)value$; break; + case 15: string_to_int_column = (java.util.Map)value$; break; + case 16: complex_column = (java.util.Map>)value$; break; + default: throw new org.apache.avro.AvroRuntimeException("Bad index"); + } + } + + /** + * Gets the value of the 'bool_column' field. + */ + public java.lang.Boolean getBoolColumn() { + return bool_column; + } + + /** + * Sets the value of the 'bool_column' field. + * @param value the value to set. + */ + public void setBoolColumn(java.lang.Boolean value) { + this.bool_column = value; + } + + /** + * Gets the value of the 'int_column' field. + */ + public java.lang.Integer getIntColumn() { + return int_column; + } + + /** + * Sets the value of the 'int_column' field. + * @param value the value to set. + */ + public void setIntColumn(java.lang.Integer value) { + this.int_column = value; + } + + /** + * Gets the value of the 'long_column' field. + */ + public java.lang.Long getLongColumn() { + return long_column; + } + + /** + * Sets the value of the 'long_column' field. + * @param value the value to set. + */ + public void setLongColumn(java.lang.Long value) { + this.long_column = value; + } + + /** + * Gets the value of the 'float_column' field. + */ + public java.lang.Float getFloatColumn() { + return float_column; + } + + /** + * Sets the value of the 'float_column' field. + * @param value the value to set. + */ + public void setFloatColumn(java.lang.Float value) { + this.float_column = value; + } + + /** + * Gets the value of the 'double_column' field. + */ + public java.lang.Double getDoubleColumn() { + return double_column; + } + + /** + * Sets the value of the 'double_column' field. + * @param value the value to set. + */ + public void setDoubleColumn(java.lang.Double value) { + this.double_column = value; + } + + /** + * Gets the value of the 'binary_column' field. + */ + public java.nio.ByteBuffer getBinaryColumn() { + return binary_column; + } + + /** + * Sets the value of the 'binary_column' field. + * @param value the value to set. + */ + public void setBinaryColumn(java.nio.ByteBuffer value) { + this.binary_column = value; + } + + /** + * Gets the value of the 'string_column' field. + */ + public java.lang.String getStringColumn() { + return string_column; + } + + /** + * Sets the value of the 'string_column' field. + * @param value the value to set. + */ + public void setStringColumn(java.lang.String value) { + this.string_column = value; + } + + /** + * Gets the value of the 'maybe_bool_column' field. + */ + public java.lang.Boolean getMaybeBoolColumn() { + return maybe_bool_column; + } + + /** + * Sets the value of the 'maybe_bool_column' field. + * @param value the value to set. + */ + public void setMaybeBoolColumn(java.lang.Boolean value) { + this.maybe_bool_column = value; + } + + /** + * Gets the value of the 'maybe_int_column' field. + */ + public java.lang.Integer getMaybeIntColumn() { + return maybe_int_column; + } + + /** + * Sets the value of the 'maybe_int_column' field. + * @param value the value to set. + */ + public void setMaybeIntColumn(java.lang.Integer value) { + this.maybe_int_column = value; + } + + /** + * Gets the value of the 'maybe_long_column' field. + */ + public java.lang.Long getMaybeLongColumn() { + return maybe_long_column; + } + + /** + * Sets the value of the 'maybe_long_column' field. + * @param value the value to set. + */ + public void setMaybeLongColumn(java.lang.Long value) { + this.maybe_long_column = value; + } + + /** + * Gets the value of the 'maybe_float_column' field. + */ + public java.lang.Float getMaybeFloatColumn() { + return maybe_float_column; + } + + /** + * Sets the value of the 'maybe_float_column' field. + * @param value the value to set. + */ + public void setMaybeFloatColumn(java.lang.Float value) { + this.maybe_float_column = value; + } + + /** + * Gets the value of the 'maybe_double_column' field. + */ + public java.lang.Double getMaybeDoubleColumn() { + return maybe_double_column; + } + + /** + * Sets the value of the 'maybe_double_column' field. + * @param value the value to set. + */ + public void setMaybeDoubleColumn(java.lang.Double value) { + this.maybe_double_column = value; + } + + /** + * Gets the value of the 'maybe_binary_column' field. + */ + public java.nio.ByteBuffer getMaybeBinaryColumn() { + return maybe_binary_column; + } + + /** + * Sets the value of the 'maybe_binary_column' field. + * @param value the value to set. + */ + public void setMaybeBinaryColumn(java.nio.ByteBuffer value) { + this.maybe_binary_column = value; + } + + /** + * Gets the value of the 'maybe_string_column' field. + */ + public java.lang.String getMaybeStringColumn() { + return maybe_string_column; + } + + /** + * Sets the value of the 'maybe_string_column' field. + * @param value the value to set. + */ + public void setMaybeStringColumn(java.lang.String value) { + this.maybe_string_column = value; + } + + /** + * Gets the value of the 'strings_column' field. + */ + public java.util.List getStringsColumn() { + return strings_column; + } + + /** + * Sets the value of the 'strings_column' field. + * @param value the value to set. + */ + public void setStringsColumn(java.util.List value) { + this.strings_column = value; + } + + /** + * Gets the value of the 'string_to_int_column' field. + */ + public java.util.Map getStringToIntColumn() { + return string_to_int_column; + } + + /** + * Sets the value of the 'string_to_int_column' field. + * @param value the value to set. + */ + public void setStringToIntColumn(java.util.Map value) { + this.string_to_int_column = value; + } + + /** + * Gets the value of the 'complex_column' field. + */ + public java.util.Map> getComplexColumn() { + return complex_column; + } + + /** + * Sets the value of the 'complex_column' field. + * @param value the value to set. + */ + public void setComplexColumn(java.util.Map> value) { + this.complex_column = value; + } + + /** Creates a new ParquetAvroCompat RecordBuilder */ + public static org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder newBuilder() { + return new org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder(); + } + + /** Creates a new ParquetAvroCompat RecordBuilder by copying an existing Builder */ + public static org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder newBuilder(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder other) { + return new org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder(other); + } + + /** Creates a new ParquetAvroCompat RecordBuilder by copying an existing ParquetAvroCompat instance */ + public static org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder newBuilder(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat other) { + return new org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder(other); + } + + /** + * RecordBuilder for ParquetAvroCompat instances. + */ + public static class Builder extends org.apache.avro.specific.SpecificRecordBuilderBase + implements org.apache.avro.data.RecordBuilder { + + private boolean bool_column; + private int int_column; + private long long_column; + private float float_column; + private double double_column; + private java.nio.ByteBuffer binary_column; + private java.lang.String string_column; + private java.lang.Boolean maybe_bool_column; + private java.lang.Integer maybe_int_column; + private java.lang.Long maybe_long_column; + private java.lang.Float maybe_float_column; + private java.lang.Double maybe_double_column; + private java.nio.ByteBuffer maybe_binary_column; + private java.lang.String maybe_string_column; + private java.util.List strings_column; + private java.util.Map string_to_int_column; + private java.util.Map> complex_column; + + /** Creates a new Builder */ + private Builder() { + super(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.SCHEMA$); + } + + /** Creates a Builder by copying an existing Builder */ + private Builder(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder other) { + super(other); + if (isValidValue(fields()[0], other.bool_column)) { + this.bool_column = data().deepCopy(fields()[0].schema(), other.bool_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int_column)) { + this.int_column = data().deepCopy(fields()[1].schema(), other.int_column); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.long_column)) { + this.long_column = data().deepCopy(fields()[2].schema(), other.long_column); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.float_column)) { + this.float_column = data().deepCopy(fields()[3].schema(), other.float_column); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.double_column)) { + this.double_column = data().deepCopy(fields()[4].schema(), other.double_column); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.binary_column)) { + this.binary_column = data().deepCopy(fields()[5].schema(), other.binary_column); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.string_column)) { + this.string_column = data().deepCopy(fields()[6].schema(), other.string_column); + fieldSetFlags()[6] = true; + } + if (isValidValue(fields()[7], other.maybe_bool_column)) { + this.maybe_bool_column = data().deepCopy(fields()[7].schema(), other.maybe_bool_column); + fieldSetFlags()[7] = true; + } + if (isValidValue(fields()[8], other.maybe_int_column)) { + this.maybe_int_column = data().deepCopy(fields()[8].schema(), other.maybe_int_column); + fieldSetFlags()[8] = true; + } + if (isValidValue(fields()[9], other.maybe_long_column)) { + this.maybe_long_column = data().deepCopy(fields()[9].schema(), other.maybe_long_column); + fieldSetFlags()[9] = true; + } + if (isValidValue(fields()[10], other.maybe_float_column)) { + this.maybe_float_column = data().deepCopy(fields()[10].schema(), other.maybe_float_column); + fieldSetFlags()[10] = true; + } + if (isValidValue(fields()[11], other.maybe_double_column)) { + this.maybe_double_column = data().deepCopy(fields()[11].schema(), other.maybe_double_column); + fieldSetFlags()[11] = true; + } + if (isValidValue(fields()[12], other.maybe_binary_column)) { + this.maybe_binary_column = data().deepCopy(fields()[12].schema(), other.maybe_binary_column); + fieldSetFlags()[12] = true; + } + if (isValidValue(fields()[13], other.maybe_string_column)) { + this.maybe_string_column = data().deepCopy(fields()[13].schema(), other.maybe_string_column); + fieldSetFlags()[13] = true; + } + if (isValidValue(fields()[14], other.strings_column)) { + this.strings_column = data().deepCopy(fields()[14].schema(), other.strings_column); + fieldSetFlags()[14] = true; + } + if (isValidValue(fields()[15], other.string_to_int_column)) { + this.string_to_int_column = data().deepCopy(fields()[15].schema(), other.string_to_int_column); + fieldSetFlags()[15] = true; + } + if (isValidValue(fields()[16], other.complex_column)) { + this.complex_column = data().deepCopy(fields()[16].schema(), other.complex_column); + fieldSetFlags()[16] = true; + } + } + + /** Creates a Builder by copying an existing ParquetAvroCompat instance */ + private Builder(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat other) { + super(org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.SCHEMA$); + if (isValidValue(fields()[0], other.bool_column)) { + this.bool_column = data().deepCopy(fields()[0].schema(), other.bool_column); + fieldSetFlags()[0] = true; + } + if (isValidValue(fields()[1], other.int_column)) { + this.int_column = data().deepCopy(fields()[1].schema(), other.int_column); + fieldSetFlags()[1] = true; + } + if (isValidValue(fields()[2], other.long_column)) { + this.long_column = data().deepCopy(fields()[2].schema(), other.long_column); + fieldSetFlags()[2] = true; + } + if (isValidValue(fields()[3], other.float_column)) { + this.float_column = data().deepCopy(fields()[3].schema(), other.float_column); + fieldSetFlags()[3] = true; + } + if (isValidValue(fields()[4], other.double_column)) { + this.double_column = data().deepCopy(fields()[4].schema(), other.double_column); + fieldSetFlags()[4] = true; + } + if (isValidValue(fields()[5], other.binary_column)) { + this.binary_column = data().deepCopy(fields()[5].schema(), other.binary_column); + fieldSetFlags()[5] = true; + } + if (isValidValue(fields()[6], other.string_column)) { + this.string_column = data().deepCopy(fields()[6].schema(), other.string_column); + fieldSetFlags()[6] = true; + } + if (isValidValue(fields()[7], other.maybe_bool_column)) { + this.maybe_bool_column = data().deepCopy(fields()[7].schema(), other.maybe_bool_column); + fieldSetFlags()[7] = true; + } + if (isValidValue(fields()[8], other.maybe_int_column)) { + this.maybe_int_column = data().deepCopy(fields()[8].schema(), other.maybe_int_column); + fieldSetFlags()[8] = true; + } + if (isValidValue(fields()[9], other.maybe_long_column)) { + this.maybe_long_column = data().deepCopy(fields()[9].schema(), other.maybe_long_column); + fieldSetFlags()[9] = true; + } + if (isValidValue(fields()[10], other.maybe_float_column)) { + this.maybe_float_column = data().deepCopy(fields()[10].schema(), other.maybe_float_column); + fieldSetFlags()[10] = true; + } + if (isValidValue(fields()[11], other.maybe_double_column)) { + this.maybe_double_column = data().deepCopy(fields()[11].schema(), other.maybe_double_column); + fieldSetFlags()[11] = true; + } + if (isValidValue(fields()[12], other.maybe_binary_column)) { + this.maybe_binary_column = data().deepCopy(fields()[12].schema(), other.maybe_binary_column); + fieldSetFlags()[12] = true; + } + if (isValidValue(fields()[13], other.maybe_string_column)) { + this.maybe_string_column = data().deepCopy(fields()[13].schema(), other.maybe_string_column); + fieldSetFlags()[13] = true; + } + if (isValidValue(fields()[14], other.strings_column)) { + this.strings_column = data().deepCopy(fields()[14].schema(), other.strings_column); + fieldSetFlags()[14] = true; + } + if (isValidValue(fields()[15], other.string_to_int_column)) { + this.string_to_int_column = data().deepCopy(fields()[15].schema(), other.string_to_int_column); + fieldSetFlags()[15] = true; + } + if (isValidValue(fields()[16], other.complex_column)) { + this.complex_column = data().deepCopy(fields()[16].schema(), other.complex_column); + fieldSetFlags()[16] = true; + } + } + + /** Gets the value of the 'bool_column' field */ + public java.lang.Boolean getBoolColumn() { + return bool_column; + } + + /** Sets the value of the 'bool_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setBoolColumn(boolean value) { + validate(fields()[0], value); + this.bool_column = value; + fieldSetFlags()[0] = true; + return this; + } + + /** Checks whether the 'bool_column' field has been set */ + public boolean hasBoolColumn() { + return fieldSetFlags()[0]; + } + + /** Clears the value of the 'bool_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearBoolColumn() { + fieldSetFlags()[0] = false; + return this; + } + + /** Gets the value of the 'int_column' field */ + public java.lang.Integer getIntColumn() { + return int_column; + } + + /** Sets the value of the 'int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setIntColumn(int value) { + validate(fields()[1], value); + this.int_column = value; + fieldSetFlags()[1] = true; + return this; + } + + /** Checks whether the 'int_column' field has been set */ + public boolean hasIntColumn() { + return fieldSetFlags()[1]; + } + + /** Clears the value of the 'int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearIntColumn() { + fieldSetFlags()[1] = false; + return this; + } + + /** Gets the value of the 'long_column' field */ + public java.lang.Long getLongColumn() { + return long_column; + } + + /** Sets the value of the 'long_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setLongColumn(long value) { + validate(fields()[2], value); + this.long_column = value; + fieldSetFlags()[2] = true; + return this; + } + + /** Checks whether the 'long_column' field has been set */ + public boolean hasLongColumn() { + return fieldSetFlags()[2]; + } + + /** Clears the value of the 'long_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearLongColumn() { + fieldSetFlags()[2] = false; + return this; + } + + /** Gets the value of the 'float_column' field */ + public java.lang.Float getFloatColumn() { + return float_column; + } + + /** Sets the value of the 'float_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setFloatColumn(float value) { + validate(fields()[3], value); + this.float_column = value; + fieldSetFlags()[3] = true; + return this; + } + + /** Checks whether the 'float_column' field has been set */ + public boolean hasFloatColumn() { + return fieldSetFlags()[3]; + } + + /** Clears the value of the 'float_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearFloatColumn() { + fieldSetFlags()[3] = false; + return this; + } + + /** Gets the value of the 'double_column' field */ + public java.lang.Double getDoubleColumn() { + return double_column; + } + + /** Sets the value of the 'double_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setDoubleColumn(double value) { + validate(fields()[4], value); + this.double_column = value; + fieldSetFlags()[4] = true; + return this; + } + + /** Checks whether the 'double_column' field has been set */ + public boolean hasDoubleColumn() { + return fieldSetFlags()[4]; + } + + /** Clears the value of the 'double_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearDoubleColumn() { + fieldSetFlags()[4] = false; + return this; + } + + /** Gets the value of the 'binary_column' field */ + public java.nio.ByteBuffer getBinaryColumn() { + return binary_column; + } + + /** Sets the value of the 'binary_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setBinaryColumn(java.nio.ByteBuffer value) { + validate(fields()[5], value); + this.binary_column = value; + fieldSetFlags()[5] = true; + return this; + } + + /** Checks whether the 'binary_column' field has been set */ + public boolean hasBinaryColumn() { + return fieldSetFlags()[5]; + } + + /** Clears the value of the 'binary_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearBinaryColumn() { + binary_column = null; + fieldSetFlags()[5] = false; + return this; + } + + /** Gets the value of the 'string_column' field */ + public java.lang.String getStringColumn() { + return string_column; + } + + /** Sets the value of the 'string_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setStringColumn(java.lang.String value) { + validate(fields()[6], value); + this.string_column = value; + fieldSetFlags()[6] = true; + return this; + } + + /** Checks whether the 'string_column' field has been set */ + public boolean hasStringColumn() { + return fieldSetFlags()[6]; + } + + /** Clears the value of the 'string_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearStringColumn() { + string_column = null; + fieldSetFlags()[6] = false; + return this; + } + + /** Gets the value of the 'maybe_bool_column' field */ + public java.lang.Boolean getMaybeBoolColumn() { + return maybe_bool_column; + } + + /** Sets the value of the 'maybe_bool_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeBoolColumn(java.lang.Boolean value) { + validate(fields()[7], value); + this.maybe_bool_column = value; + fieldSetFlags()[7] = true; + return this; + } + + /** Checks whether the 'maybe_bool_column' field has been set */ + public boolean hasMaybeBoolColumn() { + return fieldSetFlags()[7]; + } + + /** Clears the value of the 'maybe_bool_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeBoolColumn() { + maybe_bool_column = null; + fieldSetFlags()[7] = false; + return this; + } + + /** Gets the value of the 'maybe_int_column' field */ + public java.lang.Integer getMaybeIntColumn() { + return maybe_int_column; + } + + /** Sets the value of the 'maybe_int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeIntColumn(java.lang.Integer value) { + validate(fields()[8], value); + this.maybe_int_column = value; + fieldSetFlags()[8] = true; + return this; + } + + /** Checks whether the 'maybe_int_column' field has been set */ + public boolean hasMaybeIntColumn() { + return fieldSetFlags()[8]; + } + + /** Clears the value of the 'maybe_int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeIntColumn() { + maybe_int_column = null; + fieldSetFlags()[8] = false; + return this; + } + + /** Gets the value of the 'maybe_long_column' field */ + public java.lang.Long getMaybeLongColumn() { + return maybe_long_column; + } + + /** Sets the value of the 'maybe_long_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeLongColumn(java.lang.Long value) { + validate(fields()[9], value); + this.maybe_long_column = value; + fieldSetFlags()[9] = true; + return this; + } + + /** Checks whether the 'maybe_long_column' field has been set */ + public boolean hasMaybeLongColumn() { + return fieldSetFlags()[9]; + } + + /** Clears the value of the 'maybe_long_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeLongColumn() { + maybe_long_column = null; + fieldSetFlags()[9] = false; + return this; + } + + /** Gets the value of the 'maybe_float_column' field */ + public java.lang.Float getMaybeFloatColumn() { + return maybe_float_column; + } + + /** Sets the value of the 'maybe_float_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeFloatColumn(java.lang.Float value) { + validate(fields()[10], value); + this.maybe_float_column = value; + fieldSetFlags()[10] = true; + return this; + } + + /** Checks whether the 'maybe_float_column' field has been set */ + public boolean hasMaybeFloatColumn() { + return fieldSetFlags()[10]; + } + + /** Clears the value of the 'maybe_float_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeFloatColumn() { + maybe_float_column = null; + fieldSetFlags()[10] = false; + return this; + } + + /** Gets the value of the 'maybe_double_column' field */ + public java.lang.Double getMaybeDoubleColumn() { + return maybe_double_column; + } + + /** Sets the value of the 'maybe_double_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeDoubleColumn(java.lang.Double value) { + validate(fields()[11], value); + this.maybe_double_column = value; + fieldSetFlags()[11] = true; + return this; + } + + /** Checks whether the 'maybe_double_column' field has been set */ + public boolean hasMaybeDoubleColumn() { + return fieldSetFlags()[11]; + } + + /** Clears the value of the 'maybe_double_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeDoubleColumn() { + maybe_double_column = null; + fieldSetFlags()[11] = false; + return this; + } + + /** Gets the value of the 'maybe_binary_column' field */ + public java.nio.ByteBuffer getMaybeBinaryColumn() { + return maybe_binary_column; + } + + /** Sets the value of the 'maybe_binary_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeBinaryColumn(java.nio.ByteBuffer value) { + validate(fields()[12], value); + this.maybe_binary_column = value; + fieldSetFlags()[12] = true; + return this; + } + + /** Checks whether the 'maybe_binary_column' field has been set */ + public boolean hasMaybeBinaryColumn() { + return fieldSetFlags()[12]; + } + + /** Clears the value of the 'maybe_binary_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeBinaryColumn() { + maybe_binary_column = null; + fieldSetFlags()[12] = false; + return this; + } + + /** Gets the value of the 'maybe_string_column' field */ + public java.lang.String getMaybeStringColumn() { + return maybe_string_column; + } + + /** Sets the value of the 'maybe_string_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setMaybeStringColumn(java.lang.String value) { + validate(fields()[13], value); + this.maybe_string_column = value; + fieldSetFlags()[13] = true; + return this; + } + + /** Checks whether the 'maybe_string_column' field has been set */ + public boolean hasMaybeStringColumn() { + return fieldSetFlags()[13]; + } + + /** Clears the value of the 'maybe_string_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearMaybeStringColumn() { + maybe_string_column = null; + fieldSetFlags()[13] = false; + return this; + } + + /** Gets the value of the 'strings_column' field */ + public java.util.List getStringsColumn() { + return strings_column; + } + + /** Sets the value of the 'strings_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setStringsColumn(java.util.List value) { + validate(fields()[14], value); + this.strings_column = value; + fieldSetFlags()[14] = true; + return this; + } + + /** Checks whether the 'strings_column' field has been set */ + public boolean hasStringsColumn() { + return fieldSetFlags()[14]; + } + + /** Clears the value of the 'strings_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearStringsColumn() { + strings_column = null; + fieldSetFlags()[14] = false; + return this; + } + + /** Gets the value of the 'string_to_int_column' field */ + public java.util.Map getStringToIntColumn() { + return string_to_int_column; + } + + /** Sets the value of the 'string_to_int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setStringToIntColumn(java.util.Map value) { + validate(fields()[15], value); + this.string_to_int_column = value; + fieldSetFlags()[15] = true; + return this; + } + + /** Checks whether the 'string_to_int_column' field has been set */ + public boolean hasStringToIntColumn() { + return fieldSetFlags()[15]; + } + + /** Clears the value of the 'string_to_int_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearStringToIntColumn() { + string_to_int_column = null; + fieldSetFlags()[15] = false; + return this; + } + + /** Gets the value of the 'complex_column' field */ + public java.util.Map> getComplexColumn() { + return complex_column; + } + + /** Sets the value of the 'complex_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder setComplexColumn(java.util.Map> value) { + validate(fields()[16], value); + this.complex_column = value; + fieldSetFlags()[16] = true; + return this; + } + + /** Checks whether the 'complex_column' field has been set */ + public boolean hasComplexColumn() { + return fieldSetFlags()[16]; + } + + /** Clears the value of the 'complex_column' field */ + public org.apache.spark.sql.parquet.test.avro.ParquetAvroCompat.Builder clearComplexColumn() { + complex_column = null; + fieldSetFlags()[16] = false; + return this; + } + + @Override + public ParquetAvroCompat build() { + try { + ParquetAvroCompat record = new ParquetAvroCompat(); + record.bool_column = fieldSetFlags()[0] ? this.bool_column : (java.lang.Boolean) defaultValue(fields()[0]); + record.int_column = fieldSetFlags()[1] ? this.int_column : (java.lang.Integer) defaultValue(fields()[1]); + record.long_column = fieldSetFlags()[2] ? this.long_column : (java.lang.Long) defaultValue(fields()[2]); + record.float_column = fieldSetFlags()[3] ? this.float_column : (java.lang.Float) defaultValue(fields()[3]); + record.double_column = fieldSetFlags()[4] ? this.double_column : (java.lang.Double) defaultValue(fields()[4]); + record.binary_column = fieldSetFlags()[5] ? this.binary_column : (java.nio.ByteBuffer) defaultValue(fields()[5]); + record.string_column = fieldSetFlags()[6] ? this.string_column : (java.lang.String) defaultValue(fields()[6]); + record.maybe_bool_column = fieldSetFlags()[7] ? this.maybe_bool_column : (java.lang.Boolean) defaultValue(fields()[7]); + record.maybe_int_column = fieldSetFlags()[8] ? this.maybe_int_column : (java.lang.Integer) defaultValue(fields()[8]); + record.maybe_long_column = fieldSetFlags()[9] ? this.maybe_long_column : (java.lang.Long) defaultValue(fields()[9]); + record.maybe_float_column = fieldSetFlags()[10] ? this.maybe_float_column : (java.lang.Float) defaultValue(fields()[10]); + record.maybe_double_column = fieldSetFlags()[11] ? this.maybe_double_column : (java.lang.Double) defaultValue(fields()[11]); + record.maybe_binary_column = fieldSetFlags()[12] ? this.maybe_binary_column : (java.nio.ByteBuffer) defaultValue(fields()[12]); + record.maybe_string_column = fieldSetFlags()[13] ? this.maybe_string_column : (java.lang.String) defaultValue(fields()[13]); + record.strings_column = fieldSetFlags()[14] ? this.strings_column : (java.util.List) defaultValue(fields()[14]); + record.string_to_int_column = fieldSetFlags()[15] ? this.string_to_int_column : (java.util.Map) defaultValue(fields()[15]); + record.complex_column = fieldSetFlags()[16] ? this.complex_column : (java.util.Map>) defaultValue(fields()[16]); + return record; + } catch (Exception e) { + throw new org.apache.avro.AvroRuntimeException(e); + } + } + } +} diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Nested.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Nested.java new file mode 100644 index 0000000000000..281e60cc3ae34 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Nested.java @@ -0,0 +1,541 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.spark.sql.parquet.test.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-7") +public class Nested implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("Nested"); + + private static final org.apache.thrift.protocol.TField NESTED_INTS_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("nestedIntsColumn", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NESTED_STRING_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("nestedStringColumn", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new NestedStandardSchemeFactory()); + schemes.put(TupleScheme.class, new NestedTupleSchemeFactory()); + } + + public List nestedIntsColumn; // required + public String nestedStringColumn; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NESTED_INTS_COLUMN((short)1, "nestedIntsColumn"), + NESTED_STRING_COLUMN((short)2, "nestedStringColumn"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NESTED_INTS_COLUMN + return NESTED_INTS_COLUMN; + case 2: // NESTED_STRING_COLUMN + return NESTED_STRING_COLUMN; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NESTED_INTS_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("nestedIntsColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)))); + tmpMap.put(_Fields.NESTED_STRING_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("nestedStringColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(Nested.class, metaDataMap); + } + + public Nested() { + } + + public Nested( + List nestedIntsColumn, + String nestedStringColumn) + { + this(); + this.nestedIntsColumn = nestedIntsColumn; + this.nestedStringColumn = nestedStringColumn; + } + + /** + * Performs a deep copy on other. + */ + public Nested(Nested other) { + if (other.isSetNestedIntsColumn()) { + List __this__nestedIntsColumn = new ArrayList(other.nestedIntsColumn); + this.nestedIntsColumn = __this__nestedIntsColumn; + } + if (other.isSetNestedStringColumn()) { + this.nestedStringColumn = other.nestedStringColumn; + } + } + + public Nested deepCopy() { + return new Nested(this); + } + + @Override + public void clear() { + this.nestedIntsColumn = null; + this.nestedStringColumn = null; + } + + public int getNestedIntsColumnSize() { + return (this.nestedIntsColumn == null) ? 0 : this.nestedIntsColumn.size(); + } + + public java.util.Iterator getNestedIntsColumnIterator() { + return (this.nestedIntsColumn == null) ? null : this.nestedIntsColumn.iterator(); + } + + public void addToNestedIntsColumn(int elem) { + if (this.nestedIntsColumn == null) { + this.nestedIntsColumn = new ArrayList(); + } + this.nestedIntsColumn.add(elem); + } + + public List getNestedIntsColumn() { + return this.nestedIntsColumn; + } + + public Nested setNestedIntsColumn(List nestedIntsColumn) { + this.nestedIntsColumn = nestedIntsColumn; + return this; + } + + public void unsetNestedIntsColumn() { + this.nestedIntsColumn = null; + } + + /** Returns true if field nestedIntsColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetNestedIntsColumn() { + return this.nestedIntsColumn != null; + } + + public void setNestedIntsColumnIsSet(boolean value) { + if (!value) { + this.nestedIntsColumn = null; + } + } + + public String getNestedStringColumn() { + return this.nestedStringColumn; + } + + public Nested setNestedStringColumn(String nestedStringColumn) { + this.nestedStringColumn = nestedStringColumn; + return this; + } + + public void unsetNestedStringColumn() { + this.nestedStringColumn = null; + } + + /** Returns true if field nestedStringColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetNestedStringColumn() { + return this.nestedStringColumn != null; + } + + public void setNestedStringColumnIsSet(boolean value) { + if (!value) { + this.nestedStringColumn = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NESTED_INTS_COLUMN: + if (value == null) { + unsetNestedIntsColumn(); + } else { + setNestedIntsColumn((List)value); + } + break; + + case NESTED_STRING_COLUMN: + if (value == null) { + unsetNestedStringColumn(); + } else { + setNestedStringColumn((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NESTED_INTS_COLUMN: + return getNestedIntsColumn(); + + case NESTED_STRING_COLUMN: + return getNestedStringColumn(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NESTED_INTS_COLUMN: + return isSetNestedIntsColumn(); + case NESTED_STRING_COLUMN: + return isSetNestedStringColumn(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof Nested) + return this.equals((Nested)that); + return false; + } + + public boolean equals(Nested that) { + if (that == null) + return false; + + boolean this_present_nestedIntsColumn = true && this.isSetNestedIntsColumn(); + boolean that_present_nestedIntsColumn = true && that.isSetNestedIntsColumn(); + if (this_present_nestedIntsColumn || that_present_nestedIntsColumn) { + if (!(this_present_nestedIntsColumn && that_present_nestedIntsColumn)) + return false; + if (!this.nestedIntsColumn.equals(that.nestedIntsColumn)) + return false; + } + + boolean this_present_nestedStringColumn = true && this.isSetNestedStringColumn(); + boolean that_present_nestedStringColumn = true && that.isSetNestedStringColumn(); + if (this_present_nestedStringColumn || that_present_nestedStringColumn) { + if (!(this_present_nestedStringColumn && that_present_nestedStringColumn)) + return false; + if (!this.nestedStringColumn.equals(that.nestedStringColumn)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_nestedIntsColumn = true && (isSetNestedIntsColumn()); + list.add(present_nestedIntsColumn); + if (present_nestedIntsColumn) + list.add(nestedIntsColumn); + + boolean present_nestedStringColumn = true && (isSetNestedStringColumn()); + list.add(present_nestedStringColumn); + if (present_nestedStringColumn) + list.add(nestedStringColumn); + + return list.hashCode(); + } + + @Override + public int compareTo(Nested other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetNestedIntsColumn()).compareTo(other.isSetNestedIntsColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNestedIntsColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nestedIntsColumn, other.nestedIntsColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNestedStringColumn()).compareTo(other.isSetNestedStringColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNestedStringColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nestedStringColumn, other.nestedStringColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("Nested("); + boolean first = true; + + sb.append("nestedIntsColumn:"); + if (this.nestedIntsColumn == null) { + sb.append("null"); + } else { + sb.append(this.nestedIntsColumn); + } + first = false; + if (!first) sb.append(", "); + sb.append("nestedStringColumn:"); + if (this.nestedStringColumn == null) { + sb.append("null"); + } else { + sb.append(this.nestedStringColumn); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (nestedIntsColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nestedIntsColumn' was not present! Struct: " + toString()); + } + if (nestedStringColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nestedStringColumn' was not present! Struct: " + toString()); + } + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class NestedStandardSchemeFactory implements SchemeFactory { + public NestedStandardScheme getScheme() { + return new NestedStandardScheme(); + } + } + + private static class NestedStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, Nested struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NESTED_INTS_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list0 = iprot.readListBegin(); + struct.nestedIntsColumn = new ArrayList(_list0.size); + int _elem1; + for (int _i2 = 0; _i2 < _list0.size; ++_i2) + { + _elem1 = iprot.readI32(); + struct.nestedIntsColumn.add(_elem1); + } + iprot.readListEnd(); + } + struct.setNestedIntsColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NESTED_STRING_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nestedStringColumn = iprot.readString(); + struct.setNestedStringColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + + // check for required fields of primitive type, which can't be checked in the validate method + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, Nested struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.nestedIntsColumn != null) { + oprot.writeFieldBegin(NESTED_INTS_COLUMN_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.nestedIntsColumn.size())); + for (int _iter3 : struct.nestedIntsColumn) + { + oprot.writeI32(_iter3); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nestedStringColumn != null) { + oprot.writeFieldBegin(NESTED_STRING_COLUMN_FIELD_DESC); + oprot.writeString(struct.nestedStringColumn); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class NestedTupleSchemeFactory implements SchemeFactory { + public NestedTupleScheme getScheme() { + return new NestedTupleScheme(); + } + } + + private static class NestedTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, Nested struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.nestedIntsColumn.size()); + for (int _iter4 : struct.nestedIntsColumn) + { + oprot.writeI32(_iter4); + } + } + oprot.writeString(struct.nestedStringColumn); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, Nested struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list5 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.nestedIntsColumn = new ArrayList(_list5.size); + int _elem6; + for (int _i7 = 0; _i7 < _list5.size; ++_i7) + { + _elem6 = iprot.readI32(); + struct.nestedIntsColumn.add(_elem6); + } + } + struct.setNestedIntsColumnIsSet(true); + struct.nestedStringColumn = iprot.readString(); + struct.setNestedStringColumnIsSet(true); + } + } + +} + diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/ParquetThriftCompat.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/ParquetThriftCompat.java new file mode 100644 index 0000000000000..326ae9dbaa0d1 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/ParquetThriftCompat.java @@ -0,0 +1,2808 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.spark.sql.parquet.test.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +/** + * This is a test struct for testing parquet-thrift compatibility. + */ +@Generated(value = "Autogenerated by Thrift Compiler (0.9.2)", date = "2015-7-7") +public class ParquetThriftCompat implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ParquetThriftCompat"); + + private static final org.apache.thrift.protocol.TField BOOL_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("boolColumn", org.apache.thrift.protocol.TType.BOOL, (short)1); + private static final org.apache.thrift.protocol.TField BYTE_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("byteColumn", org.apache.thrift.protocol.TType.BYTE, (short)2); + private static final org.apache.thrift.protocol.TField SHORT_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("shortColumn", org.apache.thrift.protocol.TType.I16, (short)3); + private static final org.apache.thrift.protocol.TField INT_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("intColumn", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField LONG_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("longColumn", org.apache.thrift.protocol.TType.I64, (short)5); + private static final org.apache.thrift.protocol.TField DOUBLE_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("doubleColumn", org.apache.thrift.protocol.TType.DOUBLE, (short)6); + private static final org.apache.thrift.protocol.TField BINARY_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryColumn", org.apache.thrift.protocol.TType.STRING, (short)7); + private static final org.apache.thrift.protocol.TField STRING_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("stringColumn", org.apache.thrift.protocol.TType.STRING, (short)8); + private static final org.apache.thrift.protocol.TField ENUM_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("enumColumn", org.apache.thrift.protocol.TType.I32, (short)9); + private static final org.apache.thrift.protocol.TField MAYBE_BOOL_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeBoolColumn", org.apache.thrift.protocol.TType.BOOL, (short)10); + private static final org.apache.thrift.protocol.TField MAYBE_BYTE_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeByteColumn", org.apache.thrift.protocol.TType.BYTE, (short)11); + private static final org.apache.thrift.protocol.TField MAYBE_SHORT_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeShortColumn", org.apache.thrift.protocol.TType.I16, (short)12); + private static final org.apache.thrift.protocol.TField MAYBE_INT_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeIntColumn", org.apache.thrift.protocol.TType.I32, (short)13); + private static final org.apache.thrift.protocol.TField MAYBE_LONG_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeLongColumn", org.apache.thrift.protocol.TType.I64, (short)14); + private static final org.apache.thrift.protocol.TField MAYBE_DOUBLE_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeDoubleColumn", org.apache.thrift.protocol.TType.DOUBLE, (short)15); + private static final org.apache.thrift.protocol.TField MAYBE_BINARY_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeBinaryColumn", org.apache.thrift.protocol.TType.STRING, (short)16); + private static final org.apache.thrift.protocol.TField MAYBE_STRING_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeStringColumn", org.apache.thrift.protocol.TType.STRING, (short)17); + private static final org.apache.thrift.protocol.TField MAYBE_ENUM_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("maybeEnumColumn", org.apache.thrift.protocol.TType.I32, (short)18); + private static final org.apache.thrift.protocol.TField STRINGS_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("stringsColumn", org.apache.thrift.protocol.TType.LIST, (short)19); + private static final org.apache.thrift.protocol.TField INT_SET_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("intSetColumn", org.apache.thrift.protocol.TType.SET, (short)20); + private static final org.apache.thrift.protocol.TField INT_TO_STRING_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("intToStringColumn", org.apache.thrift.protocol.TType.MAP, (short)21); + private static final org.apache.thrift.protocol.TField COMPLEX_COLUMN_FIELD_DESC = new org.apache.thrift.protocol.TField("complexColumn", org.apache.thrift.protocol.TType.MAP, (short)22); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ParquetThriftCompatStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ParquetThriftCompatTupleSchemeFactory()); + } + + public boolean boolColumn; // required + public byte byteColumn; // required + public short shortColumn; // required + public int intColumn; // required + public long longColumn; // required + public double doubleColumn; // required + public ByteBuffer binaryColumn; // required + public String stringColumn; // required + /** + * + * @see Suit + */ + public Suit enumColumn; // required + public boolean maybeBoolColumn; // optional + public byte maybeByteColumn; // optional + public short maybeShortColumn; // optional + public int maybeIntColumn; // optional + public long maybeLongColumn; // optional + public double maybeDoubleColumn; // optional + public ByteBuffer maybeBinaryColumn; // optional + public String maybeStringColumn; // optional + /** + * + * @see Suit + */ + public Suit maybeEnumColumn; // optional + public List stringsColumn; // required + public Set intSetColumn; // required + public Map intToStringColumn; // required + public Map> complexColumn; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + BOOL_COLUMN((short)1, "boolColumn"), + BYTE_COLUMN((short)2, "byteColumn"), + SHORT_COLUMN((short)3, "shortColumn"), + INT_COLUMN((short)4, "intColumn"), + LONG_COLUMN((short)5, "longColumn"), + DOUBLE_COLUMN((short)6, "doubleColumn"), + BINARY_COLUMN((short)7, "binaryColumn"), + STRING_COLUMN((short)8, "stringColumn"), + /** + * + * @see Suit + */ + ENUM_COLUMN((short)9, "enumColumn"), + MAYBE_BOOL_COLUMN((short)10, "maybeBoolColumn"), + MAYBE_BYTE_COLUMN((short)11, "maybeByteColumn"), + MAYBE_SHORT_COLUMN((short)12, "maybeShortColumn"), + MAYBE_INT_COLUMN((short)13, "maybeIntColumn"), + MAYBE_LONG_COLUMN((short)14, "maybeLongColumn"), + MAYBE_DOUBLE_COLUMN((short)15, "maybeDoubleColumn"), + MAYBE_BINARY_COLUMN((short)16, "maybeBinaryColumn"), + MAYBE_STRING_COLUMN((short)17, "maybeStringColumn"), + /** + * + * @see Suit + */ + MAYBE_ENUM_COLUMN((short)18, "maybeEnumColumn"), + STRINGS_COLUMN((short)19, "stringsColumn"), + INT_SET_COLUMN((short)20, "intSetColumn"), + INT_TO_STRING_COLUMN((short)21, "intToStringColumn"), + COMPLEX_COLUMN((short)22, "complexColumn"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // BOOL_COLUMN + return BOOL_COLUMN; + case 2: // BYTE_COLUMN + return BYTE_COLUMN; + case 3: // SHORT_COLUMN + return SHORT_COLUMN; + case 4: // INT_COLUMN + return INT_COLUMN; + case 5: // LONG_COLUMN + return LONG_COLUMN; + case 6: // DOUBLE_COLUMN + return DOUBLE_COLUMN; + case 7: // BINARY_COLUMN + return BINARY_COLUMN; + case 8: // STRING_COLUMN + return STRING_COLUMN; + case 9: // ENUM_COLUMN + return ENUM_COLUMN; + case 10: // MAYBE_BOOL_COLUMN + return MAYBE_BOOL_COLUMN; + case 11: // MAYBE_BYTE_COLUMN + return MAYBE_BYTE_COLUMN; + case 12: // MAYBE_SHORT_COLUMN + return MAYBE_SHORT_COLUMN; + case 13: // MAYBE_INT_COLUMN + return MAYBE_INT_COLUMN; + case 14: // MAYBE_LONG_COLUMN + return MAYBE_LONG_COLUMN; + case 15: // MAYBE_DOUBLE_COLUMN + return MAYBE_DOUBLE_COLUMN; + case 16: // MAYBE_BINARY_COLUMN + return MAYBE_BINARY_COLUMN; + case 17: // MAYBE_STRING_COLUMN + return MAYBE_STRING_COLUMN; + case 18: // MAYBE_ENUM_COLUMN + return MAYBE_ENUM_COLUMN; + case 19: // STRINGS_COLUMN + return STRINGS_COLUMN; + case 20: // INT_SET_COLUMN + return INT_SET_COLUMN; + case 21: // INT_TO_STRING_COLUMN + return INT_TO_STRING_COLUMN; + case 22: // COMPLEX_COLUMN + return COMPLEX_COLUMN; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __BOOLCOLUMN_ISSET_ID = 0; + private static final int __BYTECOLUMN_ISSET_ID = 1; + private static final int __SHORTCOLUMN_ISSET_ID = 2; + private static final int __INTCOLUMN_ISSET_ID = 3; + private static final int __LONGCOLUMN_ISSET_ID = 4; + private static final int __DOUBLECOLUMN_ISSET_ID = 5; + private static final int __MAYBEBOOLCOLUMN_ISSET_ID = 6; + private static final int __MAYBEBYTECOLUMN_ISSET_ID = 7; + private static final int __MAYBESHORTCOLUMN_ISSET_ID = 8; + private static final int __MAYBEINTCOLUMN_ISSET_ID = 9; + private static final int __MAYBELONGCOLUMN_ISSET_ID = 10; + private static final int __MAYBEDOUBLECOLUMN_ISSET_ID = 11; + private short __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.MAYBE_BOOL_COLUMN,_Fields.MAYBE_BYTE_COLUMN,_Fields.MAYBE_SHORT_COLUMN,_Fields.MAYBE_INT_COLUMN,_Fields.MAYBE_LONG_COLUMN,_Fields.MAYBE_DOUBLE_COLUMN,_Fields.MAYBE_BINARY_COLUMN,_Fields.MAYBE_STRING_COLUMN,_Fields.MAYBE_ENUM_COLUMN}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.BOOL_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("boolColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.BYTE_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("byteColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE))); + tmpMap.put(_Fields.SHORT_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("shortColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); + tmpMap.put(_Fields.INT_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("intColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.LONG_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("longColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.DOUBLE_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("doubleColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + tmpMap.put(_Fields.BINARY_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("binaryColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.STRING_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("stringColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.ENUM_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("enumColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, Suit.class))); + tmpMap.put(_Fields.MAYBE_BOOL_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeBoolColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.MAYBE_BYTE_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeByteColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE))); + tmpMap.put(_Fields.MAYBE_SHORT_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeShortColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); + tmpMap.put(_Fields.MAYBE_INT_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeIntColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.MAYBE_LONG_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeLongColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.MAYBE_DOUBLE_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeDoubleColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + tmpMap.put(_Fields.MAYBE_BINARY_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeBinaryColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.MAYBE_STRING_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeStringColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.MAYBE_ENUM_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("maybeEnumColumn", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, Suit.class))); + tmpMap.put(_Fields.STRINGS_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("stringsColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.INT_SET_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("intSetColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.SetMetaData(org.apache.thrift.protocol.TType.SET, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)))); + tmpMap.put(_Fields.INT_TO_STRING_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("intToStringColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.COMPLEX_COLUMN, new org.apache.thrift.meta_data.FieldMetaData("complexColumn", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32), + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, Nested.class))))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ParquetThriftCompat.class, metaDataMap); + } + + public ParquetThriftCompat() { + } + + public ParquetThriftCompat( + boolean boolColumn, + byte byteColumn, + short shortColumn, + int intColumn, + long longColumn, + double doubleColumn, + ByteBuffer binaryColumn, + String stringColumn, + Suit enumColumn, + List stringsColumn, + Set intSetColumn, + Map intToStringColumn, + Map> complexColumn) + { + this(); + this.boolColumn = boolColumn; + setBoolColumnIsSet(true); + this.byteColumn = byteColumn; + setByteColumnIsSet(true); + this.shortColumn = shortColumn; + setShortColumnIsSet(true); + this.intColumn = intColumn; + setIntColumnIsSet(true); + this.longColumn = longColumn; + setLongColumnIsSet(true); + this.doubleColumn = doubleColumn; + setDoubleColumnIsSet(true); + this.binaryColumn = org.apache.thrift.TBaseHelper.copyBinary(binaryColumn); + this.stringColumn = stringColumn; + this.enumColumn = enumColumn; + this.stringsColumn = stringsColumn; + this.intSetColumn = intSetColumn; + this.intToStringColumn = intToStringColumn; + this.complexColumn = complexColumn; + } + + /** + * Performs a deep copy on other. + */ + public ParquetThriftCompat(ParquetThriftCompat other) { + __isset_bitfield = other.__isset_bitfield; + this.boolColumn = other.boolColumn; + this.byteColumn = other.byteColumn; + this.shortColumn = other.shortColumn; + this.intColumn = other.intColumn; + this.longColumn = other.longColumn; + this.doubleColumn = other.doubleColumn; + if (other.isSetBinaryColumn()) { + this.binaryColumn = org.apache.thrift.TBaseHelper.copyBinary(other.binaryColumn); + } + if (other.isSetStringColumn()) { + this.stringColumn = other.stringColumn; + } + if (other.isSetEnumColumn()) { + this.enumColumn = other.enumColumn; + } + this.maybeBoolColumn = other.maybeBoolColumn; + this.maybeByteColumn = other.maybeByteColumn; + this.maybeShortColumn = other.maybeShortColumn; + this.maybeIntColumn = other.maybeIntColumn; + this.maybeLongColumn = other.maybeLongColumn; + this.maybeDoubleColumn = other.maybeDoubleColumn; + if (other.isSetMaybeBinaryColumn()) { + this.maybeBinaryColumn = org.apache.thrift.TBaseHelper.copyBinary(other.maybeBinaryColumn); + } + if (other.isSetMaybeStringColumn()) { + this.maybeStringColumn = other.maybeStringColumn; + } + if (other.isSetMaybeEnumColumn()) { + this.maybeEnumColumn = other.maybeEnumColumn; + } + if (other.isSetStringsColumn()) { + List __this__stringsColumn = new ArrayList(other.stringsColumn); + this.stringsColumn = __this__stringsColumn; + } + if (other.isSetIntSetColumn()) { + Set __this__intSetColumn = new HashSet(other.intSetColumn); + this.intSetColumn = __this__intSetColumn; + } + if (other.isSetIntToStringColumn()) { + Map __this__intToStringColumn = new HashMap(other.intToStringColumn); + this.intToStringColumn = __this__intToStringColumn; + } + if (other.isSetComplexColumn()) { + Map> __this__complexColumn = new HashMap>(other.complexColumn.size()); + for (Map.Entry> other_element : other.complexColumn.entrySet()) { + + Integer other_element_key = other_element.getKey(); + List other_element_value = other_element.getValue(); + + Integer __this__complexColumn_copy_key = other_element_key; + + List __this__complexColumn_copy_value = new ArrayList(other_element_value.size()); + for (Nested other_element_value_element : other_element_value) { + __this__complexColumn_copy_value.add(new Nested(other_element_value_element)); + } + + __this__complexColumn.put(__this__complexColumn_copy_key, __this__complexColumn_copy_value); + } + this.complexColumn = __this__complexColumn; + } + } + + public ParquetThriftCompat deepCopy() { + return new ParquetThriftCompat(this); + } + + @Override + public void clear() { + setBoolColumnIsSet(false); + this.boolColumn = false; + setByteColumnIsSet(false); + this.byteColumn = 0; + setShortColumnIsSet(false); + this.shortColumn = 0; + setIntColumnIsSet(false); + this.intColumn = 0; + setLongColumnIsSet(false); + this.longColumn = 0; + setDoubleColumnIsSet(false); + this.doubleColumn = 0.0; + this.binaryColumn = null; + this.stringColumn = null; + this.enumColumn = null; + setMaybeBoolColumnIsSet(false); + this.maybeBoolColumn = false; + setMaybeByteColumnIsSet(false); + this.maybeByteColumn = 0; + setMaybeShortColumnIsSet(false); + this.maybeShortColumn = 0; + setMaybeIntColumnIsSet(false); + this.maybeIntColumn = 0; + setMaybeLongColumnIsSet(false); + this.maybeLongColumn = 0; + setMaybeDoubleColumnIsSet(false); + this.maybeDoubleColumn = 0.0; + this.maybeBinaryColumn = null; + this.maybeStringColumn = null; + this.maybeEnumColumn = null; + this.stringsColumn = null; + this.intSetColumn = null; + this.intToStringColumn = null; + this.complexColumn = null; + } + + public boolean isBoolColumn() { + return this.boolColumn; + } + + public ParquetThriftCompat setBoolColumn(boolean boolColumn) { + this.boolColumn = boolColumn; + setBoolColumnIsSet(true); + return this; + } + + public void unsetBoolColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __BOOLCOLUMN_ISSET_ID); + } + + /** Returns true if field boolColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetBoolColumn() { + return EncodingUtils.testBit(__isset_bitfield, __BOOLCOLUMN_ISSET_ID); + } + + public void setBoolColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __BOOLCOLUMN_ISSET_ID, value); + } + + public byte getByteColumn() { + return this.byteColumn; + } + + public ParquetThriftCompat setByteColumn(byte byteColumn) { + this.byteColumn = byteColumn; + setByteColumnIsSet(true); + return this; + } + + public void unsetByteColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __BYTECOLUMN_ISSET_ID); + } + + /** Returns true if field byteColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetByteColumn() { + return EncodingUtils.testBit(__isset_bitfield, __BYTECOLUMN_ISSET_ID); + } + + public void setByteColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __BYTECOLUMN_ISSET_ID, value); + } + + public short getShortColumn() { + return this.shortColumn; + } + + public ParquetThriftCompat setShortColumn(short shortColumn) { + this.shortColumn = shortColumn; + setShortColumnIsSet(true); + return this; + } + + public void unsetShortColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __SHORTCOLUMN_ISSET_ID); + } + + /** Returns true if field shortColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetShortColumn() { + return EncodingUtils.testBit(__isset_bitfield, __SHORTCOLUMN_ISSET_ID); + } + + public void setShortColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __SHORTCOLUMN_ISSET_ID, value); + } + + public int getIntColumn() { + return this.intColumn; + } + + public ParquetThriftCompat setIntColumn(int intColumn) { + this.intColumn = intColumn; + setIntColumnIsSet(true); + return this; + } + + public void unsetIntColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __INTCOLUMN_ISSET_ID); + } + + /** Returns true if field intColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetIntColumn() { + return EncodingUtils.testBit(__isset_bitfield, __INTCOLUMN_ISSET_ID); + } + + public void setIntColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __INTCOLUMN_ISSET_ID, value); + } + + public long getLongColumn() { + return this.longColumn; + } + + public ParquetThriftCompat setLongColumn(long longColumn) { + this.longColumn = longColumn; + setLongColumnIsSet(true); + return this; + } + + public void unsetLongColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __LONGCOLUMN_ISSET_ID); + } + + /** Returns true if field longColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetLongColumn() { + return EncodingUtils.testBit(__isset_bitfield, __LONGCOLUMN_ISSET_ID); + } + + public void setLongColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __LONGCOLUMN_ISSET_ID, value); + } + + public double getDoubleColumn() { + return this.doubleColumn; + } + + public ParquetThriftCompat setDoubleColumn(double doubleColumn) { + this.doubleColumn = doubleColumn; + setDoubleColumnIsSet(true); + return this; + } + + public void unsetDoubleColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __DOUBLECOLUMN_ISSET_ID); + } + + /** Returns true if field doubleColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetDoubleColumn() { + return EncodingUtils.testBit(__isset_bitfield, __DOUBLECOLUMN_ISSET_ID); + } + + public void setDoubleColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __DOUBLECOLUMN_ISSET_ID, value); + } + + public byte[] getBinaryColumn() { + setBinaryColumn(org.apache.thrift.TBaseHelper.rightSize(binaryColumn)); + return binaryColumn == null ? null : binaryColumn.array(); + } + + public ByteBuffer bufferForBinaryColumn() { + return org.apache.thrift.TBaseHelper.copyBinary(binaryColumn); + } + + public ParquetThriftCompat setBinaryColumn(byte[] binaryColumn) { + this.binaryColumn = binaryColumn == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(binaryColumn, binaryColumn.length)); + return this; + } + + public ParquetThriftCompat setBinaryColumn(ByteBuffer binaryColumn) { + this.binaryColumn = org.apache.thrift.TBaseHelper.copyBinary(binaryColumn); + return this; + } + + public void unsetBinaryColumn() { + this.binaryColumn = null; + } + + /** Returns true if field binaryColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetBinaryColumn() { + return this.binaryColumn != null; + } + + public void setBinaryColumnIsSet(boolean value) { + if (!value) { + this.binaryColumn = null; + } + } + + public String getStringColumn() { + return this.stringColumn; + } + + public ParquetThriftCompat setStringColumn(String stringColumn) { + this.stringColumn = stringColumn; + return this; + } + + public void unsetStringColumn() { + this.stringColumn = null; + } + + /** Returns true if field stringColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetStringColumn() { + return this.stringColumn != null; + } + + public void setStringColumnIsSet(boolean value) { + if (!value) { + this.stringColumn = null; + } + } + + /** + * + * @see Suit + */ + public Suit getEnumColumn() { + return this.enumColumn; + } + + /** + * + * @see Suit + */ + public ParquetThriftCompat setEnumColumn(Suit enumColumn) { + this.enumColumn = enumColumn; + return this; + } + + public void unsetEnumColumn() { + this.enumColumn = null; + } + + /** Returns true if field enumColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetEnumColumn() { + return this.enumColumn != null; + } + + public void setEnumColumnIsSet(boolean value) { + if (!value) { + this.enumColumn = null; + } + } + + public boolean isMaybeBoolColumn() { + return this.maybeBoolColumn; + } + + public ParquetThriftCompat setMaybeBoolColumn(boolean maybeBoolColumn) { + this.maybeBoolColumn = maybeBoolColumn; + setMaybeBoolColumnIsSet(true); + return this; + } + + public void unsetMaybeBoolColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBEBOOLCOLUMN_ISSET_ID); + } + + /** Returns true if field maybeBoolColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeBoolColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBEBOOLCOLUMN_ISSET_ID); + } + + public void setMaybeBoolColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBEBOOLCOLUMN_ISSET_ID, value); + } + + public byte getMaybeByteColumn() { + return this.maybeByteColumn; + } + + public ParquetThriftCompat setMaybeByteColumn(byte maybeByteColumn) { + this.maybeByteColumn = maybeByteColumn; + setMaybeByteColumnIsSet(true); + return this; + } + + public void unsetMaybeByteColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBEBYTECOLUMN_ISSET_ID); + } + + /** Returns true if field maybeByteColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeByteColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBEBYTECOLUMN_ISSET_ID); + } + + public void setMaybeByteColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBEBYTECOLUMN_ISSET_ID, value); + } + + public short getMaybeShortColumn() { + return this.maybeShortColumn; + } + + public ParquetThriftCompat setMaybeShortColumn(short maybeShortColumn) { + this.maybeShortColumn = maybeShortColumn; + setMaybeShortColumnIsSet(true); + return this; + } + + public void unsetMaybeShortColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBESHORTCOLUMN_ISSET_ID); + } + + /** Returns true if field maybeShortColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeShortColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBESHORTCOLUMN_ISSET_ID); + } + + public void setMaybeShortColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBESHORTCOLUMN_ISSET_ID, value); + } + + public int getMaybeIntColumn() { + return this.maybeIntColumn; + } + + public ParquetThriftCompat setMaybeIntColumn(int maybeIntColumn) { + this.maybeIntColumn = maybeIntColumn; + setMaybeIntColumnIsSet(true); + return this; + } + + public void unsetMaybeIntColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBEINTCOLUMN_ISSET_ID); + } + + /** Returns true if field maybeIntColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeIntColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBEINTCOLUMN_ISSET_ID); + } + + public void setMaybeIntColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBEINTCOLUMN_ISSET_ID, value); + } + + public long getMaybeLongColumn() { + return this.maybeLongColumn; + } + + public ParquetThriftCompat setMaybeLongColumn(long maybeLongColumn) { + this.maybeLongColumn = maybeLongColumn; + setMaybeLongColumnIsSet(true); + return this; + } + + public void unsetMaybeLongColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBELONGCOLUMN_ISSET_ID); + } + + /** Returns true if field maybeLongColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeLongColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBELONGCOLUMN_ISSET_ID); + } + + public void setMaybeLongColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBELONGCOLUMN_ISSET_ID, value); + } + + public double getMaybeDoubleColumn() { + return this.maybeDoubleColumn; + } + + public ParquetThriftCompat setMaybeDoubleColumn(double maybeDoubleColumn) { + this.maybeDoubleColumn = maybeDoubleColumn; + setMaybeDoubleColumnIsSet(true); + return this; + } + + public void unsetMaybeDoubleColumn() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAYBEDOUBLECOLUMN_ISSET_ID); + } + + /** Returns true if field maybeDoubleColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeDoubleColumn() { + return EncodingUtils.testBit(__isset_bitfield, __MAYBEDOUBLECOLUMN_ISSET_ID); + } + + public void setMaybeDoubleColumnIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAYBEDOUBLECOLUMN_ISSET_ID, value); + } + + public byte[] getMaybeBinaryColumn() { + setMaybeBinaryColumn(org.apache.thrift.TBaseHelper.rightSize(maybeBinaryColumn)); + return maybeBinaryColumn == null ? null : maybeBinaryColumn.array(); + } + + public ByteBuffer bufferForMaybeBinaryColumn() { + return org.apache.thrift.TBaseHelper.copyBinary(maybeBinaryColumn); + } + + public ParquetThriftCompat setMaybeBinaryColumn(byte[] maybeBinaryColumn) { + this.maybeBinaryColumn = maybeBinaryColumn == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(maybeBinaryColumn, maybeBinaryColumn.length)); + return this; + } + + public ParquetThriftCompat setMaybeBinaryColumn(ByteBuffer maybeBinaryColumn) { + this.maybeBinaryColumn = org.apache.thrift.TBaseHelper.copyBinary(maybeBinaryColumn); + return this; + } + + public void unsetMaybeBinaryColumn() { + this.maybeBinaryColumn = null; + } + + /** Returns true if field maybeBinaryColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeBinaryColumn() { + return this.maybeBinaryColumn != null; + } + + public void setMaybeBinaryColumnIsSet(boolean value) { + if (!value) { + this.maybeBinaryColumn = null; + } + } + + public String getMaybeStringColumn() { + return this.maybeStringColumn; + } + + public ParquetThriftCompat setMaybeStringColumn(String maybeStringColumn) { + this.maybeStringColumn = maybeStringColumn; + return this; + } + + public void unsetMaybeStringColumn() { + this.maybeStringColumn = null; + } + + /** Returns true if field maybeStringColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeStringColumn() { + return this.maybeStringColumn != null; + } + + public void setMaybeStringColumnIsSet(boolean value) { + if (!value) { + this.maybeStringColumn = null; + } + } + + /** + * + * @see Suit + */ + public Suit getMaybeEnumColumn() { + return this.maybeEnumColumn; + } + + /** + * + * @see Suit + */ + public ParquetThriftCompat setMaybeEnumColumn(Suit maybeEnumColumn) { + this.maybeEnumColumn = maybeEnumColumn; + return this; + } + + public void unsetMaybeEnumColumn() { + this.maybeEnumColumn = null; + } + + /** Returns true if field maybeEnumColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetMaybeEnumColumn() { + return this.maybeEnumColumn != null; + } + + public void setMaybeEnumColumnIsSet(boolean value) { + if (!value) { + this.maybeEnumColumn = null; + } + } + + public int getStringsColumnSize() { + return (this.stringsColumn == null) ? 0 : this.stringsColumn.size(); + } + + public java.util.Iterator getStringsColumnIterator() { + return (this.stringsColumn == null) ? null : this.stringsColumn.iterator(); + } + + public void addToStringsColumn(String elem) { + if (this.stringsColumn == null) { + this.stringsColumn = new ArrayList(); + } + this.stringsColumn.add(elem); + } + + public List getStringsColumn() { + return this.stringsColumn; + } + + public ParquetThriftCompat setStringsColumn(List stringsColumn) { + this.stringsColumn = stringsColumn; + return this; + } + + public void unsetStringsColumn() { + this.stringsColumn = null; + } + + /** Returns true if field stringsColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetStringsColumn() { + return this.stringsColumn != null; + } + + public void setStringsColumnIsSet(boolean value) { + if (!value) { + this.stringsColumn = null; + } + } + + public int getIntSetColumnSize() { + return (this.intSetColumn == null) ? 0 : this.intSetColumn.size(); + } + + public java.util.Iterator getIntSetColumnIterator() { + return (this.intSetColumn == null) ? null : this.intSetColumn.iterator(); + } + + public void addToIntSetColumn(int elem) { + if (this.intSetColumn == null) { + this.intSetColumn = new HashSet(); + } + this.intSetColumn.add(elem); + } + + public Set getIntSetColumn() { + return this.intSetColumn; + } + + public ParquetThriftCompat setIntSetColumn(Set intSetColumn) { + this.intSetColumn = intSetColumn; + return this; + } + + public void unsetIntSetColumn() { + this.intSetColumn = null; + } + + /** Returns true if field intSetColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetIntSetColumn() { + return this.intSetColumn != null; + } + + public void setIntSetColumnIsSet(boolean value) { + if (!value) { + this.intSetColumn = null; + } + } + + public int getIntToStringColumnSize() { + return (this.intToStringColumn == null) ? 0 : this.intToStringColumn.size(); + } + + public void putToIntToStringColumn(int key, String val) { + if (this.intToStringColumn == null) { + this.intToStringColumn = new HashMap(); + } + this.intToStringColumn.put(key, val); + } + + public Map getIntToStringColumn() { + return this.intToStringColumn; + } + + public ParquetThriftCompat setIntToStringColumn(Map intToStringColumn) { + this.intToStringColumn = intToStringColumn; + return this; + } + + public void unsetIntToStringColumn() { + this.intToStringColumn = null; + } + + /** Returns true if field intToStringColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetIntToStringColumn() { + return this.intToStringColumn != null; + } + + public void setIntToStringColumnIsSet(boolean value) { + if (!value) { + this.intToStringColumn = null; + } + } + + public int getComplexColumnSize() { + return (this.complexColumn == null) ? 0 : this.complexColumn.size(); + } + + public void putToComplexColumn(int key, List val) { + if (this.complexColumn == null) { + this.complexColumn = new HashMap>(); + } + this.complexColumn.put(key, val); + } + + public Map> getComplexColumn() { + return this.complexColumn; + } + + public ParquetThriftCompat setComplexColumn(Map> complexColumn) { + this.complexColumn = complexColumn; + return this; + } + + public void unsetComplexColumn() { + this.complexColumn = null; + } + + /** Returns true if field complexColumn is set (has been assigned a value) and false otherwise */ + public boolean isSetComplexColumn() { + return this.complexColumn != null; + } + + public void setComplexColumnIsSet(boolean value) { + if (!value) { + this.complexColumn = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case BOOL_COLUMN: + if (value == null) { + unsetBoolColumn(); + } else { + setBoolColumn((Boolean)value); + } + break; + + case BYTE_COLUMN: + if (value == null) { + unsetByteColumn(); + } else { + setByteColumn((Byte)value); + } + break; + + case SHORT_COLUMN: + if (value == null) { + unsetShortColumn(); + } else { + setShortColumn((Short)value); + } + break; + + case INT_COLUMN: + if (value == null) { + unsetIntColumn(); + } else { + setIntColumn((Integer)value); + } + break; + + case LONG_COLUMN: + if (value == null) { + unsetLongColumn(); + } else { + setLongColumn((Long)value); + } + break; + + case DOUBLE_COLUMN: + if (value == null) { + unsetDoubleColumn(); + } else { + setDoubleColumn((Double)value); + } + break; + + case BINARY_COLUMN: + if (value == null) { + unsetBinaryColumn(); + } else { + setBinaryColumn((ByteBuffer)value); + } + break; + + case STRING_COLUMN: + if (value == null) { + unsetStringColumn(); + } else { + setStringColumn((String)value); + } + break; + + case ENUM_COLUMN: + if (value == null) { + unsetEnumColumn(); + } else { + setEnumColumn((Suit)value); + } + break; + + case MAYBE_BOOL_COLUMN: + if (value == null) { + unsetMaybeBoolColumn(); + } else { + setMaybeBoolColumn((Boolean)value); + } + break; + + case MAYBE_BYTE_COLUMN: + if (value == null) { + unsetMaybeByteColumn(); + } else { + setMaybeByteColumn((Byte)value); + } + break; + + case MAYBE_SHORT_COLUMN: + if (value == null) { + unsetMaybeShortColumn(); + } else { + setMaybeShortColumn((Short)value); + } + break; + + case MAYBE_INT_COLUMN: + if (value == null) { + unsetMaybeIntColumn(); + } else { + setMaybeIntColumn((Integer)value); + } + break; + + case MAYBE_LONG_COLUMN: + if (value == null) { + unsetMaybeLongColumn(); + } else { + setMaybeLongColumn((Long)value); + } + break; + + case MAYBE_DOUBLE_COLUMN: + if (value == null) { + unsetMaybeDoubleColumn(); + } else { + setMaybeDoubleColumn((Double)value); + } + break; + + case MAYBE_BINARY_COLUMN: + if (value == null) { + unsetMaybeBinaryColumn(); + } else { + setMaybeBinaryColumn((ByteBuffer)value); + } + break; + + case MAYBE_STRING_COLUMN: + if (value == null) { + unsetMaybeStringColumn(); + } else { + setMaybeStringColumn((String)value); + } + break; + + case MAYBE_ENUM_COLUMN: + if (value == null) { + unsetMaybeEnumColumn(); + } else { + setMaybeEnumColumn((Suit)value); + } + break; + + case STRINGS_COLUMN: + if (value == null) { + unsetStringsColumn(); + } else { + setStringsColumn((List)value); + } + break; + + case INT_SET_COLUMN: + if (value == null) { + unsetIntSetColumn(); + } else { + setIntSetColumn((Set)value); + } + break; + + case INT_TO_STRING_COLUMN: + if (value == null) { + unsetIntToStringColumn(); + } else { + setIntToStringColumn((Map)value); + } + break; + + case COMPLEX_COLUMN: + if (value == null) { + unsetComplexColumn(); + } else { + setComplexColumn((Map>)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case BOOL_COLUMN: + return Boolean.valueOf(isBoolColumn()); + + case BYTE_COLUMN: + return Byte.valueOf(getByteColumn()); + + case SHORT_COLUMN: + return Short.valueOf(getShortColumn()); + + case INT_COLUMN: + return Integer.valueOf(getIntColumn()); + + case LONG_COLUMN: + return Long.valueOf(getLongColumn()); + + case DOUBLE_COLUMN: + return Double.valueOf(getDoubleColumn()); + + case BINARY_COLUMN: + return getBinaryColumn(); + + case STRING_COLUMN: + return getStringColumn(); + + case ENUM_COLUMN: + return getEnumColumn(); + + case MAYBE_BOOL_COLUMN: + return Boolean.valueOf(isMaybeBoolColumn()); + + case MAYBE_BYTE_COLUMN: + return Byte.valueOf(getMaybeByteColumn()); + + case MAYBE_SHORT_COLUMN: + return Short.valueOf(getMaybeShortColumn()); + + case MAYBE_INT_COLUMN: + return Integer.valueOf(getMaybeIntColumn()); + + case MAYBE_LONG_COLUMN: + return Long.valueOf(getMaybeLongColumn()); + + case MAYBE_DOUBLE_COLUMN: + return Double.valueOf(getMaybeDoubleColumn()); + + case MAYBE_BINARY_COLUMN: + return getMaybeBinaryColumn(); + + case MAYBE_STRING_COLUMN: + return getMaybeStringColumn(); + + case MAYBE_ENUM_COLUMN: + return getMaybeEnumColumn(); + + case STRINGS_COLUMN: + return getStringsColumn(); + + case INT_SET_COLUMN: + return getIntSetColumn(); + + case INT_TO_STRING_COLUMN: + return getIntToStringColumn(); + + case COMPLEX_COLUMN: + return getComplexColumn(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case BOOL_COLUMN: + return isSetBoolColumn(); + case BYTE_COLUMN: + return isSetByteColumn(); + case SHORT_COLUMN: + return isSetShortColumn(); + case INT_COLUMN: + return isSetIntColumn(); + case LONG_COLUMN: + return isSetLongColumn(); + case DOUBLE_COLUMN: + return isSetDoubleColumn(); + case BINARY_COLUMN: + return isSetBinaryColumn(); + case STRING_COLUMN: + return isSetStringColumn(); + case ENUM_COLUMN: + return isSetEnumColumn(); + case MAYBE_BOOL_COLUMN: + return isSetMaybeBoolColumn(); + case MAYBE_BYTE_COLUMN: + return isSetMaybeByteColumn(); + case MAYBE_SHORT_COLUMN: + return isSetMaybeShortColumn(); + case MAYBE_INT_COLUMN: + return isSetMaybeIntColumn(); + case MAYBE_LONG_COLUMN: + return isSetMaybeLongColumn(); + case MAYBE_DOUBLE_COLUMN: + return isSetMaybeDoubleColumn(); + case MAYBE_BINARY_COLUMN: + return isSetMaybeBinaryColumn(); + case MAYBE_STRING_COLUMN: + return isSetMaybeStringColumn(); + case MAYBE_ENUM_COLUMN: + return isSetMaybeEnumColumn(); + case STRINGS_COLUMN: + return isSetStringsColumn(); + case INT_SET_COLUMN: + return isSetIntSetColumn(); + case INT_TO_STRING_COLUMN: + return isSetIntToStringColumn(); + case COMPLEX_COLUMN: + return isSetComplexColumn(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ParquetThriftCompat) + return this.equals((ParquetThriftCompat)that); + return false; + } + + public boolean equals(ParquetThriftCompat that) { + if (that == null) + return false; + + boolean this_present_boolColumn = true; + boolean that_present_boolColumn = true; + if (this_present_boolColumn || that_present_boolColumn) { + if (!(this_present_boolColumn && that_present_boolColumn)) + return false; + if (this.boolColumn != that.boolColumn) + return false; + } + + boolean this_present_byteColumn = true; + boolean that_present_byteColumn = true; + if (this_present_byteColumn || that_present_byteColumn) { + if (!(this_present_byteColumn && that_present_byteColumn)) + return false; + if (this.byteColumn != that.byteColumn) + return false; + } + + boolean this_present_shortColumn = true; + boolean that_present_shortColumn = true; + if (this_present_shortColumn || that_present_shortColumn) { + if (!(this_present_shortColumn && that_present_shortColumn)) + return false; + if (this.shortColumn != that.shortColumn) + return false; + } + + boolean this_present_intColumn = true; + boolean that_present_intColumn = true; + if (this_present_intColumn || that_present_intColumn) { + if (!(this_present_intColumn && that_present_intColumn)) + return false; + if (this.intColumn != that.intColumn) + return false; + } + + boolean this_present_longColumn = true; + boolean that_present_longColumn = true; + if (this_present_longColumn || that_present_longColumn) { + if (!(this_present_longColumn && that_present_longColumn)) + return false; + if (this.longColumn != that.longColumn) + return false; + } + + boolean this_present_doubleColumn = true; + boolean that_present_doubleColumn = true; + if (this_present_doubleColumn || that_present_doubleColumn) { + if (!(this_present_doubleColumn && that_present_doubleColumn)) + return false; + if (this.doubleColumn != that.doubleColumn) + return false; + } + + boolean this_present_binaryColumn = true && this.isSetBinaryColumn(); + boolean that_present_binaryColumn = true && that.isSetBinaryColumn(); + if (this_present_binaryColumn || that_present_binaryColumn) { + if (!(this_present_binaryColumn && that_present_binaryColumn)) + return false; + if (!this.binaryColumn.equals(that.binaryColumn)) + return false; + } + + boolean this_present_stringColumn = true && this.isSetStringColumn(); + boolean that_present_stringColumn = true && that.isSetStringColumn(); + if (this_present_stringColumn || that_present_stringColumn) { + if (!(this_present_stringColumn && that_present_stringColumn)) + return false; + if (!this.stringColumn.equals(that.stringColumn)) + return false; + } + + boolean this_present_enumColumn = true && this.isSetEnumColumn(); + boolean that_present_enumColumn = true && that.isSetEnumColumn(); + if (this_present_enumColumn || that_present_enumColumn) { + if (!(this_present_enumColumn && that_present_enumColumn)) + return false; + if (!this.enumColumn.equals(that.enumColumn)) + return false; + } + + boolean this_present_maybeBoolColumn = true && this.isSetMaybeBoolColumn(); + boolean that_present_maybeBoolColumn = true && that.isSetMaybeBoolColumn(); + if (this_present_maybeBoolColumn || that_present_maybeBoolColumn) { + if (!(this_present_maybeBoolColumn && that_present_maybeBoolColumn)) + return false; + if (this.maybeBoolColumn != that.maybeBoolColumn) + return false; + } + + boolean this_present_maybeByteColumn = true && this.isSetMaybeByteColumn(); + boolean that_present_maybeByteColumn = true && that.isSetMaybeByteColumn(); + if (this_present_maybeByteColumn || that_present_maybeByteColumn) { + if (!(this_present_maybeByteColumn && that_present_maybeByteColumn)) + return false; + if (this.maybeByteColumn != that.maybeByteColumn) + return false; + } + + boolean this_present_maybeShortColumn = true && this.isSetMaybeShortColumn(); + boolean that_present_maybeShortColumn = true && that.isSetMaybeShortColumn(); + if (this_present_maybeShortColumn || that_present_maybeShortColumn) { + if (!(this_present_maybeShortColumn && that_present_maybeShortColumn)) + return false; + if (this.maybeShortColumn != that.maybeShortColumn) + return false; + } + + boolean this_present_maybeIntColumn = true && this.isSetMaybeIntColumn(); + boolean that_present_maybeIntColumn = true && that.isSetMaybeIntColumn(); + if (this_present_maybeIntColumn || that_present_maybeIntColumn) { + if (!(this_present_maybeIntColumn && that_present_maybeIntColumn)) + return false; + if (this.maybeIntColumn != that.maybeIntColumn) + return false; + } + + boolean this_present_maybeLongColumn = true && this.isSetMaybeLongColumn(); + boolean that_present_maybeLongColumn = true && that.isSetMaybeLongColumn(); + if (this_present_maybeLongColumn || that_present_maybeLongColumn) { + if (!(this_present_maybeLongColumn && that_present_maybeLongColumn)) + return false; + if (this.maybeLongColumn != that.maybeLongColumn) + return false; + } + + boolean this_present_maybeDoubleColumn = true && this.isSetMaybeDoubleColumn(); + boolean that_present_maybeDoubleColumn = true && that.isSetMaybeDoubleColumn(); + if (this_present_maybeDoubleColumn || that_present_maybeDoubleColumn) { + if (!(this_present_maybeDoubleColumn && that_present_maybeDoubleColumn)) + return false; + if (this.maybeDoubleColumn != that.maybeDoubleColumn) + return false; + } + + boolean this_present_maybeBinaryColumn = true && this.isSetMaybeBinaryColumn(); + boolean that_present_maybeBinaryColumn = true && that.isSetMaybeBinaryColumn(); + if (this_present_maybeBinaryColumn || that_present_maybeBinaryColumn) { + if (!(this_present_maybeBinaryColumn && that_present_maybeBinaryColumn)) + return false; + if (!this.maybeBinaryColumn.equals(that.maybeBinaryColumn)) + return false; + } + + boolean this_present_maybeStringColumn = true && this.isSetMaybeStringColumn(); + boolean that_present_maybeStringColumn = true && that.isSetMaybeStringColumn(); + if (this_present_maybeStringColumn || that_present_maybeStringColumn) { + if (!(this_present_maybeStringColumn && that_present_maybeStringColumn)) + return false; + if (!this.maybeStringColumn.equals(that.maybeStringColumn)) + return false; + } + + boolean this_present_maybeEnumColumn = true && this.isSetMaybeEnumColumn(); + boolean that_present_maybeEnumColumn = true && that.isSetMaybeEnumColumn(); + if (this_present_maybeEnumColumn || that_present_maybeEnumColumn) { + if (!(this_present_maybeEnumColumn && that_present_maybeEnumColumn)) + return false; + if (!this.maybeEnumColumn.equals(that.maybeEnumColumn)) + return false; + } + + boolean this_present_stringsColumn = true && this.isSetStringsColumn(); + boolean that_present_stringsColumn = true && that.isSetStringsColumn(); + if (this_present_stringsColumn || that_present_stringsColumn) { + if (!(this_present_stringsColumn && that_present_stringsColumn)) + return false; + if (!this.stringsColumn.equals(that.stringsColumn)) + return false; + } + + boolean this_present_intSetColumn = true && this.isSetIntSetColumn(); + boolean that_present_intSetColumn = true && that.isSetIntSetColumn(); + if (this_present_intSetColumn || that_present_intSetColumn) { + if (!(this_present_intSetColumn && that_present_intSetColumn)) + return false; + if (!this.intSetColumn.equals(that.intSetColumn)) + return false; + } + + boolean this_present_intToStringColumn = true && this.isSetIntToStringColumn(); + boolean that_present_intToStringColumn = true && that.isSetIntToStringColumn(); + if (this_present_intToStringColumn || that_present_intToStringColumn) { + if (!(this_present_intToStringColumn && that_present_intToStringColumn)) + return false; + if (!this.intToStringColumn.equals(that.intToStringColumn)) + return false; + } + + boolean this_present_complexColumn = true && this.isSetComplexColumn(); + boolean that_present_complexColumn = true && that.isSetComplexColumn(); + if (this_present_complexColumn || that_present_complexColumn) { + if (!(this_present_complexColumn && that_present_complexColumn)) + return false; + if (!this.complexColumn.equals(that.complexColumn)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_boolColumn = true; + list.add(present_boolColumn); + if (present_boolColumn) + list.add(boolColumn); + + boolean present_byteColumn = true; + list.add(present_byteColumn); + if (present_byteColumn) + list.add(byteColumn); + + boolean present_shortColumn = true; + list.add(present_shortColumn); + if (present_shortColumn) + list.add(shortColumn); + + boolean present_intColumn = true; + list.add(present_intColumn); + if (present_intColumn) + list.add(intColumn); + + boolean present_longColumn = true; + list.add(present_longColumn); + if (present_longColumn) + list.add(longColumn); + + boolean present_doubleColumn = true; + list.add(present_doubleColumn); + if (present_doubleColumn) + list.add(doubleColumn); + + boolean present_binaryColumn = true && (isSetBinaryColumn()); + list.add(present_binaryColumn); + if (present_binaryColumn) + list.add(binaryColumn); + + boolean present_stringColumn = true && (isSetStringColumn()); + list.add(present_stringColumn); + if (present_stringColumn) + list.add(stringColumn); + + boolean present_enumColumn = true && (isSetEnumColumn()); + list.add(present_enumColumn); + if (present_enumColumn) + list.add(enumColumn.getValue()); + + boolean present_maybeBoolColumn = true && (isSetMaybeBoolColumn()); + list.add(present_maybeBoolColumn); + if (present_maybeBoolColumn) + list.add(maybeBoolColumn); + + boolean present_maybeByteColumn = true && (isSetMaybeByteColumn()); + list.add(present_maybeByteColumn); + if (present_maybeByteColumn) + list.add(maybeByteColumn); + + boolean present_maybeShortColumn = true && (isSetMaybeShortColumn()); + list.add(present_maybeShortColumn); + if (present_maybeShortColumn) + list.add(maybeShortColumn); + + boolean present_maybeIntColumn = true && (isSetMaybeIntColumn()); + list.add(present_maybeIntColumn); + if (present_maybeIntColumn) + list.add(maybeIntColumn); + + boolean present_maybeLongColumn = true && (isSetMaybeLongColumn()); + list.add(present_maybeLongColumn); + if (present_maybeLongColumn) + list.add(maybeLongColumn); + + boolean present_maybeDoubleColumn = true && (isSetMaybeDoubleColumn()); + list.add(present_maybeDoubleColumn); + if (present_maybeDoubleColumn) + list.add(maybeDoubleColumn); + + boolean present_maybeBinaryColumn = true && (isSetMaybeBinaryColumn()); + list.add(present_maybeBinaryColumn); + if (present_maybeBinaryColumn) + list.add(maybeBinaryColumn); + + boolean present_maybeStringColumn = true && (isSetMaybeStringColumn()); + list.add(present_maybeStringColumn); + if (present_maybeStringColumn) + list.add(maybeStringColumn); + + boolean present_maybeEnumColumn = true && (isSetMaybeEnumColumn()); + list.add(present_maybeEnumColumn); + if (present_maybeEnumColumn) + list.add(maybeEnumColumn.getValue()); + + boolean present_stringsColumn = true && (isSetStringsColumn()); + list.add(present_stringsColumn); + if (present_stringsColumn) + list.add(stringsColumn); + + boolean present_intSetColumn = true && (isSetIntSetColumn()); + list.add(present_intSetColumn); + if (present_intSetColumn) + list.add(intSetColumn); + + boolean present_intToStringColumn = true && (isSetIntToStringColumn()); + list.add(present_intToStringColumn); + if (present_intToStringColumn) + list.add(intToStringColumn); + + boolean present_complexColumn = true && (isSetComplexColumn()); + list.add(present_complexColumn); + if (present_complexColumn) + list.add(complexColumn); + + return list.hashCode(); + } + + @Override + public int compareTo(ParquetThriftCompat other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetBoolColumn()).compareTo(other.isSetBoolColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetBoolColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.boolColumn, other.boolColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetByteColumn()).compareTo(other.isSetByteColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetByteColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.byteColumn, other.byteColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetShortColumn()).compareTo(other.isSetShortColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetShortColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.shortColumn, other.shortColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetIntColumn()).compareTo(other.isSetIntColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIntColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.intColumn, other.intColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetLongColumn()).compareTo(other.isSetLongColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetLongColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.longColumn, other.longColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetDoubleColumn()).compareTo(other.isSetDoubleColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDoubleColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.doubleColumn, other.doubleColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetBinaryColumn()).compareTo(other.isSetBinaryColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetBinaryColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.binaryColumn, other.binaryColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStringColumn()).compareTo(other.isSetStringColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStringColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stringColumn, other.stringColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetEnumColumn()).compareTo(other.isSetEnumColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetEnumColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.enumColumn, other.enumColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeBoolColumn()).compareTo(other.isSetMaybeBoolColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeBoolColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeBoolColumn, other.maybeBoolColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeByteColumn()).compareTo(other.isSetMaybeByteColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeByteColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeByteColumn, other.maybeByteColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeShortColumn()).compareTo(other.isSetMaybeShortColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeShortColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeShortColumn, other.maybeShortColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeIntColumn()).compareTo(other.isSetMaybeIntColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeIntColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeIntColumn, other.maybeIntColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeLongColumn()).compareTo(other.isSetMaybeLongColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeLongColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeLongColumn, other.maybeLongColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeDoubleColumn()).compareTo(other.isSetMaybeDoubleColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeDoubleColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeDoubleColumn, other.maybeDoubleColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeBinaryColumn()).compareTo(other.isSetMaybeBinaryColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeBinaryColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeBinaryColumn, other.maybeBinaryColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeStringColumn()).compareTo(other.isSetMaybeStringColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeStringColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeStringColumn, other.maybeStringColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaybeEnumColumn()).compareTo(other.isSetMaybeEnumColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaybeEnumColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maybeEnumColumn, other.maybeEnumColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStringsColumn()).compareTo(other.isSetStringsColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStringsColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.stringsColumn, other.stringsColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetIntSetColumn()).compareTo(other.isSetIntSetColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIntSetColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.intSetColumn, other.intSetColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetIntToStringColumn()).compareTo(other.isSetIntToStringColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetIntToStringColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.intToStringColumn, other.intToStringColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetComplexColumn()).compareTo(other.isSetComplexColumn()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetComplexColumn()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.complexColumn, other.complexColumn); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ParquetThriftCompat("); + boolean first = true; + + sb.append("boolColumn:"); + sb.append(this.boolColumn); + first = false; + if (!first) sb.append(", "); + sb.append("byteColumn:"); + sb.append(this.byteColumn); + first = false; + if (!first) sb.append(", "); + sb.append("shortColumn:"); + sb.append(this.shortColumn); + first = false; + if (!first) sb.append(", "); + sb.append("intColumn:"); + sb.append(this.intColumn); + first = false; + if (!first) sb.append(", "); + sb.append("longColumn:"); + sb.append(this.longColumn); + first = false; + if (!first) sb.append(", "); + sb.append("doubleColumn:"); + sb.append(this.doubleColumn); + first = false; + if (!first) sb.append(", "); + sb.append("binaryColumn:"); + if (this.binaryColumn == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.binaryColumn, sb); + } + first = false; + if (!first) sb.append(", "); + sb.append("stringColumn:"); + if (this.stringColumn == null) { + sb.append("null"); + } else { + sb.append(this.stringColumn); + } + first = false; + if (!first) sb.append(", "); + sb.append("enumColumn:"); + if (this.enumColumn == null) { + sb.append("null"); + } else { + sb.append(this.enumColumn); + } + first = false; + if (isSetMaybeBoolColumn()) { + if (!first) sb.append(", "); + sb.append("maybeBoolColumn:"); + sb.append(this.maybeBoolColumn); + first = false; + } + if (isSetMaybeByteColumn()) { + if (!first) sb.append(", "); + sb.append("maybeByteColumn:"); + sb.append(this.maybeByteColumn); + first = false; + } + if (isSetMaybeShortColumn()) { + if (!first) sb.append(", "); + sb.append("maybeShortColumn:"); + sb.append(this.maybeShortColumn); + first = false; + } + if (isSetMaybeIntColumn()) { + if (!first) sb.append(", "); + sb.append("maybeIntColumn:"); + sb.append(this.maybeIntColumn); + first = false; + } + if (isSetMaybeLongColumn()) { + if (!first) sb.append(", "); + sb.append("maybeLongColumn:"); + sb.append(this.maybeLongColumn); + first = false; + } + if (isSetMaybeDoubleColumn()) { + if (!first) sb.append(", "); + sb.append("maybeDoubleColumn:"); + sb.append(this.maybeDoubleColumn); + first = false; + } + if (isSetMaybeBinaryColumn()) { + if (!first) sb.append(", "); + sb.append("maybeBinaryColumn:"); + if (this.maybeBinaryColumn == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.maybeBinaryColumn, sb); + } + first = false; + } + if (isSetMaybeStringColumn()) { + if (!first) sb.append(", "); + sb.append("maybeStringColumn:"); + if (this.maybeStringColumn == null) { + sb.append("null"); + } else { + sb.append(this.maybeStringColumn); + } + first = false; + } + if (isSetMaybeEnumColumn()) { + if (!first) sb.append(", "); + sb.append("maybeEnumColumn:"); + if (this.maybeEnumColumn == null) { + sb.append("null"); + } else { + sb.append(this.maybeEnumColumn); + } + first = false; + } + if (!first) sb.append(", "); + sb.append("stringsColumn:"); + if (this.stringsColumn == null) { + sb.append("null"); + } else { + sb.append(this.stringsColumn); + } + first = false; + if (!first) sb.append(", "); + sb.append("intSetColumn:"); + if (this.intSetColumn == null) { + sb.append("null"); + } else { + sb.append(this.intSetColumn); + } + first = false; + if (!first) sb.append(", "); + sb.append("intToStringColumn:"); + if (this.intToStringColumn == null) { + sb.append("null"); + } else { + sb.append(this.intToStringColumn); + } + first = false; + if (!first) sb.append(", "); + sb.append("complexColumn:"); + if (this.complexColumn == null) { + sb.append("null"); + } else { + sb.append(this.complexColumn); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // alas, we cannot check 'boolColumn' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'byteColumn' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'shortColumn' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'intColumn' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'longColumn' because it's a primitive and you chose the non-beans generator. + // alas, we cannot check 'doubleColumn' because it's a primitive and you chose the non-beans generator. + if (binaryColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'binaryColumn' was not present! Struct: " + toString()); + } + if (stringColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'stringColumn' was not present! Struct: " + toString()); + } + if (enumColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'enumColumn' was not present! Struct: " + toString()); + } + if (stringsColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'stringsColumn' was not present! Struct: " + toString()); + } + if (intSetColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'intSetColumn' was not present! Struct: " + toString()); + } + if (intToStringColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'intToStringColumn' was not present! Struct: " + toString()); + } + if (complexColumn == null) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'complexColumn' was not present! Struct: " + toString()); + } + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ParquetThriftCompatStandardSchemeFactory implements SchemeFactory { + public ParquetThriftCompatStandardScheme getScheme() { + return new ParquetThriftCompatStandardScheme(); + } + } + + private static class ParquetThriftCompatStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ParquetThriftCompat struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // BOOL_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.boolColumn = iprot.readBool(); + struct.setBoolColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // BYTE_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) { + struct.byteColumn = iprot.readByte(); + struct.setByteColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SHORT_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I16) { + struct.shortColumn = iprot.readI16(); + struct.setShortColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // INT_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.intColumn = iprot.readI32(); + struct.setIntColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // LONG_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.longColumn = iprot.readI64(); + struct.setLongColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // DOUBLE_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.doubleColumn = iprot.readDouble(); + struct.setDoubleColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // BINARY_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.binaryColumn = iprot.readBinary(); + struct.setBinaryColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 8: // STRING_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.stringColumn = iprot.readString(); + struct.setStringColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 9: // ENUM_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.enumColumn = org.apache.spark.sql.parquet.test.thrift.Suit.findByValue(iprot.readI32()); + struct.setEnumColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 10: // MAYBE_BOOL_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.maybeBoolColumn = iprot.readBool(); + struct.setMaybeBoolColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 11: // MAYBE_BYTE_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) { + struct.maybeByteColumn = iprot.readByte(); + struct.setMaybeByteColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 12: // MAYBE_SHORT_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I16) { + struct.maybeShortColumn = iprot.readI16(); + struct.setMaybeShortColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 13: // MAYBE_INT_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.maybeIntColumn = iprot.readI32(); + struct.setMaybeIntColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 14: // MAYBE_LONG_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.maybeLongColumn = iprot.readI64(); + struct.setMaybeLongColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 15: // MAYBE_DOUBLE_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.maybeDoubleColumn = iprot.readDouble(); + struct.setMaybeDoubleColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 16: // MAYBE_BINARY_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.maybeBinaryColumn = iprot.readBinary(); + struct.setMaybeBinaryColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 17: // MAYBE_STRING_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.maybeStringColumn = iprot.readString(); + struct.setMaybeStringColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 18: // MAYBE_ENUM_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.maybeEnumColumn = org.apache.spark.sql.parquet.test.thrift.Suit.findByValue(iprot.readI32()); + struct.setMaybeEnumColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 19: // STRINGS_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list8 = iprot.readListBegin(); + struct.stringsColumn = new ArrayList(_list8.size); + String _elem9; + for (int _i10 = 0; _i10 < _list8.size; ++_i10) + { + _elem9 = iprot.readString(); + struct.stringsColumn.add(_elem9); + } + iprot.readListEnd(); + } + struct.setStringsColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 20: // INT_SET_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.SET) { + { + org.apache.thrift.protocol.TSet _set11 = iprot.readSetBegin(); + struct.intSetColumn = new HashSet(2*_set11.size); + int _elem12; + for (int _i13 = 0; _i13 < _set11.size; ++_i13) + { + _elem12 = iprot.readI32(); + struct.intSetColumn.add(_elem12); + } + iprot.readSetEnd(); + } + struct.setIntSetColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 21: // INT_TO_STRING_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map14 = iprot.readMapBegin(); + struct.intToStringColumn = new HashMap(2*_map14.size); + int _key15; + String _val16; + for (int _i17 = 0; _i17 < _map14.size; ++_i17) + { + _key15 = iprot.readI32(); + _val16 = iprot.readString(); + struct.intToStringColumn.put(_key15, _val16); + } + iprot.readMapEnd(); + } + struct.setIntToStringColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 22: // COMPLEX_COLUMN + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map18 = iprot.readMapBegin(); + struct.complexColumn = new HashMap>(2*_map18.size); + int _key19; + List _val20; + for (int _i21 = 0; _i21 < _map18.size; ++_i21) + { + _key19 = iprot.readI32(); + { + org.apache.thrift.protocol.TList _list22 = iprot.readListBegin(); + _val20 = new ArrayList(_list22.size); + Nested _elem23; + for (int _i24 = 0; _i24 < _list22.size; ++_i24) + { + _elem23 = new Nested(); + _elem23.read(iprot); + _val20.add(_elem23); + } + iprot.readListEnd(); + } + struct.complexColumn.put(_key19, _val20); + } + iprot.readMapEnd(); + } + struct.setComplexColumnIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + + // check for required fields of primitive type, which can't be checked in the validate method + if (!struct.isSetBoolColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'boolColumn' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetByteColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'byteColumn' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetShortColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'shortColumn' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetIntColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'intColumn' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetLongColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'longColumn' was not found in serialized data! Struct: " + toString()); + } + if (!struct.isSetDoubleColumn()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'doubleColumn' was not found in serialized data! Struct: " + toString()); + } + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ParquetThriftCompat struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(BOOL_COLUMN_FIELD_DESC); + oprot.writeBool(struct.boolColumn); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(BYTE_COLUMN_FIELD_DESC); + oprot.writeByte(struct.byteColumn); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(SHORT_COLUMN_FIELD_DESC); + oprot.writeI16(struct.shortColumn); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(INT_COLUMN_FIELD_DESC); + oprot.writeI32(struct.intColumn); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(LONG_COLUMN_FIELD_DESC); + oprot.writeI64(struct.longColumn); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(DOUBLE_COLUMN_FIELD_DESC); + oprot.writeDouble(struct.doubleColumn); + oprot.writeFieldEnd(); + if (struct.binaryColumn != null) { + oprot.writeFieldBegin(BINARY_COLUMN_FIELD_DESC); + oprot.writeBinary(struct.binaryColumn); + oprot.writeFieldEnd(); + } + if (struct.stringColumn != null) { + oprot.writeFieldBegin(STRING_COLUMN_FIELD_DESC); + oprot.writeString(struct.stringColumn); + oprot.writeFieldEnd(); + } + if (struct.enumColumn != null) { + oprot.writeFieldBegin(ENUM_COLUMN_FIELD_DESC); + oprot.writeI32(struct.enumColumn.getValue()); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeBoolColumn()) { + oprot.writeFieldBegin(MAYBE_BOOL_COLUMN_FIELD_DESC); + oprot.writeBool(struct.maybeBoolColumn); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeByteColumn()) { + oprot.writeFieldBegin(MAYBE_BYTE_COLUMN_FIELD_DESC); + oprot.writeByte(struct.maybeByteColumn); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeShortColumn()) { + oprot.writeFieldBegin(MAYBE_SHORT_COLUMN_FIELD_DESC); + oprot.writeI16(struct.maybeShortColumn); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeIntColumn()) { + oprot.writeFieldBegin(MAYBE_INT_COLUMN_FIELD_DESC); + oprot.writeI32(struct.maybeIntColumn); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeLongColumn()) { + oprot.writeFieldBegin(MAYBE_LONG_COLUMN_FIELD_DESC); + oprot.writeI64(struct.maybeLongColumn); + oprot.writeFieldEnd(); + } + if (struct.isSetMaybeDoubleColumn()) { + oprot.writeFieldBegin(MAYBE_DOUBLE_COLUMN_FIELD_DESC); + oprot.writeDouble(struct.maybeDoubleColumn); + oprot.writeFieldEnd(); + } + if (struct.maybeBinaryColumn != null) { + if (struct.isSetMaybeBinaryColumn()) { + oprot.writeFieldBegin(MAYBE_BINARY_COLUMN_FIELD_DESC); + oprot.writeBinary(struct.maybeBinaryColumn); + oprot.writeFieldEnd(); + } + } + if (struct.maybeStringColumn != null) { + if (struct.isSetMaybeStringColumn()) { + oprot.writeFieldBegin(MAYBE_STRING_COLUMN_FIELD_DESC); + oprot.writeString(struct.maybeStringColumn); + oprot.writeFieldEnd(); + } + } + if (struct.maybeEnumColumn != null) { + if (struct.isSetMaybeEnumColumn()) { + oprot.writeFieldBegin(MAYBE_ENUM_COLUMN_FIELD_DESC); + oprot.writeI32(struct.maybeEnumColumn.getValue()); + oprot.writeFieldEnd(); + } + } + if (struct.stringsColumn != null) { + oprot.writeFieldBegin(STRINGS_COLUMN_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.stringsColumn.size())); + for (String _iter25 : struct.stringsColumn) + { + oprot.writeString(_iter25); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.intSetColumn != null) { + oprot.writeFieldBegin(INT_SET_COLUMN_FIELD_DESC); + { + oprot.writeSetBegin(new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I32, struct.intSetColumn.size())); + for (int _iter26 : struct.intSetColumn) + { + oprot.writeI32(_iter26); + } + oprot.writeSetEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.intToStringColumn != null) { + oprot.writeFieldBegin(INT_TO_STRING_COLUMN_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRING, struct.intToStringColumn.size())); + for (Map.Entry _iter27 : struct.intToStringColumn.entrySet()) + { + oprot.writeI32(_iter27.getKey()); + oprot.writeString(_iter27.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.complexColumn != null) { + oprot.writeFieldBegin(COMPLEX_COLUMN_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.LIST, struct.complexColumn.size())); + for (Map.Entry> _iter28 : struct.complexColumn.entrySet()) + { + oprot.writeI32(_iter28.getKey()); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, _iter28.getValue().size())); + for (Nested _iter29 : _iter28.getValue()) + { + _iter29.write(oprot); + } + oprot.writeListEnd(); + } + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ParquetThriftCompatTupleSchemeFactory implements SchemeFactory { + public ParquetThriftCompatTupleScheme getScheme() { + return new ParquetThriftCompatTupleScheme(); + } + } + + private static class ParquetThriftCompatTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ParquetThriftCompat struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeBool(struct.boolColumn); + oprot.writeByte(struct.byteColumn); + oprot.writeI16(struct.shortColumn); + oprot.writeI32(struct.intColumn); + oprot.writeI64(struct.longColumn); + oprot.writeDouble(struct.doubleColumn); + oprot.writeBinary(struct.binaryColumn); + oprot.writeString(struct.stringColumn); + oprot.writeI32(struct.enumColumn.getValue()); + { + oprot.writeI32(struct.stringsColumn.size()); + for (String _iter30 : struct.stringsColumn) + { + oprot.writeString(_iter30); + } + } + { + oprot.writeI32(struct.intSetColumn.size()); + for (int _iter31 : struct.intSetColumn) + { + oprot.writeI32(_iter31); + } + } + { + oprot.writeI32(struct.intToStringColumn.size()); + for (Map.Entry _iter32 : struct.intToStringColumn.entrySet()) + { + oprot.writeI32(_iter32.getKey()); + oprot.writeString(_iter32.getValue()); + } + } + { + oprot.writeI32(struct.complexColumn.size()); + for (Map.Entry> _iter33 : struct.complexColumn.entrySet()) + { + oprot.writeI32(_iter33.getKey()); + { + oprot.writeI32(_iter33.getValue().size()); + for (Nested _iter34 : _iter33.getValue()) + { + _iter34.write(oprot); + } + } + } + } + BitSet optionals = new BitSet(); + if (struct.isSetMaybeBoolColumn()) { + optionals.set(0); + } + if (struct.isSetMaybeByteColumn()) { + optionals.set(1); + } + if (struct.isSetMaybeShortColumn()) { + optionals.set(2); + } + if (struct.isSetMaybeIntColumn()) { + optionals.set(3); + } + if (struct.isSetMaybeLongColumn()) { + optionals.set(4); + } + if (struct.isSetMaybeDoubleColumn()) { + optionals.set(5); + } + if (struct.isSetMaybeBinaryColumn()) { + optionals.set(6); + } + if (struct.isSetMaybeStringColumn()) { + optionals.set(7); + } + if (struct.isSetMaybeEnumColumn()) { + optionals.set(8); + } + oprot.writeBitSet(optionals, 9); + if (struct.isSetMaybeBoolColumn()) { + oprot.writeBool(struct.maybeBoolColumn); + } + if (struct.isSetMaybeByteColumn()) { + oprot.writeByte(struct.maybeByteColumn); + } + if (struct.isSetMaybeShortColumn()) { + oprot.writeI16(struct.maybeShortColumn); + } + if (struct.isSetMaybeIntColumn()) { + oprot.writeI32(struct.maybeIntColumn); + } + if (struct.isSetMaybeLongColumn()) { + oprot.writeI64(struct.maybeLongColumn); + } + if (struct.isSetMaybeDoubleColumn()) { + oprot.writeDouble(struct.maybeDoubleColumn); + } + if (struct.isSetMaybeBinaryColumn()) { + oprot.writeBinary(struct.maybeBinaryColumn); + } + if (struct.isSetMaybeStringColumn()) { + oprot.writeString(struct.maybeStringColumn); + } + if (struct.isSetMaybeEnumColumn()) { + oprot.writeI32(struct.maybeEnumColumn.getValue()); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ParquetThriftCompat struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.boolColumn = iprot.readBool(); + struct.setBoolColumnIsSet(true); + struct.byteColumn = iprot.readByte(); + struct.setByteColumnIsSet(true); + struct.shortColumn = iprot.readI16(); + struct.setShortColumnIsSet(true); + struct.intColumn = iprot.readI32(); + struct.setIntColumnIsSet(true); + struct.longColumn = iprot.readI64(); + struct.setLongColumnIsSet(true); + struct.doubleColumn = iprot.readDouble(); + struct.setDoubleColumnIsSet(true); + struct.binaryColumn = iprot.readBinary(); + struct.setBinaryColumnIsSet(true); + struct.stringColumn = iprot.readString(); + struct.setStringColumnIsSet(true); + struct.enumColumn = org.apache.spark.sql.parquet.test.thrift.Suit.findByValue(iprot.readI32()); + struct.setEnumColumnIsSet(true); + { + org.apache.thrift.protocol.TList _list35 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.stringsColumn = new ArrayList(_list35.size); + String _elem36; + for (int _i37 = 0; _i37 < _list35.size; ++_i37) + { + _elem36 = iprot.readString(); + struct.stringsColumn.add(_elem36); + } + } + struct.setStringsColumnIsSet(true); + { + org.apache.thrift.protocol.TSet _set38 = new org.apache.thrift.protocol.TSet(org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.intSetColumn = new HashSet(2*_set38.size); + int _elem39; + for (int _i40 = 0; _i40 < _set38.size; ++_i40) + { + _elem39 = iprot.readI32(); + struct.intSetColumn.add(_elem39); + } + } + struct.setIntSetColumnIsSet(true); + { + org.apache.thrift.protocol.TMap _map41 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.intToStringColumn = new HashMap(2*_map41.size); + int _key42; + String _val43; + for (int _i44 = 0; _i44 < _map41.size; ++_i44) + { + _key42 = iprot.readI32(); + _val43 = iprot.readString(); + struct.intToStringColumn.put(_key42, _val43); + } + } + struct.setIntToStringColumnIsSet(true); + { + org.apache.thrift.protocol.TMap _map45 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.I32, org.apache.thrift.protocol.TType.LIST, iprot.readI32()); + struct.complexColumn = new HashMap>(2*_map45.size); + int _key46; + List _val47; + for (int _i48 = 0; _i48 < _map45.size; ++_i48) + { + _key46 = iprot.readI32(); + { + org.apache.thrift.protocol.TList _list49 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + _val47 = new ArrayList(_list49.size); + Nested _elem50; + for (int _i51 = 0; _i51 < _list49.size; ++_i51) + { + _elem50 = new Nested(); + _elem50.read(iprot); + _val47.add(_elem50); + } + } + struct.complexColumn.put(_key46, _val47); + } + } + struct.setComplexColumnIsSet(true); + BitSet incoming = iprot.readBitSet(9); + if (incoming.get(0)) { + struct.maybeBoolColumn = iprot.readBool(); + struct.setMaybeBoolColumnIsSet(true); + } + if (incoming.get(1)) { + struct.maybeByteColumn = iprot.readByte(); + struct.setMaybeByteColumnIsSet(true); + } + if (incoming.get(2)) { + struct.maybeShortColumn = iprot.readI16(); + struct.setMaybeShortColumnIsSet(true); + } + if (incoming.get(3)) { + struct.maybeIntColumn = iprot.readI32(); + struct.setMaybeIntColumnIsSet(true); + } + if (incoming.get(4)) { + struct.maybeLongColumn = iprot.readI64(); + struct.setMaybeLongColumnIsSet(true); + } + if (incoming.get(5)) { + struct.maybeDoubleColumn = iprot.readDouble(); + struct.setMaybeDoubleColumnIsSet(true); + } + if (incoming.get(6)) { + struct.maybeBinaryColumn = iprot.readBinary(); + struct.setMaybeBinaryColumnIsSet(true); + } + if (incoming.get(7)) { + struct.maybeStringColumn = iprot.readString(); + struct.setMaybeStringColumnIsSet(true); + } + if (incoming.get(8)) { + struct.maybeEnumColumn = org.apache.spark.sql.parquet.test.thrift.Suit.findByValue(iprot.readI32()); + struct.setMaybeEnumColumnIsSet(true); + } + } + } + +} + diff --git a/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Suit.java b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Suit.java new file mode 100644 index 0000000000000..5315c6aae9372 --- /dev/null +++ b/sql/core/src/test/gen-java/org/apache/spark/sql/parquet/test/thrift/Suit.java @@ -0,0 +1,51 @@ +/** + * Autogenerated by Thrift Compiler (0.9.2) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.spark.sql.parquet.test.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum Suit implements org.apache.thrift.TEnum { + SPADES(0), + HEARTS(1), + DIAMONDS(2), + CLUBS(3); + + private final int value; + + private Suit(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static Suit findByValue(int value) { + switch (value) { + case 0: + return SPADES; + case 1: + return HEARTS; + case 2: + return DIAMONDS; + case 3: + return CLUBS; + default: + return null; + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetAvroCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetAvroCompatibilitySuite.scala new file mode 100644 index 0000000000000..bfa427349ff6a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetAvroCompatibilitySuite.scala @@ -0,0 +1,125 @@ +/* + * 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.parquet + +import java.nio.ByteBuffer +import java.util.{List => JList, Map => JMap} + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.fs.Path +import org.apache.parquet.avro.AvroParquetWriter + +import org.apache.spark.sql.parquet.test.avro.{Nested, ParquetAvroCompat} +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.{Row, SQLContext} + +class ParquetAvroCompatibilitySuite extends ParquetCompatibilityTest { + import ParquetCompatibilityTest._ + + override val sqlContext: SQLContext = TestSQLContext + + override protected def beforeAll(): Unit = { + super.beforeAll() + + val writer = + new AvroParquetWriter[ParquetAvroCompat]( + new Path(parquetStore.getCanonicalPath), + ParquetAvroCompat.getClassSchema) + + (0 until 10).foreach(i => writer.write(makeParquetAvroCompat(i))) + writer.close() + } + + test("Read Parquet file generated by parquet-avro") { + logInfo( + s"""Schema of the Parquet file written by parquet-avro: + |${readParquetSchema(parquetStore.getCanonicalPath)} + """.stripMargin) + + checkAnswer(sqlContext.read.parquet(parquetStore.getCanonicalPath), (0 until 10).map { i => + def nullable[T <: AnyRef]: ( => T) => T = makeNullable[T](i) + + Row( + i % 2 == 0, + i, + i.toLong * 10, + i.toFloat + 0.1f, + i.toDouble + 0.2d, + s"val_$i".getBytes, + s"val_$i", + + nullable(i % 2 == 0: java.lang.Boolean), + nullable(i: Integer), + nullable(i.toLong: java.lang.Long), + nullable(i.toFloat + 0.1f: java.lang.Float), + nullable(i.toDouble + 0.2d: java.lang.Double), + nullable(s"val_$i".getBytes), + nullable(s"val_$i"), + + Seq.tabulate(3)(n => s"arr_${i + n}"), + Seq.tabulate(3)(n => n.toString -> (i + n: Integer)).toMap, + Seq.tabulate(3) { n => + (i + n).toString -> Seq.tabulate(3) { m => + Row(Seq.tabulate(3)(j => i + j + m), s"val_${i + m}") + } + }.toMap) + }) + } + + def makeParquetAvroCompat(i: Int): ParquetAvroCompat = { + def nullable[T <: AnyRef] = makeNullable[T](i) _ + + def makeComplexColumn(i: Int): JMap[String, JList[Nested]] = { + mapAsJavaMap(Seq.tabulate(3) { n => + (i + n).toString -> seqAsJavaList(Seq.tabulate(3) { m => + Nested + .newBuilder() + .setNestedIntsColumn(seqAsJavaList(Seq.tabulate(3)(j => i + j + m))) + .setNestedStringColumn(s"val_${i + m}") + .build() + }) + }.toMap) + } + + ParquetAvroCompat + .newBuilder() + .setBoolColumn(i % 2 == 0) + .setIntColumn(i) + .setLongColumn(i.toLong * 10) + .setFloatColumn(i.toFloat + 0.1f) + .setDoubleColumn(i.toDouble + 0.2d) + .setBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes)) + .setStringColumn(s"val_$i") + + .setMaybeBoolColumn(nullable(i % 2 == 0: java.lang.Boolean)) + .setMaybeIntColumn(nullable(i: Integer)) + .setMaybeLongColumn(nullable(i.toLong: java.lang.Long)) + .setMaybeFloatColumn(nullable(i.toFloat + 0.1f: java.lang.Float)) + .setMaybeDoubleColumn(nullable(i.toDouble + 0.2d: java.lang.Double)) + .setMaybeBinaryColumn(nullable(ByteBuffer.wrap(s"val_$i".getBytes))) + .setMaybeStringColumn(nullable(s"val_$i")) + + .setStringsColumn(Seq.tabulate(3)(n => s"arr_${i + n}")) + .setStringToIntColumn( + mapAsJavaMap(Seq.tabulate(3)(n => n.toString -> (i + n: Integer)).toMap)) + .setComplexColumn(makeComplexColumn(i)) + + .build() + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala new file mode 100644 index 0000000000000..b4cdfd9e98f6f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetCompatibilityTest.scala @@ -0,0 +1,56 @@ +/* + * 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.parquet +import java.io.File + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.schema.MessageType +import org.scalatest.BeforeAndAfterAll + +import org.apache.spark.sql.QueryTest +import org.apache.spark.util.Utils + +abstract class ParquetCompatibilityTest extends QueryTest with ParquetTest with BeforeAndAfterAll { + protected var parquetStore: File = _ + + override protected def beforeAll(): Unit = { + parquetStore = Utils.createTempDir(namePrefix = "parquet-compat_") + parquetStore.delete() + } + + override protected def afterAll(): Unit = { + Utils.deleteRecursively(parquetStore) + } + + def readParquetSchema(path: String): MessageType = { + val fsPath = new Path(path) + val fs = fsPath.getFileSystem(configuration) + val parquetFiles = fs.listStatus(fsPath).toSeq.filterNot(_.getPath.getName.startsWith("_")) + val footers = ParquetFileReader.readAllFootersInParallel(configuration, parquetFiles, true) + footers.head.getParquetMetadata.getFileMetaData.getSchema + } +} + +object ParquetCompatibilityTest { + def makeNullable[T <: AnyRef](i: Int)(f: => T): T = { + if (i % 3 == 0) null.asInstanceOf[T] else f + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetThriftCompatibilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetThriftCompatibilitySuite.scala new file mode 100644 index 0000000000000..d22066cabc567 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetThriftCompatibilitySuite.scala @@ -0,0 +1,140 @@ +/* + * 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.parquet + +import java.nio.ByteBuffer +import java.util.{List => JList, Map => JMap} + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.metadata.CompressionCodecName +import org.apache.parquet.thrift.ThriftParquetWriter + +import org.apache.spark.sql.parquet.test.thrift.{Nested, ParquetThriftCompat, Suit} +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.{Row, SQLContext} + +class ParquetThriftCompatibilitySuite extends ParquetCompatibilityTest { + import ParquetCompatibilityTest._ + + override val sqlContext: SQLContext = TestSQLContext + + override protected def beforeAll(): Unit = { + super.beforeAll() + + val writer = + new ThriftParquetWriter[ParquetThriftCompat]( + new Path(parquetStore.getCanonicalPath), + classOf[ParquetThriftCompat], + CompressionCodecName.SNAPPY) + + (0 until 10).foreach(i => writer.write(makeParquetThriftCompat(i))) + writer.close() + } + + test("Read Parquet file generated by parquet-thrift") { + logInfo( + s"""Schema of the Parquet file written by parquet-thrift: + |${readParquetSchema(parquetStore.getCanonicalPath)} + """.stripMargin) + + checkAnswer(sqlContext.read.parquet(parquetStore.getCanonicalPath), (0 until 10).map { i => + def nullable[T <: AnyRef]: ( => T) => T = makeNullable[T](i) + + Row( + i % 2 == 0, + i.toByte, + (i + 1).toShort, + i + 2, + i.toLong * 10, + i.toDouble + 0.2d, + // Thrift `BINARY` values are actually unencoded `STRING` values, and thus are always + // treated as `BINARY (UTF8)` in parquet-thrift, since parquet-thrift always assume + // Thrift `STRING`s are encoded using UTF-8. + s"val_$i", + s"val_$i", + // Thrift ENUM values are converted to Parquet binaries containing UTF-8 strings + Suit.values()(i % 4).name(), + + nullable(i % 2 == 0: java.lang.Boolean), + nullable(i.toByte: java.lang.Byte), + nullable((i + 1).toShort: java.lang.Short), + nullable(i + 2: Integer), + nullable((i * 10).toLong: java.lang.Long), + nullable(i.toDouble + 0.2d: java.lang.Double), + nullable(s"val_$i"), + nullable(s"val_$i"), + nullable(Suit.values()(i % 4).name()), + + Seq.tabulate(3)(n => s"arr_${i + n}"), + // Thrift `SET`s are converted to Parquet `LIST`s + Seq(i), + Seq.tabulate(3)(n => (i + n: Integer) -> s"val_${i + n}").toMap, + Seq.tabulate(3) { n => + (i + n) -> Seq.tabulate(3) { m => + Row(Seq.tabulate(3)(j => i + j + m), s"val_${i + m}") + } + }.toMap) + }) + } + + def makeParquetThriftCompat(i: Int): ParquetThriftCompat = { + def makeComplexColumn(i: Int): JMap[Integer, JList[Nested]] = { + mapAsJavaMap(Seq.tabulate(3) { n => + (i + n: Integer) -> seqAsJavaList(Seq.tabulate(3) { m => + new Nested( + seqAsJavaList(Seq.tabulate(3)(j => i + j + m)), + s"val_${i + m}") + }) + }.toMap) + } + + val value = + new ParquetThriftCompat( + i % 2 == 0, + i.toByte, + (i + 1).toShort, + i + 2, + i.toLong * 10, + i.toDouble + 0.2d, + ByteBuffer.wrap(s"val_$i".getBytes), + s"val_$i", + Suit.values()(i % 4), + + seqAsJavaList(Seq.tabulate(3)(n => s"arr_${i + n}")), + setAsJavaSet(Set(i)), + mapAsJavaMap(Seq.tabulate(3)(n => (i + n: Integer) -> s"val_${i + n}").toMap), + makeComplexColumn(i)) + + if (i % 3 == 0) { + value + } else { + value + .setMaybeBoolColumn(i % 2 == 0) + .setMaybeByteColumn(i.toByte) + .setMaybeShortColumn((i + 1).toShort) + .setMaybeIntColumn(i + 2) + .setMaybeLongColumn(i.toLong * 10) + .setMaybeDoubleColumn(i.toDouble + 0.2d) + .setMaybeBinaryColumn(ByteBuffer.wrap(s"val_$i".getBytes)) + .setMaybeStringColumn(s"val_$i") + .setMaybeEnumColumn(Suit.values()(i % 4)) + } + } +} diff --git a/sql/core/src/test/scripts/gen-code.sh b/sql/core/src/test/scripts/gen-code.sh new file mode 100755 index 0000000000000..5d8d8ad08555c --- /dev/null +++ b/sql/core/src/test/scripts/gen-code.sh @@ -0,0 +1,31 @@ +# +# 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. +# + +cd $(dirname $0)/.. +BASEDIR=`pwd` +cd - + +rm -rf $BASEDIR/gen-java +mkdir -p $BASEDIR/gen-java + +thrift\ + --gen java\ + -out $BASEDIR/gen-java\ + $BASEDIR/thrift/parquet-compat.thrift + +avro-tools idl $BASEDIR/avro/parquet-compat.avdl > $BASEDIR/avro/parquet-compat.avpr +avro-tools compile -string protocol $BASEDIR/avro/parquet-compat.avpr $BASEDIR/gen-java diff --git a/sql/core/src/test/thrift/parquet-compat.thrift b/sql/core/src/test/thrift/parquet-compat.thrift new file mode 100644 index 0000000000000..fa5ed8c62306a --- /dev/null +++ b/sql/core/src/test/thrift/parquet-compat.thrift @@ -0,0 +1,60 @@ +/* + * 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. + */ + +namespace java org.apache.spark.sql.parquet.test.thrift + +enum Suit { + SPADES, + HEARTS, + DIAMONDS, + CLUBS +} + +struct Nested { + 1: required list nestedIntsColumn; + 2: required string nestedStringColumn; +} + +/** + * This is a test struct for testing parquet-thrift compatibility. + */ +struct ParquetThriftCompat { + 1: required bool boolColumn; + 2: required byte byteColumn; + 3: required i16 shortColumn; + 4: required i32 intColumn; + 5: required i64 longColumn; + 6: required double doubleColumn; + 7: required binary binaryColumn; + 8: required string stringColumn; + 9: required Suit enumColumn + + 10: optional bool maybeBoolColumn; + 11: optional byte maybeByteColumn; + 12: optional i16 maybeShortColumn; + 13: optional i32 maybeIntColumn; + 14: optional i64 maybeLongColumn; + 15: optional double maybeDoubleColumn; + 16: optional binary maybeBinaryColumn; + 17: optional string maybeStringColumn; + 18: optional Suit maybeEnumColumn; + + 19: required list stringsColumn; + 20: required set intSetColumn; + 21: required map intToStringColumn; + 22: required map> complexColumn; +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala new file mode 100644 index 0000000000000..bb5f1febe9ad4 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ParquetHiveCompatibilitySuite.scala @@ -0,0 +1,92 @@ +/* + * 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 + +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.parquet.ParquetCompatibilityTest +import org.apache.spark.sql.{Row, SQLConf, SQLContext} + +class ParquetHiveCompatibilitySuite extends ParquetCompatibilityTest { + import ParquetCompatibilityTest.makeNullable + + override val sqlContext: SQLContext = TestHive + + override protected def beforeAll(): Unit = { + super.beforeAll() + + withSQLConf(HiveContext.CONVERT_METASTORE_PARQUET.key -> "false") { + withTempTable("data") { + sqlContext.sql( + s"""CREATE TABLE parquet_compat( + | bool_column BOOLEAN, + | byte_column TINYINT, + | short_column SMALLINT, + | int_column INT, + | long_column BIGINT, + | float_column FLOAT, + | double_column DOUBLE, + | + | strings_column ARRAY, + | int_to_string_column MAP + |) + |STORED AS PARQUET + |LOCATION '${parquetStore.getCanonicalPath}' + """.stripMargin) + + val schema = sqlContext.table("parquet_compat").schema + val rowRDD = sqlContext.sparkContext.parallelize(makeRows).coalesce(1) + sqlContext.createDataFrame(rowRDD, schema).registerTempTable("data") + sqlContext.sql("INSERT INTO TABLE parquet_compat SELECT * FROM data") + } + } + } + + override protected def afterAll(): Unit = { + sqlContext.sql("DROP TABLE parquet_compat") + } + + test("Read Parquet file generated by parquet-hive") { + logInfo( + s"""Schema of the Parquet file written by parquet-hive: + |${readParquetSchema(parquetStore.getCanonicalPath)} + """.stripMargin) + + // Unfortunately parquet-hive doesn't add `UTF8` annotation to BINARY when writing strings. + // Have to assume all BINARY values are strings here. + withSQLConf(SQLConf.PARQUET_BINARY_AS_STRING.key -> "true") { + checkAnswer(sqlContext.read.parquet(parquetStore.getCanonicalPath), makeRows) + } + } + + def makeRows: Seq[Row] = { + (0 until 10).map { i => + def nullable[T <: AnyRef]: ( => T) => T = makeNullable[T](i) + + Row( + nullable(i % 2 == 0: java.lang.Boolean), + nullable(i.toByte: java.lang.Byte), + nullable((i + 1).toShort: java.lang.Short), + nullable(i + 2: Integer), + nullable(i.toLong * 10: java.lang.Long), + nullable(i.toFloat + 0.1f: java.lang.Float), + nullable(i.toDouble + 0.2d: java.lang.Double), + nullable(Seq.tabulate(3)(n => s"arr_${i + n}")), + nullable(Seq.tabulate(3)(n => (i + n: Integer) -> s"val_${i + n}").toMap)) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index c2e09800933b5..9d79a4b007d66 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -21,14 +21,16 @@ import java.io.File import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql._ import org.apache.spark.sql.execution.{ExecutedCommand, PhysicalRDD} import org.apache.spark.sql.hive.execution.HiveTableScan +import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.apache.spark.sql.parquet.{ParquetRelation2, ParquetTableScan} import org.apache.spark.sql.sources.{InsertIntoDataSource, InsertIntoHadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ -import org.apache.spark.sql.{DataFrame, QueryTest, Row, SQLConf, SaveMode} import org.apache.spark.util.Utils // The data where the partitioning key exists only in the directory structure. @@ -685,6 +687,31 @@ class ParquetSourceSuiteBase extends ParquetPartitioningTest { sql("drop table spark_6016_fix") } + + test("SPARK-8811: compatibility with array of struct in Hive") { + withTempPath { dir => + val path = dir.getCanonicalPath + + withTable("array_of_struct") { + val conf = Seq( + HiveContext.CONVERT_METASTORE_PARQUET.key -> "false", + SQLConf.PARQUET_BINARY_AS_STRING.key -> "true", + SQLConf.PARQUET_FOLLOW_PARQUET_FORMAT_SPEC.key -> "true") + + withSQLConf(conf: _*) { + sql( + s"""CREATE TABLE array_of_struct + |STORED AS PARQUET LOCATION '$path' + |AS SELECT '1st', '2nd', ARRAY(NAMED_STRUCT('a', 'val_a', 'b', 'val_b')) + """.stripMargin) + + checkAnswer( + sqlContext.read.parquet(path), + Row("1st", "2nd", Seq(Row("val_a", "val_b")))) + } + } + } + } } class ParquetDataSourceOnSourceSuite extends ParquetSourceSuiteBase { @@ -762,7 +789,9 @@ class ParquetDataSourceOffSourceSuite extends ParquetSourceSuiteBase { /** * A collection of tests for parquet data with various forms of partitioning. */ -abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll { +abstract class ParquetPartitioningTest extends QueryTest with SQLTestUtils with BeforeAndAfterAll { + override def sqlContext: SQLContext = TestHive + var partitionedTableDir: File = null var normalTableDir: File = null var partitionedTableDirWithKey: File = null From 381cb161ba4e3a30f2da3c4ef4ee19869d51f101 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Wed, 8 Jul 2015 16:21:28 -0700 Subject: [PATCH 65/73] [SPARK-8068] [MLLIB] Add confusionMatrix method at class MulticlassMetrics in pyspark/mllib Add confusionMatrix method at class MulticlassMetrics in pyspark/mllib Author: Yanbo Liang Closes #7286 from yanboliang/spark-8068 and squashes the following commits: 6109fe1 [Yanbo Liang] Add confusionMatrix method at class MulticlassMetrics in pyspark/mllib --- python/pyspark/mllib/evaluation.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py index c5cf3a4e7ff22..f21403707e12a 100644 --- a/python/pyspark/mllib/evaluation.py +++ b/python/pyspark/mllib/evaluation.py @@ -152,6 +152,10 @@ class MulticlassMetrics(JavaModelWrapper): >>> predictionAndLabels = sc.parallelize([(0.0, 0.0), (0.0, 1.0), (0.0, 0.0), ... (1.0, 0.0), (1.0, 1.0), (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)]) >>> metrics = MulticlassMetrics(predictionAndLabels) + >>> metrics.confusionMatrix().toArray() + array([[ 2., 1., 1.], + [ 1., 3., 0.], + [ 0., 0., 1.]]) >>> metrics.falsePositiveRate(0.0) 0.2... >>> metrics.precision(1.0) @@ -186,6 +190,13 @@ def __init__(self, predictionAndLabels): java_model = java_class(df._jdf) super(MulticlassMetrics, self).__init__(java_model) + def confusionMatrix(self): + """ + Returns confusion matrix: predicted classes are in columns, + they are ordered by class label ascending, as in "labels". + """ + return self.call("confusionMatrix") + def truePositiveRate(self, label): """ Returns true positive rate for a given label (category). From 8c32b2e870c7c250a63e838718df833edf6dea07 Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Wed, 8 Jul 2015 16:27:11 -0700 Subject: [PATCH 66/73] [SPARK-8877] [MLLIB] Public API for association rule generation Adds FPGrowth.generateAssociationRules to public API for generating association rules after mining frequent itemsets. Author: Feynman Liang Closes #7271 from feynmanliang/SPARK-8877 and squashes the following commits: 83b8baf [Feynman Liang] Add API Doc 867abff [Feynman Liang] Add FPGrowth.generateAssociationRules and change access modifiers for AssociationRules --- .../spark/mllib/fpm/AssociationRules.scala | 5 ++- .../org/apache/spark/mllib/fpm/FPGrowth.scala | 11 ++++- .../spark/mllib/fpm/FPGrowthSuite.scala | 42 +++++++++++++++++++ 3 files changed, 55 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala index 4a0f842f3338d..7e2bbfe31c1b7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/AssociationRules.scala @@ -33,7 +33,7 @@ import org.apache.spark.rdd.RDD * association rules which have a single item as the consequent. */ @Experimental -class AssociationRules private ( +class AssociationRules private[fpm] ( private var minConfidence: Double) extends Logging with Serializable { /** @@ -45,6 +45,7 @@ class AssociationRules private ( * Sets the minimal confidence (default: `0.8`). */ def setMinConfidence(minConfidence: Double): this.type = { + require(minConfidence >= 0.0 && minConfidence <= 1.0) this.minConfidence = minConfidence this } @@ -91,7 +92,7 @@ object AssociationRules { * @tparam Item item type */ @Experimental - class Rule[Item] private[mllib] ( + class Rule[Item] private[fpm] ( val antecedent: Array[Item], val consequent: Array[Item], freqUnion: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 0da59e812d5f9..9cb9a00dbd9c7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -40,7 +40,16 @@ import org.apache.spark.storage.StorageLevel * @tparam Item item type */ @Experimental -class FPGrowthModel[Item: ClassTag](val freqItemsets: RDD[FreqItemset[Item]]) extends Serializable +class FPGrowthModel[Item: ClassTag](val freqItemsets: RDD[FreqItemset[Item]]) extends Serializable { + /** + * Generates association rules for the [[Item]]s in [[freqItemsets]]. + * @param confidence minimal confidence of the rules produced + */ + def generateAssociationRules(confidence: Double): RDD[AssociationRules.Rule[Item]] = { + val associationRules = new AssociationRules(confidence) + associationRules.run(freqItemsets) + } +} /** * :: Experimental :: diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala index ddc296a428907..4a9bfdb348d9f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala @@ -132,6 +132,48 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { assert(model1.freqItemsets.count() === 625) } + test("FP-Growth String type association rule generation") { + val transactions = Seq( + "r z h k p", + "z y x w v u t s", + "s x o n r", + "x z y m t s q e", + "z", + "x z y r q t p") + .map(_.split(" ")) + val rdd = sc.parallelize(transactions, 2).cache() + + /* Verify results using the `R` code: + transactions = as(sapply( + list("r z h k p", + "z y x w v u t s", + "s x o n r", + "x z y m t s q e", + "z", + "x z y r q t p"), + FUN=function(x) strsplit(x," ",fixed=TRUE)), + "transactions") + ars = apriori(transactions, + parameter = list(support = 0.0, confidence = 0.5, target="rules", minlen=2)) + arsDF = as(ars, "data.frame") + arsDF$support = arsDF$support * length(transactions) + names(arsDF)[names(arsDF) == "support"] = "freq" + > nrow(arsDF) + [1] 23 + > sum(arsDF$confidence == 1) + [1] 23 + */ + val rules = (new FPGrowth()) + .setMinSupport(0.5) + .setNumPartitions(2) + .run(rdd) + .generateAssociationRules(0.9) + .collect() + + assert(rules.size === 23) + assert(rules.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 23) + } + test("FP-Growth using Int type") { val transactions = Seq( "1 2 3", From f472b8cdc00839780dc79be0bbe53a098cde230c Mon Sep 17 00:00:00 2001 From: Feynman Liang Date: Wed, 8 Jul 2015 16:32:00 -0700 Subject: [PATCH 67/73] [SPARK-5016] [MLLIB] Distribute GMM mixture components to executors Distribute expensive portions of computation for Gaussian mixture components (in particular, pre-computation of `MultivariateGaussian.rootSigmaInv`, the inverse covariance matrix and covariance determinant) across executors. Repost of PR#4654. Notes for reviewers: * What should be the policy for when to distribute computation. Always? When numClusters > threshold? User-specified param? TODO: * Performance testing and comparison for large number of clusters Author: Feynman Liang Closes #7166 from feynmanliang/GMM_parallel_mixtures and squashes the following commits: 4f351fa [Feynman Liang] Update heuristic and scaladoc 5ea947e [Feynman Liang] Fix parallelization logic 00eb7db [Feynman Liang] Add helper method for GMM's M step, remove distributeGaussians flag e7c8127 [Feynman Liang] Add distributeGaussians flag and tests 1da3c7f [Feynman Liang] Distribute mixtures --- .../mllib/clustering/GaussianMixture.scala | 44 +++++++++++++++---- 1 file changed, 36 insertions(+), 8 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index fc509d2ba1470..e459367333d26 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -140,6 +140,10 @@ class GaussianMixture private ( // Get length of the input vectors val d = breezeData.first().length + // Heuristic to distribute the computation of the [[MultivariateGaussian]]s, approximately when + // d > 25 except for when k is very small + val distributeGaussians = ((k - 1.0) / k) * d > 25 + // Determine initial weights and corresponding Gaussians. // If the user supplied an initial GMM, we use those values, otherwise // we start with uniform weights, a random mean from the data, and @@ -171,14 +175,25 @@ class GaussianMixture private ( // Create new distributions based on the partial assignments // (often referred to as the "M" step in literature) val sumWeights = sums.weights.sum - var i = 0 - while (i < k) { - val mu = sums.means(i) / sums.weights(i) - BLAS.syr(-sums.weights(i), Vectors.fromBreeze(mu), - Matrices.fromBreeze(sums.sigmas(i)).asInstanceOf[DenseMatrix]) - weights(i) = sums.weights(i) / sumWeights - gaussians(i) = new MultivariateGaussian(mu, sums.sigmas(i) / sums.weights(i)) - i = i + 1 + + if (distributeGaussians) { + val numPartitions = math.min(k, 1024) + val tuples = + Seq.tabulate(k)(i => (sums.means(i), sums.sigmas(i), sums.weights(i))) + val (ws, gs) = sc.parallelize(tuples, numPartitions).map { case (mean, sigma, weight) => + updateWeightsAndGaussians(mean, sigma, weight, sumWeights) + }.collect.unzip + Array.copy(ws, 0, weights, 0, ws.length) + Array.copy(gs, 0, gaussians, 0, gs.length) + } else { + var i = 0 + while (i < k) { + val (weight, gaussian) = + updateWeightsAndGaussians(sums.means(i), sums.sigmas(i), sums.weights(i), sumWeights) + weights(i) = weight + gaussians(i) = gaussian + i = i + 1 + } } llhp = llh // current becomes previous @@ -192,6 +207,19 @@ class GaussianMixture private ( /** Java-friendly version of [[run()]] */ def run(data: JavaRDD[Vector]): GaussianMixtureModel = run(data.rdd) + private def updateWeightsAndGaussians( + mean: BDV[Double], + sigma: BreezeMatrix[Double], + weight: Double, + sumWeights: Double): (Double, MultivariateGaussian) = { + val mu = (mean /= weight) + BLAS.syr(-weight, Vectors.fromBreeze(mu), + Matrices.fromBreeze(sigma).asInstanceOf[DenseMatrix]) + val newWeight = weight / sumWeights + val newGaussian = new MultivariateGaussian(mu, sigma / weight) + (newWeight, newGaussian) + } + /** Average of dense breeze vectors */ private def vectorMean(x: IndexedSeq[BV[Double]]): BDV[Double] = { val v = BDV.zeros[Double](x(0).length) From 2a4f88b6c16f2991e63b17c0e103bcd79f04dbbc Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 8 Jul 2015 18:09:39 -0700 Subject: [PATCH 68/73] [SPARK-8914][SQL] Remove RDDApi As rxin suggested in #7298 , we should consider to remove `RDDApi`. Author: Kousuke Saruta Closes #7302 from sarutak/remove-rddapi and squashes the following commits: e495d35 [Kousuke Saruta] Fixed mima cb7ebb9 [Kousuke Saruta] Removed overriding RDDApi --- project/MimaExcludes.scala | 5 ++ .../org/apache/spark/sql/DataFrame.scala | 39 ++++++----- .../scala/org/apache/spark/sql/RDDApi.scala | 67 ------------------- 3 files changed, 24 insertions(+), 87 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 7346d804632bc..57a86bf8deb64 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -70,7 +70,12 @@ object MimaExcludes { "org.apache.spark.mllib.linalg.Matrix.numNonzeros"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.mllib.linalg.Matrix.numActives") + ) ++ Seq( + // SPARK-8914 Remove RDDApi + ProblemFilters.exclude[MissingClassProblem]( + "org.apache.spark.sql.RDDApi") ) + case v if v.startsWith("1.4") => Seq( MimaBuild.excludeSparkPackage("deploy"), 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 f33e19a0cb7dd..eeefc85255d14 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 @@ -115,8 +115,7 @@ private[sql] object DataFrame { @Experimental class DataFrame private[sql]( @transient val sqlContext: SQLContext, - @DeveloperApi @transient val queryExecution: SQLContext#QueryExecution) - extends RDDApi[Row] with Serializable { + @DeveloperApi @transient val queryExecution: SQLContext#QueryExecution) extends Serializable { /** * A constructor that automatically analyzes the logical plan. @@ -1320,14 +1319,14 @@ class DataFrame private[sql]( * @group action * @since 1.3.0 */ - override def first(): Row = head() + def first(): Row = head() /** * Returns a new RDD by applying a function to all rows of this DataFrame. * @group rdd * @since 1.3.0 */ - override def map[R: ClassTag](f: Row => R): RDD[R] = rdd.map(f) + def map[R: ClassTag](f: Row => R): RDD[R] = rdd.map(f) /** * Returns a new RDD by first applying a function to all rows of this [[DataFrame]], @@ -1335,14 +1334,14 @@ class DataFrame private[sql]( * @group rdd * @since 1.3.0 */ - override def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] = rdd.flatMap(f) + def flatMap[R: ClassTag](f: Row => TraversableOnce[R]): RDD[R] = rdd.flatMap(f) /** * Returns a new RDD by applying a function to each partition of this DataFrame. * @group rdd * @since 1.3.0 */ - override def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] = { + def mapPartitions[R: ClassTag](f: Iterator[Row] => Iterator[R]): RDD[R] = { rdd.mapPartitions(f) } @@ -1351,49 +1350,49 @@ class DataFrame private[sql]( * @group rdd * @since 1.3.0 */ - override def foreach(f: Row => Unit): Unit = rdd.foreach(f) + def foreach(f: Row => Unit): Unit = rdd.foreach(f) /** * Applies a function f to each partition of this [[DataFrame]]. * @group rdd * @since 1.3.0 */ - override def foreachPartition(f: Iterator[Row] => Unit): Unit = rdd.foreachPartition(f) + def foreachPartition(f: Iterator[Row] => Unit): Unit = rdd.foreachPartition(f) /** * Returns the first `n` rows in the [[DataFrame]]. * @group action * @since 1.3.0 */ - override def take(n: Int): Array[Row] = head(n) + def take(n: Int): Array[Row] = head(n) /** * Returns an array that contains all of [[Row]]s in this [[DataFrame]]. * @group action * @since 1.3.0 */ - override def collect(): Array[Row] = queryExecution.executedPlan.executeCollect() + def collect(): Array[Row] = queryExecution.executedPlan.executeCollect() /** * Returns a Java list that contains all of [[Row]]s in this [[DataFrame]]. * @group action * @since 1.3.0 */ - override def collectAsList(): java.util.List[Row] = java.util.Arrays.asList(rdd.collect() : _*) + def collectAsList(): java.util.List[Row] = java.util.Arrays.asList(rdd.collect() : _*) /** * Returns the number of rows in the [[DataFrame]]. * @group action * @since 1.3.0 */ - override def count(): Long = groupBy().count().collect().head.getLong(0) + def count(): Long = groupBy().count().collect().head.getLong(0) /** * Returns a new [[DataFrame]] that has exactly `numPartitions` partitions. * @group rdd * @since 1.3.0 */ - override def repartition(numPartitions: Int): DataFrame = { + def repartition(numPartitions: Int): DataFrame = { Repartition(numPartitions, shuffle = true, logicalPlan) } @@ -1405,7 +1404,7 @@ class DataFrame private[sql]( * @group rdd * @since 1.4.0 */ - override def coalesce(numPartitions: Int): DataFrame = { + def coalesce(numPartitions: Int): DataFrame = { Repartition(numPartitions, shuffle = false, logicalPlan) } @@ -1415,13 +1414,13 @@ class DataFrame private[sql]( * @group dfops * @since 1.3.0 */ - override def distinct(): DataFrame = dropDuplicates() + def distinct(): DataFrame = dropDuplicates() /** * @group basic * @since 1.3.0 */ - override def persist(): this.type = { + def persist(): this.type = { sqlContext.cacheManager.cacheQuery(this) this } @@ -1430,13 +1429,13 @@ class DataFrame private[sql]( * @group basic * @since 1.3.0 */ - override def cache(): this.type = persist() + def cache(): this.type = persist() /** * @group basic * @since 1.3.0 */ - override def persist(newLevel: StorageLevel): this.type = { + def persist(newLevel: StorageLevel): this.type = { sqlContext.cacheManager.cacheQuery(this, None, newLevel) this } @@ -1445,7 +1444,7 @@ class DataFrame private[sql]( * @group basic * @since 1.3.0 */ - override def unpersist(blocking: Boolean): this.type = { + def unpersist(blocking: Boolean): this.type = { sqlContext.cacheManager.tryUncacheQuery(this, blocking) this } @@ -1454,7 +1453,7 @@ class DataFrame private[sql]( * @group basic * @since 1.3.0 */ - override def unpersist(): this.type = unpersist(blocking = false) + def unpersist(): this.type = unpersist(blocking = false) ///////////////////////////////////////////////////////////////////////////// // I/O diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala deleted file mode 100644 index 63dbab19947c0..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala +++ /dev/null @@ -1,67 +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 - -import scala.reflect.ClassTag - -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel - - -/** - * An internal interface defining the RDD-like methods for [[DataFrame]]. - * Please use [[DataFrame]] directly, and do NOT use this. - */ -private[sql] trait RDDApi[T] { - - def cache(): this.type - - def persist(): this.type - - def persist(newLevel: StorageLevel): this.type - - def unpersist(): this.type - - def unpersist(blocking: Boolean): this.type - - def map[R: ClassTag](f: T => R): RDD[R] - - def flatMap[R: ClassTag](f: T => TraversableOnce[R]): RDD[R] - - def mapPartitions[R: ClassTag](f: Iterator[T] => Iterator[R]): RDD[R] - - def foreach(f: T => Unit): Unit - - def foreachPartition(f: Iterator[T] => Unit): Unit - - def take(n: Int): Array[T] - - def collect(): Array[T] - - def collectAsList(): java.util.List[T] - - def count(): Long - - def first(): T - - def repartition(numPartitions: Int): DataFrame - - def coalesce(numPartitions: Int): DataFrame - - def distinct: DataFrame -} From 74d8d3d928cc9a7386b68588ac89ae042847d146 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 8 Jul 2015 18:22:53 -0700 Subject: [PATCH 69/73] [SPARK-8450] [SQL] [PYSARK] cleanup type converter for Python DataFrame This PR fixes the converter for Python DataFrame, especially for DecimalType Closes #7106 Author: Davies Liu Closes #7131 from davies/decimal_python and squashes the following commits: 4d3c234 [Davies Liu] Merge branch 'master' of github.com:apache/spark into decimal_python 20531d6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into decimal_python 7d73168 [Davies Liu] fix conflit 6cdd86a [Davies Liu] Merge branch 'master' of github.com:apache/spark into decimal_python 7104e97 [Davies Liu] improve type infer 9cd5a21 [Davies Liu] run python tests with SPARK_PREPEND_CLASSES 829a05b [Davies Liu] fix UDT in python c99e8c5 [Davies Liu] fix mima c46814a [Davies Liu] convert decimal for Python DataFrames --- .../apache/spark/mllib/linalg/Matrices.scala | 10 +- .../apache/spark/mllib/linalg/Vectors.scala | 16 +--- project/MimaExcludes.scala | 5 +- python/pyspark/sql/tests.py | 13 +++ python/pyspark/sql/types.py | 4 + python/run-tests.py | 3 +- .../org/apache/spark/sql/DataFrame.scala | 4 +- .../org/apache/spark/sql/SQLContext.scala | 28 +----- .../spark/sql/execution/pythonUDFs.scala | 95 ++++++++++--------- 9 files changed, 84 insertions(+), 94 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 75e7004464af9..0df07663405a3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -24,9 +24,9 @@ import scala.collection.mutable.{ArrayBuilder => MArrayBuilder, HashSet => MHash import breeze.linalg.{CSCMatrix => BSM, DenseMatrix => BDM, Matrix => BM} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.Row -import org.apache.spark.sql.types._ import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types._ /** * Trait for a local matrix. @@ -147,7 +147,7 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { )) } - override def serialize(obj: Any): Row = { + override def serialize(obj: Any): InternalRow = { val row = new GenericMutableRow(7) obj match { case sm: SparseMatrix => @@ -173,9 +173,7 @@ private[spark] class MatrixUDT extends UserDefinedType[Matrix] { override def deserialize(datum: Any): Matrix = { datum match { - // TODO: something wrong with UDT serialization, should never happen. - case m: Matrix => m - case row: Row => + case row: InternalRow => require(row.length == 7, s"MatrixUDT.deserialize given row with length ${row.length} but requires length == 7") val tpe = row.getByte(0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index c9c27425d2877..e048b01d92462 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -28,7 +28,7 @@ import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, Vector => BV} import org.apache.spark.SparkException import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.util.NumericParser -import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.types._ @@ -175,7 +175,7 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] { StructField("values", ArrayType(DoubleType, containsNull = false), nullable = true))) } - override def serialize(obj: Any): Row = { + override def serialize(obj: Any): InternalRow = { obj match { case SparseVector(size, indices, values) => val row = new GenericMutableRow(4) @@ -191,17 +191,12 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] { row.setNullAt(2) row.update(3, values.toSeq) row - // TODO: There are bugs in UDT serialization because we don't have a clear separation between - // TODO: internal SQL types and language specific types (including UDT). UDT serialize and - // TODO: deserialize may get called twice. See SPARK-7186. - case row: Row => - row } } override def deserialize(datum: Any): Vector = { datum match { - case row: Row => + case row: InternalRow => require(row.length == 4, s"VectorUDT.deserialize given row with length ${row.length} but requires length == 4") val tpe = row.getByte(0) @@ -215,11 +210,6 @@ private[spark] class VectorUDT extends UserDefinedType[Vector] { val values = row.getAs[Iterable[Double]](3).toArray new DenseVector(values) } - // TODO: There are bugs in UDT serialization because we don't have a clear separation between - // TODO: internal SQL types and language specific types (including UDT). UDT serialize and - // TODO: deserialize may get called twice. See SPARK-7186. - case v: Vector => - v } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 57a86bf8deb64..821aadd477ef3 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -63,7 +63,10 @@ object MimaExcludes { // SQL execution is considered private. excludePackage("org.apache.spark.sql.execution"), // Parquet support is considered private. - excludePackage("org.apache.spark.sql.parquet") + excludePackage("org.apache.spark.sql.parquet"), + // local function inside a method + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.sql.SQLContext.org$apache$spark$sql$SQLContext$$needsConversion$1") ) ++ Seq( // SPARK-8479 Add numNonzeros and numActives to Matrix. ProblemFilters.exclude[MissingMethodProblem]( diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 333378c7f1854..66827d48850d9 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -700,6 +700,19 @@ def test_time_with_timezone(self): self.assertTrue(now - now1 < datetime.timedelta(0.001)) self.assertTrue(now - utcnow1 < datetime.timedelta(0.001)) + def test_decimal(self): + from decimal import Decimal + schema = StructType([StructField("decimal", DecimalType(10, 5))]) + df = self.sqlCtx.createDataFrame([(Decimal("3.14159"),)], schema) + row = df.select(df.decimal + 1).first() + self.assertEqual(row[0], Decimal("4.14159")) + tmpPath = tempfile.mkdtemp() + shutil.rmtree(tmpPath) + df.write.parquet(tmpPath) + df2 = self.sqlCtx.read.parquet(tmpPath) + row = df2.first() + self.assertEqual(row[0], Decimal("3.14159")) + def test_dropna(self): schema = StructType([ StructField("name", StringType(), True), diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 160df40d65cc1..7e64cb0b54dba 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1069,6 +1069,10 @@ def _verify_type(obj, dataType): if obj is None: return + # StringType can work with any types + if isinstance(dataType, StringType): + return + if isinstance(dataType, UserDefinedType): if not (hasattr(obj, '__UDT__') and obj.__UDT__ == dataType): raise ValueError("%r is not an instance of type %r" % (obj, dataType)) diff --git a/python/run-tests.py b/python/run-tests.py index 7638854def2e8..cc560779373b3 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -72,7 +72,8 @@ def print_red(text): def run_individual_python_test(test_name, pyspark_python): - env = {'SPARK_TESTING': '1', 'PYSPARK_PYTHON': which(pyspark_python)} + env = dict(os.environ) + env.update({'SPARK_TESTING': '1', 'PYSPARK_PYTHON': which(pyspark_python)}) LOGGER.debug("Starting test(%s): %s", pyspark_python, test_name) start_time = time.time() try: 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 eeefc85255d14..d9f987ae0252f 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 @@ -1549,8 +1549,8 @@ class DataFrame private[sql]( * Converts a JavaRDD to a PythonRDD. */ protected[sql] def javaToPython: JavaRDD[Array[Byte]] = { - val fieldTypes = schema.fields.map(_.dataType) - val jrdd = rdd.map(EvaluatePython.rowToArray(_, fieldTypes)).toJavaRDD() + val structType = schema // capture it for closure + val jrdd = queryExecution.toRdd.map(EvaluatePython.toJava(_, structType)).toJavaRDD() SerDeUtil.javaToPython(jrdd) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 079f31ab8fe6d..477dea9164726 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -1044,33 +1044,7 @@ class SQLContext(@transient val sparkContext: SparkContext) rdd: RDD[Array[Any]], schema: StructType): DataFrame = { - def needsConversion(dataType: DataType): Boolean = dataType match { - case ByteType => true - case ShortType => true - case LongType => true - case FloatType => true - case DateType => true - case TimestampType => true - case StringType => true - case ArrayType(_, _) => true - case MapType(_, _, _) => true - case StructType(_) => true - case udt: UserDefinedType[_] => needsConversion(udt.sqlType) - case other => false - } - - val convertedRdd = if (schema.fields.exists(f => needsConversion(f.dataType))) { - rdd.map(m => m.zip(schema.fields).map { - case (value, field) => EvaluatePython.fromJava(value, field.dataType) - }) - } else { - rdd - } - - val rowRdd = convertedRdd.mapPartitions { iter => - iter.map { m => new GenericInternalRow(m): InternalRow} - } - + val rowRdd = rdd.map(r => EvaluatePython.fromJava(r, schema).asInstanceOf[InternalRow]) DataFrame(this, LogicalRDD(schema.toAttributes, rowRdd)(self)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala index 6946e798b71b0..1c8130b07c7fb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUDFs.scala @@ -24,20 +24,19 @@ import scala.collection.JavaConverters._ import net.razorvine.pickle.{Pickler, Unpickler} -import org.apache.spark.{Accumulator, Logging => SparkLogging} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.{PythonBroadcast, PythonRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.{Accumulator, Logging => SparkLogging} /** * A serialized version of a Python lambda function. Suitable for use in a [[PythonRDD]]. @@ -125,59 +124,86 @@ object EvaluatePython { new EvaluatePython(udf, child, AttributeReference("pythonUDF", udf.dataType)()) /** - * Helper for converting a Scala object to a java suitable for pyspark serialization. + * Helper for converting from Catalyst type to java type suitable for Pyrolite. */ def toJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { case (null, _) => null - case (row: Row, struct: StructType) => + case (row: InternalRow, struct: StructType) => val fields = struct.fields.map(field => field.dataType) - row.toSeq.zip(fields).map { - case (obj, dataType) => toJava(obj, dataType) - }.toArray + rowToArray(row, fields) case (seq: Seq[Any], array: ArrayType) => seq.map(x => toJava(x, array.elementType)).asJava - case (list: JList[_], array: ArrayType) => - list.map(x => toJava(x, array.elementType)).asJava - case (arr, array: ArrayType) if arr.getClass.isArray => - arr.asInstanceOf[Array[Any]].map(x => toJava(x, array.elementType)) case (obj: Map[_, _], mt: MapType) => obj.map { case (k, v) => (toJava(k, mt.keyType), toJava(v, mt.valueType)) }.asJava - case (ud, udt: UserDefinedType[_]) => toJava(udt.serialize(ud), udt.sqlType) + case (ud, udt: UserDefinedType[_]) => toJava(ud, udt.sqlType) case (date: Int, DateType) => DateTimeUtils.toJavaDate(date) case (t: Long, TimestampType) => DateTimeUtils.toJavaTimestamp(t) + + case (d: Decimal, _) => d.toJavaBigDecimal + case (s: UTF8String, StringType) => s.toString - // Pyrolite can handle Timestamp and Decimal case (other, _) => other } /** * Convert Row into Java Array (for pickled into Python) */ - def rowToArray(row: Row, fields: Seq[DataType]): Array[Any] = { + def rowToArray(row: InternalRow, fields: Seq[DataType]): Array[Any] = { // TODO: this is slow! row.toSeq.zip(fields).map {case (obj, dt) => toJava(obj, dt)}.toArray } - // Converts value to the type specified by the data type. - // Because Python does not have data types for TimestampType, FloatType, ShortType, and - // ByteType, we need to explicitly convert values in columns of these data types to the desired - // JVM data types. + /** + * Converts `obj` to the type specified by the data type, or returns null if the type of obj is + * unexpected. Because Python doesn't enforce the type. + */ def fromJava(obj: Any, dataType: DataType): Any = (obj, dataType) match { - // TODO: We should check nullable case (null, _) => null + case (c: Boolean, BooleanType) => c + + case (c: Int, ByteType) => c.toByte + case (c: Long, ByteType) => c.toByte + + case (c: Int, ShortType) => c.toShort + case (c: Long, ShortType) => c.toShort + + case (c: Int, IntegerType) => c + case (c: Long, IntegerType) => c.toInt + + case (c: Int, LongType) => c.toLong + case (c: Long, LongType) => c + + case (c: Double, FloatType) => c.toFloat + + case (c: Double, DoubleType) => c + + case (c: java.math.BigDecimal, dt: DecimalType) => Decimal(c) + + case (c: Int, DateType) => c + + case (c: Long, TimestampType) => c + + case (c: String, StringType) => UTF8String.fromString(c) + case (c, StringType) => + // If we get here, c is not a string. Call toString on it. + UTF8String.fromString(c.toString) + + case (c: String, BinaryType) => c.getBytes("utf-8") + case (c, BinaryType) if c.getClass.isArray && c.getClass.getComponentType.getName == "byte" => c + case (c: java.util.List[_], ArrayType(elementType, _)) => - c.map { e => fromJava(e, elementType)}: Seq[Any] + c.map { e => fromJava(e, elementType)}.toSeq case (c, ArrayType(elementType, _)) if c.getClass.isArray => - c.asInstanceOf[Array[_]].map(e => fromJava(e, elementType)): Seq[Any] + c.asInstanceOf[Array[_]].map(e => fromJava(e, elementType)).toSeq case (c: java.util.Map[_, _], MapType(keyType, valueType, _)) => c.map { case (key, value) => (fromJava(key, keyType), fromJava(value, valueType)) @@ -188,30 +214,11 @@ object EvaluatePython { case (e, f) => fromJava(e, f.dataType) }) - case (c: java.util.Calendar, DateType) => - DateTimeUtils.fromJavaDate(new java.sql.Date(c.getTimeInMillis)) - - case (c: java.util.Calendar, TimestampType) => - c.getTimeInMillis * 10000L - case (t: java.sql.Timestamp, TimestampType) => - DateTimeUtils.fromJavaTimestamp(t) - - case (_, udt: UserDefinedType[_]) => - fromJava(obj, udt.sqlType) - - case (c: Int, ByteType) => c.toByte - case (c: Long, ByteType) => c.toByte - case (c: Int, ShortType) => c.toShort - case (c: Long, ShortType) => c.toShort - case (c: Long, IntegerType) => c.toInt - case (c: Int, LongType) => c.toLong - case (c: Double, FloatType) => c.toFloat - case (c: String, StringType) => UTF8String.fromString(c) - case (c, StringType) => - // If we get here, c is not a string. Call toString on it. - UTF8String.fromString(c.toString) + case (_, udt: UserDefinedType[_]) => fromJava(obj, udt.sqlType) - case (c, _) => c + // all other unexpected type should be null, or we will have runtime exception + // TODO(davies): we could improve this by try to cast the object to expected type + case (c, _) => null } } From 28fa01e2ba146e823489f6d81c5eb3a76b20c71f Mon Sep 17 00:00:00 2001 From: Jonathan Alter Date: Thu, 9 Jul 2015 03:28:51 +0100 Subject: [PATCH 70/73] [SPARK-8927] [DOCS] Format wrong for some config descriptions A couple descriptions were not inside `` and were being displayed immediately under the section title instead of in their row. Author: Jonathan Alter Closes #7292 from jonalter/docs-config and squashes the following commits: 5ce1570 [Jonathan Alter] [DOCS] Format wrong for some config descriptions --- docs/configuration.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index bebaf6f62e90a..892c02b27df32 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1007,9 +1007,9 @@ Apart from these, the following properties are also available, and may be useful spark.rpc.numRetries 3 + Number of times to retry before an RPC task gives up. An RPC task will run at most times of this number. - @@ -1029,8 +1029,8 @@ Apart from these, the following properties are also available, and may be useful spark.rpc.lookupTimeout 120s - Duration for an RPC remote endpoint lookup operation to wait before timing out. + Duration for an RPC remote endpoint lookup operation to wait before timing out. From a290814877308c6fa9b0f78b1a81145db7651ca4 Mon Sep 17 00:00:00 2001 From: Yijie Shen Date: Wed, 8 Jul 2015 20:20:17 -0700 Subject: [PATCH 71/73] [SPARK-8866][SQL] use 1us precision for timestamp type JIRA: https://issues.apache.org/jira/browse/SPARK-8866 Author: Yijie Shen Closes #7283 from yijieshen/micro_timestamp and squashes the following commits: dc735df [Yijie Shen] update CastSuite to avoid round error 714eaea [Yijie Shen] add timestamp_udf into blacklist due to precision lose c3ca2f4 [Yijie Shen] fix unhandled case in CurrentTimestamp 8d4aa6b [Yijie Shen] use 1us precision for timestamp type --- python/pyspark/sql/types.py | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 18 ++++----- .../expressions/datetimeFunctions.scala | 2 +- .../sql/catalyst/util/DateTimeUtils.scala | 38 +++++++++---------- .../sql/catalyst/expressions/CastSuite.scala | 10 ++--- .../catalyst/util/DateTimeUtilsSuite.scala | 8 ++-- .../apache/spark/sql/json/JacksonParser.scala | 4 +- .../org/apache/spark/sql/json/JsonRDD.scala | 6 +-- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 2 +- .../execution/HiveCompatibilitySuite.scala | 6 +-- .../spark/sql/hive/HiveInspectors.scala | 4 +- 11 files changed, 50 insertions(+), 50 deletions(-) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 7e64cb0b54dba..fecfe6d71e9a7 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -775,7 +775,7 @@ def to_posix_timstamp(dt): if dt: seconds = (calendar.timegm(dt.utctimetuple()) if dt.tzinfo else time.mktime(dt.timetuple())) - return int(seconds * 1e7 + dt.microsecond * 10) + return int(seconds * 1e6 + dt.microsecond) return to_posix_timstamp else: 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 662ceeca7782d..567feca7136f9 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 @@ -186,7 +186,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case ByteType => buildCast[Byte](_, b => longToTimestamp(b.toLong)) case DateType => - buildCast[Int](_, d => DateTimeUtils.daysToMillis(d) * 10000) + buildCast[Int](_, d => DateTimeUtils.daysToMillis(d) * 1000) // TimestampWritable.decimalToTimestamp case DecimalType() => buildCast[Decimal](_, d => decimalToTimestamp(d)) @@ -207,16 +207,16 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w } private[this] def decimalToTimestamp(d: Decimal): Long = { - (d.toBigDecimal * 10000000L).longValue() + (d.toBigDecimal * 1000000L).longValue() } - // converting milliseconds to 100ns - private[this] def longToTimestamp(t: Long): Long = t * 10000L - // converting 100ns to seconds - private[this] def timestampToLong(ts: Long): Long = math.floor(ts.toDouble / 10000000L).toLong - // converting 100ns to seconds in double + // converting milliseconds to us + private[this] def longToTimestamp(t: Long): Long = t * 1000L + // converting us to seconds + private[this] def timestampToLong(ts: Long): Long = math.floor(ts.toDouble / 1000000L).toLong + // converting us to seconds in double private[this] def timestampToDouble(ts: Long): Double = { - ts / 10000000.0 + ts / 1000000.0 } // DateConverter @@ -229,7 +229,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w case TimestampType => // throw valid precision more than seconds, according to Hive. // Timestamp.nanos is in 0 to 999,999,999, no more than a second. - buildCast[Long](_, t => DateTimeUtils.millisToDays(t / 10000L)) + buildCast[Long](_, t => DateTimeUtils.millisToDays(t / 1000L)) // Hive throws this exception as a Semantic Exception // It is never possible to compare result when hive return with exception, // so we can return null diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala index a492b966a5e31..dd5ec330a771b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeFunctions.scala @@ -51,6 +51,6 @@ case class CurrentTimestamp() extends LeafExpression { override def dataType: DataType = TimestampType override def eval(input: InternalRow): Any = { - System.currentTimeMillis() * 10000L + System.currentTimeMillis() * 1000L } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala index 4269ad5d56737..c1ddee3ef0230 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala @@ -34,8 +34,8 @@ object DateTimeUtils { // see http://stackoverflow.com/questions/466321/convert-unix-timestamp-to-julian final val JULIAN_DAY_OF_EPOCH = 2440587 // and .5 final val SECONDS_PER_DAY = 60 * 60 * 24L - final val HUNDRED_NANOS_PER_SECOND = 1000L * 1000L * 10L - final val NANOS_PER_SECOND = HUNDRED_NANOS_PER_SECOND * 100 + final val MICROS_PER_SECOND = 1000L * 1000L + final val NANOS_PER_SECOND = MICROS_PER_SECOND * 1000L // Java TimeZone has no mention of thread safety. Use thread local instance to be safe. @@ -77,8 +77,8 @@ object DateTimeUtils { threadLocalDateFormat.get.format(toJavaDate(days)) // Converts Timestamp to string according to Hive TimestampWritable convention. - def timestampToString(num100ns: Long): String = { - val ts = toJavaTimestamp(num100ns) + def timestampToString(us: Long): String = { + val ts = toJavaTimestamp(us) val timestampString = ts.toString val formatted = threadLocalTimestampFormat.get.format(ts) @@ -132,52 +132,52 @@ object DateTimeUtils { } /** - * Returns a java.sql.Timestamp from number of 100ns since epoch. + * Returns a java.sql.Timestamp from number of micros since epoch. */ - def toJavaTimestamp(num100ns: Long): Timestamp = { + def toJavaTimestamp(us: Long): Timestamp = { // setNanos() will overwrite the millisecond part, so the milliseconds should be // cut off at seconds - var seconds = num100ns / HUNDRED_NANOS_PER_SECOND - var nanos = num100ns % HUNDRED_NANOS_PER_SECOND + var seconds = us / MICROS_PER_SECOND + var micros = us % MICROS_PER_SECOND // setNanos() can not accept negative value - if (nanos < 0) { - nanos += HUNDRED_NANOS_PER_SECOND + if (micros < 0) { + micros += MICROS_PER_SECOND seconds -= 1 } val t = new Timestamp(seconds * 1000) - t.setNanos(nanos.toInt * 100) + t.setNanos(micros.toInt * 1000) t } /** - * Returns the number of 100ns since epoch from java.sql.Timestamp. + * Returns the number of micros since epoch from java.sql.Timestamp. */ def fromJavaTimestamp(t: Timestamp): Long = { if (t != null) { - t.getTime() * 10000L + (t.getNanos().toLong / 100) % 10000L + t.getTime() * 1000L + (t.getNanos().toLong / 1000) % 1000L } else { 0L } } /** - * Returns the number of 100ns (hundred of nanoseconds) since epoch from Julian day + * Returns the number of microseconds since epoch from Julian day * and nanoseconds in a day */ def fromJulianDay(day: Int, nanoseconds: Long): Long = { // use Long to avoid rounding errors val seconds = (day - JULIAN_DAY_OF_EPOCH).toLong * SECONDS_PER_DAY - SECONDS_PER_DAY / 2 - seconds * HUNDRED_NANOS_PER_SECOND + nanoseconds / 100L + seconds * MICROS_PER_SECOND + nanoseconds / 1000L } /** - * Returns Julian day and nanoseconds in a day from the number of 100ns (hundred of nanoseconds) + * Returns Julian day and nanoseconds in a day from the number of microseconds */ - def toJulianDay(num100ns: Long): (Int, Long) = { - val seconds = num100ns / HUNDRED_NANOS_PER_SECOND + SECONDS_PER_DAY / 2 + def toJulianDay(us: Long): (Int, Long) = { + val seconds = us / MICROS_PER_SECOND + SECONDS_PER_DAY / 2 val day = seconds / SECONDS_PER_DAY + JULIAN_DAY_OF_EPOCH val secondsInDay = seconds % SECONDS_PER_DAY - val nanos = (num100ns % HUNDRED_NANOS_PER_SECOND) * 100L + val nanos = (us % MICROS_PER_SECOND) * 1000L (day.toInt, secondsInDay * NANOS_PER_SECOND + nanos) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala index 518961e38396f..919fdd470b79a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -293,15 +293,15 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } test("cast from timestamp") { - val millis = 15 * 1000 + 2 - val seconds = millis * 1000 + 2 + val millis = 15 * 1000 + 3 + val seconds = millis * 1000 + 3 val ts = new Timestamp(millis) val tss = new Timestamp(seconds) checkEvaluation(cast(ts, ShortType), 15.toShort) checkEvaluation(cast(ts, IntegerType), 15) checkEvaluation(cast(ts, LongType), 15.toLong) - checkEvaluation(cast(ts, FloatType), 15.002f) - checkEvaluation(cast(ts, DoubleType), 15.002) + checkEvaluation(cast(ts, FloatType), 15.003f) + checkEvaluation(cast(ts, DoubleType), 15.003) checkEvaluation(cast(cast(tss, ShortType), TimestampType), DateTimeUtils.fromJavaTimestamp(ts)) checkEvaluation(cast(cast(tss, IntegerType), TimestampType), DateTimeUtils.fromJavaTimestamp(ts)) @@ -317,7 +317,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { Decimal(1)) // A test for higher precision than millis - checkEvaluation(cast(cast(0.0000001, TimestampType), DoubleType), 0.0000001) + checkEvaluation(cast(cast(0.000001, TimestampType), DoubleType), 0.000001) checkEvaluation(cast(Double.NaN, TimestampType), null) checkEvaluation(cast(1.0 / 0.0, TimestampType), null) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala index 1d4a60c81efc5..f63ac191e7366 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeUtilsSuite.scala @@ -24,11 +24,11 @@ import org.apache.spark.SparkFunSuite class DateTimeUtilsSuite extends SparkFunSuite { - test("timestamp and 100ns") { + test("timestamp and us") { val now = new Timestamp(System.currentTimeMillis()) - now.setNanos(100) + now.setNanos(1000) val ns = DateTimeUtils.fromJavaTimestamp(now) - assert(ns % 10000000L === 1) + assert(ns % 1000000L === 1) assert(DateTimeUtils.toJavaTimestamp(ns) === now) List(-111111111111L, -1L, 0, 1L, 111111111111L).foreach { t => @@ -38,7 +38,7 @@ class DateTimeUtilsSuite extends SparkFunSuite { } } - test("100ns and julian day") { + test("us and julian day") { val (d, ns) = DateTimeUtils.toJulianDay(0) assert(d === DateTimeUtils.JULIAN_DAY_OF_EPOCH) assert(ns === DateTimeUtils.SECONDS_PER_DAY / 2 * DateTimeUtils.NANOS_PER_SECOND) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala index 4b8ab63b5ab39..381e7ed54428f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala @@ -67,10 +67,10 @@ private[sql] object JacksonParser { DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(parser.getText).getTime) case (VALUE_STRING, TimestampType) => - DateTimeUtils.stringToTime(parser.getText).getTime * 10000L + DateTimeUtils.stringToTime(parser.getText).getTime * 1000L case (VALUE_NUMBER_INT, TimestampType) => - parser.getLongValue * 10000L + parser.getLongValue * 1000L case (_, StringType) => val writer = new ByteArrayOutputStream() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 01ba05cbd14f1..b392a51bf7dce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -401,9 +401,9 @@ private[sql] object JsonRDD extends Logging { private def toTimestamp(value: Any): Long = { value match { - case value: java.lang.Integer => value.asInstanceOf[Int].toLong * 10000L - case value: java.lang.Long => value * 10000L - case value: java.lang.String => DateTimeUtils.stringToTime(value).getTime * 10000L + case value: java.lang.Integer => value.asInstanceOf[Int].toLong * 1000L + case value: java.lang.Long => value * 1000L + case value: java.lang.String => DateTimeUtils.stringToTime(value).getTime * 1000L } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 69ab1c292d221..566a52dc1b784 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -326,7 +326,7 @@ class JDBCSuite extends SparkFunSuite with BeforeAndAfter { assert(cal.get(Calendar.HOUR) === 11) assert(cal.get(Calendar.MINUTE) === 22) assert(cal.get(Calendar.SECOND) === 33) - assert(rows(0).getAs[java.sql.Timestamp](2).getNanos === 543543500) + assert(rows(0).getAs[java.sql.Timestamp](2).getNanos === 543543000) } test("test DATE types") { diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 415a81644c58f..c884c399281a8 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -254,9 +254,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // the answer is sensitive for jdk version "udf_java_method", - // Spark SQL use Long for TimestampType, lose the precision under 100ns + // Spark SQL use Long for TimestampType, lose the precision under 1us "timestamp_1", - "timestamp_2" + "timestamp_2", + "timestamp_udf" ) /** @@ -803,7 +804,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "timestamp_comparison", "timestamp_lazy", "timestamp_null", - "timestamp_udf", "touch", "transform_ppr1", "transform_ppr2", diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 4cba17524af6c..a8f2ee37cb8ed 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -267,7 +267,7 @@ private[hive] trait HiveInspectors { poi.getWritableConstantValue.getHiveDecimal) case poi: WritableConstantTimestampObjectInspector => val t = poi.getWritableConstantValue - t.getSeconds * 10000000L + t.getNanos / 100L + t.getSeconds * 1000000L + t.getNanos / 1000L case poi: WritableConstantIntObjectInspector => poi.getWritableConstantValue.get() case poi: WritableConstantDoubleObjectInspector => @@ -332,7 +332,7 @@ private[hive] trait HiveInspectors { case x: DateObjectInspector => DateTimeUtils.fromJavaDate(x.getPrimitiveJavaObject(data)) case x: TimestampObjectInspector if x.preferWritable() => val t = x.getPrimitiveWritableObject(data) - t.getSeconds * 10000000L + t.getNanos / 100 + t.getSeconds * 1000000L + t.getNanos / 1000L case ti: TimestampObjectInspector => DateTimeUtils.fromJavaTimestamp(ti.getPrimitiveJavaObject(data)) case _ => pi.getPrimitiveJavaObject(data) From b55499a44ab74e33378211fb0d6940905d7c6318 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Wed, 8 Jul 2015 20:28:05 -0700 Subject: [PATCH 72/73] [SPARK-8932] Support copy() for UnsafeRows that do not use ObjectPools We call Row.copy() in many places throughout SQL but UnsafeRow currently throws UnsupportedOperationException when copy() is called. Supporting copying when ObjectPool is used may be difficult, since we may need to handle deep-copying of objects in the pool. In addition, this copy() method needs to produce a self-contained row object which may be passed around / buffered by downstream code which does not understand the UnsafeRow format. In the long run, we'll need to figure out how to handle the ObjectPool corner cases, but this may be unnecessary if other changes are made. Therefore, in order to unblock my sort patch (#6444) I propose that we support copy() for the cases where UnsafeRow does not use an ObjectPool and continue to throw UnsupportedOperationException when an ObjectPool is used. This patch accomplishes this by modifying UnsafeRow so that it knows the size of the row's backing data in order to be able to copy it into a byte array. Author: Josh Rosen Closes #7306 from JoshRosen/SPARK-8932 and squashes the following commits: 338e6bf [Josh Rosen] Support copy for UnsafeRows that do not use ObjectPools. --- .../UnsafeFixedWidthAggregationMap.java | 12 +++-- .../sql/catalyst/expressions/UnsafeRow.java | 32 +++++++++++- .../expressions/UnsafeRowConverter.scala | 10 +++- .../expressions/UnsafeRowConverterSuite.scala | 52 ++++++++++++++----- 4 files changed, 87 insertions(+), 19 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 1e79f4b2e88e5..79d55b36dab01 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 @@ -120,9 +120,11 @@ public UnsafeFixedWidthAggregationMap( this.bufferPool = new ObjectPool(initialCapacity); InternalRow initRow = initProjection.apply(emptyRow); - this.emptyBuffer = new byte[bufferConverter.getSizeRequirement(initRow)]; + int emptyBufferSize = bufferConverter.getSizeRequirement(initRow); + this.emptyBuffer = new byte[emptyBufferSize]; int writtenLength = bufferConverter.writeRow( - initRow, emptyBuffer, PlatformDependent.BYTE_ARRAY_OFFSET, bufferPool); + initRow, emptyBuffer, PlatformDependent.BYTE_ARRAY_OFFSET, emptyBufferSize, + bufferPool); assert (writtenLength == emptyBuffer.length): "Size requirement calculation was wrong!"; // re-use the empty buffer only when there is no object saved in pool. reuseEmptyBuffer = bufferPool.size() == 0; @@ -142,6 +144,7 @@ public UnsafeRow getAggregationBuffer(InternalRow groupingKey) { groupingKey, groupingKeyConversionScratchSpace, PlatformDependent.BYTE_ARRAY_OFFSET, + groupingKeySize, keyPool); assert (groupingKeySize == actualGroupingKeySize) : "Size requirement calculation was wrong!"; @@ -157,7 +160,7 @@ public UnsafeRow getAggregationBuffer(InternalRow groupingKey) { // There is some objects referenced by emptyBuffer, so generate a new one InternalRow initRow = initProjection.apply(emptyRow); bufferConverter.writeRow(initRow, emptyBuffer, PlatformDependent.BYTE_ARRAY_OFFSET, - bufferPool); + groupingKeySize, bufferPool); } loc.putNewKey( groupingKeyConversionScratchSpace, @@ -175,6 +178,7 @@ public UnsafeRow getAggregationBuffer(InternalRow groupingKey) { address.getBaseObject(), address.getBaseOffset(), bufferConverter.numFields(), + loc.getValueLength(), bufferPool ); return currentBuffer; @@ -214,12 +218,14 @@ public MapEntry next() { keyAddress.getBaseObject(), keyAddress.getBaseOffset(), keyConverter.numFields(), + loc.getKeyLength(), keyPool ); entry.value.pointTo( valueAddress.getBaseObject(), valueAddress.getBaseOffset(), bufferConverter.numFields(), + loc.getValueLength(), bufferPool ); return entry; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index aeb64b045812f..edb7202245289 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -68,6 +68,9 @@ public final class UnsafeRow extends MutableRow { /** The number of fields in this row, used for calculating the bitset width (and in assertions) */ private int numFields; + /** The size of this row's backing data, in bytes) */ + private int sizeInBytes; + public int length() { return numFields; } /** The width of the null tracking bit set, in bytes */ @@ -95,14 +98,17 @@ public UnsafeRow() { } * @param baseObject the base object * @param baseOffset the offset within the base object * @param numFields the number of fields in this row + * @param sizeInBytes the size of this row's backing data, in bytes * @param pool the object pool to hold arbitrary objects */ - public void pointTo(Object baseObject, long baseOffset, int numFields, ObjectPool pool) { + public void pointTo( + Object baseObject, long baseOffset, int numFields, int sizeInBytes, ObjectPool pool) { assert numFields >= 0 : "numFields should >= 0"; this.bitSetWidthInBytes = calculateBitSetWidthInBytes(numFields); this.baseObject = baseObject; this.baseOffset = baseOffset; this.numFields = numFields; + this.sizeInBytes = sizeInBytes; this.pool = pool; } @@ -336,9 +342,31 @@ public double getDouble(int i) { } } + /** + * Copies this row, returning a self-contained UnsafeRow that stores its data in an internal + * byte array rather than referencing data stored in a data page. + *

+ * This method is only supported on UnsafeRows that do not use ObjectPools. + */ @Override public InternalRow copy() { - throw new UnsupportedOperationException(); + if (pool != null) { + throw new UnsupportedOperationException( + "Copy is not supported for UnsafeRows that use object pools"); + } else { + UnsafeRow rowCopy = new UnsafeRow(); + final byte[] rowDataCopy = new byte[sizeInBytes]; + PlatformDependent.copyMemory( + baseObject, + baseOffset, + rowDataCopy, + PlatformDependent.BYTE_ARRAY_OFFSET, + sizeInBytes + ); + rowCopy.pointTo( + rowDataCopy, PlatformDependent.BYTE_ARRAY_OFFSET, numFields, sizeInBytes, null); + return rowCopy; + } } @Override 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 1f395497a9839..6af5e6200e57b 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 @@ -70,10 +70,16 @@ class UnsafeRowConverter(fieldTypes: Array[DataType]) { * @param row the row to convert * @param baseObject the base object of the destination address * @param baseOffset the base offset of the destination address + * @param rowLengthInBytes the length calculated by `getSizeRequirement(row)` * @return the number of bytes written. This should be equal to `getSizeRequirement(row)`. */ - def writeRow(row: InternalRow, baseObject: Object, baseOffset: Long, pool: ObjectPool): Int = { - unsafeRow.pointTo(baseObject, baseOffset, writers.length, pool) + def writeRow( + row: InternalRow, + baseObject: Object, + baseOffset: Long, + rowLengthInBytes: Int, + pool: ObjectPool): Int = { + unsafeRow.pointTo(baseObject, baseOffset, writers.length, rowLengthInBytes, pool) if (writers.length > 0) { // zero-out the bitset diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 96d4e64ea344a..d00aeb4dfbf47 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -44,19 +44,32 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val sizeRequired: Int = converter.getSizeRequirement(row) assert(sizeRequired === 8 + (3 * 8)) val buffer: Array[Long] = new Array[Long](sizeRequired / 8) - val numBytesWritten = converter.writeRow(row, buffer, PlatformDependent.LONG_ARRAY_OFFSET, null) + val numBytesWritten = + converter.writeRow(row, buffer, PlatformDependent.LONG_ARRAY_OFFSET, sizeRequired, null) assert(numBytesWritten === sizeRequired) val unsafeRow = new UnsafeRow() - unsafeRow.pointTo(buffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, null) + unsafeRow.pointTo( + buffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, sizeRequired, null) assert(unsafeRow.getLong(0) === 0) assert(unsafeRow.getLong(1) === 1) assert(unsafeRow.getInt(2) === 2) + // We can copy UnsafeRows as long as they don't reference ObjectPools + val unsafeRowCopy = unsafeRow.copy() + assert(unsafeRowCopy.getLong(0) === 0) + assert(unsafeRowCopy.getLong(1) === 1) + assert(unsafeRowCopy.getInt(2) === 2) + unsafeRow.setLong(1, 3) assert(unsafeRow.getLong(1) === 3) unsafeRow.setInt(2, 4) assert(unsafeRow.getInt(2) === 4) + + // Mutating the original row should not have changed the copy + assert(unsafeRowCopy.getLong(0) === 0) + assert(unsafeRowCopy.getLong(1) === 1) + assert(unsafeRowCopy.getInt(2) === 2) } test("basic conversion with primitive, string and binary types") { @@ -73,12 +86,14 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { ByteArrayMethods.roundNumberOfBytesToNearestWord("Hello".getBytes.length) + ByteArrayMethods.roundNumberOfBytesToNearestWord("World".getBytes.length)) val buffer: Array[Long] = new Array[Long](sizeRequired / 8) - val numBytesWritten = converter.writeRow(row, buffer, PlatformDependent.LONG_ARRAY_OFFSET, null) + val numBytesWritten = converter.writeRow( + row, buffer, PlatformDependent.LONG_ARRAY_OFFSET, sizeRequired, null) assert(numBytesWritten === sizeRequired) val unsafeRow = new UnsafeRow() val pool = new ObjectPool(10) - unsafeRow.pointTo(buffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, pool) + unsafeRow.pointTo( + buffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, sizeRequired, pool) assert(unsafeRow.getLong(0) === 0) assert(unsafeRow.getString(1) === "Hello") assert(unsafeRow.get(2) === "World".getBytes) @@ -96,6 +111,11 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { unsafeRow.update(2, "Hello World".getBytes) assert(unsafeRow.get(2) === "Hello World".getBytes) assert(pool.size === 2) + + // We do not support copy() for UnsafeRows that reference ObjectPools + intercept[UnsupportedOperationException] { + unsafeRow.copy() + } } test("basic conversion with primitive, decimal and array") { @@ -111,12 +131,14 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val sizeRequired: Int = converter.getSizeRequirement(row) assert(sizeRequired === 8 + (8 * 3)) val buffer: Array[Long] = new Array[Long](sizeRequired / 8) - val numBytesWritten = converter.writeRow(row, buffer, PlatformDependent.LONG_ARRAY_OFFSET, pool) + val numBytesWritten = + converter.writeRow(row, buffer, PlatformDependent.LONG_ARRAY_OFFSET, sizeRequired, pool) assert(numBytesWritten === sizeRequired) assert(pool.size === 2) val unsafeRow = new UnsafeRow() - unsafeRow.pointTo(buffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, pool) + unsafeRow.pointTo( + buffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, sizeRequired, pool) assert(unsafeRow.getLong(0) === 0) assert(unsafeRow.get(1) === Decimal(1)) assert(unsafeRow.get(2) === Array(2)) @@ -142,11 +164,13 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { assert(sizeRequired === 8 + (8 * 4) + ByteArrayMethods.roundNumberOfBytesToNearestWord("Hello".getBytes.length)) val buffer: Array[Long] = new Array[Long](sizeRequired / 8) - val numBytesWritten = converter.writeRow(row, buffer, PlatformDependent.LONG_ARRAY_OFFSET, null) + val numBytesWritten = + converter.writeRow(row, buffer, PlatformDependent.LONG_ARRAY_OFFSET, sizeRequired, null) assert(numBytesWritten === sizeRequired) val unsafeRow = new UnsafeRow() - unsafeRow.pointTo(buffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, null) + unsafeRow.pointTo( + buffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, sizeRequired, null) assert(unsafeRow.getLong(0) === 0) assert(unsafeRow.getString(1) === "Hello") // Date is represented as Int in unsafeRow @@ -190,12 +214,14 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val sizeRequired: Int = converter.getSizeRequirement(rowWithAllNullColumns) val createdFromNullBuffer: Array[Long] = new Array[Long](sizeRequired / 8) val numBytesWritten = converter.writeRow( - rowWithAllNullColumns, createdFromNullBuffer, PlatformDependent.LONG_ARRAY_OFFSET, null) + rowWithAllNullColumns, createdFromNullBuffer, PlatformDependent.LONG_ARRAY_OFFSET, + sizeRequired, null) assert(numBytesWritten === sizeRequired) val createdFromNull = new UnsafeRow() createdFromNull.pointTo( - createdFromNullBuffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, null) + createdFromNullBuffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, + sizeRequired, null) for (i <- 0 to fieldTypes.length - 1) { assert(createdFromNull.isNullAt(i)) } @@ -233,10 +259,12 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers { val pool = new ObjectPool(1) val setToNullAfterCreationBuffer: Array[Long] = new Array[Long](sizeRequired / 8 + 2) converter.writeRow( - rowWithNoNullColumns, setToNullAfterCreationBuffer, PlatformDependent.LONG_ARRAY_OFFSET, pool) + rowWithNoNullColumns, setToNullAfterCreationBuffer, PlatformDependent.LONG_ARRAY_OFFSET, + sizeRequired, pool) val setToNullAfterCreation = new UnsafeRow() setToNullAfterCreation.pointTo( - setToNullAfterCreationBuffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, pool) + setToNullAfterCreationBuffer, PlatformDependent.LONG_ARRAY_OFFSET, fieldTypes.length, + sizeRequired, pool) assert(setToNullAfterCreation.isNullAt(0) === rowWithNoNullColumns.isNullAt(0)) assert(setToNullAfterCreation.getBoolean(1) === rowWithNoNullColumns.getBoolean(1)) From 47ef423f860c3109d50c7e321616b267f4296e34 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 8 Jul 2015 20:29:08 -0700 Subject: [PATCH 73/73] [SPARK-8910] Fix MiMa flaky due to port contention issue Due to the way MiMa works, we currently start a `SQLContext` pretty early on. This causes us to start a `SparkUI` that attempts to bind to port 4040. Because many tests run in parallel on the Jenkins machines, this causes port contention sometimes and fails the MiMa tests. Note that we already disabled the SparkUI for scalatests. However, the MiMa test is run before we even have a chance to load the default scalatest settings, so we need to explicitly disable the UI ourselves. Author: Andrew Or Closes #7300 from andrewor14/mima-flaky and squashes the following commits: b55a547 [Andrew Or] Do not enable SparkUI during tests --- .../scala/org/apache/spark/sql/test/TestSQLContext.scala | 8 ++++---- .../scala/org/apache/spark/sql/hive/test/TestHive.scala | 7 ++++--- 2 files changed, 8 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index 9fa394525d65c..b3a4231da91c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -26,10 +26,10 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan /** A SQLContext that can be used for local testing. */ class LocalSQLContext extends SQLContext( - new SparkContext( - "local[2]", - "TestSQLContext", - new SparkConf().set("spark.sql.testkey", "true"))) { + new SparkContext("local[2]", "TestSQLContext", new SparkConf() + .set("spark.sql.testkey", "true") + // SPARK-8910 + .set("spark.ui.enabled", "false"))) { override protected[sql] def createSession(): SQLSession = { new this.SQLSession() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index 7978fdacaedba..0f217bc66869f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -53,9 +53,10 @@ object TestHive "TestSQLContext", new SparkConf() .set("spark.sql.test", "") - .set( - "spark.sql.hive.metastore.barrierPrefixes", - "org.apache.spark.sql.hive.execution.PairSerDe"))) + .set("spark.sql.hive.metastore.barrierPrefixes", + "org.apache.spark.sql.hive.execution.PairSerDe") + // SPARK-8910 + .set("spark.ui.enabled", "false"))) /** * A locally running test instance of Spark's Hive execution engine.