From b632e775cc057492ebba6b65647d90908aa00421 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Thu, 6 Sep 2018 15:27:59 +0800 Subject: [PATCH 001/879] [SPARK-25317][CORE] Avoid perf regression in Murmur3 Hash on UTF8String ## What changes were proposed in this pull request? SPARK-10399 introduced a performance regression on the hash computation for UTF8String. The regression can be evaluated with the code attached in the JIRA. That code runs in about 120 us per method on my laptop (MacBook Pro 2.5 GHz Intel Core i7, RAM 16 GB 1600 MHz DDR3) while the code from branch 2.3 takes on the same machine about 45 us for me. After the PR, the code takes about 45 us on the master branch too. ## How was this patch tested? running the perf test from the JIRA Closes #22338 from mgaido91/SPARK-25317. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan (cherry picked from commit 64c314e22fecca1ca3fe32378fc9374d8485deec) Signed-off-by: Wenchen Fan --- .../spark/unsafe/hash/Murmur3_x86_32.java | 23 +++++++++++-------- 1 file changed, 14 insertions(+), 9 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index aff6e93d647f..566f11615430 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -19,6 +19,7 @@ import com.google.common.primitives.Ints; +import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.types.UTF8String; @@ -59,7 +60,7 @@ public static int hashUnsafeWordsBlock(MemoryBlock base, int seed) { // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. int lengthInBytes = Ints.checkedCast(base.size()); assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; - int h1 = hashBytesByIntBlock(base, seed); + int h1 = hashBytesByIntBlock(base, lengthInBytes, seed); return fmix(h1, lengthInBytes); } @@ -69,14 +70,19 @@ public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, i } public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { + return hashUnsafeBytesBlock(base, Ints.checkedCast(base.size()), seed); + } + + private static int hashUnsafeBytesBlock(MemoryBlock base, int lengthInBytes, int seed) { // This is not compatible with original and another implementations. // But remain it for backward compatibility for the components existing before 2.3. - int lengthInBytes = Ints.checkedCast(base.size()); assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByIntBlock(base.subBlock(0, lengthAligned), seed); + int h1 = hashBytesByIntBlock(base, lengthAligned, seed); + long offset = base.getBaseOffset(); + Object o = base.getBaseObject(); for (int i = lengthAligned; i < lengthInBytes; i++) { - int halfWord = base.getByte(i); + int halfWord = Platform.getByte(o, offset + i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); } @@ -84,7 +90,7 @@ public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { } public static int hashUTF8String(UTF8String str, int seed) { - return hashUnsafeBytesBlock(str.getMemoryBlock(), seed); + return hashUnsafeBytesBlock(str.getMemoryBlock(), str.numBytes(), seed); } public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { @@ -101,7 +107,7 @@ public static int hashUnsafeBytes2Block(MemoryBlock base, int seed) { int lengthInBytes = Ints.checkedCast(base.size()); assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByIntBlock(base.subBlock(0, lengthAligned), seed); + int h1 = hashBytesByIntBlock(base, lengthAligned, seed); int k1 = 0; for (int i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { k1 ^= (base.getByte(i) & 0xFF) << shift; @@ -110,11 +116,10 @@ public static int hashUnsafeBytes2Block(MemoryBlock base, int seed) { return fmix(h1, lengthInBytes); } - private static int hashBytesByIntBlock(MemoryBlock base, int seed) { - long lengthInBytes = base.size(); + private static int hashBytesByIntBlock(MemoryBlock base, int lengthInBytes, int seed) { assert (lengthInBytes % 4 == 0); int h1 = seed; - for (long i = 0; i < lengthInBytes; i += 4) { + for (int i = 0; i < lengthInBytes; i += 4) { int halfWord = base.getInt(i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); From 085f731adb9b8c82a2bf4bbcae6d889a967fbd53 Mon Sep 17 00:00:00 2001 From: Shahid Date: Thu, 6 Sep 2018 09:52:58 -0700 Subject: [PATCH 002/879] [SPARK-25268][GRAPHX] run Parallel Personalized PageRank throws serialization Exception ## What changes were proposed in this pull request? mapValues in scala is currently not serializable. To avoid the serialization issue while running pageRank, we need to use map instead of mapValues. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #22271 from shahidki31/master_latest. Authored-by: Shahid Signed-off-by: Joseph K. Bradley (cherry picked from commit 3b6591b0b064b13a411e5b8f8ee4883a69c39e2d) Signed-off-by: Joseph K. Bradley --- .../main/scala/org/apache/spark/graphx/lib/PageRank.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 96b635f9a144..1305c059b89c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -198,9 +198,11 @@ object PageRank extends Logging { val zero = Vectors.sparse(sources.size, List()).asBreeze // map of vid -> vector where for each vid, the _position of vid in source_ is set to 1.0 - val sourcesInitMap = sources.zipWithIndex.toMap.mapValues { i => - Vectors.sparse(sources.size, Array(i), Array(1.0)).asBreeze - } + val sourcesInitMap = sources.zipWithIndex.map { case (vid, i) => + val v = Vectors.sparse(sources.size, Array(i), Array(1.0)).asBreeze + (vid, v) + }.toMap + val sc = graph.vertices.sparkContext val sourcesInitMapBC = sc.broadcast(sourcesInitMap) // Initialize the PageRank graph with each edge attribute having From f2d5022233b637eb50567f7945042b3a8c9c6b25 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 6 Sep 2018 08:18:49 -0700 Subject: [PATCH 003/879] [SPARK-25328][PYTHON] Add an example for having two columns as the grouping key in group aggregate pandas UDF ## What changes were proposed in this pull request? This PR proposes to add another example for multiple grouping key in group aggregate pandas UDF since this feature could make users still confused. ## How was this patch tested? Manually tested and documentation built. Closes #22329 from HyukjinKwon/SPARK-25328. Authored-by: hyukjinkwon Signed-off-by: Bryan Cutler (cherry picked from commit 7ef6d1daf858cc9a2c390074f92aaf56c219518a) Signed-off-by: Bryan Cutler --- python/pyspark/sql/functions.py | 24 ++++++++++++++++++++---- 1 file changed, 20 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 864780e0be9b..9396b16b7ada 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2783,14 +2783,14 @@ def pandas_udf(f=None, returnType=None, functionType=None): +---+-------------------+ Alternatively, the user can define a function that takes two arguments. - In this case, the grouping key will be passed as the first argument and the data will - be passed as the second argument. The grouping key will be passed as a tuple of numpy + In this case, the grouping key(s) will be passed as the first argument and the data will + be passed as the second argument. The grouping key(s) will be passed as a tuple of numpy data types, e.g., `numpy.int32` and `numpy.float64`. The data will still be passed in as a `pandas.DataFrame` containing all columns from the original Spark DataFrame. - This is useful when the user does not want to hardcode grouping key in the function. + This is useful when the user does not want to hardcode grouping key(s) in the function. - >>> from pyspark.sql.functions import pandas_udf, PandasUDFType >>> import pandas as pd # doctest: +SKIP + >>> from pyspark.sql.functions import pandas_udf, PandasUDFType >>> df = spark.createDataFrame( ... [(1, 1.0), (1, 2.0), (2, 3.0), (2, 5.0), (2, 10.0)], ... ("id", "v")) # doctest: +SKIP @@ -2806,6 +2806,22 @@ def pandas_udf(f=None, returnType=None, functionType=None): | 1|1.5| | 2|6.0| +---+---+ + >>> @pandas_udf( + ... "id long, `ceil(v / 2)` long, v double", + ... PandasUDFType.GROUPED_MAP) # doctest: +SKIP + >>> def sum_udf(key, pdf): + ... # key is a tuple of two numpy.int64s, which is the values + ... # of 'id' and 'ceil(df.v / 2)' for the current group + ... return pd.DataFrame([key + (pdf.v.sum(),)]) + >>> df.groupby(df.id, ceil(df.v / 2)).apply(sum_udf).show() # doctest: +SKIP + +---+-----------+----+ + | id|ceil(v / 2)| v| + +---+-----------+----+ + | 2| 5|10.0| + | 1| 1| 3.0| + | 2| 3| 5.0| + | 2| 2| 3.0| + +---+-----------+----+ .. note:: If returning a new `pandas.DataFrame` constructed with a dictionary, it is recommended to explicitly index the columns by name to ensure the positions are correct, From 3682d29f45870031d9dc4e812accbfbb583cc52a Mon Sep 17 00:00:00 2001 From: liyuanjian Date: Thu, 6 Sep 2018 10:17:29 -0700 Subject: [PATCH 004/879] [SPARK-25072][PYSPARK] Forbid extra value for custom Row ## What changes were proposed in this pull request? Add value length check in `_create_row`, forbid extra value for custom Row in PySpark. ## How was this patch tested? New UT in pyspark-sql Closes #22140 from xuanyuanking/SPARK-25072. Lead-authored-by: liyuanjian Co-authored-by: Yuanjian Li Signed-off-by: Bryan Cutler (cherry picked from commit c84bc40d7f33c71eca1c08f122cd60517f34c1f8) Signed-off-by: Bryan Cutler --- python/pyspark/sql/tests.py | 4 ++++ python/pyspark/sql/types.py | 3 +++ 2 files changed, 7 insertions(+) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 81c0af0b3d81..6d9d636b23a3 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -277,6 +277,10 @@ def test_struct_field_type_name(self): struct_field = StructField("a", IntegerType()) self.assertRaises(TypeError, struct_field.typeName) + def test_invalid_create_row(self): + row_class = Row("c1", "c2") + self.assertRaises(ValueError, lambda: row_class(1, 2, 3)) + class SQLTests(ReusedSQLTestCase): diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 0b61707c8cc0..ce1d004c6c8f 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -1500,6 +1500,9 @@ def __contains__(self, item): # let object acts like class def __call__(self, *args): """create new Row object""" + if len(args) > len(self): + raise ValueError("Can not create Row with fields %s, expected %d values " + "but got %s" % (self, len(self), args)) return _create_row(self, args) def __getitem__(self, item): From a7cfe5158f5c25ae5f774e1fb45d63a67a4bb89c Mon Sep 17 00:00:00 2001 From: xuejianbest <384329882@qq.com> Date: Thu, 6 Sep 2018 07:17:37 -0700 Subject: [PATCH 005/879] [SPARK-25108][SQL] Fix the show method to display the wide character alignment problem MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This is not a perfect solution. It is designed to minimize complexity on the basis of solving problems. It is effective for English, Chinese characters, Japanese, Korean and so on. ```scala before: +---+---------------------------+-------------+ |id |中国 |s2 | +---+---------------------------+-------------+ |1 |ab |[a] | |2 |null |[中国, abc] | |3 |ab1 |[hello world]| |4 |か行 きゃ(kya) きゅ(kyu) きょ(kyo) |[“中国] | |5 |中国(你好)a |[“中(国), 312] | |6 |中国山(东)服务区 |[“中(国)] | |7 |中国山东服务区 |[中(国)] | |8 | |[中国] | +---+---------------------------+-------------+ after: +---+-----------------------------------+----------------+ |id |中国 |s2 | +---+-----------------------------------+----------------+ |1 |ab |[a] | |2 |null |[中国, abc] | |3 |ab1 |[hello world] | |4 |か行 きゃ(kya) きゅ(kyu) きょ(kyo) |[“中国] | |5 |中国(你好)a |[“中(国), 312]| |6 |中国山(东)服务区 |[“中(国)] | |7 |中国山东服务区 |[中(国)] | |8 | |[中国] | +---+-----------------------------------+----------------+ ``` ## What changes were proposed in this pull request? When there are wide characters such as Chinese characters or Japanese characters in the data, the show method has a alignment problem. Try to fix this problem. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) ![image](https://user-images.githubusercontent.com/13044869/44250564-69f6b400-a227-11e8-88b2-6cf6960377ff.png) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #22048 from xuejianbest/master. Authored-by: xuejianbest <384329882@qq.com> Signed-off-by: Sean Owen --- .../scala/org/apache/spark/util/Utils.scala | 30 ++++++++++++ .../org/apache/spark/util/UtilsSuite.scala | 21 ++++++++ .../scala/org/apache/spark/sql/Dataset.scala | 18 +++---- .../org/apache/spark/sql/DatasetSuite.scala | 49 +++++++++++++++++++ 4 files changed, 109 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 15c958d3f511..4593b057fc63 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2795,6 +2795,36 @@ private[spark] object Utils extends Logging { } } } + + /** + * Regular expression matching full width characters. + * + * Looked at all the 0x0000-0xFFFF characters (unicode) and showed them under Xshell. + * Found all the full width characters, then get the regular expression. + */ + private val fullWidthRegex = ("""[""" + + // scalastyle:off nonascii + """\u1100-\u115F""" + + """\u2E80-\uA4CF""" + + """\uAC00-\uD7A3""" + + """\uF900-\uFAFF""" + + """\uFE10-\uFE19""" + + """\uFE30-\uFE6F""" + + """\uFF00-\uFF60""" + + """\uFFE0-\uFFE6""" + + // scalastyle:on nonascii + """]""").r + + /** + * Return the number of half widths in a given string. Note that a full width character + * occupies two half widths. + * + * For a string consisting of 1 million characters, the execution of this method requires + * about 50ms. + */ + def stringHalfWidth(str: String): Int = { + if (str == null) 0 else str.length + fullWidthRegex.findAllIn(str).size + } } private[util] object CallerContext extends Logging { diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 418d2f9b8850..943b53522d64 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1184,6 +1184,27 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(Utils.getSimpleName(classOf[MalformedClassObject.MalformedClass]) === "UtilsSuite$MalformedClassObject$MalformedClass") } + + test("stringHalfWidth") { + // scalastyle:off nonascii + assert(Utils.stringHalfWidth(null) == 0) + assert(Utils.stringHalfWidth("") == 0) + assert(Utils.stringHalfWidth("ab c") == 4) + assert(Utils.stringHalfWidth("1098") == 4) + assert(Utils.stringHalfWidth("mø") == 2) + assert(Utils.stringHalfWidth("γύρ") == 3) + assert(Utils.stringHalfWidth("pê") == 2) + assert(Utils.stringHalfWidth("ー") == 2) + assert(Utils.stringHalfWidth("测") == 2) + assert(Utils.stringHalfWidth("か") == 2) + assert(Utils.stringHalfWidth("걸") == 2) + assert(Utils.stringHalfWidth("à") == 1) + assert(Utils.stringHalfWidth("焼") == 2) + assert(Utils.stringHalfWidth("羍む") == 4) + assert(Utils.stringHalfWidth("뺭ᾘ") == 3) + assert(Utils.stringHalfWidth("\u0967\u0968\u0969") == 3) + // scalastyle:on nonascii + } } private class SimpleExtension diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index db439b1ee76f..fa14aa14ee96 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -306,16 +306,16 @@ class Dataset[T] private[sql]( // Compute the width of each column for (row <- rows) { for ((cell, i) <- row.zipWithIndex) { - colWidths(i) = math.max(colWidths(i), cell.length) + colWidths(i) = math.max(colWidths(i), Utils.stringHalfWidth(cell)) } } val paddedRows = rows.map { row => row.zipWithIndex.map { case (cell, i) => if (truncate > 0) { - StringUtils.leftPad(cell, colWidths(i)) + StringUtils.leftPad(cell, colWidths(i) - Utils.stringHalfWidth(cell) + cell.length) } else { - StringUtils.rightPad(cell, colWidths(i)) + StringUtils.rightPad(cell, colWidths(i) - Utils.stringHalfWidth(cell) + cell.length) } } } @@ -337,12 +337,10 @@ class Dataset[T] private[sql]( // Compute the width of field name and data columns val fieldNameColWidth = fieldNames.foldLeft(minimumColWidth) { case (curMax, fieldName) => - math.max(curMax, fieldName.length) + math.max(curMax, Utils.stringHalfWidth(fieldName)) } val dataColWidth = dataRows.foldLeft(minimumColWidth) { case (curMax, row) => - math.max(curMax, row.map(_.length).reduceLeftOption[Int] { case (cellMax, cell) => - math.max(cellMax, cell) - }.getOrElse(0)) + math.max(curMax, row.map(cell => Utils.stringHalfWidth(cell)).max) } dataRows.zipWithIndex.foreach { case (row, i) => @@ -351,8 +349,10 @@ class Dataset[T] private[sql]( s"-RECORD $i", fieldNameColWidth + dataColWidth + 5, "-") sb.append(rowHeader).append("\n") row.zipWithIndex.map { case (cell, j) => - val fieldName = StringUtils.rightPad(fieldNames(j), fieldNameColWidth) - val data = StringUtils.rightPad(cell, dataColWidth) + val fieldName = StringUtils.rightPad(fieldNames(j), + fieldNameColWidth - Utils.stringHalfWidth(fieldNames(j)) + fieldNames(j).length) + val data = StringUtils.rightPad(cell, + dataColWidth - Utils.stringHalfWidth(cell) + cell.length) s" $fieldName | $data " }.addString(sb, "", "\n", "\n") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index cf24eba12801..ca8fbc991a3a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -969,6 +969,55 @@ class DatasetSuite extends QueryTest with SharedSQLContext { checkShowString(ds, expected) } + test("SPARK-25108 Fix the show method to display the full width character alignment problem") { + // scalastyle:off nonascii + val df = Seq( + (0, null, 1), + (0, "", 1), + (0, "ab c", 1), + (0, "1098", 1), + (0, "mø", 1), + (0, "γύρ", 1), + (0, "pê", 1), + (0, "ー", 1), + (0, "测", 1), + (0, "か", 1), + (0, "걸", 1), + (0, "à", 1), + (0, "焼", 1), + (0, "羍む", 1), + (0, "뺭ᾘ", 1), + (0, "\u0967\u0968\u0969", 1) + ).toDF("b", "a", "c") + // scalastyle:on nonascii + val ds = df.as[ClassData] + val expected = + // scalastyle:off nonascii + """+---+----+---+ + || b| a| c| + |+---+----+---+ + || 0|null| 1| + || 0| | 1| + || 0|ab c| 1| + || 0|1098| 1| + || 0| mø| 1| + || 0| γύρ| 1| + || 0| pê| 1| + || 0| ー| 1| + || 0| 测| 1| + || 0| か| 1| + || 0| 걸| 1| + || 0| à| 1| + || 0| 焼| 1| + || 0|羍む| 1| + || 0| 뺭ᾘ| 1| + || 0| १२३| 1| + |+---+----+---+ + |""".stripMargin + // scalastyle:on nonascii + checkShowString(ds, expected) + } + test( "SPARK-15112: EmbedDeserializerInFilter should not optimize plan fragment that changes schema" ) { From ff832beee0c55c11ac110261a3c48010b81a1e5f Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 7 Sep 2018 10:12:20 +0800 Subject: [PATCH 006/879] [SPARK-25208][SQL][FOLLOW-UP] Reduce code size. ## What changes were proposed in this pull request? This is a follow-up pr of #22200. When casting to decimal type, if `Cast.canNullSafeCastToDecimal()`, overflow won't happen, so we don't need to check the result of `Decimal.changePrecision()`. ## How was this patch tested? Existing tests. Closes #22352 from ueshin/issues/SPARK-25208/reduce_code_size. Authored-by: Takuya UESHIN Signed-off-by: Wenchen Fan (cherry picked from commit 1b1711e0532b1a1521054ef3b5980cdb3d70cdeb) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/expressions/Cast.scala | 37 ++++++++++++------- 1 file changed, 23 insertions(+), 14 deletions(-) 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 005350350104..8f777997bf61 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 @@ -924,27 +924,36 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String } private[this] def changePrecision(d: ExprValue, decimalType: DecimalType, - evPrim: ExprValue, evNull: ExprValue): Block = - code""" - if ($d.changePrecision(${decimalType.precision}, ${decimalType.scale})) { - $evPrim = $d; - } else { - $evNull = true; - } - """ + evPrim: ExprValue, evNull: ExprValue, canNullSafeCast: Boolean): Block = { + if (canNullSafeCast) { + code""" + |$d.changePrecision(${decimalType.precision}, ${decimalType.scale}); + |$evPrim = $d; + """.stripMargin + } else { + code""" + |if ($d.changePrecision(${decimalType.precision}, ${decimalType.scale})) { + | $evPrim = $d; + |} else { + | $evNull = true; + |} + """.stripMargin + } + } private[this] def castToDecimalCode( from: DataType, target: DecimalType, ctx: CodegenContext): CastFunction = { val tmp = ctx.freshVariable("tmpDecimal", classOf[Decimal]) + val canNullSafeCast = Cast.canNullSafeCastToDecimal(from, target) from match { case StringType => (c, evPrim, evNull) => code""" try { Decimal $tmp = Decimal.apply(new java.math.BigDecimal($c.toString())); - ${changePrecision(tmp, target, evPrim, evNull)} + ${changePrecision(tmp, target, evPrim, evNull, canNullSafeCast)} } catch (java.lang.NumberFormatException e) { $evNull = true; } @@ -953,7 +962,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String (c, evPrim, evNull) => code""" Decimal $tmp = $c ? Decimal.apply(1) : Decimal.apply(0); - ${changePrecision(tmp, target, evPrim, evNull)} + ${changePrecision(tmp, target, evPrim, evNull, canNullSafeCast)} """ case DateType => // date can't cast to decimal in Hive @@ -964,19 +973,19 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String code""" Decimal $tmp = Decimal.apply( scala.math.BigDecimal.valueOf(${timestampToDoubleCode(c)})); - ${changePrecision(tmp, target, evPrim, evNull)} + ${changePrecision(tmp, target, evPrim, evNull, canNullSafeCast)} """ case DecimalType() => (c, evPrim, evNull) => code""" Decimal $tmp = $c.clone(); - ${changePrecision(tmp, target, evPrim, evNull)} + ${changePrecision(tmp, target, evPrim, evNull, canNullSafeCast)} """ case x: IntegralType => (c, evPrim, evNull) => code""" Decimal $tmp = Decimal.apply((long) $c); - ${changePrecision(tmp, target, evPrim, evNull)} + ${changePrecision(tmp, target, evPrim, evNull, canNullSafeCast)} """ case x: FractionalType => // All other numeric types can be represented precisely as Doubles @@ -984,7 +993,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String code""" try { Decimal $tmp = Decimal.apply(scala.math.BigDecimal.valueOf((double) $c)); - ${changePrecision(tmp, target, evPrim, evNull)} + ${changePrecision(tmp, target, evPrim, evNull, canNullSafeCast)} } catch (java.lang.NumberFormatException e) { $evNull = true; } From 24a32612bdd1136c647aa321b1c1418a43d85bf4 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Thu, 6 Sep 2018 21:41:13 -0700 Subject: [PATCH 007/879] [SPARK-25330][BUILD][BRANCH-2.3] Revert Hadoop 2.7 to 2.7.3 ## What changes were proposed in this pull request? How to reproduce permission issue: ```sh # build spark ./dev/make-distribution.sh --name SPARK-25330 --tgz -Phadoop-2.7 -Phive -Phive-thriftserver -Pyarn tar -zxf spark-2.4.0-SNAPSHOT-bin-SPARK-25330.tar && cd spark-2.4.0-SNAPSHOT-bin-SPARK-25330 export HADOOP_PROXY_USER=user_a bin/spark-sql export HADOOP_PROXY_USER=user_b bin/spark-sql ``` ```java Exception in thread "main" java.lang.RuntimeException: org.apache.hadoop.security.AccessControlException: Permission denied: user=user_b, access=EXECUTE, inode="/tmp/hive-$%7Buser.name%7D/user_b/668748f2-f6c5-4325-a797-fd0a7ee7f4d4":user_b:hadoop:drwx------ at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.check(FSPermissionChecker.java:319) at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkTraverse(FSPermissionChecker.java:259) at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:205) at org.apache.hadoop.hdfs.server.namenode.FSPermissionChecker.checkPermission(FSPermissionChecker.java:190) ``` The issue occurred in this commit: https://github.com/apache/hadoop/commit/feb886f2093ea5da0cd09c69bd1360a335335c86. This pr revert Hadoop 2.7 to 2.7.3 to avoid this issue. ## How was this patch tested? unit tests and manual tests. Closes #22327 from wangyum/SPARK-25330. Authored-by: Yuming Wang Signed-off-by: Sean Owen (cherry picked from commit b0ada7dce02d101b6a04323d8185394e997caca4) Signed-off-by: Sean Owen --- assembly/README | 2 +- dev/deps/spark-deps-hadoop-2.7 | 31 +++++++++++++++---------------- docs/building-spark.md | 2 +- pom.xml | 2 +- 4 files changed, 18 insertions(+), 19 deletions(-) diff --git a/assembly/README b/assembly/README index affd281a1385..d5dafab47741 100644 --- a/assembly/README +++ b/assembly/README @@ -9,4 +9,4 @@ This module is off by default. To activate it specify the profile in the command If you need to build an assembly for a different version of Hadoop the hadoop-version system property needs to be set as in this example: - -Dhadoop.version=2.7.7 + -Dhadoop.version=2.7.3 diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 5e12ca053af5..dcb5d63aeff4 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -64,21 +64,21 @@ gson-2.2.4.jar guava-14.0.1.jar guice-3.0.jar guice-servlet-3.0.jar -hadoop-annotations-2.7.7.jar -hadoop-auth-2.7.7.jar -hadoop-client-2.7.7.jar -hadoop-common-2.7.7.jar -hadoop-hdfs-2.7.7.jar -hadoop-mapreduce-client-app-2.7.7.jar -hadoop-mapreduce-client-common-2.7.7.jar -hadoop-mapreduce-client-core-2.7.7.jar -hadoop-mapreduce-client-jobclient-2.7.7.jar -hadoop-mapreduce-client-shuffle-2.7.7.jar -hadoop-yarn-api-2.7.7.jar -hadoop-yarn-client-2.7.7.jar -hadoop-yarn-common-2.7.7.jar -hadoop-yarn-server-common-2.7.7.jar -hadoop-yarn-server-web-proxy-2.7.7.jar +hadoop-annotations-2.7.3.jar +hadoop-auth-2.7.3.jar +hadoop-client-2.7.3.jar +hadoop-common-2.7.3.jar +hadoop-hdfs-2.7.3.jar +hadoop-mapreduce-client-app-2.7.3.jar +hadoop-mapreduce-client-common-2.7.3.jar +hadoop-mapreduce-client-core-2.7.3.jar +hadoop-mapreduce-client-jobclient-2.7.3.jar +hadoop-mapreduce-client-shuffle-2.7.3.jar +hadoop-yarn-api-2.7.3.jar +hadoop-yarn-client-2.7.3.jar +hadoop-yarn-common-2.7.3.jar +hadoop-yarn-server-common-2.7.3.jar +hadoop-yarn-server-web-proxy-2.7.3.jar hk2-api-2.4.0-b34.jar hk2-locator-2.4.0-b34.jar hk2-utils-2.4.0-b34.jar @@ -117,7 +117,6 @@ jersey-guava-2.22.2.jar jersey-media-jaxb-2.22.2.jar jersey-server-2.22.2.jar jetty-6.1.26.jar -jetty-sslengine-6.1.26.jar jetty-util-6.1.26.jar jline-2.14.6.jar joda-time-2.9.3.jar diff --git a/docs/building-spark.md b/docs/building-spark.md index 1d3e0b1b7d39..1501f0bb8454 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -67,7 +67,7 @@ Examples: ./build/mvn -Pyarn -DskipTests clean package # Apache Hadoop 2.7.X and later - ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.7 -DskipTests clean package + ./build/mvn -Pyarn -Phadoop-2.7 -Dhadoop.version=2.7.3 -DskipTests clean package ## Building With Hive and JDBC Support diff --git a/pom.xml b/pom.xml index da526a1709e6..05e3b05613ef 100644 --- a/pom.xml +++ b/pom.xml @@ -2683,7 +2683,7 @@ hadoop-2.7 - 2.7.7 + 2.7.3 2.7.1 From 3644c84f51ba8e5fd2c6607afda06f5291bdf435 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 6 Sep 2018 21:43:14 -0700 Subject: [PATCH 008/879] [SPARK-22357][CORE][FOLLOWUP] SparkContext.binaryFiles ignore minPartitions parameter ## What changes were proposed in this pull request? This adds a test following https://github.com/apache/spark/pull/21638 ## How was this patch tested? Existing tests and new test. Closes #22356 from srowen/SPARK-22357.2. Authored-by: Sean Owen Signed-off-by: Sean Owen (cherry picked from commit 4e3365b577fbc9021fa237ea4e8792f5aea5d80c) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/FileSuite.scala | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index a441b9c8ab97..81b18c71f30e 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -19,10 +19,12 @@ package org.apache.spark import java.io._ import java.nio.ByteBuffer +import java.nio.charset.StandardCharsets import java.util.zip.GZIPOutputStream import scala.io.Source +import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io._ @@ -299,6 +301,25 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { } } + test("SPARK-22357 test binaryFiles minPartitions") { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local") + .set("spark.files.openCostInBytes", "0") + .set("spark.default.parallelism", "1")) + + val tempDir = Utils.createTempDir() + val tempDirPath = tempDir.getAbsolutePath + + for (i <- 0 until 8) { + val tempFile = new File(tempDir, s"part-0000$i") + Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", tempFile, + StandardCharsets.UTF_8) + } + + for (p <- Seq(1, 2, 8)) { + assert(sc.binaryFiles(tempDirPath, minPartitions = p).getNumPartitions === p) + } + } + test("fixed record length binary file as byte array") { sc = new SparkContext("local", "test") val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) From f9b476c6ad629007d9334409e4dda99119cf0053 Mon Sep 17 00:00:00 2001 From: dujunling Date: Thu, 6 Sep 2018 21:44:46 -0700 Subject: [PATCH 009/879] [SPARK-25237][SQL] Remove updateBytesReadWithFileSize in FileScanRDD ## What changes were proposed in this pull request? This pr removed the method `updateBytesReadWithFileSize` in `FileScanRDD` because it computes input metrics by file size supported in Hadoop 2.5 and earlier. The current Spark does not support the versions, so it causes wrong input metric numbers. This is rework from #22232. Closes #22232 ## How was this patch tested? Added tests in `FileBasedDataSourceSuite`. Closes #22324 from maropu/pr22232-2. Lead-authored-by: dujunling Co-authored-by: Takeshi Yamamuro Signed-off-by: Sean Owen (cherry picked from commit ed249db9c464062fbab7c6f68ad24caaa95cec82) Signed-off-by: Sean Owen --- .../execution/datasources/FileScanRDD.scala | 10 -------- .../spark/sql/FileBasedDataSourceSuite.scala | 24 +++++++++++++++++++ 2 files changed, 24 insertions(+), 10 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 99fc78ff3e49..345c9d82ca0e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -89,14 +89,6 @@ class FileScanRDD( inputMetrics.setBytesRead(existingBytesRead + getBytesReadCallback()) } - // If we can't get the bytes read from the FS stats, fall back to the file size, - // which may be inaccurate. - private def updateBytesReadWithFileSize(): Unit = { - if (currentFile != null) { - inputMetrics.incBytesRead(currentFile.length) - } - } - private[this] val files = split.asInstanceOf[FilePartition].files.toIterator private[this] var currentFile: PartitionedFile = null private[this] var currentIterator: Iterator[Object] = null @@ -139,7 +131,6 @@ class FileScanRDD( /** Advances to the next file. Returns true if a new non-empty iterator is available. */ private def nextIterator(): Boolean = { - updateBytesReadWithFileSize() if (files.hasNext) { currentFile = files.next() logInfo(s"Reading File $currentFile") @@ -208,7 +199,6 @@ class FileScanRDD( override def close(): Unit = { updateBytesRead() - updateBytesReadWithFileSize() InputFileBlockHolder.unset() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 4aa6afd69620..304ede9c5a61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -20,10 +20,13 @@ package org.apache.spark.sql import java.io.{File, FileNotFoundException} import java.util.Locale +import scala.collection.mutable + import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkException +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT, NullData, NullUDT} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -473,6 +476,27 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo } } } + + test("SPARK-25237 compute correct input metrics in FileScanRDD") { + withTempPath { p => + val path = p.getAbsolutePath + spark.range(1000).repartition(1).write.csv(path) + val bytesReads = new mutable.ArrayBuffer[Long]() + val bytesReadListener = new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead + } + } + sparkContext.addSparkListener(bytesReadListener) + try { + spark.read.csv(path).limit(1).collect() + sparkContext.listenerBus.waitUntilEmpty(1000L) + assert(bytesReads.sum === 7860) + } finally { + sparkContext.removeSparkListener(bytesReadListener) + } + } + } } object TestingUDT { From 872bad161f1dbe6acd89b75f60053bfc8b621687 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Thu, 6 Sep 2018 23:35:02 -0700 Subject: [PATCH 010/879] [SPARK-25267][SQL][TEST] Disable ConvertToLocalRelation in the test cases of sql/core and sql/hive ## What changes were proposed in this pull request? In SharedSparkSession and TestHive, we need to disable the rule ConvertToLocalRelation for better test case coverage. ## How was this patch tested? Identify the failures after excluding "ConvertToLocalRelation" rule. Closes #22270 from dilipbiswal/SPARK-25267-final. Authored-by: Dilip Biswal Signed-off-by: gatorsmile (cherry picked from commit 6d7bc5af454341f6d9bfc1e903148ad7ba8de6f9) Signed-off-by: gatorsmile --- .../scala/org/apache/spark/ml/util/MLTest.scala | 10 +++++++++- .../sql-tests/inputs/group-by-ordinal.sql | 4 +++- .../sql-tests/results/group-by-ordinal.sql.out | 4 +++- .../apache/spark/sql/DataFrameAggregateSuite.scala | 2 +- .../apache/spark/sql/DataFrameFunctionsSuite.scala | 14 ++++++++------ .../org/apache/spark/sql/DataFrameSuite.scala | 5 ++--- .../apache/spark/sql/test/SharedSparkSession.scala | 6 ++++++ .../org/apache/spark/sql/hive/test/TestHive.scala | 8 +++++++- 8 files changed, 39 insertions(+), 14 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala index 76d41f9b2371..acac171346a8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala @@ -21,12 +21,13 @@ import java.io.File import org.scalatest.Suite -import org.apache.spark.SparkContext +import org.apache.spark.{DebugFilesystem, SparkConf, SparkContext} import org.apache.spark.ml.{PredictionModel, Transformer} import org.apache.spark.ml.linalg.Vector import org.apache.spark.sql.{DataFrame, Dataset, Encoder, Row} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.test.TestSparkSession import org.apache.spark.util.Utils @@ -36,6 +37,13 @@ trait MLTest extends StreamTest with TempDirectory { self: Suite => @transient var sc: SparkContext = _ @transient var checkpointDir: String = _ + protected override def sparkConf = { + new SparkConf() + .set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) + .set("spark.unsafe.exceptionOnMemoryLeak", "true") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + } + protected override def createSparkSession: TestSparkSession = { new TestSparkSession(new SparkContext("local[2]", "MLlibUnitTest", sparkConf)) } diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql index 928f766b4add..3144833b608b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by-ordinal.sql @@ -38,7 +38,9 @@ select a, b, sum(b) from data group by 3; select a, b, sum(b) + 2 from data group by 3; -- negative case: nondeterministic expression -select a, rand(0), sum(b) from data group by a, 2; +select a, rand(0), sum(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2; -- negative case: star select * from data group by a, b, 1; diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out index 9ecbe19078dd..cf5add6a71af 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out @@ -135,7 +135,9 @@ aggregate functions are not allowed in GROUP BY, but found (sum(CAST(data.`b` AS -- !query 13 -select a, rand(0), sum(b) from data group by a, 2 +select a, rand(0), sum(b) +from +(select /*+ REPARTITION(1) */ a, b from data) group by a, 2 -- !query 13 schema struct -- !query 13 output diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 85b3ca11383f..ed110f751645 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -558,7 +558,7 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { test("SPARK-18004 limit + aggregates") { withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "true") { - val df = Seq(("a", 1), ("b", 2), ("c", 1), ("d", 5)).toDF("id", "value") + val df = Seq(("a", 1), ("b", 2), ("c", 1), ("d", 5)).toDF("id", "value").repartition(1) val limit2Df = df.limit(2) checkAnswer( limit2Df.groupBy("id").count().select($"id"), 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 156e54300e38..4b83e51fa899 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 @@ -85,14 +85,16 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { } val df5 = Seq((Seq("a", null), Seq(1, 2))).toDF("k", "v") - intercept[RuntimeException] { + val msg1 = intercept[Exception] { df5.select(map_from_arrays($"k", $"v")).collect - } + }.getMessage + assert(msg1.contains("Cannot use null as map key!")) val df6 = Seq((Seq(1, 2), Seq("a"))).toDF("k", "v") - intercept[RuntimeException] { + val msg2 = intercept[Exception] { df6.select(map_from_arrays($"k", $"v")).collect - } + }.getMessage + assert(msg2.contains("The given two arrays should have the same length")) } test("struct with column name") { @@ -2377,7 +2379,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { assert(ex2.getMessage.contains( "The number of lambda function arguments '3' does not match")) - val ex3 = intercept[RuntimeException] { + val ex3 = intercept[Exception] { dfExample1.selectExpr("transform_keys(i, (k, v) -> v)").show() } assert(ex3.getMessage.contains("Cannot use null as map key!")) @@ -2697,7 +2699,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { test("SPARK-24734: Fix containsNull of Concat for array type") { val df = Seq((Seq(1), Seq[Integer](null), Seq("a", "b"))).toDF("k1", "k2", "v") - val ex = intercept[RuntimeException] { + val ex = intercept[Exception] { df.select(map_from_arrays(concat($"k1", $"k2"), $"v")).show() } assert(ex.getMessage.contains("Cannot use null as map key")) 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 d43fcf3c6f5d..45b17b3d4958 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 @@ -40,6 +40,7 @@ import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContex import org.apache.spark.sql.test.SQLTestData.{NullInts, NullStrings, TestData2} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils +import org.apache.spark.util.random.XORShiftRandom class DataFrameSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -1729,10 +1730,8 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("SPARK-9083: sort with non-deterministic expressions") { - import org.apache.spark.util.random.XORShiftRandom - val seed = 33 - val df = (1 to 100).map(Tuple1.apply).toDF("i") + val df = (1 to 100).map(Tuple1.apply).toDF("i").repartition(1) val random = new XORShiftRandom(seed) val expected = (1 to 100).map(_ -> random.nextDouble()).sortBy(_._2).map(_._1) val actual = df.sort(rand(seed)).collect().map(_.getInt(0)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala index 8968dbf36d50..e7e0ce64963a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala @@ -24,6 +24,7 @@ import org.scalatest.concurrent.Eventually import org.apache.spark.{DebugFilesystem, SparkConf} import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.internal.SQLConf /** @@ -39,6 +40,11 @@ trait SharedSparkSession .set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) .set("spark.unsafe.exceptionOnMemoryLeak", "true") .set(SQLConf.CODEGEN_FALLBACK.key, "false") + // Disable ConvertToLocalRelation for better test coverage. Test cases built on + // LocalRelation will exercise the optimization rules better by disabling it as + // this rule may potentially block testing of other optimization rules such as + // ConstantPropagation etc. + .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) } /** 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 ee3f99ab7e9b..71f15a45d162 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 @@ -36,6 +36,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, ExternalCatalogWithListener} +import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} import org.apache.spark.sql.execution.command.CacheTableCommand @@ -59,7 +60,12 @@ object TestHive .set("spark.sql.warehouse.dir", TestHiveContext.makeWarehouseDir().toURI.getPath) // SPARK-8910 .set("spark.ui.enabled", "false") - .set("spark.unsafe.exceptionOnMemoryLeak", "true"))) + .set("spark.unsafe.exceptionOnMemoryLeak", "true") + // Disable ConvertToLocalRelation for better test coverage. Test cases built on + // LocalRelation will exercise the optimization rules better by disabling it as + // this rule may potentially block testing of other optimization rules such as + // ConstantPropagation etc. + .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName))) case class TestHiveVersion(hiveClient: HiveClient) From 95a48b909d103e59602e883d472cb03c7c434168 Mon Sep 17 00:00:00 2001 From: fjh100456 Date: Fri, 7 Sep 2018 09:28:33 -0700 Subject: [PATCH 011/879] [SPARK-21786][SQL][FOLLOWUP] Add compressionCodec test for CTAS ## What changes were proposed in this pull request? Before Apache Spark 2.3, table properties were ignored when writing data to a hive table(created with STORED AS PARQUET/ORC syntax), because the compression configurations were not passed to the FileFormatWriter in hadoopConf. Then it was fixed in #20087. But actually for CTAS with USING PARQUET/ORC syntax, table properties were ignored too when convertMastore, so the test case for CTAS not supported. Now it has been fixed in #20522 , the test case should be enabled too. ## How was this patch tested? This only re-enables the test cases of previous PR. Closes #22302 from fjh100456/compressionCodec. Authored-by: fjh100456 Signed-off-by: Dongjoon Hyun (cherry picked from commit 473f2fb3bfd0e51c40a87e475392f2e2c8f912dd) Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/hive/CompressionCodecSuite.scala | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala index 4550d350f6db..30204d122384 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CompressionCodecSuite.scala @@ -122,7 +122,7 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo """.stripMargin) } - private def writeDateToTableUsingCTAS( + private def writeDataToTableUsingCTAS( rootDir: File, tableName: String, partitionValue: Option[String], @@ -152,7 +152,7 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo usingCTAS: Boolean): String = { val partitionValue = if (isPartitioned) Some("test") else None if (usingCTAS) { - writeDateToTableUsingCTAS(tmpDir, tableName, partitionValue, format, compressionCodec) + writeDataToTableUsingCTAS(tmpDir, tableName, partitionValue, format, compressionCodec) } else { createTable(tmpDir, tableName, isPartitioned, format, compressionCodec) writeDataToTable(tableName, partitionValue) @@ -258,8 +258,7 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo def checkForTableWithCompressProp(format: String, compressCodecs: List[String]): Unit = { Seq(true, false).foreach { isPartitioned => Seq(true, false).foreach { convertMetastore => - // TODO: Also verify CTAS(usingCTAS=true) cases when the bug(SPARK-22926) is fixed. - Seq(false).foreach { usingCTAS => + Seq(true, false).foreach { usingCTAS => checkTableCompressionCodecForCodecs( format, isPartitioned, @@ -281,8 +280,7 @@ class CompressionCodecSuite extends TestHiveSingleton with ParquetTest with Befo def checkForTableWithoutCompressProp(format: String, compressCodecs: List[String]): Unit = { Seq(true, false).foreach { isPartitioned => Seq(true, false).foreach { convertMetastore => - // TODO: Also verify CTAS(usingCTAS=true) cases when the bug(SPARK-22926) is fixed. - Seq(false).foreach { usingCTAS => + Seq(true, false).foreach { usingCTAS => checkTableCompressionCodecForCodecs( format, isPartitioned, From 80567fad4e3d8d4573d4095b1e460452e597d81f Mon Sep 17 00:00:00 2001 From: Lee Dongjin Date: Fri, 7 Sep 2018 10:36:15 -0700 Subject: [PATCH 012/879] [MINOR][SS] Fix kafka-0-10-sql trivials ## What changes were proposed in this pull request? Fix unused imports & outdated comments on `kafka-0-10-sql` module. (Found while I was working on [SPARK-23539](https://github.com/apache/spark/pull/22282)) ## How was this patch tested? Existing unit tests. Closes #22342 from dongjinleekr/feature/fix-kafka-sql-trivials. Authored-by: Lee Dongjin Signed-off-by: Sean Owen (cherry picked from commit 458f5011bd52851632c3592ac35f1573bc904d50) Signed-off-by: Sean Owen --- .../spark/sql/kafka010/KafkaOffsetRangeCalculator.scala | 1 - .../scala/org/apache/spark/sql/kafka010/KafkaRelation.scala | 1 - .../spark/sql/kafka010/KafkaStreamingWriteSupport.scala | 4 ++-- 3 files changed, 2 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala index 6631ae84167c..fb209c724afb 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.sources.v2.DataSourceOptions private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Option[Int]) { require(minPartitions.isEmpty || minPartitions.get > 0) - import KafkaOffsetRangeCalculator._ /** * Calculate the offset ranges that we are going to process this batch. If `minPartitions` * is not set or is set less than or equal the number of `topicPartitions` that we're going to diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index 9d856c9494e1..e6f9d1259e43 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.kafka010 -import java.{util => ju} import java.util.UUID import org.apache.kafka.common.TopicPartition diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala index dc19312f79a2..927c56d9ce82 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala @@ -54,8 +54,8 @@ class KafkaStreamingWriteSupport( } /** - * A [[DataWriterFactory]] for Kafka writing. Will be serialized and sent to executors to generate - * the per-task data writers. + * A [[StreamingDataWriterFactory]] for Kafka writing. Will be serialized and sent to executors to + * generate the per-task data writers. * @param topic The topic that should be written to. If None, topic will be inferred from * a `topic` field in the incoming data. * @param producerParams Parameters for Kafka producers in each task. From 904192ad18ff09cc5874e09b03447dd5f7754963 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Sat, 8 Sep 2018 09:09:14 -0700 Subject: [PATCH 013/879] [SPARK-25345][ML] Deprecate public APIs from ImageSchema ## What changes were proposed in this pull request? Deprecate public APIs from ImageSchema. ## How was this patch tested? N/A Closes #22349 from WeichenXu123/image_api_deprecate. Authored-by: WeichenXu Signed-off-by: Xiangrui Meng (cherry picked from commit 08c02e637ac601df2fe890b8b5a7a049bdb4541b) Signed-off-by: Xiangrui Meng --- .../scala/org/apache/spark/ml/image/ImageSchema.scala | 4 ++++ python/pyspark/ml/image.py | 8 +++++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala b/mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala index dcc40b6668c7..0b13eefdf3f5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala @@ -198,6 +198,8 @@ object ImageSchema { * @return DataFrame with a single column "image" of images; * see ImageSchema for the details */ + @deprecated("use `spark.read.format(\"image\").load(path)` and this `readImages` will be " + + "removed in 3.0.0.", "2.4.0") def readImages(path: String): DataFrame = readImages(path, null, false, -1, false, 1.0, 0) /** @@ -218,6 +220,8 @@ object ImageSchema { * @return DataFrame with a single column "image" of images; * see ImageSchema for the details */ + @deprecated("use `spark.read.format(\"image\").load(path)` and this `readImages` will be " + + "removed in 3.0.0.", "2.4.0") def readImages( path: String, sparkSession: SparkSession, diff --git a/python/pyspark/ml/image.py b/python/pyspark/ml/image.py index ef6785b4a8ed..edb90a357854 100644 --- a/python/pyspark/ml/image.py +++ b/python/pyspark/ml/image.py @@ -25,8 +25,10 @@ """ import sys +import warnings import numpy as np + from pyspark import SparkContext from pyspark.sql.types import Row, _create_row, _parse_datatype_json_string from pyspark.sql import DataFrame, SparkSession @@ -207,6 +209,9 @@ def readImages(self, path, recursive=False, numPartitions=-1, .. note:: If sample ratio is less than 1, sampling uses a PathFilter that is efficient but potentially non-deterministic. + .. note:: Deprecated in 2.4.0. Use `spark.read.format("image").load(path)` instead and + this `readImages` will be removed in 3.0.0. + :param str path: Path to the image directory. :param bool recursive: Recursive search flag. :param int numPartitions: Number of DataFrame partitions. @@ -222,7 +227,8 @@ def readImages(self, path, recursive=False, numPartitions=-1, .. versionadded:: 2.3.0 """ - + warnings.warn("`ImageSchema.readImage` is deprecated. " + + "Use `spark.read.format(\"image\").load(path)` instead.", DeprecationWarning) spark = SparkSession.builder.getOrCreate() image_schema = spark._jvm.org.apache.spark.ml.image.ImageSchema jsession = spark._jsparkSession From 8f7d8a0977647dc96ab9259d306555bbe1c32873 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 8 Sep 2018 10:21:55 -0700 Subject: [PATCH 014/879] [SPARK-25375][SQL][TEST] Reenable qualified perm. function checks in UDFSuite ## What changes were proposed in this pull request? At Spark 2.0.0, SPARK-14335 adds some [commented-out test coverages](https://github.com/apache/spark/pull/12117/files#diff-dd4b39a56fac28b1ced6184453a47358R177 ). This PR enables them because it's supported since 2.0.0. ## How was this patch tested? Pass the Jenkins with re-enabled test coverage. Closes #22363 from dongjoon-hyun/SPARK-25375. Authored-by: Dongjoon Hyun Signed-off-by: gatorsmile (cherry picked from commit 26f74b7cb16869079aa7b60577ac05707101ee68) Signed-off-by: gatorsmile --- .../org/apache/spark/sql/hive/UDFSuite.scala | 18 ++++++++---------- 1 file changed, 8 insertions(+), 10 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala index 88cc42efd0fe..a56c6f73989a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala @@ -141,11 +141,10 @@ class UDFSuite withTempDatabase { dbName => withUserDefinedFunction(functionName -> false) { sql(s"CREATE FUNCTION $dbName.$functionName AS '$functionClass'") - // TODO: Re-enable it after can distinguish qualified and unqualified function name - // checkAnswer( - // sql(s"SELECT $dbName.myuPPer(value) from $testTableName"), - // expectedDF - // ) + checkAnswer( + sql(s"SELECT $dbName.$functionName(value) from $testTableName"), + expectedDF + ) checkAnswer( sql(s"SHOW FUNCTIONS like $dbName.$functionNameUpper"), @@ -174,11 +173,10 @@ class UDFSuite // For this block, drop function command uses default.functionName as the function name. withUserDefinedFunction(s"$dbName.$functionNameUpper" -> false) { sql(s"CREATE FUNCTION $dbName.$functionName AS '$functionClass'") - // TODO: Re-enable it after can distinguish qualified and unqualified function name - // checkAnswer( - // sql(s"SELECT $dbName.myupper(value) from $testTableName"), - // expectedDF - // ) + checkAnswer( + sql(s"SELECT $dbName.$functionName(value) from $testTableName"), + expectedDF + ) sql(s"USE $dbName") From a00a160e1e63ef2aaf3eaeebf2a3e5a5eb05d076 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sun, 9 Sep 2018 21:25:19 +0800 Subject: [PATCH 015/879] Revert [SPARK-10399] [SPARK-23879] [SPARK-23762] [SPARK-25317] ## What changes were proposed in this pull request? When running TPC-DS benchmarks on 2.4 release, npoggi and winglungngai saw more than 10% performance regression on the following queries: q67, q24a and q24b. After we applying the PR https://github.com/apache/spark/pull/22338, the performance regression still exists. If we revert the changes in https://github.com/apache/spark/pull/19222, npoggi and winglungngai found the performance regression was resolved. Thus, this PR is to revert the related changes for unblocking the 2.4 release. In the future release, we still can continue the investigation and find out the root cause of the regression. ## How was this patch tested? The existing test cases Closes #22361 from gatorsmile/revertMemoryBlock. Authored-by: gatorsmile Signed-off-by: Wenchen Fan (cherry picked from commit 0b9ccd55c2986957863dcad3b44ce80403eecfa1) Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/HiveHasher.java | 18 +- .../org/apache/spark/unsafe/Platform.java | 2 +- .../spark/unsafe/array/ByteArrayMethods.java | 15 +- .../apache/spark/unsafe/array/LongArray.java | 17 +- .../spark/unsafe/hash/Murmur3_x86_32.java | 53 ++---- .../unsafe/memory/ByteArrayMemoryBlock.java | 128 ------------- .../unsafe/memory/HeapMemoryAllocator.java | 21 +- .../spark/unsafe/memory/MemoryAllocator.java | 4 +- .../spark/unsafe/memory/MemoryBlock.java | 157 ++------------- .../spark/unsafe/memory/MemoryLocation.java | 96 ++++++---- .../unsafe/memory/OffHeapMemoryBlock.java | 105 ---------- .../unsafe/memory/OnHeapMemoryBlock.java | 132 ------------- .../unsafe/memory/UnsafeMemoryAllocator.java | 21 +- .../apache/spark/unsafe/types/UTF8String.java | 147 +++++++------- .../spark/unsafe/PlatformUtilSuite.java | 4 +- .../spark/unsafe/array/LongArraySuite.java | 5 +- .../unsafe/hash/Murmur3_x86_32Suite.java | 18 -- .../spark/unsafe/memory/MemoryBlockSuite.java | 179 ------------------ .../spark/unsafe/types/UTF8StringSuite.java | 41 ++-- .../spark/memory/TaskMemoryManager.java | 22 +-- .../shuffle/sort/ShuffleInMemorySorter.java | 14 +- .../shuffle/sort/ShuffleSortDataFormat.java | 11 +- .../unsafe/sort/UnsafeExternalSorter.java | 2 +- .../unsafe/sort/UnsafeInMemorySorter.java | 13 +- .../spark/memory/TaskMemoryManagerSuite.java | 2 +- .../util/collection/ExternalSorterSuite.scala | 7 +- .../unsafe/sort/RadixSortSuite.scala | 10 +- .../spark/ml/feature/FeatureHasher.scala | 5 +- .../spark/mllib/feature/HashingTF.scala | 2 +- .../catalyst/expressions/UnsafeArrayData.java | 4 +- .../sql/catalyst/expressions/UnsafeRow.java | 4 +- .../spark/sql/catalyst/expressions/XXH64.java | 47 ++--- .../codegen/UTF8StringBuilder.java | 35 ++-- .../spark/sql/catalyst/expressions/hash.scala | 37 ++-- .../catalyst/expressions/HiveHasherSuite.java | 21 +- .../sql/catalyst/expressions/XXH64Suite.java | 18 +- .../vectorized/OffHeapColumnVector.java | 3 +- .../sql/vectorized/ArrowColumnVector.java | 6 +- .../execution/benchmark/SortBenchmark.scala | 16 +- .../sql/execution/python/RowQueueSuite.scala | 4 +- 40 files changed, 376 insertions(+), 1070 deletions(-) delete mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilderSuite.scala => common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java (51%) delete mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java delete mode 100644 common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java delete mode 100644 common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java diff --git a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java index 62b75ae8aa01..73577437ac50 100644 --- a/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java +++ b/common/unsafe/src/main/java/org/apache/spark/sql/catalyst/expressions/HiveHasher.java @@ -17,8 +17,7 @@ package org.apache.spark.sql.catalyst.expressions; -import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.unsafe.Platform; /** * Simulates Hive's hashing function from Hive v1.2.1 @@ -39,21 +38,12 @@ public static int hashLong(long input) { return (int) ((input >>> 32) ^ input); } - public static int hashUnsafeBytesBlock(MemoryBlock mb) { - long lengthInBytes = mb.size(); + public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes) { assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int result = 0; - for (long i = 0; i < lengthInBytes; i++) { - result = (result * 31) + (int) mb.getByte(i); + for (int i = 0; i < lengthInBytes; i++) { + result = (result * 31) + (int) Platform.getByte(base, offset + i); } return result; } - - public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes) { - return hashUnsafeBytesBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes)); - } - - public static int hashUTF8String(UTF8String str) { - return hashUnsafeBytesBlock(str.getMemoryBlock()); - } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index 54dcadf3a775..aca6fca00c48 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -187,7 +187,7 @@ public static void setMemory(long address, byte value, long size) { } public static void copyMemory( - Object src, long srcOffset, Object dst, long dstOffset, long length) { + Object src, long srcOffset, Object dst, long dstOffset, long length) { // Check if dstOffset is before or after srcOffset to determine if we should copy // forward or backwards. This is necessary in case src and dst overlap. if (dstOffset < srcOffset) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java index ef0f78d95d1e..cec8c30887e2 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/ByteArrayMethods.java @@ -18,7 +18,6 @@ package org.apache.spark.unsafe.array; import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.memory.MemoryBlock; public class ByteArrayMethods { @@ -53,25 +52,15 @@ public static long roundNumberOfBytesToNearestWord(long numBytes) { public static int MAX_ROUNDED_ARRAY_LENGTH = Integer.MAX_VALUE - 15; private static final boolean unaligned = Platform.unaligned(); - /** - * MemoryBlock equality check for MemoryBlocks. - * @return true if the arrays are equal, false otherwise - */ - public static boolean arrayEqualsBlock( - MemoryBlock leftBase, long leftOffset, MemoryBlock rightBase, long rightOffset, long length) { - return arrayEquals(leftBase.getBaseObject(), leftBase.getBaseOffset() + leftOffset, - rightBase.getBaseObject(), rightBase.getBaseOffset() + rightOffset, length); - } - /** * Optimized byte array equality check for byte arrays. * @return true if the arrays are equal, false otherwise */ public static boolean arrayEquals( - Object leftBase, long leftOffset, Object rightBase, long rightOffset, long length) { + Object leftBase, long leftOffset, Object rightBase, long rightOffset, final long length) { int i = 0; - // check if starts align and we can get both offsets to be aligned + // check if stars align and we can get both offsets to be aligned if ((leftOffset % 8) == (rightOffset % 8)) { while ((leftOffset + i) % 8 != 0 && i < length) { if (Platform.getByte(leftBase, leftOffset + i) != diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java index b74d2de0691d..2cd39bd60c2a 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -17,6 +17,7 @@ package org.apache.spark.unsafe.array; +import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.memory.MemoryBlock; /** @@ -32,12 +33,16 @@ public final class LongArray { private static final long WIDTH = 8; private final MemoryBlock memory; + private final Object baseObj; + private final long baseOffset; private final long length; public LongArray(MemoryBlock memory) { assert memory.size() < (long) Integer.MAX_VALUE * 8: "Array size >= Integer.MAX_VALUE elements"; this.memory = memory; + this.baseObj = memory.getBaseObject(); + this.baseOffset = memory.getBaseOffset(); this.length = memory.size() / WIDTH; } @@ -46,11 +51,11 @@ public MemoryBlock memoryBlock() { } public Object getBaseObject() { - return memory.getBaseObject(); + return baseObj; } public long getBaseOffset() { - return memory.getBaseOffset(); + return baseOffset; } /** @@ -64,8 +69,8 @@ public long size() { * Fill this all with 0L. */ public void zeroOut() { - for (long off = 0; off < length * WIDTH; off += WIDTH) { - memory.putLong(off, 0); + for (long off = baseOffset; off < baseOffset + length * WIDTH; off += WIDTH) { + Platform.putLong(baseObj, off, 0); } } @@ -75,7 +80,7 @@ public void zeroOut() { public void set(int index, long value) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - memory.putLong(index * WIDTH, value); + Platform.putLong(baseObj, baseOffset + index * WIDTH, value); } /** @@ -84,6 +89,6 @@ public void set(int index, long value) { public long get(int index) { assert index >= 0 : "index (" + index + ") should >= 0"; assert index < length : "index (" + index + ") should < length (" + length + ")"; - return memory.getLong(index * WIDTH); + return Platform.getLong(baseObj, baseOffset + index * WIDTH); } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java index 566f11615430..d239de6083ad 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/hash/Murmur3_x86_32.java @@ -17,11 +17,7 @@ package org.apache.spark.unsafe.hash; -import com.google.common.primitives.Ints; - import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.memory.MemoryBlock; -import org.apache.spark.unsafe.types.UTF8String; /** * 32-bit Murmur3 hasher. This is based on Guava's Murmur3_32HashFunction. @@ -53,74 +49,49 @@ public static int hashInt(int input, int seed) { } public int hashUnsafeWords(Object base, long offset, int lengthInBytes) { - return hashUnsafeWordsBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); + return hashUnsafeWords(base, offset, lengthInBytes, seed); } - public static int hashUnsafeWordsBlock(MemoryBlock base, int seed) { + public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) { // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. - int lengthInBytes = Ints.checkedCast(base.size()); assert (lengthInBytes % 8 == 0): "lengthInBytes must be a multiple of 8 (word-aligned)"; - int h1 = hashBytesByIntBlock(base, lengthInBytes, seed); + int h1 = hashBytesByInt(base, offset, lengthInBytes, seed); return fmix(h1, lengthInBytes); } - public static int hashUnsafeWords(Object base, long offset, int lengthInBytes, int seed) { - // This is based on Guava's `Murmur32_Hasher.processRemaining(ByteBuffer)` method. - return hashUnsafeWordsBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); - } - - public static int hashUnsafeBytesBlock(MemoryBlock base, int seed) { - return hashUnsafeBytesBlock(base, Ints.checkedCast(base.size()), seed); - } - - private static int hashUnsafeBytesBlock(MemoryBlock base, int lengthInBytes, int seed) { + public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { // This is not compatible with original and another implementations. // But remain it for backward compatibility for the components existing before 2.3. assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByIntBlock(base, lengthAligned, seed); - long offset = base.getBaseOffset(); - Object o = base.getBaseObject(); + int h1 = hashBytesByInt(base, offset, lengthAligned, seed); for (int i = lengthAligned; i < lengthInBytes; i++) { - int halfWord = Platform.getByte(o, offset + i); + int halfWord = Platform.getByte(base, offset + i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); } return fmix(h1, lengthInBytes); } - public static int hashUTF8String(UTF8String str, int seed) { - return hashUnsafeBytesBlock(str.getMemoryBlock(), str.numBytes(), seed); - } - - public static int hashUnsafeBytes(Object base, long offset, int lengthInBytes, int seed) { - return hashUnsafeBytesBlock(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); - } - public static int hashUnsafeBytes2(Object base, long offset, int lengthInBytes, int seed) { - return hashUnsafeBytes2Block(MemoryBlock.allocateFromObject(base, offset, lengthInBytes), seed); - } - - public static int hashUnsafeBytes2Block(MemoryBlock base, int seed) { - // This is compatible with original and other implementations. + // This is compatible with original and another implementations. // Use this method for new components after Spark 2.3. - int lengthInBytes = Ints.checkedCast(base.size()); - assert (lengthInBytes >= 0) : "lengthInBytes cannot be negative"; + assert (lengthInBytes >= 0): "lengthInBytes cannot be negative"; int lengthAligned = lengthInBytes - lengthInBytes % 4; - int h1 = hashBytesByIntBlock(base, lengthAligned, seed); + int h1 = hashBytesByInt(base, offset, lengthAligned, seed); int k1 = 0; for (int i = lengthAligned, shift = 0; i < lengthInBytes; i++, shift += 8) { - k1 ^= (base.getByte(i) & 0xFF) << shift; + k1 ^= (Platform.getByte(base, offset + i) & 0xFF) << shift; } h1 ^= mixK1(k1); return fmix(h1, lengthInBytes); } - private static int hashBytesByIntBlock(MemoryBlock base, int lengthInBytes, int seed) { + private static int hashBytesByInt(Object base, long offset, int lengthInBytes, int seed) { assert (lengthInBytes % 4 == 0); int h1 = seed; for (int i = 0; i < lengthInBytes; i += 4) { - int halfWord = base.getInt(i); + int halfWord = Platform.getInt(base, offset + i); int k1 = mixK1(halfWord); h1 = mixH1(h1, k1); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java deleted file mode 100644 index 9f238632bc87..000000000000 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/ByteArrayMemoryBlock.java +++ /dev/null @@ -1,128 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.unsafe.memory; - -import com.google.common.primitives.Ints; - -import org.apache.spark.unsafe.Platform; - -/** - * A consecutive block of memory with a byte array on Java heap. - */ -public final class ByteArrayMemoryBlock extends MemoryBlock { - - private final byte[] array; - - public ByteArrayMemoryBlock(byte[] obj, long offset, long size) { - super(obj, offset, size); - this.array = obj; - assert(offset + size <= Platform.BYTE_ARRAY_OFFSET + obj.length) : - "The sum of size " + size + " and offset " + offset + " should not be larger than " + - "the size of the given memory space " + (obj.length + Platform.BYTE_ARRAY_OFFSET); - } - - public ByteArrayMemoryBlock(long length) { - this(new byte[Ints.checkedCast(length)], Platform.BYTE_ARRAY_OFFSET, length); - } - - @Override - public MemoryBlock subBlock(long offset, long size) { - checkSubBlockRange(offset, size); - if (offset == 0 && size == this.size()) return this; - return new ByteArrayMemoryBlock(array, this.offset + offset, size); - } - - public byte[] getByteArray() { return array; } - - /** - * Creates a memory block pointing to the memory used by the byte array. - */ - public static ByteArrayMemoryBlock fromArray(final byte[] array) { - return new ByteArrayMemoryBlock(array, Platform.BYTE_ARRAY_OFFSET, array.length); - } - - @Override - public int getInt(long offset) { - return Platform.getInt(array, this.offset + offset); - } - - @Override - public void putInt(long offset, int value) { - Platform.putInt(array, this.offset + offset, value); - } - - @Override - public boolean getBoolean(long offset) { - return Platform.getBoolean(array, this.offset + offset); - } - - @Override - public void putBoolean(long offset, boolean value) { - Platform.putBoolean(array, this.offset + offset, value); - } - - @Override - public byte getByte(long offset) { - return array[(int)(this.offset + offset - Platform.BYTE_ARRAY_OFFSET)]; - } - - @Override - public void putByte(long offset, byte value) { - array[(int)(this.offset + offset - Platform.BYTE_ARRAY_OFFSET)] = value; - } - - @Override - public short getShort(long offset) { - return Platform.getShort(array, this.offset + offset); - } - - @Override - public void putShort(long offset, short value) { - Platform.putShort(array, this.offset + offset, value); - } - - @Override - public long getLong(long offset) { - return Platform.getLong(array, this.offset + offset); - } - - @Override - public void putLong(long offset, long value) { - Platform.putLong(array, this.offset + offset, value); - } - - @Override - public float getFloat(long offset) { - return Platform.getFloat(array, this.offset + offset); - } - - @Override - public void putFloat(long offset, float value) { - Platform.putFloat(array, this.offset + offset, value); - } - - @Override - public double getDouble(long offset) { - return Platform.getDouble(array, this.offset + offset); - } - - @Override - public void putDouble(long offset, double value) { - Platform.putDouble(array, this.offset + offset, value); - } -} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java index 36caf80888cd..2733760dd19e 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java @@ -23,6 +23,8 @@ import java.util.LinkedList; import java.util.Map; +import org.apache.spark.unsafe.Platform; + /** * A simple {@link MemoryAllocator} that can allocate up to 16GB using a JVM long primitive array. */ @@ -56,7 +58,7 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { final long[] array = arrayReference.get(); if (array != null) { assert (array.length * 8L >= size); - MemoryBlock memory = OnHeapMemoryBlock.fromArray(array, size); + MemoryBlock memory = new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } @@ -68,7 +70,7 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { } } long[] array = new long[numWords]; - MemoryBlock memory = OnHeapMemoryBlock.fromArray(array, size); + MemoryBlock memory = new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } @@ -77,13 +79,12 @@ public MemoryBlock allocate(long size) throws OutOfMemoryError { @Override public void free(MemoryBlock memory) { - assert(memory instanceof OnHeapMemoryBlock); - assert (memory.getBaseObject() != null) : + assert (memory.obj != null) : "baseObject was null; are you trying to use the on-heap allocator to free off-heap memory?"; - assert (memory.getPageNumber() != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : + assert (memory.pageNumber != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : "page has already been freed"; - assert ((memory.getPageNumber() == MemoryBlock.NO_PAGE_NUMBER) - || (memory.getPageNumber() == MemoryBlock.FREED_IN_TMM_PAGE_NUMBER)) : + assert ((memory.pageNumber == MemoryBlock.NO_PAGE_NUMBER) + || (memory.pageNumber == MemoryBlock.FREED_IN_TMM_PAGE_NUMBER)) : "TMM-allocated pages must first be freed via TMM.freePage(), not directly in allocator " + "free()"; @@ -93,12 +94,12 @@ public void free(MemoryBlock memory) { } // Mark the page as freed (so we can detect double-frees). - memory.setPageNumber(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER); + memory.pageNumber = MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER; // As an additional layer of defense against use-after-free bugs, we mutate the // MemoryBlock to null out its reference to the long[] array. - long[] array = ((OnHeapMemoryBlock)memory).getLongArray(); - memory.resetObjAndOffset(); + long[] array = (long[]) memory.obj; + memory.setObjAndOffset(null, 0); long alignedSize = ((size + 7) / 8) * 8; if (shouldPool(alignedSize)) { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java index 38315fb97b46..7b588681d979 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryAllocator.java @@ -38,7 +38,7 @@ public interface MemoryAllocator { void free(MemoryBlock memory); - UnsafeMemoryAllocator UNSAFE = new UnsafeMemoryAllocator(); + MemoryAllocator UNSAFE = new UnsafeMemoryAllocator(); - HeapMemoryAllocator HEAP = new HeapMemoryAllocator(); + MemoryAllocator HEAP = new HeapMemoryAllocator(); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java index ca7213bbf92d..c333857358d3 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryBlock.java @@ -22,10 +22,10 @@ import org.apache.spark.unsafe.Platform; /** - * A representation of a consecutive memory block in Spark. It defines the common interfaces - * for memory accessing and mutating. + * A consecutive block of memory, starting at a {@link MemoryLocation} with a fixed size. */ -public abstract class MemoryBlock { +public class MemoryBlock extends MemoryLocation { + /** Special `pageNumber` value for pages which were not allocated by TaskMemoryManagers */ public static final int NO_PAGE_NUMBER = -1; @@ -45,163 +45,38 @@ public abstract class MemoryBlock { */ public static final int FREED_IN_ALLOCATOR_PAGE_NUMBER = -3; - @Nullable - protected Object obj; - - protected long offset; - - protected long length; + private final long length; /** * Optional page number; used when this MemoryBlock represents a page allocated by a - * TaskMemoryManager. This field can be updated using setPageNumber method so that - * this can be modified by the TaskMemoryManager, which lives in a different package. + * TaskMemoryManager. This field is public so that it can be modified by the TaskMemoryManager, + * which lives in a different package. */ - private int pageNumber = NO_PAGE_NUMBER; + public int pageNumber = NO_PAGE_NUMBER; - protected MemoryBlock(@Nullable Object obj, long offset, long length) { - if (offset < 0 || length < 0) { - throw new IllegalArgumentException( - "Length " + length + " and offset " + offset + "must be non-negative"); - } - this.obj = obj; - this.offset = offset; + public MemoryBlock(@Nullable Object obj, long offset, long length) { + super(obj, offset); this.length = length; } - protected MemoryBlock() { - this(null, 0, 0); - } - - public final Object getBaseObject() { - return obj; - } - - public final long getBaseOffset() { - return offset; - } - - public void resetObjAndOffset() { - this.obj = null; - this.offset = 0; - } - /** * Returns the size of the memory block. */ - public final long size() { + public long size() { return length; } - public final void setPageNumber(int pageNum) { - pageNumber = pageNum; - } - - public final int getPageNumber() { - return pageNumber; - } - - /** - * Fills the memory block with the specified byte value. - */ - public final void fill(byte value) { - Platform.setMemory(obj, offset, length, value); - } - - /** - * Instantiate MemoryBlock for given object type with new offset - */ - public static final MemoryBlock allocateFromObject(Object obj, long offset, long length) { - MemoryBlock mb = null; - if (obj instanceof byte[]) { - byte[] array = (byte[])obj; - mb = new ByteArrayMemoryBlock(array, offset, length); - } else if (obj instanceof long[]) { - long[] array = (long[])obj; - mb = new OnHeapMemoryBlock(array, offset, length); - } else if (obj == null) { - // we assume that to pass null pointer means off-heap - mb = new OffHeapMemoryBlock(offset, length); - } else { - throw new UnsupportedOperationException( - "Instantiate MemoryBlock for type " + obj.getClass() + " is not supported now"); - } - return mb; - } - /** - * Just instantiate the sub-block with the same type of MemoryBlock with the new size and relative - * offset from the original offset. The data is not copied. - * If parameters are invalid, an exception is thrown. + * Creates a memory block pointing to the memory used by the long array. */ - public abstract MemoryBlock subBlock(long offset, long size); - - protected void checkSubBlockRange(long offset, long size) { - if (offset < 0 || size < 0) { - throw new ArrayIndexOutOfBoundsException( - "Size " + size + " and offset " + offset + " must be non-negative"); - } - if (offset + size > length) { - throw new ArrayIndexOutOfBoundsException("The sum of size " + size + " and offset " + - offset + " should not be larger than the length " + length + " in the MemoryBlock"); - } + public static MemoryBlock fromLongArray(final long[] array) { + return new MemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8L); } /** - * getXXX/putXXX does not ensure guarantee behavior if the offset is invalid. e.g cause illegal - * memory access, throw an exception, or etc. - * getXXX/putXXX uses an index based on this.offset that includes the size of metadata such as - * JVM object header. The offset is 0-based and is expected as an logical offset in the memory - * block. + * Fills the memory block with the specified byte value. */ - public abstract int getInt(long offset); - - public abstract void putInt(long offset, int value); - - public abstract boolean getBoolean(long offset); - - public abstract void putBoolean(long offset, boolean value); - - public abstract byte getByte(long offset); - - public abstract void putByte(long offset, byte value); - - public abstract short getShort(long offset); - - public abstract void putShort(long offset, short value); - - public abstract long getLong(long offset); - - public abstract void putLong(long offset, long value); - - public abstract float getFloat(long offset); - - public abstract void putFloat(long offset, float value); - - public abstract double getDouble(long offset); - - public abstract void putDouble(long offset, double value); - - public static final void copyMemory( - MemoryBlock src, long srcOffset, MemoryBlock dst, long dstOffset, long length) { - assert(srcOffset + length <= src.length && dstOffset + length <= dst.length); - Platform.copyMemory(src.getBaseObject(), src.getBaseOffset() + srcOffset, - dst.getBaseObject(), dst.getBaseOffset() + dstOffset, length); - } - - public static final void copyMemory(MemoryBlock src, MemoryBlock dst, long length) { - assert(length <= src.length && length <= dst.length); - Platform.copyMemory(src.getBaseObject(), src.getBaseOffset(), - dst.getBaseObject(), dst.getBaseOffset(), length); - } - - public final void copyFrom(Object src, long srcOffset, long dstOffset, long length) { - assert(length <= this.length - srcOffset); - Platform.copyMemory(src, srcOffset, obj, offset + dstOffset, length); - } - - public final void writeTo(long srcOffset, Object dst, long dstOffset, long length) { - assert(length <= this.length - srcOffset); - Platform.copyMemory(obj, offset + srcOffset, dst, dstOffset, length); + public void fill(byte value) { + Platform.setMemory(obj, offset, length, value); } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilderSuite.scala b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java similarity index 51% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilderSuite.scala rename to common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java index 1b25a4b191f8..74ebc87dc978 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilderSuite.scala +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java @@ -1,42 +1,54 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.expressions.codegen - -import org.apache.spark.SparkFunSuite -import org.apache.spark.unsafe.types.UTF8String - -class UTF8StringBuilderSuite extends SparkFunSuite { - - test("basic test") { - val sb = new UTF8StringBuilder() - assert(sb.build() === UTF8String.EMPTY_UTF8) - - sb.append("") - assert(sb.build() === UTF8String.EMPTY_UTF8) - - sb.append("abcd") - assert(sb.build() === UTF8String.fromString("abcd")) - - sb.append(UTF8String.fromString("1234")) - assert(sb.build() === UTF8String.fromString("abcd1234")) - - // expect to grow an internal buffer - sb.append(UTF8String.fromString("efgijk567890")) - assert(sb.build() === UTF8String.fromString("abcd1234efgijk567890")) - } -} +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.unsafe.memory; + +import javax.annotation.Nullable; + +/** + * A memory location. Tracked either by a memory address (with off-heap allocation), + * or by an offset from a JVM object (in-heap allocation). + */ +public class MemoryLocation { + + @Nullable + Object obj; + + long offset; + + public MemoryLocation(@Nullable Object obj, long offset) { + this.obj = obj; + this.offset = offset; + } + + public MemoryLocation() { + this(null, 0); + } + + public void setObjAndOffset(Object newObj, long newOffset) { + this.obj = newObj; + this.offset = newOffset; + } + + public final Object getBaseObject() { + return obj; + } + + public final long getBaseOffset() { + return offset; + } +} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java deleted file mode 100644 index 3431b08980eb..000000000000 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OffHeapMemoryBlock.java +++ /dev/null @@ -1,105 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.unsafe.memory; - -import org.apache.spark.unsafe.Platform; - -public class OffHeapMemoryBlock extends MemoryBlock { - public static final OffHeapMemoryBlock NULL = new OffHeapMemoryBlock(0, 0); - - public OffHeapMemoryBlock(long address, long size) { - super(null, address, size); - } - - @Override - public MemoryBlock subBlock(long offset, long size) { - checkSubBlockRange(offset, size); - if (offset == 0 && size == this.size()) return this; - return new OffHeapMemoryBlock(this.offset + offset, size); - } - - @Override - public final int getInt(long offset) { - return Platform.getInt(null, this.offset + offset); - } - - @Override - public final void putInt(long offset, int value) { - Platform.putInt(null, this.offset + offset, value); - } - - @Override - public final boolean getBoolean(long offset) { - return Platform.getBoolean(null, this.offset + offset); - } - - @Override - public final void putBoolean(long offset, boolean value) { - Platform.putBoolean(null, this.offset + offset, value); - } - - @Override - public final byte getByte(long offset) { - return Platform.getByte(null, this.offset + offset); - } - - @Override - public final void putByte(long offset, byte value) { - Platform.putByte(null, this.offset + offset, value); - } - - @Override - public final short getShort(long offset) { - return Platform.getShort(null, this.offset + offset); - } - - @Override - public final void putShort(long offset, short value) { - Platform.putShort(null, this.offset + offset, value); - } - - @Override - public final long getLong(long offset) { - return Platform.getLong(null, this.offset + offset); - } - - @Override - public final void putLong(long offset, long value) { - Platform.putLong(null, this.offset + offset, value); - } - - @Override - public final float getFloat(long offset) { - return Platform.getFloat(null, this.offset + offset); - } - - @Override - public final void putFloat(long offset, float value) { - Platform.putFloat(null, this.offset + offset, value); - } - - @Override - public final double getDouble(long offset) { - return Platform.getDouble(null, this.offset + offset); - } - - @Override - public final void putDouble(long offset, double value) { - Platform.putDouble(null, this.offset + offset, value); - } -} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java deleted file mode 100644 index ee42bc27c9c5..000000000000 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/OnHeapMemoryBlock.java +++ /dev/null @@ -1,132 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.unsafe.memory; - -import com.google.common.primitives.Ints; - -import org.apache.spark.unsafe.Platform; - -/** - * A consecutive block of memory with a long array on Java heap. - */ -public final class OnHeapMemoryBlock extends MemoryBlock { - - private final long[] array; - - public OnHeapMemoryBlock(long[] obj, long offset, long size) { - super(obj, offset, size); - this.array = obj; - assert(offset + size <= obj.length * 8L + Platform.LONG_ARRAY_OFFSET) : - "The sum of size " + size + " and offset " + offset + " should not be larger than " + - "the size of the given memory space " + (obj.length * 8L + Platform.LONG_ARRAY_OFFSET); - } - - public OnHeapMemoryBlock(long size) { - this(new long[Ints.checkedCast((size + 7) / 8)], Platform.LONG_ARRAY_OFFSET, size); - } - - @Override - public MemoryBlock subBlock(long offset, long size) { - checkSubBlockRange(offset, size); - if (offset == 0 && size == this.size()) return this; - return new OnHeapMemoryBlock(array, this.offset + offset, size); - } - - public long[] getLongArray() { return array; } - - /** - * Creates a memory block pointing to the memory used by the long array. - */ - public static OnHeapMemoryBlock fromArray(final long[] array) { - return new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, array.length * 8L); - } - - public static OnHeapMemoryBlock fromArray(final long[] array, long size) { - return new OnHeapMemoryBlock(array, Platform.LONG_ARRAY_OFFSET, size); - } - - @Override - public int getInt(long offset) { - return Platform.getInt(array, this.offset + offset); - } - - @Override - public void putInt(long offset, int value) { - Platform.putInt(array, this.offset + offset, value); - } - - @Override - public boolean getBoolean(long offset) { - return Platform.getBoolean(array, this.offset + offset); - } - - @Override - public void putBoolean(long offset, boolean value) { - Platform.putBoolean(array, this.offset + offset, value); - } - - @Override - public byte getByte(long offset) { - return Platform.getByte(array, this.offset + offset); - } - - @Override - public void putByte(long offset, byte value) { - Platform.putByte(array, this.offset + offset, value); - } - - @Override - public short getShort(long offset) { - return Platform.getShort(array, this.offset + offset); - } - - @Override - public void putShort(long offset, short value) { - Platform.putShort(array, this.offset + offset, value); - } - - @Override - public long getLong(long offset) { - return Platform.getLong(array, this.offset + offset); - } - - @Override - public void putLong(long offset, long value) { - Platform.putLong(array, this.offset + offset, value); - } - - @Override - public float getFloat(long offset) { - return Platform.getFloat(array, this.offset + offset); - } - - @Override - public void putFloat(long offset, float value) { - Platform.putFloat(array, this.offset + offset, value); - } - - @Override - public double getDouble(long offset) { - return Platform.getDouble(array, this.offset + offset); - } - - @Override - public void putDouble(long offset, double value) { - Platform.putDouble(array, this.offset + offset, value); - } -} diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java index 5310bdf2779a..4368fb615ba1 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/UnsafeMemoryAllocator.java @@ -25,9 +25,9 @@ public class UnsafeMemoryAllocator implements MemoryAllocator { @Override - public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { + public MemoryBlock allocate(long size) throws OutOfMemoryError { long address = Platform.allocateMemory(size); - OffHeapMemoryBlock memory = new OffHeapMemoryBlock(address, size); + MemoryBlock memory = new MemoryBlock(null, address, size); if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_CLEAN_VALUE); } @@ -36,25 +36,22 @@ public OffHeapMemoryBlock allocate(long size) throws OutOfMemoryError { @Override public void free(MemoryBlock memory) { - assert(memory instanceof OffHeapMemoryBlock) : - "UnsafeMemoryAllocator can only free OffHeapMemoryBlock."; - if (memory == OffHeapMemoryBlock.NULL) return; - assert (memory.getPageNumber() != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : + assert (memory.obj == null) : + "baseObject not null; are you trying to use the off-heap allocator to free on-heap memory?"; + assert (memory.pageNumber != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : "page has already been freed"; - assert ((memory.getPageNumber() == MemoryBlock.NO_PAGE_NUMBER) - || (memory.getPageNumber() == MemoryBlock.FREED_IN_TMM_PAGE_NUMBER)) : + assert ((memory.pageNumber == MemoryBlock.NO_PAGE_NUMBER) + || (memory.pageNumber == MemoryBlock.FREED_IN_TMM_PAGE_NUMBER)) : "TMM-allocated pages must be freed via TMM.freePage(), not directly in allocator free()"; if (MemoryAllocator.MEMORY_DEBUG_FILL_ENABLED) { memory.fill(MemoryAllocator.MEMORY_DEBUG_FILL_FREED_VALUE); } - Platform.freeMemory(memory.offset); - // As an additional layer of defense against use-after-free bugs, we mutate the // MemoryBlock to reset its pointer. - memory.resetObjAndOffset(); + memory.offset = 0; // Mark the page as freed (so we can detect double-frees). - memory.setPageNumber(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER); + memory.pageNumber = MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER; } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java index e91fc4391425..dff4a73f3e9d 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java @@ -34,8 +34,6 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; -import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; -import org.apache.spark.unsafe.memory.MemoryBlock; import static org.apache.spark.unsafe.Platform.*; @@ -53,13 +51,12 @@ public final class UTF8String implements Comparable, Externalizable, // These are only updated by readExternal() or read() @Nonnull - private MemoryBlock base; - // While numBytes has the same value as base.size(), to keep as int avoids cast from long to int + private Object base; + private long offset; private int numBytes; - public MemoryBlock getMemoryBlock() { return base; } - public Object getBaseObject() { return base.getBaseObject(); } - public long getBaseOffset() { return base.getBaseOffset(); } + public Object getBaseObject() { return base; } + public long getBaseOffset() { return offset; } /** * A char in UTF-8 encoding can take 1-4 bytes depending on the first byte which @@ -112,8 +109,7 @@ public final class UTF8String implements Comparable, Externalizable, */ public static UTF8String fromBytes(byte[] bytes) { if (bytes != null) { - return new UTF8String( - new ByteArrayMemoryBlock(bytes, BYTE_ARRAY_OFFSET, bytes.length)); + return new UTF8String(bytes, BYTE_ARRAY_OFFSET, bytes.length); } else { return null; } @@ -126,13 +122,19 @@ public static UTF8String fromBytes(byte[] bytes) { */ public static UTF8String fromBytes(byte[] bytes, int offset, int numBytes) { if (bytes != null) { - return new UTF8String( - new ByteArrayMemoryBlock(bytes, BYTE_ARRAY_OFFSET + offset, numBytes)); + return new UTF8String(bytes, BYTE_ARRAY_OFFSET + offset, numBytes); } else { return null; } } + /** + * Creates an UTF8String from given address (base and offset) and length. + */ + public static UTF8String fromAddress(Object base, long offset, int numBytes) { + return new UTF8String(base, offset, numBytes); + } + /** * Creates an UTF8String from String. */ @@ -149,13 +151,16 @@ public static UTF8String blankString(int length) { return fromBytes(spaces); } - public UTF8String(MemoryBlock base) { + protected UTF8String(Object base, long offset, int numBytes) { this.base = base; - this.numBytes = Ints.checkedCast(base.size()); + this.offset = offset; + this.numBytes = numBytes; } // for serialization - public UTF8String() {} + public UTF8String() { + this(null, 0, 0); + } /** * Writes the content of this string into a memory address, identified by an object and an offset. @@ -163,7 +168,7 @@ public UTF8String() {} * bytes in this string. */ public void writeToMemory(Object target, long targetOffset) { - base.writeTo(0, target, targetOffset, numBytes); + Platform.copyMemory(base, offset, target, targetOffset, numBytes); } public void writeTo(ByteBuffer buffer) { @@ -183,9 +188,8 @@ public void writeTo(ByteBuffer buffer) { */ @Nonnull public ByteBuffer getByteBuffer() { - long offset = base.getBaseOffset(); - if (base instanceof ByteArrayMemoryBlock && offset >= BYTE_ARRAY_OFFSET) { - final byte[] bytes = ((ByteArrayMemoryBlock) base).getByteArray(); + if (base instanceof byte[] && offset >= BYTE_ARRAY_OFFSET) { + final byte[] bytes = (byte[]) base; // the offset includes an object header... this is only needed for unsafe copies final long arrayOffset = offset - BYTE_ARRAY_OFFSET; @@ -252,12 +256,12 @@ public long getPrefix() { long mask = 0; if (IS_LITTLE_ENDIAN) { if (numBytes >= 8) { - p = base.getLong(0); + p = Platform.getLong(base, offset); } else if (numBytes > 4) { - p = base.getLong(0); + p = Platform.getLong(base, offset); mask = (1L << (8 - numBytes) * 8) - 1; } else if (numBytes > 0) { - p = (long) base.getInt(0); + p = (long) Platform.getInt(base, offset); mask = (1L << (8 - numBytes) * 8) - 1; } else { p = 0; @@ -266,12 +270,12 @@ public long getPrefix() { } else { // byteOrder == ByteOrder.BIG_ENDIAN if (numBytes >= 8) { - p = base.getLong(0); + p = Platform.getLong(base, offset); } else if (numBytes > 4) { - p = base.getLong(0); + p = Platform.getLong(base, offset); mask = (1L << (8 - numBytes) * 8) - 1; } else if (numBytes > 0) { - p = ((long) base.getInt(0)) << 32; + p = ((long) Platform.getInt(base, offset)) << 32; mask = (1L << (8 - numBytes) * 8) - 1; } else { p = 0; @@ -286,13 +290,12 @@ public long getPrefix() { */ public byte[] getBytes() { // avoid copy if `base` is `byte[]` - long offset = base.getBaseOffset(); - if (offset == BYTE_ARRAY_OFFSET && base instanceof ByteArrayMemoryBlock - && (((ByteArrayMemoryBlock) base).getByteArray()).length == numBytes) { - return ((ByteArrayMemoryBlock) base).getByteArray(); + if (offset == BYTE_ARRAY_OFFSET && base instanceof byte[] + && ((byte[]) base).length == numBytes) { + return (byte[]) base; } else { byte[] bytes = new byte[numBytes]; - base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, numBytes); + copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); return bytes; } } @@ -322,7 +325,7 @@ public UTF8String substring(final int start, final int until) { if (i > j) { byte[] bytes = new byte[i - j]; - base.writeTo(j, bytes, BYTE_ARRAY_OFFSET, i - j); + copyMemory(base, offset + j, bytes, BYTE_ARRAY_OFFSET, i - j); return fromBytes(bytes); } else { return EMPTY_UTF8; @@ -363,14 +366,14 @@ public boolean contains(final UTF8String substring) { * Returns the byte at position `i`. */ private byte getByte(int i) { - return base.getByte(i); + return Platform.getByte(base, offset + i); } private boolean matchAt(final UTF8String s, int pos) { if (s.numBytes + pos > numBytes || pos < 0) { return false; } - return ByteArrayMethods.arrayEqualsBlock(base, pos, s.base, 0, s.numBytes); + return ByteArrayMethods.arrayEquals(base, offset + pos, s.base, s.offset, s.numBytes); } public boolean startsWith(final UTF8String prefix) { @@ -497,7 +500,8 @@ public int findInSet(UTF8String match) { for (int i = 0; i < numBytes; i++) { if (getByte(i) == (byte) ',') { if (i - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEqualsBlock(base, lastComma + 1, match.base, 0, match.numBytes)) { + ByteArrayMethods.arrayEquals(base, offset + (lastComma + 1), match.base, match.offset, + match.numBytes)) { return n; } lastComma = i; @@ -505,7 +509,8 @@ public int findInSet(UTF8String match) { } } if (numBytes - (lastComma + 1) == match.numBytes && - ByteArrayMethods.arrayEqualsBlock(base, lastComma + 1, match.base, 0, match.numBytes)) { + ByteArrayMethods.arrayEquals(base, offset + (lastComma + 1), match.base, match.offset, + match.numBytes)) { return n; } return 0; @@ -520,7 +525,7 @@ public int findInSet(UTF8String match) { private UTF8String copyUTF8String(int start, int end) { int len = end - start + 1; byte[] newBytes = new byte[len]; - base.writeTo(start, newBytes, BYTE_ARRAY_OFFSET, len); + copyMemory(base, offset + start, newBytes, BYTE_ARRAY_OFFSET, len); return UTF8String.fromBytes(newBytes); } @@ -667,7 +672,8 @@ public UTF8String reverse() { int i = 0; // position in byte while (i < numBytes) { int len = numBytesForFirstByte(getByte(i)); - base.writeTo(i, result, BYTE_ARRAY_OFFSET + result.length - i - len, len); + copyMemory(this.base, this.offset + i, result, + BYTE_ARRAY_OFFSET + result.length - i - len, len); i += len; } @@ -681,7 +687,7 @@ public UTF8String repeat(int times) { } byte[] newBytes = new byte[numBytes * times]; - base.writeTo(0, newBytes, BYTE_ARRAY_OFFSET, numBytes); + copyMemory(this.base, this.offset, newBytes, BYTE_ARRAY_OFFSET, numBytes); int copied = 1; while (copied < times) { @@ -718,7 +724,7 @@ public int indexOf(UTF8String v, int start) { if (i + v.numBytes > numBytes) { return -1; } - if (ByteArrayMethods.arrayEqualsBlock(base, i, v.base, 0, v.numBytes)) { + if (ByteArrayMethods.arrayEquals(base, offset + i, v.base, v.offset, v.numBytes)) { return c; } i += numBytesForFirstByte(getByte(i)); @@ -734,7 +740,7 @@ public int indexOf(UTF8String v, int start) { private int find(UTF8String str, int start) { assert (str.numBytes > 0); while (start <= numBytes - str.numBytes) { - if (ByteArrayMethods.arrayEqualsBlock(base, start, str.base, 0, str.numBytes)) { + if (ByteArrayMethods.arrayEquals(base, offset + start, str.base, str.offset, str.numBytes)) { return start; } start += 1; @@ -748,7 +754,7 @@ private int find(UTF8String str, int start) { private int rfind(UTF8String str, int start) { assert (str.numBytes > 0); while (start >= 0) { - if (ByteArrayMethods.arrayEqualsBlock(base, start, str.base, 0, str.numBytes)) { + if (ByteArrayMethods.arrayEquals(base, offset + start, str.base, str.offset, str.numBytes)) { return start; } start -= 1; @@ -781,7 +787,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { return EMPTY_UTF8; } byte[] bytes = new byte[idx]; - base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, idx); + copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, idx); return fromBytes(bytes); } else { @@ -801,7 +807,7 @@ public UTF8String subStringIndex(UTF8String delim, int count) { } int size = numBytes - delim.numBytes - idx; byte[] bytes = new byte[size]; - base.writeTo(idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); + copyMemory(base, offset + idx + delim.numBytes, bytes, BYTE_ARRAY_OFFSET, size); return fromBytes(bytes); } } @@ -824,15 +830,15 @@ public UTF8String rpad(int len, UTF8String pad) { UTF8String remain = pad.substring(0, spaces - padChars * count); byte[] data = new byte[this.numBytes + pad.numBytes * count + remain.numBytes]; - base.writeTo(0, data, BYTE_ARRAY_OFFSET, this.numBytes); + copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET, this.numBytes); int offset = this.numBytes; int idx = 0; while (idx < count) { - pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; offset += pad.numBytes; } - remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); return UTF8String.fromBytes(data); } @@ -860,13 +866,13 @@ public UTF8String lpad(int len, UTF8String pad) { int offset = 0; int idx = 0; while (idx < count) { - pad.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); + copyMemory(pad.base, pad.offset, data, BYTE_ARRAY_OFFSET + offset, pad.numBytes); ++ idx; offset += pad.numBytes; } - remain.base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); + copyMemory(remain.base, remain.offset, data, BYTE_ARRAY_OFFSET + offset, remain.numBytes); offset += remain.numBytes; - base.writeTo(0, data, BYTE_ARRAY_OFFSET + offset, numBytes()); + copyMemory(this.base, this.offset, data, BYTE_ARRAY_OFFSET + offset, numBytes()); return UTF8String.fromBytes(data); } @@ -891,8 +897,8 @@ public static UTF8String concat(UTF8String... inputs) { int offset = 0; for (int i = 0; i < inputs.length; i++) { int len = inputs[i].numBytes; - inputs[i].base.writeTo( - 0, + copyMemory( + inputs[i].base, inputs[i].offset, result, BYTE_ARRAY_OFFSET + offset, len); offset += len; @@ -931,8 +937,8 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { for (int i = 0, j = 0; i < inputs.length; i++) { if (inputs[i] != null) { int len = inputs[i].numBytes; - inputs[i].base.writeTo( - 0, + copyMemory( + inputs[i].base, inputs[i].offset, result, BYTE_ARRAY_OFFSET + offset, len); offset += len; @@ -940,8 +946,8 @@ public static UTF8String concatWs(UTF8String separator, UTF8String... inputs) { j++; // Add separator if this is not the last input. if (j < numInputs) { - separator.base.writeTo( - 0, + copyMemory( + separator.base, separator.offset, result, BYTE_ARRAY_OFFSET + offset, separator.numBytes); offset += separator.numBytes; @@ -1215,7 +1221,7 @@ public UTF8String clone() { public UTF8String copy() { byte[] bytes = new byte[numBytes]; - base.writeTo(0, bytes, BYTE_ARRAY_OFFSET, numBytes); + copyMemory(base, offset, bytes, BYTE_ARRAY_OFFSET, numBytes); return fromBytes(bytes); } @@ -1223,10 +1229,11 @@ public UTF8String copy() { public int compareTo(@Nonnull final UTF8String other) { int len = Math.min(numBytes, other.numBytes); int wordMax = (len / 8) * 8; - MemoryBlock rbase = other.base; + long roffset = other.offset; + Object rbase = other.base; for (int i = 0; i < wordMax; i += 8) { - long left = base.getLong(i); - long right = rbase.getLong(i); + long left = getLong(base, offset + i); + long right = getLong(rbase, roffset + i); if (left != right) { if (IS_LITTLE_ENDIAN) { return Long.compareUnsigned(Long.reverseBytes(left), Long.reverseBytes(right)); @@ -1237,7 +1244,7 @@ public int compareTo(@Nonnull final UTF8String other) { } for (int i = wordMax; i < len; i++) { // In UTF-8, the byte should be unsigned, so we should compare them as unsigned int. - int res = (getByte(i) & 0xFF) - (rbase.getByte(i) & 0xFF); + int res = (getByte(i) & 0xFF) - (Platform.getByte(rbase, roffset + i) & 0xFF); if (res != 0) { return res; } @@ -1256,7 +1263,7 @@ public boolean equals(final Object other) { if (numBytes != o.numBytes) { return false; } - return ByteArrayMethods.arrayEqualsBlock(base, 0, o.base, 0, numBytes); + return ByteArrayMethods.arrayEquals(base, offset, o.base, o.offset, numBytes); } else { return false; } @@ -1312,8 +1319,8 @@ public int levenshteinDistance(UTF8String other) { num_bytes_j != numBytesForFirstByte(s.getByte(i_bytes))) { cost = 1; } else { - cost = (ByteArrayMethods.arrayEqualsBlock(t.base, j_bytes, s.base, - i_bytes, num_bytes_j)) ? 0 : 1; + cost = (ByteArrayMethods.arrayEquals(t.base, t.offset + j_bytes, s.base, + s.offset + i_bytes, num_bytes_j)) ? 0 : 1; } d[i + 1] = Math.min(Math.min(d[i] + 1, p[i + 1] + 1), p[i] + cost); } @@ -1328,7 +1335,7 @@ public int levenshteinDistance(UTF8String other) { @Override public int hashCode() { - return Murmur3_x86_32.hashUnsafeBytesBlock(base,42); + return Murmur3_x86_32.hashUnsafeBytes(base, offset, numBytes, 42); } /** @@ -1391,10 +1398,10 @@ public void writeExternal(ObjectOutput out) throws IOException { } public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + offset = BYTE_ARRAY_OFFSET; numBytes = in.readInt(); - byte[] bytes = new byte[numBytes]; - in.readFully(bytes); - base = ByteArrayMemoryBlock.fromArray(bytes); + base = new byte[numBytes]; + in.readFully((byte[]) base); } @Override @@ -1406,10 +1413,10 @@ public void write(Kryo kryo, Output out) { @Override public void read(Kryo kryo, Input in) { - numBytes = in.readInt(); - byte[] bytes = new byte[numBytes]; - in.read(bytes); - base = ByteArrayMemoryBlock.fromArray(bytes); + this.offset = BYTE_ARRAY_OFFSET; + this.numBytes = in.readInt(); + this.base = new byte[numBytes]; + in.read((byte[]) base); } } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java index 583a148b3845..3ad9ac7b4de9 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java @@ -81,7 +81,7 @@ public void freeingOnHeapMemoryBlockResetsBaseObjectAndOffset() { MemoryAllocator.HEAP.free(block); Assert.assertNull(block.getBaseObject()); Assert.assertEquals(0, block.getBaseOffset()); - Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, block.getPageNumber()); + Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, block.pageNumber); } @Test @@ -92,7 +92,7 @@ public void freeingOffHeapMemoryBlockResetsOffset() { MemoryAllocator.UNSAFE.free(block); Assert.assertNull(block.getBaseObject()); Assert.assertEquals(0, block.getBaseOffset()); - Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, block.getPageNumber()); + Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, block.pageNumber); } @Test(expected = AssertionError.class) diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java index 8c2e98c2bfc5..fb8e53b3348f 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/array/LongArraySuite.java @@ -20,13 +20,14 @@ import org.junit.Assert; import org.junit.Test; -import org.apache.spark.unsafe.memory.OnHeapMemoryBlock; +import org.apache.spark.unsafe.memory.MemoryBlock; public class LongArraySuite { @Test public void basicTest() { - LongArray arr = new LongArray(new OnHeapMemoryBlock(16)); + long[] bytes = new long[2]; + LongArray arr = new LongArray(MemoryBlock.fromLongArray(bytes)); arr.set(0, 1L); arr.set(1, 2L); arr.set(1, 3L); diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java index d9898771720a..6348a73bf389 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/hash/Murmur3_x86_32Suite.java @@ -70,24 +70,6 @@ public void testKnownBytesInputs() { Murmur3_x86_32.hashUnsafeBytes2(tes, Platform.BYTE_ARRAY_OFFSET, tes.length, 0)); } - @Test - public void testKnownWordsInputs() { - byte[] bytes = new byte[16]; - long offset = Platform.BYTE_ARRAY_OFFSET; - for (int i = 0; i < 16; i++) { - bytes[i] = 0; - } - Assert.assertEquals(-300363099, Murmur3_x86_32.hashUnsafeWords(bytes, offset, 16, 42)); - for (int i = 0; i < 16; i++) { - bytes[i] = -1; - } - Assert.assertEquals(-1210324667, Murmur3_x86_32.hashUnsafeWords(bytes, offset, 16, 42)); - for (int i = 0; i < 16; i++) { - bytes[i] = (byte)i; - } - Assert.assertEquals(-634919701, Murmur3_x86_32.hashUnsafeWords(bytes, offset, 16, 42)); - } - @Test public void randomizedStressTest() { int size = 65536; diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java deleted file mode 100644 index ef5ff8ee70ec..000000000000 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/memory/MemoryBlockSuite.java +++ /dev/null @@ -1,179 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.unsafe.memory; - -import org.apache.spark.unsafe.Platform; -import org.junit.Assert; -import org.junit.Test; - -import java.nio.ByteOrder; - -import static org.hamcrest.core.StringContains.containsString; - -public class MemoryBlockSuite { - private static final boolean bigEndianPlatform = - ByteOrder.nativeOrder().equals(ByteOrder.BIG_ENDIAN); - - private void check(MemoryBlock memory, Object obj, long offset, int length) { - memory.setPageNumber(1); - memory.fill((byte)-1); - memory.putBoolean(0, true); - memory.putByte(1, (byte)127); - memory.putShort(2, (short)257); - memory.putInt(4, 0x20000002); - memory.putLong(8, 0x1234567089ABCDEFL); - memory.putFloat(16, 1.0F); - memory.putLong(20, 0x1234567089ABCDEFL); - memory.putDouble(28, 2.0); - MemoryBlock.copyMemory(memory, 0L, memory, 36, 4); - int[] a = new int[2]; - a[0] = 0x12345678; - a[1] = 0x13579BDF; - memory.copyFrom(a, Platform.INT_ARRAY_OFFSET, 40, 8); - byte[] b = new byte[8]; - memory.writeTo(40, b, Platform.BYTE_ARRAY_OFFSET, 8); - - Assert.assertEquals(obj, memory.getBaseObject()); - Assert.assertEquals(offset, memory.getBaseOffset()); - Assert.assertEquals(length, memory.size()); - Assert.assertEquals(1, memory.getPageNumber()); - Assert.assertEquals(true, memory.getBoolean(0)); - Assert.assertEquals((byte)127, memory.getByte(1 )); - Assert.assertEquals((short)257, memory.getShort(2)); - Assert.assertEquals(0x20000002, memory.getInt(4)); - Assert.assertEquals(0x1234567089ABCDEFL, memory.getLong(8)); - Assert.assertEquals(1.0F, memory.getFloat(16), 0); - Assert.assertEquals(0x1234567089ABCDEFL, memory.getLong(20)); - Assert.assertEquals(2.0, memory.getDouble(28), 0); - Assert.assertEquals(true, memory.getBoolean(36)); - Assert.assertEquals((byte)127, memory.getByte(37 )); - Assert.assertEquals((short)257, memory.getShort(38)); - Assert.assertEquals(a[0], memory.getInt(40)); - Assert.assertEquals(a[1], memory.getInt(44)); - if (bigEndianPlatform) { - Assert.assertEquals(a[0], - ((int)b[0] & 0xff) << 24 | ((int)b[1] & 0xff) << 16 | - ((int)b[2] & 0xff) << 8 | ((int)b[3] & 0xff)); - Assert.assertEquals(a[1], - ((int)b[4] & 0xff) << 24 | ((int)b[5] & 0xff) << 16 | - ((int)b[6] & 0xff) << 8 | ((int)b[7] & 0xff)); - } else { - Assert.assertEquals(a[0], - ((int)b[3] & 0xff) << 24 | ((int)b[2] & 0xff) << 16 | - ((int)b[1] & 0xff) << 8 | ((int)b[0] & 0xff)); - Assert.assertEquals(a[1], - ((int)b[7] & 0xff) << 24 | ((int)b[6] & 0xff) << 16 | - ((int)b[5] & 0xff) << 8 | ((int)b[4] & 0xff)); - } - for (int i = 48; i < memory.size(); i++) { - Assert.assertEquals((byte) -1, memory.getByte(i)); - } - - assert(memory.subBlock(0, memory.size()) == memory); - - try { - memory.subBlock(-8, 8); - Assert.fail(); - } catch (Exception expected) { - Assert.assertThat(expected.getMessage(), containsString("non-negative")); - } - - try { - memory.subBlock(0, -8); - Assert.fail(); - } catch (Exception expected) { - Assert.assertThat(expected.getMessage(), containsString("non-negative")); - } - - try { - memory.subBlock(0, length + 8); - Assert.fail(); - } catch (Exception expected) { - Assert.assertThat(expected.getMessage(), containsString("should not be larger than")); - } - - try { - memory.subBlock(8, length - 4); - Assert.fail(); - } catch (Exception expected) { - Assert.assertThat(expected.getMessage(), containsString("should not be larger than")); - } - - try { - memory.subBlock(length + 8, 4); - Assert.fail(); - } catch (Exception expected) { - Assert.assertThat(expected.getMessage(), containsString("should not be larger than")); - } - - memory.setPageNumber(MemoryBlock.NO_PAGE_NUMBER); - } - - @Test - public void testByteArrayMemoryBlock() { - byte[] obj = new byte[56]; - long offset = Platform.BYTE_ARRAY_OFFSET; - int length = obj.length; - - MemoryBlock memory = new ByteArrayMemoryBlock(obj, offset, length); - check(memory, obj, offset, length); - - memory = ByteArrayMemoryBlock.fromArray(obj); - check(memory, obj, offset, length); - - obj = new byte[112]; - memory = new ByteArrayMemoryBlock(obj, offset, length); - check(memory, obj, offset, length); - } - - @Test - public void testOnHeapMemoryBlock() { - long[] obj = new long[7]; - long offset = Platform.LONG_ARRAY_OFFSET; - int length = obj.length * 8; - - MemoryBlock memory = new OnHeapMemoryBlock(obj, offset, length); - check(memory, obj, offset, length); - - memory = OnHeapMemoryBlock.fromArray(obj); - check(memory, obj, offset, length); - - obj = new long[14]; - memory = new OnHeapMemoryBlock(obj, offset, length); - check(memory, obj, offset, length); - } - - @Test - public void testOffHeapArrayMemoryBlock() { - MemoryAllocator memoryAllocator = new UnsafeMemoryAllocator(); - MemoryBlock memory = memoryAllocator.allocate(56); - Object obj = memory.getBaseObject(); - long offset = memory.getBaseOffset(); - int length = 56; - - check(memory, obj, offset, length); - memoryAllocator.free(memory); - - long address = Platform.allocateMemory(112); - memory = new OffHeapMemoryBlock(address, length); - obj = memory.getBaseObject(); - offset = memory.getBaseOffset(); - check(memory, obj, offset, length); - Platform.freeMemory(address); - } -} diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index 42dda3048070..dae13f03b02f 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -25,8 +25,7 @@ import java.util.*; import com.google.common.collect.ImmutableMap; -import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; -import org.apache.spark.unsafe.memory.OnHeapMemoryBlock; +import org.apache.spark.unsafe.Platform; import org.junit.Test; import static org.junit.Assert.*; @@ -513,6 +512,21 @@ public void soundex() { assertEquals(fromString("世界千世").soundex(), fromString("世界千世")); } + @Test + public void writeToOutputStreamUnderflow() throws IOException { + // offset underflow is apparently supported? + final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); + final byte[] test = "01234567".getBytes(StandardCharsets.UTF_8); + + for (int i = 1; i <= Platform.BYTE_ARRAY_OFFSET; ++i) { + UTF8String.fromAddress(test, Platform.BYTE_ARRAY_OFFSET - i, test.length + i) + .writeTo(outputStream); + final ByteBuffer buffer = ByteBuffer.wrap(outputStream.toByteArray(), i, test.length); + assertEquals("01234567", StandardCharsets.UTF_8.decode(buffer).toString()); + outputStream.reset(); + } + } + @Test public void writeToOutputStreamSlice() throws IOException { final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); @@ -520,7 +534,7 @@ public void writeToOutputStreamSlice() throws IOException { for (int i = 0; i < test.length; ++i) { for (int j = 0; j < test.length - i; ++j) { - new UTF8String(ByteArrayMemoryBlock.fromArray(test).subBlock(i, j)) + UTF8String.fromAddress(test, Platform.BYTE_ARRAY_OFFSET + i, j) .writeTo(outputStream); assertArrayEquals(Arrays.copyOfRange(test, i, i + j), outputStream.toByteArray()); @@ -551,7 +565,7 @@ public void writeToOutputStreamOverflow() throws IOException { for (final long offset : offsets) { try { - new UTF8String(ByteArrayMemoryBlock.fromArray(test).subBlock(offset, test.length)) + fromAddress(test, BYTE_ARRAY_OFFSET + offset, test.length) .writeTo(outputStream); throw new IllegalStateException(Long.toString(offset)); @@ -578,25 +592,26 @@ public void writeToOutputStream() throws IOException { } @Test - public void writeToOutputStreamLongArray() throws IOException { + public void writeToOutputStreamIntArray() throws IOException { // verify that writes work on objects that are not byte arrays - final ByteBuffer buffer = StandardCharsets.UTF_8.encode("3千大千世界"); + final ByteBuffer buffer = StandardCharsets.UTF_8.encode("大千世界"); buffer.position(0); buffer.order(ByteOrder.nativeOrder()); final int length = buffer.limit(); - assertEquals(16, length); + assertEquals(12, length); - final int longs = length / 8; - final long[] array = new long[longs]; + final int ints = length / 4; + final int[] array = new int[ints]; - for (int i = 0; i < longs; ++i) { - array[i] = buffer.getLong(); + for (int i = 0; i < ints; ++i) { + array[i] = buffer.getInt(); } final ByteArrayOutputStream outputStream = new ByteArrayOutputStream(); - new UTF8String(OnHeapMemoryBlock.fromArray(array)).writeTo(outputStream); - assertEquals("3千大千世界", outputStream.toString("UTF-8")); + fromAddress(array, Platform.INT_ARRAY_OFFSET, length) + .writeTo(outputStream); + assertEquals("大千世界", outputStream.toString("UTF-8")); } @Test diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index 8651a639c07f..d07faf1da124 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -311,7 +311,7 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { // this could trigger spilling to free some pages. return allocatePage(size, consumer); } - page.setPageNumber(pageNumber); + page.pageNumber = pageNumber; pageTable[pageNumber] = page; if (logger.isTraceEnabled()) { logger.trace("Allocate page number {} ({} bytes)", pageNumber, acquired); @@ -323,25 +323,25 @@ public MemoryBlock allocatePage(long size, MemoryConsumer consumer) { * Free a block of memory allocated via {@link TaskMemoryManager#allocatePage}. */ public void freePage(MemoryBlock page, MemoryConsumer consumer) { - assert (page.getPageNumber() != MemoryBlock.NO_PAGE_NUMBER) : + assert (page.pageNumber != MemoryBlock.NO_PAGE_NUMBER) : "Called freePage() on memory that wasn't allocated with allocatePage()"; - assert (page.getPageNumber() != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : + assert (page.pageNumber != MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER) : "Called freePage() on a memory block that has already been freed"; - assert (page.getPageNumber() != MemoryBlock.FREED_IN_TMM_PAGE_NUMBER) : + assert (page.pageNumber != MemoryBlock.FREED_IN_TMM_PAGE_NUMBER) : "Called freePage() on a memory block that has already been freed"; - assert(allocatedPages.get(page.getPageNumber())); - pageTable[page.getPageNumber()] = null; + assert(allocatedPages.get(page.pageNumber)); + pageTable[page.pageNumber] = null; synchronized (this) { - allocatedPages.clear(page.getPageNumber()); + allocatedPages.clear(page.pageNumber); } if (logger.isTraceEnabled()) { - logger.trace("Freed page number {} ({} bytes)", page.getPageNumber(), page.size()); + logger.trace("Freed page number {} ({} bytes)", page.pageNumber, page.size()); } long pageSize = page.size(); // Clear the page number before passing the block to the MemoryAllocator's free(). // Doing this allows the MemoryAllocator to detect when a TaskMemoryManager-managed // page has been inappropriately directly freed without calling TMM.freePage(). - page.setPageNumber(MemoryBlock.FREED_IN_TMM_PAGE_NUMBER); + page.pageNumber = MemoryBlock.FREED_IN_TMM_PAGE_NUMBER; memoryManager.tungstenMemoryAllocator().free(page); releaseExecutionMemory(pageSize, consumer); } @@ -363,7 +363,7 @@ public long encodePageNumberAndOffset(MemoryBlock page, long offsetInPage) { // relative to the page's base offset; this relative offset will fit in 51 bits. offsetInPage -= page.getBaseOffset(); } - return encodePageNumberAndOffset(page.getPageNumber(), offsetInPage); + return encodePageNumberAndOffset(page.pageNumber, offsetInPage); } @VisibleForTesting @@ -434,7 +434,7 @@ public long cleanUpAllAllocatedMemory() { for (MemoryBlock page : pageTable) { if (page != null) { logger.debug("unreleased page: " + page + " in task " + taskAttemptId); - page.setPageNumber(MemoryBlock.FREED_IN_TMM_PAGE_NUMBER); + page.pageNumber = MemoryBlock.FREED_IN_TMM_PAGE_NUMBER; memoryManager.tungstenMemoryAllocator().free(page); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java index 4b48599ad311..0d069125dc60 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorter.java @@ -20,6 +20,7 @@ import java.util.Comparator; import org.apache.spark.memory.MemoryConsumer; +import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.Sorter; @@ -112,7 +113,13 @@ public void reset() { public void expandPointerArray(LongArray newArray) { assert(newArray.size() > array.size()); - MemoryBlock.copyMemory(array.memoryBlock(), newArray.memoryBlock(), pos * 8L); + Platform.copyMemory( + array.getBaseObject(), + array.getBaseOffset(), + newArray.getBaseObject(), + newArray.getBaseOffset(), + pos * 8L + ); consumer.freeArray(array); array = newArray; usableCapacity = getUsableCapacity(); @@ -181,7 +188,10 @@ public ShuffleSorterIterator getSortedIterator() { PackedRecordPointer.PARTITION_ID_START_BYTE_INDEX, PackedRecordPointer.PARTITION_ID_END_BYTE_INDEX, false, false); } else { - MemoryBlock unused = array.memoryBlock().subBlock(pos * 8L, (array.size() - pos) * 8L); + MemoryBlock unused = new MemoryBlock( + array.getBaseObject(), + array.getBaseOffset() + pos * 8L, + (array.size() - pos) * 8L); LongArray buffer = new LongArray(unused); Sorter sorter = new Sorter<>(new ShuffleSortDataFormat(buffer)); diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java index 254449e95443..717bdd79d47e 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleSortDataFormat.java @@ -17,8 +17,8 @@ package org.apache.spark.shuffle.sort; +import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.LongArray; -import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.util.collection.SortDataFormat; final class ShuffleSortDataFormat extends SortDataFormat { @@ -60,8 +60,13 @@ public void copyElement(LongArray src, int srcPos, LongArray dst, int dstPos) { @Override public void copyRange(LongArray src, int srcPos, LongArray dst, int dstPos, int length) { - MemoryBlock.copyMemory(src.memoryBlock(), srcPos * 8L, - dst.memoryBlock(),dstPos * 8L,length * 8L); + Platform.copyMemory( + src.getBaseObject(), + src.getBaseOffset() + srcPos * 8L, + dst.getBaseObject(), + dst.getBaseOffset() + dstPos * 8L, + length * 8L + ); } @Override diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 399251b80e64..5056652a2420 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -544,7 +544,7 @@ public long spill() throws IOException { // is accessing the current record. We free this page in that caller's next loadNext() // call. for (MemoryBlock page : allocatedPages) { - if (!loaded || page.getPageNumber() != + if (!loaded || page.pageNumber != ((UnsafeInMemorySorter.SortedIterator)upstream).getCurrentPageNumber()) { released += page.size(); freePage(page); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 717823ebbd32..75690ae26483 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -26,6 +26,7 @@ import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; +import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.UnsafeAlignedOffset; import org.apache.spark.unsafe.array.LongArray; import org.apache.spark.unsafe.memory.MemoryBlock; @@ -215,7 +216,12 @@ public void expandPointerArray(LongArray newArray) { if (newArray.size() < array.size()) { throw new SparkOutOfMemoryError("Not enough memory to grow pointer array"); } - MemoryBlock.copyMemory(array.memoryBlock(), newArray.memoryBlock(), pos * 8L); + Platform.copyMemory( + array.getBaseObject(), + array.getBaseOffset(), + newArray.getBaseObject(), + newArray.getBaseOffset(), + pos * 8L); consumer.freeArray(array); array = newArray; usableCapacity = getUsableCapacity(); @@ -342,7 +348,10 @@ public UnsafeSorterIterator getSortedIterator() { array, nullBoundaryPos, (pos - nullBoundaryPos) / 2L, 0, 7, radixSortSupport.sortDescending(), radixSortSupport.sortSigned()); } else { - MemoryBlock unused = array.memoryBlock().subBlock(pos * 8L, (array.size() - pos) * 8L); + MemoryBlock unused = new MemoryBlock( + array.getBaseObject(), + array.getBaseOffset() + pos * 8L, + (array.size() - pos) * 8L); LongArray buffer = new LongArray(unused); Sorter sorter = new Sorter<>(new UnsafeSortDataFormat(buffer)); diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index d7d2d0b012bd..a0664b30d6cc 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -76,7 +76,7 @@ public void freeingPageSetsPageNumberToSpecialConstant() { final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); final MemoryBlock dataPage = manager.allocatePage(256, c); c.freePage(dataPage); - Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, dataPage.getPageNumber()); + Assert.assertEquals(MemoryBlock.FREED_IN_ALLOCATOR_PAGE_NUMBER, dataPage.pageNumber); } @Test(expected = AssertionError.class) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 3e56db5ea116..47173b89e91e 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark._ import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.unsafe.array.LongArray -import org.apache.spark.unsafe.memory.OnHeapMemoryBlock +import org.apache.spark.unsafe.memory.MemoryBlock import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordPointerAndKeyPrefix, UnsafeSortDataFormat} class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { @@ -105,8 +105,9 @@ class ExternalSorterSuite extends SparkFunSuite with LocalSparkContext { // the form [150000000, 150000001, 150000002, ...., 300000000, 0, 1, 2, ..., 149999999] // that can trigger copyRange() in TimSort.mergeLo() or TimSort.mergeHi() val ref = Array.tabulate[Long](size) { i => if (i < size / 2) size / 2 + i else i } - val buf = new LongArray(OnHeapMemoryBlock.fromArray(ref)) - val tmpBuf = new LongArray(new OnHeapMemoryBlock((size/2) * 8L)) + val buf = new LongArray(MemoryBlock.fromLongArray(ref)) + val tmp = new Array[Long](size/2) + val tmpBuf = new LongArray(MemoryBlock.fromLongArray(tmp)) new Sorter(new UnsafeSortDataFormat(tmpBuf)).sort( buf, 0, size, new Comparator[RecordPointerAndKeyPrefix] { diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala index ddf3740e76a7..d5956ea32096 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/RadixSortSuite.scala @@ -27,7 +27,7 @@ import com.google.common.primitives.Ints import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging import org.apache.spark.unsafe.array.LongArray -import org.apache.spark.unsafe.memory.OnHeapMemoryBlock +import org.apache.spark.unsafe.memory.MemoryBlock import org.apache.spark.util.collection.Sorter import org.apache.spark.util.random.XORShiftRandom @@ -78,14 +78,14 @@ class RadixSortSuite extends SparkFunSuite with Logging { private def generateTestData(size: Long, rand: => Long): (Array[JLong], LongArray) = { val ref = Array.tabulate[Long](Ints.checkedCast(size)) { i => rand } val extended = ref ++ Array.fill[Long](Ints.checkedCast(size))(0) - (ref.map(i => new JLong(i)), new LongArray(OnHeapMemoryBlock.fromArray(extended))) + (ref.map(i => new JLong(i)), new LongArray(MemoryBlock.fromLongArray(extended))) } private def generateKeyPrefixTestData(size: Long, rand: => Long): (LongArray, LongArray) = { val ref = Array.tabulate[Long](Ints.checkedCast(size * 2)) { i => rand } val extended = ref ++ Array.fill[Long](Ints.checkedCast(size * 2))(0) - (new LongArray(OnHeapMemoryBlock.fromArray(ref)), - new LongArray(OnHeapMemoryBlock.fromArray(extended))) + (new LongArray(MemoryBlock.fromLongArray(ref)), + new LongArray(MemoryBlock.fromLongArray(extended))) } private def collectToArray(array: LongArray, offset: Int, length: Long): Array[Long] = { @@ -110,7 +110,7 @@ class RadixSortSuite extends SparkFunSuite with Logging { } private def referenceKeyPrefixSort(buf: LongArray, lo: Long, hi: Long, refCmp: PrefixComparator) { - val sortBuffer = new LongArray(new OnHeapMemoryBlock(buf.size() * 8L)) + val sortBuffer = new LongArray(MemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, Ints.checkedCast(lo), Ints.checkedCast(hi), new Comparator[RecordPointerAndKeyPrefix] { override def compare( diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala index dc38ee326e5e..dc18e1d34880 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/FeatureHasher.scala @@ -29,7 +29,7 @@ import org.apache.spark.mllib.feature.{HashingTF => OldHashingTF} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.hash.Murmur3_x86_32.{hashInt, hashLong, hashUnsafeBytes2Block} +import org.apache.spark.unsafe.hash.Murmur3_x86_32.{hashInt, hashLong, hashUnsafeBytes2} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils import org.apache.spark.util.collection.OpenHashMap @@ -244,7 +244,8 @@ object FeatureHasher extends DefaultParamsReadable[FeatureHasher] { case f: Float => hashInt(java.lang.Float.floatToIntBits(f), seed) case d: Double => hashLong(java.lang.Double.doubleToLongBits(d), seed) case s: String => - hashUnsafeBytes2Block(UTF8String.fromString(s).getMemoryBlock, seed) + val utf8 = UTF8String.fromString(s) + hashUnsafeBytes2(utf8.getBaseObject, utf8.getBaseOffset, utf8.numBytes(), seed) case _ => throw new SparkException("FeatureHasher with murmur3 algorithm does not " + s"support type ${term.getClass.getCanonicalName} of input data.") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala index 7b73b286fb91..8935c8496cdb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala @@ -160,7 +160,7 @@ object HashingTF { case d: Double => hashLong(java.lang.Double.doubleToLongBits(d), seed) case s: String => val utf8 = UTF8String.fromString(s) - hashUnsafeBytesBlock(utf8.getMemoryBlock(), seed) + hashUnsafeBytes(utf8.getBaseObject, utf8.getBaseOffset, utf8.numBytes(), seed) case _ => throw new SparkException("HashingTF with murmur3 algorithm does not " + s"support type ${term.getClass.getCanonicalName} of input data.") } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index 9e7b15d339ee..9002abdcfd47 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -27,7 +27,6 @@ import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; -import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -241,8 +240,7 @@ public UTF8String getUTF8String(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - MemoryBlock mb = MemoryBlock.allocateFromObject(baseObject, baseOffset + offset, size); - return new UTF8String(mb); + return UTF8String.fromAddress(baseObject, baseOffset + offset, size); } @Override 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 469b0e60cc9a..a76e6ef8c91c 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 @@ -37,7 +37,6 @@ import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.bitset.BitSetMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; -import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -417,8 +416,7 @@ public UTF8String getUTF8String(int ordinal) { final long offsetAndSize = getLong(ordinal); final int offset = (int) (offsetAndSize >> 32); final int size = (int) offsetAndSize; - MemoryBlock mb = MemoryBlock.allocateFromObject(baseObject, baseOffset + offset, size); - return new UTF8String(mb); + return UTF8String.fromAddress(baseObject, baseOffset + offset, size); } @Override diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java index 8e9c0a2e9dc8..eb5051b28407 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/XXH64.java @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.catalyst.expressions; -import org.apache.spark.unsafe.memory.MemoryBlock; +import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.types.UTF8String; // scalastyle: off @@ -72,13 +72,13 @@ public static long hashLong(long input, long seed) { return fmix(hash); } - public long hashUnsafeWordsBlock(MemoryBlock mb) { - return hashUnsafeWordsBlock(mb, seed); + public long hashUnsafeWords(Object base, long offset, int length) { + return hashUnsafeWords(base, offset, length, seed); } - public static long hashUnsafeWordsBlock(MemoryBlock mb, long seed) { - assert (mb.size() % 8 == 0) : "lengthInBytes must be a multiple of 8 (word-aligned)"; - long hash = hashBytesByWordsBlock(mb, seed); + public static long hashUnsafeWords(Object base, long offset, int length, long seed) { + assert (length % 8 == 0) : "lengthInBytes must be a multiple of 8 (word-aligned)"; + long hash = hashBytesByWords(base, offset, length, seed); return fmix(hash); } @@ -86,22 +86,20 @@ public long hashUnsafeBytes(Object base, long offset, int length) { return hashUnsafeBytes(base, offset, length, seed); } - public static long hashUnsafeBytesBlock(MemoryBlock mb, long seed) { - long offset = 0; - long length = mb.size(); + public static long hashUnsafeBytes(Object base, long offset, int length, long seed) { assert (length >= 0) : "lengthInBytes cannot be negative"; - long hash = hashBytesByWordsBlock(mb, seed); + long hash = hashBytesByWords(base, offset, length, seed); long end = offset + length; offset += length & -8; if (offset + 4L <= end) { - hash ^= (mb.getInt(offset) & 0xFFFFFFFFL) * PRIME64_1; + hash ^= (Platform.getInt(base, offset) & 0xFFFFFFFFL) * PRIME64_1; hash = Long.rotateLeft(hash, 23) * PRIME64_2 + PRIME64_3; offset += 4L; } while (offset < end) { - hash ^= (mb.getByte(offset) & 0xFFL) * PRIME64_5; + hash ^= (Platform.getByte(base, offset) & 0xFFL) * PRIME64_5; hash = Long.rotateLeft(hash, 11) * PRIME64_1; offset++; } @@ -109,11 +107,7 @@ public static long hashUnsafeBytesBlock(MemoryBlock mb, long seed) { } public static long hashUTF8String(UTF8String str, long seed) { - return hashUnsafeBytesBlock(str.getMemoryBlock(), seed); - } - - public static long hashUnsafeBytes(Object base, long offset, int length, long seed) { - return hashUnsafeBytesBlock(MemoryBlock.allocateFromObject(base, offset, length), seed); + return hashUnsafeBytes(str.getBaseObject(), str.getBaseOffset(), str.numBytes(), seed); } private static long fmix(long hash) { @@ -125,31 +119,30 @@ private static long fmix(long hash) { return hash; } - private static long hashBytesByWordsBlock(MemoryBlock mb, long seed) { - long offset = 0; - long length = mb.size(); + private static long hashBytesByWords(Object base, long offset, int length, long seed) { + long end = offset + length; long hash; if (length >= 32) { - long limit = length - 32; + long limit = end - 32; long v1 = seed + PRIME64_1 + PRIME64_2; long v2 = seed + PRIME64_2; long v3 = seed; long v4 = seed - PRIME64_1; do { - v1 += mb.getLong(offset) * PRIME64_2; + v1 += Platform.getLong(base, offset) * PRIME64_2; v1 = Long.rotateLeft(v1, 31); v1 *= PRIME64_1; - v2 += mb.getLong(offset + 8) * PRIME64_2; + v2 += Platform.getLong(base, offset + 8) * PRIME64_2; v2 = Long.rotateLeft(v2, 31); v2 *= PRIME64_1; - v3 += mb.getLong(offset + 16) * PRIME64_2; + v3 += Platform.getLong(base, offset + 16) * PRIME64_2; v3 = Long.rotateLeft(v3, 31); v3 *= PRIME64_1; - v4 += mb.getLong(offset + 24) * PRIME64_2; + v4 += Platform.getLong(base, offset + 24) * PRIME64_2; v4 = Long.rotateLeft(v4, 31); v4 *= PRIME64_1; @@ -190,9 +183,9 @@ private static long hashBytesByWordsBlock(MemoryBlock mb, long seed) { hash += length; - long limit = length - 8; + long limit = end - 8; while (offset <= limit) { - long k1 = mb.getLong(offset); + long k1 = Platform.getLong(base, offset); hash ^= Long.rotateLeft(k1 * PRIME64_2, 31) * PRIME64_1; hash = Long.rotateLeft(hash, 27) * PRIME64_1 + PRIME64_4; offset += 8L; diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java index f8000d78cd1b..f0f66bae245f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java @@ -19,8 +19,6 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; -import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; -import org.apache.spark.unsafe.memory.MemoryBlock; import org.apache.spark.unsafe.types.UTF8String; /** @@ -31,34 +29,43 @@ public class UTF8StringBuilder { private static final int ARRAY_MAX = ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH; - private ByteArrayMemoryBlock buffer; - private int length = 0; + private byte[] buffer; + private int cursor = Platform.BYTE_ARRAY_OFFSET; public UTF8StringBuilder() { // Since initial buffer size is 16 in `StringBuilder`, we set the same size here - this.buffer = new ByteArrayMemoryBlock(16); + this.buffer = new byte[16]; } // Grows the buffer by at least `neededSize` private void grow(int neededSize) { - if (neededSize > ARRAY_MAX - length) { + if (neededSize > ARRAY_MAX - totalSize()) { throw new UnsupportedOperationException( "Cannot grow internal buffer by size " + neededSize + " because the size after growing " + "exceeds size limitation " + ARRAY_MAX); } - final int requestedSize = length + neededSize; - if (buffer.size() < requestedSize) { - int newLength = requestedSize < ARRAY_MAX / 2 ? requestedSize * 2 : ARRAY_MAX; - final ByteArrayMemoryBlock tmp = new ByteArrayMemoryBlock(newLength); - MemoryBlock.copyMemory(buffer, tmp, length); + final int length = totalSize() + neededSize; + if (buffer.length < length) { + int newLength = length < ARRAY_MAX / 2 ? length * 2 : ARRAY_MAX; + final byte[] tmp = new byte[newLength]; + Platform.copyMemory( + buffer, + Platform.BYTE_ARRAY_OFFSET, + tmp, + Platform.BYTE_ARRAY_OFFSET, + totalSize()); buffer = tmp; } } + private int totalSize() { + return cursor - Platform.BYTE_ARRAY_OFFSET; + } + public void append(UTF8String value) { grow(value.numBytes()); - value.writeToMemory(buffer.getByteArray(), length + Platform.BYTE_ARRAY_OFFSET); - length += value.numBytes(); + value.writeToMemory(buffer, cursor); + cursor += value.numBytes(); } public void append(String value) { @@ -66,6 +73,6 @@ public void append(String value) { } public UTF8String build() { - return UTF8String.fromBytes(buffer.getByteArray(), 0, length); + return UTF8String.fromBytes(buffer, 0, totalSize()); } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index a754e87a1796..742a4f87a9c0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -33,7 +33,6 @@ import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.hash.Murmur3_x86_32 -import org.apache.spark.unsafe.memory.MemoryBlock import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -362,7 +361,10 @@ abstract class HashExpression[E] extends Expression { } protected def genHashString(input: String, result: String): String = { - s"$result = $hasherClassName.hashUTF8String($input, $result);" + val baseObject = s"$input.getBaseObject()" + val baseOffset = s"$input.getBaseOffset()" + val numBytes = s"$input.numBytes()" + s"$result = $hasherClassName.hashUnsafeBytes($baseObject, $baseOffset, $numBytes, $result);" } protected def genHashForMap( @@ -469,8 +471,6 @@ abstract class InterpretedHashFunction { protected def hashUnsafeBytes(base: AnyRef, offset: Long, length: Int, seed: Long): Long - protected def hashUnsafeBytesBlock(base: MemoryBlock, seed: Long): Long - /** * Computes hash of a given `value` of type `dataType`. The caller needs to check the validity * of input `value`. @@ -496,7 +496,8 @@ abstract class InterpretedHashFunction { case c: CalendarInterval => hashInt(c.months, hashLong(c.microseconds, seed)) case a: Array[Byte] => hashUnsafeBytes(a, Platform.BYTE_ARRAY_OFFSET, a.length, seed) - case s: UTF8String => hashUnsafeBytesBlock(s.getMemoryBlock(), seed) + case s: UTF8String => + hashUnsafeBytes(s.getBaseObject, s.getBaseOffset, s.numBytes(), seed) case array: ArrayData => val elementType = dataType match { @@ -583,15 +584,9 @@ object Murmur3HashFunction extends InterpretedHashFunction { Murmur3_x86_32.hashLong(l, seed.toInt) } - override protected def hashUnsafeBytes( - base: AnyRef, offset: Long, len: Int, seed: Long): Long = { + override protected def hashUnsafeBytes(base: AnyRef, offset: Long, len: Int, seed: Long): Long = { Murmur3_x86_32.hashUnsafeBytes(base, offset, len, seed.toInt) } - - override protected def hashUnsafeBytesBlock( - base: MemoryBlock, seed: Long): Long = { - Murmur3_x86_32.hashUnsafeBytesBlock(base, seed.toInt) - } } /** @@ -616,14 +611,9 @@ object XxHash64Function extends InterpretedHashFunction { override protected def hashLong(l: Long, seed: Long): Long = XXH64.hashLong(l, seed) - override protected def hashUnsafeBytes( - base: AnyRef, offset: Long, len: Int, seed: Long): Long = { + override protected def hashUnsafeBytes(base: AnyRef, offset: Long, len: Int, seed: Long): Long = { XXH64.hashUnsafeBytes(base, offset, len, seed) } - - override protected def hashUnsafeBytesBlock(base: MemoryBlock, seed: Long): Long = { - XXH64.hashUnsafeBytesBlock(base, seed) - } } /** @@ -730,7 +720,10 @@ case class HiveHash(children: Seq[Expression]) extends HashExpression[Int] { """ override protected def genHashString(input: String, result: String): String = { - s"$result = $hasherClassName.hashUTF8String($input);" + val baseObject = s"$input.getBaseObject()" + val baseOffset = s"$input.getBaseOffset()" + val numBytes = s"$input.numBytes()" + s"$result = $hasherClassName.hashUnsafeBytes($baseObject, $baseOffset, $numBytes);" } override protected def genHashForArray( @@ -824,14 +817,10 @@ object HiveHashFunction extends InterpretedHashFunction { HiveHasher.hashLong(l) } - override protected def hashUnsafeBytes( - base: AnyRef, offset: Long, len: Int, seed: Long): Long = { + override protected def hashUnsafeBytes(base: AnyRef, offset: Long, len: Int, seed: Long): Long = { HiveHasher.hashUnsafeBytes(base, offset, len) } - override protected def hashUnsafeBytesBlock( - base: MemoryBlock, seed: Long): Long = HiveHasher.hashUnsafeBytesBlock(base) - private val HIVE_DECIMAL_MAX_PRECISION = 38 private val HIVE_DECIMAL_MAX_SCALE = 38 diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java index 76930f936851..b67c6f3e6e85 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/HiveHasherSuite.java @@ -17,8 +17,7 @@ package org.apache.spark.sql.catalyst.expressions; -import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; -import org.apache.spark.unsafe.memory.MemoryBlock; +import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.types.UTF8String; import org.junit.Assert; import org.junit.Test; @@ -54,7 +53,7 @@ public void testKnownStringAndIntInputs() { for (int i = 0; i < inputs.length; i++) { UTF8String s = UTF8String.fromString("val_" + inputs[i]); - int hash = HiveHasher.hashUnsafeBytesBlock(s.getMemoryBlock()); + int hash = HiveHasher.hashUnsafeBytes(s.getBaseObject(), s.getBaseOffset(), s.numBytes()); Assert.assertEquals(expected[i], ((31 * inputs[i]) + hash)); } } @@ -90,13 +89,13 @@ public void randomizedStressTestBytes() { int byteArrSize = rand.nextInt(100) * 8; byte[] bytes = new byte[byteArrSize]; rand.nextBytes(bytes); - MemoryBlock mb = ByteArrayMemoryBlock.fromArray(bytes); Assert.assertEquals( - HiveHasher.hashUnsafeBytesBlock(mb), - HiveHasher.hashUnsafeBytesBlock(mb)); + HiveHasher.hashUnsafeBytes(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize), + HiveHasher.hashUnsafeBytes(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); - hashcodes.add(HiveHasher.hashUnsafeBytesBlock(mb)); + hashcodes.add(HiveHasher.hashUnsafeBytes( + bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); } // A very loose bound. @@ -113,13 +112,13 @@ public void randomizedStressTestPaddedStrings() { byte[] strBytes = String.valueOf(i).getBytes(StandardCharsets.UTF_8); byte[] paddedBytes = new byte[byteArrSize]; System.arraycopy(strBytes, 0, paddedBytes, 0, strBytes.length); - MemoryBlock mb = ByteArrayMemoryBlock.fromArray(paddedBytes); Assert.assertEquals( - HiveHasher.hashUnsafeBytesBlock(mb), - HiveHasher.hashUnsafeBytesBlock(mb)); + HiveHasher.hashUnsafeBytes(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize), + HiveHasher.hashUnsafeBytes(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); - hashcodes.add(HiveHasher.hashUnsafeBytesBlock(mb)); + hashcodes.add(HiveHasher.hashUnsafeBytes( + paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); } // A very loose bound. diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/XXH64Suite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/XXH64Suite.java index cd8bce623c5d..1baee91b3439 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/XXH64Suite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/XXH64Suite.java @@ -24,8 +24,6 @@ import java.util.Set; import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.memory.ByteArrayMemoryBlock; -import org.apache.spark.unsafe.memory.MemoryBlock; import org.junit.Assert; import org.junit.Test; @@ -144,13 +142,13 @@ public void randomizedStressTestBytes() { int byteArrSize = rand.nextInt(100) * 8; byte[] bytes = new byte[byteArrSize]; rand.nextBytes(bytes); - MemoryBlock mb = ByteArrayMemoryBlock.fromArray(bytes); Assert.assertEquals( - hasher.hashUnsafeWordsBlock(mb), - hasher.hashUnsafeWordsBlock(mb)); + hasher.hashUnsafeWords(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize), + hasher.hashUnsafeWords(bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); - hashcodes.add(hasher.hashUnsafeWordsBlock(mb)); + hashcodes.add(hasher.hashUnsafeWords( + bytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); } // A very loose bound. @@ -167,13 +165,13 @@ public void randomizedStressTestPaddedStrings() { byte[] strBytes = String.valueOf(i).getBytes(StandardCharsets.UTF_8); byte[] paddedBytes = new byte[byteArrSize]; System.arraycopy(strBytes, 0, paddedBytes, 0, strBytes.length); - MemoryBlock mb = ByteArrayMemoryBlock.fromArray(paddedBytes); Assert.assertEquals( - hasher.hashUnsafeWordsBlock(mb), - hasher.hashUnsafeWordsBlock(mb)); + hasher.hashUnsafeWords(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize), + hasher.hashUnsafeWords(paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); - hashcodes.add(hasher.hashUnsafeWordsBlock(mb)); + hashcodes.add(hasher.hashUnsafeWords( + paddedBytes, Platform.BYTE_ARRAY_OFFSET, byteArrSize)); } // A very loose bound. diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index 6fdadde62855..5e0cf7d370dd 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -23,7 +23,6 @@ import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; -import org.apache.spark.unsafe.memory.OffHeapMemoryBlock; import org.apache.spark.unsafe.types.UTF8String; /** @@ -207,7 +206,7 @@ public byte[] getBytes(int rowId, int count) { @Override protected UTF8String getBytesAsUTF8String(int rowId, int count) { - return new UTF8String(new OffHeapMemoryBlock(data + rowId, count)); + return UTF8String.fromAddress(null, data + rowId, count); } // diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index 1c9beda40435..5f58b031f6ae 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -25,7 +25,6 @@ import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.execution.arrow.ArrowUtils; import org.apache.spark.sql.types.*; -import org.apache.spark.unsafe.memory.OffHeapMemoryBlock; import org.apache.spark.unsafe.types.UTF8String; /** @@ -378,10 +377,9 @@ final UTF8String getUTF8String(int rowId) { if (stringResult.isSet == 0) { return null; } else { - return new UTF8String(new OffHeapMemoryBlock( + return UTF8String.fromAddress(null, stringResult.buffer.memoryAddress() + stringResult.start, - stringResult.end - stringResult.start - )); + stringResult.end - stringResult.start); } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala index 470b93efd197..50ae26a3ff9d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.benchmark import java.util.{Arrays, Comparator} import org.apache.spark.unsafe.array.LongArray -import org.apache.spark.unsafe.memory.OnHeapMemoryBlock +import org.apache.spark.unsafe.memory.MemoryBlock import org.apache.spark.util.Benchmark import org.apache.spark.util.collection.Sorter import org.apache.spark.util.collection.unsafe.sort._ @@ -36,7 +36,7 @@ import org.apache.spark.util.random.XORShiftRandom class SortBenchmark extends BenchmarkBase { private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { - val sortBuffer = new LongArray(new OnHeapMemoryBlock(buf.size() * 8L)) + val sortBuffer = new LongArray(MemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( buf, lo, hi, new Comparator[RecordPointerAndKeyPrefix] { override def compare( @@ -50,8 +50,8 @@ class SortBenchmark extends BenchmarkBase { private def generateKeyPrefixTestData(size: Int, rand: => Long): (LongArray, LongArray) = { val ref = Array.tabulate[Long](size * 2) { i => rand } val extended = ref ++ Array.fill[Long](size * 2)(0) - (new LongArray(OnHeapMemoryBlock.fromArray(ref)), - new LongArray(OnHeapMemoryBlock.fromArray(extended))) + (new LongArray(MemoryBlock.fromLongArray(ref)), + new LongArray(MemoryBlock.fromLongArray(extended))) } ignore("sort") { @@ -60,7 +60,7 @@ class SortBenchmark extends BenchmarkBase { val benchmark = new Benchmark("radix sort " + size, size) benchmark.addTimerCase("reference TimSort key prefix array") { timer => val array = Array.tabulate[Long](size * 2) { i => rand.nextLong } - val buf = new LongArray(OnHeapMemoryBlock.fromArray(array)) + val buf = new LongArray(MemoryBlock.fromLongArray(array)) timer.startTiming() referenceKeyPrefixSort(buf, 0, size, PrefixComparators.BINARY) timer.stopTiming() @@ -78,7 +78,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong & 0xff i += 1 } - val buf = new LongArray(OnHeapMemoryBlock.fromArray(array)) + val buf = new LongArray(MemoryBlock.fromLongArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() @@ -90,7 +90,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong & 0xffff i += 1 } - val buf = new LongArray(OnHeapMemoryBlock.fromArray(array)) + val buf = new LongArray(MemoryBlock.fromLongArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() @@ -102,7 +102,7 @@ class SortBenchmark extends BenchmarkBase { array(i) = rand.nextLong i += 1 } - val buf = new LongArray(OnHeapMemoryBlock.fromArray(array)) + val buf = new LongArray(MemoryBlock.fromLongArray(array)) timer.startTiming() RadixSort.sort(buf, size, 0, 7, false, false) timer.stopTiming() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala index 25ee95daa034..ffda33cf906c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -22,13 +22,13 @@ import java.io.File import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, TestMemoryManager} import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.unsafe.memory.OnHeapMemoryBlock +import org.apache.spark.unsafe.memory.MemoryBlock import org.apache.spark.util.Utils class RowQueueSuite extends SparkFunSuite { test("in-memory queue") { - val page = new OnHeapMemoryBlock((1<<10) * 8L) + val page = MemoryBlock.fromLongArray(new Array[Long](1<<10)) val queue = new InMemoryRowQueue(page, 1) { override def close() {} } From 6b7ea78aec73b8f24c2e1161254edd5ebb6c82bf Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Sun, 9 Sep 2018 09:49:13 -0500 Subject: [PATCH 016/879] [MINOR][ML] Remove `BisectingKMeansModel.setDistanceMeasure` method ## What changes were proposed in this pull request? Remove `BisectingKMeansModel.setDistanceMeasure` method. In `BisectingKMeansModel` set this param is meaningless. ## How was this patch tested? N/A Closes #22360 from WeichenXu123/bkmeans_update. Authored-by: WeichenXu Signed-off-by: Sean Owen (cherry picked from commit 88a930dfab56c15df02c7bb944444745c2921fa5) Signed-off-by: Sean Owen --- .../org/apache/spark/ml/clustering/BisectingKMeans.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 8904193cae94..5cb16cc76588 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -104,10 +104,6 @@ class BisectingKMeansModel private[ml] ( @Since("2.1.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) - /** @group expertSetParam */ - @Since("2.4.0") - def setDistanceMeasure(value: String): this.type = set(distanceMeasure, value) - @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) From c1c1bda3cecd82a926526e5e5ee24d9909cb7e49 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 9 Sep 2018 09:07:31 -0700 Subject: [PATCH 017/879] [SPARK-25368][SQL] Incorrect predicate pushdown returns wrong result ## What changes were proposed in this pull request? How to reproduce: ```scala val df1 = spark.createDataFrame(Seq( (1, 1) )).toDF("a", "b").withColumn("c", lit(null).cast("int")) val df2 = df1.union(df1).withColumn("d", spark_partition_id).filter($"c".isNotNull) df2.show +---+---+----+---+ | a| b| c| d| +---+---+----+---+ | 1| 1|null| 0| | 1| 1|null| 1| +---+---+----+---+ ``` `filter($"c".isNotNull)` was transformed to `(null <=> c#10)` before https://github.com/apache/spark/pull/19201, but it is transformed to `(c#10 = null)` since https://github.com/apache/spark/pull/20155. This pr revert it to `(null <=> c#10)` to fix this issue. ## How was this patch tested? unit tests Closes #22368 from wangyum/SPARK-25368. Authored-by: Yuming Wang Signed-off-by: gatorsmile (cherry picked from commit 77c996403d5c761f0dfea64c5b1cb7480ba1d3ac) Signed-off-by: gatorsmile --- .../catalyst/plans/logical/LogicalPlan.scala | 2 +- .../InferFiltersFromConstraintsSuite.scala | 2 +- .../org/apache/spark/sql/DataFrameSuite.scala | 17 +++++++++++++++++ 3 files changed, 19 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 0e4456ac0e6a..5f136629eb15 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -159,7 +159,7 @@ abstract class UnaryNode extends LogicalPlan { var allConstraints = child.constraints.asInstanceOf[Set[Expression]] projectList.foreach { case a @ Alias(l: Literal, _) => - allConstraints += EqualTo(a.toAttribute, l) + allConstraints += EqualNullSafe(a.toAttribute, l) case a @ Alias(e, _) => // For every alias in `projectList`, replace the reference in constraints by its attribute. allConstraints ++= allConstraints.map(_ transform { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index e4671f0d1cce..a40ba2dc38b7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -196,7 +196,7 @@ class InferFiltersFromConstraintsSuite extends PlanTest { test("constraints should be inferred from aliased literals") { val originalLeft = testRelation.subquery('left).as("left") - val optimizedLeft = testRelation.subquery('left).where(IsNotNull('a) && 'a === 2).as("left") + val optimizedLeft = testRelation.subquery('left).where(IsNotNull('a) && 'a <=> 2).as("left") val right = Project(Seq(Literal(2).as("two")), testRelation.subquery('right)).as("right") val condition = Some("left.a".attr === "right.two".attr) 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 45b17b3d4958..435b887cb3c7 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 @@ -2552,4 +2552,21 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } } + test("SPARK-25368 Incorrect predicate pushdown returns wrong result") { + def check(newCol: Column, filter: Column, result: Seq[Row]): Unit = { + val df1 = spark.createDataFrame(Seq( + (1, 1) + )).toDF("a", "b").withColumn("c", newCol) + + val df2 = df1.union(df1).withColumn("d", spark_partition_id).filter(filter) + checkAnswer(df2, result) + } + + check(lit(null).cast("int"), $"c".isNull, Seq(Row(1, 1, null, 0), Row(1, 1, null, 1))) + check(lit(null).cast("int"), $"c".isNotNull, Seq()) + check(lit(2).cast("int"), $"c".isNull, Seq()) + check(lit(2).cast("int"), $"c".isNotNull, Seq(Row(1, 1, 2, 0), Row(1, 1, 2, 1))) + check(lit(2).cast("int"), $"c" === 2, Seq(Row(1, 1, 2, 0), Row(1, 1, 2, 1))) + check(lit(2).cast("int"), $"c" =!= 2, Seq()) + } } From 0782dfa14c524131c04320e26d2b607777fe3b06 Mon Sep 17 00:00:00 2001 From: seancxmao Date: Sun, 9 Sep 2018 19:22:47 -0700 Subject: [PATCH 018/879] [SPARK-25175][SQL] Field resolution should fail if there is ambiguity for ORC native data source table persisted in metastore ## What changes were proposed in this pull request? Apache Spark doesn't create Hive table with duplicated fields in both case-sensitive and case-insensitive mode. However, if Spark creates ORC files in case-sensitive mode first and create Hive table on that location, where it's created. In this situation, field resolution should fail in case-insensitive mode. Otherwise, we don't know which columns will be returned or filtered. Previously, SPARK-25132 fixed the same issue in Parquet. Here is a simple example: ``` val data = spark.range(5).selectExpr("id as a", "id * 2 as A") spark.conf.set("spark.sql.caseSensitive", true) data.write.format("orc").mode("overwrite").save("/user/hive/warehouse/orc_data") sql("CREATE TABLE orc_data_source (A LONG) USING orc LOCATION '/user/hive/warehouse/orc_data'") spark.conf.set("spark.sql.caseSensitive", false) sql("select A from orc_data_source").show +---+ | A| +---+ | 3| | 2| | 4| | 1| | 0| +---+ ``` See #22148 for more details about parquet data source reader. ## How was this patch tested? Unit tests added. Closes #22262 from seancxmao/SPARK-25175. Authored-by: seancxmao Signed-off-by: Dongjoon Hyun (cherry picked from commit a0aed475c54079665a8e5c5cd53a2e990a4f47b4) Signed-off-by: Dongjoon Hyun --- .../execution/datasources/orc/OrcUtils.scala | 29 +++++++- .../spark/sql/FileBasedDataSourceSuite.scala | 71 ++++++++++--------- 2 files changed, 62 insertions(+), 38 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index ac062fdc092e..95fb25bf5add 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources.orc +import java.util.Locale + import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration @@ -27,7 +29,7 @@ import org.apache.spark.SparkException import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.analysis.{caseInsensitiveResolution, caseSensitiveResolution} +import org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.types._ @@ -116,8 +118,29 @@ object OrcUtils extends Logging { } }) } else { - val resolver = if (isCaseSensitive) caseSensitiveResolution else caseInsensitiveResolution - Some(requiredSchema.fieldNames.map { name => orcFieldNames.indexWhere(resolver(_, name)) }) + if (isCaseSensitive) { + Some(requiredSchema.fieldNames.map { name => + orcFieldNames.indexWhere(caseSensitiveResolution(_, name)) + }) + } else { + // Do case-insensitive resolution only if in case-insensitive mode + val caseInsensitiveOrcFieldMap = + orcFieldNames.zipWithIndex.groupBy(_._1.toLowerCase(Locale.ROOT)) + Some(requiredSchema.fieldNames.map { requiredFieldName => + caseInsensitiveOrcFieldMap + .get(requiredFieldName.toLowerCase(Locale.ROOT)) + .map { matchedOrcFields => + if (matchedOrcFields.size > 1) { + // Need to fail if there is ambiguity, i.e. more than one field is matched. + val matchedOrcFieldsString = matchedOrcFields.map(_._1).mkString("[", ", ", "]") + throw new RuntimeException(s"""Found duplicate field(s) "$requiredFieldName": """ + + s"$matchedOrcFieldsString in case-insensitive mode") + } else { + matchedOrcFields.head._2 + } + }.getOrElse(-1) + }) + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala index 304ede9c5a61..94f163708832 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/FileBasedDataSourceSuite.scala @@ -434,44 +434,45 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo } } - test(s"SPARK-25132: case-insensitive field resolution when reading from Parquet") { - withTempDir { dir => - val format = "parquet" - val tableDir = dir.getCanonicalPath + s"/$format" - val tableName = s"spark_25132_${format}" - withTable(tableName) { - val end = 5 - val data = spark.range(end).selectExpr("id as A", "id * 2 as b", "id * 3 as B") - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - data.write.format(format).mode("overwrite").save(tableDir) - } - sql(s"CREATE TABLE $tableName (a LONG, b LONG) USING $format LOCATION '$tableDir'") - - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - checkAnswer(sql(s"select a from $tableName"), data.select("A")) - checkAnswer(sql(s"select A from $tableName"), data.select("A")) - - // RuntimeException is triggered at executor side, which is then wrapped as - // SparkException at driver side - val e1 = intercept[SparkException] { - sql(s"select b from $tableName").collect() + Seq("parquet", "orc").foreach { format => + test(s"Spark native readers should respect spark.sql.caseSensitive - ${format}") { + withTempDir { dir => + val tableName = s"spark_25132_${format}_native" + val tableDir = dir.getCanonicalPath + s"/$tableName" + withTable(tableName) { + val end = 5 + val data = spark.range(end).selectExpr("id as A", "id * 2 as b", "id * 3 as B") + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + data.write.format(format).mode("overwrite").save(tableDir) } - assert( - e1.getCause.isInstanceOf[RuntimeException] && - e1.getCause.getMessage.contains( - """Found duplicate field(s) "b": [b, B] in case-insensitive mode""")) - val e2 = intercept[SparkException] { - sql(s"select B from $tableName").collect() + sql(s"CREATE TABLE $tableName (a LONG, b LONG) USING $format LOCATION '$tableDir'") + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkAnswer(sql(s"select a from $tableName"), data.select("A")) + checkAnswer(sql(s"select A from $tableName"), data.select("A")) + + // RuntimeException is triggered at executor side, which is then wrapped as + // SparkException at driver side + val e1 = intercept[SparkException] { + sql(s"select b from $tableName").collect() + } + assert( + e1.getCause.isInstanceOf[RuntimeException] && + e1.getCause.getMessage.contains( + """Found duplicate field(s) "b": [b, B] in case-insensitive mode""")) + val e2 = intercept[SparkException] { + sql(s"select B from $tableName").collect() + } + assert( + e2.getCause.isInstanceOf[RuntimeException] && + e2.getCause.getMessage.contains( + """Found duplicate field(s) "b": [b, B] in case-insensitive mode""")) } - assert( - e2.getCause.isInstanceOf[RuntimeException] && - e2.getCause.getMessage.contains( - """Found duplicate field(s) "b": [b, B] in case-insensitive mode""")) - } - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - checkAnswer(sql(s"select a from $tableName"), (0 until end).map(_ => Row(null))) - checkAnswer(sql(s"select b from $tableName"), data.select("b")) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + checkAnswer(sql(s"select a from $tableName"), (0 until end).map(_ => Row(null))) + checkAnswer(sql(s"select b from $tableName"), data.select("b")) + } } } } From c9ca3594345610148ef5d993262d3090d5b2c658 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Mon, 10 Sep 2018 13:47:19 +0800 Subject: [PATCH 019/879] [SPARK-25313][SQL][FOLLOW-UP] Fix InsertIntoHiveDirCommand output schema in Parquet issue ## What changes were proposed in this pull request? How to reproduce: ```scala spark.sql("CREATE TABLE tbl(id long)") spark.sql("INSERT OVERWRITE TABLE tbl VALUES 4") spark.sql("CREATE VIEW view1 AS SELECT id FROM tbl") spark.sql(s"INSERT OVERWRITE LOCAL DIRECTORY '/tmp/spark/parquet' " + "STORED AS PARQUET SELECT ID FROM view1") spark.read.parquet("/tmp/spark/parquet").schema scala> spark.read.parquet("/tmp/spark/parquet").schema res10: org.apache.spark.sql.types.StructType = StructType(StructField(id,LongType,true)) ``` The schema should be `StructType(StructField(ID,LongType,true))` as we `SELECT ID FROM view1`. This pr fix this issue. ## How was this patch tested? unit tests Closes #22359 from wangyum/SPARK-25313-FOLLOW-UP. Authored-by: Yuming Wang Signed-off-by: Wenchen Fan (cherry picked from commit f8b4d5aafd1923d9524415601469f8749b3d0811) Signed-off-by: Wenchen Fan --- .../command/DataWritingCommand.scala | 15 --------------- .../CreateHiveTableAsSelectCommand.scala | 4 ++-- .../execution/InsertIntoHiveDirCommand.scala | 5 ++--- .../hive/execution/InsertIntoHiveTable.scala | 1 - .../sql/hive/execution/SaveAsHiveFile.scala | 3 +-- .../sql/hive/execution/HiveDDLSuite.scala | 19 +++++++++++++++++++ 6 files changed, 24 insertions(+), 23 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala index 0a185b847206..a1bb5af1ab72 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/DataWritingCommand.scala @@ -26,7 +26,6 @@ import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.datasources.BasicWriteJobStatsTracker import org.apache.spark.sql.execution.datasources.FileFormatWriter import org.apache.spark.sql.execution.metric.SQLMetric -import org.apache.spark.sql.types.StructType import org.apache.spark.util.SerializableConfiguration /** @@ -75,18 +74,4 @@ object DataWritingCommand { attr.withName(outputName) } } - - /** - * Returns schema of logical plan with provided names. - * The length of provided names should be the same of the length of [[LogicalPlan.schema]]. - */ - def logicalPlanSchemaWithNames( - query: LogicalPlan, - names: Seq[String]): StructType = { - assert(query.schema.length == names.length, - "The length of provided names doesn't match the length of query schema.") - StructType(query.schema.zip(names).map { case (structField, outputName) => - structField.copy(name = outputName) - }) - } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 0eb2f0de0acd..aa573b54a2b6 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -69,8 +69,8 @@ case class CreateHiveTableAsSelectCommand( // add the relation into catalog, just in case of failure occurs while data // processing. assert(tableDesc.schema.isEmpty) - val schema = DataWritingCommand.logicalPlanSchemaWithNames(query, outputColumnNames) - catalog.createTable(tableDesc.copy(schema = schema), ignoreIfExists = false) + catalog.createTable( + tableDesc.copy(schema = outputColumns.toStructType), ignoreIfExists = false) try { // Read back the metadata of the table which was created just now. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 0a73aaa94bc7..a24e902074c2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -66,7 +66,7 @@ case class InsertIntoHiveDirCommand( identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")), tableType = org.apache.spark.sql.catalyst.catalog.CatalogTableType.VIEW, storage = storage, - schema = query.schema + schema = outputColumns.toStructType )) hiveTable.getMetadata.put(serdeConstants.SERIALIZATION_LIB, storage.serde.getOrElse(classOf[LazySimpleSerDe].getName)) @@ -104,8 +104,7 @@ case class InsertIntoHiveDirCommand( plan = child, hadoopConf = hadoopConf, fileSinkConf = fileSinkConf, - outputLocation = tmpPath.toString, - allColumns = outputColumns) + outputLocation = tmpPath.toString) val fs = writeToPath.getFileSystem(hadoopConf) if (overwrite && fs.exists(writeToPath)) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 75a0563e72c9..0ed464dad91b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -198,7 +198,6 @@ case class InsertIntoHiveTable( hadoopConf = hadoopConf, fileSinkConf = fileSinkConf, outputLocation = tmpLocation.toString, - allColumns = outputColumns, partitionAttributes = partitionAttributes) if (partition.nonEmpty) { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index e0f7375387d2..078968ed0145 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -51,7 +51,6 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { hadoopConf: Configuration, fileSinkConf: FileSinkDesc, outputLocation: String, - allColumns: Seq[Attribute], customPartitionLocations: Map[TablePartitionSpec, String] = Map.empty, partitionAttributes: Seq[Attribute] = Nil): Set[String] = { @@ -90,7 +89,7 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { fileFormat = new HiveFileFormat(fileSinkConf), committer = committer, outputSpec = - FileFormatWriter.OutputSpec(outputLocation, customPartitionLocations, allColumns), + FileFormatWriter.OutputSpec(outputLocation, customPartitionLocations, outputColumns), hadoopConf = hadoopConf, partitionColumns = partitionAttributes, bucketSpec = None, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 9acd5e1c248e..69ee2bbf0665 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -803,6 +803,25 @@ class HiveDDLSuite } } + test("SPARK-25313 Insert overwrite directory should output correct schema") { + withSQLConf(CONVERT_METASTORE_PARQUET.key -> "false") { + withTable("tbl") { + withView("view1") { + spark.sql("CREATE TABLE tbl(id long)") + spark.sql("INSERT OVERWRITE TABLE tbl VALUES 4") + spark.sql("CREATE VIEW view1 AS SELECT id FROM tbl") + withTempPath { path => + spark.sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path.getCanonicalPath}' " + + "STORED AS PARQUET SELECT ID FROM view1") + val expectedSchema = StructType(Seq(StructField("ID", LongType, true))) + assert(spark.read.parquet(path.toString).schema == expectedSchema) + checkAnswer(spark.read.parquet(path.toString), Seq(Row(4))) + } + } + } + } + } + test("alter table partition - storage information") { sql("CREATE TABLE boxes (height INT, length INT) PARTITIONED BY (width INT)") sql("INSERT OVERWRITE TABLE boxes PARTITION (width=4) SELECT 4, 4") From 67bc7ef7b70b6b654433bd5e56cff2f5ec6ae9bd Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 10 Sep 2018 19:18:00 +0800 Subject: [PATCH 020/879] [SPARK-24849][SPARK-24911][SQL][FOLLOW-UP] Converting a value of StructType to a DDL string ## What changes were proposed in this pull request? Add the version number for the new APIs. ## How was this patch tested? N/A Closes #22377 from gatorsmile/followup24849. Authored-by: gatorsmile Signed-off-by: Wenchen Fan (cherry picked from commit 6f6517837ba9934a280b11aba9d9be58bc131f25) Signed-off-by: Wenchen Fan --- .../main/scala/org/apache/spark/sql/types/StructField.scala | 2 ++ .../main/scala/org/apache/spark/sql/types/StructType.scala | 4 ++++ 2 files changed, 6 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala index 902cae9150ed..35f9970a0aae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -79,6 +79,8 @@ case class StructField( /** * Returns a string containing a schema in DDL format. For example, the following value: * `StructField("eventId", IntegerType)` will be converted to `eventId` INT. + * + * @since 2.4.0 */ def toDDL: String = { val comment = getComment() 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 c5ca169c955d..06289b148320 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 @@ -365,6 +365,8 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru * `StructType(Seq(StructField("eventId", IntegerType), StructField("s", StringType)))` * will be converted to `eventId` INT, `s` STRING. * The returned DDL schema can be used in a table creation. + * + * @since 2.4.0 */ def toDDL: String = fields.map(_.toDDL).mkString(",") @@ -441,6 +443,8 @@ object StructType extends AbstractDataType { /** * Creates StructType for a given DDL-formatted string, which is a comma separated list of field * definitions, e.g., a INT, b STRING. + * + * @since 2.2.0 */ def fromDDL(ddl: String): StructType = CatalystSqlParser.parseTableSchema(ddl) From 5d98c31941471bdcdc54a68f55ddaaab48f82161 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Mon, 10 Sep 2018 19:41:51 +0800 Subject: [PATCH 021/879] [SPARK-25278][SQL] Avoid duplicated Exec nodes when the same logical plan appears in the query ## What changes were proposed in this pull request? In the Planner, we collect the placeholder which need to be substituted in the query execution plan and once we plan them, we substitute the placeholder with the effective plan. In this second phase, we rely on the `==` comparison, ie. the `equals` method. This means that if two placeholder plans - which are different instances - have the same attributes (so that they are equal, according to the equal method) they are both substituted with their corresponding new physical plans. So, in such a situation, the first time we substitute both them with the first of the 2 new generated plan and the second time we substitute nothing. This is usually of no harm for the execution of the query itself, as the 2 plans are identical. But since they are the same instance, now, the local variables are shared (which is unexpected). This causes issues for the metrics collected, as the same node is executed 2 times, so the metrics are accumulated 2 times, wrongly. The PR proposes to use the `eq` method in checking which placeholder needs to be substituted,; thus in the previous situation, actually both the two different physical nodes which are created (one for each time the logical plan appears in the query plan) are used and the metrics are collected properly for each of them. ## How was this patch tested? added UT Closes #22284 from mgaido91/SPARK-25278. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan (cherry picked from commit 12e3e9f17dca11a2cddf0fb99d72b4b97517fb56) Signed-off-by: Wenchen Fan --- .../sql/catalyst/planning/QueryPlanner.scala | 2 +- .../spark/sql/execution/PlannerSuite.scala | 17 +++++++++++++++++ .../sql/execution/metric/SQLMetricsSuite.scala | 13 +++++++++++++ 3 files changed, 31 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala index bc41dd0465e3..6fa5203a06f7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -81,7 +81,7 @@ abstract class QueryPlanner[PhysicalPlan <: TreeNode[PhysicalPlan]] { childPlans.map { childPlan => // Replace the placeholder by the child plan candidateWithPlaceholders.transformUp { - case p if p == placeholder => childPlan + case p if p.eq(placeholder) => childPlan } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 3db89ecfad9f..b10da6c70be1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -704,6 +704,23 @@ class PlannerSuite extends SharedSQLContext { df.queryExecution.executedPlan.execute() } + test("SPARK-25278: physical nodes should be different instances for same logical nodes") { + val range = Range(1, 1, 1, 1) + val df = Union(range, range) + val ranges = df.queryExecution.optimizedPlan.collect { + case r: Range => r + } + assert(ranges.length == 2) + // Ensure the two Range instances are equal according to their equal method + assert(ranges.head == ranges.last) + val execRanges = df.queryExecution.sparkPlan.collect { + case r: RangeExec => r + } + assert(execRanges.length == 2) + // Ensure the two RangeExec instances are different instances + assert(!execRanges.head.eq(execRanges.last)) + } + test("SPARK-24556: always rewrite output partitioning in ReusedExchangeExec " + "and InMemoryTableScanExec") { def checkOutputPartitioningRewrite( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index a3a3f3851e21..d45eb0c27a6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -497,6 +497,19 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared } } + test("SPARK-25278: output metrics are wrong for plans repeated in the query") { + val name = "demo_view" + withView(name) { + sql(s"CREATE OR REPLACE VIEW $name AS VALUES 1,2") + val view = spark.table(name) + val union = view.union(view) + testSparkPlanMetrics(union, 1, Map( + 0L -> ("Union" -> Map()), + 1L -> ("LocalTableScan" -> Map("number of output rows" -> 2L)), + 2L -> ("LocalTableScan" -> Map("number of output rows" -> 2L)))) + } + } + test("writing data out metrics: parquet") { testMetricsNonDynamicPartition("parquet", "t1") } From ffd036a6d13814ebcc332990be1e286939cc6abe Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Mon, 10 Sep 2018 11:01:51 -0700 Subject: [PATCH 022/879] [SPARK-23672][PYTHON] Document support for nested return types in scalar with arrow udfs ## What changes were proposed in this pull request? Clarify docstring for Scalar functions ## How was this patch tested? Adds a unit test showing use similar to wordcount, there's existing unit test for array of floats as well. Closes #20908 from holdenk/SPARK-23672-document-support-for-nested-return-types-in-scalar-with-arrow-udfs. Authored-by: Holden Karau Signed-off-by: Bryan Cutler (cherry picked from commit da5685b5bb9ee7daaeb4e8f99c488ebd50c7aac3) Signed-off-by: Bryan Cutler --- python/pyspark/sql/functions.py | 3 ++- python/pyspark/sql/tests.py | 19 +++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 9396b16b7ada..81f35f54aa54 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2720,9 +2720,10 @@ def pandas_udf(f=None, returnType=None, functionType=None): 1. SCALAR A scalar UDF defines a transformation: One or more `pandas.Series` -> A `pandas.Series`. - The returnType should be a primitive data type, e.g., :class:`DoubleType`. The length of the returned `pandas.Series` must be of the same as the input `pandas.Series`. + :class:`MapType`, :class:`StructType` are currently not supported as output types. + Scalar UDFs are used with :meth:`pyspark.sql.DataFrame.withColumn` and :meth:`pyspark.sql.DataFrame.select`. diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 6d9d636b23a3..8e5bc6729dfa 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -4443,6 +4443,7 @@ def test_timestamp_dst(self): not _have_pandas or not _have_pyarrow, _pandas_requirement_message or _pyarrow_requirement_message) class PandasUDFTests(ReusedSQLTestCase): + def test_pandas_udf_basic(self): from pyspark.rdd import PythonEvalType from pyspark.sql.functions import pandas_udf, PandasUDFType @@ -4658,6 +4659,24 @@ def random_udf(v): random_udf = random_udf.asNondeterministic() return random_udf + def test_pandas_udf_tokenize(self): + from pyspark.sql.functions import pandas_udf + tokenize = pandas_udf(lambda s: s.apply(lambda str: str.split(' ')), + ArrayType(StringType())) + self.assertEqual(tokenize.returnType, ArrayType(StringType())) + df = self.spark.createDataFrame([("hi boo",), ("bye boo",)], ["vals"]) + result = df.select(tokenize("vals").alias("hi")) + self.assertEqual([Row(hi=[u'hi', u'boo']), Row(hi=[u'bye', u'boo'])], result.collect()) + + def test_pandas_udf_nested_arrays(self): + from pyspark.sql.functions import pandas_udf + tokenize = pandas_udf(lambda s: s.apply(lambda str: [str.split(' ')]), + ArrayType(ArrayType(StringType()))) + self.assertEqual(tokenize.returnType, ArrayType(ArrayType(StringType()))) + df = self.spark.createDataFrame([("hi boo",), ("bye boo",)], ["vals"]) + result = df.select(tokenize("vals").alias("hi")) + self.assertEqual([Row(hi=[[u'hi', u'boo']]), Row(hi=[[u'bye', u'boo']])], result.collect()) + def test_vectorized_udf_basic(self): from pyspark.sql.functions import pandas_udf, col, array df = self.spark.range(10).select( From fb4965a41941f3a196de77a870a8a1f29c96dac0 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Tue, 11 Sep 2018 14:16:56 +0800 Subject: [PATCH 023/879] [SPARK-25371][SQL] struct() should allow being called with 0 args ## What changes were proposed in this pull request? SPARK-21281 introduced a check for the inputs of `CreateStructLike` to be non-empty. This means that `struct()`, which was previously considered valid, now throws an Exception. This behavior change was introduced in 2.3.0. The change may break users' application on upgrade and it causes `VectorAssembler` to fail when an empty `inputCols` is defined. The PR removes the added check making `struct()` valid again. ## How was this patch tested? added UT Closes #22373 from mgaido91/SPARK-25371. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan (cherry picked from commit 0736e72a66735664b191fc363f54e3c522697dba) Signed-off-by: Wenchen Fan --- .../org/apache/spark/ml/feature/VectorAssemblerSuite.scala | 5 +++++ .../spark/sql/catalyst/expressions/complexTypeCreator.scala | 5 +---- .../scala/org/apache/spark/sql/DataFrameFunctionsSuite.scala | 2 -- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala index ed15a1d88a26..a4d388fd321d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala @@ -256,4 +256,9 @@ class VectorAssemblerSuite assert(runWithMetadata("keep", additional_filter = "id1 > 2").count() == 4) } + test("SPARK-25371: VectorAssembler with empty inputCols") { + val vectorAssembler = new VectorAssembler().setInputCols(Array()).setOutputCol("a") + val output = vectorAssembler.transform(dfWithNullsAndNaNs) + assert(output.select("a").limit(1).collect().head == Row(Vectors.sparse(0, Seq.empty))) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 077a6dc93bd1..aba9c6c8ad6f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -379,10 +379,7 @@ trait CreateNamedStructLike extends Expression { } override def checkInputDataTypes(): TypeCheckResult = { - if (children.length < 1) { - TypeCheckResult.TypeCheckFailure( - s"input to function $prettyName requires at least one argument") - } else if (children.size % 2 != 0) { + if (children.size % 2 != 0) { TypeCheckResult.TypeCheckFailure(s"$prettyName expects an even number of arguments.") } else { val invalidNames = nameExprs.filterNot(e => e.foldable && e.dataType == StringType) 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 4b83e51fa899..121db442c77f 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 @@ -2677,8 +2677,6 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { val funcsMustHaveAtLeastOneArg = ("coalesce", (df: DataFrame) => df.select(coalesce())) :: ("coalesce", (df: DataFrame) => df.selectExpr("coalesce()")) :: - ("named_struct", (df: DataFrame) => df.select(struct())) :: - ("named_struct", (df: DataFrame) => df.selectExpr("named_struct()")) :: ("hash", (df: DataFrame) => df.select(hash())) :: ("hash", (df: DataFrame) => df.selectExpr("hash()")) :: Nil funcsMustHaveAtLeastOneArg.foreach { case (name, func) => From b7efca7ece484ee85091b1b50bbc84ad779f9bfe Mon Sep 17 00:00:00 2001 From: Mario Molina Date: Tue, 11 Sep 2018 20:47:14 +0800 Subject: [PATCH 024/879] [SPARK-17916][SPARK-25241][SQL][FOLLOW-UP] Fix empty string being parsed as null when nullValue is set. ## What changes were proposed in this pull request? In the PR, I propose new CSV option `emptyValue` and an update in the SQL Migration Guide which describes how to revert previous behavior when empty strings were not written at all. Since Spark 2.4, empty strings are saved as `""` to distinguish them from saved `null`s. Closes #22234 Closes #22367 ## How was this patch tested? It was tested by `CSVSuite` and new tests added in the PR #22234 Closes #22389 from MaxGekk/csv-empty-value-master. Lead-authored-by: Mario Molina Co-authored-by: Maxim Gekk Signed-off-by: hyukjinkwon (cherry picked from commit c9cb393dc414ae98093c1541d09fa3c8663ce276) Signed-off-by: hyukjinkwon --- docs/sql-programming-guide.md | 1 + python/pyspark/sql/readwriter.py | 12 +- python/pyspark/sql/streaming.py | 7 +- .../apache/spark/sql/DataFrameReader.scala | 1 + .../apache/spark/sql/DataFrameWriter.scala | 1 + .../datasources/csv/CSVOptions.scala | 19 ++- .../sql/streaming/DataStreamReader.scala | 1 + .../resources/test-data/cars-empty-value.csv | 4 + .../execution/datasources/csv/CSVSuite.scala | 111 ++++++++++++++++++ 9 files changed, 149 insertions(+), 8 deletions(-) create mode 100644 sql/core/src/test/resources/test-data/cars-empty-value.csv diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 374909456927..9da7d64322eb 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1897,6 +1897,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.parallelFileListingInStatsComputation.enabled` to `False`. - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. + - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. ## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 49f4e6b2ede1..3ca5d548ae7d 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -349,7 +349,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None, maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None, columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None, - samplingRatio=None, enforceSchema=None): + samplingRatio=None, enforceSchema=None, emptyValue=None): """Loads a CSV file and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if @@ -444,6 +444,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non different, ``\0`` otherwise. :param samplingRatio: defines fraction of rows used for schema inferring. If None is set, it uses the default value, ``1.0``. + :param emptyValue: sets the string representation of an empty value. If None is set, it uses + the default value, empty string. >>> df = spark.read.csv('python/test_support/sql/ages.csv') >>> df.dtypes @@ -463,7 +465,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine, charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, samplingRatio=samplingRatio, - enforceSchema=enforceSchema) + enforceSchema=enforceSchema, emptyValue=emptyValue) if isinstance(path, basestring): path = [path] if type(path) == list: @@ -859,7 +861,7 @@ def text(self, path, compression=None, lineSep=None): def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=None, header=None, nullValue=None, escapeQuotes=None, quoteAll=None, dateFormat=None, timestampFormat=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, - charToEscapeQuoteEscaping=None, encoding=None): + charToEscapeQuoteEscaping=None, encoding=None, emptyValue=None): """Saves the content of the :class:`DataFrame` in CSV format at the specified path. :param path: the path in any Hadoop supported file system @@ -911,6 +913,8 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No different, ``\0`` otherwise.. :param encoding: sets the encoding (charset) of saved csv files. If None is set, the default UTF-8 charset will be used. + :param emptyValue: sets the string representation of an empty value. If None is set, it uses + the default value, ``""``. >>> df.write.csv(os.path.join(tempfile.mkdtemp(), 'data')) """ @@ -921,7 +925,7 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No ignoreLeadingWhiteSpace=ignoreLeadingWhiteSpace, ignoreTrailingWhiteSpace=ignoreTrailingWhiteSpace, charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, - encoding=encoding) + encoding=encoding, emptyValue=emptyValue) self._jwrite.csv(path) @since(1.5) diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index ee13778a7dcd..522900bf6684 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -564,7 +564,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non negativeInf=None, dateFormat=None, timestampFormat=None, maxColumns=None, maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None, columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None, - enforceSchema=None): + enforceSchema=None, emptyValue=None): """Loads a CSV file stream and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if @@ -658,6 +658,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non the quote character. If None is set, the default value is escape character when escape and quote characters are different, ``\0`` otherwise.. + :param emptyValue: sets the string representation of an empty value. If None is set, it uses + the default value, empty string. >>> csv_sdf = spark.readStream.csv(tempfile.mkdtemp(), schema = sdf_schema) >>> csv_sdf.isStreaming @@ -674,7 +676,8 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non maxCharsPerColumn=maxCharsPerColumn, maxMalformedLogPerPartition=maxMalformedLogPerPartition, mode=mode, columnNameOfCorruptRecord=columnNameOfCorruptRecord, multiLine=multiLine, - charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, enforceSchema=enforceSchema) + charToEscapeQuoteEscaping=charToEscapeQuoteEscaping, enforceSchema=enforceSchema, + emptyValue=emptyValue) if isinstance(path, basestring): return self._df(self._jreader.csv(path)) else: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 0cfcc45fb3d3..e6c2cba79841 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -571,6 +571,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * whitespaces from values being read should be skipped. *
  • `nullValue` (default empty string): sets the string representation of a null value. Since * 2.0.1, this applies to all supported types including the string type.
  • + *
  • `emptyValue` (default empty string): sets the string representation of an empty value.
  • *
  • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
  • *
  • `positiveInf` (default `Inf`): sets the string representation of a positive infinity * value.
  • diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index eca2d5b97190..dfb8c4718550 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -635,6 +635,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * enclosed in quotes. Default is to only escape values containing a quote character. *
  • `header` (default `false`): writes the names of columns as the first line.
  • *
  • `nullValue` (default empty string): sets the string representation of a null value.
  • + *
  • `emptyValue` (default `""`): sets the string representation of an empty value.
  • *
  • `encoding` (by default it is not set): specifies encoding (charset) of saved csv * files. If it is not set, the UTF-8 charset will be used.
  • *
  • `compression` (default `null`): compression codec to use when saving to file. This can be diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala index fab8d62da0c1..492a21be6df3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala @@ -162,6 +162,21 @@ class CSVOptions( */ val enforceSchema = getBool("enforceSchema", default = true) + + /** + * String representation of an empty value in read and in write. + */ + val emptyValue = parameters.get("emptyValue") + /** + * The string is returned when CSV reader doesn't have any characters for input value, + * or an empty quoted string `""`. Default value is empty string. + */ + val emptyValueInRead = emptyValue.getOrElse("") + /** + * The value is used instead of an empty string in write. Default value is `""` + */ + val emptyValueInWrite = emptyValue.getOrElse("\"\"") + def asWriterSettings: CsvWriterSettings = { val writerSettings = new CsvWriterSettings() val format = writerSettings.getFormat @@ -173,7 +188,7 @@ class CSVOptions( writerSettings.setIgnoreLeadingWhitespaces(ignoreLeadingWhiteSpaceFlagInWrite) writerSettings.setIgnoreTrailingWhitespaces(ignoreTrailingWhiteSpaceFlagInWrite) writerSettings.setNullValue(nullValue) - writerSettings.setEmptyValue("\"\"") + writerSettings.setEmptyValue(emptyValueInWrite) writerSettings.setSkipEmptyLines(true) writerSettings.setQuoteAllFields(quoteAll) writerSettings.setQuoteEscapingEnabled(escapeQuotes) @@ -194,7 +209,7 @@ class CSVOptions( settings.setInputBufferSize(inputBufferSize) settings.setMaxColumns(maxColumns) settings.setNullValue(nullValue) - settings.setEmptyValue("") + settings.setEmptyValue(emptyValueInRead) settings.setMaxCharsPerColumn(maxCharsPerColumn) settings.setUnescapedQuoteHandling(UnescapedQuoteHandling.STOP_AT_DELIMITER) settings diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 39e9e1ad426b..2a4db4afbe00 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -327,6 +327,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * whitespaces from values being read should be skipped.
  • *
  • `nullValue` (default empty string): sets the string representation of a null value. Since * 2.0.1, this applies to all supported types including the string type.
  • + *
  • `emptyValue` (default empty string): sets the string representation of an empty value.
  • *
  • `nanValue` (default `NaN`): sets the string representation of a non-number" value.
  • *
  • `positiveInf` (default `Inf`): sets the string representation of a positive infinity * value.
  • diff --git a/sql/core/src/test/resources/test-data/cars-empty-value.csv b/sql/core/src/test/resources/test-data/cars-empty-value.csv new file mode 100644 index 000000000000..0f20a2f23ac0 --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars-empty-value.csv @@ -0,0 +1,4 @@ +year,make,model,comment,blank +"2012","Tesla","S","","" +1997,Ford,E350,"Go get one now they are going fast", +2015,Chevy,Volt,,"" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 5a1d6679ebbd..2b39a0b1f52e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -50,6 +50,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te private val carsAltFile = "test-data/cars-alternative.csv" private val carsUnbalancedQuotesFile = "test-data/cars-unbalanced-quotes.csv" private val carsNullFile = "test-data/cars-null.csv" + private val carsEmptyValueFile = "test-data/cars-empty-value.csv" private val carsBlankColName = "test-data/cars-blank-column-name.csv" private val emptyFile = "test-data/empty.csv" private val commentsFile = "test-data/comments.csv" @@ -668,6 +669,70 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te assert(results(2).toSeq === Array(null, "Chevy", "Volt", null, null)) } + test("empty fields with user defined empty values") { + + // year,make,model,comment,blank + val dataSchema = StructType(List( + StructField("year", IntegerType, nullable = true), + StructField("make", StringType, nullable = false), + StructField("model", StringType, nullable = false), + StructField("comment", StringType, nullable = true), + StructField("blank", StringType, nullable = true))) + val cars = spark.read + .format("csv") + .schema(dataSchema) + .option("header", "true") + .option("emptyValue", "empty") + .load(testFile(carsEmptyValueFile)) + + verifyCars(cars, withHeader = true, checkValues = false) + val results = cars.collect() + assert(results(0).toSeq === Array(2012, "Tesla", "S", "empty", "empty")) + assert(results(1).toSeq === + Array(1997, "Ford", "E350", "Go get one now they are going fast", null)) + assert(results(2).toSeq === Array(2015, "Chevy", "Volt", null, "empty")) + } + + test("save csv with empty fields with user defined empty values") { + withTempDir { dir => + val csvDir = new File(dir, "csv").getCanonicalPath + + // year,make,model,comment,blank + val dataSchema = StructType(List( + StructField("year", IntegerType, nullable = true), + StructField("make", StringType, nullable = false), + StructField("model", StringType, nullable = false), + StructField("comment", StringType, nullable = true), + StructField("blank", StringType, nullable = true))) + val cars = spark.read + .format("csv") + .schema(dataSchema) + .option("header", "true") + .option("nullValue", "NULL") + .load(testFile(carsEmptyValueFile)) + + cars.coalesce(1).write + .format("csv") + .option("header", "true") + .option("emptyValue", "empty") + .option("nullValue", null) + .save(csvDir) + + val carsCopy = spark.read + .format("csv") + .schema(dataSchema) + .option("header", "true") + .load(csvDir) + + verifyCars(carsCopy, withHeader = true, checkValues = false) + val results = carsCopy.collect() + assert(results(0).toSeq === Array(2012, "Tesla", "S", "empty", "empty")) + assert(results(1).toSeq === + Array(1997, "Ford", "E350", "Go get one now they are going fast", null)) + assert(results(2).toSeq === Array(2015, "Chevy", "Volt", null, "empty")) + } + } + test("save csv with compression codec option") { withTempDir { dir => val csvDir = new File(dir, "csv").getCanonicalPath @@ -1375,6 +1440,52 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te } } + test("SPARK-25241: An empty string should not be coerced to null when emptyValue is passed.") { + val litNull: String = null + val df = Seq( + (1, "John Doe"), + (2, ""), + (3, "-"), + (4, litNull) + ).toDF("id", "name") + + // Checks for new behavior where a null is not coerced to an empty string when `emptyValue` is + // set to anything but an empty string literal. + withTempPath { path => + df.write + .option("emptyValue", "-") + .csv(path.getAbsolutePath) + val computed = spark.read + .option("emptyValue", "-") + .schema(df.schema) + .csv(path.getAbsolutePath) + val expected = Seq( + (1, "John Doe"), + (2, "-"), + (3, "-"), + (4, "-") + ).toDF("id", "name") + + checkAnswer(computed, expected) + } + // Keeps the old behavior where empty string us coerced to emptyValue is not passed. + withTempPath { path => + df.write + .csv(path.getAbsolutePath) + val computed = spark.read + .schema(df.schema) + .csv(path.getAbsolutePath) + val expected = Seq( + (1, "John Doe"), + (2, litNull), + (3, "-"), + (4, litNull) + ).toDF("id", "name") + + checkAnswer(computed, expected) + } + } + test("SPARK-24329: skip lines with comments, and one or multiple whitespaces") { val schema = new StructType().add("colA", StringType) val ds = spark From 0b8bfbe12b8a368836d7ddc8445de18b7ee42cde Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 11 Sep 2018 08:57:42 -0700 Subject: [PATCH 025/879] [SPARK-25389][SQL] INSERT OVERWRITE DIRECTORY STORED AS should prevent duplicate fields ## What changes were proposed in this pull request? Like `INSERT OVERWRITE DIRECTORY USING` syntax, `INSERT OVERWRITE DIRECTORY STORED AS` should not generate files with duplicate fields because Spark cannot read those files back. **INSERT OVERWRITE DIRECTORY USING** ```scala scala> sql("INSERT OVERWRITE DIRECTORY 'file:///tmp/parquet' USING parquet SELECT 'id', 'id2' id") ... ERROR InsertIntoDataSourceDirCommand: Failed to write to directory ... org.apache.spark.sql.AnalysisException: Found duplicate column(s) when inserting into file:/tmp/parquet: `id`; ``` **INSERT OVERWRITE DIRECTORY STORED AS** ```scala scala> sql("INSERT OVERWRITE DIRECTORY 'file:///tmp/parquet' STORED AS parquet SELECT 'id', 'id2' id") // It generates corrupted files scala> spark.read.parquet("/tmp/parquet").show 18/09/09 22:09:57 WARN DataSource: Found duplicate column(s) in the data schema and the partition schema: `id`; ``` ## How was this patch tested? Pass the Jenkins with newly added test cases. Closes #22378 from dongjoon-hyun/SPARK-25389. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 77579aa8c35b0d98bbeac3c828bf68a1d190d13e) Signed-off-by: Dongjoon Hyun --- .../execution/InsertIntoHiveDirCommand.scala | 5 ++++ .../apache/spark/sql/hive/InsertSuite.scala | 24 +++++++++++++++++++ 2 files changed, 29 insertions(+) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index a24e902074c2..0c694910b06d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.hive.client.HiveClientImpl +import org.apache.spark.sql.util.SchemaUtils /** * Command for writing the results of `query` to file system. @@ -61,6 +62,10 @@ case class InsertIntoHiveDirCommand( override def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = { assert(storage.locationUri.nonEmpty) + SchemaUtils.checkColumnNameDuplication( + outputColumnNames, + s"when inserting into ${storage.locationUri.get}", + sparkSession.sessionState.conf.caseSensitiveAnalysis) val hiveTable = HiveClientImpl.toHiveTable(CatalogTable( identifier = TableIdentifier(storage.locationUri.get.toString, Some("default")), diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index ab91727049ff..5879748d05b2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -750,4 +751,27 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } } + + Seq("LOCAL", "").foreach { local => + Seq(true, false).foreach { caseSensitivity => + Seq("orc", "parquet").foreach { format => + test(s"SPARK-25389 INSERT OVERWRITE $local DIRECTORY ... STORED AS with duplicated names" + + s"(caseSensitivity=$caseSensitivity, format=$format)") { + withTempDir { dir => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> s"$caseSensitivity") { + val m = intercept[AnalysisException] { + sql( + s""" + |INSERT OVERWRITE $local DIRECTORY '${dir.toURI}' + |STORED AS $format + |SELECT 'id', 'id2' ${if (caseSensitivity) "id" else "ID"} + """.stripMargin) + }.getMessage + assert(m.contains("Found duplicate column(s) when inserting into")) + } + } + } + } + } + } } From 4414e026097c74aadd252b541c9d3009cd7e9d09 Mon Sep 17 00:00:00 2001 From: Gera Shegalov Date: Tue, 11 Sep 2018 09:28:32 -0700 Subject: [PATCH 026/879] [SPARK-25221][DEPLOY] Consistent trailing whitespace treatment of conf values ## What changes were proposed in this pull request? Stop trimming values of properties loaded from a file ## How was this patch tested? Added unit test demonstrating the issue hit in production. Closes #22213 from gerashegalov/gera/SPARK-25221. Authored-by: Gera Shegalov Signed-off-by: Marcelo Vanzin (cherry picked from commit bcb9a8c83f4e6835af5dc51f1be7f964b8fa49a3) Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/util/Utils.scala | 31 ++++++++++-- .../spark/deploy/SparkSubmitSuite.scala | 47 +++++++++++++++++++ .../org/apache/spark/util/UtilsSuite.scala | 28 +++++++++++ 3 files changed, 103 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 4593b057fc63..14f68cd6f350 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -19,7 +19,6 @@ package org.apache.spark.util import java.io._ import java.lang.{Byte => JByte} -import java.lang.InternalError import java.lang.management.{LockInfo, ManagementFactory, MonitorInfo, ThreadInfo} import java.lang.reflect.InvocationTargetException import java.math.{MathContext, RoundingMode} @@ -2052,6 +2051,30 @@ private[spark] object Utils extends Logging { } } + /** + * Implements the same logic as JDK `java.lang.String#trim` by removing leading and trailing + * non-printable characters less or equal to '\u0020' (SPACE) but preserves natural line + * delimiters according to [[java.util.Properties]] load method. The natural line delimiters are + * removed by JDK during load. Therefore any remaining ones have been specifically provided and + * escaped by the user, and must not be ignored + * + * @param str + * @return the trimmed value of str + */ + private[util] def trimExceptCRLF(str: String): String = { + val nonSpaceOrNaturalLineDelimiter: Char => Boolean = { ch => + ch > ' ' || ch == '\r' || ch == '\n' + } + + val firstPos = str.indexWhere(nonSpaceOrNaturalLineDelimiter) + val lastPos = str.lastIndexWhere(nonSpaceOrNaturalLineDelimiter) + if (firstPos >= 0 && lastPos >= 0) { + str.substring(firstPos, lastPos + 1) + } else { + "" + } + } + /** Load properties present in the given file. */ def getPropertiesFromFile(filename: String): Map[String, String] = { val file = new File(filename) @@ -2062,8 +2085,10 @@ private[spark] object Utils extends Logging { try { val properties = new Properties() properties.load(inReader) - properties.stringPropertyNames().asScala.map( - k => (k, properties.getProperty(k).trim)).toMap + properties.stringPropertyNames().asScala + .map { k => (k, trimExceptCRLF(properties.getProperty(k))) } + .toMap + } catch { case e: IOException => throw new SparkException(s"Failed when loading Spark properties from $filename", e) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index f829fecc3084..9eae3605d073 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -1144,6 +1144,53 @@ class SparkSubmitSuite conf1.get(PY_FILES.key) should be (s"s3a://${pyFile.getAbsolutePath}") conf1.get("spark.submit.pyFiles") should (startWith("/")) } + + test("handles natural line delimiters in --properties-file and --conf uniformly") { + val delimKey = "spark.my.delimiter." + val LF = "\n" + val CR = "\r" + + val lineFeedFromCommandLine = s"${delimKey}lineFeedFromCommandLine" -> LF + val leadingDelimKeyFromFile = s"${delimKey}leadingDelimKeyFromFile" -> s"${LF}blah" + val trailingDelimKeyFromFile = s"${delimKey}trailingDelimKeyFromFile" -> s"blah${CR}" + val infixDelimFromFile = s"${delimKey}infixDelimFromFile" -> s"${CR}blah${LF}" + val nonDelimSpaceFromFile = s"${delimKey}nonDelimSpaceFromFile" -> " blah\f" + + val testProps = Seq(leadingDelimKeyFromFile, trailingDelimKeyFromFile, infixDelimFromFile, + nonDelimSpaceFromFile) + + val props = new java.util.Properties() + val propsFile = File.createTempFile("test-spark-conf", ".properties", + Utils.createTempDir()) + val propsOutputStream = new FileOutputStream(propsFile) + try { + testProps.foreach { case (k, v) => props.put(k, v) } + props.store(propsOutputStream, "test whitespace") + } finally { + propsOutputStream.close() + } + + val clArgs = Seq( + "--class", "org.SomeClass", + "--conf", s"${lineFeedFromCommandLine._1}=${lineFeedFromCommandLine._2}", + "--conf", "spark.master=yarn", + "--properties-file", propsFile.getPath, + "thejar.jar") + + val appArgs = new SparkSubmitArguments(clArgs) + val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) + + Seq( + lineFeedFromCommandLine, + leadingDelimKeyFromFile, + trailingDelimKeyFromFile, + infixDelimFromFile + ).foreach { case (k, v) => + conf.get(k) should be (v) + } + + conf.get(nonDelimSpaceFromFile._1) should be ("blah") + } } object SparkSubmitSuite extends SparkFunSuite with TimeLimits { diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 943b53522d64..39f4fba78583 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -1205,6 +1205,34 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging { assert(Utils.stringHalfWidth("\u0967\u0968\u0969") == 3) // scalastyle:on nonascii } + + test("trimExceptCRLF standalone") { + val crlfSet = Set("\r", "\n") + val nonPrintableButCRLF = (0 to 32).map(_.toChar.toString).toSet -- crlfSet + + // identity for CRLF + crlfSet.foreach { s => Utils.trimExceptCRLF(s) === s } + + // empty for other non-printables + nonPrintableButCRLF.foreach { s => assert(Utils.trimExceptCRLF(s) === "") } + + // identity for a printable string + assert(Utils.trimExceptCRLF("a") === "a") + + // identity for strings with CRLF + crlfSet.foreach { s => + assert(Utils.trimExceptCRLF(s"${s}a") === s"${s}a") + assert(Utils.trimExceptCRLF(s"a${s}") === s"a${s}") + assert(Utils.trimExceptCRLF(s"b${s}b") === s"b${s}b") + } + + // trim nonPrintableButCRLF except when inside a string + nonPrintableButCRLF.foreach { s => + assert(Utils.trimExceptCRLF(s"${s}a") === "a") + assert(Utils.trimExceptCRLF(s"a${s}") === "a") + assert(Utils.trimExceptCRLF(s"b${s}b") === s"b${s}b") + } + } } private class SimpleExtension From 16127e844f8334e1152b2e3ed3d878ec8de13dfa Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 11 Sep 2018 10:31:06 -0700 Subject: [PATCH 027/879] [SPARK-24889][CORE] Update block info when unpersist rdds ## What changes were proposed in this pull request? We will update block info coming from executors, at the timing like caching a RDD. However, when removing RDDs with unpersisting, we don't ask to update block info. So the block info is not updated. We can fix this with few options: 1. Ask to update block info when unpersisting This is simplest but changes driver-executor communication a bit. 2. Update block info when processing the event of unpersisting RDD We send a `SparkListenerUnpersistRDD` event when unpersisting RDD. When processing this event, we can update block info of the RDD. This only changes event processing code so the risk seems to be lower. Currently this patch takes option 2 for lower risk. If we agree first option has no risk, we can change to it. ## How was this patch tested? Unit tests. Closes #22341 from viirya/SPARK-24889. Authored-by: Liang-Chi Hsieh Signed-off-by: Marcelo Vanzin (cherry picked from commit 14f3ad20932535fe952428bf255e7eddd8fa1b58) Signed-off-by: Marcelo Vanzin --- .../spark/status/AppStatusListener.scala | 64 ++++++++++++++----- .../org/apache/spark/status/LiveEntity.scala | 4 ++ .../org/apache/spark/storage/RDDInfo.scala | 2 +- .../spark/status/AppStatusListenerSuite.scala | 29 +++++++++ 4 files changed, 82 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 91b75e485299..b9f604b6b80d 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -669,7 +669,37 @@ private[spark] class AppStatusListener( } override def onUnpersistRDD(event: SparkListenerUnpersistRDD): Unit = { - liveRDDs.remove(event.rddId) + liveRDDs.remove(event.rddId).foreach { liveRDD => + val storageLevel = liveRDD.info.storageLevel + + // Use RDD partition info to update executor block info. + liveRDD.getPartitions().foreach { case (_, part) => + part.executors.foreach { executorId => + liveExecutors.get(executorId).foreach { exec => + exec.rddBlocks = exec.rddBlocks - 1 + } + } + } + + val now = System.nanoTime() + + // Use RDD distribution to update executor memory and disk usage info. + liveRDD.getDistributions().foreach { case (executorId, rddDist) => + liveExecutors.get(executorId).foreach { exec => + if (exec.hasMemoryInfo) { + if (storageLevel.useOffHeap) { + exec.usedOffHeap = addDeltaToValue(exec.usedOffHeap, -rddDist.offHeapUsed) + } else { + exec.usedOnHeap = addDeltaToValue(exec.usedOnHeap, -rddDist.onHeapUsed) + } + } + exec.memoryUsed = addDeltaToValue(exec.memoryUsed, -rddDist.memoryUsed) + exec.diskUsed = addDeltaToValue(exec.diskUsed, -rddDist.diskUsed) + maybeUpdate(exec, now) + } + } + } + kvstore.delete(classOf[RDDStorageInfoWrapper], event.rddId) } @@ -728,6 +758,11 @@ private[spark] class AppStatusListener( .sortBy(_.stageId) } + /** + * Apply a delta to a value, but ensure that it doesn't go negative. + */ + private def addDeltaToValue(old: Long, delta: Long): Long = math.max(0, old + delta) + private def updateRDDBlock(event: SparkListenerBlockUpdated, block: RDDBlockId): Unit = { val now = System.nanoTime() val executorId = event.blockUpdatedInfo.blockManagerId.executorId @@ -737,9 +772,6 @@ private[spark] class AppStatusListener( val diskDelta = event.blockUpdatedInfo.diskSize * (if (storageLevel.useDisk) 1 else -1) val memoryDelta = event.blockUpdatedInfo.memSize * (if (storageLevel.useMemory) 1 else -1) - // Function to apply a delta to a value, but ensure that it doesn't go negative. - def newValue(old: Long, delta: Long): Long = math.max(0, old + delta) - val updatedStorageLevel = if (storageLevel.isValid) { Some(storageLevel.description) } else { @@ -756,13 +788,13 @@ private[spark] class AppStatusListener( maybeExec.foreach { exec => if (exec.hasMemoryInfo) { if (storageLevel.useOffHeap) { - exec.usedOffHeap = newValue(exec.usedOffHeap, memoryDelta) + exec.usedOffHeap = addDeltaToValue(exec.usedOffHeap, memoryDelta) } else { - exec.usedOnHeap = newValue(exec.usedOnHeap, memoryDelta) + exec.usedOnHeap = addDeltaToValue(exec.usedOnHeap, memoryDelta) } } - exec.memoryUsed = newValue(exec.memoryUsed, memoryDelta) - exec.diskUsed = newValue(exec.diskUsed, diskDelta) + exec.memoryUsed = addDeltaToValue(exec.memoryUsed, memoryDelta) + exec.diskUsed = addDeltaToValue(exec.diskUsed, diskDelta) } // Update the block entry in the RDD info, keeping track of the deltas above so that we @@ -790,8 +822,8 @@ private[spark] class AppStatusListener( // Only update the partition if it's still stored in some executor, otherwise get rid of it. if (executors.nonEmpty) { partition.update(executors, rdd.storageLevel, - newValue(partition.memoryUsed, memoryDelta), - newValue(partition.diskUsed, diskDelta)) + addDeltaToValue(partition.memoryUsed, memoryDelta), + addDeltaToValue(partition.diskUsed, diskDelta)) } else { rdd.removePartition(block.name) } @@ -799,14 +831,14 @@ private[spark] class AppStatusListener( maybeExec.foreach { exec => if (exec.rddBlocks + rddBlocksDelta > 0) { val dist = rdd.distribution(exec) - dist.memoryUsed = newValue(dist.memoryUsed, memoryDelta) - dist.diskUsed = newValue(dist.diskUsed, diskDelta) + dist.memoryUsed = addDeltaToValue(dist.memoryUsed, memoryDelta) + dist.diskUsed = addDeltaToValue(dist.diskUsed, diskDelta) if (exec.hasMemoryInfo) { if (storageLevel.useOffHeap) { - dist.offHeapUsed = newValue(dist.offHeapUsed, memoryDelta) + dist.offHeapUsed = addDeltaToValue(dist.offHeapUsed, memoryDelta) } else { - dist.onHeapUsed = newValue(dist.onHeapUsed, memoryDelta) + dist.onHeapUsed = addDeltaToValue(dist.onHeapUsed, memoryDelta) } } dist.lastUpdate = null @@ -825,8 +857,8 @@ private[spark] class AppStatusListener( } } - rdd.memoryUsed = newValue(rdd.memoryUsed, memoryDelta) - rdd.diskUsed = newValue(rdd.diskUsed, diskDelta) + rdd.memoryUsed = addDeltaToValue(rdd.memoryUsed, memoryDelta) + rdd.diskUsed = addDeltaToValue(rdd.diskUsed, diskDelta) update(rdd, now) } diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 79e3f13b826c..ce737ac0e769 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -538,6 +538,10 @@ private class LiveRDD(val info: RDDInfo) extends LiveEntity { distributions.get(exec.executorId) } + def getPartitions(): scala.collection.Map[String, LiveRDDPartition] = partitions + + def getDistributions(): scala.collection.Map[String, LiveRDDDistribution] = distributions + override protected def doUpdate(): Any = { val dists = if (distributions.nonEmpty) { Some(distributions.values.map(_.toApi()).toSeq) diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 9ccc8f9cc585..64e5c8b1c4bb 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -55,7 +55,7 @@ class RDDInfo( } private[spark] object RDDInfo { - private val callsiteForm = SparkEnv.get.conf.get(EVENT_LOG_CALLSITE_FORM) + private lazy val callsiteForm = SparkEnv.get.conf.get(EVENT_LOG_CALLSITE_FORM) def fromRdd(rdd: RDD[_]): RDDInfo = { val rddName = Option(rdd.name).getOrElse(Utils.getFormattedClassName(rdd)) diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index ea80fea90534..a6ccd17283fc 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -881,12 +881,41 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(dist.memoryRemaining === maxMemory - rdd2b1.memSize - rdd1b2.memSize ) } + // Add block1 of rdd1 back to bm 1. + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo(bm1, rdd1b1.blockId, level, rdd1b1.memSize, rdd1b1.diskSize))) + + check[ExecutorSummaryWrapper](bm1.executorId) { exec => + assert(exec.info.rddBlocks === 3L) + assert(exec.info.memoryUsed === rdd1b1.memSize + rdd1b2.memSize + rdd2b1.memSize) + assert(exec.info.diskUsed === rdd1b1.diskSize + rdd1b2.diskSize + rdd2b1.diskSize) + } + // Unpersist RDD1. listener.onUnpersistRDD(SparkListenerUnpersistRDD(rdd1b1.rddId)) intercept[NoSuchElementException] { check[RDDStorageInfoWrapper](rdd1b1.rddId) { _ => () } } + // executor1 now only contains block1 from rdd2. + check[ExecutorSummaryWrapper](bm1.executorId) { exec => + assert(exec.info.rddBlocks === 1L) + assert(exec.info.memoryUsed === rdd2b1.memSize) + assert(exec.info.diskUsed === rdd2b1.diskSize) + } + + // Unpersist RDD2. + listener.onUnpersistRDD(SparkListenerUnpersistRDD(rdd2b1.rddId)) + intercept[NoSuchElementException] { + check[RDDStorageInfoWrapper](rdd2b1.rddId) { _ => () } + } + + check[ExecutorSummaryWrapper](bm1.executorId) { exec => + assert(exec.info.rddBlocks === 0L) + assert(exec.info.memoryUsed === 0) + assert(exec.info.diskUsed === 0) + } + // Update a StreamBlock. val stream1 = StreamBlockId(1, 1L) listener.onBlockUpdated(SparkListenerBlockUpdated( From 99b37a91871f8bf070d43080f1c58475548c99fd Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 11 Sep 2018 14:46:03 -0500 Subject: [PATCH 028/879] [SPARK-25398] Minor bugs from comparing unrelated types MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Correct some comparisons between unrelated types to what they seem to… have been trying to do ## How was this patch tested? Existing tests. Closes #22384 from srowen/SPARK-25398. Authored-by: Sean Owen Signed-off-by: Sean Owen (cherry picked from commit cfbdd6a1f5906b848c520d3365cc4034992215d9) Signed-off-by: Sean Owen --- .../org/apache/spark/status/LiveEntity.scala | 4 +--- .../apache/spark/util/ClosureCleaner.scala | 2 +- .../ExternalAppendOnlyMapSuite.scala | 4 +--- .../cluster/mesos/MesosClusterScheduler.scala | 20 +++++++++---------- .../mesos/MesosClusterSchedulerSuite.scala | 14 ++++++------- ...esosFineGrainedSchedulerBackendSuite.scala | 2 +- .../spark/deploy/yarn/ClientSuite.scala | 2 +- .../PropagateEmptyRelationSuite.scala | 2 +- .../sql/catalyst/util/UnsafeArraySuite.scala | 4 ++-- .../org/apache/spark/sql/DatasetSuite.scala | 2 +- .../parquet/ParquetSchemaSuite.scala | 2 +- 11 files changed, 27 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index ce737ac0e769..093b477cdcdd 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -33,7 +33,6 @@ import org.apache.spark.storage.RDDInfo import org.apache.spark.ui.SparkUI import org.apache.spark.util.AccumulatorContext import org.apache.spark.util.collection.OpenHashSet -import org.apache.spark.util.kvstore.KVStore /** * A mutable representation of a live entity in Spark (jobs, stages, tasks, et al). Every live @@ -585,8 +584,7 @@ private object LiveEntityHelpers { .filter { acc => // We don't need to store internal or SQL accumulables as their values will be shown in // other places, so drop them to reduce the memory usage. - !acc.internal && (!acc.metadata.isDefined || - acc.metadata.get != Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER)) + !acc.internal && acc.metadata != Some(AccumulatorContext.SQL_ACCUM_IDENTIFIER) } .map { acc => new v1.AccumulableInfo( diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index b6c300c4778b..43d62561e8eb 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -175,7 +175,7 @@ private[spark] object ClosureCleaner extends Logging { closure.getClass.isSynthetic && closure .getClass - .getInterfaces.exists(_.getName.equals("scala.Serializable")) + .getInterfaces.exists(_.getName == "scala.Serializable") if (isClosureCandidate) { try { diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index d542ba0b6640..8a2f2ffe0acf 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.util.collection -import java.util.Objects - import scala.collection.mutable.ArrayBuffer import scala.ref.WeakReference @@ -509,7 +507,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite .sorted assert(it.isEmpty) - assert(keys == (0 until 100)) + assert(keys == (0 until 100).toList) assert(map.numSpills == 0) // these asserts try to show that we're no longer holding references to the underlying map. diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index 7d80eedcc43c..cb1bcba651be 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -202,7 +202,7 @@ private[spark] class MesosClusterScheduler( } else if (removeFromPendingRetryDrivers(submissionId)) { k.success = true k.message = "Removed driver while it's being retried" - } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) { + } else if (finishedDrivers.exists(_.driverDescription.submissionId == submissionId)) { k.success = false k.message = "Driver already terminated" } else { @@ -222,21 +222,21 @@ private[spark] class MesosClusterScheduler( } s.submissionId = submissionId stateLock.synchronized { - if (queuedDrivers.exists(_.submissionId.equals(submissionId))) { + if (queuedDrivers.exists(_.submissionId == submissionId)) { s.success = true s.driverState = "QUEUED" } else if (launchedDrivers.contains(submissionId)) { s.success = true s.driverState = "RUNNING" launchedDrivers(submissionId).mesosTaskStatus.foreach(state => s.message = state.toString) - } else if (finishedDrivers.exists(_.driverDescription.submissionId.equals(submissionId))) { + } else if (finishedDrivers.exists(_.driverDescription.submissionId == submissionId)) { s.success = true s.driverState = "FINISHED" finishedDrivers .find(d => d.driverDescription.submissionId.equals(submissionId)).get.mesosTaskStatus .foreach(state => s.message = state.toString) - } else if (pendingRetryDrivers.exists(_.submissionId.equals(submissionId))) { - val status = pendingRetryDrivers.find(_.submissionId.equals(submissionId)) + } else if (pendingRetryDrivers.exists(_.submissionId == submissionId)) { + val status = pendingRetryDrivers.find(_.submissionId == submissionId) .get.retryState.get.lastFailureStatus s.success = true s.driverState = "RETRYING" @@ -254,13 +254,13 @@ private[spark] class MesosClusterScheduler( */ def getDriverState(submissionId: String): Option[MesosDriverState] = { stateLock.synchronized { - queuedDrivers.find(_.submissionId.equals(submissionId)) + queuedDrivers.find(_.submissionId == submissionId) .map(d => new MesosDriverState("QUEUED", d)) .orElse(launchedDrivers.get(submissionId) .map(d => new MesosDriverState("RUNNING", d.driverDescription, Some(d)))) - .orElse(finishedDrivers.find(_.driverDescription.submissionId.equals(submissionId)) + .orElse(finishedDrivers.find(_.driverDescription.submissionId == submissionId) .map(d => new MesosDriverState("FINISHED", d.driverDescription, Some(d)))) - .orElse(pendingRetryDrivers.find(_.submissionId.equals(submissionId)) + .orElse(pendingRetryDrivers.find(_.submissionId == submissionId) .map(d => new MesosDriverState("RETRYING", d))) } } @@ -814,7 +814,7 @@ private[spark] class MesosClusterScheduler( status: Int): Unit = {} private def removeFromQueuedDrivers(subId: String): Boolean = { - val index = queuedDrivers.indexWhere(_.submissionId.equals(subId)) + val index = queuedDrivers.indexWhere(_.submissionId == subId) if (index != -1) { queuedDrivers.remove(index) queuedDriversState.expunge(subId) @@ -834,7 +834,7 @@ private[spark] class MesosClusterScheduler( } private def removeFromPendingRetryDrivers(subId: String): Boolean = { - val index = pendingRetryDrivers.indexWhere(_.submissionId.equals(subId)) + val index = pendingRetryDrivers.indexWhere(_.submissionId == subId) if (index != -1) { pendingRetryDrivers.remove(index) pendingRetryDriversState.expunge(subId) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala index e534b9d7e3ed..082d4bcfdf83 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala @@ -21,7 +21,7 @@ import java.util.{Collection, Collections, Date} import scala.collection.JavaConverters._ -import org.apache.mesos.Protos.{Environment, Secret, TaskState => MesosTaskState, _} +import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} import org.apache.mesos.Protos.Value.{Scalar, Type} import org.apache.mesos.SchedulerDriver import org.mockito.{ArgumentCaptor, Matchers} @@ -146,14 +146,14 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi assert(scheduler.getResource(resources, "cpus") == 1.5) assert(scheduler.getResource(resources, "mem") == 1200) val resourcesSeq: Seq[Resource] = resources.asScala - val cpus = resourcesSeq.filter(_.getName.equals("cpus")).toList + val cpus = resourcesSeq.filter(_.getName == "cpus").toList assert(cpus.size == 2) - assert(cpus.exists(_.getRole().equals("role2"))) - assert(cpus.exists(_.getRole().equals("*"))) - val mem = resourcesSeq.filter(_.getName.equals("mem")).toList + assert(cpus.exists(_.getRole() == "role2")) + assert(cpus.exists(_.getRole() == "*")) + val mem = resourcesSeq.filter(_.getName == "mem").toList assert(mem.size == 2) - assert(mem.exists(_.getRole().equals("role2"))) - assert(mem.exists(_.getRole().equals("*"))) + assert(mem.exists(_.getRole() == "role2")) + assert(mem.exists(_.getRole() == "*")) verify(driver, times(1)).launchTasks( Matchers.eq(Collections.singleton(offer.getId)), diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala index 31f84310485a..1ead4b1ed7c7 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosFineGrainedSchedulerBackendSuite.scala @@ -106,7 +106,7 @@ class MesosFineGrainedSchedulerBackendSuite // uri is null. val (executorInfo, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id") val executorResources = executorInfo.getResourcesList - val cpus = executorResources.asScala.find(_.getName.equals("cpus")).get.getScalar.getValue + val cpus = executorResources.asScala.find(_.getName == "cpus").get.getScalar.getValue assert(cpus === mesosExecutorCores) } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 7fa597167f3f..26013a109c42 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -191,7 +191,7 @@ class ClientSuite extends SparkFunSuite with Matchers { appContext.getQueue should be ("staging-queue") appContext.getAMContainerSpec should be (containerLaunchContext) appContext.getApplicationType should be ("SPARK") - appContext.getClass.getMethods.filter(_.getName.equals("getApplicationTags")).foreach{ method => + appContext.getClass.getMethods.filter(_.getName == "getApplicationTags").foreach { method => val tags = method.invoke(appContext).asInstanceOf[java.util.Set[String]] tags should contain allOf ("tag1", "dup", "tag2", "multi word") tags.asScala.count(_.nonEmpty) should be (4) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala index f1ce7543ffdc..d395bba105a7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala @@ -147,7 +147,7 @@ class PropagateEmptyRelationSuite extends PlanTest { .where(false) .select('a) .where('a > 1) - .where('a != 200) + .where('a =!= 200) .orderBy('a.asc) val optimized = Optimize.execute(query.analyze) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index 8f75c14192c9..755c8897cada 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -114,7 +114,7 @@ class UnsafeArraySuite extends SparkFunSuite { assert(unsafeDate.isInstanceOf[UnsafeArrayData]) assert(unsafeDate.numElements == dateArray.length) dateArray.zipWithIndex.map { case (e, i) => - assert(unsafeDate.get(i, DateType) == e) + assert(unsafeDate.get(i, DateType).asInstanceOf[Int] == e) } val unsafeTimestamp = ExpressionEncoder[Array[Long]].resolveAndBind(). @@ -122,7 +122,7 @@ class UnsafeArraySuite extends SparkFunSuite { assert(unsafeTimestamp.isInstanceOf[UnsafeArrayData]) assert(unsafeTimestamp.numElements == timestampArray.length) timestampArray.zipWithIndex.map { case (e, i) => - assert(unsafeTimestamp.get(i, TimestampType) == e) + assert(unsafeTimestamp.get(i, TimestampType).asInstanceOf[Long] == e) } Seq(decimalArray4_1, decimalArray20_20).map { decimalArray => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index ca8fbc991a3a..4e593ff046a5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -611,7 +611,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ).toDF("id", "stringData") val sampleDF = df.sample(false, 0.7, 50) // After sampling, sampleDF doesn't contain id=1. - assert(!sampleDF.select("id").collect.contains(1)) + assert(!sampleDF.select("id").as[Int].collect.contains(1)) // simpleUdf should not encounter id=1. checkAnswer(sampleDF.select(simpleUdf($"id")), List.fill(sampleDF.count.toInt)(Row(1))) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 7eefedb8ff5b..528a4d0ca800 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -427,7 +427,7 @@ class ParquetSchemaSuite extends ParquetSchemaTest { assert(errMsg.startsWith("Parquet column cannot be converted in file")) val file = errMsg.substring("Parquet column cannot be converted in file ".length, errMsg.indexOf(". ")) - val col = spark.read.parquet(file).schema.fields.filter(_.name.equals("a")) + val col = spark.read.parquet(file).schema.fields.filter(_.name == "a") assert(col.length == 1) if (col(0).dataType == StringType) { assert(errMsg.contains("Column: [a], Expected: int, Found: BINARY")) From 3a6ef8b7e2d17fe22458bfd249f45b5a5ce269ec Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 11 Sep 2018 14:52:58 -0500 Subject: [PATCH 029/879] Revert "[SPARK-23820][CORE] Enable use of long form of callsite in logs" This reverts commit e58dadb77ed6cac3e1b2a037a6449e5a6e7f2cec. --- .../org/apache/spark/internal/config/package.scala | 3 --- .../main/scala/org/apache/spark/storage/RDDInfo.scala | 10 +--------- 2 files changed, 1 insertion(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 319e664a1967..c74040cbd0db 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -72,9 +72,6 @@ package object config { private[spark] val EVENT_LOG_OVERWRITE = ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) - private[spark] val EVENT_LOG_CALLSITE_FORM = - ConfigBuilder("spark.eventLog.callsite").stringConf.createWithDefault("short") - private[spark] val EXECUTOR_CLASS_PATH = ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.createOptional diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 64e5c8b1c4bb..e5abbf745cc4 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -17,9 +17,7 @@ package org.apache.spark.storage -import org.apache.spark.SparkEnv import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.config._ import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.util.Utils @@ -55,16 +53,10 @@ class RDDInfo( } private[spark] object RDDInfo { - private lazy val callsiteForm = SparkEnv.get.conf.get(EVENT_LOG_CALLSITE_FORM) - def fromRdd(rdd: RDD[_]): RDDInfo = { val rddName = Option(rdd.name).getOrElse(Utils.getFormattedClassName(rdd)) val parentIds = rdd.dependencies.map(_.rdd.id) - val callSite = callsiteForm match { - case "short" => rdd.creationSite.shortForm - case "long" => rdd.creationSite.longForm - } new RDDInfo(rdd.id, rddName, rdd.partitions.length, - rdd.getStorageLevel, parentIds, callSite, rdd.scope) + rdd.getStorageLevel, parentIds, rdd.creationSite.shortForm, rdd.scope) } } From 0dbf1450f7965c27ce9329c7dad351ff8b8072dc Mon Sep 17 00:00:00 2001 From: Mukul Murthy Date: Tue, 11 Sep 2018 15:53:15 -0700 Subject: [PATCH 030/879] [SPARK-25399][SS] Continuous processing state should not affect microbatch execution jobs ## What changes were proposed in this pull request? The leftover state from running a continuous processing streaming job should not affect later microbatch execution jobs. If a continuous processing job runs and the same thread gets reused for a microbatch execution job in the same environment, the microbatch job could get wrong answers because it can attempt to load the wrong version of the state. ## How was this patch tested? New and existing unit tests Closes #22386 from mukulmurthy/25399-streamthread. Authored-by: Mukul Murthy Signed-off-by: Tathagata Das (cherry picked from commit 9f5c5b4cca7d4eaa30a3f8adb4cb1eebe3f77c7a) Signed-off-by: Tathagata Das --- .../streaming/MicroBatchExecution.scala | 2 ++ .../execution/streaming/StreamExecution.scala | 1 + .../continuous/ContinuousExecution.scala | 2 ++ .../streaming/state/StateStoreRDD.scala | 12 +++++-- .../spark/sql/streaming/StreamSuite.scala | 33 +++++++++++++++++-- 5 files changed, 45 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index b1cafd67820c..2cac86599ef1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -511,6 +511,8 @@ class MicroBatchExecution( sparkSessionToRunBatch.sparkContext.setLocalProperty( MicroBatchExecution.BATCH_ID_KEY, currentBatchId.toString) + sparkSessionToRunBatch.sparkContext.setLocalProperty( + StreamExecution.IS_CONTINUOUS_PROCESSING, false.toString) reportTimeTaken("queryPlanning") { lastExecution = new IncrementalExecution( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index a39bb715c991..f6c60c1c9212 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -529,6 +529,7 @@ abstract class StreamExecution( object StreamExecution { val QUERY_ID_KEY = "sql.streaming.queryId" + val IS_CONTINUOUS_PROCESSING = "__is_continuous_processing" def isInterruptionException(e: Throwable): Boolean = e match { // InterruptedIOException - thrown when an I/O operation is interrupted diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index 4ddebb33b79d..ccca72667a21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -209,6 +209,8 @@ class ContinuousExecution( scan.readSupport.asInstanceOf[ContinuousReadSupport] -> scan.scanConfig }.head + sparkSessionForQuery.sparkContext.setLocalProperty( + StreamExecution.IS_CONTINUOUS_PROCESSING, true.toString) sparkSessionForQuery.sparkContext.setLocalProperty( ContinuousExecution.START_EPOCH_KEY, currentBatchId.toString) // Add another random ID on top of the run ID, to distinguish epoch coordinators across diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala index 3f11b8f79943..4a69a48fed75 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStoreRDD.scala @@ -23,6 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, TaskContext} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.streaming.continuous.EpochTracker import org.apache.spark.sql.internal.SessionState import org.apache.spark.sql.types.StructType @@ -74,9 +75,14 @@ class StateStoreRDD[T: ClassTag, U: ClassTag]( // If we're in continuous processing mode, we should get the store version for the current // epoch rather than the one at planning time. - val currentVersion = EpochTracker.getCurrentEpoch match { - case None => storeVersion - case Some(value) => value + val isContinuous = Option(ctxt.getLocalProperty(StreamExecution.IS_CONTINUOUS_PROCESSING)) + .map(_.toBoolean).getOrElse(false) + val currentVersion = if (isContinuous) { + val epoch = EpochTracker.getCurrentEpoch + assert(epoch.isDefined, "Current epoch must be defined for continuous processing streams.") + epoch.get + } else { + storeVersion } store = StateStore.get( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index bf509b1976ed..f55ddb5419d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -29,13 +29,14 @@ import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{SparkConf, SparkContext, TaskContext} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.Range import org.apache.spark.sql.catalyst.streaming.InternalOutputModes import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProvider} import org.apache.spark.sql.functions._ @@ -788,7 +789,7 @@ class StreamSuite extends StreamTest { val query = input .toDS() .map { i => - while (!org.apache.spark.TaskContext.get().isInterrupted()) { + while (!TaskContext.get().isInterrupted()) { // keep looping till interrupted by query.stop() Thread.sleep(100) } @@ -1029,6 +1030,34 @@ class StreamSuite extends StreamTest { false)) } + test("is_continuous_processing property should be false for microbatch processing") { + val input = MemoryStream[Int] + val df = input.toDS() + .map(i => TaskContext.get().getLocalProperty(StreamExecution.IS_CONTINUOUS_PROCESSING)) + testStream(df) ( + AddData(input, 1), + CheckAnswer("false") + ) + } + + test("is_continuous_processing property should be true for continuous processing") { + val input = ContinuousMemoryStream[Int] + val stream = input.toDS() + .map(i => TaskContext.get().getLocalProperty(StreamExecution.IS_CONTINUOUS_PROCESSING)) + .writeStream.format("memory") + .queryName("output") + .trigger(Trigger.Continuous("1 seconds")) + .start() + try { + input.addData(1) + stream.processAllAvailable() + } finally { + stream.stop() + } + + checkAnswer(spark.sql("select * from output"), Row("true")) + } + for (e <- Seq( new InterruptedException, new InterruptedIOException, From 40e4db0eb72be7640bd8b5b319ad4ba99c9dc846 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 12 Sep 2018 21:11:22 +0800 Subject: [PATCH 031/879] [SPARK-25402][SQL] Null handling in BooleanSimplification ## What changes were proposed in this pull request? This PR is to fix the null handling in BooleanSimplification. In the rule BooleanSimplification, there are two cases that do not properly handle null values. The optimization is not right if either side is null. This PR is to fix them. ## How was this patch tested? Added test cases Closes #22390 from gatorsmile/fixBooleanSimplification. Authored-by: gatorsmile Signed-off-by: Wenchen Fan (cherry picked from commit 79cc59718fdf7785bdc37a26bb8df4c6151114a6) Signed-off-by: Wenchen Fan --- .../sql/catalyst/optimizer/expressions.scala | 13 ++++-- .../BooleanSimplificationSuite.scala | 45 +++++++++++++++++-- .../org/apache/spark/sql/DataFrameSuite.scala | 10 +++++ 3 files changed, 60 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 5629b7289422..f8037588fa71 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -263,10 +263,15 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { case TrueLiteral Or _ => TrueLiteral case _ Or TrueLiteral => TrueLiteral - case a And b if Not(a).semanticEquals(b) => FalseLiteral - case a Or b if Not(a).semanticEquals(b) => TrueLiteral - case a And b if a.semanticEquals(Not(b)) => FalseLiteral - case a Or b if a.semanticEquals(Not(b)) => TrueLiteral + case a And b if Not(a).semanticEquals(b) => + If(IsNull(a), Literal.create(null, a.dataType), FalseLiteral) + case a And b if a.semanticEquals(Not(b)) => + If(IsNull(b), Literal.create(null, b.dataType), FalseLiteral) + + case a Or b if Not(a).semanticEquals(b) => + If(IsNull(a), Literal.create(null, a.dataType), TrueLiteral) + case a Or b if a.semanticEquals(Not(b)) => + If(IsNull(b), Literal.create(null, b.dataType), TrueLiteral) case a And b if a.semanticEquals(b) => a case a Or b if a.semanticEquals(b) => a diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 653c07f1835c..6cd1108eef33 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.BooleanType class BooleanSimplificationSuite extends PlanTest with PredicateHelper { @@ -37,6 +38,7 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { Batch("Constant Folding", FixedPoint(50), NullPropagation, ConstantFolding, + SimplifyConditionals, BooleanSimplification, PruneFilters) :: Nil } @@ -48,6 +50,14 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { testRelation.output, Seq(Row(1, 2, 3, "abc")) ) + val testNotNullableRelation = LocalRelation('a.int.notNull, 'b.int.notNull, 'c.int.notNull, + 'd.string.notNull, 'e.boolean.notNull, 'f.boolean.notNull, 'g.boolean.notNull, + 'h.boolean.notNull) + + val testNotNullableRelationWithData = LocalRelation.fromExternalRows( + testNotNullableRelation.output, Seq(Row(1, 2, 3, "abc")) + ) + private def checkCondition(input: Expression, expected: LogicalPlan): Unit = { val plan = testRelationWithData.where(input).analyze val actual = Optimize.execute(plan) @@ -61,6 +71,13 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { comparePlans(actual, correctAnswer) } + private def checkConditionInNotNullableRelation( + input: Expression, expected: LogicalPlan): Unit = { + val plan = testNotNullableRelationWithData.where(input).analyze + val actual = Optimize.execute(plan) + comparePlans(actual, expected) + } + test("a && a => a") { checkCondition(Literal(1) < 'a && Literal(1) < 'a, Literal(1) < 'a) checkCondition(Literal(1) < 'a && Literal(1) < 'a && Literal(1) < 'a, Literal(1) < 'a) @@ -174,10 +191,30 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { } test("Complementation Laws") { - checkCondition('a && !'a, testRelation) - checkCondition(!'a && 'a, testRelation) + checkConditionInNotNullableRelation('e && !'e, testNotNullableRelation) + checkConditionInNotNullableRelation(!'e && 'e, testNotNullableRelation) + + checkConditionInNotNullableRelation('e || !'e, testNotNullableRelationWithData) + checkConditionInNotNullableRelation(!'e || 'e, testNotNullableRelationWithData) + } + + test("Complementation Laws - null handling") { + checkCondition('e && !'e, + testRelationWithData.where(If('e.isNull, Literal.create(null, BooleanType), false)).analyze) + checkCondition(!'e && 'e, + testRelationWithData.where(If('e.isNull, Literal.create(null, BooleanType), false)).analyze) + + checkCondition('e || !'e, + testRelationWithData.where(If('e.isNull, Literal.create(null, BooleanType), true)).analyze) + checkCondition(!'e || 'e, + testRelationWithData.where(If('e.isNull, Literal.create(null, BooleanType), true)).analyze) + } + + test("Complementation Laws - negative case") { + checkCondition('e && !'f, testRelationWithData.where('e && !'f).analyze) + checkCondition(!'f && 'e, testRelationWithData.where(!'f && 'e).analyze) - checkCondition('a || !'a, testRelationWithData) - checkCondition(!'a || 'a, testRelationWithData) + checkCondition('e || !'f, testRelationWithData.where('e || !'f).analyze) + checkCondition(!'f || 'e, testRelationWithData.where(!'f || 'e).analyze) } } 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 435b887cb3c7..279b7b8d49f5 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 @@ -2569,4 +2569,14 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { check(lit(2).cast("int"), $"c" === 2, Seq(Row(1, 1, 2, 0), Row(1, 1, 2, 1))) check(lit(2).cast("int"), $"c" =!= 2, Seq()) } + + test("SPARK-25402 Null handling in BooleanSimplification") { + val schema = StructType.fromDDL("a boolean, b int") + val rows = Seq(Row(null, 1)) + + val rdd = sparkContext.parallelize(rows) + val df = spark.createDataFrame(rdd, schema) + + checkAnswer(df.where("(NOT a) OR a"), Seq.empty) + } } From 071babbab5a49b7106d61b0c9a18672bd67e1786 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 12 Sep 2018 22:54:05 +0800 Subject: [PATCH 032/879] [SPARK-25352][SQL] Perform ordered global limit when limit number is bigger than topKSortFallbackThreshold ## What changes were proposed in this pull request? We have optimization on global limit to evenly distribute limit rows across all partitions. This optimization doesn't work for ordered results. For a query ending with sort + limit, in most cases it is performed by `TakeOrderedAndProjectExec`. But if limit number is bigger than `SQLConf.TOP_K_SORT_FALLBACK_THRESHOLD`, global limit will be used. At this moment, we need to do ordered global limit. ## How was this patch tested? Unit tests. Closes #22344 from viirya/SPARK-25352. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan (cherry picked from commit 2f422398b524eacc89ab58e423bb134ae3ca3941) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/SparkStrategies.scala | 44 ++++++--- .../apache/spark/sql/execution/limit.scala | 7 +- .../org/apache/spark/sql/DataFrameSuite.scala | 22 ++++- .../spark/sql/execution/LimitSuite.scala | 81 ++++++++++++++++ .../TakeOrderedAndProjectSuite.scala | 94 +++++++++++-------- 5 files changed, 192 insertions(+), 56 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/LimitSuite.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 dbc6db62bd82..7c8ce316f964 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 @@ -68,22 +68,42 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object SpecialLimits extends Strategy { override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case ReturnAnswer(rootPlan) => rootPlan match { - case Limit(IntegerLiteral(limit), Sort(order, true, child)) - if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil - case Limit(IntegerLiteral(limit), Project(projectList, Sort(order, true, child))) - if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, order, projectList, planLater(child)) :: Nil + case Limit(IntegerLiteral(limit), s@Sort(order, true, child)) => + if (limit < conf.topKSortFallbackThreshold) { + TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil + } else { + GlobalLimitExec(limit, + LocalLimitExec(limit, planLater(s)), + orderedLimit = true) :: Nil + } + case Limit(IntegerLiteral(limit), p@Project(projectList, Sort(order, true, child))) => + if (limit < conf.topKSortFallbackThreshold) { + TakeOrderedAndProjectExec(limit, order, projectList, planLater(child)) :: Nil + } else { + GlobalLimitExec(limit, + LocalLimitExec(limit, planLater(p)), + orderedLimit = true) :: Nil + } case Limit(IntegerLiteral(limit), child) => CollectLimitExec(limit, planLater(child)) :: Nil case other => planLater(other) :: Nil } - case Limit(IntegerLiteral(limit), Sort(order, true, child)) - if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil - case Limit(IntegerLiteral(limit), Project(projectList, Sort(order, true, child))) - if limit < conf.topKSortFallbackThreshold => - TakeOrderedAndProjectExec(limit, order, projectList, planLater(child)) :: Nil + case Limit(IntegerLiteral(limit), s@Sort(order, true, child)) => + if (limit < conf.topKSortFallbackThreshold) { + TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil + } else { + GlobalLimitExec(limit, + LocalLimitExec(limit, planLater(s)), + orderedLimit = true) :: Nil + } + case Limit(IntegerLiteral(limit), p@Project(projectList, Sort(order, true, child))) => + if (limit < conf.topKSortFallbackThreshold) { + TakeOrderedAndProjectExec(limit, order, projectList, planLater(child)) :: Nil + } else { + GlobalLimitExec(limit, + LocalLimitExec(limit, planLater(p)), + orderedLimit = true) :: Nil + } case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index fb46970e38f3..1a09632f93ca 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -98,7 +98,8 @@ case class LocalLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode wi /** * Take the `limit` elements of the child output. */ -case class GlobalLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode { +case class GlobalLimitExec(limit: Int, child: SparkPlan, + orderedLimit: Boolean = false) extends UnaryExecNode { override def output: Seq[Attribute] = child.output @@ -126,7 +127,9 @@ case class GlobalLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode { // When enabled, Spark goes to take rows at each partition repeatedly until reaching // limit number. When disabled, Spark takes all rows at first partition, then rows // at second partition ..., until reaching limit number. - val flatGlobalLimit = sqlContext.conf.limitFlatGlobalLimit + // The optimization is disabled when it is needed to keep the original order of rows + // before global sort, e.g., select * from table order by col limit 10. + val flatGlobalLimit = sqlContext.conf.limitFlatGlobalLimit && !orderedLimit val shuffled = new ShuffledRowRDD(shuffleDependency) 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 279b7b8d49f5..f001b138f4b8 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 @@ -31,7 +31,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.Uuid import org.apache.spark.sql.catalyst.plans.logical.{Filter, OneRowRelation, Union} -import org.apache.spark.sql.execution.{FilterExec, QueryExecution, WholeStageCodegenExec} +import org.apache.spark.sql.execution.{FilterExec, QueryExecution, TakeOrderedAndProjectExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.functions._ @@ -2552,6 +2552,26 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } } + test("SPARK-25352: Ordered global limit when more than topKSortFallbackThreshold ") { + withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "true") { + val baseDf = spark.range(1000).toDF.repartition(3).sort("id") + + withSQLConf(SQLConf.TOP_K_SORT_FALLBACK_THRESHOLD.key -> "100") { + val expected = baseDf.limit(99) + val takeOrderedNode1 = expected.queryExecution.executedPlan + .find(_.isInstanceOf[TakeOrderedAndProjectExec]) + assert(takeOrderedNode1.isDefined) + + val result = baseDf.limit(100) + val takeOrderedNode2 = result.queryExecution.executedPlan + .find(_.isInstanceOf[TakeOrderedAndProjectExec]) + assert(takeOrderedNode2.isEmpty) + + checkAnswer(expected, result.collect().take(99)) + } + } + } + test("SPARK-25368 Incorrect predicate pushdown returns wrong result") { def check(newCol: Column, filter: Column, result: Seq[Row]): Unit = { val df1 = spark.createDataFrame(Seq( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/LimitSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/LimitSuite.scala new file mode 100644 index 000000000000..a7840a5fcfae --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/LimitSuite.scala @@ -0,0 +1,81 @@ +/* + * 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 + +import scala.util.Random + +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + + +class LimitSuite extends SparkPlanTest with SharedSQLContext { + + private var rand: Random = _ + private var seed: Long = 0 + + protected override def beforeAll(): Unit = { + super.beforeAll() + seed = System.currentTimeMillis() + rand = new Random(seed) + } + + test("Produce ordered global limit if more than topKSortFallbackThreshold") { + withSQLConf(SQLConf.TOP_K_SORT_FALLBACK_THRESHOLD.key -> "100") { + val df = LimitTest.generateRandomInputData(spark, rand).sort("a") + + val globalLimit = df.limit(99).queryExecution.executedPlan.collect { + case g: GlobalLimitExec => g + } + assert(globalLimit.size == 0) + + val topKSort = df.limit(99).queryExecution.executedPlan.collect { + case t: TakeOrderedAndProjectExec => t + } + assert(topKSort.size == 1) + + val orderedGlobalLimit = df.limit(100).queryExecution.executedPlan.collect { + case g: GlobalLimitExec => g + } + assert(orderedGlobalLimit.size == 1 && orderedGlobalLimit(0).orderedLimit == true) + } + } + + test("Ordered global limit") { + val baseDf = LimitTest.generateRandomInputData(spark, rand) + .select("a").repartition(3).sort("a") + + withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "true") { + val orderedGlobalLimit = GlobalLimitExec(3, baseDf.queryExecution.sparkPlan, + orderedLimit = true) + val orderedGlobalLimitResult = SparkPlanTest.executePlan(orderedGlobalLimit, spark.sqlContext) + .map(_.getInt(0)) + + val globalLimit = GlobalLimitExec(3, baseDf.queryExecution.sparkPlan, orderedLimit = false) + val globalLimitResult = SparkPlanTest.executePlan(globalLimit, spark.sqlContext) + .map(_.getInt(0)) + + // Global limit without order takes values at each partition sequentially. + // After global sort, the values in second partition must be larger than the values + // in first partition. + assert(orderedGlobalLimitResult(0) == globalLimitResult(0)) + assert(orderedGlobalLimitResult(1) < globalLimitResult(1)) + assert(orderedGlobalLimitResult(2) < globalLimitResult(2)) + } + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala index f076959dfdf7..9322204063af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution import scala.util.Random -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{DataFrame, Row, SparkSession} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.internal.SQLConf @@ -32,28 +32,10 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { private var rand: Random = _ private var seed: Long = 0 - private val originalLimitFlatGlobalLimit = SQLConf.get.getConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT) - protected override def beforeAll(): Unit = { super.beforeAll() seed = System.currentTimeMillis() rand = new Random(seed) - - // Disable the optimization to make Sort-Limit match `TakeOrderedAndProject` semantics. - SQLConf.get.setConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT, false) - } - - protected override def afterAll() = { - SQLConf.get.setConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT, originalLimitFlatGlobalLimit) - super.afterAll() - } - - private def generateRandomInputData(): DataFrame = { - val schema = new StructType() - .add("a", IntegerType, nullable = false) - .add("b", IntegerType, nullable = false) - val inputData = Seq.fill(10000)(Row(rand.nextInt(), rand.nextInt())) - spark.createDataFrame(sparkContext.parallelize(Random.shuffle(inputData), 10), schema) } /** @@ -66,32 +48,62 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { val sortOrder = 'a.desc :: 'b.desc :: Nil test("TakeOrderedAndProject.doExecute without project") { - withClue(s"seed = $seed") { - checkThatPlansAgree( - generateRandomInputData(), - input => - noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, input.output, input)), - input => - GlobalLimitExec(limit, - LocalLimitExec(limit, - SortExec(sortOrder, true, input))), - sortAnswers = false) + withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "false") { + withClue(s"seed = $seed") { + checkThatPlansAgree( + LimitTest.generateRandomInputData(spark, rand), + input => + noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, input.output, input)), + input => + GlobalLimitExec(limit, + LocalLimitExec(limit, + SortExec(sortOrder, true, input))), + sortAnswers = false) + } } } test("TakeOrderedAndProject.doExecute with project") { - withClue(s"seed = $seed") { - checkThatPlansAgree( - generateRandomInputData(), - input => - noOpFilter( - TakeOrderedAndProjectExec(limit, sortOrder, Seq(input.output.last), input)), - input => - GlobalLimitExec(limit, - LocalLimitExec(limit, - ProjectExec(Seq(input.output.last), - SortExec(sortOrder, true, input)))), - sortAnswers = false) + withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "false") { + withClue(s"seed = $seed") { + checkThatPlansAgree( + LimitTest.generateRandomInputData(spark, rand), + input => + noOpFilter( + TakeOrderedAndProjectExec(limit, sortOrder, Seq(input.output.last), input)), + input => + GlobalLimitExec(limit, + LocalLimitExec(limit, + ProjectExec(Seq(input.output.last), + SortExec(sortOrder, true, input)))), + sortAnswers = false) + } } } + + test("TakeOrderedAndProject.doExecute equals to ordered global limit") { + withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "true") { + withClue(s"seed = $seed") { + checkThatPlansAgree( + LimitTest.generateRandomInputData(spark, rand), + input => + noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, input.output, input)), + input => + GlobalLimitExec(limit, + LocalLimitExec(limit, + SortExec(sortOrder, true, input)), orderedLimit = true), + sortAnswers = false) + } + } + } +} + +object LimitTest { + def generateRandomInputData(spark: SparkSession, rand: Random): DataFrame = { + val schema = new StructType() + .add("a", IntegerType, nullable = false) + .add("b", IntegerType, nullable = false) + val inputData = Seq.fill(10000)(Row(rand.nextInt(), rand.nextInt())) + spark.createDataFrame(spark.sparkContext.parallelize(Random.shuffle(inputData), 10), schema) + } } From 4c1428fa2b29c371458977427561d2b4bb9daa5b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 12 Sep 2018 17:43:40 +0000 Subject: [PATCH 033/879] [SPARK-25363][SQL] Fix schema pruning in where clause by ignoring unnecessary root fields ## What changes were proposed in this pull request? Schema pruning doesn't work if nested column is used in where clause. For example, ``` sql("select name.first from contacts where name.first = 'David'") == Physical Plan == *(1) Project [name#19.first AS first#40] +- *(1) Filter (isnotnull(name#19) && (name#19.first = David)) +- *(1) FileScan parquet [name#19] Batched: false, Format: Parquet, PartitionFilters: [], PushedFilters: [IsNotNull(name)], ReadSchema: struct> ``` In above query plan, the scan node reads the entire schema of `name` column. This issue is reported by: https://github.com/apache/spark/pull/21320#issuecomment-419290197 The cause is that we infer a root field from expression `IsNotNull(name)`. However, for such expression, we don't really use the nested fields of this root field, so we can ignore the unnecessary nested fields. ## How was this patch tested? Unit tests. Closes #22357 from viirya/SPARK-25363. Authored-by: Liang-Chi Hsieh Signed-off-by: DB Tsai (cherry picked from commit 3030b82c89d3e45a2e361c469fbc667a1e43b854) Signed-off-by: DB Tsai --- .../parquet/ParquetSchemaPruning.scala | 34 ++++++-- .../parquet/ParquetSchemaPruningSuite.scala | 77 ++++++++++++++++--- 2 files changed, 96 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala index 6a46b5f8edc5..91080b15727d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.datasources.parquet -import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeReference, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} import org.apache.spark.sql.catalyst.rules.Rule @@ -110,7 +110,17 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { val projectionRootFields = projects.flatMap(getRootFields) val filterRootFields = filters.flatMap(getRootFields) - (projectionRootFields ++ filterRootFields).distinct + // Kind of expressions don't need to access any fields of a root fields, e.g., `IsNotNull`. + // For them, if there are any nested fields accessed in the query, we don't need to add root + // field access of above expressions. + // For example, for a query `SELECT name.first FROM contacts WHERE name IS NOT NULL`, + // we don't need to read nested fields of `name` struct other than `first` field. + val (rootFields, optRootFields) = (projectionRootFields ++ filterRootFields) + .distinct.partition(_.contentAccessed) + + optRootFields.filter { opt => + !rootFields.exists(_.field.name == opt.field.name) + } ++ rootFields } /** @@ -156,7 +166,7 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { // in the resulting schema may differ from their ordering in the logical relation's // original schema val mergedSchema = requestedRootFields - .map { case RootField(field, _) => StructType(Array(field)) } + .map { case root: RootField => StructType(Array(root.field)) } .reduceLeft(_ merge _) val dataSchemaFieldNames = fileDataSchema.fieldNames.toSet val mergedDataSchema = @@ -199,6 +209,15 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { case att: Attribute => RootField(StructField(att.name, att.dataType, att.nullable), derivedFromAtt = true) :: Nil case SelectedField(field) => RootField(field, derivedFromAtt = false) :: Nil + // Root field accesses by `IsNotNull` and `IsNull` are special cases as the expressions + // don't actually use any nested fields. These root field accesses might be excluded later + // if there are any nested fields accesses in the query plan. + case IsNotNull(SelectedField(field)) => + RootField(field, derivedFromAtt = false, contentAccessed = false) :: Nil + case IsNull(SelectedField(field)) => + RootField(field, derivedFromAtt = false, contentAccessed = false) :: Nil + case IsNotNull(_: Attribute) | IsNull(_: Attribute) => + expr.children.flatMap(getRootFields).map(_.copy(contentAccessed = false)) case _ => expr.children.flatMap(getRootFields) } @@ -250,8 +269,11 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { } /** - * A "root" schema field (aka top-level, no-parent) and whether it was derived from - * an attribute or had a proper child. + * This represents a "root" schema field (aka top-level, no-parent). `field` is the + * `StructField` for field name and datatype. `derivedFromAtt` indicates whether it + * was derived from an attribute or had a proper child. `contentAccessed` means whether + * it was accessed with its content by the expressions refer it. */ - private case class RootField(field: StructField, derivedFromAtt: Boolean) + private case class RootField(field: StructField, derivedFromAtt: Boolean, + contentAccessed: Boolean = true) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index eb99654fa78f..7b132af4f691 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -35,22 +35,29 @@ class ParquetSchemaPruningSuite with SchemaPruningTest with SharedSQLContext { case class FullName(first: String, middle: String, last: String) + case class Company(name: String, address: String) + case class Employer(id: Int, company: Company) case class Contact( id: Int, name: FullName, address: String, pets: Int, friends: Array[FullName] = Array.empty, - relatives: Map[String, FullName] = Map.empty) + relatives: Map[String, FullName] = Map.empty, + employer: Employer = null) val janeDoe = FullName("Jane", "X.", "Doe") val johnDoe = FullName("John", "Y.", "Doe") val susanSmith = FullName("Susan", "Z.", "Smith") + val employer = Employer(0, Company("abc", "123 Business Street")) + val employerWithNullCompany = Employer(1, null) + private val contacts = Contact(0, janeDoe, "123 Main Street", 1, friends = Array(susanSmith), - relatives = Map("brother" -> johnDoe)) :: - Contact(1, johnDoe, "321 Wall Street", 3, relatives = Map("sister" -> janeDoe)) :: Nil + relatives = Map("brother" -> johnDoe), employer = employer) :: + Contact(1, johnDoe, "321 Wall Street", 3, relatives = Map("sister" -> janeDoe), + employer = employerWithNullCompany) :: Nil case class Name(first: String, last: String) case class BriefContact(id: Int, name: Name, address: String) @@ -66,13 +73,14 @@ class ParquetSchemaPruningSuite pets: Int, friends: Array[FullName] = Array(), relatives: Map[String, FullName] = Map(), + employer: Employer = null, p: Int) case class BriefContactWithDataPartitionColumn(id: Int, name: Name, address: String, p: Int) private val contactsWithDataPartitionColumn = - contacts.map { case Contact(id, name, address, pets, friends, relatives) => - ContactWithDataPartitionColumn(id, name, address, pets, friends, relatives, 1) } + contacts.map { case Contact(id, name, address, pets, friends, relatives, employer) => + ContactWithDataPartitionColumn(id, name, address, pets, friends, relatives, employer, 1) } private val briefContactsWithDataPartitionColumn = briefContacts.map { case BriefContact(id, name, address) => BriefContactWithDataPartitionColumn(id, name, address, 2) } @@ -155,6 +163,60 @@ class ParquetSchemaPruningSuite Row(null) :: Row(null) :: Nil) } + testSchemaPruning("select a single complex field and in where clause") { + val query1 = sql("select name.first from contacts where name.first = 'Jane'") + checkScan(query1, "struct>") + checkAnswer(query1, Row("Jane") :: Nil) + + val query2 = sql("select name.first, name.last from contacts where name.first = 'Jane'") + checkScan(query2, "struct>") + checkAnswer(query2, Row("Jane", "Doe") :: Nil) + + val query3 = sql("select name.first from contacts " + + "where employer.company.name = 'abc' and p = 1") + checkScan(query3, "struct," + + "employer:struct>>") + checkAnswer(query3, Row("Jane") :: Nil) + + val query4 = sql("select name.first, employer.company.name from contacts " + + "where employer.company is not null and p = 1") + checkScan(query4, "struct," + + "employer:struct>>") + checkAnswer(query4, Row("Jane", "abc") :: Nil) + } + + testSchemaPruning("select nullable complex field and having is not null predicate") { + val query = sql("select employer.company from contacts " + + "where employer is not null and p = 1") + checkScan(query, "struct>>") + checkAnswer(query, Row(Row("abc", "123 Business Street")) :: Row(null) :: Nil) + } + + testSchemaPruning("select a single complex field and is null expression in project") { + val query = sql("select name.first, address is not null from contacts") + checkScan(query, "struct,address:string>") + checkAnswer(query.orderBy("id"), + Row("Jane", true) :: Row("John", true) :: Row("Janet", true) :: Row("Jim", true) :: Nil) + } + + testSchemaPruning("select a single complex field array and in clause") { + val query = sql("select friends.middle from contacts where friends.first[0] = 'Susan'") + checkScan(query, + "struct>>") + checkAnswer(query.orderBy("id"), + Row(Array("Z.")) :: Nil) + } + + testSchemaPruning("select a single complex field from a map entry and in clause") { + val query = + sql("select relatives[\"brother\"].middle from contacts " + + "where relatives[\"brother\"].first = 'John'") + checkScan(query, + "struct>>") + checkAnswer(query.orderBy("id"), + Row("Y.") :: Nil) + } + private def testSchemaPruning(testName: String)(testThunk: => Unit) { withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { test(s"Spark vectorized reader - without partition data column - $testName") { @@ -238,10 +300,7 @@ class ParquetSchemaPruningSuite testMixedCasePruning("filter with different-case column names") { val query = sql("select id from mixedcase where Col2.b = 2") - // Pruning with filters is currently unsupported. As-is, the file reader will read the id column - // and the entire coL2 struct. Once pruning with filters has been implemented we can uncomment - // this line - // checkScan(query, "struct>") + checkScan(query, "struct>") checkAnswer(query.orderBy("id"), Row(1) :: Nil) } From 15d2e9d7d2f0d5ecefd69bdc3f8a149670b05e79 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 12 Sep 2018 11:25:24 -0700 Subject: [PATCH 034/879] [SPARK-24882][SQL] Revert [] improve data source v2 API from branch 2.4 ## What changes were proposed in this pull request? As discussed in the dev list, we don't want to include https://github.com/apache/spark/pull/22009 in Spark 2.4, as it needs data source v2 users to change the implementation intensitively, while they need to change again in next release. ## How was this patch tested? existing tests Author: Wenchen Fan Closes #22388 from cloud-fan/revert. --- ...port.scala => KafkaContinuousReader.scala} | 133 +++--- ...port.scala => KafkaMicroBatchReader.scala} | 103 ++-- .../sql/kafka010/KafkaSourceProvider.scala | 37 +- ...eSupport.scala => KafkaStreamWriter.scala} | 14 +- .../kafka010/KafkaContinuousSourceSuite.scala | 12 +- .../sql/kafka010/KafkaContinuousTest.scala | 8 +- .../kafka010/KafkaMicroBatchSourceSuite.scala | 35 +- .../sql/sources/v2/ContinuousReadSupport.java | 46 ++ .../v2/ContinuousReadSupportProvider.java | 70 --- .../spark/sql/sources/v2/DataSourceV2.java | 10 +- .../sql/sources/v2/MicroBatchReadSupport.java | 52 +++ .../v2/MicroBatchReadSupportProvider.java | 70 --- ...dSupportProvider.java => ReadSupport.java} | 42 +- .../sql/sources/v2/SessionConfigSupport.java | 7 +- .../sql/sources/v2/StreamWriteSupport.java | 52 +++ .../v2/StreamingWriteSupportProvider.java | 54 --- ...SupportProvider.java => WriteSupport.java} | 32 +- .../sources/v2/reader/BatchReadSupport.java | 51 -- ...der.java => ContinuousInputPartition.java} | 15 +- .../sources/v2/reader/DataSourceReader.java | 75 +++ .../sql/sources/v2/reader/InputPartition.java | 26 +- ...nReader.java => InputPartitionReader.java} | 18 +- .../v2/reader/PartitionReaderFactory.java | 66 --- .../sql/sources/v2/reader/ReadSupport.java | 50 -- .../sql/sources/v2/reader/ScanConfig.java | 45 -- .../sql/sources/v2/reader/Statistics.java | 2 +- .../v2/reader/SupportsPushDownFilters.java | 6 +- .../SupportsPushDownRequiredColumns.java | 8 +- .../v2/reader/SupportsReportPartitioning.java | 12 +- .../v2/reader/SupportsReportStatistics.java | 14 +- .../v2/reader/SupportsScanColumnarBatch.java | 53 +++ .../partitioning/ClusteredDistribution.java | 4 +- .../v2/reader/partitioning/Distribution.java | 6 +- .../v2/reader/partitioning/Partitioning.java | 5 +- ...va => ContinuousInputPartitionReader.java} | 23 +- .../ContinuousPartitionReaderFactory.java | 40 -- .../streaming/ContinuousReadSupport.java | 77 --- .../v2/reader/streaming/ContinuousReader.java | 79 ++++ .../streaming/MicroBatchReadSupport.java | 60 --- .../v2/reader/streaming/MicroBatchReader.java | 75 +++ .../sources/v2/reader/streaming/Offset.java | 4 +- .../streaming/StreamingReadSupport.java | 49 -- ...riteSupport.java => DataSourceWriter.java} | 25 +- .../sql/sources/v2/writer/DataWriter.java | 16 +- .../sources/v2/writer/DataWriterFactory.java | 23 +- .../v2/writer/WriterCommitMessage.java | 9 +- ...ingWriteSupport.java => StreamWriter.java} | 32 +- .../streaming/StreamingDataWriterFactory.java | 59 --- .../apache/spark/sql/DataFrameReader.scala | 4 +- .../apache/spark/sql/DataFrameWriter.scala | 8 +- .../datasources/v2/DataSourceRDD.scala | 44 +- .../datasources/v2/DataSourceV2Relation.scala | 72 ++- .../datasources/v2/DataSourceV2ScanExec.scala | 65 +-- .../datasources/v2/DataSourceV2Strategy.scala | 49 +- .../datasources/v2/DataSourceV2Utils.scala | 9 - .../v2/WriteToDataSourceV2Exec.scala | 40 +- .../streaming/MicroBatchExecution.scala | 91 ++-- .../streaming/ProgressReporter.scala | 8 +- .../SimpleStreamingScanConfigBuilder.scala | 40 -- .../streaming/StreamingRelation.scala | 6 +- .../sql/execution/streaming/console.scala | 14 +- .../continuous/ContinuousDataSourceRDD.scala | 37 +- .../continuous/ContinuousExecution.scala | 51 +- .../ContinuousQueuedDataReader.scala | 29 +- .../ContinuousRateStreamSource.scala | 60 +-- .../ContinuousTextSocketSource.scala | 72 ++- .../continuous/ContinuousWriteRDD.scala | 7 +- .../continuous/EpochCoordinator.scala | 18 +- .../WriteToContinuousDataSource.scala | 4 +- .../WriteToContinuousDataSourceExec.scala | 10 +- .../sql/execution/streaming/memory.scala | 51 +- ...WriteSupport.scala => ConsoleWriter.scala} | 11 +- .../sources/ContinuousMemoryStream.scala | 76 +-- ...ider.scala => ForeachWriterProvider.scala} | 31 +- .../sources/MicroBatchWritSupport.scala | 51 -- ...adSupport.scala => MicroBatchWriter.scala} | 22 +- .../sources/PackedRowWriterFactory.scala | 9 +- ...scala => RateStreamMicroBatchReader.scala} | 79 ++-- .../sources/RateStreamProvider.scala | 27 +- .../streaming/sources/memoryV2.scala | 35 +- .../execution/streaming/sources/socket.scala | 114 +++-- .../sql/streaming/DataStreamReader.scala | 52 +-- .../sql/streaming/DataStreamWriter.scala | 9 +- .../sql/streaming/StreamingQueryManager.scala | 4 +- .../sources/v2/JavaAdvancedDataSourceV2.java | 147 +++--- .../sql/sources/v2/JavaBatchDataSourceV2.java | 114 +++++ .../sources/v2/JavaColumnarDataSourceV2.java | 114 ----- .../v2/JavaPartitionAwareDataSource.java | 81 ++-- .../v2/JavaSchemaRequiredDataSource.java | 26 +- .../sources/v2/JavaSimpleDataSourceV2.java | 68 ++- .../sql/sources/v2/JavaSimpleReadSupport.java | 99 ---- ...pache.spark.sql.sources.DataSourceRegister | 4 +- .../streaming/MemorySinkV2Suite.scala | 2 +- ...rtSuite.scala => ConsoleWriterSuite.scala} | 4 +- .../sources/RateStreamProviderSuite.scala | 84 ++-- .../sources/TextSocketStreamSuite.scala | 81 ++-- .../sql/sources/v2/DataSourceV2Suite.scala | 438 ++++++++---------- .../sources/v2/SimpleWritableDataSource.scala | 110 +++-- .../spark/sql/streaming/StreamTest.scala | 2 +- .../StreamingQueryListenerSuite.scala | 4 +- .../sql/streaming/StreamingQuerySuite.scala | 58 ++- .../ContinuousQueuedDataReaderSuite.scala | 45 +- .../continuous/ContinuousSuite.scala | 2 +- .../continuous/EpochCoordinatorSuite.scala | 18 +- .../sources/StreamingDataSourceV2Suite.scala | 95 ++-- 105 files changed, 2129 insertions(+), 2580 deletions(-) rename external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/{KafkaContinuousReadSupport.scala => KafkaContinuousReader.scala} (74%) rename external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/{KafkaMicroBatchReadSupport.scala => KafkaMicroBatchReader.scala} (84%) rename external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/{KafkaStreamingWriteSupport.scala => KafkaStreamWriter.scala} (91%) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java rename sql/core/src/main/java/org/apache/spark/sql/sources/v2/{BatchReadSupportProvider.java => ReadSupport.java} (59%) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamWriteSupport.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java rename sql/core/src/main/java/org/apache/spark/sql/sources/v2/{BatchWriteSupportProvider.java => WriteSupport.java} (58%) delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java rename sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/{ScanConfigBuilder.java => ContinuousInputPartition.java} (61%) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceReader.java rename sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/{PartitionReader.java => InputPartitionReader.java} (67%) delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsScanColumnarBatch.java rename sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/{ContinuousPartitionReader.java => ContinuousInputPartitionReader.java} (60%) delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReader.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java create mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReader.java delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java rename sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/{BatchWriteSupport.java => DataSourceWriter.java} (79%) rename sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/{StreamingWriteSupport.java => StreamWriter.java} (78%) delete mode 100644 sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SimpleStreamingScanConfigBuilder.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/{ConsoleWriteSupport.scala => ConsoleWriter.scala} (86%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/{ForeachWriteSupportProvider.scala => ForeachWriterProvider.scala} (82%) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/{RateControlMicroBatchReadSupport.scala => MicroBatchWriter.scala} (50%) rename sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/{RateStreamMicroBatchReadSupport.scala => RateStreamMicroBatchReader.scala} (78%) create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaBatchDataSourceV2.java delete mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java delete mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java rename sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/{ConsoleWriteSupportSuite.scala => ConsoleWriterSuite.scala} (98%) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala similarity index 74% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala index 1753a28fba2f..8ce56a249622 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala @@ -25,15 +25,16 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.TaskContext import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming._ +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, Offset, PartitionOffset} import org.apache.spark.sql.types.StructType /** - * A [[ContinuousReadSupport]] for data from kafka. + * A [[ContinuousReader]] for data from kafka. * * @param offsetReader a reader used to get kafka offsets. Note that the actual data will be * read by per-task consumers generated later. @@ -46,49 +47,70 @@ import org.apache.spark.sql.types.StructType * scenarios, where some offsets after the specified initial ones can't be * properly read. */ -class KafkaContinuousReadSupport( +class KafkaContinuousReader( offsetReader: KafkaOffsetReader, kafkaParams: ju.Map[String, Object], sourceOptions: Map[String, String], metadataPath: String, initialOffsets: KafkaOffsetRangeLimit, failOnDataLoss: Boolean) - extends ContinuousReadSupport with Logging { + extends ContinuousReader with Logging { + + private lazy val session = SparkSession.getActiveSession.get + private lazy val sc = session.sparkContext private val pollTimeoutMs = sourceOptions.getOrElse("kafkaConsumer.pollTimeoutMs", "512").toLong - override def initialOffset(): Offset = { - val offsets = initialOffsets match { - case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) - case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets()) - case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss) - } - logInfo(s"Initial offsets: $offsets") - offsets - } + // Initialized when creating reader factories. If this diverges from the partitions at the latest + // offsets, we need to reconfigure. + // Exposed outside this object only for unit tests. + @volatile private[sql] var knownPartitions: Set[TopicPartition] = _ - override def fullSchema(): StructType = KafkaOffsetReader.kafkaSchema + override def readSchema: StructType = KafkaOffsetReader.kafkaSchema - override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = { - new KafkaContinuousScanConfigBuilder(fullSchema(), start, offsetReader, reportDataLoss) + private var offset: Offset = _ + override def setStartOffset(start: ju.Optional[Offset]): Unit = { + offset = start.orElse { + val offsets = initialOffsets match { + case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets()) + case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss) + } + logInfo(s"Initial offsets: $offsets") + offsets + } } + override def getStartOffset(): Offset = offset + override def deserializeOffset(json: String): Offset = { KafkaSourceOffset(JsonUtils.partitionOffsets(json)) } - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val startOffsets = config.asInstanceOf[KafkaContinuousScanConfig].startOffsets + override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = { + import scala.collection.JavaConverters._ + + val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(offset) + + val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet + val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet) + val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq) + + val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet) + if (deletedPartitions.nonEmpty) { + reportDataLoss(s"Some partitions were deleted: $deletedPartitions") + } + + val startOffsets = newPartitionOffsets ++ + oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_)) + knownPartitions = startOffsets.keySet + startOffsets.toSeq.map { case (topicPartition, start) => KafkaContinuousInputPartition( - topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss) - }.toArray - } - - override def createContinuousReaderFactory( - config: ScanConfig): ContinuousPartitionReaderFactory = { - KafkaContinuousReaderFactory + topicPartition, start, kafkaParams, pollTimeoutMs, failOnDataLoss + ): InputPartition[InternalRow] + }.asJava } /** Stop this source and free any resources it has allocated. */ @@ -105,9 +127,8 @@ class KafkaContinuousReadSupport( KafkaSourceOffset(mergedMap) } - override def needsReconfiguration(config: ScanConfig): Boolean = { - val knownPartitions = config.asInstanceOf[KafkaContinuousScanConfig].knownPartitions - offsetReader.fetchLatestOffsets().keySet != knownPartitions + override def needsReconfiguration(): Boolean = { + knownPartitions != null && offsetReader.fetchLatestOffsets().keySet != knownPartitions } override def toString(): String = s"KafkaSource[$offsetReader]" @@ -141,51 +162,23 @@ case class KafkaContinuousInputPartition( startOffset: Long, kafkaParams: ju.Map[String, Object], pollTimeoutMs: Long, - failOnDataLoss: Boolean) extends InputPartition - -object KafkaContinuousReaderFactory extends ContinuousPartitionReaderFactory { - override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = { - val p = partition.asInstanceOf[KafkaContinuousInputPartition] - new KafkaContinuousPartitionReader( - p.topicPartition, p.startOffset, p.kafkaParams, p.pollTimeoutMs, p.failOnDataLoss) + failOnDataLoss: Boolean) extends ContinuousInputPartition[InternalRow] { + + override def createContinuousReader( + offset: PartitionOffset): InputPartitionReader[InternalRow] = { + val kafkaOffset = offset.asInstanceOf[KafkaSourcePartitionOffset] + require(kafkaOffset.topicPartition == topicPartition, + s"Expected topicPartition: $topicPartition, but got: ${kafkaOffset.topicPartition}") + new KafkaContinuousInputPartitionReader( + topicPartition, kafkaOffset.partitionOffset, kafkaParams, pollTimeoutMs, failOnDataLoss) } -} - -class KafkaContinuousScanConfigBuilder( - schema: StructType, - startOffset: Offset, - offsetReader: KafkaOffsetReader, - reportDataLoss: String => Unit) - extends ScanConfigBuilder { - - override def build(): ScanConfig = { - val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(startOffset) - - val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet - val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet) - val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq) - val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet) - if (deletedPartitions.nonEmpty) { - reportDataLoss(s"Some partitions were deleted: $deletedPartitions") - } - - val startOffsets = newPartitionOffsets ++ - oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_)) - KafkaContinuousScanConfig(schema, startOffsets) + override def createPartitionReader(): KafkaContinuousInputPartitionReader = { + new KafkaContinuousInputPartitionReader( + topicPartition, startOffset, kafkaParams, pollTimeoutMs, failOnDataLoss) } } -case class KafkaContinuousScanConfig( - readSchema: StructType, - startOffsets: Map[TopicPartition, Long]) - extends ScanConfig { - - // Created when building the scan config builder. If this diverges from the partitions at the - // latest offsets, we need to reconfigure the kafka read support. - def knownPartitions: Set[TopicPartition] = startOffsets.keySet -} - /** * A per-task data reader for continuous Kafka processing. * @@ -196,12 +189,12 @@ case class KafkaContinuousScanConfig( * @param failOnDataLoss Flag indicating whether data reader should fail if some offsets * are skipped. */ -class KafkaContinuousPartitionReader( +class KafkaContinuousInputPartitionReader( topicPartition: TopicPartition, startOffset: Long, kafkaParams: ju.Map[String, Object], pollTimeoutMs: Long, - failOnDataLoss: Boolean) extends ContinuousPartitionReader[InternalRow] { + failOnDataLoss: Boolean) extends ContinuousInputPartitionReader[InternalRow] { private val consumer = KafkaDataConsumer.acquire(topicPartition, kafkaParams, useCache = false) private val converter = new KafkaRecordToUnsafeRowConverter diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala similarity index 84% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala index bb4de674c3c7..8cc989fce197 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala @@ -21,6 +21,8 @@ import java.{util => ju} import java.io._ import java.nio.charset.StandardCharsets +import scala.collection.JavaConverters._ + import org.apache.commons.io.IOUtils import org.apache.spark.SparkEnv @@ -29,17 +31,16 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset, SimpleStreamingScanConfig, SimpleStreamingScanConfigBuilder} -import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchReadSupport +import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset} import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} import org.apache.spark.sql.sources.v2.DataSourceOptions -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset} +import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader} +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset} import org.apache.spark.sql.types.StructType import org.apache.spark.util.UninterruptibleThread /** - * A [[MicroBatchReadSupport]] that reads data from Kafka. + * A [[MicroBatchReader]] that reads data from Kafka. * * The [[KafkaSourceOffset]] is the custom [[Offset]] defined for this source that contains * a map of TopicPartition -> offset. Note that this offset is 1 + (available offset). For @@ -54,13 +55,17 @@ import org.apache.spark.util.UninterruptibleThread * To avoid this issue, you should make sure stopping the query before stopping the Kafka brokers * and not use wrong broker addresses. */ -private[kafka010] class KafkaMicroBatchReadSupport( +private[kafka010] class KafkaMicroBatchReader( kafkaOffsetReader: KafkaOffsetReader, executorKafkaParams: ju.Map[String, Object], options: DataSourceOptions, metadataPath: String, startingOffsets: KafkaOffsetRangeLimit, - failOnDataLoss: Boolean) extends RateControlMicroBatchReadSupport with Logging { + failOnDataLoss: Boolean) + extends MicroBatchReader with Logging { + + private var startPartitionOffsets: PartitionOffsetMap = _ + private var endPartitionOffsets: PartitionOffsetMap = _ private val pollTimeoutMs = options.getLong( "kafkaConsumer.pollTimeoutMs", @@ -70,40 +75,34 @@ private[kafka010] class KafkaMicroBatchReadSupport( Option(options.get("maxOffsetsPerTrigger").orElse(null)).map(_.toLong) private val rangeCalculator = KafkaOffsetRangeCalculator(options) - - private var endPartitionOffsets: KafkaSourceOffset = _ - /** * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only * called in StreamExecutionThread. Otherwise, interrupting a thread while running * `KafkaConsumer.poll` may hang forever (KAFKA-1894). */ - override def initialOffset(): Offset = { - KafkaSourceOffset(getOrCreateInitialPartitionOffsets()) - } - - override def latestOffset(start: Offset): Offset = { - val startPartitionOffsets = start.asInstanceOf[KafkaSourceOffset].partitionToOffsets - val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets() - endPartitionOffsets = KafkaSourceOffset(maxOffsetsPerTrigger.map { maxOffsets => - rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets) - }.getOrElse { - latestPartitionOffsets - }) - endPartitionOffsets - } - - override def fullSchema(): StructType = KafkaOffsetReader.kafkaSchema - - override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end)) + private lazy val initialPartitionOffsets = getOrCreateInitialPartitionOffsets() + + override def setOffsetRange(start: ju.Optional[Offset], end: ju.Optional[Offset]): Unit = { + // Make sure initialPartitionOffsets is initialized + initialPartitionOffsets + + startPartitionOffsets = Option(start.orElse(null)) + .map(_.asInstanceOf[KafkaSourceOffset].partitionToOffsets) + .getOrElse(initialPartitionOffsets) + + endPartitionOffsets = Option(end.orElse(null)) + .map(_.asInstanceOf[KafkaSourceOffset].partitionToOffsets) + .getOrElse { + val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets() + maxOffsetsPerTrigger.map { maxOffsets => + rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets) + }.getOrElse { + latestPartitionOffsets + } + } } - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val sc = config.asInstanceOf[SimpleStreamingScanConfig] - val startPartitionOffsets = sc.start.asInstanceOf[KafkaSourceOffset].partitionToOffsets - val endPartitionOffsets = sc.end.get.asInstanceOf[KafkaSourceOffset].partitionToOffsets - + override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = { // Find the new partitions, and get their earliest offsets val newPartitions = endPartitionOffsets.keySet.diff(startPartitionOffsets.keySet) val newPartitionInitialOffsets = kafkaOffsetReader.fetchEarliestOffsets(newPartitions.toSeq) @@ -145,19 +144,26 @@ private[kafka010] class KafkaMicroBatchReadSupport( // Generate factories based on the offset ranges offsetRanges.map { range => - KafkaMicroBatchInputPartition( - range, executorKafkaParams, pollTimeoutMs, failOnDataLoss, reuseKafkaConsumer) - }.toArray + new KafkaMicroBatchInputPartition( + range, executorKafkaParams, pollTimeoutMs, failOnDataLoss, reuseKafkaConsumer + ): InputPartition[InternalRow] + }.asJava + } + + override def getStartOffset: Offset = { + KafkaSourceOffset(startPartitionOffsets) } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - KafkaMicroBatchReaderFactory + override def getEndOffset: Offset = { + KafkaSourceOffset(endPartitionOffsets) } override def deserializeOffset(json: String): Offset = { KafkaSourceOffset(JsonUtils.partitionOffsets(json)) } + override def readSchema(): StructType = KafkaOffsetReader.kafkaSchema + override def commit(end: Offset): Unit = {} override def stop(): Unit = { @@ -300,23 +306,22 @@ private[kafka010] case class KafkaMicroBatchInputPartition( executorKafkaParams: ju.Map[String, Object], pollTimeoutMs: Long, failOnDataLoss: Boolean, - reuseKafkaConsumer: Boolean) extends InputPartition + reuseKafkaConsumer: Boolean) extends InputPartition[InternalRow] { -private[kafka010] object KafkaMicroBatchReaderFactory extends PartitionReaderFactory { - override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - val p = partition.asInstanceOf[KafkaMicroBatchInputPartition] - KafkaMicroBatchPartitionReader(p.offsetRange, p.executorKafkaParams, p.pollTimeoutMs, - p.failOnDataLoss, p.reuseKafkaConsumer) - } + override def preferredLocations(): Array[String] = offsetRange.preferredLoc.toArray + + override def createPartitionReader(): InputPartitionReader[InternalRow] = + new KafkaMicroBatchInputPartitionReader(offsetRange, executorKafkaParams, pollTimeoutMs, + failOnDataLoss, reuseKafkaConsumer) } -/** A [[PartitionReader]] for reading Kafka data in a micro-batch streaming query. */ -private[kafka010] case class KafkaMicroBatchPartitionReader( +/** A [[InputPartitionReader]] for reading Kafka data in a micro-batch streaming query. */ +private[kafka010] case class KafkaMicroBatchInputPartitionReader( offsetRange: KafkaOffsetRange, executorKafkaParams: ju.Map[String, Object], pollTimeoutMs: Long, failOnDataLoss: Boolean, - reuseKafkaConsumer: Boolean) extends PartitionReader[InternalRow] with Logging { + reuseKafkaConsumer: Boolean) extends InputPartitionReader[InternalRow] with Logging { private val consumer = KafkaDataConsumer.acquire( offsetRange.topicPartition, executorKafkaParams, reuseKafkaConsumer) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 28c9853bfea9..d225c1ea6b7f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -30,8 +30,9 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SparkSession, SQLContext} import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.sources._ -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, MicroBatchReadSupport, StreamWriteSupport} +import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousInputPartitionReader +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType @@ -45,9 +46,9 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with StreamSinkProvider with RelationProvider with CreatableRelationProvider - with StreamingWriteSupportProvider - with ContinuousReadSupportProvider - with MicroBatchReadSupportProvider + with StreamWriteSupport + with ContinuousReadSupport + with MicroBatchReadSupport with Logging { import KafkaSourceProvider._ @@ -107,12 +108,13 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } /** - * Creates a [[org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport]] to read - * batches of Kafka data in a micro-batch streaming query. + * Creates a [[org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader]] to read batches + * of Kafka data in a micro-batch streaming query. */ - override def createMicroBatchReadSupport( + override def createMicroBatchReader( + schema: Optional[StructType], metadataPath: String, - options: DataSourceOptions): KafkaMicroBatchReadSupport = { + options: DataSourceOptions): KafkaMicroBatchReader = { val parameters = options.asMap().asScala.toMap validateStreamOptions(parameters) @@ -138,7 +140,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister parameters, driverGroupIdPrefix = s"$uniqueGroupId-driver") - new KafkaMicroBatchReadSupport( + new KafkaMicroBatchReader( kafkaOffsetReader, kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), options, @@ -148,12 +150,13 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } /** - * Creates a [[org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport]] to read + * Creates a [[ContinuousInputPartitionReader]] to read * Kafka data in a continuous streaming query. */ - override def createContinuousReadSupport( + override def createContinuousReader( + schema: Optional[StructType], metadataPath: String, - options: DataSourceOptions): KafkaContinuousReadSupport = { + options: DataSourceOptions): KafkaContinuousReader = { val parameters = options.asMap().asScala.toMap validateStreamOptions(parameters) // Each running query should use its own group id. Otherwise, the query may be only assigned @@ -178,7 +181,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister parameters, driverGroupIdPrefix = s"$uniqueGroupId-driver") - new KafkaContinuousReadSupport( + new KafkaContinuousReader( kafkaOffsetReader, kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), parameters, @@ -267,11 +270,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } } - override def createStreamingWriteSupport( + override def createStreamWriter( queryId: String, schema: StructType, mode: OutputMode, - options: DataSourceOptions): StreamingWriteSupport = { + options: DataSourceOptions): StreamWriter = { import scala.collection.JavaConverters._ val spark = SparkSession.getActiveSession.get @@ -282,7 +285,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister KafkaWriter.validateQuery( schema.toAttributes, new java.util.HashMap[String, Object](producerParams.asJava), topic) - new KafkaStreamingWriteSupport(topic, producerParams, schema) + new KafkaStreamWriter(topic, producerParams, schema) } private def strategy(caseInsensitiveParams: Map[String, String]) = diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala similarity index 91% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala index 927c56d9ce82..97c577d5a8b9 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamWriter.scala @@ -23,7 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport} +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter import org.apache.spark.sql.types.StructType /** @@ -33,20 +33,20 @@ import org.apache.spark.sql.types.StructType case object KafkaWriterCommitMessage extends WriterCommitMessage /** - * A [[StreamingWriteSupport]] for Kafka writing. Responsible for generating the writer factory. + * A [[StreamWriter]] for Kafka writing. Responsible for generating the writer factory. * * @param topic The topic this writer is responsible for. If None, topic will be inferred from * a `topic` field in the incoming data. * @param producerParams Parameters for Kafka producers in each task. * @param schema The schema of the input data. */ -class KafkaStreamingWriteSupport( +class KafkaStreamWriter( topic: Option[String], producerParams: Map[String, String], schema: StructType) - extends StreamingWriteSupport { + extends StreamWriter { validateQuery(schema.toAttributes, producerParams.toMap[String, Object].asJava, topic) - override def createStreamingWriterFactory(): KafkaStreamWriterFactory = + override def createWriterFactory(): KafkaStreamWriterFactory = KafkaStreamWriterFactory(topic, producerParams, schema) override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} @@ -63,9 +63,9 @@ class KafkaStreamingWriteSupport( */ case class KafkaStreamWriterFactory( topic: Option[String], producerParams: Map[String, String], schema: StructType) - extends StreamingDataWriterFactory { + extends DataWriterFactory[InternalRow] { - override def createWriter( + override def createDataWriter( partitionId: Int, taskId: Long, epochId: Long): DataWriter[InternalRow] = { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala index af510219a6f6..a0e5818dbbb6 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.clients.producer.ProducerRecord import org.apache.spark.sql.Dataset -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec +import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.streaming.Trigger @@ -207,13 +207,11 @@ class KafkaContinuousSourceTopicDeletionSuite extends KafkaContinuousTest { testUtils.createTopic(topic2, partitions = 5) eventually(timeout(streamingTimeout)) { assert( - query.lastExecution.executedPlan.collectFirst { - case scan: DataSourceV2ScanExec - if scan.readSupport.isInstanceOf[KafkaContinuousReadSupport] => - scan.scanConfig.asInstanceOf[KafkaContinuousScanConfig] - }.exists { config => + query.lastExecution.logical.collectFirst { + case StreamingDataSourceV2Relation(_, _, _, r: KafkaContinuousReader) => r + }.exists { r => // Ensure the new topic is present and the old topic is gone. - config.knownPartitions.exists(_.topic == topic2) + r.knownPartitions.exists(_.topic == topic2) }, s"query never reconfigured to new topic $topic2") } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala index fa6bdc20bd4f..fa1468a3943c 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala @@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.spark.SparkContext import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec +import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.streaming.Trigger @@ -46,10 +46,8 @@ trait KafkaContinuousTest extends KafkaSourceTest { testUtils.addPartitions(topic, newCount) eventually(timeout(streamingTimeout)) { assert( - query.lastExecution.executedPlan.collectFirst { - case scan: DataSourceV2ScanExec - if scan.readSupport.isInstanceOf[KafkaContinuousReadSupport] => - scan.scanConfig.asInstanceOf[KafkaContinuousScanConfig] + query.lastExecution.logical.collectFirst { + case StreamingDataSourceV2Relation(_, _, _, r: KafkaContinuousReader) => r }.exists(_.knownPartitions.size == newCount), s"query never reconfigured to $newCount partitions") } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 8e246dbbf5d7..65615fdb5b3e 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010 import java.io._ import java.nio.charset.StandardCharsets.UTF_8 import java.nio.file.{Files, Paths} -import java.util.Locale +import java.util.{Locale, Optional} import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.atomic.AtomicInteger @@ -28,7 +28,7 @@ import scala.collection.JavaConverters._ import scala.io.Source import scala.util.Random -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} +import org.apache.kafka.clients.producer.{ProducerRecord, RecordMetadata} import org.apache.kafka.common.TopicPartition import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ @@ -40,9 +40,11 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.functions.{count, window} import org.apache.spark.sql.kafka010.KafkaSourceProvider._ import org.apache.spark.sql.sources.v2.DataSourceOptions +import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest} import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType abstract class KafkaSourceTest extends StreamTest with SharedSQLContext with KafkaTest { @@ -112,16 +114,14 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext with Kaf query.nonEmpty, "Cannot add data when there is no query for finding the active kafka source") - val sources: Seq[BaseStreamingSource] = { + val sources = { query.get.logicalPlan.collect { case StreamingExecutionRelation(source: KafkaSource, _) => source - case StreamingExecutionRelation(source: KafkaMicroBatchReadSupport, _) => source + case StreamingExecutionRelation(source: KafkaMicroBatchReader, _) => source } ++ (query.get.lastExecution match { case null => Seq() case e => e.logical.collect { - case r: StreamingDataSourceV2Relation - if r.readSupport.isInstanceOf[KafkaContinuousReadSupport] => - r.readSupport.asInstanceOf[KafkaContinuousReadSupport] + case StreamingDataSourceV2Relation(_, _, _, reader: KafkaContinuousReader) => reader } }) }.distinct @@ -905,7 +905,7 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase { makeSureGetOffsetCalled, AssertOnQuery { query => query.logicalPlan.collect { - case StreamingExecutionRelation(_: KafkaMicroBatchReadSupport, _) => true + case StreamingExecutionRelation(_: KafkaMicroBatchReader, _) => true }.nonEmpty } ) @@ -930,16 +930,17 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase { "kafka.bootstrap.servers" -> testUtils.brokerAddress, "subscribe" -> topic ) ++ Option(minPartitions).map { p => "minPartitions" -> p} - val readSupport = provider.createMicroBatchReadSupport( - dir.getAbsolutePath, new DataSourceOptions(options.asJava)) - val config = readSupport.newScanConfigBuilder( - KafkaSourceOffset(Map(tp -> 0L)), - KafkaSourceOffset(Map(tp -> 100L))).build() - val inputPartitions = readSupport.planInputPartitions(config) + val reader = provider.createMicroBatchReader( + Optional.empty[StructType], dir.getAbsolutePath, new DataSourceOptions(options.asJava)) + reader.setOffsetRange( + Optional.of[OffsetV2](KafkaSourceOffset(Map(tp -> 0L))), + Optional.of[OffsetV2](KafkaSourceOffset(Map(tp -> 100L))) + ) + val factories = reader.planInputPartitions().asScala .map(_.asInstanceOf[KafkaMicroBatchInputPartition]) - withClue(s"minPartitions = $minPartitions generated factories $inputPartitions\n\t") { - assert(inputPartitions.size == numPartitionsGenerated) - inputPartitions.foreach { f => assert(f.reuseKafkaConsumer == reusesConsumers) } + withClue(s"minPartitions = $minPartitions generated factories $factories\n\t") { + assert(factories.size == numPartitionsGenerated) + factories.foreach { f => assert(f.reuseKafkaConsumer == reusesConsumers) } } } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java new file mode 100644 index 000000000000..7df5a451ae5f --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupport.java @@ -0,0 +1,46 @@ +/* + * 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.sources.v2; + +import java.util.Optional; + +import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader; +import org.apache.spark.sql.types.StructType; + +/** + * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to + * provide data reading ability for continuous stream processing. + */ +@InterfaceStability.Evolving +public interface ContinuousReadSupport extends DataSourceV2 { + /** + * Creates a {@link ContinuousReader} to scan the data from this data source. + * + * @param schema the user provided schema, or empty() if none was provided + * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure + * recovery. Readers for the same logical source in the same query + * will be given the same checkpointLocation. + * @param options the options for the returned data source reader, which is an immutable + * case-insensitive string-to-string map. + */ + ContinuousReader createContinuousReader( + Optional schema, + String checkpointLocation, + DataSourceOptions options); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java deleted file mode 100644 index 824c290518ac..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java +++ /dev/null @@ -1,70 +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.sources.v2; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; -import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport; -import org.apache.spark.sql.types.StructType; - -/** - * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to - * provide data reading ability for continuous stream processing. - * - * This interface is used to create {@link ContinuousReadSupport} instances when end users run - * {@code SparkSession.readStream.format(...).option(...).load()} with a continuous trigger. - */ -@InterfaceStability.Evolving -public interface ContinuousReadSupportProvider extends DataSourceV2 { - - /** - * Creates a {@link ContinuousReadSupport} instance to scan the data from this streaming data - * source with a user specified schema, which is called by Spark at the beginning of each - * continuous streaming query. - * - * By default this method throws {@link UnsupportedOperationException}, implementations should - * override this method to handle user specified schema. - * - * @param schema the user provided schema. - * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure - * recovery. Readers for the same logical source in the same query - * will be given the same checkpointLocation. - * @param options the options for the returned data source reader, which is an immutable - * case-insensitive string-to-string map. - */ - default ContinuousReadSupport createContinuousReadSupport( - StructType schema, - String checkpointLocation, - DataSourceOptions options) { - return DataSourceV2Utils.failForUserSpecifiedSchema(this); - } - - /** - * Creates a {@link ContinuousReadSupport} instance to scan the data from this streaming data - * source, which is called by Spark at the beginning of each continuous streaming query. - * - * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure - * recovery. Readers for the same logical source in the same query - * will be given the same checkpointLocation. - * @param options the options for the returned data source reader, which is an immutable - * case-insensitive string-to-string map. - */ - ContinuousReadSupport createContinuousReadSupport( - String checkpointLocation, - DataSourceOptions options); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java index 6e31e84bf6c7..6234071320dc 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java @@ -22,13 +22,9 @@ /** * The base interface for data source v2. Implementations must have a public, 0-arg constructor. * - * Note that this is an empty interface. Data source implementations must mix in interfaces such as - * {@link BatchReadSupportProvider} or {@link BatchWriteSupportProvider}, which can provide - * batch or streaming read/write support instances. Otherwise it's just a dummy data source which - * is un-readable/writable. - * - * If Spark fails to execute any methods in the implementations of this interface (by throwing an - * exception), the read action will fail and no Spark job will be submitted. + * Note that this is an empty interface. Data source implementations should mix-in at least one of + * the plug-in interfaces like {@link ReadSupport} and {@link WriteSupport}. Otherwise it's just + * a dummy data source which is un-readable/writable. */ @InterfaceStability.Evolving public interface DataSourceV2 {} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java new file mode 100644 index 000000000000..7f4a2c9593c7 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupport.java @@ -0,0 +1,52 @@ +/* + * 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.sources.v2; + +import java.util.Optional; + +import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader; +import org.apache.spark.sql.types.StructType; + +/** + * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to + * provide streaming micro-batch data reading ability. + */ +@InterfaceStability.Evolving +public interface MicroBatchReadSupport extends DataSourceV2 { + /** + * Creates a {@link MicroBatchReader} to read batches of data from this data source in a + * streaming query. + * + * The execution engine will create a micro-batch reader at the start of a streaming query, + * alternate calls to setOffsetRange and planInputPartitions for each batch to process, and + * then call stop() when the execution is complete. Note that a single query may have multiple + * executions due to restart or failure recovery. + * + * @param schema the user provided schema, or empty() if none was provided + * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure + * recovery. Readers for the same logical source in the same query + * will be given the same checkpointLocation. + * @param options the options for the returned data source reader, which is an immutable + * case-insensitive string-to-string map. + */ + MicroBatchReader createMicroBatchReader( + Optional schema, + String checkpointLocation, + DataSourceOptions options); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java deleted file mode 100644 index 61c08e7fa89d..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java +++ /dev/null @@ -1,70 +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.sources.v2; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; -import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport; -import org.apache.spark.sql.types.StructType; - -/** - * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to - * provide data reading ability for micro-batch stream processing. - * - * This interface is used to create {@link MicroBatchReadSupport} instances when end users run - * {@code SparkSession.readStream.format(...).option(...).load()} with a micro-batch trigger. - */ -@InterfaceStability.Evolving -public interface MicroBatchReadSupportProvider extends DataSourceV2 { - - /** - * Creates a {@link MicroBatchReadSupport} instance to scan the data from this streaming data - * source with a user specified schema, which is called by Spark at the beginning of each - * micro-batch streaming query. - * - * By default this method throws {@link UnsupportedOperationException}, implementations should - * override this method to handle user specified schema. - * - * @param schema the user provided schema. - * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure - * recovery. Readers for the same logical source in the same query - * will be given the same checkpointLocation. - * @param options the options for the returned data source reader, which is an immutable - * case-insensitive string-to-string map. - */ - default MicroBatchReadSupport createMicroBatchReadSupport( - StructType schema, - String checkpointLocation, - DataSourceOptions options) { - return DataSourceV2Utils.failForUserSpecifiedSchema(this); - } - - /** - * Creates a {@link MicroBatchReadSupport} instance to scan the data from this streaming data - * source, which is called by Spark at the beginning of each micro-batch streaming query. - * - * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure - * recovery. Readers for the same logical source in the same query - * will be given the same checkpointLocation. - * @param options the options for the returned data source reader, which is an immutable - * case-insensitive string-to-string map. - */ - MicroBatchReadSupport createMicroBatchReadSupport( - String checkpointLocation, - DataSourceOptions options); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java similarity index 59% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java rename to sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java index f403dc619e86..80ac08ee5ff5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ReadSupport.java @@ -18,44 +18,48 @@ package org.apache.spark.sql.sources.v2; import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; -import org.apache.spark.sql.sources.v2.reader.BatchReadSupport; +import org.apache.spark.sql.sources.DataSourceRegister; +import org.apache.spark.sql.sources.v2.reader.DataSourceReader; import org.apache.spark.sql.types.StructType; /** * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to - * provide data reading ability for batch processing. - * - * This interface is used to create {@link BatchReadSupport} instances when end users run - * {@code SparkSession.read.format(...).option(...).load()}. + * provide data reading ability and scan the data from the data source. */ @InterfaceStability.Evolving -public interface BatchReadSupportProvider extends DataSourceV2 { +public interface ReadSupport extends DataSourceV2 { /** - * Creates a {@link BatchReadSupport} instance to load the data from this data source with a user - * specified schema, which is called by Spark at the beginning of each batch query. - * - * Spark will call this method at the beginning of each batch query to create a - * {@link BatchReadSupport} instance. + * Creates a {@link DataSourceReader} to scan the data from this data source. * - * By default this method throws {@link UnsupportedOperationException}, implementations should - * override this method to handle user specified schema. + * If this method fails (by throwing an exception), the action will fail and no Spark job will be + * submitted. * * @param schema the user specified schema. * @param options the options for the returned data source reader, which is an immutable * case-insensitive string-to-string map. + * + * By default this method throws {@link UnsupportedOperationException}, implementations should + * override this method to handle user specified schema. */ - default BatchReadSupport createBatchReadSupport(StructType schema, DataSourceOptions options) { - return DataSourceV2Utils.failForUserSpecifiedSchema(this); + default DataSourceReader createReader(StructType schema, DataSourceOptions options) { + String name; + if (this instanceof DataSourceRegister) { + name = ((DataSourceRegister) this).shortName(); + } else { + name = this.getClass().getName(); + } + throw new UnsupportedOperationException(name + " does not support user specified schema"); } /** - * Creates a {@link BatchReadSupport} instance to scan the data from this data source, which is - * called by Spark at the beginning of each batch query. + * Creates a {@link DataSourceReader} to scan the data from this data source. + * + * If this method fails (by throwing an exception), the action will fail and no Spark job will be + * submitted. * * @param options the options for the returned data source reader, which is an immutable * case-insensitive string-to-string map. */ - BatchReadSupport createBatchReadSupport(DataSourceOptions options); + DataSourceReader createReader(DataSourceOptions options); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java index bbe430e29926..926c6fd8fd22 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java @@ -28,10 +28,9 @@ public interface SessionConfigSupport extends DataSourceV2 { /** - * Key prefix of the session configs to propagate, which is usually the data source name. Spark - * will extract all session configs that starts with `spark.datasource.$keyPrefix`, turn - * `spark.datasource.$keyPrefix.xxx -> yyy` into `xxx -> yyy`, and propagate them to all - * data source operations in this session. + * Key prefix of the session configs to propagate. Spark will extract all session configs that + * starts with `spark.datasource.$keyPrefix`, turn `spark.datasource.$keyPrefix.xxx -> yyy` + * into `xxx -> yyy`, and propagate them to all data source operations in this session. */ String keyPrefix(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamWriteSupport.java new file mode 100644 index 000000000000..a77b01497269 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamWriteSupport.java @@ -0,0 +1,52 @@ +/* + * 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.sources.v2; + +import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.execution.streaming.BaseStreamingSink; +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter; +import org.apache.spark.sql.streaming.OutputMode; +import org.apache.spark.sql.types.StructType; + +/** + * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to + * provide data writing ability for structured streaming. + */ +@InterfaceStability.Evolving +public interface StreamWriteSupport extends DataSourceV2, BaseStreamingSink { + + /** + * Creates an optional {@link StreamWriter} to save the data to this data source. Data + * sources can return None if there is no writing needed to be done. + * + * @param queryId A unique string for the writing query. It's possible that there are many + * writing queries running at the same time, and the returned + * {@link DataSourceWriter} can use this id to distinguish itself from others. + * @param schema the schema of the data to be written. + * @param mode the output mode which determines what successive epoch output means to this + * sink, please refer to {@link OutputMode} for more details. + * @param options the options for the returned data source writer, which is an immutable + * case-insensitive string-to-string map. + */ + StreamWriter createStreamWriter( + String queryId, + StructType schema, + OutputMode mode, + DataSourceOptions options); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java deleted file mode 100644 index f9ca85d8089b..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java +++ /dev/null @@ -1,54 +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.sources.v2; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.streaming.BaseStreamingSink; -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport; -import org.apache.spark.sql.streaming.OutputMode; -import org.apache.spark.sql.types.StructType; - -/** - * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to - * provide data writing ability for structured streaming. - * - * This interface is used to create {@link StreamingWriteSupport} instances when end users run - * {@code Dataset.writeStream.format(...).option(...).start()}. - */ -@InterfaceStability.Evolving -public interface StreamingWriteSupportProvider extends DataSourceV2, BaseStreamingSink { - - /** - * Creates a {@link StreamingWriteSupport} instance to save the data to this data source, which is - * called by Spark at the beginning of each streaming query. - * - * @param queryId A unique string for the writing query. It's possible that there are many - * writing queries running at the same time, and the returned - * {@link StreamingWriteSupport} can use this id to distinguish itself from others. - * @param schema the schema of the data to be written. - * @param mode the output mode which determines what successive epoch output means to this - * sink, please refer to {@link OutputMode} for more details. - * @param options the options for the returned data source writer, which is an immutable - * case-insensitive string-to-string map. - */ - StreamingWriteSupport createStreamingWriteSupport( - String queryId, - StructType schema, - OutputMode mode, - DataSourceOptions options); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/WriteSupport.java similarity index 58% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java rename to sql/core/src/main/java/org/apache/spark/sql/sources/v2/WriteSupport.java index bd10c3353bf1..048787a7a0a0 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/WriteSupport.java @@ -21,39 +21,33 @@ import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.sources.v2.writer.BatchWriteSupport; +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; import org.apache.spark.sql.types.StructType; /** * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to - * provide data writing ability for batch processing. - * - * This interface is used to create {@link BatchWriteSupport} instances when end users run - * {@code Dataset.write.format(...).option(...).save()}. + * provide data writing ability and save the data to the data source. */ @InterfaceStability.Evolving -public interface BatchWriteSupportProvider extends DataSourceV2 { +public interface WriteSupport extends DataSourceV2 { /** - * Creates an optional {@link BatchWriteSupport} instance to save the data to this data source, - * which is called by Spark at the beginning of each batch query. + * Creates an optional {@link DataSourceWriter} to save the data to this data source. Data + * sources can return None if there is no writing needed to be done according to the save mode. * - * Data sources can return None if there is no writing needed to be done according to the save - * mode. + * If this method fails (by throwing an exception), the action will fail and no Spark job will be + * submitted. * - * @param queryId A unique string for the writing query. It's possible that there are many - * writing queries running at the same time, and the returned - * {@link BatchWriteSupport} can use this id to distinguish itself from others. + * @param writeUUID A unique string for the writing job. It's possible that there are many writing + * jobs running at the same time, and the returned {@link DataSourceWriter} can + * use this job id to distinguish itself from other jobs. * @param schema the schema of the data to be written. * @param mode the save mode which determines what to do when the data are already in this data * source, please refer to {@link SaveMode} for more details. * @param options the options for the returned data source writer, which is an immutable * case-insensitive string-to-string map. - * @return a write support to write data to this data source. + * @return a writer to append data to this data source */ - Optional createBatchWriteSupport( - String queryId, - StructType schema, - SaveMode mode, - DataSourceOptions options); + Optional createWriter( + String writeUUID, StructType schema, SaveMode mode, DataSourceOptions options); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java deleted file mode 100644 index 452ee86675b4..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java +++ /dev/null @@ -1,51 +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.sources.v2.reader; - -import org.apache.spark.annotation.InterfaceStability; - -/** - * An interface that defines how to load the data from data source for batch processing. - * - * The execution engine will get an instance of this interface from a data source provider - * (e.g. {@link org.apache.spark.sql.sources.v2.BatchReadSupportProvider}) at the start of a batch - * query, then call {@link #newScanConfigBuilder()} and create an instance of {@link ScanConfig}. - * The {@link ScanConfigBuilder} can apply operator pushdown and keep the pushdown result in - * {@link ScanConfig}. The {@link ScanConfig} will be used to create input partitions and reader - * factory to scan data from the data source with a Spark job. - */ -@InterfaceStability.Evolving -public interface BatchReadSupport extends ReadSupport { - - /** - * Returns a builder of {@link ScanConfig}. Spark will call this method and create a - * {@link ScanConfig} for each data scanning job. - * - * The builder can take some query specific information to do operators pushdown, and keep these - * information in the created {@link ScanConfig}. - * - * This is the first step of the data scan. All other methods in {@link BatchReadSupport} needs - * to take {@link ScanConfig} as an input. - */ - ScanConfigBuilder newScanConfigBuilder(); - - /** - * Returns a factory, which produces one {@link PartitionReader} for one {@link InputPartition}. - */ - PartitionReaderFactory createReaderFactory(ScanConfig config); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousInputPartition.java similarity index 61% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java rename to sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousInputPartition.java index 4c0eedfddfe2..dcb87715d0b6 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ContinuousInputPartition.java @@ -18,13 +18,18 @@ package org.apache.spark.sql.sources.v2.reader; import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset; /** - * An interface for building the {@link ScanConfig}. Implementations can mixin those - * SupportsPushDownXYZ interfaces to do operator pushdown, and keep the operator pushdown result in - * the returned {@link ScanConfig}. + * A mix-in interface for {@link InputPartition}. Continuous input partitions can + * implement this interface to provide creating {@link InputPartitionReader} with particular offset. */ @InterfaceStability.Evolving -public interface ScanConfigBuilder { - ScanConfig build(); +public interface ContinuousInputPartition extends InputPartition { + /** + * Create an input partition reader with particular offset as its startOffset. + * + * @param offset offset want to set as the input partition reader's startOffset. + */ + InputPartitionReader createContinuousReader(PartitionOffset offset); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceReader.java new file mode 100644 index 000000000000..da98fab1284e --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/DataSourceReader.java @@ -0,0 +1,75 @@ +/* + * 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.sources.v2.reader; + +import java.util.List; + +import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.sources.v2.DataSourceOptions; +import org.apache.spark.sql.sources.v2.ReadSupport; +import org.apache.spark.sql.types.StructType; + +/** + * A data source reader that is returned by + * {@link ReadSupport#createReader(DataSourceOptions)} or + * {@link ReadSupport#createReader(StructType, DataSourceOptions)}. + * It can mix in various query optimization interfaces to speed up the data scan. The actual scan + * logic is delegated to {@link InputPartition}s, which are returned by + * {@link #planInputPartitions()}. + * + * There are mainly 3 kinds of query optimizations: + * 1. Operators push-down. E.g., filter push-down, required columns push-down(aka column + * pruning), etc. Names of these interfaces start with `SupportsPushDown`. + * 2. Information Reporting. E.g., statistics reporting, ordering reporting, etc. + * Names of these interfaces start with `SupportsReporting`. + * 3. Columnar scan if implements {@link SupportsScanColumnarBatch}. + * + * If an exception was throw when applying any of these query optimizations, the action will fail + * and no Spark job will be submitted. + * + * Spark first applies all operator push-down optimizations that this data source supports. Then + * Spark collects information this data source reported for further optimizations. Finally Spark + * issues the scan request and does the actual data reading. + */ +@InterfaceStability.Evolving +public interface DataSourceReader { + + /** + * Returns the actual schema of this data source reader, which may be different from the physical + * schema of the underlying storage, as column pruning or other optimizations may happen. + * + * If this method fails (by throwing an exception), the action will fail and no Spark job will be + * submitted. + */ + StructType readSchema(); + + /** + * Returns a list of {@link InputPartition}s. Each {@link InputPartition} is responsible for + * creating a data reader to output data of one RDD partition. The number of input partitions + * returned here is the same as the number of RDD partitions this scan outputs. + * + * Note that, this may not be a full scan if the data source reader mixes in other optimization + * interfaces like column pruning, filter push-down, etc. These optimizations are applied before + * Spark issues the scan request. + * + * If this method fails (by throwing an exception), the action will fail and no Spark job will be + * submitted. + */ + List> planInputPartitions(); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java index 95c30de907e4..f2038d0de3ff 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java @@ -22,18 +22,18 @@ import org.apache.spark.annotation.InterfaceStability; /** - * A serializable representation of an input partition returned by - * {@link ReadSupport#planInputPartitions(ScanConfig)}. + * An input partition returned by {@link DataSourceReader#planInputPartitions()} and is + * responsible for creating the actual data reader of one RDD partition. + * The relationship between {@link InputPartition} and {@link InputPartitionReader} + * is similar to the relationship between {@link Iterable} and {@link java.util.Iterator}. * - * Note that {@link InputPartition} will be serialized and sent to executors, then - * {@link PartitionReader} will be created by - * {@link PartitionReaderFactory#createReader(InputPartition)} or - * {@link PartitionReaderFactory#createColumnarReader(InputPartition)} on executors to do - * the actual reading. So {@link InputPartition} must be serializable while {@link PartitionReader} - * doesn't need to be. + * Note that {@link InputPartition}s will be serialized and sent to executors, then + * {@link InputPartitionReader}s will be created on executors to do the actual reading. So + * {@link InputPartition} must be serializable while {@link InputPartitionReader} doesn't need to + * be. */ @InterfaceStability.Evolving -public interface InputPartition extends Serializable { +public interface InputPartition extends Serializable { /** * The preferred locations where the input partition reader returned by this partition can run @@ -51,4 +51,12 @@ public interface InputPartition extends Serializable { default String[] preferredLocations() { return new String[0]; } + + /** + * Returns an input partition reader to do the actual reading work. + * + * If this method fails (by throwing an exception), the corresponding Spark task would fail and + * get retried until hitting the maximum retry times. + */ + InputPartitionReader createPartitionReader(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartitionReader.java similarity index 67% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java rename to sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartitionReader.java index 04ff8d0a19fc..f3ff7f5cc0f2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartitionReader.java @@ -23,27 +23,31 @@ import org.apache.spark.annotation.InterfaceStability; /** - * A partition reader returned by {@link PartitionReaderFactory#createReader(InputPartition)} or - * {@link PartitionReaderFactory#createColumnarReader(InputPartition)}. It's responsible for - * outputting data for a RDD partition. + * An input partition reader returned by {@link InputPartition#createPartitionReader()} and is + * responsible for outputting data for a RDD partition. * * Note that, Currently the type `T` can only be {@link org.apache.spark.sql.catalyst.InternalRow} - * for normal data sources, or {@link org.apache.spark.sql.vectorized.ColumnarBatch} for columnar - * data sources(whose {@link PartitionReaderFactory#supportColumnarReads(InputPartition)} - * returns true). + * for normal data source readers, {@link org.apache.spark.sql.vectorized.ColumnarBatch} for data + * source readers that mix in {@link SupportsScanColumnarBatch}. */ @InterfaceStability.Evolving -public interface PartitionReader extends Closeable { +public interface InputPartitionReader extends Closeable { /** * Proceed to next record, returns false if there is no more records. * + * If this method fails (by throwing an exception), the corresponding Spark task would fail and + * get retried until hitting the maximum retry times. + * * @throws IOException if failure happens during disk/network IO like reading files. */ boolean next() throws IOException; /** * Return the current record. This method should return same value until `next` is called. + * + * If this method fails (by throwing an exception), the corresponding Spark task would fail and + * get retried until hitting the maximum retry times. */ T get(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java deleted file mode 100644 index f35de9310eee..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java +++ /dev/null @@ -1,66 +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.sources.v2.reader; - -import java.io.Serializable; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.vectorized.ColumnarBatch; - -/** - * A factory used to create {@link PartitionReader} instances. - * - * If Spark fails to execute any methods in the implementations of this interface or in the returned - * {@link PartitionReader} (by throwing an exception), corresponding Spark task would fail and - * get retried until hitting the maximum retry times. - */ -@InterfaceStability.Evolving -public interface PartitionReaderFactory extends Serializable { - - /** - * Returns a row-based partition reader to read data from the given {@link InputPartition}. - * - * Implementations probably need to cast the input partition to the concrete - * {@link InputPartition} class defined for the data source. - */ - PartitionReader createReader(InputPartition partition); - - /** - * Returns a columnar partition reader to read data from the given {@link InputPartition}. - * - * Implementations probably need to cast the input partition to the concrete - * {@link InputPartition} class defined for the data source. - */ - default PartitionReader createColumnarReader(InputPartition partition) { - throw new UnsupportedOperationException("Cannot create columnar reader."); - } - - /** - * Returns true if the given {@link InputPartition} should be read by Spark in a columnar way. - * This means, implementations must also implement {@link #createColumnarReader(InputPartition)} - * for the input partitions that this method returns true. - * - * As of Spark 2.4, Spark can only read all input partition in a columnar way, or none of them. - * Data source can't mix columnar and row-based partitions. This may be relaxed in future - * versions. - */ - default boolean supportColumnarReads(InputPartition partition) { - return false; - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java deleted file mode 100644 index a58ddb288f1e..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java +++ /dev/null @@ -1,50 +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.sources.v2.reader; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.types.StructType; - -/** - * The base interface for all the batch and streaming read supports. Data sources should implement - * concrete read support interfaces like {@link BatchReadSupport}. - * - * If Spark fails to execute any methods in the implementations of this interface (by throwing an - * exception), the read action will fail and no Spark job will be submitted. - */ -@InterfaceStability.Evolving -public interface ReadSupport { - - /** - * Returns the full schema of this data source, which is usually the physical schema of the - * underlying storage. This full schema should not be affected by column pruning or other - * optimizations. - */ - StructType fullSchema(); - - /** - * Returns a list of {@link InputPartition input partitions}. Each {@link InputPartition} - * represents a data split that can be processed by one Spark task. The number of input - * partitions returned here is the same as the number of RDD partitions this scan outputs. - * - * Note that, this may not be a full scan if the data source supports optimization like filter - * push-down. Implementations should check the input {@link ScanConfig} and adjust the resulting - * {@link InputPartition input partitions}. - */ - InputPartition[] planInputPartitions(ScanConfig config); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java deleted file mode 100644 index 7462ce282058..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java +++ /dev/null @@ -1,45 +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.sources.v2.reader; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.types.StructType; - -/** - * An interface that carries query specific information for the data scanning job, like operator - * pushdown information and streaming query offsets. This is defined as an empty interface, and data - * sources should define their own {@link ScanConfig} classes. - * - * For APIs that take a {@link ScanConfig} as input, like - * {@link ReadSupport#planInputPartitions(ScanConfig)}, - * {@link BatchReadSupport#createReaderFactory(ScanConfig)} and - * {@link SupportsReportStatistics#estimateStatistics(ScanConfig)}, implementations mostly need to - * cast the input {@link ScanConfig} to the concrete {@link ScanConfig} class of the data source. - */ -@InterfaceStability.Evolving -public interface ScanConfig { - - /** - * Returns the actual schema of this data source reader, which may be different from the physical - * schema of the underlying storage, as column pruning or other optimizations may happen. - * - * If this method fails (by throwing an exception), the action will fail and no Spark job will be - * submitted. - */ - StructType readSchema(); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java index 44799c7d4913..031c7a73c367 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java @@ -23,7 +23,7 @@ /** * An interface to represent statistics for a data source, which is returned by - * {@link SupportsReportStatistics#estimateStatistics(ScanConfig)}. + * {@link SupportsReportStatistics#estimateStatistics()}. */ @InterfaceStability.Evolving public interface Statistics { diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java index 5e7985f645a0..7e0020f38a73 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java @@ -21,11 +21,11 @@ import org.apache.spark.sql.sources.Filter; /** - * A mix-in interface for {@link ScanConfigBuilder}. Data sources can implement this interface to - * push down filters to the data source and reduce the size of the data to be read. + * A mix-in interface for {@link DataSourceReader}. Data source readers can implement this + * interface to push down filters to the data source and reduce the size of the data to be read. */ @InterfaceStability.Evolving -public interface SupportsPushDownFilters extends ScanConfigBuilder { +public interface SupportsPushDownFilters extends DataSourceReader { /** * Pushes down filters, and returns filters that need to be evaluated after scanning. diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java index edb164937d6e..427b4d00a112 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java @@ -21,12 +21,12 @@ import org.apache.spark.sql.types.StructType; /** - * A mix-in interface for {@link ScanConfigBuilder}. Data sources can implement this + * A mix-in interface for {@link DataSourceReader}. Data source readers can implement this * interface to push down required columns to the data source and only read these columns during * scan to reduce the size of the data to be read. */ @InterfaceStability.Evolving -public interface SupportsPushDownRequiredColumns extends ScanConfigBuilder { +public interface SupportsPushDownRequiredColumns extends DataSourceReader { /** * Applies column pruning w.r.t. the given requiredSchema. @@ -35,8 +35,8 @@ public interface SupportsPushDownRequiredColumns extends ScanConfigBuilder { * also OK to do the pruning partially, e.g., a data source may not be able to prune nested * fields, and only prune top-level columns. * - * Note that, {@link ScanConfig#readSchema()} implementation should take care of the column - * pruning applied here. + * Note that, data source readers should update {@link DataSourceReader#readSchema()} after + * applying column pruning. */ void pruneColumns(StructType requiredSchema); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java index db62cd451536..6b60da7c4dc1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java @@ -21,17 +21,17 @@ import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning; /** - * A mix in interface for {@link BatchReadSupport}. Data sources can implement this interface to - * report data partitioning and try to avoid shuffle at Spark side. + * A mix in interface for {@link DataSourceReader}. Data source readers can implement this + * interface to report data partitioning and try to avoid shuffle at Spark side. * - * Note that, when a {@link ReadSupport} implementation creates exactly one {@link InputPartition}, - * Spark may avoid adding a shuffle even if the reader does not implement this interface. + * Note that, when the reader creates exactly one {@link InputPartition}, Spark may avoid + * adding a shuffle even if the reader does not implement this interface. */ @InterfaceStability.Evolving -public interface SupportsReportPartitioning extends ReadSupport { +public interface SupportsReportPartitioning extends DataSourceReader { /** * Returns the output data partitioning that this reader guarantees. */ - Partitioning outputPartitioning(ScanConfig config); + Partitioning outputPartitioning(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java index 1831488ba096..44d0ce3c6e74 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java @@ -20,18 +20,18 @@ import org.apache.spark.annotation.InterfaceStability; /** - * A mix in interface for {@link BatchReadSupport}. Data sources can implement this interface to - * report statistics to Spark. + * A mix in interface for {@link DataSourceReader}. Data source readers can implement this + * interface to report statistics to Spark. * * As of Spark 2.4, statistics are reported to the optimizer before any operator is pushed to the - * data source. Implementations that return more accurate statistics based on pushed operators will - * not improve query performance until the planner can push operators before getting stats. + * DataSourceReader. Implementations that return more accurate statistics based on pushed operators + * will not improve query performance until the planner can push operators before getting stats. */ @InterfaceStability.Evolving -public interface SupportsReportStatistics extends ReadSupport { +public interface SupportsReportStatistics extends DataSourceReader { /** - * Returns the estimated statistics of this data source scan. + * Returns the estimated statistics of this data source. */ - Statistics estimateStatistics(ScanConfig config); + Statistics estimateStatistics(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsScanColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsScanColumnarBatch.java new file mode 100644 index 000000000000..f4da686740d1 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsScanColumnarBatch.java @@ -0,0 +1,53 @@ +/* + * 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.sources.v2.reader; + +import java.util.List; + +import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.vectorized.ColumnarBatch; + +/** + * A mix-in interface for {@link DataSourceReader}. Data source readers can implement this + * interface to output {@link ColumnarBatch} and make the scan faster. + */ +@InterfaceStability.Evolving +public interface SupportsScanColumnarBatch extends DataSourceReader { + @Override + default List> planInputPartitions() { + throw new IllegalStateException( + "planInputPartitions not supported by default within SupportsScanColumnarBatch."); + } + + /** + * Similar to {@link DataSourceReader#planInputPartitions()}, but returns columnar data + * in batches. + */ + List> planBatchInputPartitions(); + + /** + * Returns true if the concrete data source reader can read data in batch according to the scan + * properties like required columns, pushes filters, etc. It's possible that the implementation + * can only support some certain columns with certain types. Users can overwrite this method and + * {@link #planInputPartitions()} to fallback to normal read path under some conditions. + */ + default boolean enableBatchRead() { + return true; + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java index 6764d4b7665c..38ca5fc6387b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java @@ -18,12 +18,12 @@ package org.apache.spark.sql.sources.v2.reader.partitioning; import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; +import org.apache.spark.sql.sources.v2.reader.InputPartitionReader; /** * A concrete implementation of {@link Distribution}. Represents a distribution where records that * share the same values for the {@link #clusteredColumns} will be produced by the same - * {@link PartitionReader}. + * {@link InputPartitionReader}. */ @InterfaceStability.Evolving public class ClusteredDistribution implements Distribution { diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java index 364a3f553923..5e32ba6952e1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java @@ -18,14 +18,14 @@ package org.apache.spark.sql.sources.v2.reader.partitioning; import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; +import org.apache.spark.sql.sources.v2.reader.InputPartitionReader; /** * An interface to represent data distribution requirement, which specifies how the records should - * be distributed among the data partitions (one {@link PartitionReader} outputs data for one + * be distributed among the data partitions (one {@link InputPartitionReader} outputs data for one * partition). * Note that this interface has nothing to do with the data ordering inside one - * partition(the output records of a single {@link PartitionReader}). + * partition(the output records of a single {@link InputPartitionReader}). * * The instance of this interface is created and provided by Spark, then consumed by * {@link Partitioning#satisfy(Distribution)}. This means data source developers don't need to diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java index fb0b6f1df43b..f460f6bfe3bb 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java @@ -19,13 +19,12 @@ import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.ScanConfig; import org.apache.spark.sql.sources.v2.reader.SupportsReportPartitioning; /** * An interface to represent the output data partitioning for a data source, which is returned by - * {@link SupportsReportPartitioning#outputPartitioning(ScanConfig)}. Note that this should work - * like a snapshot. Once created, it should be deterministic and always report the same number of + * {@link SupportsReportPartitioning#outputPartitioning()}. Note that this should work like a + * snapshot. Once created, it should be deterministic and always report the same number of * partitions and the same "satisfy" result for a certain distribution. */ @InterfaceStability.Evolving diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousInputPartitionReader.java similarity index 60% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java rename to sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousInputPartitionReader.java index 9101c8a44d34..7b0ba0bbdda9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousInputPartitionReader.java @@ -18,20 +18,19 @@ package org.apache.spark.sql.sources.v2.reader.streaming; import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.sources.v2.reader.PartitionReader; +import org.apache.spark.sql.sources.v2.reader.InputPartitionReader; /** - * A variation on {@link PartitionReader} for use with continuous streaming processing. + * A variation on {@link InputPartitionReader} for use with streaming in continuous processing mode. */ @InterfaceStability.Evolving -public interface ContinuousPartitionReader extends PartitionReader { - - /** - * Get the offset of the current record, or the start offset if no records have been read. - * - * The execution engine will call this method along with get() to keep track of the current - * offset. When an epoch ends, the offset of the previous record in each partition will be saved - * as a restart checkpoint. - */ - PartitionOffset getOffset(); +public interface ContinuousInputPartitionReader extends InputPartitionReader { + /** + * Get the offset of the current record, or the start offset if no records have been read. + * + * The execution engine will call this method along with get() to keep track of the current + * offset. When an epoch ends, the offset of the previous record in each partition will be saved + * as a restart checkpoint. + */ + PartitionOffset getOffset(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java deleted file mode 100644 index 2d9f1ca1686a..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java +++ /dev/null @@ -1,40 +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.sources.v2.reader.streaming; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; -import org.apache.spark.sql.vectorized.ColumnarBatch; - -/** - * A variation on {@link PartitionReaderFactory} that returns {@link ContinuousPartitionReader} - * instead of {@link org.apache.spark.sql.sources.v2.reader.PartitionReader}. It's used for - * continuous streaming processing. - */ -@InterfaceStability.Evolving -public interface ContinuousPartitionReaderFactory extends PartitionReaderFactory { - @Override - ContinuousPartitionReader createReader(InputPartition partition); - - @Override - default ContinuousPartitionReader createColumnarReader(InputPartition partition) { - throw new UnsupportedOperationException("Cannot create columnar reader."); - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java deleted file mode 100644 index 9a3ad2eb8a80..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java +++ /dev/null @@ -1,77 +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.sources.v2.reader.streaming; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.streaming.BaseStreamingSource; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.ScanConfig; -import org.apache.spark.sql.sources.v2.reader.ScanConfigBuilder; - -/** - * An interface that defines how to load the data from data source for continuous streaming - * processing. - * - * The execution engine will get an instance of this interface from a data source provider - * (e.g. {@link org.apache.spark.sql.sources.v2.ContinuousReadSupportProvider}) at the start of a - * streaming query, then call {@link #newScanConfigBuilder(Offset)} and create an instance of - * {@link ScanConfig} for the duration of the streaming query or until - * {@link #needsReconfiguration(ScanConfig)} is true. The {@link ScanConfig} will be used to create - * input partitions and reader factory to scan data with a Spark job for its duration. At the end - * {@link #stop()} will be called when the streaming execution is completed. Note that a single - * query may have multiple executions due to restart or failure recovery. - */ -@InterfaceStability.Evolving -public interface ContinuousReadSupport extends StreamingReadSupport, BaseStreamingSource { - - /** - * Returns a builder of {@link ScanConfig}. Spark will call this method and create a - * {@link ScanConfig} for each data scanning job. - * - * The builder can take some query specific information to do operators pushdown, store streaming - * offsets, etc., and keep these information in the created {@link ScanConfig}. - * - * This is the first step of the data scan. All other methods in {@link ContinuousReadSupport} - * needs to take {@link ScanConfig} as an input. - */ - ScanConfigBuilder newScanConfigBuilder(Offset start); - - /** - * Returns a factory, which produces one {@link ContinuousPartitionReader} for one - * {@link InputPartition}. - */ - ContinuousPartitionReaderFactory createContinuousReaderFactory(ScanConfig config); - - /** - * Merge partitioned offsets coming from {@link ContinuousPartitionReader} instances - * for each partition to a single global offset. - */ - Offset mergeOffsets(PartitionOffset[] offsets); - - /** - * The execution engine will call this method in every epoch to determine if new input - * partitions need to be generated, which may be required if for example the underlying - * source system has had partitions added or removed. - * - * If true, the query will be shut down and restarted with a new {@link ContinuousReadSupport} - * instance. - */ - default boolean needsReconfiguration(ScanConfig config) { - return false; - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReader.java new file mode 100644 index 000000000000..6e960bedf802 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReader.java @@ -0,0 +1,79 @@ +/* + * 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.sources.v2.reader.streaming; + +import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.execution.streaming.BaseStreamingSource; +import org.apache.spark.sql.sources.v2.reader.DataSourceReader; + +import java.util.Optional; + +/** + * A mix-in interface for {@link DataSourceReader}. Data source readers can implement this + * interface to allow reading in a continuous processing mode stream. + * + * Implementations must ensure each partition reader is a {@link ContinuousInputPartitionReader}. + * + * Note: This class currently extends {@link BaseStreamingSource} to maintain compatibility with + * DataSource V1 APIs. This extension will be removed once we get rid of V1 completely. + */ +@InterfaceStability.Evolving +public interface ContinuousReader extends BaseStreamingSource, DataSourceReader { + /** + * Merge partitioned offsets coming from {@link ContinuousInputPartitionReader} instances + * for each partition to a single global offset. + */ + Offset mergeOffsets(PartitionOffset[] offsets); + + /** + * Deserialize a JSON string into an Offset of the implementation-defined offset type. + * @throws IllegalArgumentException if the JSON does not encode a valid offset for this reader + */ + Offset deserializeOffset(String json); + + /** + * Set the desired start offset for partitions created from this reader. The scan will + * start from the first record after the provided offset, or from an implementation-defined + * inferred starting point if no offset is provided. + */ + void setStartOffset(Optional start); + + /** + * Return the specified or inferred start offset for this reader. + * + * @throws IllegalStateException if setStartOffset has not been called + */ + Offset getStartOffset(); + + /** + * The execution engine will call this method in every epoch to determine if new input + * partitions need to be generated, which may be required if for example the underlying + * source system has had partitions added or removed. + * + * If true, the query will be shut down and restarted with a new reader. + */ + default boolean needsReconfiguration() { + return false; + } + + /** + * Informs the source that Spark has completed processing all data for offsets less than or + * equal to `end` and will only request offsets greater than `end` in the future. + */ + void commit(Offset end); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java deleted file mode 100644 index edb0db11bff2..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java +++ /dev/null @@ -1,60 +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.sources.v2.reader.streaming; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.streaming.BaseStreamingSource; -import org.apache.spark.sql.sources.v2.reader.*; - -/** - * An interface that defines how to scan the data from data source for micro-batch streaming - * processing. - * - * The execution engine will get an instance of this interface from a data source provider - * (e.g. {@link org.apache.spark.sql.sources.v2.MicroBatchReadSupportProvider}) at the start of a - * streaming query, then call {@link #newScanConfigBuilder(Offset, Offset)} and create an instance - * of {@link ScanConfig} for each micro-batch. The {@link ScanConfig} will be used to create input - * partitions and reader factory to scan a micro-batch with a Spark job. At the end {@link #stop()} - * will be called when the streaming execution is completed. Note that a single query may have - * multiple executions due to restart or failure recovery. - */ -@InterfaceStability.Evolving -public interface MicroBatchReadSupport extends StreamingReadSupport, BaseStreamingSource { - - /** - * Returns a builder of {@link ScanConfig}. Spark will call this method and create a - * {@link ScanConfig} for each data scanning job. - * - * The builder can take some query specific information to do operators pushdown, store streaming - * offsets, etc., and keep these information in the created {@link ScanConfig}. - * - * This is the first step of the data scan. All other methods in {@link MicroBatchReadSupport} - * needs to take {@link ScanConfig} as an input. - */ - ScanConfigBuilder newScanConfigBuilder(Offset start, Offset end); - - /** - * Returns a factory, which produces one {@link PartitionReader} for one {@link InputPartition}. - */ - PartitionReaderFactory createReaderFactory(ScanConfig config); - - /** - * Returns the most recent offset available. - */ - Offset latestOffset(); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReader.java new file mode 100644 index 000000000000..0159c731762d --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReader.java @@ -0,0 +1,75 @@ +/* + * 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.sources.v2.reader.streaming; + +import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.sources.v2.reader.DataSourceReader; +import org.apache.spark.sql.execution.streaming.BaseStreamingSource; + +import java.util.Optional; + +/** + * A mix-in interface for {@link DataSourceReader}. Data source readers can implement this + * interface to indicate they allow micro-batch streaming reads. + * + * Note: This class currently extends {@link BaseStreamingSource} to maintain compatibility with + * DataSource V1 APIs. This extension will be removed once we get rid of V1 completely. + */ +@InterfaceStability.Evolving +public interface MicroBatchReader extends DataSourceReader, BaseStreamingSource { + /** + * Set the desired offset range for input partitions created from this reader. Partition readers + * will generate only data within (`start`, `end`]; that is, from the first record after `start` + * to the record with offset `end`. + * + * @param start The initial offset to scan from. If not specified, scan from an + * implementation-specified start point, such as the earliest available record. + * @param end The last offset to include in the scan. If not specified, scan up to an + * implementation-defined endpoint, such as the last available offset + * or the start offset plus a target batch size. + */ + void setOffsetRange(Optional start, Optional end); + + /** + * Returns the specified (if explicitly set through setOffsetRange) or inferred start offset + * for this reader. + * + * @throws IllegalStateException if setOffsetRange has not been called + */ + Offset getStartOffset(); + + /** + * Return the specified (if explicitly set through setOffsetRange) or inferred end offset + * for this reader. + * + * @throws IllegalStateException if setOffsetRange has not been called + */ + Offset getEndOffset(); + + /** + * Deserialize a JSON string into an Offset of the implementation-defined offset type. + * @throws IllegalArgumentException if the JSON does not encode a valid offset for this reader + */ + Offset deserializeOffset(String json); + + /** + * Informs the source that Spark has completed processing all data for offsets less than or + * equal to `end` and will only request offsets greater than `end` in the future. + */ + void commit(Offset end); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java index 6cf27734867c..e41c0351edc8 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java @@ -20,8 +20,8 @@ import org.apache.spark.annotation.InterfaceStability; /** - * An abstract representation of progress through a {@link MicroBatchReadSupport} or - * {@link ContinuousReadSupport}. + * An abstract representation of progress through a {@link MicroBatchReader} or + * {@link ContinuousReader}. * During execution, offsets provided by the data source implementation will be logged and used as * restart checkpoints. Each source should provide an offset implementation which the source can use * to reconstruct a position in the stream up to which data has been seen/processed. diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java deleted file mode 100644 index 84872d1ebc26..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java +++ /dev/null @@ -1,49 +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.sources.v2.reader.streaming; - -import org.apache.spark.sql.sources.v2.reader.ReadSupport; - -/** - * A base interface for streaming read support. This is package private and is invisible to data - * sources. Data sources should implement concrete streaming read support interfaces: - * {@link MicroBatchReadSupport} or {@link ContinuousReadSupport}. - */ -interface StreamingReadSupport extends ReadSupport { - - /** - * Returns the initial offset for a streaming query to start reading from. Note that the - * streaming data source should not assume that it will start reading from its initial offset: - * if Spark is restarting an existing query, it will restart from the check-pointed offset rather - * than the initial one. - */ - Offset initialOffset(); - - /** - * Deserialize a JSON string into an Offset of the implementation-defined offset type. - * - * @throws IllegalArgumentException if the JSON does not encode a valid offset for this reader - */ - Offset deserializeOffset(String json); - - /** - * Informs the source that Spark has completed processing all data for offsets less than or - * equal to `end` and will only request offsets greater than `end` in the future. - */ - void commit(Offset end); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataSourceWriter.java similarity index 79% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java rename to sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataSourceWriter.java index 0ec9e05d6a02..385fc294fea8 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataSourceWriter.java @@ -18,13 +18,28 @@ package org.apache.spark.sql.sources.v2.writer; import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.SaveMode; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.sources.v2.DataSourceOptions; +import org.apache.spark.sql.sources.v2.StreamWriteSupport; +import org.apache.spark.sql.sources.v2.WriteSupport; +import org.apache.spark.sql.streaming.OutputMode; +import org.apache.spark.sql.types.StructType; /** - * An interface that defines how to write the data to data source for batch processing. + * A data source writer that is returned by + * {@link WriteSupport#createWriter(String, StructType, SaveMode, DataSourceOptions)}/ + * {@link StreamWriteSupport#createStreamWriter( + * String, StructType, OutputMode, DataSourceOptions)}. + * It can mix in various writing optimization interfaces to speed up the data saving. The actual + * writing logic is delegated to {@link DataWriter}. + * + * If an exception was throw when applying any of these writing optimizations, the action will fail + * and no Spark job will be submitted. * * The writing procedure is: - * 1. Create a writer factory by {@link #createBatchWriterFactory()}, serialize and send it to all - * the partitions of the input data(RDD). + * 1. Create a writer factory by {@link #createWriterFactory()}, serialize and send it to all the + * partitions of the input data(RDD). * 2. For each partition, create the data writer, and write the data of the partition with this * writer. If all the data are written successfully, call {@link DataWriter#commit()}. If * exception happens during the writing, call {@link DataWriter#abort()}. @@ -38,7 +53,7 @@ * Please refer to the documentation of commit/abort methods for detailed specifications. */ @InterfaceStability.Evolving -public interface BatchWriteSupport { +public interface DataSourceWriter { /** * Creates a writer factory which will be serialized and sent to executors. @@ -46,7 +61,7 @@ public interface BatchWriteSupport { * If this method fails (by throwing an exception), the action will fail and no Spark job will be * submitted. */ - DataWriterFactory createBatchWriterFactory(); + DataWriterFactory createWriterFactory(); /** * Returns whether Spark should use the commit coordinator to ensure that at most one task for diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java index 5fb067966ee6..27dc5ea224fe 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java @@ -22,7 +22,7 @@ import org.apache.spark.annotation.InterfaceStability; /** - * A data writer returned by {@link DataWriterFactory#createWriter(int, long)} and is + * A data writer returned by {@link DataWriterFactory#createDataWriter(int, long, long)} and is * responsible for writing data for an input RDD partition. * * One Spark task has one exclusive data writer, so there is no thread-safe concern. @@ -36,11 +36,11 @@ * * If this data writer succeeds(all records are successfully written and {@link #commit()} * succeeds), a {@link WriterCommitMessage} will be sent to the driver side and pass to - * {@link BatchWriteSupport#commit(WriterCommitMessage[])} with commit messages from other data + * {@link DataSourceWriter#commit(WriterCommitMessage[])} with commit messages from other data * writers. If this data writer fails(one record fails to write or {@link #commit()} fails), an * exception will be sent to the driver side, and Spark may retry this writing task a few times. - * In each retry, {@link DataWriterFactory#createWriter(int, long)} will receive a - * different `taskId`. Spark will call {@link BatchWriteSupport#abort(WriterCommitMessage[])} + * In each retry, {@link DataWriterFactory#createDataWriter(int, long, long)} will receive a + * different `taskId`. Spark will call {@link DataSourceWriter#abort(WriterCommitMessage[])} * when the configured number of retries is exhausted. * * Besides the retry mechanism, Spark may launch speculative tasks if the existing writing task @@ -71,11 +71,11 @@ public interface DataWriter { /** * Commits this writer after all records are written successfully, returns a commit message which * will be sent back to driver side and passed to - * {@link BatchWriteSupport#commit(WriterCommitMessage[])}. + * {@link DataSourceWriter#commit(WriterCommitMessage[])}. * * The written data should only be visible to data source readers after - * {@link BatchWriteSupport#commit(WriterCommitMessage[])} succeeds, which means this method - * should still "hide" the written data and ask the {@link BatchWriteSupport} at driver side to + * {@link DataSourceWriter#commit(WriterCommitMessage[])} succeeds, which means this method + * should still "hide" the written data and ask the {@link DataSourceWriter} at driver side to * do the final commit via {@link WriterCommitMessage}. * * If this method fails (by throwing an exception), {@link #abort()} will be called and this @@ -93,7 +93,7 @@ public interface DataWriter { * failed. * * If this method fails(by throwing an exception), the underlying data source may have garbage - * that need to be cleaned by {@link BatchWriteSupport#abort(WriterCommitMessage[])} or manually, + * that need to be cleaned by {@link DataSourceWriter#abort(WriterCommitMessage[])} or manually, * but these garbage should not be visible to data source readers. * * @throws IOException if failure happens during disk/network IO like writing files. diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java index 19a36dd23245..3d337b6e0bdf 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java @@ -19,20 +19,18 @@ import java.io.Serializable; -import org.apache.spark.TaskContext; import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.catalyst.InternalRow; /** - * A factory of {@link DataWriter} returned by {@link BatchWriteSupport#createBatchWriterFactory()}, + * A factory of {@link DataWriter} returned by {@link DataSourceWriter#createWriterFactory()}, * which is responsible for creating and initializing the actual data writer at executor side. * * Note that, the writer factory will be serialized and sent to executors, then the data writer - * will be created on executors and do the actual writing. So this interface must be + * will be created on executors and do the actual writing. So {@link DataWriterFactory} must be * serializable and {@link DataWriter} doesn't need to be. */ @InterfaceStability.Evolving -public interface DataWriterFactory extends Serializable { +public interface DataWriterFactory extends Serializable { /** * Returns a data writer to do the actual writing work. Note that, Spark will reuse the same data @@ -40,16 +38,19 @@ public interface DataWriterFactory extends Serializable { * are responsible for defensive copies if necessary, e.g. copy the data before buffer it in a * list. * - * If this method fails (by throwing an exception), the corresponding Spark write task would fail - * and get retried until hitting the maximum retry times. + * If this method fails (by throwing an exception), the action will fail and no Spark job will be + * submitted. * * @param partitionId A unique id of the RDD partition that the returned writer will process. * Usually Spark processes many RDD partitions at the same time, * implementations should use the partition id to distinguish writers for * different partitions. - * @param taskId The task id returned by {@link TaskContext#taskAttemptId()}. Spark may run - * multiple tasks for the same partition (due to speculation or task failures, - * for example). + * @param taskId A unique identifier for a task that is performing the write of the partition + * data. Spark may run multiple tasks for the same partition (due to speculation + * or task failures, for example). + * @param epochId A monotonically increasing id for streaming queries that are split in to + * discrete periods of execution. For non-streaming queries, + * this ID will always be 0. */ - DataWriter createWriter(int partitionId, long taskId); + DataWriter createDataWriter(int partitionId, long taskId, long epochId); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java index 123335c414e9..9e38836c0edf 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java @@ -19,16 +19,15 @@ import java.io.Serializable; -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport; import org.apache.spark.annotation.InterfaceStability; /** * A commit message returned by {@link DataWriter#commit()} and will be sent back to the driver side - * as the input parameter of {@link BatchWriteSupport#commit(WriterCommitMessage[])} or - * {@link StreamingWriteSupport#commit(long, WriterCommitMessage[])}. + * as the input parameter of {@link DataSourceWriter#commit(WriterCommitMessage[])}. * - * This is an empty interface, data sources should define their own message class and use it when - * generating messages at executor side and handling the messages at driver side. + * This is an empty interface, data sources should define their own message class and use it in + * their {@link DataWriter#commit()} and {@link DataSourceWriter#commit(WriterCommitMessage[])} + * implementations. */ @InterfaceStability.Evolving public interface WriterCommitMessage extends Serializable {} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamWriter.java similarity index 78% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java rename to sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamWriter.java index 3fdfac5e1c84..a316b2a4c1d8 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java +++ b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamWriter.java @@ -18,36 +18,27 @@ package org.apache.spark.sql.sources.v2.writer.streaming; import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter; import org.apache.spark.sql.sources.v2.writer.DataWriter; import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; /** - * An interface that defines how to write the data to data source for streaming processing. + * A {@link DataSourceWriter} for use with structured streaming. * * Streaming queries are divided into intervals of data called epochs, with a monotonically * increasing numeric ID. This writer handles commits and aborts for each successive epoch. */ @InterfaceStability.Evolving -public interface StreamingWriteSupport { - - /** - * Creates a writer factory which will be serialized and sent to executors. - * - * If this method fails (by throwing an exception), the action will fail and no Spark job will be - * submitted. - */ - StreamingDataWriterFactory createStreamingWriterFactory(); - +public interface StreamWriter extends DataSourceWriter { /** * Commits this writing job for the specified epoch with a list of commit messages. The commit * messages are collected from successful data writers and are produced by * {@link DataWriter#commit()}. * * If this method fails (by throwing an exception), this writing job is considered to have been - * failed, and the execution engine will attempt to call - * {@link #abort(long, WriterCommitMessage[])}. + * failed, and the execution engine will attempt to call {@link #abort(WriterCommitMessage[])}. * - * The execution engine may call `commit` multiple times for the same epoch in some circumstances. + * The execution engine may call commit() multiple times for the same epoch in some circumstances. * To support exactly-once data semantics, implementations must ensure that multiple commits for * the same epoch are idempotent. */ @@ -55,8 +46,7 @@ public interface StreamingWriteSupport { /** * Aborts this writing job because some data writers are failed and keep failing when retried, or - * the Spark job fails with some unknown reasons, or {@link #commit(long, WriterCommitMessage[])} - * fails. + * the Spark job fails with some unknown reasons, or {@link #commit(WriterCommitMessage[])} fails. * * If this method fails (by throwing an exception), the underlying data source may require manual * cleanup. @@ -68,4 +58,14 @@ public interface StreamingWriteSupport { * clean up the data left by data writers. */ void abort(long epochId, WriterCommitMessage[] messages); + + default void commit(WriterCommitMessage[] messages) { + throw new UnsupportedOperationException( + "Commit without epoch should not be called with StreamWriter"); + } + + default void abort(WriterCommitMessage[] messages) { + throw new UnsupportedOperationException( + "Abort without epoch should not be called with StreamWriter"); + } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java deleted file mode 100644 index a4da24fc5ae6..000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java +++ /dev/null @@ -1,59 +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.sources.v2.writer.streaming; - -import java.io.Serializable; - -import org.apache.spark.TaskContext; -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.sources.v2.writer.DataWriter; - -/** - * A factory of {@link DataWriter} returned by - * {@link StreamingWriteSupport#createStreamingWriterFactory()}, which is responsible for creating - * and initializing the actual data writer at executor side. - * - * Note that, the writer factory will be serialized and sent to executors, then the data writer - * will be created on executors and do the actual writing. So this interface must be - * serializable and {@link DataWriter} doesn't need to be. - */ -@InterfaceStability.Evolving -public interface StreamingDataWriterFactory extends Serializable { - - /** - * Returns a data writer to do the actual writing work. Note that, Spark will reuse the same data - * object instance when sending data to the data writer, for better performance. Data writers - * are responsible for defensive copies if necessary, e.g. copy the data before buffer it in a - * list. - * - * If this method fails (by throwing an exception), the corresponding Spark write task would fail - * and get retried until hitting the maximum retry times. - * - * @param partitionId A unique id of the RDD partition that the returned writer will process. - * Usually Spark processes many RDD partitions at the same time, - * implementations should use the partition id to distinguish writers for - * different partitions. - * @param taskId The task id returned by {@link TaskContext#taskAttemptId()}. Spark may run - * multiple tasks for the same partition (due to speculation or task failures, - * for example). - * @param epochId A monotonically increasing id for streaming queries that are split in to - * discrete periods of execution. - */ - DataWriter createWriter(int partitionId, long taskId, long epochId); -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index e6c2cba79841..371ec7051879 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -37,7 +37,7 @@ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils -import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, DataSourceOptions, DataSourceV2} +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport} import org.apache.spark.sql.types.{StringType, StructType} import org.apache.spark.unsafe.types.UTF8String @@ -194,7 +194,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) if (classOf[DataSourceV2].isAssignableFrom(cls)) { val ds = cls.newInstance().asInstanceOf[DataSourceV2] - if (ds.isInstanceOf[BatchReadSupportProvider]) { + if (ds.isInstanceOf[ReadSupport]) { val sessionOptions = DataSourceV2Utils.extractSessionConfigs( ds = ds, conf = sparkSession.sessionState.conf) val pathsOption = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index dfb8c4718550..4aeddfd96d17 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -240,7 +240,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { if (classOf[DataSourceV2].isAssignableFrom(cls)) { val source = cls.newInstance().asInstanceOf[DataSourceV2] source match { - case provider: BatchWriteSupportProvider => + case ws: WriteSupport => val options = extraOptions ++ DataSourceV2Utils.extractSessionConfigs(source, df.sparkSession.sessionState.conf) @@ -251,10 +251,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } } else { - val writer = provider.createBatchWriteSupport( - UUID.randomUUID().toString, - df.logicalPlan.output.toStructType, - mode, + val writer = ws.createWriter( + UUID.randomUUID.toString, df.logicalPlan.output.toStructType, mode, new DataSourceOptions(options.asJava)) if (writer.isPresent) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala index f62f7349d1da..782829887c44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceRDD.scala @@ -17,22 +17,19 @@ package org.apache.spark.sql.execution.datasources.v2 -import org.apache.spark._ +import scala.reflect.ClassTag + +import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskContext} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.sources.v2.reader.InputPartition -class DataSourceRDDPartition(val index: Int, val inputPartition: InputPartition) +class DataSourceRDDPartition[T : ClassTag](val index: Int, val inputPartition: InputPartition[T]) extends Partition with Serializable -// TODO: we should have 2 RDDs: an RDD[InternalRow] for row-based scan, an `RDD[ColumnarBatch]` for -// columnar scan. -class DataSourceRDD( +class DataSourceRDD[T: ClassTag]( sc: SparkContext, - @transient private val inputPartitions: Seq[InputPartition], - partitionReaderFactory: PartitionReaderFactory, - columnarReads: Boolean) - extends RDD[InternalRow](sc, Nil) { + @transient private val inputPartitions: Seq[InputPartition[T]]) + extends RDD[T](sc, Nil) { override protected def getPartitions: Array[Partition] = { inputPartitions.zipWithIndex.map { @@ -40,21 +37,11 @@ class DataSourceRDD( }.toArray } - private def castPartition(split: Partition): DataSourceRDDPartition = split match { - case p: DataSourceRDDPartition => p - case _ => throw new SparkException(s"[BUG] Not a DataSourceRDDPartition: $split") - } - - override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { - val inputPartition = castPartition(split).inputPartition - val reader: PartitionReader[_] = if (columnarReads) { - partitionReaderFactory.createColumnarReader(inputPartition) - } else { - partitionReaderFactory.createReader(inputPartition) - } - + override def compute(split: Partition, context: TaskContext): Iterator[T] = { + val reader = split.asInstanceOf[DataSourceRDDPartition[T]].inputPartition + .createPartitionReader() context.addTaskCompletionListener[Unit](_ => reader.close()) - val iter = new Iterator[Any] { + val iter = new Iterator[T] { private[this] var valuePrepared = false override def hasNext: Boolean = { @@ -64,7 +51,7 @@ class DataSourceRDD( valuePrepared } - override def next(): Any = { + override def next(): T = { if (!hasNext) { throw new java.util.NoSuchElementException("End of stream") } @@ -72,11 +59,10 @@ class DataSourceRDD( reader.get() } } - // TODO: SPARK-25083 remove the type erasure hack in data source scan - new InterruptibleIterator(context, iter.asInstanceOf[Iterator[InternalRow]]) + new InterruptibleIterator(context, iter) } override def getPreferredLocations(split: Partition): Seq[String] = { - castPartition(split).inputPartition.preferredLocations() + split.asInstanceOf[DataSourceRDDPartition[T]].inputPartition.preferredLocations() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala index f7e29593a635..abc5fb979250 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -27,21 +27,21 @@ import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelat import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, BatchWriteSupportProvider, DataSourceOptions, DataSourceV2} -import org.apache.spark.sql.sources.v2.reader.{BatchReadSupport, ReadSupport, ScanConfigBuilder, SupportsReportStatistics} -import org.apache.spark.sql.sources.v2.writer.BatchWriteSupport +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, ReadSupport, WriteSupport} +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsReportStatistics} +import org.apache.spark.sql.sources.v2.writer.DataSourceWriter import org.apache.spark.sql.types.StructType /** * A logical plan representing a data source v2 scan. * * @param source An instance of a [[DataSourceV2]] implementation. - * @param options The options for this scan. Used to create fresh [[BatchWriteSupport]]. - * @param userSpecifiedSchema The user-specified schema for this scan. + * @param options The options for this scan. Used to create fresh [[DataSourceReader]]. + * @param userSpecifiedSchema The user-specified schema for this scan. Used to create fresh + * [[DataSourceReader]]. */ case class DataSourceV2Relation( source: DataSourceV2, - readSupport: BatchReadSupport, output: Seq[AttributeReference], options: Map[String, String], tableIdent: Option[TableIdentifier] = None, @@ -58,12 +58,13 @@ case class DataSourceV2Relation( override def simpleString: String = "RelationV2 " + metadataString - def newWriteSupport(): BatchWriteSupport = source.createWriteSupport(options, schema) + def newReader(): DataSourceReader = source.createReader(options, userSpecifiedSchema) - override def computeStats(): Statistics = readSupport match { + def newWriter(): DataSourceWriter = source.createWriter(options, schema) + + override def computeStats(): Statistics = newReader match { case r: SupportsReportStatistics => - val statistics = r.estimateStatistics(readSupport.newScanConfigBuilder().build()) - Statistics(sizeInBytes = statistics.sizeInBytes().orElse(conf.defaultSizeInBytes)) + Statistics(sizeInBytes = r.estimateStatistics.sizeInBytes().orElse(conf.defaultSizeInBytes)) case _ => Statistics(sizeInBytes = conf.defaultSizeInBytes) } @@ -84,8 +85,7 @@ case class StreamingDataSourceV2Relation( output: Seq[AttributeReference], source: DataSourceV2, options: Map[String, String], - readSupport: ReadSupport, - scanConfigBuilder: ScanConfigBuilder) + reader: DataSourceReader) extends LeafNode with MultiInstanceRelation with DataSourceV2StringFormat { override def isStreaming: Boolean = true @@ -99,8 +99,7 @@ case class StreamingDataSourceV2Relation( // TODO: unify the equal/hashCode implementation for all data source v2 query plans. override def equals(other: Any): Boolean = other match { case other: StreamingDataSourceV2Relation => - output == other.output && readSupport.getClass == other.readSupport.getClass && - options == other.options + output == other.output && reader.getClass == other.reader.getClass && options == other.options case _ => false } @@ -108,10 +107,9 @@ case class StreamingDataSourceV2Relation( Seq(output, source, options).hashCode() } - override def computeStats(): Statistics = readSupport match { + override def computeStats(): Statistics = reader match { case r: SupportsReportStatistics => - val statistics = r.estimateStatistics(scanConfigBuilder.build()) - Statistics(sizeInBytes = statistics.sizeInBytes().orElse(conf.defaultSizeInBytes)) + Statistics(sizeInBytes = r.estimateStatistics.sizeInBytes().orElse(conf.defaultSizeInBytes)) case _ => Statistics(sizeInBytes = conf.defaultSizeInBytes) } @@ -119,19 +117,19 @@ case class StreamingDataSourceV2Relation( object DataSourceV2Relation { private implicit class SourceHelpers(source: DataSourceV2) { - def asReadSupportProvider: BatchReadSupportProvider = { + def asReadSupport: ReadSupport = { source match { - case provider: BatchReadSupportProvider => - provider + case support: ReadSupport => + support case _ => throw new AnalysisException(s"Data source is not readable: $name") } } - def asWriteSupportProvider: BatchWriteSupportProvider = { + def asWriteSupport: WriteSupport = { source match { - case provider: BatchWriteSupportProvider => - provider + case support: WriteSupport => + support case _ => throw new AnalysisException(s"Data source is not writable: $name") } @@ -146,26 +144,23 @@ object DataSourceV2Relation { } } - def createReadSupport( + def createReader( options: Map[String, String], - userSpecifiedSchema: Option[StructType]): BatchReadSupport = { + userSpecifiedSchema: Option[StructType]): DataSourceReader = { val v2Options = new DataSourceOptions(options.asJava) userSpecifiedSchema match { case Some(s) => - asReadSupportProvider.createBatchReadSupport(s, v2Options) + asReadSupport.createReader(s, v2Options) case _ => - asReadSupportProvider.createBatchReadSupport(v2Options) + asReadSupport.createReader(v2Options) } } - def createWriteSupport( + def createWriter( options: Map[String, String], - schema: StructType): BatchWriteSupport = { - asWriteSupportProvider.createBatchWriteSupport( - UUID.randomUUID().toString, - schema, - SaveMode.Append, - new DataSourceOptions(options.asJava)).get + schema: StructType): DataSourceWriter = { + val v2Options = new DataSourceOptions(options.asJava) + asWriteSupport.createWriter(UUID.randomUUID.toString, schema, SaveMode.Append, v2Options).get } } @@ -174,16 +169,15 @@ object DataSourceV2Relation { options: Map[String, String], tableIdent: Option[TableIdentifier] = None, userSpecifiedSchema: Option[StructType] = None): DataSourceV2Relation = { - val readSupport = source.createReadSupport(options, userSpecifiedSchema) - val output = readSupport.fullSchema().toAttributes + val reader = source.createReader(options, userSpecifiedSchema) val ident = tableIdent.orElse(tableFromOptions(options)) DataSourceV2Relation( - source, readSupport, output, options, ident, userSpecifiedSchema) + source, reader.readSchema().toAttributes, options, ident, userSpecifiedSchema) } private def tableFromOptions(options: Map[String, String]): Option[TableIdentifier] = { options - .get(DataSourceOptions.TABLE_KEY) - .map(TableIdentifier(_, options.get(DataSourceOptions.DATABASE_KEY))) + .get(DataSourceOptions.TABLE_KEY) + .map(TableIdentifier(_, options.get(DataSourceOptions.DATABASE_KEY))) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala index 04a97735d024..c8494f97f176 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.datasources.v2 +import scala.collection.JavaConverters._ + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ @@ -26,7 +28,8 @@ import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, WholeSta import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.sources.v2.DataSourceV2 import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory, ContinuousReadSupport, MicroBatchReadSupport} +import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader +import org.apache.spark.sql.vectorized.ColumnarBatch /** * Physical plan node for scanning data from a data source. @@ -36,8 +39,7 @@ case class DataSourceV2ScanExec( @transient source: DataSourceV2, @transient options: Map[String, String], @transient pushedFilters: Seq[Expression], - @transient readSupport: ReadSupport, - @transient scanConfig: ScanConfig) + @transient reader: DataSourceReader) extends LeafExecNode with DataSourceV2StringFormat with ColumnarBatchScan { override def simpleString: String = "ScanV2 " + metadataString @@ -45,8 +47,7 @@ case class DataSourceV2ScanExec( // TODO: unify the equal/hashCode implementation for all data source v2 query plans. override def equals(other: Any): Boolean = other match { case other: DataSourceV2ScanExec => - output == other.output && readSupport.getClass == other.readSupport.getClass && - options == other.options + output == other.output && reader.getClass == other.reader.getClass && options == other.options case _ => false } @@ -54,39 +55,36 @@ case class DataSourceV2ScanExec( Seq(output, source, options).hashCode() } - override def outputPartitioning: physical.Partitioning = readSupport match { - case _ if partitions.length == 1 => + override def outputPartitioning: physical.Partitioning = reader match { + case r: SupportsScanColumnarBatch if r.enableBatchRead() && batchPartitions.size == 1 => + SinglePartition + + case r: SupportsScanColumnarBatch if !r.enableBatchRead() && partitions.size == 1 => + SinglePartition + + case r if !r.isInstanceOf[SupportsScanColumnarBatch] && partitions.size == 1 => SinglePartition case s: SupportsReportPartitioning => new DataSourcePartitioning( - s.outputPartitioning(scanConfig), AttributeMap(output.map(a => a -> a.name))) + s.outputPartitioning(), AttributeMap(output.map(a => a -> a.name))) case _ => super.outputPartitioning } - private lazy val partitions: Seq[InputPartition] = readSupport.planInputPartitions(scanConfig) - - private lazy val readerFactory = readSupport match { - case r: BatchReadSupport => r.createReaderFactory(scanConfig) - case r: MicroBatchReadSupport => r.createReaderFactory(scanConfig) - case r: ContinuousReadSupport => r.createContinuousReaderFactory(scanConfig) - case _ => throw new IllegalStateException("unknown read support: " + readSupport) + private lazy val partitions: Seq[InputPartition[InternalRow]] = { + reader.planInputPartitions().asScala } - // TODO: clean this up when we have dedicated scan plan for continuous streaming. - override val supportsBatch: Boolean = { - require(partitions.forall(readerFactory.supportColumnarReads) || - !partitions.exists(readerFactory.supportColumnarReads), - "Cannot mix row-based and columnar input partitions.") - - partitions.exists(readerFactory.supportColumnarReads) + private lazy val batchPartitions: Seq[InputPartition[ColumnarBatch]] = reader match { + case r: SupportsScanColumnarBatch if r.enableBatchRead() => + assert(!reader.isInstanceOf[ContinuousReader], + "continuous stream reader does not support columnar read yet.") + r.planBatchInputPartitions().asScala } - private lazy val inputRDD: RDD[InternalRow] = readSupport match { - case _: ContinuousReadSupport => - assert(!supportsBatch, - "continuous stream reader does not support columnar read yet.") + private lazy val inputRDD: RDD[InternalRow] = reader match { + case _: ContinuousReader => EpochCoordinatorRef.get( sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY), sparkContext.env) @@ -95,17 +93,22 @@ case class DataSourceV2ScanExec( sparkContext, sqlContext.conf.continuousStreamingExecutorQueueSize, sqlContext.conf.continuousStreamingExecutorPollIntervalMs, - partitions, - schema, - readerFactory.asInstanceOf[ContinuousPartitionReaderFactory]) + partitions).asInstanceOf[RDD[InternalRow]] + + case r: SupportsScanColumnarBatch if r.enableBatchRead() => + new DataSourceRDD(sparkContext, batchPartitions).asInstanceOf[RDD[InternalRow]] case _ => - new DataSourceRDD( - sparkContext, partitions, readerFactory.asInstanceOf[PartitionReaderFactory], supportsBatch) + new DataSourceRDD(sparkContext, partitions).asInstanceOf[RDD[InternalRow]] } override def inputRDDs(): Seq[RDD[InternalRow]] = Seq(inputRDD) + override val supportsBatch: Boolean = reader match { + case r: SupportsScanColumnarBatch if r.enableBatchRead() => true + case _ => false + } + override protected def needsUnsafeRowConversion: Boolean = false override protected def doExecute(): RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 9a3109e7c199..9d97d3b58f30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -26,8 +26,8 @@ import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, Rep import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, SupportsPushDownFilters, SupportsPushDownRequiredColumns} +import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReader object DataSourceV2Strategy extends Strategy { @@ -37,9 +37,9 @@ object DataSourceV2Strategy extends Strategy { * @return pushed filter and post-scan filters. */ private def pushFilters( - configBuilder: ScanConfigBuilder, + reader: DataSourceReader, filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - configBuilder match { + reader match { case r: SupportsPushDownFilters => // A map from translated data source filters to original catalyst filter expressions. val translatedFilterToExpr = mutable.HashMap.empty[sources.Filter, Expression] @@ -71,43 +71,41 @@ object DataSourceV2Strategy extends Strategy { /** * Applies column pruning to the data source, w.r.t. the references of the given expressions. * - * @return the created `ScanConfig`(since column pruning is the last step of operator pushdown), - * and new output attributes after column pruning. + * @return new output attributes after column pruning. */ // TODO: nested column pruning. private def pruneColumns( - configBuilder: ScanConfigBuilder, + reader: DataSourceReader, relation: DataSourceV2Relation, - exprs: Seq[Expression]): (ScanConfig, Seq[AttributeReference]) = { - configBuilder match { + exprs: Seq[Expression]): Seq[AttributeReference] = { + reader match { case r: SupportsPushDownRequiredColumns => val requiredColumns = AttributeSet(exprs.flatMap(_.references)) val neededOutput = relation.output.filter(requiredColumns.contains) if (neededOutput != relation.output) { r.pruneColumns(neededOutput.toStructType) - val config = r.build() val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap - config -> config.readSchema().toAttributes.map { + r.readSchema().toAttributes.map { // We have to keep the attribute id during transformation. a => a.withExprId(nameToAttr(a.name).exprId) } } else { - r.build() -> relation.output + relation.output } - case _ => configBuilder.build() -> relation.output + case _ => relation.output } } override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(project, filters, relation: DataSourceV2Relation) => - val configBuilder = relation.readSupport.newScanConfigBuilder() + val reader = relation.newReader() // `pushedFilters` will be pushed down and evaluated in the underlying data sources. // `postScanFilters` need to be evaluated after the scan. // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter. - val (pushedFilters, postScanFilters) = pushFilters(configBuilder, filters) - val (config, output) = pruneColumns(configBuilder, relation, project ++ postScanFilters) + val (pushedFilters, postScanFilters) = pushFilters(reader, filters) + val output = pruneColumns(reader, relation, project ++ postScanFilters) logInfo( s""" |Pushing operators to ${relation.source.getClass} @@ -117,12 +115,7 @@ object DataSourceV2Strategy extends Strategy { """.stripMargin) val scan = DataSourceV2ScanExec( - output, - relation.source, - relation.options, - pushedFilters, - relation.readSupport, - config) + output, relation.source, relation.options, pushedFilters, reader) val filterCondition = postScanFilters.reduceLeftOption(And) val withFilter = filterCondition.map(FilterExec(_, scan)).getOrElse(scan) @@ -131,26 +124,22 @@ object DataSourceV2Strategy extends Strategy { ProjectExec(project, withFilter) :: Nil case r: StreamingDataSourceV2Relation => - // TODO: support operator pushdown for streaming data sources. - val scanConfig = r.scanConfigBuilder.build() // ensure there is a projection, which will produce unsafe rows required by some operators ProjectExec(r.output, - DataSourceV2ScanExec( - r.output, r.source, r.options, r.pushedFilters, r.readSupport, scanConfig)) :: Nil + DataSourceV2ScanExec(r.output, r.source, r.options, r.pushedFilters, r.reader)) :: Nil case WriteToDataSourceV2(writer, query) => WriteToDataSourceV2Exec(writer, planLater(query)) :: Nil case AppendData(r: DataSourceV2Relation, query, _) => - WriteToDataSourceV2Exec(r.newWriteSupport(), planLater(query)) :: Nil + WriteToDataSourceV2Exec(r.newWriter(), planLater(query)) :: Nil case WriteToContinuousDataSource(writer, query) => WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil case Repartition(1, false, child) => - val isContinuous = child.find { - case s: StreamingDataSourceV2Relation => s.readSupport.isInstanceOf[ContinuousReadSupport] - case _ => false + val isContinuous = child.collectFirst { + case StreamingDataSourceV2Relation(_, _, _, r: ContinuousReader) => r }.isDefined if (isContinuous) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala index e9cc3991155c..5267f5f1580c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala @@ -21,7 +21,6 @@ import java.util.regex.Pattern import org.apache.spark.internal.Logging import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.sources.v2.{DataSourceV2, SessionConfigSupport} private[sql] object DataSourceV2Utils extends Logging { @@ -56,12 +55,4 @@ private[sql] object DataSourceV2Utils extends Logging { case _ => Map.empty } - - def failForUserSpecifiedSchema[T](ds: DataSourceV2): T = { - val name = ds match { - case register: DataSourceRegister => register.shortName() - case _ => ds.getClass.getName - } - throw new UnsupportedOperationException(name + " source does not support user-specified schema") - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala index c3f7b690ef63..59ebb9bc5431 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/WriteToDataSourceV2Exec.scala @@ -23,11 +23,15 @@ import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.executor.CommitDeniedException import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.streaming.MicroBatchExecution import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils /** @@ -35,8 +39,7 @@ import org.apache.spark.util.Utils * specific logical plans, like [[org.apache.spark.sql.catalyst.plans.logical.AppendData]]. */ @deprecated("Use specific logical plans like AppendData instead", "2.4.0") -case class WriteToDataSourceV2(writeSupport: BatchWriteSupport, query: LogicalPlan) - extends LogicalPlan { +case class WriteToDataSourceV2(writer: DataSourceWriter, query: LogicalPlan) extends LogicalPlan { override def children: Seq[LogicalPlan] = Seq(query) override def output: Seq[Attribute] = Nil } @@ -44,48 +47,46 @@ case class WriteToDataSourceV2(writeSupport: BatchWriteSupport, query: LogicalPl /** * The physical plan for writing data into data source v2. */ -case class WriteToDataSourceV2Exec(writeSupport: BatchWriteSupport, query: SparkPlan) - extends SparkPlan { - +case class WriteToDataSourceV2Exec(writer: DataSourceWriter, query: SparkPlan) extends SparkPlan { override def children: Seq[SparkPlan] = Seq(query) override def output: Seq[Attribute] = Nil override protected def doExecute(): RDD[InternalRow] = { - val writerFactory = writeSupport.createBatchWriterFactory() - val useCommitCoordinator = writeSupport.useCommitCoordinator + val writeTask = writer.createWriterFactory() + val useCommitCoordinator = writer.useCommitCoordinator val rdd = query.execute() val messages = new Array[WriterCommitMessage](rdd.partitions.length) - logInfo(s"Start processing data source write support: $writeSupport. " + + logInfo(s"Start processing data source writer: $writer. " + s"The input RDD has ${messages.length} partitions.") try { sparkContext.runJob( rdd, (context: TaskContext, iter: Iterator[InternalRow]) => - DataWritingSparkTask.run(writerFactory, context, iter, useCommitCoordinator), + DataWritingSparkTask.run(writeTask, context, iter, useCommitCoordinator), rdd.partitions.indices, (index, message: WriterCommitMessage) => { messages(index) = message - writeSupport.onDataWriterCommit(message) + writer.onDataWriterCommit(message) } ) - logInfo(s"Data source write support $writeSupport is committing.") - writeSupport.commit(messages) - logInfo(s"Data source write support $writeSupport committed.") + logInfo(s"Data source writer $writer is committing.") + writer.commit(messages) + logInfo(s"Data source writer $writer committed.") } catch { case cause: Throwable => - logError(s"Data source write support $writeSupport is aborting.") + logError(s"Data source writer $writer is aborting.") try { - writeSupport.abort(messages) + writer.abort(messages) } catch { case t: Throwable => - logError(s"Data source write support $writeSupport failed to abort.") + logError(s"Data source writer $writer failed to abort.") cause.addSuppressed(t) throw new SparkException("Writing job failed.", cause) } - logError(s"Data source write support $writeSupport aborted.") + logError(s"Data source writer $writer aborted.") cause match { // Only wrap non fatal exceptions. case NonFatal(e) => throw new SparkException("Writing job aborted.", e) @@ -99,7 +100,7 @@ case class WriteToDataSourceV2Exec(writeSupport: BatchWriteSupport, query: Spark object DataWritingSparkTask extends Logging { def run( - writerFactory: DataWriterFactory, + writeTask: DataWriterFactory[InternalRow], context: TaskContext, iter: Iterator[InternalRow], useCommitCoordinator: Boolean): WriterCommitMessage = { @@ -108,7 +109,8 @@ object DataWritingSparkTask extends Logging { val partId = context.partitionId() val taskId = context.taskAttemptId() val attemptId = context.attemptNumber() - val dataWriter = writerFactory.createWriter(partId, taskId) + val epochId = Option(context.getLocalProperty(MicroBatchExecution.BATCH_ID_KEY)).getOrElse("0") + val dataWriter = writeTask.createDataWriter(partId, taskId, epochId.toLong) // write the data and commit this writer. Utils.tryWithSafeFinallyAndFailureCallbacks(block = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 2cac86599ef1..7a007b6eb34d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.streaming +import java.util.Optional + import scala.collection.JavaConverters._ import scala.collection.mutable.{Map => MutableMap} @@ -26,9 +28,9 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, CurrentBatchTimestamp, import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Project} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, WriteToDataSourceV2} -import org.apache.spark.sql.execution.streaming.sources.{MicroBatchWritSupport, RateControlMicroBatchReadSupport} -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2} +import org.apache.spark.sql.execution.streaming.sources.MicroBatchWriter +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, MicroBatchReadSupport, StreamWriteSupport} +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset => OffsetV2} import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} import org.apache.spark.util.{Clock, Utils} @@ -49,8 +51,8 @@ class MicroBatchExecution( @volatile protected var sources: Seq[BaseStreamingSource] = Seq.empty - private val readSupportToDataSourceMap = - MutableMap.empty[MicroBatchReadSupport, (DataSourceV2, Map[String, String])] + private val readerToDataSourceMap = + MutableMap.empty[MicroBatchReader, (DataSourceV2, Map[String, String])] private val triggerExecutor = trigger match { case t: ProcessingTime => ProcessingTimeExecutor(t, triggerClock) @@ -89,19 +91,20 @@ class MicroBatchExecution( StreamingExecutionRelation(source, output)(sparkSession) }) case s @ StreamingRelationV2( - dataSourceV2: MicroBatchReadSupportProvider, sourceName, options, output, _) if + dataSourceV2: MicroBatchReadSupport, sourceName, options, output, _) if !disabledSources.contains(dataSourceV2.getClass.getCanonicalName) => v2ToExecutionRelationMap.getOrElseUpdate(s, { // Materialize source to avoid creating it in every batch val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId" - val readSupport = dataSourceV2.createMicroBatchReadSupport( + val reader = dataSourceV2.createMicroBatchReader( + Optional.empty(), // user specified schema metadataPath, new DataSourceOptions(options.asJava)) nextSourceId += 1 - readSupportToDataSourceMap(readSupport) = dataSourceV2 -> options - logInfo(s"Using MicroBatchReadSupport [$readSupport] from " + + readerToDataSourceMap(reader) = dataSourceV2 -> options + logInfo(s"Using MicroBatchReader [$reader] from " + s"DataSourceV2 named '$sourceName' [$dataSourceV2]") - StreamingExecutionRelation(readSupport, output)(sparkSession) + StreamingExecutionRelation(reader, output)(sparkSession) }) case s @ StreamingRelationV2(dataSourceV2, sourceName, _, output, v1Relation) => v2ToExecutionRelationMap.getOrElseUpdate(s, { @@ -341,19 +344,19 @@ class MicroBatchExecution( reportTimeTaken("getOffset") { (s, s.getOffset) } - case s: RateControlMicroBatchReadSupport => - updateStatusMessage(s"Getting offsets from $s") - reportTimeTaken("latestOffset") { - val startOffset = availableOffsets - .get(s).map(off => s.deserializeOffset(off.json)) - .getOrElse(s.initialOffset()) - (s, Option(s.latestOffset(startOffset))) - } - case s: MicroBatchReadSupport => + case s: MicroBatchReader => updateStatusMessage(s"Getting offsets from $s") - reportTimeTaken("latestOffset") { - (s, Option(s.latestOffset())) + reportTimeTaken("setOffsetRange") { + // Once v1 streaming source execution is gone, we can refactor this away. + // For now, we set the range here to get the source to infer the available end offset, + // get that offset, and then set the range again when we later execute. + s.setOffsetRange( + toJava(availableOffsets.get(s).map(off => s.deserializeOffset(off.json))), + Optional.empty()) } + + val currentOffset = reportTimeTaken("getEndOffset") { s.getEndOffset() } + (s, Option(currentOffset)) }.toMap availableOffsets ++= latestOffsets.filter { case (_, o) => o.nonEmpty }.mapValues(_.get) @@ -393,8 +396,8 @@ class MicroBatchExecution( if (prevBatchOff.isDefined) { prevBatchOff.get.toStreamProgress(sources).foreach { case (src: Source, off) => src.commit(off) - case (readSupport: MicroBatchReadSupport, off) => - readSupport.commit(readSupport.deserializeOffset(off.json)) + case (reader: MicroBatchReader, off) => + reader.commit(reader.deserializeOffset(off.json)) case (src, _) => throw new IllegalArgumentException( s"Unknown source is found at constructNextBatch: $src") @@ -438,34 +441,30 @@ class MicroBatchExecution( s"${batch.queryExecution.logical}") logDebug(s"Retrieving data from $source: $current -> $available") Some(source -> batch.logicalPlan) - - // TODO(cloud-fan): for data source v2, the new batch is just a new `ScanConfigBuilder`, but - // to be compatible with streaming source v1, we return a logical plan as a new batch here. - case (readSupport: MicroBatchReadSupport, available) - if committedOffsets.get(readSupport).map(_ != available).getOrElse(true) => - val current = committedOffsets.get(readSupport).map { - off => readSupport.deserializeOffset(off.json) - } - val endOffset: OffsetV2 = available match { - case v1: SerializedOffset => readSupport.deserializeOffset(v1.json) + case (reader: MicroBatchReader, available) + if committedOffsets.get(reader).map(_ != available).getOrElse(true) => + val current = committedOffsets.get(reader).map(off => reader.deserializeOffset(off.json)) + val availableV2: OffsetV2 = available match { + case v1: SerializedOffset => reader.deserializeOffset(v1.json) case v2: OffsetV2 => v2 } - val startOffset = current.getOrElse(readSupport.initialOffset) - val scanConfigBuilder = readSupport.newScanConfigBuilder(startOffset, endOffset) - logDebug(s"Retrieving data from $readSupport: $current -> $endOffset") + reader.setOffsetRange( + toJava(current), + Optional.of(availableV2)) + logDebug(s"Retrieving data from $reader: $current -> $availableV2") - val (source, options) = readSupport match { + val (source, options) = reader match { // `MemoryStream` is special. It's for test only and doesn't have a `DataSourceV2` // implementation. We provide a fake one here for explain. case _: MemoryStream[_] => MemoryStreamDataSource -> Map.empty[String, String] // Provide a fake value here just in case something went wrong, e.g. the reader gives // a wrong `equals` implementation. - case _ => readSupportToDataSourceMap.getOrElse(readSupport, { + case _ => readerToDataSourceMap.getOrElse(reader, { FakeDataSourceV2 -> Map.empty[String, String] }) } - Some(readSupport -> StreamingDataSourceV2Relation( - readSupport.fullSchema().toAttributes, source, options, readSupport, scanConfigBuilder)) + Some(reader -> StreamingDataSourceV2Relation( + reader.readSchema().toAttributes, source, options, reader)) case _ => None } } @@ -499,13 +498,13 @@ class MicroBatchExecution( val triggerLogicalPlan = sink match { case _: Sink => newAttributePlan - case s: StreamingWriteSupportProvider => - val writer = s.createStreamingWriteSupport( + case s: StreamWriteSupport => + val writer = s.createStreamWriter( s"$runId", newAttributePlan.schema, outputMode, new DataSourceOptions(extraOptions.asJava)) - WriteToDataSourceV2(new MicroBatchWritSupport(currentBatchId, writer), newAttributePlan) + WriteToDataSourceV2(new MicroBatchWriter(currentBatchId, writer), newAttributePlan) case _ => throw new IllegalArgumentException(s"unknown sink type for $sink") } @@ -533,7 +532,7 @@ class MicroBatchExecution( SQLExecution.withNewExecutionId(sparkSessionToRunBatch, lastExecution) { sink match { case s: Sink => s.addBatch(currentBatchId, nextBatch) - case _: StreamingWriteSupportProvider => + case _: StreamWriteSupport => // This doesn't accumulate any data - it just forces execution of the microbatch writer. nextBatch.collect() } @@ -557,6 +556,10 @@ class MicroBatchExecution( awaitProgressLock.unlock() } } + + private def toJava(scalaOption: Option[OffsetV2]): Optional[OffsetV2] = { + Optional.ofNullable(scalaOption.orNull) + } } object MicroBatchExecution { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala index d4b50655c721..6a380ab89ff7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ProgressReporter.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalP import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec -import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport +import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader import org.apache.spark.sql.streaming._ import org.apache.spark.sql.streaming.StreamingQueryListener.QueryProgressEvent import org.apache.spark.util.Clock @@ -251,7 +251,7 @@ trait ProgressReporter extends Logging { // Check whether the streaming query's logical plan has only V2 data sources val allStreamingLeaves = logicalPlan.collect { case s: StreamingExecutionRelation => s } - allStreamingLeaves.forall { _.source.isInstanceOf[MicroBatchReadSupport] } + allStreamingLeaves.forall { _.source.isInstanceOf[MicroBatchReader] } } if (onlyDataSourceV2Sources) { @@ -278,7 +278,7 @@ trait ProgressReporter extends Logging { new IdentityHashMap[DataSourceV2ScanExec, DataSourceV2ScanExec]() lastExecution.executedPlan.collectLeaves().foreach { - case s: DataSourceV2ScanExec if s.readSupport.isInstanceOf[BaseStreamingSource] => + case s: DataSourceV2ScanExec if s.reader.isInstanceOf[BaseStreamingSource] => uniqueStreamingExecLeavesMap.put(s, s) case _ => } @@ -286,7 +286,7 @@ trait ProgressReporter extends Logging { val sourceToInputRowsTuples = uniqueStreamingExecLeavesMap.values.asScala.map { execLeaf => val numRows = execLeaf.metrics.get("numOutputRows").map(_.value).getOrElse(0L) - val source = execLeaf.readSupport.asInstanceOf[BaseStreamingSource] + val source = execLeaf.reader.asInstanceOf[BaseStreamingSource] source -> numRows }.toSeq logDebug("Source -> # input rows\n\t" + sourceToInputRowsTuples.mkString("\n\t")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SimpleStreamingScanConfigBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SimpleStreamingScanConfigBuilder.scala deleted file mode 100644 index 1be071614d92..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SimpleStreamingScanConfigBuilder.scala +++ /dev/null @@ -1,40 +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.execution.streaming - -import org.apache.spark.sql.sources.v2.reader.{ScanConfig, ScanConfigBuilder} -import org.apache.spark.sql.types.StructType - -/** - * A very simple [[ScanConfigBuilder]] implementation that creates a simple [[ScanConfig]] to - * carry schema and offsets for streaming data sources. - */ -class SimpleStreamingScanConfigBuilder( - schema: StructType, - start: Offset, - end: Option[Offset] = None) - extends ScanConfigBuilder { - - override def build(): ScanConfig = SimpleStreamingScanConfig(schema, start, end) -} - -case class SimpleStreamingScanConfig( - readSchema: StructType, - start: Offset, - end: Option[Offset]) - extends ScanConfig diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala index 4b696dfa5735..24195b5657e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceV2} +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceV2} object StreamingRelation { def apply(dataSource: DataSource): StreamingRelation = { @@ -83,7 +83,7 @@ case class StreamingExecutionRelation( // We have to pack in the V1 data source as a shim, for the case when a source implements // continuous processing (which is always V2) but only has V1 microbatch support. We don't -// know at read time whether the query is continuous or not, so we need to be able to +// know at read time whether the query is conntinuous or not, so we need to be able to // swap a V1 relation back in. /** * Used to link a [[DataSourceV2]] into a streaming @@ -113,7 +113,7 @@ case class StreamingRelationV2( * Used to link a [[DataSourceV2]] into a continuous processing execution. */ case class ContinuousExecutionRelation( - source: ContinuousReadSupportProvider, + source: ContinuousReadSupport, extraOptions: Map[String, String], output: Seq[Attribute])(session: SparkSession) extends LeafNode with MultiInstanceRelation { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala index 9c5c16f4f5d1..cfba1001c6de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala @@ -18,10 +18,10 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.sql._ -import org.apache.spark.sql.execution.streaming.sources.ConsoleWriteSupport +import org.apache.spark.sql.execution.streaming.sources.ConsoleWriter import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister} -import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamingWriteSupportProvider} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamWriteSupport} +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType @@ -31,16 +31,16 @@ case class ConsoleRelation(override val sqlContext: SQLContext, data: DataFrame) } class ConsoleSinkProvider extends DataSourceV2 - with StreamingWriteSupportProvider + with StreamWriteSupport with DataSourceRegister with CreatableRelationProvider { - override def createStreamingWriteSupport( + override def createStreamWriter( queryId: String, schema: StructType, mode: OutputMode, - options: DataSourceOptions): StreamingWriteSupport = { - new ConsoleWriteSupport(schema, options) + options: DataSourceOptions): StreamWriter = { + new ConsoleWriter(schema, options) } def createRelation( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala index b68f67e0b22d..554a0b0573f4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousDataSourceRDD.scala @@ -21,13 +21,12 @@ import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousPartitionReaderFactory -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousInputPartitionReader import org.apache.spark.util.NextIterator class ContinuousDataSourceRDDPartition( val index: Int, - val inputPartition: InputPartition) + val inputPartition: InputPartition[InternalRow]) extends Partition with Serializable { // This is semantically a lazy val - it's initialized once the first time a call to @@ -50,22 +49,15 @@ class ContinuousDataSourceRDD( sc: SparkContext, dataQueueSize: Int, epochPollIntervalMs: Long, - private val inputPartitions: Seq[InputPartition], - schema: StructType, - partitionReaderFactory: ContinuousPartitionReaderFactory) + private val readerInputPartitions: Seq[InputPartition[InternalRow]]) extends RDD[InternalRow](sc, Nil) { override protected def getPartitions: Array[Partition] = { - inputPartitions.zipWithIndex.map { + readerInputPartitions.zipWithIndex.map { case (inputPartition, index) => new ContinuousDataSourceRDDPartition(index, inputPartition) }.toArray } - private def castPartition(split: Partition): ContinuousDataSourceRDDPartition = split match { - case p: ContinuousDataSourceRDDPartition => p - case _ => throw new SparkException(s"[BUG] Not a ContinuousDataSourceRDDPartition: $split") - } - /** * Initialize the shared reader for this partition if needed, then read rows from it until * it returns null to signal the end of the epoch. @@ -77,12 +69,10 @@ class ContinuousDataSourceRDD( } val readerForPartition = { - val partition = castPartition(split) + val partition = split.asInstanceOf[ContinuousDataSourceRDDPartition] if (partition.queueReader == null) { - val partitionReader = partitionReaderFactory.createReader( - partition.inputPartition) - partition.queueReader = new ContinuousQueuedDataReader( - partition.index, partitionReader, schema, context, dataQueueSize, epochPollIntervalMs) + partition.queueReader = + new ContinuousQueuedDataReader(partition, context, dataQueueSize, epochPollIntervalMs) } partition.queueReader @@ -103,6 +93,17 @@ class ContinuousDataSourceRDD( } override def getPreferredLocations(split: Partition): Seq[String] = { - castPartition(split).inputPartition.preferredLocations() + split.asInstanceOf[ContinuousDataSourceRDDPartition].inputPartition.preferredLocations() + } +} + +object ContinuousDataSourceRDD { + private[continuous] def getContinuousReader( + reader: InputPartitionReader[InternalRow]): ContinuousInputPartitionReader[_] = { + reader match { + case r: ContinuousInputPartitionReader[InternalRow] => r + case _ => + throw new IllegalStateException(s"Unknown continuous reader type ${reader.getClass}") + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index ccca72667a21..f104422b868f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -29,12 +29,13 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, CurrentDate, CurrentTimestamp} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SQLExecution -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanExec, StreamingDataSourceV2Relation} +import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, WriteToDataSourceV2} import org.apache.spark.sql.execution.streaming.{ContinuousExecutionRelation, StreamingRelationV2, _} import org.apache.spark.sql.sources.v2 -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, StreamingWriteSupportProvider} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, PartitionOffset} +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, StreamWriteSupport} +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, PartitionOffset} import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} +import org.apache.spark.sql.types.StructType import org.apache.spark.util.{Clock, Utils} class ContinuousExecution( @@ -42,7 +43,7 @@ class ContinuousExecution( name: String, checkpointRoot: String, analyzedPlan: LogicalPlan, - sink: StreamingWriteSupportProvider, + sink: StreamWriteSupport, trigger: Trigger, triggerClock: Clock, outputMode: OutputMode, @@ -52,7 +53,7 @@ class ContinuousExecution( sparkSession, name, checkpointRoot, analyzedPlan, sink, trigger, triggerClock, outputMode, deleteCheckpointOnStop) { - @volatile protected var continuousSources: Seq[ContinuousReadSupport] = Seq() + @volatile protected var continuousSources: Seq[ContinuousReader] = Seq() override protected def sources: Seq[BaseStreamingSource] = continuousSources // For use only in test harnesses. @@ -62,8 +63,7 @@ class ContinuousExecution( val toExecutionRelationMap = MutableMap[StreamingRelationV2, ContinuousExecutionRelation]() analyzedPlan.transform { case r @ StreamingRelationV2( - source: ContinuousReadSupportProvider, _, extraReaderOptions, output, _) => - // TODO: shall we create `ContinuousReadSupport` here instead of each reconfiguration? + source: ContinuousReadSupport, _, extraReaderOptions, output, _) => toExecutionRelationMap.getOrElseUpdate(r, { ContinuousExecutionRelation(source, extraReaderOptions, output)(sparkSession) }) @@ -148,7 +148,8 @@ class ContinuousExecution( val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId" nextSourceId += 1 - dataSource.createContinuousReadSupport( + dataSource.createContinuousReader( + java.util.Optional.empty[StructType](), metadataPath, new DataSourceOptions(extraReaderOptions.asJava)) } @@ -159,9 +160,9 @@ class ContinuousExecution( var insertedSourceId = 0 val withNewSources = logicalPlan transform { case ContinuousExecutionRelation(source, options, output) => - val readSupport = continuousSources(insertedSourceId) + val reader = continuousSources(insertedSourceId) insertedSourceId += 1 - val newOutput = readSupport.fullSchema().toAttributes + val newOutput = reader.readSchema().toAttributes assert(output.size == newOutput.size, s"Invalid reader: ${Utils.truncatedString(output, ",")} != " + @@ -169,10 +170,9 @@ class ContinuousExecution( replacements ++= output.zip(newOutput) val loggedOffset = offsets.offsets(0) - val realOffset = loggedOffset.map(off => readSupport.deserializeOffset(off.json)) - val startOffset = realOffset.getOrElse(readSupport.initialOffset) - val scanConfigBuilder = readSupport.newScanConfigBuilder(startOffset) - StreamingDataSourceV2Relation(newOutput, source, options, readSupport, scanConfigBuilder) + val realOffset = loggedOffset.map(off => reader.deserializeOffset(off.json)) + reader.setStartOffset(java.util.Optional.ofNullable(realOffset.orNull)) + StreamingDataSourceV2Relation(newOutput, source, options, reader) } // Rewire the plan to use the new attributes that were returned by the source. @@ -185,13 +185,17 @@ class ContinuousExecution( "CurrentTimestamp and CurrentDate not yet supported for continuous processing") } - val writer = sink.createStreamingWriteSupport( + val writer = sink.createStreamWriter( s"$runId", triggerLogicalPlan.schema, outputMode, new DataSourceOptions(extraOptions.asJava)) val withSink = WriteToContinuousDataSource(writer, triggerLogicalPlan) + val reader = withSink.collect { + case StreamingDataSourceV2Relation(_, _, _, r: ContinuousReader) => r + }.head + reportTimeTaken("queryPlanning") { lastExecution = new IncrementalExecution( sparkSessionForQuery, @@ -204,11 +208,6 @@ class ContinuousExecution( lastExecution.executedPlan // Force the lazy generation of execution plan } - val (readSupport, scanConfig) = lastExecution.executedPlan.collect { - case scan: DataSourceV2ScanExec if scan.readSupport.isInstanceOf[ContinuousReadSupport] => - scan.readSupport.asInstanceOf[ContinuousReadSupport] -> scan.scanConfig - }.head - sparkSessionForQuery.sparkContext.setLocalProperty( StreamExecution.IS_CONTINUOUS_PROCESSING, true.toString) sparkSessionForQuery.sparkContext.setLocalProperty( @@ -226,16 +225,14 @@ class ContinuousExecution( // Use the parent Spark session for the endpoint since it's where this query ID is registered. val epochEndpoint = EpochCoordinatorRef.create( - writer, readSupport, this, epochCoordinatorId, currentBatchId, sparkSession, SparkEnv.get) + writer, reader, this, epochCoordinatorId, currentBatchId, sparkSession, SparkEnv.get) val epochUpdateThread = new Thread(new Runnable { override def run: Unit = { try { triggerExecutor.execute(() => { startTrigger() - val shouldReconfigure = readSupport.needsReconfiguration(scanConfig) && - state.compareAndSet(ACTIVE, RECONFIGURING) - if (shouldReconfigure) { + if (reader.needsReconfiguration() && state.compareAndSet(ACTIVE, RECONFIGURING)) { if (queryExecutionThread.isAlive) { queryExecutionThread.interrupt() } @@ -285,12 +282,10 @@ class ContinuousExecution( * Report ending partition offsets for the given reader at the given epoch. */ def addOffset( - epoch: Long, - readSupport: ContinuousReadSupport, - partitionOffsets: Seq[PartitionOffset]): Unit = { + epoch: Long, reader: ContinuousReader, partitionOffsets: Seq[PartitionOffset]): Unit = { assert(continuousSources.length == 1, "only one continuous source supported currently") - val globalOffset = readSupport.mergeOffsets(partitionOffsets.toArray) + val globalOffset = reader.mergeOffsets(partitionOffsets.toArray) val oldOffset = synchronized { offsetLog.add(epoch, OffsetSeq.fill(globalOffset)) offsetLog.get(epoch - 1) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala index 65c5fc63c2f4..ec1dabd7da3e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousQueuedDataReader.scala @@ -25,9 +25,8 @@ import scala.util.control.NonFatal import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.UnsafeProjection -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReader, PartitionOffset} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader} +import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset import org.apache.spark.util.ThreadUtils /** @@ -38,14 +37,15 @@ import org.apache.spark.util.ThreadUtils * offsets across epochs. Each compute() should call the next() method here until null is returned. */ class ContinuousQueuedDataReader( - partitionIndex: Int, - reader: ContinuousPartitionReader[InternalRow], - schema: StructType, + partition: ContinuousDataSourceRDDPartition, context: TaskContext, dataQueueSize: Int, epochPollIntervalMs: Long) extends Closeable { + private val reader = partition.inputPartition.createPartitionReader() + // Important sequencing - we must get our starting point before the provider threads start running - private var currentOffset: PartitionOffset = reader.getOffset + private var currentOffset: PartitionOffset = + ContinuousDataSourceRDD.getContinuousReader(reader).getOffset /** * The record types in the read buffer. @@ -66,7 +66,7 @@ class ContinuousQueuedDataReader( epochMarkerExecutor.scheduleWithFixedDelay( epochMarkerGenerator, 0, epochPollIntervalMs, TimeUnit.MILLISECONDS) - private val dataReaderThread = new DataReaderThread(schema) + private val dataReaderThread = new DataReaderThread dataReaderThread.setDaemon(true) dataReaderThread.start() @@ -113,7 +113,7 @@ class ContinuousQueuedDataReader( currentEntry match { case EpochMarker => epochCoordEndpoint.send(ReportPartitionOffset( - partitionIndex, EpochTracker.getCurrentEpoch.get, currentOffset)) + partition.index, EpochTracker.getCurrentEpoch.get, currentOffset)) null case ContinuousRow(row, offset) => currentOffset = offset @@ -128,16 +128,16 @@ class ContinuousQueuedDataReader( /** * The data component of [[ContinuousQueuedDataReader]]. Pushes (row, offset) to the queue when - * a new row arrives to the [[ContinuousPartitionReader]]. + * a new row arrives to the [[InputPartitionReader]]. */ - class DataReaderThread(schema: StructType) extends Thread( + class DataReaderThread extends Thread( s"continuous-reader--${context.partitionId()}--" + s"${context.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY)}") with Logging { @volatile private[continuous] var failureReason: Throwable = _ - private val toUnsafe = UnsafeProjection.create(schema) override def run(): Unit = { TaskContext.setTaskContext(context) + val baseReader = ContinuousDataSourceRDD.getContinuousReader(reader) try { while (!shouldStop()) { if (!reader.next()) { @@ -149,9 +149,8 @@ class ContinuousQueuedDataReader( return } } - // `InternalRow#copy` may not be properly implemented, for safety we convert to unsafe row - // before copy here. - queue.put(ContinuousRow(toUnsafe(reader.get()).copy(), reader.getOffset)) + + queue.put(ContinuousRow(reader.get().copy(), baseReader.getOffset)) } } catch { case _: InterruptedException => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index a6cde2b8a710..551e07c3db86 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -17,22 +17,24 @@ package org.apache.spark.sql.execution.streaming.continuous +import scala.collection.JavaConverters._ + import org.json4s.DefaultFormats import org.json4s.jackson.Serialization import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.streaming.{RateStreamOffset, SimpleStreamingScanConfig, SimpleStreamingScanConfigBuilder, ValueRunTimeMsPair} +import org.apache.spark.sql.execution.streaming.{RateStreamOffset, ValueRunTimeMsPair} import org.apache.spark.sql.execution.streaming.sources.RateStreamProvider import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming._ +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, Offset, PartitionOffset} import org.apache.spark.sql.types.StructType case class RateStreamPartitionOffset( partition: Int, currentValue: Long, currentTimeMs: Long) extends PartitionOffset -class RateStreamContinuousReadSupport(options: DataSourceOptions) extends ContinuousReadSupport { +class RateStreamContinuousReader(options: DataSourceOptions) extends ContinuousReader { implicit val defaultFormats: DefaultFormats = DefaultFormats val creationTime = System.currentTimeMillis() @@ -54,18 +56,18 @@ class RateStreamContinuousReadSupport(options: DataSourceOptions) extends Contin RateStreamOffset(Serialization.read[Map[Int, ValueRunTimeMsPair]](json)) } - override def fullSchema(): StructType = RateStreamProvider.SCHEMA + override def readSchema(): StructType = RateStreamProvider.SCHEMA - override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start) - } + private var offset: Offset = _ - override def initialOffset: Offset = createInitialOffset(numPartitions, creationTime) + override def setStartOffset(offset: java.util.Optional[Offset]): Unit = { + this.offset = offset.orElse(createInitialOffset(numPartitions, creationTime)) + } - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val startOffset = config.asInstanceOf[SimpleStreamingScanConfig].start + override def getStartOffset(): Offset = offset - val partitionStartMap = startOffset match { + override def planInputPartitions(): java.util.List[InputPartition[InternalRow]] = { + val partitionStartMap = offset match { case off: RateStreamOffset => off.partitionToValueAndRunTimeMs case off => throw new IllegalArgumentException( @@ -88,12 +90,8 @@ class RateStreamContinuousReadSupport(options: DataSourceOptions) extends Contin i, numPartitions, perPartitionRate) - }.toArray - } - - override def createContinuousReaderFactory( - config: ScanConfig): ContinuousPartitionReaderFactory = { - RateStreamContinuousReaderFactory + .asInstanceOf[InputPartition[InternalRow]] + }.asJava } override def commit(end: Offset): Unit = {} @@ -120,23 +118,33 @@ case class RateStreamContinuousInputPartition( partitionIndex: Int, increment: Long, rowsPerSecond: Double) - extends InputPartition - -object RateStreamContinuousReaderFactory extends ContinuousPartitionReaderFactory { - override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = { - val p = partition.asInstanceOf[RateStreamContinuousInputPartition] - new RateStreamContinuousPartitionReader( - p.startValue, p.startTimeMs, p.partitionIndex, p.increment, p.rowsPerSecond) + extends ContinuousInputPartition[InternalRow] { + + override def createContinuousReader( + offset: PartitionOffset): InputPartitionReader[InternalRow] = { + val rateStreamOffset = offset.asInstanceOf[RateStreamPartitionOffset] + require(rateStreamOffset.partition == partitionIndex, + s"Expected partitionIndex: $partitionIndex, but got: ${rateStreamOffset.partition}") + new RateStreamContinuousInputPartitionReader( + rateStreamOffset.currentValue, + rateStreamOffset.currentTimeMs, + partitionIndex, + increment, + rowsPerSecond) } + + override def createPartitionReader(): InputPartitionReader[InternalRow] = + new RateStreamContinuousInputPartitionReader( + startValue, startTimeMs, partitionIndex, increment, rowsPerSecond) } -class RateStreamContinuousPartitionReader( +class RateStreamContinuousInputPartitionReader( startValue: Long, startTimeMs: Long, partitionIndex: Int, increment: Long, rowsPerSecond: Double) - extends ContinuousPartitionReader[InternalRow] { + extends ContinuousInputPartitionReader[InternalRow] { private var nextReadTime: Long = startTimeMs private val readTimeIncrement: Long = (1000 / rowsPerSecond).toLong diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala index 28ab2448a663..56bfefd91aaf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala @@ -20,9 +20,10 @@ package org.apache.spark.sql.execution.streaming.continuous import java.io.{BufferedReader, InputStreamReader, IOException} import java.net.Socket import java.sql.Timestamp -import java.util.Calendar +import java.util.{Calendar, List => JList} import javax.annotation.concurrent.GuardedBy +import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer import org.json4s.{DefaultFormats, NoTypeHints} @@ -33,26 +34,24 @@ import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.streaming.{Offset => _, _} +import org.apache.spark.sql.execution.streaming.{ContinuousRecordEndpoint, ContinuousRecordPartitionOffset, GetRecord} import org.apache.spark.sql.execution.streaming.sources.TextSocketReader import org.apache.spark.sql.sources.v2.DataSourceOptions -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming._ +import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader} +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, Offset, PartitionOffset} import org.apache.spark.sql.types.StructType import org.apache.spark.util.RpcUtils /** - * A ContinuousReadSupport that reads text lines through a TCP socket, designed only for tutorials - * and debugging. This ContinuousReadSupport will *not* work in production applications due to - * multiple reasons, including no support for fault recovery. + * A ContinuousReader that reads text lines through a TCP socket, designed only for tutorials and + * debugging. This ContinuousReader will *not* work in production applications due to multiple + * reasons, including no support for fault recovery. * * The driver maintains a socket connection to the host-port, keeps the received messages in * buckets and serves the messages to the executors via a RPC endpoint. */ -class TextSocketContinuousReadSupport(options: DataSourceOptions) - extends ContinuousReadSupport with Logging { - +class TextSocketContinuousReader(options: DataSourceOptions) extends ContinuousReader with Logging { implicit val defaultFormats: DefaultFormats = DefaultFormats private val host: String = options.get("host").get() @@ -74,8 +73,7 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions) @GuardedBy("this") private var currentOffset: Int = -1 - // Exposed for tests. - private[spark] var startOffset: TextSocketOffset = _ + private var startOffset: TextSocketOffset = _ private val recordEndpoint = new ContinuousRecordEndpoint(buckets, this) @volatile private var endpointRef: RpcEndpointRef = _ @@ -96,16 +94,16 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions) TextSocketOffset(Serialization.read[List[Int]](json)) } - override def initialOffset(): Offset = { - startOffset = TextSocketOffset(List.fill(numPartitions)(0)) - startOffset + override def setStartOffset(offset: java.util.Optional[Offset]): Unit = { + this.startOffset = offset + .orElse(TextSocketOffset(List.fill(numPartitions)(0))) + .asInstanceOf[TextSocketOffset] + recordEndpoint.setStartOffsets(startOffset.offsets) } - override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start) - } + override def getStartOffset: Offset = startOffset - override def fullSchema(): StructType = { + override def readSchema(): StructType = { if (includeTimestamp) { TextSocketReader.SCHEMA_TIMESTAMP } else { @@ -113,10 +111,8 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions) } } - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val startOffset = config.asInstanceOf[SimpleStreamingScanConfig] - .start.asInstanceOf[TextSocketOffset] - recordEndpoint.setStartOffsets(startOffset.offsets) + override def planInputPartitions(): JList[InputPartition[InternalRow]] = { + val endpointName = s"TextSocketContinuousReaderEndpoint-${java.util.UUID.randomUUID()}" endpointRef = recordEndpoint.rpcEnv.setupEndpoint(endpointName, recordEndpoint) @@ -136,13 +132,10 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions) startOffset.offsets.zipWithIndex.map { case (offset, i) => - TextSocketContinuousInputPartition(endpointName, i, offset, includeTimestamp) - }.toArray - } + TextSocketContinuousInputPartition( + endpointName, i, offset, includeTimestamp): InputPartition[InternalRow] + }.asJava - override def createContinuousReaderFactory( - config: ScanConfig): ContinuousPartitionReaderFactory = { - TextSocketReaderFactory } override def commit(end: Offset): Unit = synchronized { @@ -197,7 +190,7 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions) logWarning(s"Stream closed by $host:$port") return } - TextSocketContinuousReadSupport.this.synchronized { + TextSocketContinuousReader.this.synchronized { currentOffset += 1 val newData = (line, Timestamp.valueOf( @@ -228,30 +221,25 @@ case class TextSocketContinuousInputPartition( driverEndpointName: String, partitionId: Int, startOffset: Int, - includeTimestamp: Boolean) extends InputPartition - - -object TextSocketReaderFactory extends ContinuousPartitionReaderFactory { + includeTimestamp: Boolean) +extends InputPartition[InternalRow] { - override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = { - val p = partition.asInstanceOf[TextSocketContinuousInputPartition] - new TextSocketContinuousPartitionReader( - p.driverEndpointName, p.partitionId, p.startOffset, p.includeTimestamp) - } + override def createPartitionReader(): InputPartitionReader[InternalRow] = + new TextSocketContinuousInputPartitionReader(driverEndpointName, partitionId, startOffset, + includeTimestamp) } - /** * Continuous text socket input partition reader. * * Polls the driver endpoint for new records. */ -class TextSocketContinuousPartitionReader( +class TextSocketContinuousInputPartitionReader( driverEndpointName: String, partitionId: Int, startOffset: Int, includeTimestamp: Boolean) - extends ContinuousPartitionReader[InternalRow] { + extends ContinuousInputPartitionReader[InternalRow] { private val endpoint = RpcUtils.makeDriverRef( driverEndpointName, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala index a08411d746ab..967dbe24a370 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousWriteRDD.scala @@ -20,8 +20,7 @@ package org.apache.spark.sql.execution.streaming.continuous import org.apache.spark.{Partition, SparkEnv, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.writer.DataWriter -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingDataWriterFactory +import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} import org.apache.spark.util.Utils /** @@ -32,7 +31,7 @@ import org.apache.spark.util.Utils * * We keep repeating prev.compute() and writing new epochs until the query is shut down. */ -class ContinuousWriteRDD(var prev: RDD[InternalRow], writerFactory: StreamingDataWriterFactory) +class ContinuousWriteRDD(var prev: RDD[InternalRow], writeTask: DataWriterFactory[InternalRow]) extends RDD[Unit](prev) { override val partitioner = prev.partitioner @@ -51,7 +50,7 @@ class ContinuousWriteRDD(var prev: RDD[InternalRow], writerFactory: StreamingDat Utils.tryWithSafeFinallyAndFailureCallbacks(block = { try { val dataIterator = prev.compute(split, context) - dataWriter = writerFactory.createWriter( + dataWriter = writeTask.createDataWriter( context.partitionId(), context.taskAttemptId(), EpochTracker.getCurrentEpoch.get) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala index 2238ce26e7b4..8877ebeb2673 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala @@ -23,9 +23,9 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, PartitionOffset} +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, PartitionOffset} import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter import org.apache.spark.util.RpcUtils private[continuous] sealed trait EpochCoordinatorMessage extends Serializable @@ -82,15 +82,15 @@ private[sql] object EpochCoordinatorRef extends Logging { * Create a reference to a new [[EpochCoordinator]]. */ def create( - writeSupport: StreamingWriteSupport, - readSupport: ContinuousReadSupport, + writer: StreamWriter, + reader: ContinuousReader, query: ContinuousExecution, epochCoordinatorId: String, startEpoch: Long, session: SparkSession, env: SparkEnv): RpcEndpointRef = synchronized { val coordinator = new EpochCoordinator( - writeSupport, readSupport, query, startEpoch, session, env.rpcEnv) + writer, reader, query, startEpoch, session, env.rpcEnv) val ref = env.rpcEnv.setupEndpoint(endpointName(epochCoordinatorId), coordinator) logInfo("Registered EpochCoordinator endpoint") ref @@ -115,8 +115,8 @@ private[sql] object EpochCoordinatorRef extends Logging { * have both committed and reported an end offset for a given epoch. */ private[continuous] class EpochCoordinator( - writeSupport: StreamingWriteSupport, - readSupport: ContinuousReadSupport, + writer: StreamWriter, + reader: ContinuousReader, query: ContinuousExecution, startEpoch: Long, session: SparkSession, @@ -198,7 +198,7 @@ private[continuous] class EpochCoordinator( s"and is ready to be committed. Committing epoch $epoch.") // Sequencing is important here. We must commit to the writer before recording the commit // in the query, or we will end up dropping the commit if we restart in the middle. - writeSupport.commit(epoch, messages.toArray) + writer.commit(epoch, messages.toArray) query.commit(epoch) } @@ -220,7 +220,7 @@ private[continuous] class EpochCoordinator( partitionOffsets.collect { case ((e, _), o) if e == epoch => o } if (thisEpochOffsets.size == numReaderPartitions) { logDebug(s"Epoch $epoch has offsets reported from all partitions: $thisEpochOffsets") - query.addOffset(epoch, readSupport, thisEpochOffsets.toSeq) + query.addOffset(epoch, reader, thisEpochOffsets.toSeq) resolveCommitsAtEpoch(epoch) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala index 7ad21cc304e7..943c731a7052 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala @@ -19,13 +19,13 @@ package org.apache.spark.sql.execution.streaming.continuous import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter /** * The logical plan for writing data in a continuous stream. */ case class WriteToContinuousDataSource( - writeSupport: StreamingWriteSupport, query: LogicalPlan) extends LogicalPlan { + writer: StreamWriter, query: LogicalPlan) extends LogicalPlan { override def children: Seq[LogicalPlan] = Seq(query) override def output: Seq[Attribute] = Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala index c216b6138385..927d3a84e296 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala @@ -26,21 +26,21 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.streaming.StreamExecution -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter /** - * The physical plan for writing data into a continuous processing [[StreamingWriteSupport]]. + * The physical plan for writing data into a continuous processing [[StreamWriter]]. */ -case class WriteToContinuousDataSourceExec(writeSupport: StreamingWriteSupport, query: SparkPlan) +case class WriteToContinuousDataSourceExec(writer: StreamWriter, query: SparkPlan) extends SparkPlan with Logging { override def children: Seq[SparkPlan] = Seq(query) override def output: Seq[Attribute] = Nil override protected def doExecute(): RDD[InternalRow] = { - val writerFactory = writeSupport.createStreamingWriterFactory() + val writerFactory = writer.createWriterFactory() val rdd = new ContinuousWriteRDD(query.execute(), writerFactory) - logInfo(s"Start processing data source write support: $writeSupport. " + + logInfo(s"Start processing data source writer: $writer. " + s"The input RDD has ${rdd.partitions.length} partitions.") EpochCoordinatorRef.get( sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index adf52aba21a0..f81abdcc3711 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -17,9 +17,12 @@ package org.apache.spark.sql.execution.streaming +import java.{util => ju} +import java.util.Optional import java.util.concurrent.atomic.AtomicInteger import javax.annotation.concurrent.GuardedBy +import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, ListBuffer} import scala.util.control.NonFatal @@ -31,8 +34,8 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow} import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2} +import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader} +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset => OffsetV2} import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -64,7 +67,7 @@ abstract class MemoryStreamBase[A : Encoder](sqlContext: SQLContext) extends Bas addData(data.toTraversable) } - def fullSchema(): StructType = encoder.schema + def readSchema(): StructType = encoder.schema protected def logicalPlan: LogicalPlan @@ -77,7 +80,7 @@ abstract class MemoryStreamBase[A : Encoder](sqlContext: SQLContext) extends Bas * available. */ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) - extends MemoryStreamBase[A](sqlContext) with MicroBatchReadSupport with Logging { + extends MemoryStreamBase[A](sqlContext) with MicroBatchReader with Logging { protected val logicalPlan: LogicalPlan = StreamingExecutionRelation(this, attributes)(sqlContext.sparkSession) @@ -119,22 +122,24 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) override def toString: String = s"MemoryStream[${Utils.truncatedString(output, ",")}]" - override def deserializeOffset(json: String): OffsetV2 = LongOffset(json.toLong) + override def setOffsetRange(start: Optional[OffsetV2], end: Optional[OffsetV2]): Unit = { + synchronized { + startOffset = start.orElse(LongOffset(-1)).asInstanceOf[LongOffset] + endOffset = end.orElse(currentOffset).asInstanceOf[LongOffset] + } + } - override def initialOffset: OffsetV2 = LongOffset(-1) + override def deserializeOffset(json: String): OffsetV2 = LongOffset(json.toLong) - override def latestOffset(): OffsetV2 = { - if (currentOffset.offset == -1) null else currentOffset + override def getStartOffset: OffsetV2 = synchronized { + if (startOffset.offset == -1) null else startOffset } - override def newScanConfigBuilder(start: OffsetV2, end: OffsetV2): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end)) + override def getEndOffset: OffsetV2 = synchronized { + if (endOffset.offset == -1) null else endOffset } - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val sc = config.asInstanceOf[SimpleStreamingScanConfig] - val startOffset = sc.start.asInstanceOf[LongOffset] - val endOffset = sc.end.get.asInstanceOf[LongOffset] + override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = { synchronized { // Compute the internal batch numbers to fetch: [startOrdinal, endOrdinal) val startOrdinal = startOffset.offset.toInt + 1 @@ -151,15 +156,11 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) logDebug(generateDebugString(newBlocks.flatten, startOrdinal, endOrdinal)) newBlocks.map { block => - new MemoryStreamInputPartition(block) - }.toArray + new MemoryStreamInputPartition(block): InputPartition[InternalRow] + }.asJava } } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - MemoryStreamReaderFactory - } - private def generateDebugString( rows: Seq[UnsafeRow], startOrdinal: Int, @@ -200,12 +201,10 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } -class MemoryStreamInputPartition(val records: Array[UnsafeRow]) extends InputPartition - -object MemoryStreamReaderFactory extends PartitionReaderFactory { - override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - val records = partition.asInstanceOf[MemoryStreamInputPartition].records - new PartitionReader[InternalRow] { +class MemoryStreamInputPartition(records: Array[UnsafeRow]) + extends InputPartition[InternalRow] { + override def createPartitionReader(): InputPartitionReader[InternalRow] = { + new InputPartitionReader[InternalRow] { private var currentIndex = -1 override def next(): Boolean = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala similarity index 86% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala index 833e62f35ede..fd45ba509091 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriter.scala @@ -19,15 +19,16 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.internal.Logging import org.apache.spark.sql.{Dataset, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.sources.v2.DataSourceOptions -import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport} +import org.apache.spark.sql.sources.v2.writer.{DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter import org.apache.spark.sql.types.StructType /** Common methods used to create writes for the the console sink */ -class ConsoleWriteSupport(schema: StructType, options: DataSourceOptions) - extends StreamingWriteSupport with Logging { +class ConsoleWriter(schema: StructType, options: DataSourceOptions) + extends StreamWriter with Logging { // Number of rows to display, by default 20 rows protected val numRowsToShow = options.getInt("numRows", 20) @@ -38,7 +39,7 @@ class ConsoleWriteSupport(schema: StructType, options: DataSourceOptions) assert(SparkSession.getActiveSession.isDefined) protected val spark = SparkSession.getActiveSession.get - def createStreamingWriterFactory(): StreamingDataWriterFactory = PackedRowWriterFactory + def createWriterFactory(): DataWriterFactory[InternalRow] = PackedRowWriterFactory override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { // We have to print a "Batch" label for the epoch for compatibility with the pre-data source V2 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala index dbcc4483e577..4a32217f149b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala @@ -17,22 +17,26 @@ package org.apache.spark.sql.execution.streaming.sources +import java.{util => ju} +import java.util.Optional import java.util.concurrent.atomic.AtomicInteger import javax.annotation.concurrent.GuardedBy +import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer import org.json4s.NoTypeHints import org.json4s.jackson.Serialization import org.apache.spark.{SparkEnv, TaskContext} -import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.sql.{Encoder, SQLContext} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.streaming.{Offset => _, _} -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions} -import org.apache.spark.sql.sources.v2.reader.{InputPartition, ScanConfig, ScanConfigBuilder} -import org.apache.spark.sql.sources.v2.reader.streaming._ +import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions} +import org.apache.spark.sql.sources.v2.reader.InputPartition +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, Offset, PartitionOffset} +import org.apache.spark.sql.types.StructType import org.apache.spark.util.RpcUtils /** @@ -44,9 +48,7 @@ import org.apache.spark.util.RpcUtils * the specified offset within the list, or null if that offset doesn't yet have a record. */ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPartitions: Int = 2) - extends MemoryStreamBase[A](sqlContext) - with ContinuousReadSupportProvider with ContinuousReadSupport { - + extends MemoryStreamBase[A](sqlContext) with ContinuousReader with ContinuousReadSupport { private implicit val formats = Serialization.formats(NoTypeHints) protected val logicalPlan = @@ -57,6 +59,9 @@ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPa @GuardedBy("this") private val records = Seq.fill(numPartitions)(new ListBuffer[A]) + @GuardedBy("this") + private var startOffset: ContinuousMemoryStreamOffset = _ + private val recordEndpoint = new ContinuousRecordEndpoint(records, this) @volatile private var endpointRef: RpcEndpointRef = _ @@ -70,8 +75,15 @@ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPa ContinuousMemoryStreamOffset((0 until numPartitions).map(i => (i, records(i).size)).toMap) } - override def initialOffset(): Offset = { - ContinuousMemoryStreamOffset((0 until numPartitions).map(i => (i, 0)).toMap) + override def setStartOffset(start: Optional[Offset]): Unit = synchronized { + // Inferred initial offset is position 0 in each partition. + startOffset = start.orElse { + ContinuousMemoryStreamOffset((0 until numPartitions).map(i => (i, 0)).toMap) + }.asInstanceOf[ContinuousMemoryStreamOffset] + } + + override def getStartOffset: Offset = synchronized { + startOffset } override def deserializeOffset(json: String): ContinuousMemoryStreamOffset = { @@ -86,40 +98,34 @@ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPa ) } - override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start) - } - - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val startOffset = config.asInstanceOf[SimpleStreamingScanConfig] - .start.asInstanceOf[ContinuousMemoryStreamOffset] + override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = { synchronized { val endpointName = s"ContinuousMemoryStreamRecordEndpoint-${java.util.UUID.randomUUID()}-$id" endpointRef = recordEndpoint.rpcEnv.setupEndpoint(endpointName, recordEndpoint) startOffset.partitionNums.map { - case (part, index) => ContinuousMemoryStreamInputPartition(endpointName, part, index) - }.toArray + case (part, index) => + new ContinuousMemoryStreamInputPartition( + endpointName, part, index): InputPartition[InternalRow] + }.toList.asJava } } - override def createContinuousReaderFactory( - config: ScanConfig): ContinuousPartitionReaderFactory = { - ContinuousMemoryStreamReaderFactory - } - override def stop(): Unit = { if (endpointRef != null) recordEndpoint.rpcEnv.stop(endpointRef) } override def commit(end: Offset): Unit = {} - // ContinuousReadSupportProvider implementation + // ContinuousReadSupport implementation // This is necessary because of how StreamTest finds the source for AddDataMemory steps. - override def createContinuousReadSupport( + def createContinuousReader( + schema: Optional[StructType], checkpointLocation: String, - options: DataSourceOptions): ContinuousReadSupport = this + options: DataSourceOptions): ContinuousReader = { + this + } } object ContinuousMemoryStream { @@ -135,16 +141,12 @@ object ContinuousMemoryStream { /** * An input partition for continuous memory stream. */ -case class ContinuousMemoryStreamInputPartition( +class ContinuousMemoryStreamInputPartition( driverEndpointName: String, partition: Int, - startOffset: Int) extends InputPartition - -object ContinuousMemoryStreamReaderFactory extends ContinuousPartitionReaderFactory { - override def createReader(partition: InputPartition): ContinuousPartitionReader[InternalRow] = { - val p = partition.asInstanceOf[ContinuousMemoryStreamInputPartition] - new ContinuousMemoryStreamPartitionReader(p.driverEndpointName, p.partition, p.startOffset) - } + startOffset: Int) extends InputPartition[InternalRow] { + override def createPartitionReader: ContinuousMemoryStreamInputPartitionReader = + new ContinuousMemoryStreamInputPartitionReader(driverEndpointName, partition, startOffset) } /** @@ -152,10 +154,10 @@ object ContinuousMemoryStreamReaderFactory extends ContinuousPartitionReaderFact * * Polls the driver endpoint for new records. */ -class ContinuousMemoryStreamPartitionReader( +class ContinuousMemoryStreamInputPartitionReader( driverEndpointName: String, partition: Int, - startOffset: Int) extends ContinuousPartitionReader[InternalRow] { + startOffset: Int) extends ContinuousInputPartitionReader[InternalRow] { private val endpoint = RpcUtils.makeDriverRef( driverEndpointName, SparkEnv.get.conf, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriteSupportProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterProvider.scala similarity index 82% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriteSupportProvider.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterProvider.scala index 4218fd51ad20..e8ce21cc1204 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriteSupportProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterProvider.scala @@ -22,9 +22,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.python.PythonForeachWriter -import org.apache.spark.sql.sources.v2.{DataSourceOptions, StreamingWriteSupportProvider} -import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport} +import org.apache.spark.sql.sources.v2.{DataSourceOptions, StreamWriteSupport} +import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType @@ -37,21 +37,20 @@ import org.apache.spark.sql.types.StructType * a [[ExpressionEncoder]] or a direct converter function. * @tparam T The expected type of the sink. */ -case class ForeachWriteSupportProvider[T]( +case class ForeachWriterProvider[T]( writer: ForeachWriter[T], - converter: Either[ExpressionEncoder[T], InternalRow => T]) - extends StreamingWriteSupportProvider { + converter: Either[ExpressionEncoder[T], InternalRow => T]) extends StreamWriteSupport { - override def createStreamingWriteSupport( + override def createStreamWriter( queryId: String, schema: StructType, mode: OutputMode, - options: DataSourceOptions): StreamingWriteSupport = { - new StreamingWriteSupport { + options: DataSourceOptions): StreamWriter = { + new StreamWriter { override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} - override def createStreamingWriterFactory(): StreamingDataWriterFactory = { + override def createWriterFactory(): DataWriterFactory[InternalRow] = { val rowConverter: InternalRow => T = converter match { case Left(enc) => val boundEnc = enc.resolveAndBind( @@ -69,16 +68,16 @@ case class ForeachWriteSupportProvider[T]( } } -object ForeachWriteSupportProvider { +object ForeachWriterProvider { def apply[T]( writer: ForeachWriter[T], - encoder: ExpressionEncoder[T]): ForeachWriteSupportProvider[_] = { + encoder: ExpressionEncoder[T]): ForeachWriterProvider[_] = { writer match { case pythonWriter: PythonForeachWriter => - new ForeachWriteSupportProvider[UnsafeRow]( + new ForeachWriterProvider[UnsafeRow]( pythonWriter, Right((x: InternalRow) => x.asInstanceOf[UnsafeRow])) case _ => - new ForeachWriteSupportProvider[T](writer, Left(encoder)) + new ForeachWriterProvider[T](writer, Left(encoder)) } } } @@ -86,8 +85,8 @@ object ForeachWriteSupportProvider { case class ForeachWriterFactory[T]( writer: ForeachWriter[T], rowConverter: InternalRow => T) - extends StreamingDataWriterFactory { - override def createWriter( + extends DataWriterFactory[InternalRow] { + override def createDataWriter( partitionId: Int, taskId: Long, epochId: Long): ForeachDataWriter[T] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala deleted file mode 100644 index 9f88416871f8..000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala +++ /dev/null @@ -1,51 +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.execution.streaming.sources - -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.writer.{BatchWriteSupport, DataWriter, DataWriterFactory, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport} - -/** - * A [[BatchWriteSupport]] used to hook V2 stream writers into a microbatch plan. It implements - * the non-streaming interface, forwarding the epoch ID determined at construction to a wrapped - * streaming write support. - */ -class MicroBatchWritSupport(eppchId: Long, val writeSupport: StreamingWriteSupport) - extends BatchWriteSupport { - - override def commit(messages: Array[WriterCommitMessage]): Unit = { - writeSupport.commit(eppchId, messages) - } - - override def abort(messages: Array[WriterCommitMessage]): Unit = { - writeSupport.abort(eppchId, messages) - } - - override def createBatchWriterFactory(): DataWriterFactory = { - new MicroBatchWriterFactory(eppchId, writeSupport.createStreamingWriterFactory()) - } -} - -class MicroBatchWriterFactory(epochId: Long, streamingWriterFactory: StreamingDataWriterFactory) - extends DataWriterFactory { - - override def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = { - streamingWriterFactory.createWriter(partitionId, taskId, epochId) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWriter.scala similarity index 50% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchReadSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWriter.scala index 90680ea38fbd..2d43a7bb7787 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWriter.scala @@ -17,15 +17,21 @@ package org.apache.spark.sql.execution.streaming.sources -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.writer.{DataSourceWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter -// A special `MicroBatchReadSupport` that can get latestOffset with a start offset. -trait RateControlMicroBatchReadSupport extends MicroBatchReadSupport { - - override def latestOffset(): Offset = { - throw new IllegalAccessException( - "latestOffset should not be called for RateControlMicroBatchReadSupport") +/** + * A [[DataSourceWriter]] used to hook V2 stream writers into a microbatch plan. It implements + * the non-streaming interface, forwarding the batch ID determined at construction to a wrapped + * streaming writer. + */ +class MicroBatchWriter(batchId: Long, val writer: StreamWriter) extends DataSourceWriter { + override def commit(messages: Array[WriterCommitMessage]): Unit = { + writer.commit(batchId, messages) } - def latestOffset(start: Offset): Offset + override def abort(messages: Array[WriterCommitMessage]): Unit = writer.abort(batchId, messages) + + override def createWriterFactory(): DataWriterFactory[InternalRow] = writer.createWriterFactory() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala index ac3c71cc222b..f26e11d842b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala @@ -21,18 +21,17 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.writer.{BatchWriteSupport, DataWriter, DataWriterFactory, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingDataWriterFactory +import org.apache.spark.sql.sources.v2.writer.{DataSourceWriter, DataWriter, DataWriterFactory, WriterCommitMessage} /** * A simple [[DataWriterFactory]] whose tasks just pack rows into the commit message for delivery - * to a [[BatchWriteSupport]] on the driver. + * to a [[DataSourceWriter]] on the driver. * * Note that, because it sends all rows to the driver, this factory will generally be unsuitable * for production-quality sinks. It's intended for use in tests. */ -case object PackedRowWriterFactory extends StreamingDataWriterFactory { - override def createWriter( +case object PackedRowWriterFactory extends DataWriterFactory[InternalRow] { + override def createDataWriter( partitionId: Int, taskId: Long, epochId: Long): DataWriter[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala similarity index 78% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReadSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala index f5364047adff..9e0d95493216 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReader.scala @@ -19,24 +19,27 @@ package org.apache.spark.sql.execution.streaming.sources import java.io._ import java.nio.charset.StandardCharsets +import java.util.Optional import java.util.concurrent.TimeUnit +import scala.collection.JavaConverters._ + import org.apache.commons.io.IOUtils import org.apache.spark.internal.Logging import org.apache.spark.network.util.JavaUtils -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset} +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset} import org.apache.spark.sql.types.StructType import org.apache.spark.util.{ManualClock, SystemClock} -class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLocation: String) - extends MicroBatchReadSupport with Logging { +class RateStreamMicroBatchReader(options: DataSourceOptions, checkpointLocation: String) + extends MicroBatchReader with Logging { import RateStreamProvider._ private[sources] val clock = { @@ -103,30 +106,38 @@ class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLoca @volatile private var lastTimeMs: Long = creationTimeMs - override def initialOffset(): Offset = LongOffset(0L) + private var start: LongOffset = _ + private var end: LongOffset = _ - override def latestOffset(): Offset = { - val now = clock.getTimeMillis() - if (lastTimeMs < now) { - lastTimeMs = now - } - LongOffset(TimeUnit.MILLISECONDS.toSeconds(lastTimeMs - creationTimeMs)) - } + override def readSchema(): StructType = SCHEMA - override def deserializeOffset(json: String): Offset = { - LongOffset(json.toLong) + override def setOffsetRange(start: Optional[Offset], end: Optional[Offset]): Unit = { + this.start = start.orElse(LongOffset(0L)).asInstanceOf[LongOffset] + this.end = end.orElse { + val now = clock.getTimeMillis() + if (lastTimeMs < now) { + lastTimeMs = now + } + LongOffset(TimeUnit.MILLISECONDS.toSeconds(lastTimeMs - creationTimeMs)) + }.asInstanceOf[LongOffset] } - override def fullSchema(): StructType = SCHEMA + override def getStartOffset(): Offset = { + if (start == null) throw new IllegalStateException("start offset not set") + start + } + override def getEndOffset(): Offset = { + if (end == null) throw new IllegalStateException("end offset not set") + end + } - override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end)) + override def deserializeOffset(json: String): Offset = { + LongOffset(json.toLong) } - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val sc = config.asInstanceOf[SimpleStreamingScanConfig] - val startSeconds = sc.start.asInstanceOf[LongOffset].offset - val endSeconds = sc.end.get.asInstanceOf[LongOffset].offset + override def planInputPartitions(): java.util.List[InputPartition[InternalRow]] = { + val startSeconds = LongOffset.convert(start).map(_.offset).getOrElse(0L) + val endSeconds = LongOffset.convert(end).map(_.offset).getOrElse(0L) assert(startSeconds <= endSeconds, s"startSeconds($startSeconds) > endSeconds($endSeconds)") if (endSeconds > maxSeconds) { throw new ArithmeticException("Integer overflow. Max offset with " + @@ -142,7 +153,7 @@ class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLoca s"rangeStart: $rangeStart, rangeEnd: $rangeEnd") if (rangeStart == rangeEnd) { - return Array.empty + return List.empty.asJava } val localStartTimeMs = creationTimeMs + TimeUnit.SECONDS.toMillis(startSeconds) @@ -159,11 +170,8 @@ class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLoca (0 until numPartitions).map { p => new RateStreamMicroBatchInputPartition( p, numPartitions, rangeStart, rangeEnd, localStartTimeMs, relativeMsPerValue) - }.toArray - } - - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - RateStreamMicroBatchReaderFactory + : InputPartition[InternalRow] + }.toList.asJava } override def commit(end: Offset): Unit = {} @@ -175,29 +183,26 @@ class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLoca s"numPartitions=${options.get(NUM_PARTITIONS).orElse("default")}" } -case class RateStreamMicroBatchInputPartition( +class RateStreamMicroBatchInputPartition( partitionId: Int, numPartitions: Int, rangeStart: Long, rangeEnd: Long, localStartTimeMs: Long, - relativeMsPerValue: Double) extends InputPartition + relativeMsPerValue: Double) extends InputPartition[InternalRow] { -object RateStreamMicroBatchReaderFactory extends PartitionReaderFactory { - override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - val p = partition.asInstanceOf[RateStreamMicroBatchInputPartition] - new RateStreamMicroBatchPartitionReader(p.partitionId, p.numPartitions, p.rangeStart, - p.rangeEnd, p.localStartTimeMs, p.relativeMsPerValue) - } + override def createPartitionReader(): InputPartitionReader[InternalRow] = + new RateStreamMicroBatchInputPartitionReader(partitionId, numPartitions, rangeStart, rangeEnd, + localStartTimeMs, relativeMsPerValue) } -class RateStreamMicroBatchPartitionReader( +class RateStreamMicroBatchInputPartitionReader( partitionId: Int, numPartitions: Int, rangeStart: Long, rangeEnd: Long, localStartTimeMs: Long, - relativeMsPerValue: Double) extends PartitionReader[InternalRow] { + relativeMsPerValue: Double) extends InputPartitionReader[InternalRow] { private var count: Long = 0 override def next(): Boolean = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala index 6942dfbfe0ec..6bdd492f0cb3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala @@ -17,11 +17,14 @@ package org.apache.spark.sql.execution.streaming.sources +import java.util.Optional + import org.apache.spark.network.util.JavaUtils -import org.apache.spark.sql.execution.streaming.continuous.RateStreamContinuousReadSupport +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.execution.streaming.continuous.RateStreamContinuousReader import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, MicroBatchReadSupport} +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, MicroBatchReader} import org.apache.spark.sql.types._ /** @@ -39,12 +42,13 @@ import org.apache.spark.sql.types._ * be resource constrained, and `numPartitions` can be tweaked to help reach the desired speed. */ class RateStreamProvider extends DataSourceV2 - with MicroBatchReadSupportProvider with ContinuousReadSupportProvider with DataSourceRegister { + with MicroBatchReadSupport with ContinuousReadSupport with DataSourceRegister { import RateStreamProvider._ - override def createMicroBatchReadSupport( + override def createMicroBatchReader( + schema: Optional[StructType], checkpointLocation: String, - options: DataSourceOptions): MicroBatchReadSupport = { + options: DataSourceOptions): MicroBatchReader = { if (options.get(ROWS_PER_SECOND).isPresent) { val rowsPerSecond = options.get(ROWS_PER_SECOND).get().toLong if (rowsPerSecond <= 0) { @@ -70,14 +74,17 @@ class RateStreamProvider extends DataSourceV2 } } - new RateStreamMicroBatchReadSupport(options, checkpointLocation) + if (schema.isPresent) { + throw new AnalysisException("The rate source does not support a user-specified schema.") + } + + new RateStreamMicroBatchReader(options, checkpointLocation) } - override def createContinuousReadSupport( + override def createContinuousReader( + schema: Optional[StructType], checkpointLocation: String, - options: DataSourceOptions): ContinuousReadSupport = { - new RateStreamContinuousReadSupport(options) - } + options: DataSourceOptions): ContinuousReader = new RateStreamContinuousReader(options) override def shortName(): String = "rate" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala index c50dc7bcb8da..cb76e8650339 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala @@ -32,9 +32,9 @@ import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics} import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.{Append, Complete, Update} import org.apache.spark.sql.execution.streaming.{MemorySinkBase, Sink} -import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamingWriteSupportProvider} +import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamWriteSupport} import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport} +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType @@ -42,15 +42,13 @@ import org.apache.spark.sql.types.StructType * A sink that stores the results in memory. This [[Sink]] is primarily intended for use in unit * tests and does not provide durability. */ -class MemorySinkV2 extends DataSourceV2 with StreamingWriteSupportProvider - with MemorySinkBase with Logging { - - override def createStreamingWriteSupport( +class MemorySinkV2 extends DataSourceV2 with StreamWriteSupport with MemorySinkBase with Logging { + override def createStreamWriter( queryId: String, schema: StructType, mode: OutputMode, - options: DataSourceOptions): StreamingWriteSupport = { - new MemoryStreamingWriteSupport(this, mode, schema) + options: DataSourceOptions): StreamWriter = { + new MemoryStreamWriter(this, mode, schema) } private case class AddedData(batchId: Long, data: Array[Row]) @@ -122,13 +120,10 @@ class MemorySinkV2 extends DataSourceV2 with StreamingWriteSupportProvider case class MemoryWriterCommitMessage(partition: Int, data: Seq[Row]) extends WriterCommitMessage {} -class MemoryStreamingWriteSupport( - val sink: MemorySinkV2, outputMode: OutputMode, schema: StructType) - extends StreamingWriteSupport { +class MemoryStreamWriter(val sink: MemorySinkV2, outputMode: OutputMode, schema: StructType) + extends StreamWriter { - override def createStreamingWriterFactory: MemoryWriterFactory = { - MemoryWriterFactory(outputMode, schema) - } + override def createWriterFactory: MemoryWriterFactory = MemoryWriterFactory(outputMode, schema) override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { val newRows = messages.flatMap { @@ -143,19 +138,13 @@ class MemoryStreamingWriteSupport( } case class MemoryWriterFactory(outputMode: OutputMode, schema: StructType) - extends DataWriterFactory with StreamingDataWriterFactory { + extends DataWriterFactory[InternalRow] { - override def createWriter( - partitionId: Int, - taskId: Long): DataWriter[InternalRow] = { - new MemoryDataWriter(partitionId, outputMode, schema) - } - - override def createWriter( + override def createDataWriter( partitionId: Int, taskId: Long, epochId: Long): DataWriter[InternalRow] = { - createWriter(partitionId, taskId) + new MemoryDataWriter(partitionId, outputMode, schema) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala index b2a573eae504..874c479db95d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala @@ -20,10 +20,11 @@ package org.apache.spark.sql.execution.streaming.sources import java.io.{BufferedReader, InputStreamReader, IOException} import java.net.Socket import java.text.SimpleDateFormat -import java.util.{Calendar, Locale} +import java.util.{Calendar, List => JList, Locale, Optional} import java.util.concurrent.atomic.AtomicBoolean import javax.annotation.concurrent.GuardedBy +import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer import scala.util.{Failure, Success, Try} @@ -31,15 +32,16 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.streaming.{LongOffset, SimpleStreamingScanConfig, SimpleStreamingScanConfigBuilder} -import org.apache.spark.sql.execution.streaming.continuous.TextSocketContinuousReadSupport +import org.apache.spark.sql.execution.streaming.LongOffset +import org.apache.spark.sql.execution.streaming.continuous.TextSocketContinuousReader import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, DataSourceV2, MicroBatchReadSupportProvider} -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, MicroBatchReadSupport, Offset} +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, DataSourceV2, MicroBatchReadSupport} +import org.apache.spark.sql.sources.v2.reader.{InputPartition, InputPartitionReader} +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, MicroBatchReader, Offset} import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} import org.apache.spark.unsafe.types.UTF8String +// Shared object for micro-batch and continuous reader object TextSocketReader { val SCHEMA_REGULAR = StructType(StructField("value", StringType) :: Nil) val SCHEMA_TIMESTAMP = StructType(StructField("value", StringType) :: @@ -48,12 +50,14 @@ object TextSocketReader { } /** - * A MicroBatchReadSupport that reads text lines through a TCP socket, designed only for tutorials - * and debugging. This MicroBatchReadSupport will *not* work in production applications due to - * multiple reasons, including no support for fault recovery. + * A MicroBatchReader that reads text lines through a TCP socket, designed only for tutorials and + * debugging. This MicroBatchReader will *not* work in production applications due to multiple + * reasons, including no support for fault recovery. */ -class TextSocketMicroBatchReadSupport(options: DataSourceOptions) - extends MicroBatchReadSupport with Logging { +class TextSocketMicroBatchReader(options: DataSourceOptions) extends MicroBatchReader with Logging { + + private var startOffset: Offset = _ + private var endOffset: Offset = _ private val host: String = options.get("host").get() private val port: Int = options.get("port").get().toInt @@ -99,7 +103,7 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions) logWarning(s"Stream closed by $host:$port") return } - TextSocketMicroBatchReadSupport.this.synchronized { + TextSocketMicroBatchReader.this.synchronized { val newData = ( UTF8String.fromString(line), DateTimeUtils.fromMillis(Calendar.getInstance().getTimeInMillis) @@ -116,15 +120,24 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions) readThread.start() } - override def initialOffset(): Offset = LongOffset(-1L) + override def setOffsetRange(start: Optional[Offset], end: Optional[Offset]): Unit = synchronized { + startOffset = start.orElse(LongOffset(-1L)) + endOffset = end.orElse(currentOffset) + } - override def latestOffset(): Offset = currentOffset + override def getStartOffset(): Offset = { + Option(startOffset).getOrElse(throw new IllegalStateException("start offset not set")) + } + + override def getEndOffset(): Offset = { + Option(endOffset).getOrElse(throw new IllegalStateException("end offset not set")) + } override def deserializeOffset(json: String): Offset = { LongOffset(json.toLong) } - override def fullSchema(): StructType = { + override def readSchema(): StructType = { if (options.getBoolean("includeTimestamp", false)) { TextSocketReader.SCHEMA_TIMESTAMP } else { @@ -132,14 +145,12 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions) } } - override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end)) - } + override def planInputPartitions(): JList[InputPartition[InternalRow]] = { + assert(startOffset != null && endOffset != null, + "start offset and end offset should already be set before create read tasks.") - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val sc = config.asInstanceOf[SimpleStreamingScanConfig] - val startOrdinal = sc.start.asInstanceOf[LongOffset].offset.toInt + 1 - val endOrdinal = sc.end.get.asInstanceOf[LongOffset].offset.toInt + 1 + val startOrdinal = LongOffset.convert(startOffset).get.offset.toInt + 1 + val endOrdinal = LongOffset.convert(endOffset).get.offset.toInt + 1 // Internal buffer only holds the batches after lastOffsetCommitted val rawList = synchronized { @@ -161,29 +172,26 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions) slices(idx % numPartitions).append(r) } - slices.map(TextSocketInputPartition) - } + (0 until numPartitions).map { i => + val slice = slices(i) + new InputPartition[InternalRow] { + override def createPartitionReader(): InputPartitionReader[InternalRow] = + new InputPartitionReader[InternalRow] { + private var currentIdx = -1 - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - new PartitionReaderFactory { - override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - val slice = partition.asInstanceOf[TextSocketInputPartition].slice - new PartitionReader[InternalRow] { - private var currentIdx = -1 + override def next(): Boolean = { + currentIdx += 1 + currentIdx < slice.size + } - override def next(): Boolean = { - currentIdx += 1 - currentIdx < slice.size - } + override def get(): InternalRow = { + InternalRow(slice(currentIdx)._1, slice(currentIdx)._2) + } - override def get(): InternalRow = { - InternalRow(slice(currentIdx)._1, slice(currentIdx)._2) + override def close(): Unit = {} } - - override def close(): Unit = {} - } } - } + }.toList.asJava } override def commit(end: Offset): Unit = synchronized { @@ -219,11 +227,8 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions) override def toString: String = s"TextSocketV2[host: $host, port: $port]" } -case class TextSocketInputPartition(slice: ListBuffer[(UTF8String, Long)]) extends InputPartition - class TextSocketSourceProvider extends DataSourceV2 - with MicroBatchReadSupportProvider with ContinuousReadSupportProvider - with DataSourceRegister with Logging { + with MicroBatchReadSupport with ContinuousReadSupport with DataSourceRegister with Logging { private def checkParameters(params: DataSourceOptions): Unit = { logWarning("The socket source should not be used for production applications! " + @@ -243,18 +248,27 @@ class TextSocketSourceProvider extends DataSourceV2 } } - override def createMicroBatchReadSupport( + override def createMicroBatchReader( + schema: Optional[StructType], checkpointLocation: String, - options: DataSourceOptions): MicroBatchReadSupport = { + options: DataSourceOptions): MicroBatchReader = { checkParameters(options) - new TextSocketMicroBatchReadSupport(options) + if (schema.isPresent) { + throw new AnalysisException("The socket source does not support a user-specified schema.") + } + + new TextSocketMicroBatchReader(options) } - override def createContinuousReadSupport( + override def createContinuousReader( + schema: Optional[StructType], checkpointLocation: String, - options: DataSourceOptions): ContinuousReadSupport = { + options: DataSourceOptions): ContinuousReader = { checkParameters(options) - new TextSocketContinuousReadSupport(options) + if (schema.isPresent) { + throw new AnalysisException("The socket source does not support a user-specified schema.") + } + new TextSocketContinuousReader(options) } /** String that represents the format that this data source provider uses. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 2a4db4afbe00..7eb5db5ed0ee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming -import java.util.Locale +import java.util.{Locale, Optional} import scala.collection.JavaConverters._ @@ -28,8 +28,8 @@ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} import org.apache.spark.sql.sources.StreamSourceProvider -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, MicroBatchReadSupportProvider} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, MicroBatchReadSupport} +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, MicroBatchReadSupport} +import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReader import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -172,21 +172,19 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo case _ => None } ds match { - case s: MicroBatchReadSupportProvider => - var tempReadSupport: MicroBatchReadSupport = null + case s: MicroBatchReadSupport => + var tempReader: MicroBatchReader = null val schema = try { - val tmpCheckpointPath = Utils.createTempDir(namePrefix = s"tempCP").getCanonicalPath - tempReadSupport = if (userSpecifiedSchema.isDefined) { - s.createMicroBatchReadSupport(userSpecifiedSchema.get, tmpCheckpointPath, options) - } else { - s.createMicroBatchReadSupport(tmpCheckpointPath, options) - } - tempReadSupport.fullSchema() + tempReader = s.createMicroBatchReader( + Optional.ofNullable(userSpecifiedSchema.orNull), + Utils.createTempDir(namePrefix = s"temporaryReader").getCanonicalPath, + options) + tempReader.readSchema() } finally { // Stop tempReader to avoid side-effect thing - if (tempReadSupport != null) { - tempReadSupport.stop() - tempReadSupport = null + if (tempReader != null) { + tempReader.stop() + tempReader = null } } Dataset.ofRows( @@ -194,28 +192,16 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo StreamingRelationV2( s, source, extraOptions.toMap, schema.toAttributes, v1Relation)(sparkSession)) - case s: ContinuousReadSupportProvider => - var tempReadSupport: ContinuousReadSupport = null - val schema = try { - val tmpCheckpointPath = Utils.createTempDir(namePrefix = s"tempCP").getCanonicalPath - tempReadSupport = if (userSpecifiedSchema.isDefined) { - s.createContinuousReadSupport(userSpecifiedSchema.get, tmpCheckpointPath, options) - } else { - s.createContinuousReadSupport(tmpCheckpointPath, options) - } - tempReadSupport.fullSchema() - } finally { - // Stop tempReader to avoid side-effect thing - if (tempReadSupport != null) { - tempReadSupport.stop() - tempReadSupport = null - } - } + case s: ContinuousReadSupport => + val tempReader = s.createContinuousReader( + Optional.ofNullable(userSpecifiedSchema.orNull), + Utils.createTempDir(namePrefix = s"temporaryReader").getCanonicalPath, + options) Dataset.ofRows( sparkSession, StreamingRelationV2( s, source, extraOptions.toMap, - schema.toAttributes, v1Relation)(sparkSession)) + tempReader.readSchema().toAttributes, v1Relation)(sparkSession)) case _ => // Code path for data source v1. Dataset.ofRows(sparkSession, StreamingRelation(v1DataSource)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 7866e4f70f14..3b9a56ffdde4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -21,7 +21,7 @@ import java.util.Locale import scala.collection.JavaConverters._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.{InterfaceStability, Since} import org.apache.spark.api.java.function.VoidFunction2 import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes @@ -30,7 +30,7 @@ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.execution.streaming.sources._ -import org.apache.spark.sql.sources.v2.StreamingWriteSupportProvider +import org.apache.spark.sql.sources.v2.StreamWriteSupport /** * Interface used to write a streaming `Dataset` to external storage systems (e.g. file systems, @@ -270,7 +270,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { query } else if (source == "foreach") { assertNotPartitioned("foreach") - val sink = ForeachWriteSupportProvider[T](foreachWriter, ds.exprEnc) + val sink = ForeachWriterProvider[T](foreachWriter, ds.exprEnc) df.sparkSession.sessionState.streamingQueryManager.startQuery( extraOptions.get("queryName"), extraOptions.get("checkpointLocation"), @@ -299,8 +299,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { val ds = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) val disabledSources = df.sparkSession.sqlContext.conf.disabledV2StreamingWriters.split(",") val sink = ds.newInstance() match { - case w: StreamingWriteSupportProvider - if !disabledSources.contains(w.getClass.getCanonicalName) => w + case w: StreamWriteSupport if !disabledSources.contains(w.getClass.getCanonicalName) => w case _ => val ds = DataSource( df.sparkSession, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index cd52d991d55c..25bb05212d66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS -import org.apache.spark.sql.sources.v2.StreamingWriteSupportProvider +import org.apache.spark.sql.sources.v2.StreamWriteSupport import org.apache.spark.util.{Clock, SystemClock, Utils} /** @@ -256,7 +256,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo } (sink, trigger) match { - case (v2Sink: StreamingWriteSupportProvider, trigger: ContinuousTrigger) => + case (v2Sink: StreamWriteSupport, trigger: ContinuousTrigger) => if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) { UnsupportedOperationChecker.checkForContinuous(analyzedPlan, outputMode) } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java index 5602310219a7..e4cead9df429 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java @@ -24,71 +24,29 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.sources.Filter; import org.apache.spark.sql.sources.GreaterThan; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; import org.apache.spark.sql.sources.v2.DataSourceOptions; import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.ReadSupport; import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.StructType; -public class JavaAdvancedDataSourceV2 implements DataSourceV2, BatchReadSupportProvider { +public class JavaAdvancedDataSourceV2 implements DataSourceV2, ReadSupport { - public class ReadSupport extends JavaSimpleReadSupport { - @Override - public ScanConfigBuilder newScanConfigBuilder() { - return new AdvancedScanConfigBuilder(); - } - - @Override - public InputPartition[] planInputPartitions(ScanConfig config) { - Filter[] filters = ((AdvancedScanConfigBuilder) config).filters; - List res = new ArrayList<>(); - - Integer lowerBound = null; - for (Filter filter : filters) { - if (filter instanceof GreaterThan) { - GreaterThan f = (GreaterThan) filter; - if ("i".equals(f.attribute()) && f.value() instanceof Integer) { - lowerBound = (Integer) f.value(); - break; - } - } - } - - if (lowerBound == null) { - res.add(new JavaRangeInputPartition(0, 5)); - res.add(new JavaRangeInputPartition(5, 10)); - } else if (lowerBound < 4) { - res.add(new JavaRangeInputPartition(lowerBound + 1, 5)); - res.add(new JavaRangeInputPartition(5, 10)); - } else if (lowerBound < 9) { - res.add(new JavaRangeInputPartition(lowerBound + 1, 10)); - } - - return res.stream().toArray(InputPartition[]::new); - } - - @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { - StructType requiredSchema = ((AdvancedScanConfigBuilder) config).requiredSchema; - return new AdvancedReaderFactory(requiredSchema); - } - } - - public static class AdvancedScanConfigBuilder implements ScanConfigBuilder, ScanConfig, - SupportsPushDownFilters, SupportsPushDownRequiredColumns { + public class Reader implements DataSourceReader, SupportsPushDownRequiredColumns, + SupportsPushDownFilters { // Exposed for testing. public StructType requiredSchema = new StructType().add("i", "int").add("j", "int"); public Filter[] filters = new Filter[0]; @Override - public void pruneColumns(StructType requiredSchema) { - this.requiredSchema = requiredSchema; + public StructType readSchema() { + return requiredSchema; } @Override - public StructType readSchema() { - return requiredSchema; + public void pruneColumns(StructType requiredSchema) { + this.requiredSchema = requiredSchema; } @Override @@ -121,54 +79,79 @@ public Filter[] pushedFilters() { } @Override - public ScanConfig build() { - return this; + public List> planInputPartitions() { + List> res = new ArrayList<>(); + + Integer lowerBound = null; + for (Filter filter : filters) { + if (filter instanceof GreaterThan) { + GreaterThan f = (GreaterThan) filter; + if ("i".equals(f.attribute()) && f.value() instanceof Integer) { + lowerBound = (Integer) f.value(); + break; + } + } + } + + if (lowerBound == null) { + res.add(new JavaAdvancedInputPartition(0, 5, requiredSchema)); + res.add(new JavaAdvancedInputPartition(5, 10, requiredSchema)); + } else if (lowerBound < 4) { + res.add(new JavaAdvancedInputPartition(lowerBound + 1, 5, requiredSchema)); + res.add(new JavaAdvancedInputPartition(5, 10, requiredSchema)); + } else if (lowerBound < 9) { + res.add(new JavaAdvancedInputPartition(lowerBound + 1, 10, requiredSchema)); + } + + return res; } } - static class AdvancedReaderFactory implements PartitionReaderFactory { - StructType requiredSchema; + static class JavaAdvancedInputPartition implements InputPartition, + InputPartitionReader { + private int start; + private int end; + private StructType requiredSchema; - AdvancedReaderFactory(StructType requiredSchema) { + JavaAdvancedInputPartition(int start, int end, StructType requiredSchema) { + this.start = start; + this.end = end; this.requiredSchema = requiredSchema; } @Override - public PartitionReader createReader(InputPartition partition) { - JavaRangeInputPartition p = (JavaRangeInputPartition) partition; - return new PartitionReader() { - private int current = p.start - 1; - - @Override - public boolean next() throws IOException { - current += 1; - return current < p.end; - } + public InputPartitionReader createPartitionReader() { + return new JavaAdvancedInputPartition(start - 1, end, requiredSchema); + } - @Override - public InternalRow get() { - Object[] values = new Object[requiredSchema.size()]; - for (int i = 0; i < values.length; i++) { - if ("i".equals(requiredSchema.apply(i).name())) { - values[i] = current; - } else if ("j".equals(requiredSchema.apply(i).name())) { - values[i] = -current; - } - } - return new GenericInternalRow(values); + @Override + public boolean next() { + start += 1; + return start < end; + } + + @Override + public InternalRow get() { + Object[] values = new Object[requiredSchema.size()]; + for (int i = 0; i < values.length; i++) { + if ("i".equals(requiredSchema.apply(i).name())) { + values[i] = start; + } else if ("j".equals(requiredSchema.apply(i).name())) { + values[i] = -start; } + } + return new GenericInternalRow(values); + } - @Override - public void close() throws IOException { + @Override + public void close() throws IOException { - } - }; } } @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); + public DataSourceReader createReader(DataSourceOptions options) { + return new Reader(); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaBatchDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaBatchDataSourceV2.java new file mode 100644 index 000000000000..97d6176d0255 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaBatchDataSourceV2.java @@ -0,0 +1,114 @@ +/* + * 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 test.org.apache.spark.sql.sources.v2; + +import java.io.IOException; +import java.util.List; + +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector; +import org.apache.spark.sql.sources.v2.DataSourceOptions; +import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.ReadSupport; +import org.apache.spark.sql.sources.v2.reader.*; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.vectorized.ColumnVector; +import org.apache.spark.sql.vectorized.ColumnarBatch; + + +public class JavaBatchDataSourceV2 implements DataSourceV2, ReadSupport { + + class Reader implements DataSourceReader, SupportsScanColumnarBatch { + private final StructType schema = new StructType().add("i", "int").add("j", "int"); + + @Override + public StructType readSchema() { + return schema; + } + + @Override + public List> planBatchInputPartitions() { + return java.util.Arrays.asList( + new JavaBatchInputPartition(0, 50), new JavaBatchInputPartition(50, 90)); + } + } + + static class JavaBatchInputPartition + implements InputPartition, InputPartitionReader { + private int start; + private int end; + + private static final int BATCH_SIZE = 20; + + private OnHeapColumnVector i; + private OnHeapColumnVector j; + private ColumnarBatch batch; + + JavaBatchInputPartition(int start, int end) { + this.start = start; + this.end = end; + } + + @Override + public InputPartitionReader createPartitionReader() { + this.i = new OnHeapColumnVector(BATCH_SIZE, DataTypes.IntegerType); + this.j = new OnHeapColumnVector(BATCH_SIZE, DataTypes.IntegerType); + ColumnVector[] vectors = new ColumnVector[2]; + vectors[0] = i; + vectors[1] = j; + this.batch = new ColumnarBatch(vectors); + return this; + } + + @Override + public boolean next() { + i.reset(); + j.reset(); + int count = 0; + while (start < end && count < BATCH_SIZE) { + i.putInt(count, start); + j.putInt(count, -start); + start += 1; + count += 1; + } + + if (count == 0) { + return false; + } else { + batch.setNumRows(count); + return true; + } + } + + @Override + public ColumnarBatch get() { + return batch; + } + + @Override + public void close() throws IOException { + batch.close(); + } + } + + + @Override + public DataSourceReader createReader(DataSourceOptions options) { + return new Reader(); + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java deleted file mode 100644 index 28a933039831..000000000000 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java +++ /dev/null @@ -1,114 +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 test.org.apache.spark.sql.sources.v2; - -import java.io.IOException; - -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; -import org.apache.spark.sql.sources.v2.DataSourceOptions; -import org.apache.spark.sql.sources.v2.DataSourceV2; -import org.apache.spark.sql.sources.v2.reader.*; -import org.apache.spark.sql.types.DataTypes; -import org.apache.spark.sql.vectorized.ColumnVector; -import org.apache.spark.sql.vectorized.ColumnarBatch; - - -public class JavaColumnarDataSourceV2 implements DataSourceV2, BatchReadSupportProvider { - - class ReadSupport extends JavaSimpleReadSupport { - - @Override - public InputPartition[] planInputPartitions(ScanConfig config) { - InputPartition[] partitions = new InputPartition[2]; - partitions[0] = new JavaRangeInputPartition(0, 50); - partitions[1] = new JavaRangeInputPartition(50, 90); - return partitions; - } - - @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { - return new ColumnarReaderFactory(); - } - } - - static class ColumnarReaderFactory implements PartitionReaderFactory { - private static final int BATCH_SIZE = 20; - - @Override - public boolean supportColumnarReads(InputPartition partition) { - return true; - } - - @Override - public PartitionReader createReader(InputPartition partition) { - throw new UnsupportedOperationException(""); - } - - @Override - public PartitionReader createColumnarReader(InputPartition partition) { - JavaRangeInputPartition p = (JavaRangeInputPartition) partition; - OnHeapColumnVector i = new OnHeapColumnVector(BATCH_SIZE, DataTypes.IntegerType); - OnHeapColumnVector j = new OnHeapColumnVector(BATCH_SIZE, DataTypes.IntegerType); - ColumnVector[] vectors = new ColumnVector[2]; - vectors[0] = i; - vectors[1] = j; - ColumnarBatch batch = new ColumnarBatch(vectors); - - return new PartitionReader() { - private int current = p.start; - - @Override - public boolean next() throws IOException { - i.reset(); - j.reset(); - int count = 0; - while (current < p.end && count < BATCH_SIZE) { - i.putInt(count, current); - j.putInt(count, -current); - current += 1; - count += 1; - } - - if (count == 0) { - return false; - } else { - batch.setNumRows(count); - return true; - } - } - - @Override - public ColumnarBatch get() { - return batch; - } - - @Override - public void close() throws IOException { - batch.close(); - } - }; - } - } - - @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); - } -} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java index 18a11dde8219..2d21324f5ece 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java @@ -19,34 +19,38 @@ import java.io.IOException; import java.util.Arrays; +import java.util.List; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.sources.v2.*; +import org.apache.spark.sql.sources.v2.DataSourceOptions; +import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.ReadSupport; import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.sources.v2.reader.partitioning.ClusteredDistribution; import org.apache.spark.sql.sources.v2.reader.partitioning.Distribution; import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning; +import org.apache.spark.sql.types.StructType; -public class JavaPartitionAwareDataSource implements DataSourceV2, BatchReadSupportProvider { +public class JavaPartitionAwareDataSource implements DataSourceV2, ReadSupport { - class ReadSupport extends JavaSimpleReadSupport implements SupportsReportPartitioning { + class Reader implements DataSourceReader, SupportsReportPartitioning { + private final StructType schema = new StructType().add("a", "int").add("b", "int"); @Override - public InputPartition[] planInputPartitions(ScanConfig config) { - InputPartition[] partitions = new InputPartition[2]; - partitions[0] = new SpecificInputPartition(new int[]{1, 1, 3}, new int[]{4, 4, 6}); - partitions[1] = new SpecificInputPartition(new int[]{2, 4, 4}, new int[]{6, 2, 2}); - return partitions; + public StructType readSchema() { + return schema; } @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { - return new SpecificReaderFactory(); + public List> planInputPartitions() { + return java.util.Arrays.asList( + new SpecificInputPartition(new int[]{1, 1, 3}, new int[]{4, 4, 6}), + new SpecificInputPartition(new int[]{2, 4, 4}, new int[]{6, 2, 2})); } @Override - public Partitioning outputPartitioning(ScanConfig config) { + public Partitioning outputPartitioning() { return new MyPartitioning(); } } @@ -62,53 +66,50 @@ public int numPartitions() { public boolean satisfy(Distribution distribution) { if (distribution instanceof ClusteredDistribution) { String[] clusteredCols = ((ClusteredDistribution) distribution).clusteredColumns; - return Arrays.asList(clusteredCols).contains("i"); + return Arrays.asList(clusteredCols).contains("a"); } return false; } } - static class SpecificInputPartition implements InputPartition { - int[] i; - int[] j; + static class SpecificInputPartition implements InputPartition, + InputPartitionReader { + + private int[] i; + private int[] j; + private int current = -1; SpecificInputPartition(int[] i, int[] j) { assert i.length == j.length; this.i = i; this.j = j; } - } - static class SpecificReaderFactory implements PartitionReaderFactory { + @Override + public boolean next() throws IOException { + current += 1; + return current < i.length; + } + + @Override + public InternalRow get() { + return new GenericInternalRow(new Object[] {i[current], j[current]}); + } + + @Override + public void close() throws IOException { + + } @Override - public PartitionReader createReader(InputPartition partition) { - SpecificInputPartition p = (SpecificInputPartition) partition; - return new PartitionReader() { - private int current = -1; - - @Override - public boolean next() throws IOException { - current += 1; - return current < p.i.length; - } - - @Override - public InternalRow get() { - return new GenericInternalRow(new Object[] {p.i[current], p.j[current]}); - } - - @Override - public void close() throws IOException { - - } - }; + public InputPartitionReader createPartitionReader() { + return this; } } @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); + public DataSourceReader createReader(DataSourceOptions options) { + return new Reader(); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java index cc9ac04a0dad..6fd6a44d2c4d 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java @@ -17,39 +17,43 @@ package test.org.apache.spark.sql.sources.v2; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; +import java.util.List; + +import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.DataSourceOptions; import org.apache.spark.sql.sources.v2.DataSourceV2; -import org.apache.spark.sql.sources.v2.reader.*; +import org.apache.spark.sql.sources.v2.ReadSupport; +import org.apache.spark.sql.sources.v2.reader.DataSourceReader; +import org.apache.spark.sql.sources.v2.reader.InputPartition; import org.apache.spark.sql.types.StructType; -public class JavaSchemaRequiredDataSource implements DataSourceV2, BatchReadSupportProvider { +public class JavaSchemaRequiredDataSource implements DataSourceV2, ReadSupport { - class ReadSupport extends JavaSimpleReadSupport { + class Reader implements DataSourceReader { private final StructType schema; - ReadSupport(StructType schema) { + Reader(StructType schema) { this.schema = schema; } @Override - public StructType fullSchema() { + public StructType readSchema() { return schema; } @Override - public InputPartition[] planInputPartitions(ScanConfig config) { - return new InputPartition[0]; + public List> planInputPartitions() { + return java.util.Collections.emptyList(); } } @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { + public DataSourceReader createReader(DataSourceOptions options) { throw new IllegalArgumentException("requires a user-supplied schema"); } @Override - public BatchReadSupport createBatchReadSupport(StructType schema, DataSourceOptions options) { - return new ReadSupport(schema); + public DataSourceReader createReader(StructType schema, DataSourceOptions options) { + return new Reader(schema); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java index 2cdbba84ec4a..274dc3745bcf 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java @@ -17,26 +17,72 @@ package test.org.apache.spark.sql.sources.v2; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; +import java.io.IOException; +import java.util.List; + +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.sources.v2.DataSourceV2; import org.apache.spark.sql.sources.v2.DataSourceOptions; -import org.apache.spark.sql.sources.v2.reader.*; +import org.apache.spark.sql.sources.v2.ReadSupport; +import org.apache.spark.sql.sources.v2.reader.InputPartitionReader; +import org.apache.spark.sql.sources.v2.reader.InputPartition; +import org.apache.spark.sql.sources.v2.reader.DataSourceReader; +import org.apache.spark.sql.types.StructType; + +public class JavaSimpleDataSourceV2 implements DataSourceV2, ReadSupport { + + class Reader implements DataSourceReader { + private final StructType schema = new StructType().add("i", "int").add("j", "int"); + + @Override + public StructType readSchema() { + return schema; + } + + @Override + public List> planInputPartitions() { + return java.util.Arrays.asList( + new JavaSimpleInputPartition(0, 5), + new JavaSimpleInputPartition(5, 10)); + } + } + + static class JavaSimpleInputPartition implements InputPartition, + InputPartitionReader { -public class JavaSimpleDataSourceV2 implements DataSourceV2, BatchReadSupportProvider { + private int start; + private int end; - class ReadSupport extends JavaSimpleReadSupport { + JavaSimpleInputPartition(int start, int end) { + this.start = start; + this.end = end; + } + + @Override + public InputPartitionReader createPartitionReader() { + return new JavaSimpleInputPartition(start - 1, end); + } @Override - public InputPartition[] planInputPartitions(ScanConfig config) { - InputPartition[] partitions = new InputPartition[2]; - partitions[0] = new JavaRangeInputPartition(0, 5); - partitions[1] = new JavaRangeInputPartition(5, 10); - return partitions; + public boolean next() { + start += 1; + return start < end; + } + + @Override + public InternalRow get() { + return new GenericInternalRow(new Object[] {start, -start}); + } + + @Override + public void close() throws IOException { + } } @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); + public DataSourceReader createReader(DataSourceOptions options) { + return new Reader(); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java deleted file mode 100644 index 685f9b9747e8..000000000000 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java +++ /dev/null @@ -1,99 +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 test.org.apache.spark.sql.sources.v2; - -import java.io.IOException; - -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.sources.v2.reader.*; -import org.apache.spark.sql.types.StructType; - -abstract class JavaSimpleReadSupport implements BatchReadSupport { - - @Override - public StructType fullSchema() { - return new StructType().add("i", "int").add("j", "int"); - } - - @Override - public ScanConfigBuilder newScanConfigBuilder() { - return new JavaNoopScanConfigBuilder(fullSchema()); - } - - @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { - return new JavaSimpleReaderFactory(); - } -} - -class JavaNoopScanConfigBuilder implements ScanConfigBuilder, ScanConfig { - - private StructType schema; - - JavaNoopScanConfigBuilder(StructType schema) { - this.schema = schema; - } - - @Override - public ScanConfig build() { - return this; - } - - @Override - public StructType readSchema() { - return schema; - } -} - -class JavaSimpleReaderFactory implements PartitionReaderFactory { - - @Override - public PartitionReader createReader(InputPartition partition) { - JavaRangeInputPartition p = (JavaRangeInputPartition) partition; - return new PartitionReader() { - private int current = p.start - 1; - - @Override - public boolean next() throws IOException { - current += 1; - return current < p.end; - } - - @Override - public InternalRow get() { - return new GenericInternalRow(new Object[] {current, -current}); - } - - @Override - public void close() throws IOException { - - } - }; - } -} - -class JavaRangeInputPartition implements InputPartition { - int start; - int end; - - JavaRangeInputPartition(int start, int end) { - this.start = start; - this.end = end; - } -} diff --git a/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index a36b0cfa6ff1..46b38bed1c0f 100644 --- a/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -9,6 +9,6 @@ org.apache.spark.sql.streaming.sources.FakeReadMicroBatchOnly org.apache.spark.sql.streaming.sources.FakeReadContinuousOnly org.apache.spark.sql.streaming.sources.FakeReadBothModes org.apache.spark.sql.streaming.sources.FakeReadNeitherMode -org.apache.spark.sql.streaming.sources.FakeWriteSupportProvider +org.apache.spark.sql.streaming.sources.FakeWrite org.apache.spark.sql.streaming.sources.FakeNoWrite -org.apache.spark.sql.streaming.sources.FakeWriteSupportProviderV1Fallback +org.apache.spark.sql.streaming.sources.FakeWriteV1Fallback diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala index 61857365ac98..7bb2cf59f5ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala @@ -43,7 +43,7 @@ class MemorySinkV2Suite extends StreamTest with BeforeAndAfter { test("streaming writer") { val sink = new MemorySinkV2 - val writeSupport = new MemoryStreamingWriteSupport( + val writeSupport = new MemoryStreamWriter( sink, OutputMode.Append(), new StructType().add("i", "int")) writeSupport.commit(0, Array( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriterSuite.scala similarity index 98% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriterSuite.scala index 5884380271f0..55acf2ba28d2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupportSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriterSuite.scala @@ -19,10 +19,12 @@ package org.apache.spark.sql.execution.streaming.sources import java.io.ByteArrayOutputStream +import org.scalatest.time.SpanSugar._ + import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.streaming.{StreamTest, Trigger} -class ConsoleWriteSupportSuite extends StreamTest { +class ConsoleWriterSuite extends StreamTest { import testImplicits._ test("microbatch - default") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala index dd74af873c2e..5ca13b89735b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala @@ -17,18 +17,20 @@ package org.apache.spark.sql.execution.streaming.sources +import java.util.Optional import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.functions._ -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, MicroBatchReadSupportProvider} +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, MicroBatchReadSupport} import org.apache.spark.sql.sources.v2.reader.streaming.Offset import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.util.ManualClock @@ -41,7 +43,7 @@ class RateSourceSuite extends StreamTest { override def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) = { assert(query.nonEmpty) val rateSource = query.get.logicalPlan.collect { - case StreamingExecutionRelation(source: RateStreamMicroBatchReadSupport, _) => source + case StreamingExecutionRelation(source: RateStreamMicroBatchReader, _) => source }.head rateSource.clock.asInstanceOf[ManualClock].advance(TimeUnit.SECONDS.toMillis(seconds)) @@ -54,10 +56,10 @@ class RateSourceSuite extends StreamTest { test("microbatch in registry") { withTempDir { temp => DataSource.lookupDataSource("rate", spark.sqlContext.conf).newInstance() match { - case ds: MicroBatchReadSupportProvider => - val readSupport = ds.createMicroBatchReadSupport( - temp.getCanonicalPath, DataSourceOptions.empty()) - assert(readSupport.isInstanceOf[RateStreamMicroBatchReadSupport]) + case ds: MicroBatchReadSupport => + val reader = ds.createMicroBatchReader( + Optional.empty(), temp.getCanonicalPath, DataSourceOptions.empty()) + assert(reader.isInstanceOf[RateStreamMicroBatchReader]) case _ => throw new IllegalStateException("Could not find read support for rate") } @@ -67,7 +69,7 @@ class RateSourceSuite extends StreamTest { test("compatible with old path in registry") { DataSource.lookupDataSource("org.apache.spark.sql.execution.streaming.RateSourceProvider", spark.sqlContext.conf).newInstance() match { - case ds: MicroBatchReadSupportProvider => + case ds: MicroBatchReadSupport => assert(ds.isInstanceOf[RateStreamProvider]) case _ => throw new IllegalStateException("Could not find read support for rate") @@ -139,19 +141,30 @@ class RateSourceSuite extends StreamTest { ) } + test("microbatch - set offset") { + withTempDir { temp => + val reader = new RateStreamMicroBatchReader(DataSourceOptions.empty(), temp.getCanonicalPath) + val startOffset = LongOffset(0L) + val endOffset = LongOffset(1L) + reader.setOffsetRange(Optional.of(startOffset), Optional.of(endOffset)) + assert(reader.getStartOffset() == startOffset) + assert(reader.getEndOffset() == endOffset) + } + } + test("microbatch - infer offsets") { withTempDir { temp => - val readSupport = new RateStreamMicroBatchReadSupport( + val reader = new RateStreamMicroBatchReader( new DataSourceOptions( Map("numPartitions" -> "1", "rowsPerSecond" -> "100", "useManualClock" -> "true").asJava), temp.getCanonicalPath) - readSupport.clock.asInstanceOf[ManualClock].advance(100000) - val startOffset = readSupport.initialOffset() - startOffset match { + reader.clock.asInstanceOf[ManualClock].advance(100000) + reader.setOffsetRange(Optional.empty(), Optional.empty()) + reader.getStartOffset() match { case r: LongOffset => assert(r.offset === 0L) case _ => throw new IllegalStateException("unexpected offset type") } - readSupport.latestOffset() match { + reader.getEndOffset() match { case r: LongOffset => assert(r.offset >= 100) case _ => throw new IllegalStateException("unexpected offset type") } @@ -160,16 +173,15 @@ class RateSourceSuite extends StreamTest { test("microbatch - predetermined batch size") { withTempDir { temp => - val readSupport = new RateStreamMicroBatchReadSupport( + val reader = new RateStreamMicroBatchReader( new DataSourceOptions(Map("numPartitions" -> "1", "rowsPerSecond" -> "20").asJava), temp.getCanonicalPath) val startOffset = LongOffset(0L) val endOffset = LongOffset(1L) - val config = readSupport.newScanConfigBuilder(startOffset, endOffset).build() - val tasks = readSupport.planInputPartitions(config) - val readerFactory = readSupport.createReaderFactory(config) + reader.setOffsetRange(Optional.of(startOffset), Optional.of(endOffset)) + val tasks = reader.planInputPartitions() assert(tasks.size == 1) - val dataReader = readerFactory.createReader(tasks(0)) + val dataReader = tasks.get(0).createPartitionReader() val data = ArrayBuffer[InternalRow]() while (dataReader.next()) { data.append(dataReader.get()) @@ -180,25 +192,24 @@ class RateSourceSuite extends StreamTest { test("microbatch - data read") { withTempDir { temp => - val readSupport = new RateStreamMicroBatchReadSupport( + val reader = new RateStreamMicroBatchReader( new DataSourceOptions(Map("numPartitions" -> "11", "rowsPerSecond" -> "33").asJava), temp.getCanonicalPath) val startOffset = LongOffset(0L) val endOffset = LongOffset(1L) - val config = readSupport.newScanConfigBuilder(startOffset, endOffset).build() - val tasks = readSupport.planInputPartitions(config) - val readerFactory = readSupport.createReaderFactory(config) + reader.setOffsetRange(Optional.of(startOffset), Optional.of(endOffset)) + val tasks = reader.planInputPartitions() assert(tasks.size == 11) - val readData = tasks - .map(readerFactory.createReader) + val readData = tasks.asScala + .map(_.createPartitionReader()) .flatMap { reader => val buf = scala.collection.mutable.ListBuffer[InternalRow]() while (reader.next()) buf.append(reader.get()) buf } - assert(readData.map(_.getLong(1)).sorted === 0.until(33).toArray) + assert(readData.map(_.getLong(1)).sorted == Range(0, 33)) } } @@ -309,44 +320,41 @@ class RateSourceSuite extends StreamTest { } test("user-specified schema given") { - val exception = intercept[UnsupportedOperationException] { + val exception = intercept[AnalysisException] { spark.readStream .format("rate") .schema(spark.range(1).schema) .load() } assert(exception.getMessage.contains( - "rate source does not support user-specified schema")) + "rate source does not support a user-specified schema")) } test("continuous in registry") { DataSource.lookupDataSource("rate", spark.sqlContext.conf).newInstance() match { - case ds: ContinuousReadSupportProvider => - val readSupport = ds.createContinuousReadSupport( - "", DataSourceOptions.empty()) - assert(readSupport.isInstanceOf[RateStreamContinuousReadSupport]) + case ds: ContinuousReadSupport => + val reader = ds.createContinuousReader(Optional.empty(), "", DataSourceOptions.empty()) + assert(reader.isInstanceOf[RateStreamContinuousReader]) case _ => throw new IllegalStateException("Could not find read support for continuous rate") } } test("continuous data") { - val readSupport = new RateStreamContinuousReadSupport( + val reader = new RateStreamContinuousReader( new DataSourceOptions(Map("numPartitions" -> "2", "rowsPerSecond" -> "20").asJava)) - val config = readSupport.newScanConfigBuilder(readSupport.initialOffset).build() - val tasks = readSupport.planInputPartitions(config) - val readerFactory = readSupport.createContinuousReaderFactory(config) + reader.setStartOffset(Optional.empty()) + val tasks = reader.planInputPartitions() assert(tasks.size == 2) val data = scala.collection.mutable.ListBuffer[InternalRow]() - tasks.foreach { + tasks.asScala.foreach { case t: RateStreamContinuousInputPartition => - val startTimeMs = readSupport.initialOffset() + val startTimeMs = reader.getStartOffset() .asInstanceOf[RateStreamOffset] .partitionToValueAndRunTimeMs(t.partitionIndex) .runTimeMs - val r = readerFactory.createReader(t) - .asInstanceOf[RateStreamContinuousPartitionReader] + val r = t.createPartitionReader().asInstanceOf[RateStreamContinuousInputPartitionReader] for (rowIndex <- 0 to 9) { r.next() data.append(r.get()) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala index 409156e5ebc7..48e5cf75bf8b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala @@ -21,6 +21,7 @@ import java.net.{InetSocketAddress, SocketException} import java.nio.ByteBuffer import java.nio.channels.ServerSocketChannel import java.sql.Timestamp +import java.util.Optional import java.util.concurrent.LinkedBlockingQueue import scala.collection.JavaConverters._ @@ -33,8 +34,8 @@ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.{DataSourceOptions, MicroBatchReadSupportProvider} -import org.apache.spark.sql.sources.v2.reader.streaming.Offset +import org.apache.spark.sql.sources.v2.{DataSourceOptions, MicroBatchReadSupport} +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReader, Offset} import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -48,9 +49,14 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before serverThread.join() serverThread = null } + if (batchReader != null) { + batchReader.stop() + batchReader = null + } } private var serverThread: ServerThread = null + private var batchReader: MicroBatchReader = null case class AddSocketData(data: String*) extends AddData { override def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) = { @@ -59,7 +65,7 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before "Cannot add data when there is no query for finding the active socket source") val sources = query.get.logicalPlan.collect { - case StreamingExecutionRelation(source: TextSocketMicroBatchReadSupport, _) => source + case StreamingExecutionRelation(source: TextSocketMicroBatchReader, _) => source } if (sources.isEmpty) { throw new Exception( @@ -85,7 +91,7 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before test("backward compatibility with old path") { DataSource.lookupDataSource("org.apache.spark.sql.execution.streaming.TextSocketSourceProvider", spark.sqlContext.conf).newInstance() match { - case ds: MicroBatchReadSupportProvider => + case ds: MicroBatchReadSupport => assert(ds.isInstanceOf[TextSocketSourceProvider]) case _ => throw new IllegalStateException("Could not find socket source") @@ -175,16 +181,16 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before test("params not given") { val provider = new TextSocketSourceProvider intercept[AnalysisException] { - provider.createMicroBatchReadSupport( - "", new DataSourceOptions(Map.empty[String, String].asJava)) + provider.createMicroBatchReader(Optional.empty(), "", + new DataSourceOptions(Map.empty[String, String].asJava)) } intercept[AnalysisException] { - provider.createMicroBatchReadSupport( - "", new DataSourceOptions(Map("host" -> "localhost").asJava)) + provider.createMicroBatchReader(Optional.empty(), "", + new DataSourceOptions(Map("host" -> "localhost").asJava)) } intercept[AnalysisException] { - provider.createMicroBatchReadSupport( - "", new DataSourceOptions(Map("port" -> "1234").asJava)) + provider.createMicroBatchReader(Optional.empty(), "", + new DataSourceOptions(Map("port" -> "1234").asJava)) } } @@ -193,7 +199,7 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before val params = Map("host" -> "localhost", "port" -> "1234", "includeTimestamp" -> "fasle") intercept[AnalysisException] { val a = new DataSourceOptions(params.asJava) - provider.createMicroBatchReadSupport("", a) + provider.createMicroBatchReader(Optional.empty(), "", a) } } @@ -203,12 +209,12 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before StructField("name", StringType) :: StructField("area", StringType) :: Nil) val params = Map("host" -> "localhost", "port" -> "1234") - val exception = intercept[UnsupportedOperationException] { - provider.createMicroBatchReadSupport( - userSpecifiedSchema, "", new DataSourceOptions(params.asJava)) + val exception = intercept[AnalysisException] { + provider.createMicroBatchReader( + Optional.of(userSpecifiedSchema), "", new DataSourceOptions(params.asJava)) } assert(exception.getMessage.contains( - "socket source does not support user-specified schema")) + "socket source does not support a user-specified schema")) } test("input row metrics") { @@ -299,27 +305,25 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before serverThread = new ServerThread() serverThread.start() - val readSupport = new TextSocketContinuousReadSupport( + val reader = new TextSocketContinuousReader( new DataSourceOptions(Map("numPartitions" -> "2", "host" -> "localhost", "port" -> serverThread.port.toString).asJava)) - - val scanConfig = readSupport.newScanConfigBuilder(readSupport.initialOffset()).build() - val tasks = readSupport.planInputPartitions(scanConfig) + reader.setStartOffset(Optional.empty()) + val tasks = reader.planInputPartitions() assert(tasks.size == 2) val numRecords = 10 val data = scala.collection.mutable.ListBuffer[Int]() val offsets = scala.collection.mutable.ListBuffer[Int]() - val readerFactory = readSupport.createContinuousReaderFactory(scanConfig) import org.scalatest.time.SpanSugar._ failAfter(5 seconds) { // inject rows, read and check the data and offsets for (i <- 0 until numRecords) { serverThread.enqueue(i.toString) } - tasks.foreach { + tasks.asScala.foreach { case t: TextSocketContinuousInputPartition => - val r = readerFactory.createReader(t).asInstanceOf[TextSocketContinuousPartitionReader] + val r = t.createPartitionReader().asInstanceOf[TextSocketContinuousInputPartitionReader] for (i <- 0 until numRecords / 2) { r.next() offsets.append(r.getOffset().asInstanceOf[ContinuousRecordPartitionOffset].offset) @@ -335,15 +339,16 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before data.clear() case _ => throw new IllegalStateException("Unexpected task type") } - assert(readSupport.startOffset.offsets == List(3, 3)) - readSupport.commit(TextSocketOffset(List(5, 5))) - assert(readSupport.startOffset.offsets == List(5, 5)) + assert(reader.getStartOffset.asInstanceOf[TextSocketOffset].offsets == List(3, 3)) + reader.commit(TextSocketOffset(List(5, 5))) + assert(reader.getStartOffset.asInstanceOf[TextSocketOffset].offsets == List(5, 5)) } def commitOffset(partition: Int, offset: Int): Unit = { - val offsetsToCommit = readSupport.startOffset.offsets.updated(partition, offset) - readSupport.commit(TextSocketOffset(offsetsToCommit)) - assert(readSupport.startOffset.offsets == offsetsToCommit) + val offsetsToCommit = reader.getStartOffset.asInstanceOf[TextSocketOffset] + .offsets.updated(partition, offset) + reader.commit(TextSocketOffset(offsetsToCommit)) + assert(reader.getStartOffset.asInstanceOf[TextSocketOffset].offsets == offsetsToCommit) } } @@ -351,13 +356,14 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before serverThread = new ServerThread() serverThread.start() - val readSupport = new TextSocketContinuousReadSupport( + val reader = new TextSocketContinuousReader( new DataSourceOptions(Map("numPartitions" -> "2", "host" -> "localhost", "port" -> serverThread.port.toString).asJava)) - - readSupport.startOffset = TextSocketOffset(List(5, 5)) + reader.setStartOffset(Optional.of(TextSocketOffset(List(5, 5)))) + // ok to commit same offset + reader.setStartOffset(Optional.of(TextSocketOffset(List(5, 5)))) assertThrows[IllegalStateException] { - readSupport.commit(TextSocketOffset(List(6, 6))) + reader.commit(TextSocketOffset(List(6, 6))) } } @@ -365,12 +371,12 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before serverThread = new ServerThread() serverThread.start() - val readSupport = new TextSocketContinuousReadSupport( + val reader = new TextSocketContinuousReader( new DataSourceOptions(Map("numPartitions" -> "2", "host" -> "localhost", "includeTimestamp" -> "true", "port" -> serverThread.port.toString).asJava)) - val scanConfig = readSupport.newScanConfigBuilder(readSupport.initialOffset()).build() - val tasks = readSupport.planInputPartitions(scanConfig) + reader.setStartOffset(Optional.empty()) + val tasks = reader.planInputPartitions() assert(tasks.size == 2) val numRecords = 4 @@ -378,10 +384,9 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before for (i <- 0 until numRecords) { serverThread.enqueue(i.toString) } - val readerFactory = readSupport.createContinuousReaderFactory(scanConfig) - tasks.foreach { + tasks.asScala.foreach { case t: TextSocketContinuousInputPartition => - val r = readerFactory.createReader(t).asInstanceOf[TextSocketContinuousPartitionReader] + val r = t.createPartitionReader().asInstanceOf[TextSocketContinuousInputPartitionReader] for (i <- 0 until numRecords / 2) { r.next() assert(r.get().get(0, TextSocketReader.SCHEMA_TIMESTAMP) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala index f6c3e0ce82e3..12beca257a0a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.sources.v2 +import java.util.{ArrayList, List => JList} + import test.org.apache.spark.sql.sources.v2._ import org.apache.spark.SparkException @@ -36,21 +38,6 @@ import org.apache.spark.sql.vectorized.ColumnarBatch class DataSourceV2Suite extends QueryTest with SharedSQLContext { import testImplicits._ - private def getScanConfig(query: DataFrame): AdvancedScanConfigBuilder = { - query.queryExecution.executedPlan.collect { - case d: DataSourceV2ScanExec => - d.scanConfig.asInstanceOf[AdvancedScanConfigBuilder] - }.head - } - - private def getJavaScanConfig( - query: DataFrame): JavaAdvancedDataSourceV2.AdvancedScanConfigBuilder = { - query.queryExecution.executedPlan.collect { - case d: DataSourceV2ScanExec => - d.scanConfig.asInstanceOf[JavaAdvancedDataSourceV2.AdvancedScanConfigBuilder] - }.head - } - test("simplest implementation") { Seq(classOf[SimpleDataSourceV2], classOf[JavaSimpleDataSourceV2]).foreach { cls => withClue(cls.getName) { @@ -63,6 +50,18 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { } test("advanced implementation") { + def getReader(query: DataFrame): AdvancedDataSourceV2#Reader = { + query.queryExecution.executedPlan.collect { + case d: DataSourceV2ScanExec => d.reader.asInstanceOf[AdvancedDataSourceV2#Reader] + }.head + } + + def getJavaReader(query: DataFrame): JavaAdvancedDataSourceV2#Reader = { + query.queryExecution.executedPlan.collect { + case d: DataSourceV2ScanExec => d.reader.asInstanceOf[JavaAdvancedDataSourceV2#Reader] + }.head + } + Seq(classOf[AdvancedDataSourceV2], classOf[JavaAdvancedDataSourceV2]).foreach { cls => withClue(cls.getName) { val df = spark.read.format(cls.getName).load() @@ -71,58 +70,58 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { val q1 = df.select('j) checkAnswer(q1, (0 until 10).map(i => Row(-i))) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q1) - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + val reader = getReader(q1) + assert(reader.filters.isEmpty) + assert(reader.requiredSchema.fieldNames === Seq("j")) } else { - val config = getJavaScanConfig(q1) - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + val reader = getJavaReader(q1) + assert(reader.filters.isEmpty) + assert(reader.requiredSchema.fieldNames === Seq("j")) } val q2 = df.filter('i > 3) checkAnswer(q2, (4 until 10).map(i => Row(i, -i))) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q2) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i", "j")) + val reader = getReader(q2) + assert(reader.filters.flatMap(_.references).toSet == Set("i")) + assert(reader.requiredSchema.fieldNames === Seq("i", "j")) } else { - val config = getJavaScanConfig(q2) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i", "j")) + val reader = getJavaReader(q2) + assert(reader.filters.flatMap(_.references).toSet == Set("i")) + assert(reader.requiredSchema.fieldNames === Seq("i", "j")) } val q3 = df.select('i).filter('i > 6) checkAnswer(q3, (7 until 10).map(i => Row(i))) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q3) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i")) + val reader = getReader(q3) + assert(reader.filters.flatMap(_.references).toSet == Set("i")) + assert(reader.requiredSchema.fieldNames === Seq("i")) } else { - val config = getJavaScanConfig(q3) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i")) + val reader = getJavaReader(q3) + assert(reader.filters.flatMap(_.references).toSet == Set("i")) + assert(reader.requiredSchema.fieldNames === Seq("i")) } val q4 = df.select('j).filter('j < -10) checkAnswer(q4, Nil) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q4) + val reader = getReader(q4) // 'j < 10 is not supported by the testing data source. - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + assert(reader.filters.isEmpty) + assert(reader.requiredSchema.fieldNames === Seq("j")) } else { - val config = getJavaScanConfig(q4) + val reader = getJavaReader(q4) // 'j < 10 is not supported by the testing data source. - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + assert(reader.filters.isEmpty) + assert(reader.requiredSchema.fieldNames === Seq("j")) } } } } test("columnar batch scan implementation") { - Seq(classOf[ColumnarDataSourceV2], classOf[JavaColumnarDataSourceV2]).foreach { cls => + Seq(classOf[BatchDataSourceV2], classOf[JavaBatchDataSourceV2]).foreach { cls => withClue(cls.getName) { val df = spark.read.format(cls.getName).load() checkAnswer(df, (0 until 90).map(i => Row(i, -i))) @@ -154,25 +153,25 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { val df = spark.read.format(cls.getName).load() checkAnswer(df, Seq(Row(1, 4), Row(1, 4), Row(3, 6), Row(2, 6), Row(4, 2), Row(4, 2))) - val groupByColA = df.groupBy('i).agg(sum('j)) + val groupByColA = df.groupBy('a).agg(sum('b)) checkAnswer(groupByColA, Seq(Row(1, 8), Row(2, 6), Row(3, 6), Row(4, 4))) assert(groupByColA.queryExecution.executedPlan.collectFirst { case e: ShuffleExchangeExec => e }.isEmpty) - val groupByColAB = df.groupBy('i, 'j).agg(count("*")) + val groupByColAB = df.groupBy('a, 'b).agg(count("*")) checkAnswer(groupByColAB, Seq(Row(1, 4, 2), Row(2, 6, 1), Row(3, 6, 1), Row(4, 2, 2))) assert(groupByColAB.queryExecution.executedPlan.collectFirst { case e: ShuffleExchangeExec => e }.isEmpty) - val groupByColB = df.groupBy('j).agg(sum('i)) + val groupByColB = df.groupBy('b).agg(sum('a)) checkAnswer(groupByColB, Seq(Row(2, 8), Row(4, 2), Row(6, 5))) assert(groupByColB.queryExecution.executedPlan.collectFirst { case e: ShuffleExchangeExec => e }.isDefined) - val groupByAPlusB = df.groupBy('i + 'j).agg(count("*")) + val groupByAPlusB = df.groupBy('a + 'b).agg(count("*")) checkAnswer(groupByAPlusB, Seq(Row(5, 2), Row(6, 2), Row(8, 1), Row(9, 1))) assert(groupByAPlusB.queryExecution.executedPlan.collectFirst { case e: ShuffleExchangeExec => e @@ -273,30 +272,36 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { } test("SPARK-23301: column pruning with arbitrary expressions") { + def getReader(query: DataFrame): AdvancedDataSourceV2#Reader = { + query.queryExecution.executedPlan.collect { + case d: DataSourceV2ScanExec => d.reader.asInstanceOf[AdvancedDataSourceV2#Reader] + }.head + } + val df = spark.read.format(classOf[AdvancedDataSourceV2].getName).load() val q1 = df.select('i + 1) checkAnswer(q1, (1 until 11).map(i => Row(i))) - val config1 = getScanConfig(q1) - assert(config1.requiredSchema.fieldNames === Seq("i")) + val reader1 = getReader(q1) + assert(reader1.requiredSchema.fieldNames === Seq("i")) val q2 = df.select(lit(1)) checkAnswer(q2, (0 until 10).map(i => Row(1))) - val config2 = getScanConfig(q2) - assert(config2.requiredSchema.isEmpty) + val reader2 = getReader(q2) + assert(reader2.requiredSchema.isEmpty) // 'j === 1 can't be pushed down, but we should still be able do column pruning val q3 = df.filter('j === -1).select('j * 2) checkAnswer(q3, Row(-2)) - val config3 = getScanConfig(q3) - assert(config3.filters.isEmpty) - assert(config3.requiredSchema.fieldNames === Seq("j")) + val reader3 = getReader(q3) + assert(reader3.filters.isEmpty) + assert(reader3.requiredSchema.fieldNames === Seq("j")) // column pruning should work with other operators. val q4 = df.sort('i).limit(1).select('i + 1) checkAnswer(q4, Row(1)) - val config4 = getScanConfig(q4) - assert(config4.requiredSchema.fieldNames === Seq("i")) + val reader4 = getReader(q4) + assert(reader4.requiredSchema.fieldNames === Seq("i")) } test("SPARK-23315: get output from canonicalized data source v2 related plans") { @@ -319,291 +324,240 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { } } +class SimpleSinglePartitionSource extends DataSourceV2 with ReadSupport { -case class RangeInputPartition(start: Int, end: Int) extends InputPartition - -case class NoopScanConfigBuilder(readSchema: StructType) extends ScanConfigBuilder with ScanConfig { - override def build(): ScanConfig = this -} - -object SimpleReaderFactory extends PartitionReaderFactory { - override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - val RangeInputPartition(start, end) = partition - new PartitionReader[InternalRow] { - private var current = start - 1 - - override def next(): Boolean = { - current += 1 - current < end - } - - override def get(): InternalRow = InternalRow(current, -current) + class Reader extends DataSourceReader { + override def readSchema(): StructType = new StructType().add("i", "int").add("j", "int") - override def close(): Unit = {} + override def planInputPartitions(): JList[InputPartition[InternalRow]] = { + java.util.Arrays.asList(new SimpleInputPartition(0, 5)) } } + + override def createReader(options: DataSourceOptions): DataSourceReader = new Reader } -abstract class SimpleReadSupport extends BatchReadSupport { - override def fullSchema(): StructType = new StructType().add("i", "int").add("j", "int") +// This class is used by pyspark tests. If this class is modified/moved, make sure pyspark +// tests still pass. +class SimpleDataSourceV2 extends DataSourceV2 with ReadSupport { - override def newScanConfigBuilder(): ScanConfigBuilder = { - NoopScanConfigBuilder(fullSchema()) - } + class Reader extends DataSourceReader { + override def readSchema(): StructType = new StructType().add("i", "int").add("j", "int") - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - SimpleReaderFactory + override def planInputPartitions(): JList[InputPartition[InternalRow]] = { + java.util.Arrays.asList(new SimpleInputPartition(0, 5), new SimpleInputPartition(5, 10)) + } } + + override def createReader(options: DataSourceOptions): DataSourceReader = new Reader } +class SimpleInputPartition(start: Int, end: Int) + extends InputPartition[InternalRow] + with InputPartitionReader[InternalRow] { + private var current = start - 1 -class SimpleSinglePartitionSource extends DataSourceV2 with BatchReadSupportProvider { + override def createPartitionReader(): InputPartitionReader[InternalRow] = + new SimpleInputPartition(start, end) - class ReadSupport extends SimpleReadSupport { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - Array(RangeInputPartition(0, 5)) - } + override def next(): Boolean = { + current += 1 + current < end } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport - } + override def get(): InternalRow = InternalRow(current, -current) + + override def close(): Unit = {} } -// This class is used by pyspark tests. If this class is modified/moved, make sure pyspark -// tests still pass. -class SimpleDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider { +class AdvancedDataSourceV2 extends DataSourceV2 with ReadSupport { - class ReadSupport extends SimpleReadSupport { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - Array(RangeInputPartition(0, 5), RangeInputPartition(5, 10)) - } - } + class Reader extends DataSourceReader + with SupportsPushDownRequiredColumns with SupportsPushDownFilters { - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport - } -} + var requiredSchema = new StructType().add("i", "int").add("j", "int") + var filters = Array.empty[Filter] + override def pruneColumns(requiredSchema: StructType): Unit = { + this.requiredSchema = requiredSchema + } -class AdvancedDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider { + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + val (supported, unsupported) = filters.partition { + case GreaterThan("i", _: Int) => true + case _ => false + } + this.filters = supported + unsupported + } - class ReadSupport extends SimpleReadSupport { - override def newScanConfigBuilder(): ScanConfigBuilder = new AdvancedScanConfigBuilder() + override def pushedFilters(): Array[Filter] = filters - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val filters = config.asInstanceOf[AdvancedScanConfigBuilder].filters + override def readSchema(): StructType = { + requiredSchema + } + override def planInputPartitions(): JList[InputPartition[InternalRow]] = { val lowerBound = filters.collectFirst { case GreaterThan("i", v: Int) => v } - val res = scala.collection.mutable.ArrayBuffer.empty[InputPartition] + val res = new ArrayList[InputPartition[InternalRow]] if (lowerBound.isEmpty) { - res.append(RangeInputPartition(0, 5)) - res.append(RangeInputPartition(5, 10)) + res.add(new AdvancedInputPartition(0, 5, requiredSchema)) + res.add(new AdvancedInputPartition(5, 10, requiredSchema)) } else if (lowerBound.get < 4) { - res.append(RangeInputPartition(lowerBound.get + 1, 5)) - res.append(RangeInputPartition(5, 10)) + res.add(new AdvancedInputPartition(lowerBound.get + 1, 5, requiredSchema)) + res.add(new AdvancedInputPartition(5, 10, requiredSchema)) } else if (lowerBound.get < 9) { - res.append(RangeInputPartition(lowerBound.get + 1, 10)) + res.add(new AdvancedInputPartition(lowerBound.get + 1, 10, requiredSchema)) } - res.toArray - } - - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - val requiredSchema = config.asInstanceOf[AdvancedScanConfigBuilder].requiredSchema - new AdvancedReaderFactory(requiredSchema) + res } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport - } + override def createReader(options: DataSourceOptions): DataSourceReader = new Reader } -class AdvancedScanConfigBuilder extends ScanConfigBuilder with ScanConfig - with SupportsPushDownRequiredColumns with SupportsPushDownFilters { +class AdvancedInputPartition(start: Int, end: Int, requiredSchema: StructType) + extends InputPartition[InternalRow] with InputPartitionReader[InternalRow] { - var requiredSchema = new StructType().add("i", "int").add("j", "int") - var filters = Array.empty[Filter] + private var current = start - 1 - override def pruneColumns(requiredSchema: StructType): Unit = { - this.requiredSchema = requiredSchema + override def createPartitionReader(): InputPartitionReader[InternalRow] = { + new AdvancedInputPartition(start, end, requiredSchema) } - override def readSchema(): StructType = requiredSchema + override def close(): Unit = {} - override def pushFilters(filters: Array[Filter]): Array[Filter] = { - val (supported, unsupported) = filters.partition { - case GreaterThan("i", _: Int) => true - case _ => false - } - this.filters = supported - unsupported + override def next(): Boolean = { + current += 1 + current < end } - override def pushedFilters(): Array[Filter] = filters - - override def build(): ScanConfig = this -} - -class AdvancedReaderFactory(requiredSchema: StructType) extends PartitionReaderFactory { - override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - val RangeInputPartition(start, end) = partition - new PartitionReader[InternalRow] { - private var current = start - 1 - - override def next(): Boolean = { - current += 1 - current < end - } - - override def get(): InternalRow = { - val values = requiredSchema.map(_.name).map { - case "i" => current - case "j" => -current - } - InternalRow.fromSeq(values) - } - - override def close(): Unit = {} + override def get(): InternalRow = { + val values = requiredSchema.map(_.name).map { + case "i" => current + case "j" => -current } + InternalRow.fromSeq(values) } } -class SchemaRequiredDataSource extends DataSourceV2 with BatchReadSupportProvider { +class SchemaRequiredDataSource extends DataSourceV2 with ReadSupport { - class ReadSupport(val schema: StructType) extends SimpleReadSupport { - override def fullSchema(): StructType = schema - - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = - Array.empty + class Reader(val readSchema: StructType) extends DataSourceReader { + override def planInputPartitions(): JList[InputPartition[InternalRow]] = + java.util.Collections.emptyList() } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { + override def createReader(options: DataSourceOptions): DataSourceReader = { throw new IllegalArgumentException("requires a user-supplied schema") } - override def createBatchReadSupport( - schema: StructType, options: DataSourceOptions): BatchReadSupport = { - new ReadSupport(schema) + override def createReader(schema: StructType, options: DataSourceOptions): DataSourceReader = { + new Reader(schema) } } -class ColumnarDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider { +class BatchDataSourceV2 extends DataSourceV2 with ReadSupport { - class ReadSupport extends SimpleReadSupport { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - Array(RangeInputPartition(0, 50), RangeInputPartition(50, 90)) - } + class Reader extends DataSourceReader with SupportsScanColumnarBatch { + override def readSchema(): StructType = new StructType().add("i", "int").add("j", "int") - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - ColumnarReaderFactory + override def planBatchInputPartitions(): JList[InputPartition[ColumnarBatch]] = { + java.util.Arrays.asList( + new BatchInputPartitionReader(0, 50), new BatchInputPartitionReader(50, 90)) } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport - } + override def createReader(options: DataSourceOptions): DataSourceReader = new Reader } -object ColumnarReaderFactory extends PartitionReaderFactory { - private final val BATCH_SIZE = 20 +class BatchInputPartitionReader(start: Int, end: Int) + extends InputPartition[ColumnarBatch] with InputPartitionReader[ColumnarBatch] { - override def supportColumnarReads(partition: InputPartition): Boolean = true + private final val BATCH_SIZE = 20 + private lazy val i = new OnHeapColumnVector(BATCH_SIZE, IntegerType) + private lazy val j = new OnHeapColumnVector(BATCH_SIZE, IntegerType) + private lazy val batch = new ColumnarBatch(Array(i, j)) - override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - throw new UnsupportedOperationException - } + private var current = start - override def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch] = { - val RangeInputPartition(start, end) = partition - new PartitionReader[ColumnarBatch] { - private lazy val i = new OnHeapColumnVector(BATCH_SIZE, IntegerType) - private lazy val j = new OnHeapColumnVector(BATCH_SIZE, IntegerType) - private lazy val batch = new ColumnarBatch(Array(i, j)) - - private var current = start - - override def next(): Boolean = { - i.reset() - j.reset() - - var count = 0 - while (current < end && count < BATCH_SIZE) { - i.putInt(count, current) - j.putInt(count, -current) - current += 1 - count += 1 - } + override def createPartitionReader(): InputPartitionReader[ColumnarBatch] = this - if (count == 0) { - false - } else { - batch.setNumRows(count) - true - } - } + override def next(): Boolean = { + i.reset() + j.reset() - override def get(): ColumnarBatch = batch + var count = 0 + while (current < end && count < BATCH_SIZE) { + i.putInt(count, current) + j.putInt(count, -current) + current += 1 + count += 1 + } - override def close(): Unit = batch.close() + if (count == 0) { + false + } else { + batch.setNumRows(count) + true } } + + override def get(): ColumnarBatch = { + batch + } + + override def close(): Unit = batch.close() } +class PartitionAwareDataSource extends DataSourceV2 with ReadSupport { -class PartitionAwareDataSource extends DataSourceV2 with BatchReadSupportProvider { + class Reader extends DataSourceReader with SupportsReportPartitioning { + override def readSchema(): StructType = new StructType().add("a", "int").add("b", "int") - class ReadSupport extends SimpleReadSupport with SupportsReportPartitioning { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + override def planInputPartitions(): JList[InputPartition[InternalRow]] = { // Note that we don't have same value of column `a` across partitions. - Array( - SpecificInputPartition(Array(1, 1, 3), Array(4, 4, 6)), - SpecificInputPartition(Array(2, 4, 4), Array(6, 2, 2))) - } - - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - SpecificReaderFactory + java.util.Arrays.asList( + new SpecificInputPartitionReader(Array(1, 1, 3), Array(4, 4, 6)), + new SpecificInputPartitionReader(Array(2, 4, 4), Array(6, 2, 2))) } - override def outputPartitioning(config: ScanConfig): Partitioning = new MyPartitioning + override def outputPartitioning(): Partitioning = new MyPartitioning } class MyPartitioning extends Partitioning { override def numPartitions(): Int = 2 override def satisfy(distribution: Distribution): Boolean = distribution match { - case c: ClusteredDistribution => c.clusteredColumns.contains("i") + case c: ClusteredDistribution => c.clusteredColumns.contains("a") case _ => false } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport - } + override def createReader(options: DataSourceOptions): DataSourceReader = new Reader } -case class SpecificInputPartition(i: Array[Int], j: Array[Int]) extends InputPartition +class SpecificInputPartitionReader(i: Array[Int], j: Array[Int]) + extends InputPartition[InternalRow] + with InputPartitionReader[InternalRow] { + assert(i.length == j.length) -object SpecificReaderFactory extends PartitionReaderFactory { - override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - val p = partition.asInstanceOf[SpecificInputPartition] - new PartitionReader[InternalRow] { - private var current = -1 + private var current = -1 - override def next(): Boolean = { - current += 1 - current < p.i.length - } + override def createPartitionReader(): InputPartitionReader[InternalRow] = this - override def get(): InternalRow = InternalRow(p.i(current), p.j(current)) - - override def close(): Unit = {} - } + override def next(): Boolean = { + current += 1 + current < i.length } + + override def get(): InternalRow = InternalRow(i(current), j(current)) + + override def close(): Unit = {} } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala index 952241b0b6be..e1b8e9c44d72 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala @@ -18,36 +18,34 @@ package org.apache.spark.sql.sources.v2 import java.io.{BufferedReader, InputStreamReader, IOException} -import java.util.Optional +import java.util.{Collections, List => JList, Optional} import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.{FileSystem, FSDataInputStream, Path} import org.apache.spark.SparkContext import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.sources.v2.reader.{DataSourceReader, InputPartition, InputPartitionReader} import org.apache.spark.sql.sources.v2.writer._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.util.SerializableConfiguration /** * A HDFS based transactional writable data source. - * Each task writes data to `target/_temporary/queryId/$jobId-$partitionId-$attemptNumber`. - * Each job moves files from `target/_temporary/queryId/` to `target`. + * Each task writes data to `target/_temporary/jobId/$jobId-$partitionId-$attemptNumber`. + * Each job moves files from `target/_temporary/jobId/` to `target`. */ -class SimpleWritableDataSource extends DataSourceV2 - with BatchReadSupportProvider with BatchWriteSupportProvider { +class SimpleWritableDataSource extends DataSourceV2 with ReadSupport with WriteSupport { private val schema = new StructType().add("i", "long").add("j", "long") - class ReadSupport(path: String, conf: Configuration) extends SimpleReadSupport { + class Reader(path: String, conf: Configuration) extends DataSourceReader { + override def readSchema(): StructType = schema - override def fullSchema(): StructType = schema - - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + override def planInputPartitions(): JList[InputPartition[InternalRow]] = { val dataPath = new Path(path) val fs = dataPath.getFileSystem(conf) if (fs.exists(dataPath)) { @@ -55,23 +53,21 @@ class SimpleWritableDataSource extends DataSourceV2 val name = status.getPath.getName name.startsWith("_") || name.startsWith(".") }.map { f => - CSVInputPartitionReader(f.getPath.toUri.toString) - }.toArray + val serializableConf = new SerializableConfiguration(conf) + new SimpleCSVInputPartitionReader( + f.getPath.toUri.toString, + serializableConf): InputPartition[InternalRow] + }.toList.asJava } else { - Array.empty + Collections.emptyList() } } - - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - val serializableConf = new SerializableConfiguration(conf) - new CSVReaderFactory(serializableConf) - } } - class WritSupport(queryId: String, path: String, conf: Configuration) extends BatchWriteSupport { - override def createBatchWriterFactory(): DataWriterFactory = { + class Writer(jobId: String, path: String, conf: Configuration) extends DataSourceWriter { + override def createWriterFactory(): DataWriterFactory[InternalRow] = { SimpleCounter.resetCounter - new CSVDataWriterFactory(path, queryId, new SerializableConfiguration(conf)) + new CSVDataWriterFactory(path, jobId, new SerializableConfiguration(conf)) } override def onDataWriterCommit(message: WriterCommitMessage): Unit = { @@ -80,7 +76,7 @@ class SimpleWritableDataSource extends DataSourceV2 override def commit(messages: Array[WriterCommitMessage]): Unit = { val finalPath = new Path(path) - val jobPath = new Path(new Path(finalPath, "_temporary"), queryId) + val jobPath = new Path(new Path(finalPath, "_temporary"), jobId) val fs = jobPath.getFileSystem(conf) try { for (file <- fs.listStatus(jobPath).map(_.getPath)) { @@ -95,23 +91,23 @@ class SimpleWritableDataSource extends DataSourceV2 } override def abort(messages: Array[WriterCommitMessage]): Unit = { - val jobPath = new Path(new Path(path, "_temporary"), queryId) + val jobPath = new Path(new Path(path, "_temporary"), jobId) val fs = jobPath.getFileSystem(conf) fs.delete(jobPath, true) } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { + override def createReader(options: DataSourceOptions): DataSourceReader = { val path = new Path(options.get("path").get()) val conf = SparkContext.getActive.get.hadoopConfiguration - new ReadSupport(path.toUri.toString, conf) + new Reader(path.toUri.toString, conf) } - override def createBatchWriteSupport( - queryId: String, + override def createWriter( + jobId: String, schema: StructType, mode: SaveMode, - options: DataSourceOptions): Optional[BatchWriteSupport] = { + options: DataSourceOptions): Optional[DataSourceWriter] = { assert(DataType.equalsStructurally(schema.asNullable, this.schema.asNullable)) assert(!SparkContext.getActive.get.conf.getBoolean("spark.speculation", false)) @@ -134,42 +130,39 @@ class SimpleWritableDataSource extends DataSourceV2 } val pathStr = path.toUri.toString - Optional.of(new WritSupport(queryId, pathStr, conf)) + Optional.of(new Writer(jobId, pathStr, conf)) } } -case class CSVInputPartitionReader(path: String) extends InputPartition +class SimpleCSVInputPartitionReader(path: String, conf: SerializableConfiguration) + extends InputPartition[InternalRow] with InputPartitionReader[InternalRow] { -class CSVReaderFactory(conf: SerializableConfiguration) - extends PartitionReaderFactory { + @transient private var lines: Iterator[String] = _ + @transient private var currentLine: String = _ + @transient private var inputStream: FSDataInputStream = _ - override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - val path = partition.asInstanceOf[CSVInputPartitionReader].path + override def createPartitionReader(): InputPartitionReader[InternalRow] = { val filePath = new Path(path) val fs = filePath.getFileSystem(conf.value) + inputStream = fs.open(filePath) + lines = new BufferedReader(new InputStreamReader(inputStream)) + .lines().iterator().asScala + this + } - new PartitionReader[InternalRow] { - private val inputStream = fs.open(filePath) - private val lines = new BufferedReader(new InputStreamReader(inputStream)) - .lines().iterator().asScala - - private var currentLine: String = _ - - override def next(): Boolean = { - if (lines.hasNext) { - currentLine = lines.next() - true - } else { - false - } - } + override def next(): Boolean = { + if (lines.hasNext) { + currentLine = lines.next() + true + } else { + false + } + } - override def get(): InternalRow = InternalRow(currentLine.split(",").map(_.trim.toLong): _*) + override def get(): InternalRow = InternalRow(currentLine.split(",").map(_.trim.toLong): _*) - override def close(): Unit = { - inputStream.close() - } - } + override def close(): Unit = { + inputStream.close() } } @@ -190,11 +183,12 @@ private[v2] object SimpleCounter { } class CSVDataWriterFactory(path: String, jobId: String, conf: SerializableConfiguration) - extends DataWriterFactory { + extends DataWriterFactory[InternalRow] { - override def createWriter( + override def createDataWriter( partitionId: Int, - taskId: Long): DataWriter[InternalRow] = { + taskId: Long, + epochId: Long): DataWriter[InternalRow] = { val jobPath = new Path(new Path(path, "_temporary"), jobId) val filePath = new Path(jobPath, s"$jobId-$partitionId-$taskId") val fs = filePath.getFileSystem(conf.value) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index 491dc34afa14..35644c58cf79 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -686,7 +686,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be plan .collect { case r: StreamingExecutionRelation => r.source - case r: StreamingDataSourceV2Relation => r.readSupport + case r: StreamingDataSourceV2Relation => r.reader } .zipWithIndex .find(_._1 == source) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index fe77a1b4469c..0f15cd6e5a50 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -299,9 +299,9 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { try { val input = new MemoryStream[Int](0, sqlContext) { @volatile var numTriggers = 0 - override def latestOffset(): OffsetV2 = { + override def getEndOffset: OffsetV2 = { numTriggers += 1 - super.latestOffset() + super.getEndOffset } } val clock = new StreamManualClock() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index 1dd817545a96..0278e2a36890 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.streaming +import java.{util => ju} +import java.util.Optional import java.util.concurrent.CountDownLatch import scala.collection.mutable @@ -30,12 +32,13 @@ import org.scalatest.mockito.MockitoSugar import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Literal, Rand, Randn, Shuffle, Uuid} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.TestForeachWriter import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.{InputPartition, ScanConfig} +import org.apache.spark.sql.sources.v2.reader.InputPartition import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} import org.apache.spark.sql.streaming.util.{BlockingSource, MockSourceProvider, StreamManualClock} import org.apache.spark.sql.types.StructType @@ -212,17 +215,25 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi private def dataAdded: Boolean = currentOffset.offset != -1 - // latestOffset should take 50 ms the first time it is called after data is added - override def latestOffset(): OffsetV2 = synchronized { - if (dataAdded) clock.waitTillTime(1050) - super.latestOffset() + // setOffsetRange should take 50 ms the first time it is called after data is added + override def setOffsetRange(start: Optional[OffsetV2], end: Optional[OffsetV2]): Unit = { + synchronized { + if (dataAdded) clock.waitTillTime(1050) + super.setOffsetRange(start, end) + } + } + + // getEndOffset should take 100 ms the first time it is called after data is added + override def getEndOffset(): OffsetV2 = synchronized { + if (dataAdded) clock.waitTillTime(1150) + super.getEndOffset() } // getBatch should take 100 ms the first time it is called - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + override def planInputPartitions(): ju.List[InputPartition[InternalRow]] = { synchronized { - clock.waitTillTime(1150) - super.planInputPartitions(config) + clock.waitTillTime(1350) + super.planInputPartitions() } } } @@ -263,26 +274,34 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi AssertOnQuery(_.status.message === "Waiting for next trigger"), AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), - // Test status and progress when `latestOffset` is being called + // Test status and progress when setOffsetRange is being called AddData(inputData, 1, 2), - AdvanceManualClock(1000), // time = 1000 to start new trigger, will block on `latestOffset` + AdvanceManualClock(1000), // time = 1000 to start new trigger, will block on setOffsetRange AssertStreamExecThreadIsWaitingForTime(1050), AssertOnQuery(_.status.isDataAvailable === false), AssertOnQuery(_.status.isTriggerActive === true), AssertOnQuery(_.status.message.startsWith("Getting offsets from")), AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), - AdvanceManualClock(50), // time = 1050 to unblock `latestOffset` + AdvanceManualClock(50), // time = 1050 to unblock setOffsetRange AssertClockTime(1050), - // will block on `planInputPartitions` that needs 1350 - AssertStreamExecThreadIsWaitingForTime(1150), + AssertStreamExecThreadIsWaitingForTime(1150), // will block on getEndOffset that needs 1150 + AssertOnQuery(_.status.isDataAvailable === false), + AssertOnQuery(_.status.isTriggerActive === true), + AssertOnQuery(_.status.message.startsWith("Getting offsets from")), + AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), + + AdvanceManualClock(100), // time = 1150 to unblock getEndOffset + AssertClockTime(1150), + // will block on planInputPartitions that needs 1350 + AssertStreamExecThreadIsWaitingForTime(1350), AssertOnQuery(_.status.isDataAvailable === true), AssertOnQuery(_.status.isTriggerActive === true), AssertOnQuery(_.status.message === "Processing new data"), AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), - AdvanceManualClock(100), // time = 1150 to unblock `planInputPartitions` - AssertClockTime(1150), + AdvanceManualClock(200), // time = 1350 to unblock planInputPartitions + AssertClockTime(1350), AssertStreamExecThreadIsWaitingForTime(1500), // will block on map task that needs 1500 AssertOnQuery(_.status.isDataAvailable === true), AssertOnQuery(_.status.isTriggerActive === true), @@ -290,7 +309,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), // Test status and progress while batch processing has completed - AdvanceManualClock(350), // time = 1500 to unblock map task + AdvanceManualClock(150), // time = 1500 to unblock map task AssertClockTime(1500), CheckAnswer(2), AssertStreamExecThreadIsWaitingForTime(2000), // will block until the next trigger @@ -310,10 +329,11 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi assert(progress.numInputRows === 2) assert(progress.processedRowsPerSecond === 4.0) - assert(progress.durationMs.get("latestOffset") === 50) - assert(progress.durationMs.get("queryPlanning") === 100) + assert(progress.durationMs.get("setOffsetRange") === 50) + assert(progress.durationMs.get("getEndOffset") === 100) + assert(progress.durationMs.get("queryPlanning") === 200) assert(progress.durationMs.get("walCommit") === 0) - assert(progress.durationMs.get("addBatch") === 350) + assert(progress.durationMs.get("addBatch") === 150) assert(progress.durationMs.get("triggerExecution") === 500) assert(progress.sources.length === 1) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala index d6819eacd07c..4f198819b58d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala @@ -22,15 +22,16 @@ import java.util.concurrent.{ArrayBlockingQueue, BlockingQueue} import org.mockito.Mockito._ import org.scalatest.mockito.MockitoSugar -import org.apache.spark.{SparkEnv, TaskContext} -import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.{SparkEnv, SparkFunSuite, TaskContext} +import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.execution.streaming.continuous._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReader, ContinuousReadSupport, PartitionOffset} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.reader.InputPartition +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousInputPartitionReader, ContinuousReader, PartitionOffset} +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter import org.apache.spark.sql.streaming.StreamTest -import org.apache.spark.sql.types.{DataType, IntegerType, StructType} +import org.apache.spark.sql.types.{DataType, IntegerType} class ContinuousQueuedDataReaderSuite extends StreamTest with MockitoSugar { case class LongPartitionOffset(offset: Long) extends PartitionOffset @@ -43,8 +44,8 @@ class ContinuousQueuedDataReaderSuite extends StreamTest with MockitoSugar { override def beforeEach(): Unit = { super.beforeEach() epochEndpoint = EpochCoordinatorRef.create( - mock[StreamingWriteSupport], - mock[ContinuousReadSupport], + mock[StreamWriter], + mock[ContinuousReader], mock[ContinuousExecution], coordinatorId, startEpoch, @@ -72,26 +73,26 @@ class ContinuousQueuedDataReaderSuite extends StreamTest with MockitoSugar { */ private def setup(): (BlockingQueue[UnsafeRow], ContinuousQueuedDataReader) = { val queue = new ArrayBlockingQueue[UnsafeRow](1024) - val partitionReader = new ContinuousPartitionReader[InternalRow] { - var index = -1 - var curr: UnsafeRow = _ - - override def next() = { - curr = queue.take() - index += 1 - true - } + val factory = new InputPartition[InternalRow] { + override def createPartitionReader() = new ContinuousInputPartitionReader[InternalRow] { + var index = -1 + var curr: UnsafeRow = _ + + override def next() = { + curr = queue.take() + index += 1 + true + } - override def get = curr + override def get = curr - override def getOffset = LongPartitionOffset(index) + override def getOffset = LongPartitionOffset(index) - override def close() = {} + override def close() = {} + } } val reader = new ContinuousQueuedDataReader( - 0, - partitionReader, - new StructType().add("i", "int"), + new ContinuousDataSourceRDDPartition(0, factory), mockContext, dataQueueSize = sqlContext.conf.continuousStreamingExecutorQueueSize, epochPollIntervalMs = sqlContext.conf.continuousStreamingExecutorPollIntervalMs) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala index 3d21bc63e0cc..4980b0cd41f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala @@ -41,7 +41,7 @@ class ContinuousSuiteBase extends StreamTest { case s: ContinuousExecution => assert(numTriggers >= 2, "must wait for at least 2 triggers to ensure query is initialized") val reader = s.lastExecution.executedPlan.collectFirst { - case DataSourceV2ScanExec(_, _, _, _, r: RateStreamContinuousReadSupport, _) => r + case DataSourceV2ScanExec(_, _, _, _, r: RateStreamContinuousReader) => r }.get val deltaMs = numTriggers * 1000 + 300 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala index 3c973d8ebc70..82836dced9df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala @@ -27,9 +27,9 @@ import org.apache.spark._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.LocalSparkSession import org.apache.spark.sql.execution.streaming.continuous._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, PartitionOffset} +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, PartitionOffset} import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter import org.apache.spark.sql.test.TestSparkSession class EpochCoordinatorSuite @@ -40,20 +40,20 @@ class EpochCoordinatorSuite private var epochCoordinator: RpcEndpointRef = _ - private var writeSupport: StreamingWriteSupport = _ + private var writer: StreamWriter = _ private var query: ContinuousExecution = _ private var orderVerifier: InOrder = _ override def beforeEach(): Unit = { - val reader = mock[ContinuousReadSupport] - writeSupport = mock[StreamingWriteSupport] + val reader = mock[ContinuousReader] + writer = mock[StreamWriter] query = mock[ContinuousExecution] - orderVerifier = inOrder(writeSupport, query) + orderVerifier = inOrder(writer, query) spark = new TestSparkSession() epochCoordinator - = EpochCoordinatorRef.create(writeSupport, reader, query, "test", 1, spark, SparkEnv.get) + = EpochCoordinatorRef.create(writer, reader, query, "test", 1, spark, SparkEnv.get) } test("single epoch") { @@ -209,12 +209,12 @@ class EpochCoordinatorSuite } private def verifyCommit(epoch: Long): Unit = { - orderVerifier.verify(writeSupport).commit(eqTo(epoch), any()) + orderVerifier.verify(writer).commit(eqTo(epoch), any()) orderVerifier.verify(query).commit(epoch) } private def verifyNoCommitFor(epoch: Long): Unit = { - verify(writeSupport, never()).commit(eqTo(epoch), any()) + verify(writer, never()).commit(eqTo(epoch), any()) verify(query, never()).commit(epoch) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index aeef4c8fe933..52b833a19c23 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -17,74 +17,73 @@ package org.apache.spark.sql.streaming.sources +import java.util.Optional + import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.streaming.{RateStreamOffset, Sink, StreamingQueryWrapper} import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} -import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReaderFactory, ScanConfig, ScanConfigBuilder} -import org.apache.spark.sql.sources.v2.reader.streaming._ -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, MicroBatchReadSupport, StreamWriteSupport} +import org.apache.spark.sql.sources.v2.reader.InputPartition +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, MicroBatchReader, Offset, PartitionOffset} +import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter import org.apache.spark.sql.streaming.{OutputMode, StreamTest, Trigger} import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils -case class FakeReadSupport() extends MicroBatchReadSupport with ContinuousReadSupport { - override def deserializeOffset(json: String): Offset = RateStreamOffset(Map()) - override def commit(end: Offset): Unit = {} - override def stop(): Unit = {} - override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = RateStreamOffset(Map()) - override def fullSchema(): StructType = StructType(Seq()) - override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = null - override def initialOffset(): Offset = RateStreamOffset(Map()) - override def latestOffset(): Offset = RateStreamOffset(Map()) - override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = null - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - throw new IllegalStateException("fake source - cannot actually read") - } - override def createContinuousReaderFactory( - config: ScanConfig): ContinuousPartitionReaderFactory = { - throw new IllegalStateException("fake source - cannot actually read") - } - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { +case class FakeReader() extends MicroBatchReader with ContinuousReader { + def setOffsetRange(start: Optional[Offset], end: Optional[Offset]): Unit = {} + def getStartOffset: Offset = RateStreamOffset(Map()) + def getEndOffset: Offset = RateStreamOffset(Map()) + def deserializeOffset(json: String): Offset = RateStreamOffset(Map()) + def commit(end: Offset): Unit = {} + def readSchema(): StructType = StructType(Seq()) + def stop(): Unit = {} + def mergeOffsets(offsets: Array[PartitionOffset]): Offset = RateStreamOffset(Map()) + def setStartOffset(start: Optional[Offset]): Unit = {} + + def planInputPartitions(): java.util.ArrayList[InputPartition[InternalRow]] = { throw new IllegalStateException("fake source - cannot actually read") } } -trait FakeMicroBatchReadSupportProvider extends MicroBatchReadSupportProvider { - override def createMicroBatchReadSupport( +trait FakeMicroBatchReadSupport extends MicroBatchReadSupport { + override def createMicroBatchReader( + schema: Optional[StructType], checkpointLocation: String, - options: DataSourceOptions): MicroBatchReadSupport = FakeReadSupport() + options: DataSourceOptions): MicroBatchReader = FakeReader() } -trait FakeContinuousReadSupportProvider extends ContinuousReadSupportProvider { - override def createContinuousReadSupport( +trait FakeContinuousReadSupport extends ContinuousReadSupport { + override def createContinuousReader( + schema: Optional[StructType], checkpointLocation: String, - options: DataSourceOptions): ContinuousReadSupport = FakeReadSupport() + options: DataSourceOptions): ContinuousReader = FakeReader() } -trait FakeStreamingWriteSupportProvider extends StreamingWriteSupportProvider { - override def createStreamingWriteSupport( +trait FakeStreamWriteSupport extends StreamWriteSupport { + override def createStreamWriter( queryId: String, schema: StructType, mode: OutputMode, - options: DataSourceOptions): StreamingWriteSupport = { + options: DataSourceOptions): StreamWriter = { throw new IllegalStateException("fake sink - cannot actually write") } } -class FakeReadMicroBatchOnly extends DataSourceRegister with FakeMicroBatchReadSupportProvider { +class FakeReadMicroBatchOnly extends DataSourceRegister with FakeMicroBatchReadSupport { override def shortName(): String = "fake-read-microbatch-only" } -class FakeReadContinuousOnly extends DataSourceRegister with FakeContinuousReadSupportProvider { +class FakeReadContinuousOnly extends DataSourceRegister with FakeContinuousReadSupport { override def shortName(): String = "fake-read-continuous-only" } class FakeReadBothModes extends DataSourceRegister - with FakeMicroBatchReadSupportProvider with FakeContinuousReadSupportProvider { + with FakeMicroBatchReadSupport with FakeContinuousReadSupport { override def shortName(): String = "fake-read-microbatch-continuous" } @@ -92,7 +91,7 @@ class FakeReadNeitherMode extends DataSourceRegister { override def shortName(): String = "fake-read-neither-mode" } -class FakeWriteSupportProvider extends DataSourceRegister with FakeStreamingWriteSupportProvider { +class FakeWrite extends DataSourceRegister with FakeStreamWriteSupport { override def shortName(): String = "fake-write-microbatch-continuous" } @@ -107,8 +106,8 @@ class FakeSink extends Sink { override def addBatch(batchId: Long, data: DataFrame): Unit = {} } -class FakeWriteSupportProviderV1Fallback extends DataSourceRegister - with FakeStreamingWriteSupportProvider with StreamSinkProvider { +class FakeWriteV1Fallback extends DataSourceRegister + with FakeStreamWriteSupport with StreamSinkProvider { override def createSink( sqlContext: SQLContext, @@ -191,11 +190,11 @@ class StreamingDataSourceV2Suite extends StreamTest { val v2Query = testPositiveCase( "fake-read-microbatch-continuous", "fake-write-v1-fallback", Trigger.Once()) assert(v2Query.asInstanceOf[StreamingQueryWrapper].streamingQuery.sink - .isInstanceOf[FakeWriteSupportProviderV1Fallback]) + .isInstanceOf[FakeWriteV1Fallback]) // Ensure we create a V1 sink with the config. Note the config is a comma separated // list, including other fake entries. - val fullSinkName = classOf[FakeWriteSupportProviderV1Fallback].getName + val fullSinkName = "org.apache.spark.sql.streaming.sources.FakeWriteV1Fallback" withSQLConf(SQLConf.DISABLED_V2_STREAMING_WRITERS.key -> s"a,b,c,test,$fullSinkName,d,e") { val v1Query = testPositiveCase( "fake-read-microbatch-continuous", "fake-write-v1-fallback", Trigger.Once()) @@ -219,37 +218,35 @@ class StreamingDataSourceV2Suite extends StreamTest { val writeSource = DataSource.lookupDataSource(write, spark.sqlContext.conf).newInstance() (readSource, writeSource, trigger) match { // Valid microbatch queries. - case (_: MicroBatchReadSupportProvider, _: StreamingWriteSupportProvider, t) + case (_: MicroBatchReadSupport, _: StreamWriteSupport, t) if !t.isInstanceOf[ContinuousTrigger] => testPositiveCase(read, write, trigger) // Valid continuous queries. - case (_: ContinuousReadSupportProvider, _: StreamingWriteSupportProvider, - _: ContinuousTrigger) => + case (_: ContinuousReadSupport, _: StreamWriteSupport, _: ContinuousTrigger) => testPositiveCase(read, write, trigger) // Invalid - can't read at all case (r, _, _) - if !r.isInstanceOf[MicroBatchReadSupportProvider] - && !r.isInstanceOf[ContinuousReadSupportProvider] => + if !r.isInstanceOf[MicroBatchReadSupport] + && !r.isInstanceOf[ContinuousReadSupport] => testNegativeCase(read, write, trigger, s"Data source $read does not support streamed reading") // Invalid - can't write - case (_, w, _) if !w.isInstanceOf[StreamingWriteSupportProvider] => + case (_, w, _) if !w.isInstanceOf[StreamWriteSupport] => testNegativeCase(read, write, trigger, s"Data source $write does not support streamed writing") // Invalid - trigger is continuous but reader is not - case (r, _: StreamingWriteSupportProvider, _: ContinuousTrigger) - if !r.isInstanceOf[ContinuousReadSupportProvider] => + case (r, _: StreamWriteSupport, _: ContinuousTrigger) + if !r.isInstanceOf[ContinuousReadSupport] => testNegativeCase(read, write, trigger, s"Data source $read does not support continuous processing") // Invalid - trigger is microbatch but reader is not case (r, _, t) - if !r.isInstanceOf[MicroBatchReadSupportProvider] && - !t.isInstanceOf[ContinuousTrigger] => + if !r.isInstanceOf[MicroBatchReadSupport] && !t.isInstanceOf[ContinuousTrigger] => testPostCreationNegativeCase(read, write, trigger, s"Data source $read does not support microbatch processing") } From 71f70130f1b2b4ec70595627f0a02a88e2c0e27d Mon Sep 17 00:00:00 2001 From: Michael Mior Date: Thu, 13 Sep 2018 09:45:25 +0800 Subject: [PATCH 035/879] [SPARK-23820][CORE] Enable use of long form of callsite in logs This is a rework of #21433 to address some concerns there. Closes #22398 from michaelmior/long-callsite2. Authored-by: Michael Mior Signed-off-by: Wenchen Fan (cherry picked from commit ab25c967905ca0973fc2f30b8523246bb9244206) Signed-off-by: Wenchen Fan --- .../org/apache/spark/internal/config/package.scala | 3 +++ .../main/scala/org/apache/spark/storage/RDDInfo.scala | 11 ++++++++++- docs/configuration.md | 7 +++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index c74040cbd0db..7f1eb1e52461 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -72,6 +72,9 @@ package object config { private[spark] val EVENT_LOG_OVERWRITE = ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) + private[spark] val EVENT_LOG_CALLSITE_LONG_FORM = + ConfigBuilder("spark.eventLog.longForm.enabled").booleanConf.createWithDefault(false) + private[spark] val EXECUTOR_CLASS_PATH = ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.createOptional diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index e5abbf745cc4..19f86569c1e3 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -17,7 +17,9 @@ package org.apache.spark.storage +import org.apache.spark.SparkEnv import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.config._ import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.util.Utils @@ -53,10 +55,17 @@ class RDDInfo( } private[spark] object RDDInfo { + private val callsiteLongForm = SparkEnv.get.conf.get(EVENT_LOG_CALLSITE_LONG_FORM) + def fromRdd(rdd: RDD[_]): RDDInfo = { val rddName = Option(rdd.name).getOrElse(Utils.getFormattedClassName(rdd)) val parentIds = rdd.dependencies.map(_.rdd.id) + val callSite = if (callsiteLongForm) { + rdd.creationSite.longForm + } else { + rdd.creationSite.shortForm + } new RDDInfo(rdd.id, rddName, rdd.partitions.length, - rdd.getStorageLevel, parentIds, rdd.creationSite.shortForm, rdd.scope) + rdd.getStorageLevel, parentIds, callSite, rdd.scope) } } diff --git a/docs/configuration.md b/docs/configuration.md index f344bcd20087..a3e59a05f7bd 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -746,6 +746,13 @@ Apart from these, the following properties are also available, and may be useful *Warning*: This will increase the size of the event log considerably. + + spark.eventLog.longForm.enabled + false + + If true, use the long form of call sites in the event log. Otherwise use the short form. + + spark.eventLog.compress false From 776dc42c1326764233a4466172330b74b98df7aa Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Thu, 13 Sep 2018 09:51:49 +0800 Subject: [PATCH 036/879] [SPARK-25387][SQL] Fix for NPE caused by bad CSV input ## What changes were proposed in this pull request? The PR fixes NPE in `UnivocityParser` caused by malformed CSV input. In some cases, `uniVocity` parser can return `null` for bad input. In the PR, I propose to check result of parsing and not propagate NPE to upper layers. ## How was this patch tested? I added a test which reproduce the issue and tested by `CSVSuite`. Closes #22374 from MaxGekk/npe-on-bad-csv. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Wenchen Fan (cherry picked from commit 083c9447671719e0bd67312e3d572f6160c06a4a) Signed-off-by: Wenchen Fan --- .../datasources/csv/CSVDataSource.scala | 36 ++++++++++--------- .../datasources/csv/UnivocityParser.scala | 7 +++- .../execution/datasources/csv/CSVSuite.scala | 11 +++++- 3 files changed, 35 insertions(+), 19 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala index 2b86054c0ffc..e840ff168250 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala @@ -240,23 +240,25 @@ object TextInputCSVDataSource extends CSVDataSource { sparkSession: SparkSession, csv: Dataset[String], maybeFirstLine: Option[String], - parsedOptions: CSVOptions): StructType = maybeFirstLine match { - case Some(firstLine) => - val firstRow = new CsvParser(parsedOptions.asParserSettings).parseLine(firstLine) - val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis - val header = makeSafeHeader(firstRow, caseSensitive, parsedOptions) - val sampled: Dataset[String] = CSVUtils.sample(csv, parsedOptions) - val tokenRDD = sampled.rdd.mapPartitions { iter => - val filteredLines = CSVUtils.filterCommentAndEmpty(iter, parsedOptions) - val linesWithoutHeader = - CSVUtils.filterHeaderLine(filteredLines, firstLine, parsedOptions) - val parser = new CsvParser(parsedOptions.asParserSettings) - linesWithoutHeader.map(parser.parseLine) - } - CSVInferSchema.infer(tokenRDD, header, parsedOptions) - case None => - // If the first line could not be read, just return the empty schema. - StructType(Nil) + parsedOptions: CSVOptions): StructType = { + val csvParser = new CsvParser(parsedOptions.asParserSettings) + maybeFirstLine.map(csvParser.parseLine(_)) match { + case Some(firstRow) if firstRow != null => + val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + val header = makeSafeHeader(firstRow, caseSensitive, parsedOptions) + val sampled: Dataset[String] = CSVUtils.sample(csv, parsedOptions) + val tokenRDD = sampled.rdd.mapPartitions { iter => + val filteredLines = CSVUtils.filterCommentAndEmpty(iter, parsedOptions) + val linesWithoutHeader = + CSVUtils.filterHeaderLine(filteredLines, maybeFirstLine.get, parsedOptions) + val parser = new CsvParser(parsedOptions.asParserSettings) + linesWithoutHeader.map(parser.parseLine) + } + CSVInferSchema.infer(tokenRDD, header, parsedOptions) + case _ => + // If the first line could not be read, just return the empty schema. + StructType(Nil) + } } private def createBaseDataset( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala index e15af425b264..9088d43905e2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala @@ -216,7 +216,12 @@ class UnivocityParser( } private def convert(tokens: Array[String]): InternalRow = { - if (tokens.length != parsedSchema.length) { + if (tokens == null) { + throw BadRecordException( + () => getCurrentInput, + () => None, + new RuntimeException("Malformed CSV record")) + } else if (tokens.length != parsedSchema.length) { // If the number of tokens doesn't match the schema, we should treat it as a malformed record. // However, we still have chance to parse some of the tokens, by adding extra null tokens in // the tail if the number is smaller, or by dropping extra tokens if the number is larger. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 2b39a0b1f52e..f70df0bcecde 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -34,7 +34,7 @@ import org.apache.log4j.{AppenderSkeleton, LogManager} import org.apache.log4j.spi.LoggingEvent import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row, UDT} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} @@ -1811,4 +1811,13 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te checkCount(2) countForMalformedCSV(0, Seq("")) } + + test("SPARK-25387: bad input should not cause NPE") { + val schema = StructType(StructField("a", IntegerType) :: Nil) + val input = spark.createDataset(Seq("\u0000\u0000\u0001234")) + + checkAnswer(spark.read.schema(schema).csv(input), Row(null)) + checkAnswer(spark.read.option("multiLine", true).schema(schema).csv(input), Row(null)) + assert(spark.read.csv(input).collect().toSet == Set(Row())) + } } From 6f4d647e07ef527ef93c4fc849a478008a52bc80 Mon Sep 17 00:00:00 2001 From: LantaoJin Date: Thu, 13 Sep 2018 09:57:34 +0800 Subject: [PATCH 037/879] [SPARK-25357][SQL] Add metadata to SparkPlanInfo to dump more information like file path to event log ## What changes were proposed in this pull request? Field metadata removed from SparkPlanInfo in #18600 . Corresponding, many meta data was also removed from event SparkListenerSQLExecutionStart in Spark event log. If we want to analyze event log to get all input paths, we couldn't get them. Instead, simpleString of SparkPlanInfo JSON only display 100 characters, it won't help. Before 2.3, the fragment of SparkListenerSQLExecutionStart in event log looks like below (It contains the metadata field which has the intact information): >{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart", Location: InMemoryFileIndex[hdfs://cluster1/sys/edw/test1/test2/test3/test4..., "metadata": {"Location": "InMemoryFileIndex[hdfs://cluster1/sys/edw/test1/test2/test3/test4/test5/snapshot/dt=20180904]","ReadSchema":"struct"} After #18600, metadata field was removed. >{"Event":"org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart", Location: InMemoryFileIndex[hdfs://cluster1/sys/edw/test1/test2/test3/test4..., So I add this field back to SparkPlanInfo class. Then it will log out the meta data to event log. Intact information in event log is very useful for offline job analysis. ## How was this patch tested? Unit test Closes #22353 from LantaoJin/SPARK-25357. Authored-by: LantaoJin Signed-off-by: Wenchen Fan (cherry picked from commit 6dc5921e66d56885b95c07e56e687f9f6c1eaca7) Signed-off-by: Wenchen Fan --- .../apache/spark/sql/execution/SparkPlanInfo.scala | 12 ++++++++---- .../spark/sql/execution/SQLJsonProtocolSuite.scala | 2 +- .../apache/spark/sql/execution/SparkPlanSuite.scala | 8 ++++++++ 3 files changed, 17 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 2a2315896831..59ffd1638111 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution -import com.fasterxml.jackson.annotation.JsonIgnoreProperties - import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.SQLMetricInfo @@ -28,11 +26,11 @@ import org.apache.spark.sql.execution.metric.SQLMetricInfo * Stores information about a SQL SparkPlan. */ @DeveloperApi -@JsonIgnoreProperties(Array("metadata")) // The metadata field was removed in Spark 2.3. class SparkPlanInfo( val nodeName: String, val simpleString: String, val children: Seq[SparkPlanInfo], + val metadata: Map[String, String], val metrics: Seq[SQLMetricInfo]) { override def hashCode(): Int = { @@ -59,6 +57,12 @@ private[execution] object SparkPlanInfo { new SQLMetricInfo(metric.name.getOrElse(key), metric.id, metric.metricType) } - new SparkPlanInfo(plan.nodeName, plan.simpleString, children.map(fromSparkPlan), metrics) + // dump the file scan metadata (e.g file path) to event log + val metadata = plan match { + case fileScan: FileSourceScanExec => fileScan.metadata + case _ => Map[String, String]() + } + new SparkPlanInfo(plan.nodeName, plan.simpleString, children.map(fromSparkPlan), + metadata, metrics) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala index c2e62b987e0c..08e40e28d3d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala @@ -46,7 +46,7 @@ class SQLJsonProtocolSuite extends SparkFunSuite { """.stripMargin val reconstructedEvent = JsonProtocol.sparkEventFromJson(parse(SQLExecutionStartJsonString)) val expectedEvent = SparkListenerSQLExecutionStart(0, "test desc", "test detail", "test plan", - new SparkPlanInfo("TestNode", "test string", Nil, Nil), 0) + new SparkPlanInfo("TestNode", "test string", Nil, Map(), Nil), 0) assert(reconstructedEvent == expectedEvent) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala index 34dc6f37c0e4..47ff372992b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala @@ -50,4 +50,12 @@ class SparkPlanSuite extends QueryTest with SharedSQLContext { } } } + + test("SPARK-25357 SparkPlanInfo of FileScan contains nonEmpty metadata") { + withTempPath { path => + spark.range(5).write.parquet(path.getAbsolutePath) + val f = spark.read.parquet(path.getAbsolutePath) + assert(SparkPlanInfo.fromSparkPlan(f.queryExecution.sparkPlan).metadata.nonEmpty) + } + } } From ae5c7bb204c52dd18cfb63e5c621537023e36539 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 13 Sep 2018 11:19:43 +0800 Subject: [PATCH 038/879] [SPARK-25238][PYTHON] lint-python: Fix W605 warnings for pycodestyle 2.4 (This change is a subset of the changes needed for the JIRA; see https://github.com/apache/spark/pull/22231) ## What changes were proposed in this pull request? Use raw strings and simpler regex syntax consistently in Python, which also avoids warnings from pycodestyle about accidentally relying Python's non-escaping of non-reserved chars in normal strings. Also, fix a few long lines. ## How was this patch tested? Existing tests, and some manual double-checking of the behavior of regexes in Python 2/3 to be sure. Closes #22400 from srowen/SPARK-25238.2. Authored-by: Sean Owen Signed-off-by: hyukjinkwon (cherry picked from commit 08c76b5d39127ae207d9d1fff99c2551e6ce2581) Signed-off-by: hyukjinkwon --- dev/create-release/generate-contributors.py | 10 +++++----- dev/create-release/releaseutils.py | 2 +- dev/merge_spark_pr.py | 4 ++-- dev/run-tests-jenkins.py | 3 ++- dev/run-tests.py | 2 +- python/pyspark/ml/classification.py | 4 ++-- python/pyspark/ml/clustering.py | 16 ++++++++-------- python/pyspark/ml/feature.py | 16 ++++++++-------- python/pyspark/ml/fpm.py | 2 +- python/pyspark/ml/regression.py | 20 ++++++++++---------- python/pyspark/mllib/clustering.py | 2 +- python/pyspark/mllib/evaluation.py | 4 ++-- python/pyspark/mllib/feature.py | 2 +- python/pyspark/rdd.py | 2 +- python/pyspark/shell.py | 2 +- python/pyspark/sql/functions.py | 14 ++++++++------ python/pyspark/sql/readwriter.py | 12 ++++++------ python/pyspark/sql/streaming.py | 2 +- python/pyspark/sql/types.py | 2 +- python/pyspark/storagelevel.py | 4 ++-- python/pyspark/util.py | 2 +- python/run-tests.py | 2 +- 22 files changed, 66 insertions(+), 63 deletions(-) diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py index 131d81c8a75c..d9135173419a 100755 --- a/dev/create-release/generate-contributors.py +++ b/dev/create-release/generate-contributors.py @@ -67,7 +67,7 @@ print("Release tag: %s" % RELEASE_TAG) print("Previous release tag: %s" % PREVIOUS_RELEASE_TAG) print("Number of commits in this range: %s" % len(new_commits)) -print +print("") def print_indented(_list): @@ -88,10 +88,10 @@ def print_indented(_list): def is_release(commit_title): - return re.findall("\[release\]", commit_title.lower()) or \ - "preparing spark release" in commit_title.lower() or \ - "preparing development version" in commit_title.lower() or \ - "CHANGES.txt" in commit_title + return ("[release]" in commit_title.lower() or + "preparing spark release" in commit_title.lower() or + "preparing development version" in commit_title.lower() or + "CHANGES.txt" in commit_title) def is_maintenance(commit_title): diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index 8cc990d87184..f273b337fdb4 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -235,7 +235,7 @@ def translate_component(component, commit_hash, warnings): # Parse components in the commit message # The returned components are already filtered and translated def find_components(commit, commit_hash): - components = re.findall("\[\w*\]", commit.lower()) + components = re.findall(r"\[\w*\]", commit.lower()) components = [translate_component(c, commit_hash) for c in components if c in known_components] return components diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 81daa909e019..cca6f405e89a 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -274,7 +274,7 @@ def resolve_jira_issue(merge_branches, comment, default_jira_id=""): versions = sorted(versions, key=lambda x: x.name, reverse=True) versions = filter(lambda x: x.raw['released'] is False, versions) # Consider only x.y.z versions - versions = filter(lambda x: re.match('\d+\.\d+\.\d+', x.name), versions) + versions = filter(lambda x: re.match(r'\d+\.\d+\.\d+', x.name), versions) default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) for v in default_fix_versions: @@ -403,7 +403,7 @@ def standardize_jira_ref(text): # Extract spark component(s): # Look for alphanumeric chars, spaces, dashes, periods, and/or commas - pattern = re.compile(r'(\[[\w\s,-\.]+\])', re.IGNORECASE) + pattern = re.compile(r'(\[[\w\s,.-]+\])', re.IGNORECASE) for component in pattern.findall(text): components.append(component.upper()) text = text.replace(component, '') diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index e6fe3b82ed20..6e943898ffed 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -115,7 +115,8 @@ def run_tests(tests_timeout): os.path.join(SPARK_HOME, 'dev', 'run-tests')]).wait() failure_note_by_errcode = { - 1: 'executing the `dev/run-tests` script', # error to denote run-tests script failures + # error to denote run-tests script failures: + 1: 'executing the `dev/run-tests` script', # noqa: W605 ERROR_CODES["BLOCK_GENERAL"]: 'some tests', ERROR_CODES["BLOCK_RAT"]: 'RAT tests', ERROR_CODES["BLOCK_SCALA_STYLE"]: 'Scala style tests', diff --git a/dev/run-tests.py b/dev/run-tests.py index d9d3789ac125..f534637b80d6 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -169,7 +169,7 @@ def determine_java_version(java_exe): # find raw version string, eg 'java version "1.8.0_25"' raw_version_str = next(x for x in raw_output_lines if " version " in x) - match = re.search('(\d+)\.(\d+)\.(\d+)', raw_version_str) + match = re.search(r'(\d+)\.(\d+)\.(\d+)', raw_version_str) major = int(match.group(1)) minor = int(match.group(2)) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index d5963f4f7042..ce028512357f 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -773,8 +773,8 @@ def roc(self): which is a Dataframe having two fields (FPR, TPR) with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. - .. seealso:: `Wikipedia reference \ - `_ + .. seealso:: `Wikipedia reference + `_ .. note:: This ignores instance weights (setting all to 1.0) from `LogisticRegression.weightCol`. This will change in later Spark diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index ab449bc3f8f5..5ef4e765ea4e 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -1202,21 +1202,21 @@ class PowerIterationClustering(HasMaxIter, HasWeightCol, JavaParams, JavaMLReada .. note:: Experimental Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by - Lin and Cohen. From the abstract: + `Lin and Cohen `_. From the abstract: PIC finds a very low-dimensional embedding of a dataset using truncated power iteration on a normalized pair-wise similarity matrix of the data. This class is not yet an Estimator/Transformer, use :py:func:`assignClusters` method to run the PowerIterationClustering algorithm. - .. seealso:: `Wikipedia on Spectral clustering \ - `_ + .. seealso:: `Wikipedia on Spectral clustering + `_ - >>> data = [(1, 0, 0.5), \ - (2, 0, 0.5), (2, 1, 0.7), \ - (3, 0, 0.5), (3, 1, 0.7), (3, 2, 0.9), \ - (4, 0, 0.5), (4, 1, 0.7), (4, 2, 0.9), (4, 3, 1.1), \ - (5, 0, 0.5), (5, 1, 0.7), (5, 2, 0.9), (5, 3, 1.1), (5, 4, 1.3)] + >>> data = [(1, 0, 0.5), + ... (2, 0, 0.5), (2, 1, 0.7), + ... (3, 0, 0.5), (3, 1, 0.7), (3, 2, 0.9), + ... (4, 0, 0.5), (4, 1, 0.7), (4, 2, 0.9), (4, 3, 1.1), + ... (5, 0, 0.5), (5, 1, 0.7), (5, 2, 0.9), (5, 3, 1.1), (5, 4, 1.3)] >>> df = spark.createDataFrame(data).toDF("src", "dst", "weight") >>> pic = PowerIterationClustering(k=2, maxIter=40, weightCol="weight") >>> assignments = pic.assignClusters(df) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 760aa82168f5..eccb7acae5b9 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -207,8 +207,8 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp distance space. The output will be vectors of configurable dimension. Hash values in the same dimension are calculated by the same hash function. - .. seealso:: `Stable Distributions \ - `_ + .. seealso:: `Stable Distributions + `_ .. seealso:: `Hashing for Similarity Search: A Survey `_ >>> from pyspark.ml.linalg import Vectors @@ -303,7 +303,7 @@ def _create_model(self, java_model): class BucketedRandomProjectionLSHModel(LSHModel, JavaMLReadable, JavaMLWritable): - """ + r""" .. note:: Experimental Model fitted by :py:class:`BucketedRandomProjectionLSH`, where multiple random vectors are @@ -653,8 +653,8 @@ class DCT(JavaTransformer, HasInputCol, HasOutputCol, JavaMLReadable, JavaMLWrit The return vector is scaled such that the transform matrix is unitary (aka scaled DCT-II). - .. seealso:: `More information on Wikipedia \ - `_. + .. seealso:: `More information on Wikipedia + `_. >>> from pyspark.ml.linalg import Vectors >>> df1 = spark.createDataFrame([(Vectors.dense([5.0, 8.0, 6.0]),)], ["vec"]) @@ -1353,7 +1353,7 @@ def _create_model(self, java_model): class MinHashLSHModel(LSHModel, JavaMLReadable, JavaMLWritable): - """ + r""" .. note:: Experimental Model produced by :py:class:`MinHashLSH`, where where multiple hash functions are stored. Each @@ -1362,8 +1362,8 @@ class MinHashLSHModel(LSHModel, JavaMLReadable, JavaMLWritable): :math:`h_i(x) = ((x \cdot a_i + b_i) \mod prime)` This hash family is approximately min-wise independent according to the reference. - .. seealso:: Tom Bohman, Colin Cooper, and Alan Frieze. "Min-wise independent linear \ - permutations." Electronic Journal of Combinatorics 7 (2000): R26. + .. seealso:: Tom Bohman, Colin Cooper, and Alan Frieze. "Min-wise independent linear + permutations." Electronic Journal of Combinatorics 7 (2000): R26. .. versionadded:: 2.2.0 """ diff --git a/python/pyspark/ml/fpm.py b/python/pyspark/ml/fpm.py index c2b29b73460f..886ad8409ca6 100644 --- a/python/pyspark/ml/fpm.py +++ b/python/pyspark/ml/fpm.py @@ -158,7 +158,7 @@ class FPGrowth(JavaEstimator, HasItemsCol, HasPredictionCol, HasMinSupport, HasNumPartitions, HasMinConfidence, JavaMLWritable, JavaMLReadable): - """ + r""" .. note:: Experimental A parallel FP-growth algorithm to mine frequent itemsets. The algorithm is described in diff --git a/python/pyspark/ml/regression.py b/python/pyspark/ml/regression.py index 513ca5a9df85..98f436135184 100644 --- a/python/pyspark/ml/regression.py +++ b/python/pyspark/ml/regression.py @@ -188,8 +188,8 @@ def intercept(self): @property @since("2.3.0") def scale(self): - """ - The value by which \|y - X'w\| is scaled down when loss is "huber", otherwise 1.0. + r""" + The value by which :math:`\|y - X'w\|` is scaled down when loss is "huber", otherwise 1.0. """ return self._call_java("scale") @@ -279,12 +279,12 @@ def featuresCol(self): @property @since("2.0.0") def explainedVariance(self): - """ + r""" Returns the explained variance regression score. - explainedVariance = 1 - variance(y - \hat{y}) / variance(y) + explainedVariance = :math:`1 - \frac{variance(y - \hat{y})}{variance(y)}` - .. seealso:: `Wikipedia explain variation \ - `_ + .. seealso:: `Wikipedia explain variation + `_ .. note:: This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. This will change in later Spark @@ -339,8 +339,8 @@ def r2(self): """ Returns R^2, the coefficient of determination. - .. seealso:: `Wikipedia coefficient of determination \ - `_ + .. seealso:: `Wikipedia coefficient of determination + `_ .. note:: This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. This will change in later Spark @@ -354,8 +354,8 @@ def r2adj(self): """ Returns Adjusted R^2, the adjusted coefficient of determination. - .. seealso:: `Wikipedia coefficient of determination, Adjusted R^2 \ - `_ + .. seealso:: `Wikipedia coefficient of determination, Adjusted R^2 + `_ .. note:: This ignores instance weights (setting all to 1.0) from `LinearRegression.weightCol`. This will change in later Spark versions. diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index b09469b9f5c2..b1a8af6bcc09 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -647,7 +647,7 @@ class PowerIterationClustering(object): @classmethod @since('1.5.0') def train(cls, rdd, k, maxIterations=100, initMode="random"): - """ + r""" :param rdd: An RDD of (i, j, s\ :sub:`ij`\) tuples representing the affinity matrix, which is the matrix A in the PIC paper. The diff --git a/python/pyspark/mllib/evaluation.py b/python/pyspark/mllib/evaluation.py index 6c65da58e4e2..0bb0ca37c1ab 100644 --- a/python/pyspark/mllib/evaluation.py +++ b/python/pyspark/mllib/evaluation.py @@ -117,9 +117,9 @@ def __init__(self, predictionAndObservations): @property @since('1.4.0') def explainedVariance(self): - """ + r""" Returns the explained variance regression score. - explainedVariance = 1 - variance(y - \hat{y}) / variance(y) + explainedVariance = :math:`1 - \frac{variance(y - \hat{y})}{variance(y)}` """ return self.call("explainedVariance") diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 40ecd2e0ff4b..6d7d4d61db04 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -59,7 +59,7 @@ def transform(self, vector): class Normalizer(VectorTransformer): - """ + r""" Normalizes samples individually to unit L\ :sup:`p`\ norm For any 1 <= `p` < float('inf'), normalizes samples using diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index b317156885e5..ccf39e1ffbe9 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2399,7 +2399,7 @@ def barrier(self): :return: an :class:`RDDBarrier` instance that provides actions within a barrier stage. .. seealso:: :class:`BarrierTaskContext` - .. seealso:: `SPIP: Barrier Execution Mode \ + .. seealso:: `SPIP: Barrier Execution Mode `_ .. seealso:: `Design Doc `_ diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 472c3cd4452f..65e3bdbc05ce 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -54,7 +54,7 @@ sqlContext = spark._wrapped sqlCtx = sqlContext -print("""Welcome to +print(r"""Welcome to ____ __ / __/__ ___ _____/ /__ _\ \/ _ \/ _ `/ __/ '_/ diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 81f35f54aa54..e288ec818b40 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -283,7 +283,8 @@ def approxCountDistinct(col, rsd=None): @since(2.1) def approx_count_distinct(col, rsd=None): - """Aggregate function: returns a new :class:`Column` for approximate distinct count of column `col`. + """Aggregate function: returns a new :class:`Column` for approximate distinct count of + column `col`. :param rsd: maximum estimation error allowed (default = 0.05). For rsd < 0.01, it is more efficient to use :func:`countDistinct` @@ -346,7 +347,8 @@ def coalesce(*cols): @since(1.6) def corr(col1, col2): - """Returns a new :class:`Column` for the Pearson Correlation Coefficient for ``col1`` and ``col2``. + """Returns a new :class:`Column` for the Pearson Correlation Coefficient for ``col1`` + and ``col2``. >>> a = range(20) >>> b = [2 * x for x in range(20)] @@ -1688,14 +1690,14 @@ def split(str, pattern): @ignore_unicode_prefix @since(1.5) def regexp_extract(str, pattern, idx): - """Extract a specific group matched by a Java regex, from the specified string column. + r"""Extract a specific group matched by a Java regex, from the specified string column. If the regex did not match, or the specified group did not match, an empty string is returned. >>> df = spark.createDataFrame([('100-200',)], ['str']) - >>> df.select(regexp_extract('str', '(\d+)-(\d+)', 1).alias('d')).collect() + >>> df.select(regexp_extract('str', r'(\d+)-(\d+)', 1).alias('d')).collect() [Row(d=u'100')] >>> df = spark.createDataFrame([('foo',)], ['str']) - >>> df.select(regexp_extract('str', '(\d+)', 1).alias('d')).collect() + >>> df.select(regexp_extract('str', r'(\d+)', 1).alias('d')).collect() [Row(d=u'')] >>> df = spark.createDataFrame([('aaaac',)], ['str']) >>> df.select(regexp_extract('str', '(a+)(b)?(c)', 2).alias('d')).collect() @@ -1712,7 +1714,7 @@ def regexp_replace(str, pattern, replacement): """Replace all substrings of the specified string value that match regexp with rep. >>> df = spark.createDataFrame([('100-200',)], ['str']) - >>> df.select(regexp_replace('str', '(\\d+)', '--').alias('d')).collect() + >>> df.select(regexp_replace('str', r'(\d+)', '--').alias('d')).collect() [Row(d=u'-----')] """ sc = SparkContext._active_spark_context diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index 3ca5d548ae7d..690b13072244 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -350,7 +350,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None, columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None, samplingRatio=None, enforceSchema=None, emptyValue=None): - """Loads a CSV file and returns the result as a :class:`DataFrame`. + r"""Loads a CSV file and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if ``inferSchema`` is enabled. To avoid going through the entire data once, disable @@ -519,8 +519,8 @@ def jdbc(self, url, table, column=None, lowerBound=None, upperBound=None, numPar If both ``column`` and ``predicates`` are specified, ``column`` will be used. - .. note:: Don't create too many partitions in parallel on a large cluster; \ - otherwise Spark might crash your external database systems. + .. note:: Don't create too many partitions in parallel on a large cluster; + otherwise Spark might crash your external database systems. :param url: a JDBC URL of the form ``jdbc:subprotocol:subname`` :param table: the name of the table @@ -862,7 +862,7 @@ def csv(self, path, mode=None, compression=None, sep=None, quote=None, escape=No header=None, nullValue=None, escapeQuotes=None, quoteAll=None, dateFormat=None, timestampFormat=None, ignoreLeadingWhiteSpace=None, ignoreTrailingWhiteSpace=None, charToEscapeQuoteEscaping=None, encoding=None, emptyValue=None): - """Saves the content of the :class:`DataFrame` in CSV format at the specified path. + r"""Saves the content of the :class:`DataFrame` in CSV format at the specified path. :param path: the path in any Hadoop supported file system :param mode: specifies the behavior of the save operation when data already exists. @@ -962,8 +962,8 @@ def orc(self, path, mode=None, partitionBy=None, compression=None): def jdbc(self, url, table, mode=None, properties=None): """Saves the content of the :class:`DataFrame` to an external database table via JDBC. - .. note:: Don't create too many partitions in parallel on a large cluster; \ - otherwise Spark might crash your external database systems. + .. note:: Don't create too many partitions in parallel on a large cluster; + otherwise Spark might crash your external database systems. :param url: a JDBC URL of the form ``jdbc:subprotocol:subname`` :param table: Name of the table in the external database. diff --git a/python/pyspark/sql/streaming.py b/python/pyspark/sql/streaming.py index 522900bf6684..b18453b2a4f9 100644 --- a/python/pyspark/sql/streaming.py +++ b/python/pyspark/sql/streaming.py @@ -565,7 +565,7 @@ def csv(self, path, schema=None, sep=None, encoding=None, quote=None, escape=Non maxCharsPerColumn=None, maxMalformedLogPerPartition=None, mode=None, columnNameOfCorruptRecord=None, multiLine=None, charToEscapeQuoteEscaping=None, enforceSchema=None, emptyValue=None): - """Loads a CSV file stream and returns the result as a :class:`DataFrame`. + r"""Loads a CSV file stream and returns the result as a :class:`DataFrame`. This function will go through the input once to determine the input schema if ``inferSchema`` is enabled. To avoid going through the entire data once, disable diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index ce1d004c6c8f..1d24c40e5858 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -752,7 +752,7 @@ def __eq__(self, other): for v in [ArrayType, MapType, StructType]) -_FIXED_DECIMAL = re.compile("decimal\\(\\s*(\\d+)\\s*,\\s*(\\d+)\\s*\\)") +_FIXED_DECIMAL = re.compile(r"decimal\(\s*(\d+)\s*,\s*(\d+)\s*\)") def _parse_datatype_string(s): diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index ef012d27cb22..7f29646c0743 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -58,8 +58,8 @@ def __str__(self): StorageLevel.OFF_HEAP = StorageLevel(True, True, True, False, 1) """ -.. note:: The following four storage level constants are deprecated in 2.0, since the records \ -will always be serialized in Python. +.. note:: The following four storage level constants are deprecated in 2.0, since the records + will always be serialized in Python. """ StorageLevel.MEMORY_ONLY_SER = StorageLevel.MEMORY_ONLY """.. note:: Deprecated in 2.0, use ``StorageLevel.MEMORY_ONLY`` instead.""" diff --git a/python/pyspark/util.py b/python/pyspark/util.py index f015542c8799..f906f4959543 100644 --- a/python/pyspark/util.py +++ b/python/pyspark/util.py @@ -80,7 +80,7 @@ def majorMinorVersion(sparkVersion): (2, 3) """ - m = re.search('^(\d+)\.(\d+)(\..*)?$', sparkVersion) + m = re.search(r'^(\d+)\.(\d+)(\..*)?$', sparkVersion) if m is not None: return (int(m.group(1)), int(m.group(2))) else: diff --git a/python/run-tests.py b/python/run-tests.py index 4c90926cfa35..ccbdfac3f385 100755 --- a/python/run-tests.py +++ b/python/run-tests.py @@ -138,7 +138,7 @@ def run_individual_python_test(target_dir, test_name, pyspark_python): # 2 (or --verbose option is enabled). decoded_lines = map(lambda line: line.decode(), iter(per_test_output)) skipped_tests = list(filter( - lambda line: re.search('test_.* \(pyspark\..*\) ... skipped ', line), + lambda line: re.search(r'test_.* \(pyspark\..*\) ... skipped ', line), decoded_lines)) skipped_counts = len(skipped_tests) if skipped_counts > 0: From abb5196c7ef685e1027eb1b0b09f4559d3eba015 Mon Sep 17 00:00:00 2001 From: Stavros Kontopoulos Date: Wed, 12 Sep 2018 22:02:59 -0700 Subject: [PATCH 039/879] [SPARK-25295][K8S] Fix executor names collision ## What changes were proposed in this pull request? Fixes the collision issue with spark executor names in client mode, see SPARK-25295 for the details. It follows the cluster name convention as app-name will be used as the prefix and if that is not defined we use "spark" as the default prefix. Eg. `spark-pi-1536781360723-exec-1` where spark-pi is the name of the app passed at the config side or transformed if it contains illegal characters. Also fixes the issue with spark app name having spaces in cluster mode. If you run the Spark Pi test in client mode it passes. The tricky part is the user may set the app name: https://github.com/apache/spark/blob/3030b82c89d3e45a2e361c469fbc667a1e43b854/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala#L30 If i do: ``` ./bin/spark-submit ... --deploy-mode cluster --name "spark pi" ... ``` it will fail as the app name is used for the prefix of driver's pod name and it cannot have spaces (according to k8s conventions). ## How was this patch tested? Manually by running spark job in client mode. To reproduce do: ``` kubectl create -f service.yaml kubectl create -f pod.yaml ``` service.yaml : ``` kind: Service apiVersion: v1 metadata: name: spark-test-app-1-svc spec: clusterIP: None selector: spark-app-selector: spark-test-app-1 ports: - protocol: TCP name: driver-port port: 7077 targetPort: 7077 - protocol: TCP name: block-manager port: 10000 targetPort: 10000 ``` pod.yaml: ``` apiVersion: v1 kind: Pod metadata: name: spark-test-app-1 labels: spark-app-selector: spark-test-app-1 spec: containers: - name: spark-test image: skonto/spark:k8s-client-fix imagePullPolicy: Always command: - 'sh' - '-c' - "/opt/spark/bin/spark-submit --verbose --master k8s://https://kubernetes.default.svc --deploy-mode client --class org.apache.spark.examples.SparkPi --conf spark.app.name=spark --conf spark.executor.instances=1 --conf spark.kubernetes.container.image=skonto/spark:k8s-client-fix --conf spark.kubernetes.container.image.pullPolicy=Always --conf spark.kubernetes.authenticate.oauthTokenFile=/var/run/secrets/kubernetes.io/serviceaccount/token --conf spark.kubernetes.authenticate.caCertFile=/var/run/secrets/kubernetes.io/serviceaccount/ca.crt --conf spark.executor.memory=500m --conf spark.executor.cores=1 --conf spark.executor.instances=1 --conf spark.driver.host=spark-test-app-1-svc.default.svc --conf spark.driver.port=7077 --conf spark.driver.blockManager.port=10000 local:///opt/spark/examples/jars/spark-examples_2.11-2.4.0-SNAPSHOT.jar 1000000" ``` Closes #22405 from skonto/fix-k8s-client-mode-executor-names. Authored-by: Stavros Kontopoulos Signed-off-by: Yinan Li (cherry picked from commit 3e75a9fa24f8629d068b5fbbc7356ce2603fa58d) Signed-off-by: Yinan Li --- .../spark/deploy/k8s/KubernetesConf.scala | 13 +++++++++++- .../submit/KubernetesClientApplication.scala | 21 +++++++++++++++---- .../k8s/ExecutorPodsAllocatorSuite.scala | 16 +++++++++++--- 3 files changed, 42 insertions(+), 8 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala index 3aa35d419073..cae6e7d5ad51 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesConf.scala @@ -24,6 +24,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ +import org.apache.spark.deploy.k8s.submit.KubernetesClientApplication._ import org.apache.spark.internal.config.ConfigEntry @@ -220,10 +221,20 @@ private[spark] object KubernetesConf { val executorVolumes = KubernetesVolumeUtils.parseVolumesWithPrefix( sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX).map(_.get) + // If no prefix is defined then we are in pure client mode + // (not the one used by cluster mode inside the container) + val appResourceNamePrefix = { + if (sparkConf.getOption(KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key).isEmpty) { + getResourceNamePrefix(getAppName(sparkConf)) + } else { + sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) + } + } + KubernetesConf( sparkConf.clone(), KubernetesExecutorSpecificConf(executorId, driverPod), - sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX), + appResourceNamePrefix, appId, executorLabels, executorAnnotations, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala index 986c950ab365..edeaa380194a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesClientApplication.scala @@ -211,11 +211,8 @@ private[spark] class KubernetesClientApplication extends SparkApplication { // considerably restrictive, e.g. must be no longer than 63 characters in length. So we generate // a unique app ID (captured by spark.app.id) in the format below. val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" - val launchTime = System.currentTimeMillis() val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) - val kubernetesResourceNamePrefix = { - s"$appName-$launchTime".toLowerCase.replaceAll("\\.", "-") - } + val kubernetesResourceNamePrefix = KubernetesClientApplication.getResourceNamePrefix(appName) sparkConf.set(KUBERNETES_PYSPARK_PY_FILES, clientArguments.maybePyFiles.getOrElse("")) val kubernetesConf = KubernetesConf.createDriverConf( sparkConf, @@ -254,3 +251,19 @@ private[spark] class KubernetesClientApplication extends SparkApplication { } } } + +private[spark] object KubernetesClientApplication { + + def getAppName(conf: SparkConf): String = conf.getOption("spark.app.name").getOrElse("spark") + + def getResourceNamePrefix(appName: String): String = { + val launchTime = System.currentTimeMillis() + s"$appName-$launchTime" + .trim + .toLowerCase + .replaceAll("\\s+", "-") + .replaceAll("\\.", "-") + .replaceAll("[^a-z0-9\\-]", "") + .replaceAll("-+", "-") + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index e847f8590d35..0e617b002101 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -167,13 +167,23 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { executorSpecificConf.executorId, TEST_SPARK_APP_ID, Some(driverPod)) - k8sConf.sparkConf.getAll.toMap == conf.getAll.toMap && + + // Set prefixes to a common string since KUBERNETES_EXECUTOR_POD_NAME_PREFIX + // has not be set for the tests and thus KubernetesConf will use a random + // string for the prefix, based on the app name, and this comparison here will fail. + val k8sConfCopy = k8sConf + .copy(appResourceNamePrefix = "") + .copy(sparkConf = conf) + val expectedK8sConfCopy = expectedK8sConf + .copy(appResourceNamePrefix = "") + .copy(sparkConf = conf) + + k8sConf.sparkConf.getAll.toMap == conf.getAll.toMap && // Since KubernetesConf.createExecutorConf clones the SparkConf object, force // deep equality comparison for the SparkConf object and use object equality // comparison on all other fields. - k8sConf.copy(sparkConf = conf) == expectedK8sConf.copy(sparkConf = conf) + k8sConfCopy == expectedK8sConfCopy } } }) - } From e7f511ad0803f4a25c657ea25a63a70c6f33367a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 13 Sep 2018 14:21:00 +0200 Subject: [PATCH 040/879] [SPARK-25352][SQL][FOLLOWUP] Add helper method and address style issue ## What changes were proposed in this pull request? This follow-up patch addresses [the review comment](https://github.com/apache/spark/pull/22344/files#r217070658) by adding a helper method to simplify code and fixing style issue. ## How was this patch tested? Existing unit tests. Author: Liang-Chi Hsieh Closes #22409 from viirya/SPARK-25352-followup. (cherry picked from commit 5b761c537a600115450b53817bee0679d5c2bb97) Signed-off-by: Herman van Hovell --- .../spark/sql/execution/SparkStrategies.scala | 55 ++++++++----------- 1 file changed, 23 insertions(+), 32 deletions(-) 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 7c8ce316f964..89442a70283f 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 @@ -66,44 +66,35 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { * Plans special cases of limit operators. */ object SpecialLimits extends Strategy { - override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case ReturnAnswer(rootPlan) => rootPlan match { - case Limit(IntegerLiteral(limit), s@Sort(order, true, child)) => - if (limit < conf.topKSortFallbackThreshold) { + private def decideTopRankNode(limit: Int, child: LogicalPlan): Seq[SparkPlan] = { + if (limit < conf.topKSortFallbackThreshold) { + child match { + case Sort(order, true, child) => TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil - } else { - GlobalLimitExec(limit, - LocalLimitExec(limit, planLater(s)), - orderedLimit = true) :: Nil - } - case Limit(IntegerLiteral(limit), p@Project(projectList, Sort(order, true, child))) => - if (limit < conf.topKSortFallbackThreshold) { + case Project(projectList, Sort(order, true, child)) => TakeOrderedAndProjectExec(limit, order, projectList, planLater(child)) :: Nil - } else { - GlobalLimitExec(limit, - LocalLimitExec(limit, planLater(p)), - orderedLimit = true) :: Nil - } + } + } else { + GlobalLimitExec(limit, + LocalLimitExec(limit, planLater(child)), + orderedLimit = true) :: Nil + } + } + + override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case ReturnAnswer(rootPlan) => rootPlan match { + case Limit(IntegerLiteral(limit), s @ Sort(order, true, child)) => + decideTopRankNode(limit, s) + case Limit(IntegerLiteral(limit), p @ Project(projectList, Sort(order, true, child))) => + decideTopRankNode(limit, p) case Limit(IntegerLiteral(limit), child) => CollectLimitExec(limit, planLater(child)) :: Nil case other => planLater(other) :: Nil } - case Limit(IntegerLiteral(limit), s@Sort(order, true, child)) => - if (limit < conf.topKSortFallbackThreshold) { - TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil - } else { - GlobalLimitExec(limit, - LocalLimitExec(limit, planLater(s)), - orderedLimit = true) :: Nil - } - case Limit(IntegerLiteral(limit), p@Project(projectList, Sort(order, true, child))) => - if (limit < conf.topKSortFallbackThreshold) { - TakeOrderedAndProjectExec(limit, order, projectList, planLater(child)) :: Nil - } else { - GlobalLimitExec(limit, - LocalLimitExec(limit, planLater(p)), - orderedLimit = true) :: Nil - } + case Limit(IntegerLiteral(limit), s @ Sort(order, true, child)) => + decideTopRankNode(limit, s) + case Limit(IntegerLiteral(limit), p @ Project(projectList, Sort(order, true, child))) => + decideTopRankNode(limit, p) case _ => Nil } } From cc19f424bc7d405acdec024a983345ec986b25fc Mon Sep 17 00:00:00 2001 From: LucaCanali Date: Thu, 13 Sep 2018 10:19:21 -0500 Subject: [PATCH 041/879] [SPARK-25170][DOC] Add list and short description of Spark Executor Task Metrics to the documentation. ## What changes were proposed in this pull request? Add description of Executor Task Metrics to the documentation. Closes #22397 from LucaCanali/docMonitoringTaskMetrics. Authored-by: LucaCanali Signed-off-by: Sean Owen (cherry picked from commit 45c4ebc8171d75fc0d169bb8071a4c43263d283e) Signed-off-by: Sean Owen --- docs/monitoring.md | 152 +++++++++++++++++++++++++++++++++++++++++++++ 1 file changed, 152 insertions(+) diff --git a/docs/monitoring.md b/docs/monitoring.md index 2717dd091c75..f6d52ef4597e 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -388,6 +388,158 @@ value triggering garbage collection on jobs, and `spark.ui.retainedStages` that Note that the garbage collection takes place on playback: it is possible to retrieve more entries by increasing these values and restarting the history server. +### Executor Task Metrics + +The REST API exposes the values of the Task Metrics collected by Spark executors with the granularity +of task execution. The metrics can be used for performance troubleshooting and workload characterization. +A list of the available metrics, with a short description: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Spark Executor Task Metric nameShort description
    executorRunTimeElapsed time the executor spent running this task. This includes time fetching shuffle data. + The value is expressed in milliseconds.
    executorCpuTimeCPU time the executor spent running this task. This includes time fetching shuffle data. + The value is expressed in nanoseconds.
    executorDeserializeTimeElapsed time spent to deserialize this task. The value is expressed in milliseconds.
    executorDeserializeCpuTimeCPU time taken on the executor to deserialize this task. The value is expressed + in nanoseconds.
    resultSizeThe number of bytes this task transmitted back to the driver as the TaskResult.
    jvmGCTimeElapsed time the JVM spent in garbage collection while executing this task. + The value is expressed in milliseconds.
    resultSerializationTimeElapsed time spent serializing the task result. The value is expressed in milliseconds.
    memoryBytesSpilledThe number of in-memory bytes spilled by this task.
    diskBytesSpilledThe number of on-disk bytes spilled by this task.
    peakExecutionMemoryPeak memory used by internal data structures created during shuffles, aggregations and + joins. The value of this accumulator should be approximately the sum of the peak sizes + across all such data structures created in this task. For SQL jobs, this only tracks all + unsafe operators and ExternalSort.
    inputMetrics.*Metrics related to reading data from [[org.apache.spark.rdd.HadoopRDD]] + or from persisted data.
        .bytesReadTotal number of bytes read.
        .recordsReadTotal number of records read.
    outputMetrics.*Metrics related to writing data externally (e.g. to a distributed filesystem), + defined only in tasks with output.
        .bytesWrittenTotal number of bytes written
        .recordsWrittenTotal number of records written
    shuffleReadMetrics.*Metrics related to shuffle read operations.
        .recordsReadNumber of records read in shuffle operations
        .remoteBlocksFetchedNumber of remote blocks fetched in shuffle operations
        .localBlocksFetchedNumber of local (as opposed to read from a remote executor) blocks fetched + in shuffle operations
        .totalBlocksFetchedNumber of blocks fetched in shuffle operations (both local and remote)
        .remoteBytesReadNumber of remote bytes read in shuffle operations
        .localBytesReadNumber of bytes read in shuffle operations from local disk (as opposed to + read from a remote executor)
        .totalBytesReadNumber of bytes read in shuffle operations (both local and remote)
        .remoteBytesReadToDiskNumber of remote bytes read to disk in shuffle operations. + Large blocks are fetched to disk in shuffle read operations, as opposed to + being read into memory, which is the default behavior.
        .fetchWaitTimeTime the task spent waiting for remote shuffle blocks. + This only includes the time blocking on shuffle input data. + For instance if block B is being fetched while the task is still not finished + processing block A, it is not considered to be blocking on block B. + The value is expressed in milliseconds.
    shuffleWriteMetrics.*Metrics related to operations writing shuffle data.
        .bytesWrittenNumber of bytes written in shuffle operations
        .recordsWrittenNumber of records written in shuffle operations
        .writeTimeTime spent blocking on writes to disk or buffer cache. The value is expressed + in nanoseconds.
    + + + ### API Versioning Policy These endpoints have been strongly versioned to make it easier to develop applications on top. From 35a84baa5b0e7ee68b1a810c77d92b7b39e47a02 Mon Sep 17 00:00:00 2001 From: Michael Allman Date: Thu, 13 Sep 2018 17:08:45 +0000 Subject: [PATCH 042/879] [SPARK-25406][SQL] For ParquetSchemaPruningSuite.scala, move calls to `withSQLConf` inside calls to `test` (Link to Jira: https://issues.apache.org/jira/browse/SPARK-25406) ## What changes were proposed in this pull request? The current use of `withSQLConf` in `ParquetSchemaPruningSuite.scala` is incorrect. The desired configuration settings are not being set when running the test cases. This PR fixes that defective usage and addresses the test failures that were previously masked by that defect. ## How was this patch tested? I added code to relevant test cases to print the expected SQL configuration settings and found that the settings were not being set as expected. When I changed the order of calls to `test` and `withSQLConf` I found that the configuration settings were being set as expected. Closes #22394 from mallman/spark-25406-fix_broken_schema_pruning_tests. Authored-by: Michael Allman Signed-off-by: DB Tsai (cherry picked from commit a7e5aa6cd430d0a49bb6dac92c007fab189db3a3) Signed-off-by: DB Tsai --- .../parquet/ParquetSchemaPruningSuite.scala | 63 +++++++++++-------- 1 file changed, 38 insertions(+), 25 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 7b132af4f691..434c4414edeb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -218,20 +218,24 @@ class ParquetSchemaPruningSuite } private def testSchemaPruning(testName: String)(testThunk: => Unit) { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { - test(s"Spark vectorized reader - without partition data column - $testName") { + test(s"Spark vectorized reader - without partition data column - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { withContacts(testThunk) } - test(s"Spark vectorized reader - with partition data column - $testName") { + } + test(s"Spark vectorized reader - with partition data column - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { withContactsWithDataPartitionColumn(testThunk) } } - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - test(s"Parquet-mr reader - without partition data column - $testName") { + test(s"Parquet-mr reader - without partition data column - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { withContacts(testThunk) } - test(s"Parquet-mr reader - with partition data column - $testName") { + } + test(s"Parquet-mr reader - with partition data column - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { withContactsWithDataPartitionColumn(testThunk) } } @@ -271,7 +275,7 @@ class ParquetSchemaPruningSuite MixedCase(1, "r1c1", MixedCaseColumn("123", 2)) :: Nil - testMixedCasePruning("select with exact column names") { + testExactCaseQueryPruning("select with exact column names") { val query = sql("select CoL1, coL2.B from mixedcase") checkScan(query, "struct>") checkAnswer(query.orderBy("id"), @@ -280,7 +284,7 @@ class ParquetSchemaPruningSuite Nil) } - testMixedCasePruning("select with lowercase column names") { + testMixedCaseQueryPruning("select with lowercase column names") { val query = sql("select col1, col2.b from mixedcase") checkScan(query, "struct>") checkAnswer(query.orderBy("id"), @@ -289,7 +293,7 @@ class ParquetSchemaPruningSuite Nil) } - testMixedCasePruning("select with different-case column names") { + testMixedCaseQueryPruning("select with different-case column names") { val query = sql("select cOL1, cOl2.b from mixedcase") checkScan(query, "struct>") checkAnswer(query.orderBy("id"), @@ -298,34 +302,43 @@ class ParquetSchemaPruningSuite Nil) } - testMixedCasePruning("filter with different-case column names") { + testMixedCaseQueryPruning("filter with different-case column names") { val query = sql("select id from mixedcase where Col2.b = 2") checkScan(query, "struct>") checkAnswer(query.orderBy("id"), Row(1) :: Nil) } - private def testMixedCasePruning(testName: String)(testThunk: => Unit) { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", - SQLConf.CASE_SENSITIVE.key -> "true") { - test(s"Spark vectorized reader - case-sensitive parser - mixed-case schema - $testName") { - withMixedCaseData(testThunk) + // Tests schema pruning for a query whose column and field names are exactly the same as the table + // schema's column and field names. N.B. this implies that `testThunk` should pass using either a + // case-sensitive or case-insensitive query parser + private def testExactCaseQueryPruning(testName: String)(testThunk: => Unit) { + test(s"Spark vectorized reader - case-sensitive parser - mixed-case schema - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", + SQLConf.CASE_SENSITIVE.key -> "true") { + withMixedCaseData(testThunk) } } - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", - SQLConf.CASE_SENSITIVE.key -> "false") { - test(s"Parquet-mr reader - case-insensitive parser - mixed-case schema - $testName") { + test(s"Parquet-mr reader - case-sensitive parser - mixed-case schema - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", + SQLConf.CASE_SENSITIVE.key -> "true") { withMixedCaseData(testThunk) } } - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", - SQLConf.CASE_SENSITIVE.key -> "false") { - test(s"Spark vectorized reader - case-insensitive parser - mixed-case schema - $testName") { - withMixedCaseData(testThunk) + testMixedCaseQueryPruning(testName)(testThunk) + } + + // Tests schema pruning for a query whose column and field names may differ in case from the table + // schema's column and field names + private def testMixedCaseQueryPruning(testName: String)(testThunk: => Unit) { + test(s"Spark vectorized reader - case-insensitive parser - mixed-case schema - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", + SQLConf.CASE_SENSITIVE.key -> "false") { + withMixedCaseData(testThunk) } } - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", - SQLConf.CASE_SENSITIVE.key -> "true") { - test(s"Parquet-mr reader - case-sensitive parser - mixed-case schema - $testName") { + test(s"Parquet-mr reader - case-insensitive parser - mixed-case schema - $testName") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", + SQLConf.CASE_SENSITIVE.key -> "false") { withMixedCaseData(testThunk) } } From 9273be09f64f23d70c13fd80479cc41ebd514313 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 13 Sep 2018 14:11:55 -0500 Subject: [PATCH 043/879] [SPARK-25400][CORE][TEST] Increase test timeouts We've seen some flakiness in jenkins in SchedulerIntegrationSuite which looks like it just needs a longer timeout. Closes #22385 from squito/SPARK-25400. Authored-by: Imran Rashid Signed-off-by: Sean Owen (cherry picked from commit 9deddbb13edebfefb3fd03f063679ed12e73c575) Signed-off-by: Sean Owen --- .../apache/spark/scheduler/BlacklistIntegrationSuite.scala | 1 - .../apache/spark/scheduler/SchedulerIntegrationSuite.scala | 7 +++---- 2 files changed, 3 insertions(+), 5 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index d3bbfd11d406..fe22d70850c7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -24,7 +24,6 @@ import org.apache.spark.internal.config class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorMockBackend]{ val badHost = "host-0" - val duration = Duration(10, SECONDS) /** * This backend just always fails if the task is executed on a bad host, but otherwise succeeds diff --git a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala index 2d409d94ca1b..ff0f99b5c94d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -51,6 +51,9 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa var taskScheduler: TestTaskScheduler = null var scheduler: DAGScheduler = null var backend: T = _ + // Even though the tests aren't doing much, occassionally we see flakiness from pauses over + // a second (probably from GC?) so we leave a long timeout in here + val duration = Duration(10, SECONDS) override def beforeEach(): Unit = { if (taskScheduler != null) { @@ -539,7 +542,6 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor } withBackend(runBackend _) { val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) - val duration = Duration(1, SECONDS) awaitJobTermination(jobFuture, duration) } assert(results === (0 until 10).map { _ -> 42 }.toMap) @@ -592,7 +594,6 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor } withBackend(runBackend _) { val jobFuture = submit(d, (0 until 30).toArray) - val duration = Duration(1, SECONDS) awaitJobTermination(jobFuture, duration) } assert(results === (0 until 30).map { idx => idx -> (4321 + idx) }.toMap) @@ -634,7 +635,6 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor } withBackend(runBackend _) { val jobFuture = submit(shuffledRdd, (0 until 10).toArray) - val duration = Duration(1, SECONDS) awaitJobTermination(jobFuture, duration) } assertDataStructuresEmpty() @@ -649,7 +649,6 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor } withBackend(runBackend _) { val jobFuture = submit(new MockRDD(sc, 10, Nil), (0 until 10).toArray) - val duration = Duration(1, SECONDS) awaitJobTermination(jobFuture, duration) assert(failure.getMessage.contains("test task failure")) } From 1220ab8a0738b5f67dc522df5e3e77ffc83d207a Mon Sep 17 00:00:00 2001 From: Wenchen Date: Fri, 14 Sep 2018 12:39:46 +0000 Subject: [PATCH 044/879] Preparing Spark release v2.4.0-rc1 --- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 2 +- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 42 files changed, 42 insertions(+), 42 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 9608c96fd536..63ab510eb683 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 8c148359c302..b10e11849a74 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 8ca7733507f1..74c6d232b36a 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 05335df61a66..fbdc979d120d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 564e6583c909..53d1b2b58f1f 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 2f04abe8c7e8..98145481adc7 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index ba127408e1c5..f0c8ebe25e63 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 152785473039..c635cd3b9394 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 5fa3a86de6b0..ed3efffa180b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 095fadb93fe5..c3252a3e695d 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,7 +14,7 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.0-SNAPSHOT +SPARK_VERSION: 2.4.0 SPARK_VERSION_SHORT: 2.4.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.8" diff --git a/examples/pom.xml b/examples/pom.xml index 868110b8e35e..c7b8354b0968 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 8f118ba48201..a91c13362caa 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 431339d41219..db239f23e1de 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 7cd1ec4c9c09..cf19fed1bef5 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index f810aa80e878..da5dba82f3cf 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 498e88f665eb..e591ce802256 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index f80f8e3a0183..e8b13cdc2bef 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 8588e8be052e..c3d3b888656b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index a97fd35bfbb7..468ace0ff8d7 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 6be17a81f3fe..db8fda66c3cd 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 6d1c4789f382..aa1e1267f57b 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 68fded515626..7d07e18f04a2 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 491589396559..f545f212b2f0 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 027157e53d51..6b388ffcf18b 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 0f5dc548600b..fc369c73edc0 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 2c39a7df0146..f4cb94147d9e 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 912eb6b6d2a0..22f60efae9af 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 53286fe93478..b17e67d78b98 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index f07d7f24fd31..bf0d406da9ed 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 05e3b05613ef..dc434b7bb1c5 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index b9c2c4ced71d..53e3424394ba 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.0.dev0" +__version__ = "2.4.0" diff --git a/repl/pom.xml b/repl/pom.xml index e8464a688336..4532f0b8686e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 920f0f6ebf2c..47fe3f2765b2 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 614705c1ed66..f59895e9a1bd 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 3995d0afeb5f..0eacbbfac2b9 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 37e25ceecb88..2afdc15be446 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 7d23637e2834..ef1d012d2f80 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index ba17f5f33f2b..f4be074642ae 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 9f247f9224c7..e5c1065451f7 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c55ba32fa458..b522bfeac130 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 4497e53b6598..8e71adabb3ec 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 242219e29f50..c00e1b851ba0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0-SNAPSHOT + 2.4.0 ../pom.xml From 8cdf7f4c9345f8a58adffcf048fb84cc618cffcf Mon Sep 17 00:00:00 2001 From: Wenchen Date: Fri, 14 Sep 2018 12:41:28 +0000 Subject: [PATCH 045/879] Preparing development version 2.4.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index f52d785e05cd..714b6f18b19d 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.0 +Version: 2.4.1 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 63ab510eb683..ee0de73f21d5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b10e11849a74..b89e0fed2a66 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 74c6d232b36a..3105a8182afc 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index fbdc979d120d..f85248b53080 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 53d1b2b58f1f..3350990c0461 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 98145481adc7..9a2cb90a2f30 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index f0c8ebe25e63..9b927a9ba525 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index c635cd3b9394..718875f88e74 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ed3efffa180b..583a2fff7638 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index c3252a3e695d..20b6495d27df 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.0 -SPARK_VERSION_SHORT: 2.4.0 +SPARK_VERSION: 2.4.1-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.8" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index c7b8354b0968..737ebfd7900c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index a91c13362caa..7c6b78ac2b31 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index db239f23e1de..6343a15c43a8 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index cf19fed1bef5..2f4886832b75 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index da5dba82f3cf..9404e2a52d90 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index e591ce802256..73dbc8d99449 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index e8b13cdc2bef..912d82b764e5 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index c3d3b888656b..8454f5c108dc 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 468ace0ff8d7..05f7cf870af8 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index db8fda66c3cd..0e935ac112af 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index aa1e1267f57b..7977389718cb 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7d07e18f04a2..7193df69900d 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f545f212b2f0..4b318b00ff5b 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6b388ffcf18b..6fbb1b7db6aa 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index fc369c73edc0..9a42d9a69967 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index f4cb94147d9e..702f60ab9f00 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 22f60efae9af..62322da714bd 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index b17e67d78b98..c8e45b27c78f 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bf0d406da9ed..6201e0dd3c9a 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index dc434b7bb1c5..47b8772a8676 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 53e3424394ba..87282141fa64 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.0" +__version__ = "2.4.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 4532f0b8686e..d09548348eb5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 47fe3f2765b2..a9d9c8de9959 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index f59895e9a1bd..793167103179 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 0eacbbfac2b9..e6457289b859 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 2afdc15be446..e70711ce7257 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ef1d012d2f80..65cf33f32972 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f4be074642ae..c327b85ee866 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index e5c1065451f7..d9d742444e7e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b522bfeac130..c428be67aa3d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 8e71adabb3ec..37e96b807653 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c00e1b851ba0..dd00c2f6cd2c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml From 59054fa89b1f39f0d5d83cfe0b531ec39517f8fe Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 14 Sep 2018 09:25:27 -0700 Subject: [PATCH 046/879] [SPARK-25431][SQL][EXAMPLES] Fix function examples and unify the format of the example results. ## What changes were proposed in this pull request? There are some mistakes in examples of newly added functions. Also the format of the example results are not unified. We should fix and unify them. ## How was this patch tested? Manually executed the examples. Closes #22421 from ueshin/issues/SPARK-25431/fix_examples. Authored-by: Takuya UESHIN Signed-off-by: gatorsmile (cherry picked from commit 9c25d7f735ed8c49c795babea3fda3cab226e7cb) Signed-off-by: gatorsmile --- .../expressions/collectionOperations.scala | 49 ++++++++++--------- .../expressions/complexTypeCreator.scala | 4 +- .../expressions/higherOrderFunctions.scala | 32 ++++++------ 3 files changed, 43 insertions(+), 42 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index ea6fcccddfd4..3ad21ec5e51f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -131,7 +131,7 @@ case class Size(child: Expression) extends UnaryExpression with ExpectsInputType examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b')); - [1,2] + [1, 2] """) case class MapKeys(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -320,7 +320,7 @@ case class ArraysZip(children: Seq[Expression]) extends Expression with ExpectsI examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b')); - ["a","b"] + ["a", "b"] """) case class MapValues(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -348,7 +348,7 @@ case class MapValues(child: Expression) examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b')); - [(1,"a"),(2,"b")] + [[1, "a"], [2, "b"]] """, since = "2.4.0") case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -516,7 +516,7 @@ case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInp examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b'), map(2, 'c', 3, 'd')); - [[1 -> "a"], [2 -> "b"], [2 -> "c"], [3 -> "d"]] + [1 -> "a", 2 -> "b", 2 -> "c", 3 -> "d"] """, since = "2.4.0") case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpression { @@ -718,7 +718,7 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres examples = """ Examples: > SELECT _FUNC_(array(struct(1, 'a'), struct(2, 'b'))); - {1:"a",2:"b"} + [1 -> "a", 2 -> "b"] """, since = "2.4.0") case class MapFromEntries(child: Expression) extends UnaryExpression { @@ -1071,7 +1071,7 @@ object ArraySortLike { examples = """ Examples: > SELECT _FUNC_(array('b', 'd', null, 'c', 'a'), true); - [null,"a","b","c","d"] + [null, "a", "b", "c", "d"] """) // scalastyle:on line.size.limit case class SortArray(base: Expression, ascendingOrder: Expression) @@ -1129,7 +1129,7 @@ case class SortArray(base: Expression, ascendingOrder: Expression) examples = """ Examples: > SELECT _FUNC_(array('b', 'd', null, 'c', 'a')); - ["a","b","c","d",null] + ["a", "b", "c", "d", null] """, since = "2.4.0") // scalastyle:on line.size.limit @@ -1254,7 +1254,7 @@ case class Shuffle(child: Expression, randomSeed: Option[Long] = None) examples = """ Examples: > SELECT _FUNC_('Spark SQL'); - LQS krapS + "LQS krapS" > SELECT _FUNC_(array(2, 1, 4, 3)); [3, 4, 1, 2] """, @@ -1634,9 +1634,9 @@ case class ArraysOverlap(left: Expression, right: Expression) examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3, 4), 2, 2); - [2,3] + [2, 3] > SELECT _FUNC_(array(1, 2, 3, 4), -2, 2); - [3,4] + [3, 4] """, since = "2.4.0") // scalastyle:on line.size.limit case class Slice(x: Expression, start: Expression, length: Expression) @@ -1745,11 +1745,11 @@ case class Slice(x: Expression, start: Expression, length: Expression) examples = """ Examples: > SELECT _FUNC_(array('hello', 'world'), ' '); - hello world + "hello world" > SELECT _FUNC_(array('hello', null ,'world'), ' '); - hello world + "hello world" > SELECT _FUNC_(array('hello', null ,'world'), ' ', ','); - hello , world + "hello , world" """, since = "2.4.0") case class ArrayJoin( array: Expression, @@ -2236,10 +2236,11 @@ case class ElementAt(left: Expression, right: Expression) extends GetMapValueUti examples = """ Examples: > SELECT _FUNC_('Spark', 'SQL'); - SparkSQL + "SparkSQL" > SELECT _FUNC_(array(1, 2, 3), array(4, 5), array(6)); - | [1,2,3,4,5,6] - """) + [1, 2, 3, 4, 5, 6] + """, + note = "Concat logic for arrays is available since 2.4.0.") case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpression { private def allowedTypes: Seq[AbstractDataType] = Seq(StringType, BinaryType, ArrayType) @@ -2427,8 +2428,8 @@ case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpressio usage = "_FUNC_(arrayOfArrays) - Transforms an array of arrays into a single array.", examples = """ Examples: - > SELECT _FUNC_(array(array(1, 2), array(3, 4)); - [1,2,3,4] + > SELECT _FUNC_(array(array(1, 2), array(3, 4))); + [1, 2, 3, 4] """, since = "2.4.0") case class Flatten(child: Expression) extends UnaryExpression { @@ -2934,7 +2935,7 @@ object Sequence { examples = """ Examples: > SELECT _FUNC_('123', 2); - ['123', '123'] + ["123", "123"] """, since = "2.4.0") case class ArrayRepeat(left: Expression, right: Expression) @@ -3055,7 +3056,7 @@ case class ArrayRepeat(left: Expression, right: Expression) examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3, null, 3), 3); - [1,2,null] + [1, 2, null] """, since = "2.4.0") case class ArrayRemove(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -3245,7 +3246,7 @@ trait ArraySetLike { examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3, null, 3)); - [1,2,3,null] + [1, 2, 3, null] """, since = "2.4.0") case class ArrayDistinct(child: Expression) extends UnaryExpression with ArraySetLike with ExpectsInputTypes { @@ -3421,7 +3422,7 @@ object ArrayBinaryLike { examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5)); - array(1, 2, 3, 5) + [1, 2, 3, 5] """, since = "2.4.0") case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLike @@ -3632,7 +3633,7 @@ object ArrayUnion { examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5)); - array(1, 3) + [1, 3] """, since = "2.4.0") case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBinaryLike @@ -3873,7 +3874,7 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5)); - array(2) + [2] """, since = "2.4.0") case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryLike diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index aba9c6c8ad6f..117fa3e9aa51 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -248,8 +248,8 @@ case class CreateMap(children: Seq[Expression]) extends Expression { in keys should not be null""", examples = """ Examples: - > SELECT _FUNC_([1.0, 3.0], ['2', '4']); - {1.0:"2",3.0:"4"} + > SELECT _FUNC_(array(1.0, 3.0), array('2', '4')); + [1.0 -> "2", 3.0 -> "4"] """, since = "2.4.0") case class MapFromArrays(left: Expression, right: Expression) extends BinaryExpression with ExpectsInputTypes { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index 2bb6b20b944d..3ef2ec03099e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -209,9 +209,9 @@ trait MapBasedSimpleHigherOrderFunction extends SimpleHigherOrderFunction { examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), x -> x + 1); - array(2, 3, 4) + [2, 3, 4] > SELECT _FUNC_(array(1, 2, 3), (x, i) -> x + i); - array(1, 3, 5) + [1, 3, 5] """, since = "2.4.0") case class ArrayTransform( @@ -318,7 +318,7 @@ case class MapFilter( examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 1); - array(1, 3) + [1, 3] """, since = "2.4.0") case class ArrayFilter( @@ -499,10 +499,10 @@ case class ArrayAggregate( usage = "_FUNC_(expr, func) - Transforms elements in a map using the function.", examples = """ Examples: - > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + 1); - map(array(2, 3, 4), array(1, 2, 3)) - > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); - map(array(2, 4, 6), array(1, 2, 3)) + > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + 1); + [2 -> 1, 3 -> 2, 4 -> 3] + > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); + [2 -> 1, 4 -> 2, 6 -> 3] """, since = "2.4.0") case class TransformKeys( @@ -549,10 +549,10 @@ case class TransformKeys( usage = "_FUNC_(expr, func) - Transforms values in the map using the function.", examples = """ Examples: - > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> v + 1); - map(array(1, 2, 3), array(2, 3, 4)) - > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); - map(array(1, 2, 3), array(2, 4, 6)) + > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> v + 1); + [1 -> 2, 2 -> 3, 3 -> 4] + > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); + [1 -> 2, 2 -> 4, 3 -> 6] """, since = "2.4.0") case class TransformValues( @@ -603,7 +603,7 @@ case class TransformValues( examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b'), map(1, 'x', 2, 'y'), (k, v1, v2) -> concat(v1, v2)); - {1:"ax",2:"by"} + [1 -> "ax", 2 -> "by"] """, since = "2.4.0") case class MapZipWith(left: Expression, right: Expression, function: Expression) @@ -777,11 +777,11 @@ case class MapZipWith(left: Expression, right: Expression, function: Expression) examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)); - array(('a', 1), ('b', 2), ('c', 3)) - > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y)); - array(4, 6) + [["a", 1], ["b", 2], ["c", 3]] + > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y); + [4, 6] > SELECT _FUNC_(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, y)); - array('ad', 'be', 'cf') + ["ad", "be", "cf"] """, since = "2.4.0") // scalastyle:on line.size.limit From d3f5475a1a9efc1dcffbf5a4697f8431b0588e9e Mon Sep 17 00:00:00 2001 From: cclauss Date: Fri, 14 Sep 2018 20:13:07 -0500 Subject: [PATCH 047/879] [SPARK-25238][PYTHON] lint-python: Upgrade pycodestyle to v2.4.0 See https://pycodestyle.readthedocs.io/en/latest/developer.html#changes for changes made in this release. ## What changes were proposed in this pull request? Upgrade pycodestyle to v2.4.0 ## How was this patch tested? __pycodestyle__ Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #22231 from cclauss/patch-1. Authored-by: cclauss Signed-off-by: Sean Owen (cherry picked from commit 9bb798f2e6eefd9edb7b6d9980a894557c107bd3) Signed-off-by: Sean Owen --- dev/lint-python | 2 +- dev/run-tests-jenkins.py | 4 ++-- dev/tox.ini | 2 +- python/pyspark/sql/functions.py | 2 +- 4 files changed, 5 insertions(+), 5 deletions(-) diff --git a/dev/lint-python b/dev/lint-python index f738af9c4976..1e944b0baf08 100755 --- a/dev/lint-python +++ b/dev/lint-python @@ -36,7 +36,7 @@ compile_status="${PIPESTATUS[0]}" # Get pycodestyle at runtime so that we don't rely on it being installed on the build server. # See: https://github.com/apache/spark/pull/1744#issuecomment-50982162 # Updated to the latest official version of pep8. pep8 is formally renamed to pycodestyle. -PYCODESTYLE_VERSION="2.3.1" +PYCODESTYLE_VERSION="2.4.0" PYCODESTYLE_SCRIPT_PATH="$SPARK_ROOT_DIR/dev/pycodestyle-$PYCODESTYLE_VERSION.py" PYCODESTYLE_SCRIPT_REMOTE_PATH="https://raw.githubusercontent.com/PyCQA/pycodestyle/$PYCODESTYLE_VERSION/pycodestyle.py" diff --git a/dev/run-tests-jenkins.py b/dev/run-tests-jenkins.py index 6e943898ffed..eca88f2391bf 100755 --- a/dev/run-tests-jenkins.py +++ b/dev/run-tests-jenkins.py @@ -116,7 +116,7 @@ def run_tests(tests_timeout): failure_note_by_errcode = { # error to denote run-tests script failures: - 1: 'executing the `dev/run-tests` script', # noqa: W605 + 1: 'executing the `dev/run-tests` script', ERROR_CODES["BLOCK_GENERAL"]: 'some tests', ERROR_CODES["BLOCK_RAT"]: 'RAT tests', ERROR_CODES["BLOCK_SCALA_STYLE"]: 'Scala style tests', @@ -131,7 +131,7 @@ def run_tests(tests_timeout): ERROR_CODES["BLOCK_PYSPARK_UNIT_TESTS"]: 'PySpark unit tests', ERROR_CODES["BLOCK_PYSPARK_PIP_TESTS"]: 'PySpark pip packaging tests', ERROR_CODES["BLOCK_SPARKR_UNIT_TESTS"]: 'SparkR unit tests', - ERROR_CODES["BLOCK_TIMEOUT"]: 'from timeout after a configured wait of \`%s\`' % ( + ERROR_CODES["BLOCK_TIMEOUT"]: 'from timeout after a configured wait of `%s`' % ( tests_timeout) } diff --git a/dev/tox.ini b/dev/tox.ini index 28dad8f3b5c7..6ec223b743b4 100644 --- a/dev/tox.ini +++ b/dev/tox.ini @@ -14,6 +14,6 @@ # limitations under the License. [pycodestyle] -ignore=E402,E731,E241,W503,E226,E722,E741,E305 +ignore=E226,E241,E305,E402,E722,E731,E741,W503,W504 max-line-length=100 exclude=cloudpickle.py,heapq3.py,shared.py,python/docs/conf.py,work/*/*.py,python/.eggs/*,dist/* diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index e288ec818b40..6da5237d18de 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1711,7 +1711,7 @@ def regexp_extract(str, pattern, idx): @ignore_unicode_prefix @since(1.5) def regexp_replace(str, pattern, replacement): - """Replace all substrings of the specified string value that match regexp with rep. + r"""Replace all substrings of the specified string value that match regexp with rep. >>> df = spark.createDataFrame([('100-200',)], ['str']) >>> df.select(regexp_replace('str', r'(\d+)', '--').alias('d')).collect() From ae2ca0e5ddc477dea3fdffafe5b69f548b502692 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sat, 15 Sep 2018 12:51:46 +0900 Subject: [PATCH 048/879] Revert "[SPARK-25431][SQL][EXAMPLES] Fix function examples and unify the format of the example results." This reverts commit 59054fa89b1f39f0d5d83cfe0b531ec39517f8fe. --- .../expressions/collectionOperations.scala | 49 +++++++++---------- .../expressions/complexTypeCreator.scala | 4 +- .../expressions/higherOrderFunctions.scala | 32 ++++++------ 3 files changed, 42 insertions(+), 43 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 3ad21ec5e51f..ea6fcccddfd4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -131,7 +131,7 @@ case class Size(child: Expression) extends UnaryExpression with ExpectsInputType examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b')); - [1, 2] + [1,2] """) case class MapKeys(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -320,7 +320,7 @@ case class ArraysZip(children: Seq[Expression]) extends Expression with ExpectsI examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b')); - ["a", "b"] + ["a","b"] """) case class MapValues(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -348,7 +348,7 @@ case class MapValues(child: Expression) examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b')); - [[1, "a"], [2, "b"]] + [(1,"a"),(2,"b")] """, since = "2.4.0") case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -516,7 +516,7 @@ case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInp examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b'), map(2, 'c', 3, 'd')); - [1 -> "a", 2 -> "b", 2 -> "c", 3 -> "d"] + [[1 -> "a"], [2 -> "b"], [2 -> "c"], [3 -> "d"]] """, since = "2.4.0") case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpression { @@ -718,7 +718,7 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres examples = """ Examples: > SELECT _FUNC_(array(struct(1, 'a'), struct(2, 'b'))); - [1 -> "a", 2 -> "b"] + {1:"a",2:"b"} """, since = "2.4.0") case class MapFromEntries(child: Expression) extends UnaryExpression { @@ -1071,7 +1071,7 @@ object ArraySortLike { examples = """ Examples: > SELECT _FUNC_(array('b', 'd', null, 'c', 'a'), true); - [null, "a", "b", "c", "d"] + [null,"a","b","c","d"] """) // scalastyle:on line.size.limit case class SortArray(base: Expression, ascendingOrder: Expression) @@ -1129,7 +1129,7 @@ case class SortArray(base: Expression, ascendingOrder: Expression) examples = """ Examples: > SELECT _FUNC_(array('b', 'd', null, 'c', 'a')); - ["a", "b", "c", "d", null] + ["a","b","c","d",null] """, since = "2.4.0") // scalastyle:on line.size.limit @@ -1254,7 +1254,7 @@ case class Shuffle(child: Expression, randomSeed: Option[Long] = None) examples = """ Examples: > SELECT _FUNC_('Spark SQL'); - "LQS krapS" + LQS krapS > SELECT _FUNC_(array(2, 1, 4, 3)); [3, 4, 1, 2] """, @@ -1634,9 +1634,9 @@ case class ArraysOverlap(left: Expression, right: Expression) examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3, 4), 2, 2); - [2, 3] + [2,3] > SELECT _FUNC_(array(1, 2, 3, 4), -2, 2); - [3, 4] + [3,4] """, since = "2.4.0") // scalastyle:on line.size.limit case class Slice(x: Expression, start: Expression, length: Expression) @@ -1745,11 +1745,11 @@ case class Slice(x: Expression, start: Expression, length: Expression) examples = """ Examples: > SELECT _FUNC_(array('hello', 'world'), ' '); - "hello world" + hello world > SELECT _FUNC_(array('hello', null ,'world'), ' '); - "hello world" + hello world > SELECT _FUNC_(array('hello', null ,'world'), ' ', ','); - "hello , world" + hello , world """, since = "2.4.0") case class ArrayJoin( array: Expression, @@ -2236,11 +2236,10 @@ case class ElementAt(left: Expression, right: Expression) extends GetMapValueUti examples = """ Examples: > SELECT _FUNC_('Spark', 'SQL'); - "SparkSQL" + SparkSQL > SELECT _FUNC_(array(1, 2, 3), array(4, 5), array(6)); - [1, 2, 3, 4, 5, 6] - """, - note = "Concat logic for arrays is available since 2.4.0.") + | [1,2,3,4,5,6] + """) case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpression { private def allowedTypes: Seq[AbstractDataType] = Seq(StringType, BinaryType, ArrayType) @@ -2428,8 +2427,8 @@ case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpressio usage = "_FUNC_(arrayOfArrays) - Transforms an array of arrays into a single array.", examples = """ Examples: - > SELECT _FUNC_(array(array(1, 2), array(3, 4))); - [1, 2, 3, 4] + > SELECT _FUNC_(array(array(1, 2), array(3, 4)); + [1,2,3,4] """, since = "2.4.0") case class Flatten(child: Expression) extends UnaryExpression { @@ -2935,7 +2934,7 @@ object Sequence { examples = """ Examples: > SELECT _FUNC_('123', 2); - ["123", "123"] + ['123', '123'] """, since = "2.4.0") case class ArrayRepeat(left: Expression, right: Expression) @@ -3056,7 +3055,7 @@ case class ArrayRepeat(left: Expression, right: Expression) examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3, null, 3), 3); - [1, 2, null] + [1,2,null] """, since = "2.4.0") case class ArrayRemove(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -3246,7 +3245,7 @@ trait ArraySetLike { examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3, null, 3)); - [1, 2, 3, null] + [1,2,3,null] """, since = "2.4.0") case class ArrayDistinct(child: Expression) extends UnaryExpression with ArraySetLike with ExpectsInputTypes { @@ -3422,7 +3421,7 @@ object ArrayBinaryLike { examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5)); - [1, 2, 3, 5] + array(1, 2, 3, 5) """, since = "2.4.0") case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLike @@ -3633,7 +3632,7 @@ object ArrayUnion { examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5)); - [1, 3] + array(1, 3) """, since = "2.4.0") case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBinaryLike @@ -3874,7 +3873,7 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5)); - [2] + array(2) """, since = "2.4.0") case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryLike diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 117fa3e9aa51..aba9c6c8ad6f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -248,8 +248,8 @@ case class CreateMap(children: Seq[Expression]) extends Expression { in keys should not be null""", examples = """ Examples: - > SELECT _FUNC_(array(1.0, 3.0), array('2', '4')); - [1.0 -> "2", 3.0 -> "4"] + > SELECT _FUNC_([1.0, 3.0], ['2', '4']); + {1.0:"2",3.0:"4"} """, since = "2.4.0") case class MapFromArrays(left: Expression, right: Expression) extends BinaryExpression with ExpectsInputTypes { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index 3ef2ec03099e..2bb6b20b944d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -209,9 +209,9 @@ trait MapBasedSimpleHigherOrderFunction extends SimpleHigherOrderFunction { examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), x -> x + 1); - [2, 3, 4] + array(2, 3, 4) > SELECT _FUNC_(array(1, 2, 3), (x, i) -> x + i); - [1, 3, 5] + array(1, 3, 5) """, since = "2.4.0") case class ArrayTransform( @@ -318,7 +318,7 @@ case class MapFilter( examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 1); - [1, 3] + array(1, 3) """, since = "2.4.0") case class ArrayFilter( @@ -499,10 +499,10 @@ case class ArrayAggregate( usage = "_FUNC_(expr, func) - Transforms elements in a map using the function.", examples = """ Examples: - > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + 1); - [2 -> 1, 3 -> 2, 4 -> 3] - > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); - [2 -> 1, 4 -> 2, 6 -> 3] + > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + 1); + map(array(2, 3, 4), array(1, 2, 3)) + > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); + map(array(2, 4, 6), array(1, 2, 3)) """, since = "2.4.0") case class TransformKeys( @@ -549,10 +549,10 @@ case class TransformKeys( usage = "_FUNC_(expr, func) - Transforms values in the map using the function.", examples = """ Examples: - > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> v + 1); - [1 -> 2, 2 -> 3, 3 -> 4] - > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); - [1 -> 2, 2 -> 4, 3 -> 6] + > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> v + 1); + map(array(1, 2, 3), array(2, 3, 4)) + > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); + map(array(1, 2, 3), array(2, 4, 6)) """, since = "2.4.0") case class TransformValues( @@ -603,7 +603,7 @@ case class TransformValues( examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b'), map(1, 'x', 2, 'y'), (k, v1, v2) -> concat(v1, v2)); - [1 -> "ax", 2 -> "by"] + {1:"ax",2:"by"} """, since = "2.4.0") case class MapZipWith(left: Expression, right: Expression, function: Expression) @@ -777,11 +777,11 @@ case class MapZipWith(left: Expression, right: Expression, function: Expression) examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)); - [["a", 1], ["b", 2], ["c", 3]] - > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y); - [4, 6] + array(('a', 1), ('b', 2), ('c', 3)) + > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y)); + array(4, 6) > SELECT _FUNC_(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, y)); - ["ad", "be", "cf"] + array('ad', 'be', 'cf') """, since = "2.4.0") // scalastyle:on line.size.limit From b40e5feec2660891590e21807133a508cbd004d3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 15 Sep 2018 17:48:39 -0700 Subject: [PATCH 049/879] [SPARK-25438][SQL][TEST] Fix FilterPushdownBenchmark to use the same memory assumption ## What changes were proposed in this pull request? This PR aims to fix three things in `FilterPushdownBenchmark`. **1. Use the same memory assumption.** The following configurations are used in ORC and Parquet. - Memory buffer for writing - parquet.block.size (default: 128MB) - orc.stripe.size (default: 64MB) - Compression chunk size - parquet.page.size (default: 1MB) - orc.compress.size (default: 256KB) SPARK-24692 used 1MB, the default value of `parquet.page.size`, for `parquet.block.size` and `orc.stripe.size`. But, it missed to match `orc.compress.size`. So, the current benchmark shows the result from ORC with 256KB memory for compression and Parquet with 1MB. To compare correctly, we need to be consistent. **2. Dictionary encoding should not be enforced for all cases.** SPARK-24206 enforced dictionary encoding for all test cases. This PR recovers the default behavior in general and enforces dictionary encoding only in case of `prepareStringDictTable`. **3. Generate test result on AWS r3.xlarge** SPARK-24206 generated the result on AWS in order to reproduce and compare easily. This PR also aims to update the result on the same machine again in the same reason. Specifically, AWS r3.xlarge with Instance Store is used. ## How was this patch tested? Manual. Enable the test cases and run `FilterPushdownBenchmark` on `AWS r3.xlarge`. It takes about 4 hours 15 minutes. Closes #22427 from dongjoon-hyun/SPARK-25438. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit fefaa3c30df2c56046370081cb51bfe68d26976b) Signed-off-by: Dongjoon Hyun --- .../FilterPushdownBenchmark-results.txt | 912 ++++++++---------- .../benchmark/FilterPushdownBenchmark.scala | 11 +- 2 files changed, 428 insertions(+), 495 deletions(-) diff --git a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt index a75a15c99328..e680ddff53dd 100644 --- a/sql/core/benchmarks/FilterPushdownBenchmark-results.txt +++ b/sql/core/benchmarks/FilterPushdownBenchmark-results.txt @@ -2,737 +2,669 @@ Pushdown for many distinct value case ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 0 string row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8970 / 9122 1.8 570.3 1.0X -Parquet Vectorized (Pushdown) 471 / 491 33.4 30.0 19.0X -Native ORC Vectorized 7661 / 7853 2.1 487.0 1.2X -Native ORC Vectorized (Pushdown) 1134 / 1161 13.9 72.1 7.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 11405 / 11485 1.4 725.1 1.0X +Parquet Vectorized (Pushdown) 675 / 690 23.3 42.9 16.9X +Native ORC Vectorized 7127 / 7170 2.2 453.1 1.6X +Native ORC Vectorized (Pushdown) 519 / 541 30.3 33.0 22.0X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 0 string row ('7864320' < value < '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 9246 / 9297 1.7 587.8 1.0X -Parquet Vectorized (Pushdown) 480 / 488 32.8 30.5 19.3X -Native ORC Vectorized 7838 / 7850 2.0 498.3 1.2X -Native ORC Vectorized (Pushdown) 1054 / 1118 14.9 67.0 8.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 11457 / 11473 1.4 728.4 1.0X +Parquet Vectorized (Pushdown) 656 / 686 24.0 41.7 17.5X +Native ORC Vectorized 7328 / 7342 2.1 465.9 1.6X +Native ORC Vectorized (Pushdown) 539 / 565 29.2 34.2 21.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 string row (value = '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8989 / 9100 1.7 571.5 1.0X -Parquet Vectorized (Pushdown) 448 / 467 35.1 28.5 20.1X -Native ORC Vectorized 7680 / 7768 2.0 488.3 1.2X -Native ORC Vectorized (Pushdown) 1067 / 1118 14.7 67.8 8.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 11878 / 11888 1.3 755.2 1.0X +Parquet Vectorized (Pushdown) 630 / 654 25.0 40.1 18.9X +Native ORC Vectorized 7342 / 7362 2.1 466.8 1.6X +Native ORC Vectorized (Pushdown) 519 / 537 30.3 33.0 22.9X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 string row (value <=> '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 9115 / 9266 1.7 579.5 1.0X -Parquet Vectorized (Pushdown) 466 / 492 33.7 29.7 19.5X -Native ORC Vectorized 7800 / 7914 2.0 495.9 1.2X -Native ORC Vectorized (Pushdown) 1075 / 1102 14.6 68.4 8.5X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 11423 / 11440 1.4 726.2 1.0X +Parquet Vectorized (Pushdown) 625 / 643 25.2 39.7 18.3X +Native ORC Vectorized 7315 / 7335 2.2 465.1 1.6X +Native ORC Vectorized (Pushdown) 507 / 520 31.0 32.2 22.5X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 string row ('7864320' <= value <= '7864320'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 9099 / 9237 1.7 578.5 1.0X -Parquet Vectorized (Pushdown) 462 / 475 34.1 29.3 19.7X -Native ORC Vectorized 7847 / 7925 2.0 498.9 1.2X -Native ORC Vectorized (Pushdown) 1078 / 1114 14.6 68.5 8.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 11440 / 11478 1.4 727.3 1.0X +Parquet Vectorized (Pushdown) 634 / 652 24.8 40.3 18.0X +Native ORC Vectorized 7311 / 7324 2.2 464.8 1.6X +Native ORC Vectorized (Pushdown) 517 / 548 30.4 32.8 22.1X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select all string rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 19303 / 19547 0.8 1227.3 1.0X -Parquet Vectorized (Pushdown) 19924 / 20089 0.8 1266.7 1.0X -Native ORC Vectorized 18725 / 19079 0.8 1190.5 1.0X -Native ORC Vectorized (Pushdown) 19310 / 19492 0.8 1227.7 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 20750 / 20872 0.8 1319.3 1.0X +Parquet Vectorized (Pushdown) 21002 / 21032 0.7 1335.3 1.0X +Native ORC Vectorized 16714 / 16742 0.9 1062.6 1.2X +Native ORC Vectorized (Pushdown) 16926 / 16965 0.9 1076.1 1.2X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 0 int row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8117 / 8323 1.9 516.1 1.0X -Parquet Vectorized (Pushdown) 484 / 494 32.5 30.8 16.8X -Native ORC Vectorized 6811 / 7036 2.3 433.0 1.2X -Native ORC Vectorized (Pushdown) 1061 / 1082 14.8 67.5 7.6X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10510 / 10532 1.5 668.2 1.0X +Parquet Vectorized (Pushdown) 642 / 665 24.5 40.8 16.4X +Native ORC Vectorized 6609 / 6618 2.4 420.2 1.6X +Native ORC Vectorized (Pushdown) 502 / 512 31.4 31.9 21.0X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 0 int row (7864320 < value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8105 / 8140 1.9 515.3 1.0X -Parquet Vectorized (Pushdown) 478 / 505 32.9 30.4 17.0X -Native ORC Vectorized 6914 / 7211 2.3 439.6 1.2X -Native ORC Vectorized (Pushdown) 1044 / 1064 15.1 66.4 7.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10505 / 10514 1.5 667.9 1.0X +Parquet Vectorized (Pushdown) 659 / 673 23.9 41.9 15.9X +Native ORC Vectorized 6634 / 6641 2.4 421.8 1.6X +Native ORC Vectorized (Pushdown) 513 / 526 30.7 32.6 20.5X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 int row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7983 / 8116 2.0 507.6 1.0X -Parquet Vectorized (Pushdown) 464 / 487 33.9 29.5 17.2X -Native ORC Vectorized 6703 / 6774 2.3 426.1 1.2X -Native ORC Vectorized (Pushdown) 1017 / 1058 15.5 64.6 7.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10555 / 10570 1.5 671.1 1.0X +Parquet Vectorized (Pushdown) 651 / 668 24.2 41.4 16.2X +Native ORC Vectorized 6721 / 6728 2.3 427.3 1.6X +Native ORC Vectorized (Pushdown) 508 / 519 31.0 32.3 20.8X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 int row (value <=> 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7942 / 7983 2.0 504.9 1.0X -Parquet Vectorized (Pushdown) 468 / 479 33.6 29.7 17.0X -Native ORC Vectorized 6677 / 6779 2.4 424.5 1.2X -Native ORC Vectorized (Pushdown) 1021 / 1068 15.4 64.9 7.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10556 / 10566 1.5 671.1 1.0X +Parquet Vectorized (Pushdown) 647 / 654 24.3 41.1 16.3X +Native ORC Vectorized 6716 / 6728 2.3 427.0 1.6X +Native ORC Vectorized (Pushdown) 510 / 521 30.9 32.4 20.7X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 int row (7864320 <= value <= 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7909 / 7958 2.0 502.8 1.0X -Parquet Vectorized (Pushdown) 485 / 494 32.4 30.8 16.3X -Native ORC Vectorized 6751 / 6846 2.3 429.2 1.2X -Native ORC Vectorized (Pushdown) 1043 / 1077 15.1 66.3 7.6X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10556 / 10565 1.5 671.1 1.0X +Parquet Vectorized (Pushdown) 649 / 654 24.2 41.3 16.3X +Native ORC Vectorized 6700 / 6712 2.3 426.0 1.6X +Native ORC Vectorized (Pushdown) 509 / 520 30.9 32.3 20.8X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 int row (7864319 < value < 7864321): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8010 / 8033 2.0 509.2 1.0X -Parquet Vectorized (Pushdown) 472 / 489 33.3 30.0 17.0X -Native ORC Vectorized 6655 / 6808 2.4 423.1 1.2X -Native ORC Vectorized (Pushdown) 1015 / 1067 15.5 64.5 7.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10547 / 10566 1.5 670.5 1.0X +Parquet Vectorized (Pushdown) 649 / 653 24.2 41.3 16.3X +Native ORC Vectorized 6703 / 6713 2.3 426.2 1.6X +Native ORC Vectorized (Pushdown) 510 / 520 30.8 32.5 20.7X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 10% int rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8983 / 9035 1.8 571.1 1.0X -Parquet Vectorized (Pushdown) 2204 / 2231 7.1 140.1 4.1X -Native ORC Vectorized 7864 / 8011 2.0 500.0 1.1X -Native ORC Vectorized (Pushdown) 2674 / 2789 5.9 170.0 3.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 11478 / 11525 1.4 729.7 1.0X +Parquet Vectorized (Pushdown) 2576 / 2587 6.1 163.8 4.5X +Native ORC Vectorized 7633 / 7657 2.1 485.3 1.5X +Native ORC Vectorized (Pushdown) 2076 / 2096 7.6 132.0 5.5X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 50% int rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 12723 / 12903 1.2 808.9 1.0X -Parquet Vectorized (Pushdown) 9112 / 9282 1.7 579.3 1.4X -Native ORC Vectorized 12090 / 12230 1.3 768.7 1.1X -Native ORC Vectorized (Pushdown) 9242 / 9372 1.7 587.6 1.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 14785 / 14802 1.1 940.0 1.0X +Parquet Vectorized (Pushdown) 9971 / 9977 1.6 633.9 1.5X +Native ORC Vectorized 11082 / 11107 1.4 704.6 1.3X +Native ORC Vectorized (Pushdown) 8061 / 8073 2.0 512.5 1.8X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 90% int rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 16453 / 16678 1.0 1046.1 1.0X -Parquet Vectorized (Pushdown) 15997 / 16262 1.0 1017.0 1.0X -Native ORC Vectorized 16652 / 17070 0.9 1058.7 1.0X -Native ORC Vectorized (Pushdown) 15843 / 16112 1.0 1007.2 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 18174 / 18214 0.9 1155.5 1.0X +Parquet Vectorized (Pushdown) 17387 / 17403 0.9 1105.5 1.0X +Native ORC Vectorized 14465 / 14492 1.1 919.7 1.3X +Native ORC Vectorized (Pushdown) 14024 / 14041 1.1 891.6 1.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select all int rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 17098 / 17254 0.9 1087.1 1.0X -Parquet Vectorized (Pushdown) 17302 / 17529 0.9 1100.1 1.0X -Native ORC Vectorized 16790 / 17098 0.9 1067.5 1.0X -Native ORC Vectorized (Pushdown) 17329 / 17914 0.9 1101.7 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 19004 / 19014 0.8 1208.2 1.0X +Parquet Vectorized (Pushdown) 19219 / 19232 0.8 1221.9 1.0X +Native ORC Vectorized 15266 / 15290 1.0 970.6 1.2X +Native ORC Vectorized (Pushdown) 15469 / 15482 1.0 983.5 1.2X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select all int rows (value > -1): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 17088 / 17392 0.9 1086.4 1.0X -Parquet Vectorized (Pushdown) 17609 / 17863 0.9 1119.5 1.0X -Native ORC Vectorized 18334 / 69831 0.9 1165.7 0.9X -Native ORC Vectorized (Pushdown) 17465 / 17629 0.9 1110.4 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 19036 / 19052 0.8 1210.3 1.0X +Parquet Vectorized (Pushdown) 19287 / 19306 0.8 1226.2 1.0X +Native ORC Vectorized 15311 / 15371 1.0 973.5 1.2X +Native ORC Vectorized (Pushdown) 15517 / 15590 1.0 986.5 1.2X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select all int rows (value != -1): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 16903 / 17233 0.9 1074.6 1.0X -Parquet Vectorized (Pushdown) 16945 / 17032 0.9 1077.3 1.0X -Native ORC Vectorized 16377 / 16762 1.0 1041.2 1.0X -Native ORC Vectorized (Pushdown) 16950 / 17212 0.9 1077.7 1.0X +Parquet Vectorized 19072 / 19102 0.8 1212.6 1.0X +Parquet Vectorized (Pushdown) 19288 / 19318 0.8 1226.3 1.0X +Native ORC Vectorized 15277 / 15293 1.0 971.3 1.2X +Native ORC Vectorized (Pushdown) 15479 / 15499 1.0 984.1 1.2X ================================================================================================ Pushdown for few distinct value case (use dictionary encoding) ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 0 distinct string row (value IS NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7245 / 7322 2.2 460.7 1.0X -Parquet Vectorized (Pushdown) 378 / 389 41.6 24.0 19.2X -Native ORC Vectorized 6720 / 6778 2.3 427.2 1.1X -Native ORC Vectorized (Pushdown) 1009 / 1032 15.6 64.2 7.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10250 / 10274 1.5 651.7 1.0X +Parquet Vectorized (Pushdown) 571 / 576 27.5 36.3 17.9X +Native ORC Vectorized 8651 / 8660 1.8 550.0 1.2X +Native ORC Vectorized (Pushdown) 909 / 933 17.3 57.8 11.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 0 distinct string row ('100' < value < '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7627 / 7795 2.1 484.9 1.0X -Parquet Vectorized (Pushdown) 384 / 406 41.0 24.4 19.9X -Native ORC Vectorized 6724 / 7824 2.3 427.5 1.1X -Native ORC Vectorized (Pushdown) 968 / 986 16.3 61.5 7.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10420 / 10426 1.5 662.5 1.0X +Parquet Vectorized (Pushdown) 574 / 579 27.4 36.5 18.2X +Native ORC Vectorized 8973 / 8982 1.8 570.5 1.2X +Native ORC Vectorized (Pushdown) 916 / 955 17.2 58.2 11.4X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 distinct string row (value = '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7157 / 7534 2.2 455.0 1.0X -Parquet Vectorized (Pushdown) 542 / 565 29.0 34.5 13.2X -Native ORC Vectorized 6716 / 7214 2.3 427.0 1.1X -Native ORC Vectorized (Pushdown) 1212 / 1288 13.0 77.0 5.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10428 / 10441 1.5 663.0 1.0X +Parquet Vectorized (Pushdown) 789 / 809 19.9 50.2 13.2X +Native ORC Vectorized 9042 / 9055 1.7 574.9 1.2X +Native ORC Vectorized (Pushdown) 1130 / 1145 13.9 71.8 9.2X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 distinct string row (value <=> '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7368 / 7552 2.1 468.4 1.0X -Parquet Vectorized (Pushdown) 544 / 556 28.9 34.6 13.5X -Native ORC Vectorized 6740 / 6867 2.3 428.5 1.1X -Native ORC Vectorized (Pushdown) 1230 / 1426 12.8 78.2 6.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10402 / 10416 1.5 661.3 1.0X +Parquet Vectorized (Pushdown) 791 / 806 19.9 50.3 13.2X +Native ORC Vectorized 9042 / 9055 1.7 574.9 1.2X +Native ORC Vectorized (Pushdown) 1112 / 1145 14.1 70.7 9.4X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 distinct string row ('100' <= value <= '100'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7427 / 7734 2.1 472.2 1.0X -Parquet Vectorized (Pushdown) 556 / 568 28.3 35.4 13.3X -Native ORC Vectorized 6847 / 7059 2.3 435.3 1.1X -Native ORC Vectorized (Pushdown) 1226 / 1230 12.8 77.9 6.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10548 / 10563 1.5 670.6 1.0X +Parquet Vectorized (Pushdown) 790 / 796 19.9 50.2 13.4X +Native ORC Vectorized 9144 / 9153 1.7 581.3 1.2X +Native ORC Vectorized (Pushdown) 1117 / 1148 14.1 71.0 9.4X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select all distinct string rows (value IS NOT NULL): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 16998 / 17311 0.9 1080.7 1.0X -Parquet Vectorized (Pushdown) 16977 / 17250 0.9 1079.4 1.0X -Native ORC Vectorized 18447 / 19852 0.9 1172.8 0.9X -Native ORC Vectorized (Pushdown) 16614 / 17102 0.9 1056.3 1.0X +Parquet Vectorized 20445 / 20469 0.8 1299.8 1.0X +Parquet Vectorized (Pushdown) 20686 / 20699 0.8 1315.2 1.0X +Native ORC Vectorized 18851 / 18953 0.8 1198.5 1.1X +Native ORC Vectorized (Pushdown) 19255 / 19268 0.8 1224.2 1.1X ================================================================================================ Pushdown benchmark for StringStartsWith ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz StringStartsWith filter: (value like '10%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 9705 / 10814 1.6 617.0 1.0X -Parquet Vectorized (Pushdown) 3086 / 3574 5.1 196.2 3.1X -Native ORC Vectorized 10094 / 10695 1.6 641.8 1.0X -Native ORC Vectorized (Pushdown) 9611 / 9999 1.6 611.0 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 14265 / 15213 1.1 907.0 1.0X +Parquet Vectorized (Pushdown) 4228 / 4870 3.7 268.8 3.4X +Native ORC Vectorized 10116 / 10977 1.6 643.2 1.4X +Native ORC Vectorized (Pushdown) 10653 / 11376 1.5 677.3 1.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz StringStartsWith filter: (value like '1000%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8016 / 8183 2.0 509.7 1.0X -Parquet Vectorized (Pushdown) 444 / 457 35.4 28.2 18.0X -Native ORC Vectorized 6970 / 7169 2.3 443.2 1.2X -Native ORC Vectorized (Pushdown) 7447 / 7503 2.1 473.5 1.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 11499 / 11539 1.4 731.1 1.0X +Parquet Vectorized (Pushdown) 669 / 672 23.5 42.5 17.2X +Native ORC Vectorized 7343 / 7363 2.1 466.8 1.6X +Native ORC Vectorized (Pushdown) 7559 / 7568 2.1 480.6 1.5X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz StringStartsWith filter: (value like '786432%'): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7908 / 8046 2.0 502.8 1.0X -Parquet Vectorized (Pushdown) 408 / 429 38.6 25.9 19.4X -Native ORC Vectorized 7021 / 7100 2.2 446.4 1.1X -Native ORC Vectorized (Pushdown) 7310 / 7490 2.2 464.8 1.1X +Parquet Vectorized 11463 / 11468 1.4 728.8 1.0X +Parquet Vectorized (Pushdown) 647 / 651 24.3 41.1 17.7X +Native ORC Vectorized 7322 / 7338 2.1 465.5 1.6X +Native ORC Vectorized (Pushdown) 7533 / 7544 2.1 478.9 1.5X ================================================================================================ Pushdown benchmark for decimal ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 decimal(9, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 4546 / 4743 3.5 289.0 1.0X -Parquet Vectorized (Pushdown) 161 / 175 98.0 10.2 28.3X -Native ORC Vectorized 5721 / 5842 2.7 363.7 0.8X -Native ORC Vectorized (Pushdown) 1019 / 1070 15.4 64.8 4.5X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 5543 / 5564 2.8 352.4 1.0X +Parquet Vectorized (Pushdown) 168 / 174 93.7 10.7 33.0X +Native ORC Vectorized 4992 / 5052 3.2 317.4 1.1X +Native ORC Vectorized (Pushdown) 840 / 850 18.7 53.4 6.6X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 10% decimal(9, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 6340 / 7236 2.5 403.1 1.0X -Parquet Vectorized (Pushdown) 3052 / 3164 5.2 194.1 2.1X -Native ORC Vectorized 8370 / 9214 1.9 532.1 0.8X -Native ORC Vectorized (Pushdown) 4137 / 4242 3.8 263.0 1.5X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 7312 / 7358 2.2 464.9 1.0X +Parquet Vectorized (Pushdown) 3008 / 3078 5.2 191.2 2.4X +Native ORC Vectorized 6775 / 6798 2.3 430.7 1.1X +Native ORC Vectorized (Pushdown) 6819 / 6832 2.3 433.5 1.1X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 50% decimal(9, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 12976 / 13249 1.2 825.0 1.0X -Parquet Vectorized (Pushdown) 12655 / 13570 1.2 804.6 1.0X -Native ORC Vectorized 15562 / 15950 1.0 989.4 0.8X -Native ORC Vectorized (Pushdown) 15042 / 15668 1.0 956.3 0.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 13232 / 13241 1.2 841.3 1.0X +Parquet Vectorized (Pushdown) 12555 / 12569 1.3 798.2 1.1X +Native ORC Vectorized 12597 / 12627 1.2 800.9 1.1X +Native ORC Vectorized (Pushdown) 12677 / 12711 1.2 806.0 1.0X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 90% decimal(9, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 14303 / 14616 1.1 909.3 1.0X -Parquet Vectorized (Pushdown) 14380 / 14649 1.1 914.3 1.0X -Native ORC Vectorized 16964 / 17358 0.9 1078.5 0.8X -Native ORC Vectorized (Pushdown) 17255 / 17874 0.9 1097.0 0.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 14725 / 14729 1.1 936.2 1.0X +Parquet Vectorized (Pushdown) 14781 / 14800 1.1 939.7 1.0X +Native ORC Vectorized 15360 / 15453 1.0 976.5 1.0X +Native ORC Vectorized (Pushdown) 15444 / 15466 1.0 981.9 1.0X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 decimal(18, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 4701 / 6416 3.3 298.9 1.0X -Parquet Vectorized (Pushdown) 128 / 164 122.8 8.1 36.7X -Native ORC Vectorized 5698 / 7904 2.8 362.3 0.8X -Native ORC Vectorized (Pushdown) 913 / 942 17.2 58.0 5.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 5746 / 5763 2.7 365.3 1.0X +Parquet Vectorized (Pushdown) 166 / 169 94.8 10.6 34.6X +Native ORC Vectorized 5007 / 5023 3.1 318.3 1.1X +Native ORC Vectorized (Pushdown) 2629 / 2640 6.0 167.1 2.2X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 10% decimal(18, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 5376 / 5461 2.9 341.8 1.0X -Parquet Vectorized (Pushdown) 1479 / 1543 10.6 94.0 3.6X -Native ORC Vectorized 6640 / 6748 2.4 422.2 0.8X -Native ORC Vectorized (Pushdown) 2438 / 2479 6.5 155.0 2.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 6827 / 6864 2.3 434.0 1.0X +Parquet Vectorized (Pushdown) 1809 / 1827 8.7 115.0 3.8X +Native ORC Vectorized 6287 / 6296 2.5 399.7 1.1X +Native ORC Vectorized (Pushdown) 6364 / 6377 2.5 404.6 1.1X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 50% decimal(18, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 9224 / 9356 1.7 586.5 1.0X -Parquet Vectorized (Pushdown) 7172 / 7415 2.2 456.0 1.3X -Native ORC Vectorized 11017 / 11408 1.4 700.4 0.8X -Native ORC Vectorized (Pushdown) 8771 / 10218 1.8 557.7 1.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 11315 / 11342 1.4 719.4 1.0X +Parquet Vectorized (Pushdown) 8431 / 8450 1.9 536.0 1.3X +Native ORC Vectorized 11591 / 11611 1.4 736.9 1.0X +Native ORC Vectorized (Pushdown) 11424 / 11475 1.4 726.3 1.0X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 90% decimal(18, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 13933 / 15990 1.1 885.8 1.0X -Parquet Vectorized (Pushdown) 12683 / 12942 1.2 806.4 1.1X -Native ORC Vectorized 16344 / 20196 1.0 1039.1 0.9X -Native ORC Vectorized (Pushdown) 15162 / 16627 1.0 964.0 0.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 15703 / 15712 1.0 998.4 1.0X +Parquet Vectorized (Pushdown) 14982 / 15009 1.0 952.5 1.0X +Native ORC Vectorized 16887 / 16955 0.9 1073.7 0.9X +Native ORC Vectorized (Pushdown) 16518 / 16530 1.0 1050.2 1.0X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 decimal(38, 2) row (value = 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7102 / 8282 2.2 451.5 1.0X -Parquet Vectorized (Pushdown) 124 / 150 126.4 7.9 57.1X -Native ORC Vectorized 5811 / 6883 2.7 369.5 1.2X -Native ORC Vectorized (Pushdown) 1121 / 1502 14.0 71.3 6.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 8101 / 8130 1.9 515.1 1.0X +Parquet Vectorized (Pushdown) 184 / 187 85.6 11.7 44.1X +Native ORC Vectorized 4998 / 5027 3.1 317.8 1.6X +Native ORC Vectorized (Pushdown) 165 / 168 95.6 10.5 49.2X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 10% decimal(38, 2) rows (value < 1572864): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 6894 / 7562 2.3 438.3 1.0X -Parquet Vectorized (Pushdown) 1863 / 1980 8.4 118.4 3.7X -Native ORC Vectorized 6812 / 6848 2.3 433.1 1.0X -Native ORC Vectorized (Pushdown) 2511 / 2598 6.3 159.7 2.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 9405 / 9447 1.7 597.9 1.0X +Parquet Vectorized (Pushdown) 2269 / 2275 6.9 144.2 4.1X +Native ORC Vectorized 6167 / 6203 2.6 392.1 1.5X +Native ORC Vectorized (Pushdown) 1783 / 1787 8.8 113.3 5.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 50% decimal(38, 2) rows (value < 7864320): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 11732 / 12183 1.3 745.9 1.0X -Parquet Vectorized (Pushdown) 8912 / 9945 1.8 566.6 1.3X -Native ORC Vectorized 11499 / 12387 1.4 731.1 1.0X -Native ORC Vectorized (Pushdown) 9328 / 9382 1.7 593.1 1.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 14700 / 14707 1.1 934.6 1.0X +Parquet Vectorized (Pushdown) 10699 / 10712 1.5 680.2 1.4X +Native ORC Vectorized 10687 / 10703 1.5 679.5 1.4X +Native ORC Vectorized (Pushdown) 8364 / 8415 1.9 531.8 1.8X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 90% decimal(38, 2) rows (value < 14155776): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 16272 / 16328 1.0 1034.6 1.0X -Parquet Vectorized (Pushdown) 15714 / 18100 1.0 999.1 1.0X -Native ORC Vectorized 16539 / 18897 1.0 1051.5 1.0X -Native ORC Vectorized (Pushdown) 16328 / 17306 1.0 1038.1 1.0X +Parquet Vectorized 19780 / 19894 0.8 1257.6 1.0X +Parquet Vectorized (Pushdown) 19003 / 19025 0.8 1208.1 1.0X +Native ORC Vectorized 15385 / 15404 1.0 978.2 1.3X +Native ORC Vectorized (Pushdown) 15032 / 15060 1.0 955.7 1.3X ================================================================================================ Pushdown benchmark for InSet -> InFilters ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 5, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7993 / 8104 2.0 508.2 1.0X -Parquet Vectorized (Pushdown) 507 / 532 31.0 32.2 15.8X -Native ORC Vectorized 6922 / 7163 2.3 440.1 1.2X -Native ORC Vectorized (Pushdown) 1017 / 1058 15.5 64.6 7.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10521 / 10534 1.5 668.9 1.0X +Parquet Vectorized (Pushdown) 677 / 691 23.2 43.1 15.5X +Native ORC Vectorized 6768 / 6776 2.3 430.3 1.6X +Native ORC Vectorized (Pushdown) 501 / 512 31.4 31.8 21.0X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 5, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7855 / 7963 2.0 499.4 1.0X -Parquet Vectorized (Pushdown) 503 / 516 31.3 32.0 15.6X -Native ORC Vectorized 6825 / 6954 2.3 433.9 1.2X -Native ORC Vectorized (Pushdown) 1019 / 1044 15.4 64.8 7.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10531 / 10538 1.5 669.5 1.0X +Parquet Vectorized (Pushdown) 677 / 718 23.2 43.0 15.6X +Native ORC Vectorized 6765 / 6773 2.3 430.1 1.6X +Native ORC Vectorized (Pushdown) 499 / 507 31.5 31.7 21.1X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 5, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7858 / 7928 2.0 499.6 1.0X -Parquet Vectorized (Pushdown) 490 / 519 32.1 31.1 16.0X -Native ORC Vectorized 7079 / 7966 2.2 450.1 1.1X -Native ORC Vectorized (Pushdown) 1276 / 1673 12.3 81.1 6.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10540 / 10553 1.5 670.1 1.0X +Parquet Vectorized (Pushdown) 678 / 710 23.2 43.1 15.5X +Native ORC Vectorized 6787 / 6794 2.3 431.5 1.6X +Native ORC Vectorized (Pushdown) 501 / 509 31.4 31.9 21.0X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 10, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8007 / 11155 2.0 509.0 1.0X -Parquet Vectorized (Pushdown) 519 / 540 30.3 33.0 15.4X -Native ORC Vectorized 6848 / 7072 2.3 435.4 1.2X -Native ORC Vectorized (Pushdown) 1026 / 1050 15.3 65.2 7.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10551 / 10559 1.5 670.8 1.0X +Parquet Vectorized (Pushdown) 703 / 708 22.4 44.7 15.0X +Native ORC Vectorized 6791 / 6802 2.3 431.7 1.6X +Native ORC Vectorized (Pushdown) 519 / 526 30.3 33.0 20.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 10, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7876 / 7956 2.0 500.7 1.0X -Parquet Vectorized (Pushdown) 521 / 535 30.2 33.1 15.1X -Native ORC Vectorized 7051 / 7368 2.2 448.3 1.1X -Native ORC Vectorized (Pushdown) 1014 / 1035 15.5 64.5 7.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10561 / 10565 1.5 671.4 1.0X +Parquet Vectorized (Pushdown) 711 / 716 22.1 45.2 14.9X +Native ORC Vectorized 6791 / 6806 2.3 431.8 1.6X +Native ORC Vectorized (Pushdown) 529 / 537 29.8 33.6 20.0X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 10, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7897 / 8229 2.0 502.1 1.0X -Parquet Vectorized (Pushdown) 513 / 530 30.7 32.6 15.4X -Native ORC Vectorized 6730 / 6990 2.3 427.9 1.2X -Native ORC Vectorized (Pushdown) 1003 / 1036 15.7 63.8 7.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10572 / 10590 1.5 672.1 1.0X +Parquet Vectorized (Pushdown) 713 / 716 22.1 45.3 14.8X +Native ORC Vectorized 6808 / 6815 2.3 432.9 1.6X +Native ORC Vectorized (Pushdown) 530 / 541 29.7 33.7 19.9X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 50, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7967 / 8175 2.0 506.5 1.0X -Parquet Vectorized (Pushdown) 8155 / 8434 1.9 518.5 1.0X -Native ORC Vectorized 7002 / 7107 2.2 445.2 1.1X -Native ORC Vectorized (Pushdown) 1092 / 1139 14.4 69.4 7.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10871 / 10882 1.4 691.2 1.0X +Parquet Vectorized (Pushdown) 11104 / 11110 1.4 706.0 1.0X +Native ORC Vectorized 7088 / 7104 2.2 450.7 1.5X +Native ORC Vectorized (Pushdown) 665 / 677 23.6 42.3 16.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 50, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8032 / 8122 2.0 510.7 1.0X -Parquet Vectorized (Pushdown) 8141 / 8908 1.9 517.6 1.0X -Native ORC Vectorized 7140 / 7387 2.2 454.0 1.1X -Native ORC Vectorized (Pushdown) 1156 / 1220 13.6 73.5 6.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10861 / 10867 1.4 690.5 1.0X +Parquet Vectorized (Pushdown) 11094 / 11099 1.4 705.3 1.0X +Native ORC Vectorized 7075 / 7092 2.2 449.8 1.5X +Native ORC Vectorized (Pushdown) 718 / 733 21.9 45.6 15.1X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 50, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8088 / 8350 1.9 514.2 1.0X -Parquet Vectorized (Pushdown) 8629 / 8702 1.8 548.6 0.9X -Native ORC Vectorized 7480 / 7886 2.1 475.6 1.1X -Native ORC Vectorized (Pushdown) 1106 / 1145 14.2 70.3 7.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10868 / 10887 1.4 691.0 1.0X +Parquet Vectorized (Pushdown) 11100 / 11106 1.4 705.7 1.0X +Native ORC Vectorized 7087 / 7093 2.2 450.6 1.5X +Native ORC Vectorized (Pushdown) 712 / 731 22.1 45.3 15.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 100, distribution: 10): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8028 / 8165 2.0 510.4 1.0X -Parquet Vectorized (Pushdown) 8349 / 8674 1.9 530.8 1.0X -Native ORC Vectorized 7107 / 7354 2.2 451.8 1.1X -Native ORC Vectorized (Pushdown) 1175 / 1207 13.4 74.7 6.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10850 / 10888 1.4 689.8 1.0X +Parquet Vectorized (Pushdown) 11086 / 11105 1.4 704.9 1.0X +Native ORC Vectorized 7090 / 7101 2.2 450.8 1.5X +Native ORC Vectorized (Pushdown) 867 / 882 18.1 55.1 12.5X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 100, distribution: 50): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8041 / 8195 2.0 511.2 1.0X -Parquet Vectorized (Pushdown) 8466 / 8604 1.9 538.2 0.9X -Native ORC Vectorized 7116 / 7286 2.2 452.4 1.1X -Native ORC Vectorized (Pushdown) 1197 / 1214 13.1 76.1 6.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10816 / 10819 1.5 687.7 1.0X +Parquet Vectorized (Pushdown) 11052 / 11059 1.4 702.7 1.0X +Native ORC Vectorized 7037 / 7044 2.2 447.4 1.5X +Native ORC Vectorized (Pushdown) 919 / 931 17.1 58.4 11.8X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz InSet -> InFilters (values count: 100, distribution: 90): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7998 / 8311 2.0 508.5 1.0X -Parquet Vectorized (Pushdown) 9366 / 11257 1.7 595.5 0.9X -Native ORC Vectorized 7856 / 9273 2.0 499.5 1.0X -Native ORC Vectorized (Pushdown) 1350 / 1747 11.7 85.8 5.9X +Parquet Vectorized 10807 / 10815 1.5 687.1 1.0X +Parquet Vectorized (Pushdown) 11047 / 11054 1.4 702.4 1.0X +Native ORC Vectorized 7042 / 7047 2.2 447.7 1.5X +Native ORC Vectorized (Pushdown) 950 / 961 16.6 60.4 11.4X ================================================================================================ Pushdown benchmark for tinyint ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 tinyint row (value = CAST(63 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 3461 / 3997 4.5 220.1 1.0X -Parquet Vectorized (Pushdown) 270 / 315 58.4 17.1 12.8X -Native ORC Vectorized 4107 / 5372 3.8 261.1 0.8X -Native ORC Vectorized (Pushdown) 778 / 1553 20.2 49.5 4.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 6034 / 6048 2.6 383.6 1.0X +Parquet Vectorized (Pushdown) 333 / 344 47.2 21.2 18.1X +Native ORC Vectorized 3240 / 3307 4.9 206.0 1.9X +Native ORC Vectorized (Pushdown) 330 / 341 47.6 21.0 18.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 10% tinyint rows (value < CAST(12 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 4771 / 6655 3.3 303.3 1.0X -Parquet Vectorized (Pushdown) 1322 / 1606 11.9 84.0 3.6X -Native ORC Vectorized 4437 / 4572 3.5 282.1 1.1X -Native ORC Vectorized (Pushdown) 1781 / 1976 8.8 113.2 2.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 6759 / 6800 2.3 429.7 1.0X +Parquet Vectorized (Pushdown) 1533 / 1537 10.3 97.5 4.4X +Native ORC Vectorized 3863 / 3874 4.1 245.6 1.7X +Native ORC Vectorized (Pushdown) 1235 / 1248 12.7 78.5 5.5X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 50% tinyint rows (value < CAST(63 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 7433 / 7752 2.1 472.6 1.0X -Parquet Vectorized (Pushdown) 5863 / 5913 2.7 372.8 1.3X -Native ORC Vectorized 7986 / 8084 2.0 507.7 0.9X -Native ORC Vectorized (Pushdown) 6522 / 6608 2.4 414.6 1.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10247 / 10289 1.5 651.5 1.0X +Parquet Vectorized (Pushdown) 7430 / 7453 2.1 472.4 1.4X +Native ORC Vectorized 6995 / 7009 2.2 444.7 1.5X +Native ORC Vectorized (Pushdown) 5561 / 5571 2.8 353.6 1.8X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 90% tinyint rows (value < CAST(114 AS tinyint)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 11190 / 11519 1.4 711.4 1.0X -Parquet Vectorized (Pushdown) 10861 / 11206 1.4 690.5 1.0X -Native ORC Vectorized 11622 / 12196 1.4 738.9 1.0X -Native ORC Vectorized (Pushdown) 11377 / 11654 1.4 723.3 1.0X +Parquet Vectorized 13949 / 13991 1.1 886.9 1.0X +Parquet Vectorized (Pushdown) 13486 / 13511 1.2 857.4 1.0X +Native ORC Vectorized 10149 / 10186 1.5 645.3 1.4X +Native ORC Vectorized (Pushdown) 9889 / 9905 1.6 628.7 1.4X ================================================================================================ Pushdown benchmark for Timestamp ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 timestamp stored as INT96 row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 4784 / 4956 3.3 304.2 1.0X -Parquet Vectorized (Pushdown) 4838 / 4917 3.3 307.6 1.0X -Native ORC Vectorized 3923 / 4173 4.0 249.4 1.2X -Native ORC Vectorized (Pushdown) 894 / 943 17.6 56.8 5.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 6307 / 6310 2.5 401.0 1.0X +Parquet Vectorized (Pushdown) 6360 / 6397 2.5 404.3 1.0X +Native ORC Vectorized 2912 / 2917 5.4 185.1 2.2X +Native ORC Vectorized (Pushdown) 138 / 141 114.4 8.7 45.9X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 10% timestamp stored as INT96 rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 5686 / 5901 2.8 361.5 1.0X -Parquet Vectorized (Pushdown) 5555 / 5895 2.8 353.2 1.0X -Native ORC Vectorized 4844 / 4957 3.2 308.0 1.2X -Native ORC Vectorized (Pushdown) 2141 / 2230 7.3 136.1 2.7X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 7225 / 7233 2.2 459.4 1.0X +Parquet Vectorized (Pushdown) 7250 / 7255 2.2 461.0 1.0X +Native ORC Vectorized 3772 / 3783 4.2 239.8 1.9X +Native ORC Vectorized (Pushdown) 1277 / 1282 12.3 81.2 5.7X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 50% timestamp stored as INT96 rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 9100 / 9421 1.7 578.6 1.0X -Parquet Vectorized (Pushdown) 9122 / 9496 1.7 580.0 1.0X -Native ORC Vectorized 8365 / 8874 1.9 531.9 1.1X -Native ORC Vectorized (Pushdown) 7128 / 7376 2.2 453.2 1.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10952 / 10965 1.4 696.3 1.0X +Parquet Vectorized (Pushdown) 10985 / 10998 1.4 698.4 1.0X +Native ORC Vectorized 7178 / 7227 2.2 456.3 1.5X +Native ORC Vectorized (Pushdown) 5825 / 5830 2.7 370.3 1.9X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 90% timestamp stored as INT96 rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 12764 / 13120 1.2 811.5 1.0X -Parquet Vectorized (Pushdown) 12656 / 13003 1.2 804.7 1.0X -Native ORC Vectorized 13096 / 13233 1.2 832.6 1.0X -Native ORC Vectorized (Pushdown) 12710 / 15611 1.2 808.1 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 14560 / 14583 1.1 925.7 1.0X +Parquet Vectorized (Pushdown) 14608 / 14620 1.1 928.7 1.0X +Native ORC Vectorized 10601 / 10640 1.5 674.0 1.4X +Native ORC Vectorized (Pushdown) 10392 / 10406 1.5 660.7 1.4X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 timestamp stored as TIMESTAMP_MICROS row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 4381 / 4796 3.6 278.5 1.0X -Parquet Vectorized (Pushdown) 122 / 137 129.3 7.7 36.0X -Native ORC Vectorized 3913 / 3988 4.0 248.8 1.1X -Native ORC Vectorized (Pushdown) 905 / 945 17.4 57.6 4.8X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 5653 / 5658 2.8 359.4 1.0X +Parquet Vectorized (Pushdown) 165 / 169 95.1 10.5 34.2X +Native ORC Vectorized 2918 / 2921 5.4 185.5 1.9X +Native ORC Vectorized (Pushdown) 137 / 145 114.9 8.7 41.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 10% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 5145 / 5184 3.1 327.1 1.0X -Parquet Vectorized (Pushdown) 1426 / 1519 11.0 90.7 3.6X -Native ORC Vectorized 4827 / 4901 3.3 306.9 1.1X -Native ORC Vectorized (Pushdown) 2133 / 2210 7.4 135.6 2.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 6540 / 6552 2.4 415.8 1.0X +Parquet Vectorized (Pushdown) 1610 / 1614 9.8 102.3 4.1X +Native ORC Vectorized 3775 / 3788 4.2 240.0 1.7X +Native ORC Vectorized (Pushdown) 1274 / 1277 12.3 81.0 5.1X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 50% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 9234 / 9516 1.7 587.1 1.0X -Parquet Vectorized (Pushdown) 6752 / 7046 2.3 429.3 1.4X -Native ORC Vectorized 8418 / 8998 1.9 535.2 1.1X -Native ORC Vectorized (Pushdown) 7199 / 7314 2.2 457.7 1.3X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10259 / 10278 1.5 652.3 1.0X +Parquet Vectorized (Pushdown) 7591 / 7601 2.1 482.6 1.4X +Native ORC Vectorized 7185 / 7194 2.2 456.8 1.4X +Native ORC Vectorized (Pushdown) 5828 / 5843 2.7 370.6 1.8X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 90% timestamp stored as TIMESTAMP_MICROS rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 12414 / 12458 1.3 789.2 1.0X -Parquet Vectorized (Pushdown) 12094 / 12249 1.3 768.9 1.0X -Native ORC Vectorized 12198 / 13755 1.3 775.5 1.0X -Native ORC Vectorized (Pushdown) 12205 / 12431 1.3 776.0 1.0X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 13850 / 13868 1.1 880.5 1.0X +Parquet Vectorized (Pushdown) 13433 / 13450 1.2 854.0 1.0X +Native ORC Vectorized 10635 / 10669 1.5 676.1 1.3X +Native ORC Vectorized (Pushdown) 10437 / 10448 1.5 663.6 1.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 timestamp stored as TIMESTAMP_MILLIS row (value = CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 4369 / 4515 3.6 277.8 1.0X -Parquet Vectorized (Pushdown) 116 / 125 136.2 7.3 37.8X -Native ORC Vectorized 3965 / 4703 4.0 252.1 1.1X -Native ORC Vectorized (Pushdown) 892 / 1162 17.6 56.7 4.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 5884 / 5888 2.7 374.1 1.0X +Parquet Vectorized (Pushdown) 166 / 170 94.7 10.6 35.4X +Native ORC Vectorized 2913 / 2916 5.4 185.2 2.0X +Native ORC Vectorized (Pushdown) 136 / 144 115.4 8.7 43.2X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 10% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(1572864 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 5211 / 5409 3.0 331.3 1.0X -Parquet Vectorized (Pushdown) 1427 / 1438 11.0 90.7 3.7X -Native ORC Vectorized 4719 / 4883 3.3 300.1 1.1X -Native ORC Vectorized (Pushdown) 2191 / 2228 7.2 139.3 2.4X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 6763 / 6776 2.3 430.0 1.0X +Parquet Vectorized (Pushdown) 1634 / 1638 9.6 103.9 4.1X +Native ORC Vectorized 3777 / 3785 4.2 240.1 1.8X +Native ORC Vectorized (Pushdown) 1276 / 1279 12.3 81.2 5.3X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 50% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(7864320 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 8716 / 8953 1.8 554.2 1.0X -Parquet Vectorized (Pushdown) 6632 / 6968 2.4 421.7 1.3X -Native ORC Vectorized 8376 / 9118 1.9 532.5 1.0X -Native ORC Vectorized (Pushdown) 7218 / 7609 2.2 458.9 1.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_151-b12 on Mac OS X 10.12.6 -Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz +Parquet Vectorized 10460 / 10469 1.5 665.0 1.0X +Parquet Vectorized (Pushdown) 7689 / 7698 2.0 488.9 1.4X +Native ORC Vectorized 7190 / 7197 2.2 457.1 1.5X +Native ORC Vectorized (Pushdown) 5820 / 5834 2.7 370.0 1.8X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 90% timestamp stored as TIMESTAMP_MILLIS rows (value < CAST(14155776 AS timestamp)): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 12264 / 12452 1.3 779.7 1.0X -Parquet Vectorized (Pushdown) 11766 / 11927 1.3 748.0 1.0X -Native ORC Vectorized 12101 / 12301 1.3 769.3 1.0X -Native ORC Vectorized (Pushdown) 11983 / 12651 1.3 761.9 1.0X +Parquet Vectorized 14033 / 14039 1.1 892.2 1.0X +Parquet Vectorized (Pushdown) 13608 / 13636 1.2 865.2 1.0X +Native ORC Vectorized 10635 / 10686 1.5 676.2 1.3X +Native ORC Vectorized (Pushdown) 10420 / 10442 1.5 662.5 1.3X ================================================================================================ Pushdown benchmark with many filters ================================================================================================ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.13.6 -Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz - +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 row with 1 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 158 / 182 0.0 158442969.0 1.0X -Parquet Vectorized (Pushdown) 150 / 158 0.0 149718289.0 1.1X -Native ORC Vectorized 141 / 148 0.0 141259852.0 1.1X -Native ORC Vectorized (Pushdown) 142 / 147 0.0 142016472.0 1.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.13.6 -Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz +Parquet Vectorized 319 / 323 0.0 318789986.0 1.0X +Parquet Vectorized (Pushdown) 323 / 347 0.0 322755287.0 1.0X +Native ORC Vectorized 316 / 336 0.0 315670745.0 1.0X +Native ORC Vectorized (Pushdown) 317 / 320 0.0 317392594.0 1.0X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 row with 250 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 1013 / 1026 0.0 1013194322.0 1.0X -Parquet Vectorized (Pushdown) 1326 / 1332 0.0 1326301956.0 0.8X -Native ORC Vectorized 1005 / 1010 0.0 1005266379.0 1.0X -Native ORC Vectorized (Pushdown) 1068 / 1071 0.0 1067964993.0 0.9X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.13.6 -Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz +Parquet Vectorized 2192 / 2218 0.0 2191883823.0 1.0X +Parquet Vectorized (Pushdown) 2675 / 2687 0.0 2675439029.0 0.8X +Native ORC Vectorized 2158 / 2162 0.0 2157646071.0 1.0X +Native ORC Vectorized (Pushdown) 2309 / 2326 0.0 2309096612.0 0.9X +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Select 1 row with 500 filters: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Parquet Vectorized 3598 / 3614 0.0 3598001202.0 1.0X -Parquet Vectorized (Pushdown) 4282 / 4333 0.0 4281849770.0 0.8X -Native ORC Vectorized 3594 / 3619 0.0 3593551548.0 1.0X -Native ORC Vectorized (Pushdown) 3834 / 3840 0.0 3834240570.0 0.9X +Parquet Vectorized 6219 / 6248 0.0 6218727737.0 1.0X +Parquet Vectorized (Pushdown) 7376 / 7436 0.0 7375977710.0 0.8X +Native ORC Vectorized 6252 / 6279 0.0 6252473320.0 1.0X +Native ORC Vectorized (Pushdown) 6858 / 6876 0.0 6857854486.0 0.9X + + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index 8596abd1b4ff..d6dfdec45a0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -53,7 +53,8 @@ class FilterPushdownBenchmark extends SparkFunSuite with BenchmarkBeforeAndAfter private val numRows = 1024 * 1024 * 15 private val width = 5 private val mid = numRows / 2 - private val blockSize = 1048576 + // For Parquet/ORC, we will use the same value for block size and compression size + private val blockSize = org.apache.parquet.hadoop.ParquetWriter.DEFAULT_PAGE_SIZE private val spark = SparkSession.builder().config(conf).getOrCreate() @@ -130,16 +131,16 @@ class FilterPushdownBenchmark extends SparkFunSuite with BenchmarkBeforeAndAfter } val df = spark.range(numRows).selectExpr(selectExpr: _*).sort("value") - saveAsTable(df, dir) + saveAsTable(df, dir, true) } - private def saveAsTable(df: DataFrame, dir: File): Unit = { + private def saveAsTable(df: DataFrame, dir: File, useDictionary: Boolean = false): Unit = { val orcPath = dir.getCanonicalPath + "/orc" val parquetPath = dir.getCanonicalPath + "/parquet" - // To always turn on dictionary encoding, we set 1.0 at the threshold (the default is 0.8) df.write.mode("overwrite") - .option("orc.dictionary.key.threshold", 1.0) + .option("orc.dictionary.key.threshold", if (useDictionary) 1.0 else 0.8) + .option("orc.compress.size", blockSize) .option("orc.stripe.size", blockSize).orc(orcPath) spark.read.orc(orcPath).createOrReplaceTempView("orcTable") From b839721f3cea2b9d9af73ab4fd9dad225025ec86 Mon Sep 17 00:00:00 2001 From: npoggi Date: Sat, 15 Sep 2018 20:06:08 -0700 Subject: [PATCH 050/879] [SPARK-25439][TESTS][SQL] Fixes TPCHQuerySuite datatype of customer.c_nationkey to BIGINT according to spec ## What changes were proposed in this pull request? Fixes TPCH DDL datatype of `customer.c_nationkey` from `STRING` to `BIGINT` according to spec and `nation.nationkey` in `TPCHQuerySuite.scala`. The rest of the keys are OK. Note, this will lead to **non-comparable previous results** to new runs involving the customer table. ## How was this patch tested? Manual tests Author: npoggi Closes #22430 from npoggi/SPARK-25439_Fix-TPCH-customer-c_nationkey. (cherry picked from commit 02c2963f895b9d78d7f6d9972cacec4ef55fa278) Signed-off-by: gatorsmile --- .../src/test/scala/org/apache/spark/sql/TPCHQuerySuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TPCHQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/TPCHQuerySuite.scala index e3e700529bba..b32d95d0b286 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TPCHQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TPCHQuerySuite.scala @@ -69,7 +69,7 @@ class TPCHQuerySuite extends BenchmarkQueryTest { sql( """ |CREATE TABLE `customer` (`c_custkey` BIGINT, `c_name` STRING, `c_address` STRING, - |`c_nationkey` STRING, `c_phone` STRING, `c_acctbal` DECIMAL(10,0), + |`c_nationkey` BIGINT, `c_phone` STRING, `c_acctbal` DECIMAL(10,0), |`c_mktsegment` STRING, `c_comment` STRING) |USING parquet """.stripMargin) From 60af706b4c49fa1be1b2b1223490c98868c801c3 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 16 Sep 2018 04:14:19 +0000 Subject: [PATCH 051/879] [SPARK-24418][FOLLOWUP][DOC] Update docs to show Scala 2.11.12 ## What changes were proposed in this pull request? SPARK-24418 upgrades Scala to 2.11.12. This PR updates Scala version in docs. - https://spark.apache.org/docs/latest/quick-start.html#self-contained-applications (screenshot) ![screen1](https://user-images.githubusercontent.com/9700541/45590509-9c5f0400-b8ee-11e8-9293-e48d297db894.png) - https://spark.apache.org/docs/latest/rdd-programming-guide.html#working-with-key-value-pairs (Scala, Java) (These are hyperlink updates) - https://spark.apache.org/docs/latest/streaming-flume-integration.html#configuring-flume-1 (screenshot) ![screen2](https://user-images.githubusercontent.com/9700541/45590511-a123b800-b8ee-11e8-97a5-b7f2288229c2.png) ## How was this patch tested? Manual. ```bash $ cd docs $ SKIP_API=1 jekyll build ``` Closes #22431 from dongjoon-hyun/SPARK-24418. Authored-by: Dongjoon Hyun Signed-off-by: DB Tsai (cherry picked from commit bfcf7426057a964b3cee90089aab6c003addc4fb) Signed-off-by: DB Tsai --- docs/_config.yml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/_config.yml b/docs/_config.yml index 20b6495d27df..72473771c2e9 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -17,7 +17,7 @@ include: SPARK_VERSION: 2.4.1-SNAPSHOT SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" -SCALA_VERSION: "2.11.8" +SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 SPARK_ISSUE_TRACKER_URL: https://issues.apache.org/jira/browse/SPARK SPARK_GITHUB_URL: https://github.com/apache/spark From 1cb1e43012e57e649d77524f8ff2de231f52c66a Mon Sep 17 00:00:00 2001 From: Michael Chirico Date: Sun, 16 Sep 2018 12:57:44 -0700 Subject: [PATCH 052/879] [MINOR][DOCS] Axe deprecated doc refs Continuation of #22370. Summary of discussion there: There is some inconsistency in the R manual w.r.t. supercedent functions linking back to deprecated functions. - `createOrReplaceTempView` and `createTable` both link back to functions which are deprecated (`registerTempTable` and `createExternalTable`, respectively) - `sparkR.session` and `dropTempView` do _not_ link back to deprecated functions This PR takes the view that it is preferable _not_ to link back to deprecated functions, and removes these references from `?createOrReplaceTempView` and `?createTable`. As `registerTempTable` was included in the `SparkDataFrame functions` `family` of functions, other documentation pages which included a link to `?registerTempTable` will similarly be altered. Author: Michael Chirico Author: Michael Chirico Closes #22393 from MichaelChirico/axe_deprecated_doc_refs. (cherry picked from commit a1dd78255a3ae023820b2f245cd39f0c57a32fb1) Signed-off-by: Felix Cheung --- R/pkg/R/DataFrame.R | 1 - R/pkg/R/catalog.R | 1 - 2 files changed, 2 deletions(-) diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 4f2d4c7c002d..458decaf4766 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -503,7 +503,6 @@ setMethod("createOrReplaceTempView", #' @param x A SparkDataFrame #' @param tableName A character vector containing the name of the table #' -#' @family SparkDataFrame functions #' @seealso \link{createOrReplaceTempView} #' @rdname registerTempTable-deprecated #' @name registerTempTable diff --git a/R/pkg/R/catalog.R b/R/pkg/R/catalog.R index baf4d861fcf8..c2d0fc38786b 100644 --- a/R/pkg/R/catalog.R +++ b/R/pkg/R/catalog.R @@ -69,7 +69,6 @@ createExternalTable <- function(x, ...) { #' @param ... additional named parameters as options for the data source. #' @return A SparkDataFrame. #' @rdname createTable -#' @seealso \link{createExternalTable} #' @examples #'\dontrun{ #' sparkR.session() From fb1539ad876d0878dde56258af53399dfdf706eb Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 17 Sep 2018 11:07:51 +0800 Subject: [PATCH 053/879] [SPARK-22713][CORE][TEST][FOLLOWUP] Fix flaky ExternalAppendOnlyMapSuite due to timeout ## What changes were proposed in this pull request? SPARK-22713 uses [`eventually` with the default timeout `150ms`](https://github.com/apache/spark/pull/21369/files#diff-5bbb6a931b7e4d6a31e4938f51935682R462). It causes flakiness because it's executed once when GC is slow. ```scala eventually { System.gc() ... } ``` **Failures** ```scala org.scalatest.exceptions.TestFailedDueToTimeoutException: The code passed to eventually never returned normally. Attempted 1 times over 501.22261 milliseconds. Last failure message: tmpIsNull was false. ``` - master-test-sbt-hadoop-2.7 [4916](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/4916) [4907](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/4907) [4906](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.7/4906) - spark-master-test-sbt-hadoop-2.6 [4979](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4979) [4974](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4974) [4967](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4967) [4966](https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-sbt-hadoop-2.6/4966) ## How was this patch tested? Pass the Jenkins. Closes #22432 from dongjoon-hyun/SPARK-22713. Authored-by: Dongjoon Hyun Signed-off-by: Wenchen Fan (cherry picked from commit 538e0478783160d8fab2dc76fd8fc7b469cb4e19) Signed-off-by: Wenchen Fan --- .../spark/util/collection/ExternalAppendOnlyMapSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 8a2f2ffe0acf..cd2526578413 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.util.collection import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration._ import scala.ref.WeakReference import org.scalatest.Matchers @@ -457,7 +458,7 @@ class ExternalAppendOnlyMapSuite extends SparkFunSuite // https://github.com/scala/scala/blob/2.13.x/test/junit/scala/tools/testing/AssertUtil.scala // (lines 69-89) // assert(map.currentMap == null) - eventually { + eventually(timeout(5 seconds), interval(200 milliseconds)) { System.gc() // direct asserts introduced some macro generated code that held a reference to the map val tmpIsNull = null == underlyingMapRef.get.orNull From e368efcf54b478848f6d625d6cbce63f0d1ce5b9 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Mon, 17 Sep 2018 15:21:18 +0800 Subject: [PATCH 054/879] [SPARK-24685][BUILD][FOLLOWUP] Fix the nonexist profile name in release script ## What changes were proposed in this pull request? `without-hadoop` profile doesn't exist in Maven, instead the name should be `hadoop-provided`, this is a regression introduced by SPARK-24685. So here fix it. ## How was this patch tested? Local test. Closes #22434 from jerryshao/SPARK-24685-followup. Authored-by: jerryshao Signed-off-by: Wenchen Fan (cherry picked from commit b66e14dc96011a83f5ea0df8708ecb02a154ed1d) Signed-off-by: Wenchen Fan --- dev/create-release/release-build.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 73610a333591..ca066bed133d 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -271,7 +271,7 @@ if [[ "$1" == "package" ]]; then BINARY_PKGS_ARGS["hadoop2.7"]="-Phadoop-2.7 $HIVE_PROFILES" if ! is_dry_run; then BINARY_PKGS_ARGS["hadoop2.6"]="-Phadoop-2.6 $HIVE_PROFILES" - BINARY_PKGS_ARGS["without-hadoop"]="-Pwithout-hadoop" + BINARY_PKGS_ARGS["without-hadoop"]="-Phadoop-provided" if [[ $SPARK_VERSION < "2.2." ]]; then BINARY_PKGS_ARGS["hadoop2.4"]="-Phadoop-2.4 $HIVE_PROFILES" BINARY_PKGS_ARGS["hadoop2.3"]="-Phadoop-2.3 $HIVE_PROFILES" From 43c9b108545adcd0f99bd4408759fbee440c560f Mon Sep 17 00:00:00 2001 From: s71955 Date: Mon, 17 Sep 2018 19:22:27 +0800 Subject: [PATCH 055/879] [SPARK-23425][SQL][FOLLOWUP] Support wildcards in HDFS path for loadtable command. What changes were proposed in this pull request Updated the Migration guide for the behavior changes done in the JIRA issue SPARK-23425. How was this patch tested? Manually verified. Closes #22396 from sujith71955/master_newtest. Authored-by: s71955 Signed-off-by: Wenchen Fan (cherry picked from commit 619c949019feccd3fc2c9e58a841c655d05216f3) Signed-off-by: Wenchen Fan --- docs/sql-programming-guide.md | 1 + .../spark/sql/hive/execution/SQLQuerySuite.scala | 15 +++++++++++++++ 2 files changed, 16 insertions(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 9da7d64322eb..e262987ab23d 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1898,6 +1898,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.parallelFileListingInStatsComputation.enabled` to `False`. - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. + - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. ## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above 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 20c4c36c0509..e49aea267026 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 @@ -1916,6 +1916,21 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("SPARK-23425 Test LOAD DATA LOCAL INPATH with space in file name") { + withTempDir { dir => + val path = dir.toURI.toString.stripSuffix("/") + val dirPath = dir.getAbsoluteFile + for (i <- 1 to 3) { + Files.write(s"$i", new File(dirPath, s"part-r-0000 $i"), StandardCharsets.UTF_8) + } + withTable("load_t") { + sql("CREATE TABLE load_t (a STRING)") + sql(s"LOAD DATA LOCAL INPATH '$path/part-r-0000 1' INTO TABLE load_t") + checkAnswer(sql("SELECT * FROM load_t"), Seq(Row("1"))) + } + } + } + test("Support wildcard character in folderlevel for LOAD DATA LOCAL INPATH") { withTempDir { dir => val path = dir.toURI.toString.stripSuffix("/") From 56f706867f306df1836f5c976eba02074c53ed4d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 17 Sep 2018 19:33:51 +0800 Subject: [PATCH 056/879] [SPARK-25427][SQL][TEST] Add BloomFilter creation test cases ## What changes were proposed in this pull request? Spark supports BloomFilter creation for ORC files. This PR aims to add test coverages to prevent accidental regressions like [SPARK-12417](https://issues.apache.org/jira/browse/SPARK-12417). ## How was this patch tested? Pass the Jenkins with newly added test cases. Closes #22418 from dongjoon-hyun/SPARK-25427. Authored-by: Dongjoon Hyun Signed-off-by: Wenchen Fan (cherry picked from commit 0dd61ec47df7078fd4f77d8c58ecf26c630c700e) Signed-off-by: Wenchen Fan --- .../datasources/orc/OrcSourceSuite.scala | 69 +++++++++++++++++++ .../sql/hive/orc/HiveOrcSourceSuite.scala | 9 +++ 2 files changed, 78 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala index 02bfb7197ffc..b6bb1d7ba4ce 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcSourceSuite.scala @@ -21,7 +21,12 @@ import java.io.File import java.sql.Timestamp import java.util.Locale +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.orc.OrcConf.COMPRESS +import org.apache.orc.OrcFile +import org.apache.orc.OrcProto.Stream.Kind +import org.apache.orc.impl.RecordReaderImpl import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.Row @@ -50,6 +55,66 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { .createOrReplaceTempView("orc_temp_table") } + protected def testBloomFilterCreation(bloomFilterKind: Kind) { + val tableName = "bloomFilter" + + withTempDir { dir => + withTable(tableName) { + val sqlStatement = orcImp match { + case "native" => + s""" + |CREATE TABLE $tableName (a INT, b STRING) + |USING ORC + |OPTIONS ( + | path '${dir.toURI}', + | orc.bloom.filter.columns '*', + | orc.bloom.filter.fpp 0.1 + |) + """.stripMargin + case "hive" => + s""" + |CREATE TABLE $tableName (a INT, b STRING) + |STORED AS ORC + |LOCATION '${dir.toURI}' + |TBLPROPERTIES ( + | orc.bloom.filter.columns='*', + | orc.bloom.filter.fpp=0.1 + |) + """.stripMargin + case impl => + throw new UnsupportedOperationException(s"Unknown ORC implementation: $impl") + } + + sql(sqlStatement) + sql(s"INSERT INTO $tableName VALUES (1, 'str')") + + val partFiles = dir.listFiles() + .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) + assert(partFiles.length === 1) + + val orcFilePath = new Path(partFiles.head.getAbsolutePath) + val readerOptions = OrcFile.readerOptions(new Configuration()) + val reader = OrcFile.createReader(orcFilePath, readerOptions) + var recordReader: RecordReaderImpl = null + try { + recordReader = reader.rows.asInstanceOf[RecordReaderImpl] + + // BloomFilter array is created for all types; `struct`, int (`a`), string (`b`) + val sargColumns = Array(true, true, true) + val orcIndex = recordReader.readRowIndex(0, null, sargColumns) + + // Check the types and counts of bloom filters + assert(orcIndex.getBloomFilterKinds.forall(_ === bloomFilterKind)) + assert(orcIndex.getBloomFilterIndex.forall(_.getBloomFilterCount > 0)) + } finally { + if (recordReader != null) { + recordReader.close() + } + } + } + } + } + test("create temporary orc table") { checkAnswer(sql("SELECT COUNT(*) FROM normal_orc_source"), Row(10)) @@ -215,4 +280,8 @@ class OrcSourceSuite extends OrcSuite with SharedSQLContext { |) """.stripMargin) } + + test("Check BloomFilter creation") { + testBloomFilterCreation(Kind.BLOOM_FILTER_UTF8) // After ORC-101 + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala index d84f9a382820..c1ae2f6861cb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.orc.OrcSuite +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.types._ @@ -173,4 +174,12 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { assert(msg.contains("ORC data source does not support calendarinterval data type.")) } } + + test("Check BloomFilter creation") { + Seq(true, false).foreach { convertMetastore => + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> s"$convertMetastore") { + testBloomFilterCreation(org.apache.orc.OrcProto.Stream.Kind.BLOOM_FILTER) // Before ORC-101 + } + } + } } From d05596ebd57b24f8caf6d76a5c9a8684f45a41b6 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Mon, 17 Sep 2018 20:40:42 +0800 Subject: [PATCH 057/879] [SPARK-25431][SQL][EXAMPLES] Fix function examples and the example results. ## What changes were proposed in this pull request? There are some mistakes in examples of newly added functions. Also the format of the example results are not unified. We should fix them. ## How was this patch tested? Manually executed the examples. Closes #22437 from ueshin/issues/SPARK-25431/fix_examples_2. Authored-by: Takuya UESHIN Signed-off-by: hyukjinkwon (cherry picked from commit 8cf6fd1c2342949916fedb5a7f712177b22585fa) Signed-off-by: hyukjinkwon --- .../expressions/collectionOperations.scala | 37 ++++++++++--------- .../expressions/complexTypeCreator.scala | 2 +- .../expressions/higherOrderFunctions.scala | 32 ++++++++-------- 3 files changed, 36 insertions(+), 35 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index ea6fcccddfd4..cc9edcfd41d0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -159,9 +159,9 @@ case class MapKeys(child: Expression) examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array(2, 3, 4)); - [[1, 2], [2, 3], [3, 4]] + [{"0":1,"1":2},{"0":2,"1":3},{"0":3,"1":4}] > SELECT _FUNC_(array(1, 2), array(2, 3), array(3, 4)); - [[1, 2, 3], [2, 3, 4]] + [{"0":1,"1":2,"2":3},{"0":2,"1":3,"2":4}] """, since = "2.4.0") case class ArraysZip(children: Seq[Expression]) extends Expression with ExpectsInputTypes { @@ -348,7 +348,7 @@ case class MapValues(child: Expression) examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b')); - [(1,"a"),(2,"b")] + [{"key":1,"value":"a"},{"key":2,"value":"b"}] """, since = "2.4.0") case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInputTypes { @@ -516,7 +516,7 @@ case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInp examples = """ Examples: > SELECT _FUNC_(map(1, 'a', 2, 'b'), map(2, 'c', 3, 'd')); - [[1 -> "a"], [2 -> "b"], [2 -> "c"], [3 -> "d"]] + {1:"a",2:"c",3:"d"} """, since = "2.4.0") case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpression { @@ -1171,9 +1171,9 @@ case class ArraySort(child: Expression) extends UnaryExpression with ArraySortLi examples = """ Examples: > SELECT _FUNC_(array(1, 20, 3, 5)); - [3, 1, 5, 20] + [3,1,5,20] > SELECT _FUNC_(array(1, 20, null, 3)); - [20, null, 3, 1] + [20,null,3,1] """, note = "The function is non-deterministic.", since = "2.4.0") @@ -1256,7 +1256,7 @@ case class Shuffle(child: Expression, randomSeed: Option[Long] = None) > SELECT _FUNC_('Spark SQL'); LQS krapS > SELECT _FUNC_(array(2, 1, 4, 3)); - [3, 4, 1, 2] + [3,4,1,2] """, since = "1.5.0", note = "Reverse logic for arrays is available since 2.4.0." @@ -2123,7 +2123,7 @@ case class ArrayPosition(left: Expression, right: Expression) > SELECT _FUNC_(array(1, 2, 3), 2); 2 > SELECT _FUNC_(map(1, 'a', 2, 'b'), 2); - "b" + b """, since = "2.4.0") case class ElementAt(left: Expression, right: Expression) extends GetMapValueUtil { @@ -2238,8 +2238,9 @@ case class ElementAt(left: Expression, right: Expression) extends GetMapValueUti > SELECT _FUNC_('Spark', 'SQL'); SparkSQL > SELECT _FUNC_(array(1, 2, 3), array(4, 5), array(6)); - | [1,2,3,4,5,6] - """) + [1,2,3,4,5,6] + """, + note = "Concat logic for arrays is available since 2.4.0.") case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpression { private def allowedTypes: Seq[AbstractDataType] = Seq(StringType, BinaryType, ArrayType) @@ -2427,7 +2428,7 @@ case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpressio usage = "_FUNC_(arrayOfArrays) - Transforms an array of arrays into a single array.", examples = """ Examples: - > SELECT _FUNC_(array(array(1, 2), array(3, 4)); + > SELECT _FUNC_(array(array(1, 2), array(3, 4))); [1,2,3,4] """, since = "2.4.0") @@ -2556,11 +2557,11 @@ case class Flatten(child: Expression) extends UnaryExpression { examples = """ Examples: > SELECT _FUNC_(1, 5); - [1, 2, 3, 4, 5] + [1,2,3,4,5] > SELECT _FUNC_(5, 1); - [5, 4, 3, 2, 1] + [5,4,3,2,1] > SELECT _FUNC_(to_date('2018-01-01'), to_date('2018-03-01'), interval 1 month); - [2018-01-01, 2018-02-01, 2018-03-01] + [2018-01-01,2018-02-01,2018-03-01] """, since = "2.4.0" ) @@ -2934,7 +2935,7 @@ object Sequence { examples = """ Examples: > SELECT _FUNC_('123', 2); - ['123', '123'] + ["123","123"] """, since = "2.4.0") case class ArrayRepeat(left: Expression, right: Expression) @@ -3421,7 +3422,7 @@ object ArrayBinaryLike { examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5)); - array(1, 2, 3, 5) + [1,2,3,5] """, since = "2.4.0") case class ArrayUnion(left: Expression, right: Expression) extends ArrayBinaryLike @@ -3632,7 +3633,7 @@ object ArrayUnion { examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5)); - array(1, 3) + [1,3] """, since = "2.4.0") case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBinaryLike @@ -3873,7 +3874,7 @@ case class ArrayIntersect(left: Expression, right: Expression) extends ArrayBina examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array(1, 3, 5)); - array(2) + [2] """, since = "2.4.0") case class ArrayExcept(left: Expression, right: Expression) extends ArrayBinaryLike diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index aba9c6c8ad6f..fd8b5e94fe48 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -248,7 +248,7 @@ case class CreateMap(children: Seq[Expression]) extends Expression { in keys should not be null""", examples = """ Examples: - > SELECT _FUNC_([1.0, 3.0], ['2', '4']); + > SELECT _FUNC_(array(1.0, 3.0), array('2', '4')); {1.0:"2",3.0:"4"} """, since = "2.4.0") case class MapFromArrays(left: Expression, right: Expression) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index 2bb6b20b944d..b07d9466ba0d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -209,9 +209,9 @@ trait MapBasedSimpleHigherOrderFunction extends SimpleHigherOrderFunction { examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), x -> x + 1); - array(2, 3, 4) + [2,3,4] > SELECT _FUNC_(array(1, 2, 3), (x, i) -> x + i); - array(1, 3, 5) + [1,3,5] """, since = "2.4.0") case class ArrayTransform( @@ -268,7 +268,7 @@ usage = "_FUNC_(expr, func) - Filters entries in a map using the function.", examples = """ Examples: > SELECT _FUNC_(map(1, 0, 2, 2, 3, -1), (k, v) -> k > v); - [1 -> 0, 3 -> -1] + {1:0,3:-1} """, since = "2.4.0") case class MapFilter( @@ -318,7 +318,7 @@ case class MapFilter( examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 1); - array(1, 3) + [1,3] """, since = "2.4.0") case class ArrayFilter( @@ -499,10 +499,10 @@ case class ArrayAggregate( usage = "_FUNC_(expr, func) - Transforms elements in a map using the function.", examples = """ Examples: - > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + 1); - map(array(2, 3, 4), array(1, 2, 3)) - > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); - map(array(2, 4, 6), array(1, 2, 3)) + > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + 1); + {2:1,3:2,4:3} + > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); + {2:1,4:2,6:3} """, since = "2.4.0") case class TransformKeys( @@ -549,10 +549,10 @@ case class TransformKeys( usage = "_FUNC_(expr, func) - Transforms values in the map using the function.", examples = """ Examples: - > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> v + 1); - map(array(1, 2, 3), array(2, 3, 4)) - > SELECT _FUNC_(map(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); - map(array(1, 2, 3), array(2, 4, 6)) + > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> v + 1); + {1:2,2:3,3:4} + > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); + {1:2,2:4,3:6} """, since = "2.4.0") case class TransformValues( @@ -777,11 +777,11 @@ case class MapZipWith(left: Expression, right: Expression, function: Expression) examples = """ Examples: > SELECT _FUNC_(array(1, 2, 3), array('a', 'b', 'c'), (x, y) -> (y, x)); - array(('a', 1), ('b', 2), ('c', 3)) - > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y)); - array(4, 6) + [{"y":"a","x":1},{"y":"b","x":2},{"y":"c","x":3}] + > SELECT _FUNC_(array(1, 2), array(3, 4), (x, y) -> x + y); + [4,6] > SELECT _FUNC_(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, y)); - array('ad', 'be', 'cf') + ["ad","be","cf"] """, since = "2.4.0") // scalastyle:on line.size.limit From 08f7b145cf8f335263f670a3a2516e78ceaf1015 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 17 Sep 2018 08:54:44 -0500 Subject: [PATCH 058/879] [SPARK-24654][BUILD][FOLLOWUP] Update, fix LICENSE and NOTICE, and specialize for source vs binary ## What changes were proposed in this pull request? Fix location of licenses-binary in binary release, and remove binary items from source release ## How was this patch tested? N/A Closes #22436 from srowen/SPARK-24654.2. Authored-by: Sean Owen Signed-off-by: Sean Owen (cherry picked from commit 30aa37fca45ec0ad4f30076bc855d1a201cfc097) Signed-off-by: Sean Owen --- dev/create-release/release-build.sh | 4 ++++ dev/make-distribution.sh | 1 - 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index ca066bed133d..098aa5745e34 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -171,6 +171,10 @@ if [[ "$1" == "package" ]]; then # Source and binary tarballs echo "Packaging release source tarballs" cp -r spark spark-$SPARK_VERSION + # For source release, exclude copy of binary license/notice + rm spark-$SPARK_VERSION/LICENSE-binary + rm spark-$SPARK_VERSION/NOTICE-binary + rm -r spark-$SPARK_VERSION/licenses-binary tar cvzf spark-$SPARK_VERSION.tgz spark-$SPARK_VERSION echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour --output spark-$SPARK_VERSION.tgz.asc \ --detach-sig spark-$SPARK_VERSION.tgz diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index ad99ce55806a..126d39dfe20e 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -212,7 +212,6 @@ cp -r "$SPARK_HOME/examples/src/main" "$DISTDIR/examples/src/" # Copy license and ASF files cp "$SPARK_HOME/LICENSE-binary" "$DISTDIR/LICENSE" -mkdir -p "$DISTDIR/licenses" cp -r "$SPARK_HOME/licenses-binary" "$DISTDIR/licenses" cp "$SPARK_HOME/NOTICE-binary" "$DISTDIR/NOTICE" From 963af1301131f63a955288edcd6bc76956bb6447 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 13 Aug 2018 21:35:34 -0500 Subject: [PATCH 059/879] [PYSPARK] Updates to pyspark broadcast (cherry picked from commit 58419b92673c46911c25bc6c6b13397f880c6424) --- .../apache/spark/api/python/PythonRDD.scala | 299 +++++++++++++++--- .../spark/api/python/PythonRunner.scala | 52 ++- .../spark/api/python/PythonRDDSuite.scala | 23 +- dev/sparktestsupport/modules.py | 2 + python/pyspark/broadcast.py | 58 +++- python/pyspark/context.py | 64 +++- python/pyspark/serializers.py | 51 +++ python/pyspark/sql/session.py | 12 +- python/pyspark/sql/tests.py | 45 ++- python/pyspark/test_broadcast.py | 126 ++++++++ python/pyspark/test_serializers.py | 90 ++++++ python/pyspark/tests.py | 9 +- python/pyspark/worker.py | 22 +- .../spark/sql/api/python/PythonSQLUtils.scala | 47 ++- .../sql/execution/arrow/ArrowConverters.scala | 9 +- 15 files changed, 789 insertions(+), 120 deletions(-) create mode 100644 python/pyspark/test_broadcast.py create mode 100644 python/pyspark/test_serializers.py diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index e639a842754b..8b5a7a9aefea 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -24,8 +24,10 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.concurrent.Promise +import scala.concurrent.duration.Duration import scala.language.existentials -import scala.util.control.NonFatal +import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec @@ -37,6 +39,7 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream import org.apache.spark.internal.Logging +import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -169,27 +172,34 @@ private[spark] object PythonRDD extends Logging { def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { - val file = new DataInputStream(new FileInputStream(filename)) + readRDDFromInputStream(sc.sc, new FileInputStream(filename), parallelism) + } + + def readRDDFromInputStream( + sc: SparkContext, + in: InputStream, + parallelism: Int): JavaRDD[Array[Byte]] = { + val din = new DataInputStream(in) try { val objs = new mutable.ArrayBuffer[Array[Byte]] try { while (true) { - val length = file.readInt() + val length = din.readInt() val obj = new Array[Byte](length) - file.readFully(obj) + din.readFully(obj) objs += obj } } catch { case eof: EOFException => // No-op } - JavaRDD.fromRDD(sc.sc.parallelize(objs, parallelism)) + JavaRDD.fromRDD(sc.parallelize(objs, parallelism)) } finally { - file.close() + din.close() } } - def readBroadcastFromFile(sc: JavaSparkContext, path: String): Broadcast[PythonBroadcast] = { - sc.broadcast(new PythonBroadcast(path)) + def setupBroadcast(path: String): PythonBroadcast = { + new PythonBroadcast(path) } def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { @@ -419,34 +429,15 @@ private[spark] object PythonRDD extends Logging { */ private[spark] def serveToStream( threadName: String)(writeFunc: OutputStream => Unit): Array[Any] = { - val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) - // Close the socket if no connection in 15 seconds - serverSocket.setSoTimeout(15000) - - new Thread(threadName) { - setDaemon(true) - override def run() { - try { - val sock = serverSocket.accept() - authHelper.authClient(sock) - - val out = new BufferedOutputStream(sock.getOutputStream) - Utils.tryWithSafeFinally { - writeFunc(out) - } { - out.close() - sock.close() - } - } catch { - case NonFatal(e) => - logError(s"Error while sending iterator", e) - } finally { - serverSocket.close() - } + val (port, secret) = PythonServer.setupOneConnectionServer(authHelper, threadName) { s => + val out = new BufferedOutputStream(s.getOutputStream()) + Utils.tryWithSafeFinally { + writeFunc(out) + } { + out.close() } - }.start() - - Array(serverSocket.getLocalPort, authHelper.secret) + } + Array(port, secret) } private def getMergedConf(confAsMap: java.util.HashMap[String, String], @@ -664,13 +655,11 @@ private[spark] class PythonAccumulatorV2( } } -/** - * A Wrapper for Python Broadcast, which is written into disk by Python. It also will - * write the data into disk after deserialization, then Python can read it from disks. - */ // scalastyle:off no.finalize private[spark] class PythonBroadcast(@transient var path: String) extends Serializable - with Logging { + with Logging { + + private var encryptionServer: PythonServer[Unit] = null /** * Read data from disks, then copy it to `out` @@ -713,5 +702,235 @@ private[spark] class PythonBroadcast(@transient var path: String) extends Serial } super.finalize() } + + def setupEncryptionServer(): Array[Any] = { + encryptionServer = new PythonServer[Unit]("broadcast-encrypt-server") { + override def handleConnection(sock: Socket): Unit = { + val env = SparkEnv.get + val in = sock.getInputStream() + val dir = new File(Utils.getLocalDir(env.conf)) + val file = File.createTempFile("broadcast", "", dir) + path = file.getAbsolutePath + val out = env.serializerManager.wrapForEncryption(new FileOutputStream(path)) + DechunkedInputStream.dechunkAndCopyToOutput(in, out) + } + } + Array(encryptionServer.port, encryptionServer.secret) + } + + def waitTillDataReceived(): Unit = encryptionServer.getResult() } // scalastyle:on no.finalize + +/** + * The inverse of pyspark's ChunkedStream for sending data of unknown size. + * + * We might be serializing a really large object from python -- we don't want + * python to buffer the whole thing in memory, nor can it write to a file, + * so we don't know the length in advance. So python writes it in chunks, each chunk + * preceeded by a length, till we get a "length" of -1 which serves as EOF. + * + * Tested from python tests. + */ +private[spark] class DechunkedInputStream(wrapped: InputStream) extends InputStream with Logging { + private val din = new DataInputStream(wrapped) + private var remainingInChunk = din.readInt() + + override def read(): Int = { + val into = new Array[Byte](1) + val n = read(into, 0, 1) + if (n == -1) { + -1 + } else { + // if you just cast a byte to an int, then anything > 127 is negative, which is interpreted + // as an EOF + val b = into(0) + if (b < 0) { + 256 + b + } else { + b + } + } + } + + override def read(dest: Array[Byte], off: Int, len: Int): Int = { + if (remainingInChunk == -1) { + return -1 + } + var destSpace = len + var destPos = off + while (destSpace > 0 && remainingInChunk != -1) { + val toCopy = math.min(remainingInChunk, destSpace) + val read = din.read(dest, destPos, toCopy) + destPos += read + destSpace -= read + remainingInChunk -= read + if (remainingInChunk == 0) { + remainingInChunk = din.readInt() + } + } + assert(destSpace == 0 || remainingInChunk == -1) + return destPos - off + } + + override def close(): Unit = wrapped.close() +} + +private[spark] object DechunkedInputStream { + + /** + * Dechunks the input, copies to output, and closes both input and the output safely. + */ + def dechunkAndCopyToOutput(chunked: InputStream, out: OutputStream): Unit = { + val dechunked = new DechunkedInputStream(chunked) + Utils.tryWithSafeFinally { + Utils.copyStream(dechunked, out) + } { + JavaUtils.closeQuietly(out) + JavaUtils.closeQuietly(dechunked) + } + } +} + +/** + * Creates a server in the jvm to communicate with python for handling one batch of data, with + * authentication and error handling. + */ +private[spark] abstract class PythonServer[T]( + authHelper: SocketAuthHelper, + threadName: String) { + + def this(env: SparkEnv, threadName: String) = this(new SocketAuthHelper(env.conf), threadName) + def this(threadName: String) = this(SparkEnv.get, threadName) + + val (port, secret) = PythonServer.setupOneConnectionServer(authHelper, threadName) { sock => + promise.complete(Try(handleConnection(sock))) + } + + /** + * Handle a connection which has already been authenticated. Any error from this function + * will clean up this connection and the entire server, and get propogated to [[getResult]]. + */ + def handleConnection(sock: Socket): T + + val promise = Promise[T]() + + /** + * Blocks indefinitely for [[handleConnection]] to finish, and returns that result. If + * handleConnection throws an exception, this will throw an exception which includes the original + * exception as a cause. + */ + def getResult(): T = { + getResult(Duration.Inf) + } + + def getResult(wait: Duration): T = { + ThreadUtils.awaitResult(promise.future, wait) + } + +} + +private[spark] object PythonServer { + + /** + * Create a socket server and run user function on the socket in a background thread. + * + * The socket server can only accept one connection, or close if no connection + * in 15 seconds. + * + * The thread will terminate after the supplied user function, or if there are any exceptions. + * + * If you need to get a result of the supplied function, create a subclass of [[PythonServer]] + * + * @return The port number of a local socket and the secret for authentication. + */ + def setupOneConnectionServer( + authHelper: SocketAuthHelper, + threadName: String) + (func: Socket => Unit): (Int, String) = { + val serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) + // Close the socket if no connection in 15 seconds + serverSocket.setSoTimeout(15000) + + new Thread(threadName) { + setDaemon(true) + override def run(): Unit = { + var sock: Socket = null + try { + sock = serverSocket.accept() + authHelper.authClient(sock) + func(sock) + } finally { + JavaUtils.closeQuietly(serverSocket) + JavaUtils.closeQuietly(sock) + } + } + }.start() + (serverSocket.getLocalPort, authHelper.secret) + } +} + +/** + * Sends decrypted broadcast data to python worker. See [[PythonRunner]] for entire protocol. + */ +private[spark] class EncryptedPythonBroadcastServer( + val env: SparkEnv, + val idsAndFiles: Seq[(Long, String)]) + extends PythonServer[Unit]("broadcast-decrypt-server") with Logging { + + override def handleConnection(socket: Socket): Unit = { + val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream())) + var socketIn: InputStream = null + // send the broadcast id, then the decrypted data. We don't need to send the length, the + // the python pickle module just needs a stream. + Utils.tryWithSafeFinally { + (idsAndFiles).foreach { case (id, path) => + out.writeLong(id) + val in = env.serializerManager.wrapForEncryption(new FileInputStream(path)) + Utils.tryWithSafeFinally { + Utils.copyStream(in, out, false) + } { + in.close() + } + } + logTrace("waiting for python to accept broadcast data over socket") + out.flush() + socketIn = socket.getInputStream() + socketIn.read() + logTrace("done serving broadcast data") + } { + JavaUtils.closeQuietly(socketIn) + JavaUtils.closeQuietly(out) + } + } + + def waitTillBroadcastDataSent(): Unit = { + getResult() + } +} + +/** + * Helper for making RDD[Array[Byte]] from some python data, by reading the data from python + * over a socket. This is used in preference to writing data to a file when encryption is enabled. + */ +private[spark] abstract class PythonRDDServer + extends PythonServer[JavaRDD[Array[Byte]]]("pyspark-parallelize-server") { + + def handleConnection(sock: Socket): JavaRDD[Array[Byte]] = { + val in = sock.getInputStream() + val dechunkedInput: InputStream = new DechunkedInputStream(in) + streamToRDD(dechunkedInput) + } + + protected def streamToRDD(input: InputStream): RDD[Array[Byte]] + +} + +private[spark] class PythonParallelizeServer(sc: SparkContext, parallelism: Int) + extends PythonRDDServer { + + override protected def streamToRDD(input: InputStream): RDD[Array[Byte]] = { + PythonRDD.readRDDFromInputStream(sc, input, parallelism) + } +} + diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 4c53bc269a10..6e53a044e9a8 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -289,19 +289,51 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( val newBids = broadcastVars.map(_.id).toSet // number of different broadcasts val toRemove = oldBids.diff(newBids) - val cnt = toRemove.size + newBids.diff(oldBids).size + val addedBids = newBids.diff(oldBids) + val cnt = toRemove.size + addedBids.size + val needsDecryptionServer = env.serializerManager.encryptionEnabled && addedBids.nonEmpty + dataOut.writeBoolean(needsDecryptionServer) dataOut.writeInt(cnt) - for (bid <- toRemove) { - // remove the broadcast from worker - dataOut.writeLong(- bid - 1) // bid >= 0 - oldBids.remove(bid) + def sendBidsToRemove(): Unit = { + for (bid <- toRemove) { + // remove the broadcast from worker + dataOut.writeLong(-bid - 1) // bid >= 0 + oldBids.remove(bid) + } } - for (broadcast <- broadcastVars) { - if (!oldBids.contains(broadcast.id)) { + if (needsDecryptionServer) { + // if there is encryption, we setup a server which reads the encrypted files, and sends + // the decrypted data to python + val idsAndFiles = broadcastVars.flatMap { broadcast => + if (!oldBids.contains(broadcast.id)) { + Some((broadcast.id, broadcast.value.path)) + } else { + None + } + } + val server = new EncryptedPythonBroadcastServer(env, idsAndFiles) + dataOut.writeInt(server.port) + logTrace(s"broadcast decryption server setup on ${server.port}") + PythonRDD.writeUTF(server.secret, dataOut) + sendBidsToRemove() + idsAndFiles.foreach { case (id, _) => // send new broadcast - dataOut.writeLong(broadcast.id) - PythonRDD.writeUTF(broadcast.value.path, dataOut) - oldBids.add(broadcast.id) + dataOut.writeLong(id) + oldBids.add(id) + } + dataOut.flush() + logTrace("waiting for python to read decrypted broadcast data from server") + server.waitTillBroadcastDataSent() + logTrace("done sending decrypted data to python") + } else { + sendBidsToRemove() + for (broadcast <- broadcastVars) { + if (!oldBids.contains(broadcast.id)) { + // send new broadcast + dataOut.writeLong(broadcast.id) + PythonRDD.writeUTF(broadcast.value.path, dataOut) + oldBids.add(broadcast.id) + } } } dataOut.flush() diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala index 05b4e67412f2..6f9b583898c3 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala @@ -18,9 +18,13 @@ package org.apache.spark.api.python import java.io.{ByteArrayOutputStream, DataOutputStream} +import java.net.{InetAddress, Socket} import java.nio.charset.StandardCharsets -import org.apache.spark.SparkFunSuite +import scala.concurrent.duration.Duration + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.security.SocketAuthHelper class PythonRDDSuite extends SparkFunSuite { @@ -44,4 +48,21 @@ class PythonRDDSuite extends SparkFunSuite { ("a".getBytes(StandardCharsets.UTF_8), null), (null, "b".getBytes(StandardCharsets.UTF_8))), buffer) } + + test("python server error handling") { + val authHelper = new SocketAuthHelper(new SparkConf()) + val errorServer = new ExceptionPythonServer(authHelper) + val client = new Socket(InetAddress.getLoopbackAddress(), errorServer.port) + authHelper.authToServer(client) + val ex = intercept[Exception] { errorServer.getResult(Duration(1, "second")) } + assert(ex.getCause().getMessage().contains("exception within handleConnection")) + } + + class ExceptionPythonServer(authHelper: SocketAuthHelper) + extends PythonServer[Unit](authHelper, "error-server") { + + override def handleConnection(sock: Socket): Unit = { + throw new Exception("exception within handleConnection") + } + } } diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index 2aa355504bf2..e267fbfa623b 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -387,6 +387,8 @@ def __hash__(self): "pyspark.profiler", "pyspark.shuffle", "pyspark.tests", + "pyspark.test_broadcast", + "pyspark.test_serializers", "pyspark.util", ] ) diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index b3dfc99962a3..1c7f2a7418df 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -15,13 +15,16 @@ # limitations under the License. # +import gc import os +import socket import sys -import gc from tempfile import NamedTemporaryFile import threading from pyspark.cloudpickle import print_exec +from pyspark.java_gateway import local_connect_and_auth +from pyspark.serializers import ChunkedStream from pyspark.util import _exception_message if sys.version < '3': @@ -64,19 +67,43 @@ class Broadcast(object): >>> large_broadcast = sc.broadcast(range(10000)) """ - def __init__(self, sc=None, value=None, pickle_registry=None, path=None): + def __init__(self, sc=None, value=None, pickle_registry=None, path=None, + sock_file=None): """ Should not be called directly by users -- use L{SparkContext.broadcast()} instead. """ if sc is not None: + # we're on the driver. We want the pickled data to end up in a file (maybe encrypted) f = NamedTemporaryFile(delete=False, dir=sc._temp_dir) - self._path = self.dump(value, f) - self._jbroadcast = sc._jvm.PythonRDD.readBroadcastFromFile(sc._jsc, self._path) + self._path = f.name + python_broadcast = sc._jvm.PythonRDD.setupBroadcast(self._path) + if sc._encryption_enabled: + # with encryption, we ask the jvm to do the encryption for us, we send it data + # over a socket + port, auth_secret = python_broadcast.setupEncryptionServer() + (encryption_sock_file, _) = local_connect_and_auth(port, auth_secret) + broadcast_out = ChunkedStream(encryption_sock_file, 8192) + else: + # no encryption, we can just write pickled data directly to the file from python + broadcast_out = f + self.dump(value, broadcast_out) + if sc._encryption_enabled: + python_broadcast.waitTillDataReceived() + self._jbroadcast = sc._jsc.broadcast(python_broadcast) self._pickle_registry = pickle_registry else: + # we're on an executor self._jbroadcast = None - self._path = path + if sock_file is not None: + # the jvm is doing decryption for us. Read the value + # immediately from the sock_file + self._value = self.load(sock_file) + else: + # the jvm just dumps the pickled data in path -- we'll unpickle lazily when + # the value is requested + assert(path is not None) + self._path = path def dump(self, value, f): try: @@ -89,24 +116,25 @@ def dump(self, value, f): print_exec(sys.stderr) raise pickle.PicklingError(msg) f.close() - return f.name - def load(self, path): + def load_from_path(self, path): with open(path, 'rb', 1 << 20) as f: - # pickle.load() may create lots of objects, disable GC - # temporary for better performance - gc.disable() - try: - return pickle.load(f) - finally: - gc.enable() + return self.load(f) + + def load(self, file): + # "file" could also be a socket + gc.disable() + try: + return pickle.load(file) + finally: + gc.enable() @property def value(self): """ Return the broadcasted value """ if not hasattr(self, "_value") and self._path is not None: - self._value = self.load(self._path) + self._value = self.load_from_path(self._path) return self._value def unpersist(self, blocking=False): diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 4cabae4b2f50..2c92c29a1cc1 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -33,9 +33,9 @@ from pyspark.broadcast import Broadcast, BroadcastPickleRegistry from pyspark.conf import SparkConf from pyspark.files import SparkFiles -from pyspark.java_gateway import launch_gateway +from pyspark.java_gateway import launch_gateway, local_connect_and_auth from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ - PairDeserializer, AutoBatchedSerializer, NoOpSerializer + PairDeserializer, AutoBatchedSerializer, NoOpSerializer, ChunkedStream from pyspark.storagelevel import StorageLevel from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix from pyspark.traceback_utils import CallSite, first_spark_call @@ -189,6 +189,13 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, self._javaAccumulator = self._jvm.PythonAccumulatorV2(host, port, auth_token) self._jsc.sc().register(self._javaAccumulator) + # If encryption is enabled, we need to setup a server in the jvm to read broadcast + # data via a socket. + # scala's mangled names w/ $ in them require special treatment. + encryption_conf = self._jvm.org.apache.spark.internal.config.__getattr__("package$")\ + .__getattr__("MODULE$").IO_ENCRYPTION_ENABLED() + self._encryption_enabled = self._jsc.sc().conf().get(encryption_conf) + self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python') self.pythonVer = "%d.%d" % sys.version_info[:2] @@ -498,23 +505,46 @@ def f(split, iterator): def reader_func(temp_filename): return self._jvm.PythonRDD.readRDDFromFile(self._jsc, temp_filename, numSlices) - jrdd = self._serialize_to_jvm(c, serializer, reader_func) + def createRDDServer(): + return self._jvm.PythonParallelizeServer(self._jsc.sc(), numSlices) + + jrdd = self._serialize_to_jvm(c, serializer, reader_func, createRDDServer) return RDD(jrdd, self, serializer) - def _serialize_to_jvm(self, data, serializer, reader_func): - """ - Calling the Java parallelize() method with an ArrayList is too slow, - because it sends O(n) Py4J commands. As an alternative, serialized - objects are written to a file and loaded through textFile(). - """ - tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) - try: - serializer.dump_stream(data, tempFile) - tempFile.close() - return reader_func(tempFile.name) - finally: - # readRDDFromFile eagerily reads the file so we can delete right after. - os.unlink(tempFile.name) + def _serialize_to_jvm(self, data, serializer, reader_func, createRDDServer): + """ + Using py4j to send a large dataset to the jvm is really slow, so we use either a file + or a socket if we have encryption enabled. + :param data: + :param serializer: + :param reader_func: A function which takes a filename and reads in the data in the jvm and + returns a JavaRDD. Only used when encryption is disabled. + :param createRDDServer: A function which creates a PythonRDDServer in the jvm to + accept the serialized data, for use when encryption is enabled. + :return: + """ + if self._encryption_enabled: + # with encryption, we open a server in java and send the data directly + server = createRDDServer() + (sock_file, _) = local_connect_and_auth(server.port(), server.secret()) + chunked_out = ChunkedStream(sock_file, 8192) + serializer.dump_stream(data, chunked_out) + chunked_out.close() + # this call will block until the server has read all the data and processed it (or + # throws an exception) + r = server.getResult() + return r + else: + # without encryption, we serialize to a file, and we read the file in java and + # parallelize from there. + tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) + try: + serializer.dump_stream(data, tempFile) + tempFile.close() + return reader_func(tempFile.name) + finally: + # we eagerily reads the file so we can delete right after. + os.unlink(tempFile.name) def pickleFile(self, name, minPartitions=None): """ diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 48006778e86f..ff9a612b77f6 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -731,6 +731,57 @@ def write_with_length(obj, stream): stream.write(obj) +class ChunkedStream(object): + + """ + This is a file-like object takes a stream of data, of unknown length, and breaks it into fixed + length frames. The intended use case is serializing large data and sending it immediately over + a socket -- we do not want to buffer the entire data before sending it, but the receiving end + needs to know whether or not there is more data coming. + + It works by buffering the incoming data in some fixed-size chunks. If the buffer is full, it + first sends the buffer size, then the data. This repeats as long as there is more data to send. + When this is closed, it sends the length of whatever data is in the buffer, then that data, and + finally a "length" of -1 to indicate the stream has completed. + """ + + def __init__(self, wrapped, buffer_size): + self.buffer_size = buffer_size + self.buffer = bytearray(buffer_size) + self.current_pos = 0 + self.wrapped = wrapped + + def write(self, bytes): + byte_pos = 0 + byte_remaining = len(bytes) + while byte_remaining > 0: + new_pos = byte_remaining + self.current_pos + if new_pos < self.buffer_size: + # just put it in our buffer + self.buffer[self.current_pos:new_pos] = bytes[byte_pos:] + self.current_pos = new_pos + byte_remaining = 0 + else: + # fill the buffer, send the length then the contents, and start filling again + space_left = self.buffer_size - self.current_pos + new_byte_pos = byte_pos + space_left + self.buffer[self.current_pos:self.buffer_size] = bytes[byte_pos:new_byte_pos] + write_int(self.buffer_size, self.wrapped) + self.wrapped.write(self.buffer) + byte_remaining -= space_left + byte_pos = new_byte_pos + self.current_pos = 0 + + def close(self): + # if there is anything left in the buffer, write it out first + if self.current_pos > 0: + write_int(self.current_pos, self.wrapped) + self.wrapped.write(self.buffer[:self.current_pos]) + # -1 length indicates to the receiving end that we're done. + write_int(-1, self.wrapped) + self.wrapped.close() + + if __name__ == '__main__': import doctest (failure_count, test_count) = doctest.testmod() diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 87d8d6a59a6e..51a38ebfd19f 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -539,12 +539,18 @@ def _create_from_pandas_with_arrow(self, pdf, schema, timezone): struct.names[i] = name schema = struct + jsqlContext = self._wrapped._jsqlContext + def reader_func(temp_filename): - return self._jvm.PythonSQLUtils.arrowReadStreamFromFile( - self._wrapped._jsqlContext, temp_filename, schema.json()) + return self._jvm.PythonSQLUtils.readArrowStreamFromFile(jsqlContext, temp_filename) + + def create_RDD_server(): + return self._jvm.ArrowRDDServer(jsqlContext) # Create Spark DataFrame from Arrow stream file, using one batch per partition - jdf = self._sc._serialize_to_jvm(batches, ArrowStreamSerializer(), reader_func) + jrdd = self._sc._serialize_to_jvm(batches, ArrowStreamSerializer(), reader_func, + create_RDD_server) + jdf = self._jvm.PythonSQLUtils.toDataFrame(jrdd, schema.json(), jsqlContext) df = DataFrame(jdf, self._wrapped) df._schema = schema return df diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 8e5bc6729dfa..08d7cfadc084 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -26,6 +26,7 @@ import pydoc import shutil import tempfile +import threading import pickle import functools import time @@ -228,12 +229,12 @@ def sql_conf(self, pairs): class ReusedSQLTestCase(ReusedPySparkTestCase, SQLTestUtils): @classmethod def setUpClass(cls): - ReusedPySparkTestCase.setUpClass() + super(ReusedSQLTestCase, cls).setUpClass() cls.spark = SparkSession(cls.sc) @classmethod def tearDownClass(cls): - ReusedPySparkTestCase.tearDownClass() + super(ReusedSQLTestCase, cls).tearDownClass() cls.spark.stop() def assertPandasEqual(self, expected, result): @@ -4105,7 +4106,8 @@ def setUpClass(cls): from decimal import Decimal from distutils.version import LooseVersion import pyarrow as pa - ReusedSQLTestCase.setUpClass() + super(ArrowTests, cls).setUpClass() + cls.warnings_lock = threading.Lock() # Synchronize default timezone between Python and Java cls.tz_prev = os.environ.get("TZ", None) # save current tz if set @@ -4146,7 +4148,7 @@ def tearDownClass(cls): if cls.tz_prev is not None: os.environ["TZ"] = cls.tz_prev time.tzset() - ReusedSQLTestCase.tearDownClass() + super(ArrowTests, cls).tearDownClass() def create_pandas_data_frame(self): import pandas as pd @@ -4166,15 +4168,18 @@ def test_toPandas_fallback_enabled(self): schema = StructType([StructField("map", MapType(StringType(), IntegerType()), True)]) df = self.spark.createDataFrame([({u'a': 1},)], schema=schema) with QuietTest(self.sc): - with warnings.catch_warnings(record=True) as warns: - pdf = df.toPandas() - # Catch and check the last UserWarning. - user_warns = [ - warn.message for warn in warns if isinstance(warn.message, UserWarning)] - self.assertTrue(len(user_warns) > 0) - self.assertTrue( - "Attempting non-optimization" in _exception_message(user_warns[-1])) - self.assertPandasEqual(pdf, pd.DataFrame({u'map': [{u'a': 1}]})) + with self.warnings_lock: + with warnings.catch_warnings(record=True) as warns: + # we want the warnings to appear even if this test is run from a subclass + warnings.simplefilter("always") + pdf = df.toPandas() + # Catch and check the last UserWarning. + user_warns = [ + warn.message for warn in warns if isinstance(warn.message, UserWarning)] + self.assertTrue(len(user_warns) > 0) + self.assertTrue( + "Attempting non-optimization" in _exception_message(user_warns[-1])) + self.assertPandasEqual(pdf, pd.DataFrame({u'map': [{u'a': 1}]})) def test_toPandas_fallback_disabled(self): from distutils.version import LooseVersion @@ -4183,8 +4188,9 @@ def test_toPandas_fallback_disabled(self): schema = StructType([StructField("map", MapType(StringType(), IntegerType()), True)]) df = self.spark.createDataFrame([(None,)], schema=schema) with QuietTest(self.sc): - with self.assertRaisesRegexp(Exception, 'Unsupported type'): - df.toPandas() + with self.warnings_lock: + with self.assertRaisesRegexp(Exception, 'Unsupported type'): + df.toPandas() # TODO: remove BinaryType check once minimum pyarrow version is 0.10.0 if LooseVersion(pa.__version__) < LooseVersion("0.10.0"): @@ -4396,6 +4402,8 @@ def test_createDataFrame_fallback_enabled(self): with QuietTest(self.sc): with self.sql_conf({"spark.sql.execution.arrow.fallback.enabled": True}): with warnings.catch_warnings(record=True) as warns: + # we want the warnings to appear even if this test is run from a subclass + warnings.simplefilter("always") df = self.spark.createDataFrame( pd.DataFrame([[{u'a': 1}]]), "a: map") # Catch and check the last UserWarning. @@ -4439,6 +4447,13 @@ def test_timestamp_dst(self): self.assertPandasEqual(pdf, df_from_pandas.toPandas()) +class EncryptionArrowTests(ArrowTests): + + @classmethod + def conf(cls): + return super(EncryptionArrowTests, cls).conf().set("spark.io.encryption.enabled", "true") + + @unittest.skipIf( not _have_pandas or not _have_pyarrow, _pandas_requirement_message or _pyarrow_requirement_message) diff --git a/python/pyspark/test_broadcast.py b/python/pyspark/test_broadcast.py new file mode 100644 index 000000000000..a00329c18ad8 --- /dev/null +++ b/python/pyspark/test_broadcast.py @@ -0,0 +1,126 @@ +# +# 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. +# + +import os +import random +import tempfile +import unittest + +try: + import xmlrunner +except ImportError: + xmlrunner = None + +from pyspark.broadcast import Broadcast +from pyspark.conf import SparkConf +from pyspark.context import SparkContext +from pyspark.java_gateway import launch_gateway +from pyspark.serializers import ChunkedStream + + +class BroadcastTest(unittest.TestCase): + + def tearDown(self): + if getattr(self, "sc", None) is not None: + self.sc.stop() + self.sc = None + + def _test_encryption_helper(self, vs): + """ + Creates a broadcast variables for each value in vs, and runs a simple job to make sure the + value is the same when it's read in the executors. Also makes sure there are no task + failures. + """ + bs = [self.sc.broadcast(value=v) for v in vs] + exec_values = self.sc.parallelize(range(2)).map(lambda x: [b.value for b in bs]).collect() + for ev in exec_values: + self.assertEqual(ev, vs) + # make sure there are no task failures + status = self.sc.statusTracker() + for jid in status.getJobIdsForGroup(): + for sid in status.getJobInfo(jid).stageIds: + stage_info = status.getStageInfo(sid) + self.assertEqual(0, stage_info.numFailedTasks) + + def _test_multiple_broadcasts(self, *extra_confs): + """ + Test broadcast variables make it OK to the executors. Tests multiple broadcast variables, + and also multiple jobs. + """ + conf = SparkConf() + for key, value in extra_confs: + conf.set(key, value) + conf.setMaster("local-cluster[2,1,1024]") + self.sc = SparkContext(conf=conf) + self._test_encryption_helper([5]) + self._test_encryption_helper([5, 10, 20]) + + def test_broadcast_with_encryption(self): + self._test_multiple_broadcasts(("spark.io.encryption.enabled", "true")) + + def test_broadcast_no_encryption(self): + self._test_multiple_broadcasts() + + +class BroadcastFrameProtocolTest(unittest.TestCase): + + @classmethod + def setUpClass(cls): + gateway = launch_gateway(SparkConf()) + cls._jvm = gateway.jvm + cls.longMessage = True + random.seed(42) + + def _test_chunked_stream(self, data, py_buf_size): + # write data using the chunked protocol from python. + chunked_file = tempfile.NamedTemporaryFile(delete=False) + dechunked_file = tempfile.NamedTemporaryFile(delete=False) + dechunked_file.close() + try: + out = ChunkedStream(chunked_file, py_buf_size) + out.write(data) + out.close() + # now try to read it in java + jin = self._jvm.java.io.FileInputStream(chunked_file.name) + jout = self._jvm.java.io.FileOutputStream(dechunked_file.name) + self._jvm.DechunkedInputStream.dechunkAndCopyToOutput(jin, jout) + # java should have decoded it back to the original data + self.assertEqual(len(data), os.stat(dechunked_file.name).st_size) + with open(dechunked_file.name, "rb") as f: + byte = f.read(1) + idx = 0 + while byte: + self.assertEqual(data[idx], bytearray(byte)[0], msg="idx = " + str(idx)) + byte = f.read(1) + idx += 1 + finally: + os.unlink(chunked_file.name) + os.unlink(dechunked_file.name) + + def test_chunked_stream(self): + def random_bytes(n): + return bytearray(random.getrandbits(8) for _ in range(n)) + for data_length in [1, 10, 100, 10000]: + for buffer_length in [1, 2, 5, 8192]: + self._test_chunked_stream(random_bytes(data_length), buffer_length) + +if __name__ == '__main__': + from pyspark.test_broadcast import * + if xmlrunner: + unittest.main(testRunner=xmlrunner.XMLTestRunner(output='target/test-reports'), verbosity=2) + else: + unittest.main(verbosity=2) diff --git a/python/pyspark/test_serializers.py b/python/pyspark/test_serializers.py new file mode 100644 index 000000000000..5b43729f9ebb --- /dev/null +++ b/python/pyspark/test_serializers.py @@ -0,0 +1,90 @@ +# +# 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. +# + +import io +import math +import struct +import sys +import unittest + +try: + import xmlrunner +except ImportError: + xmlrunner = None + +from pyspark import serializers + + +def read_int(b): + return struct.unpack("!i", b)[0] + + +def write_int(i): + return struct.pack("!i", i) + + +class SerializersTest(unittest.TestCase): + + def test_chunked_stream(self): + original_bytes = bytearray(range(100)) + for data_length in [1, 10, 100]: + for buffer_length in [1, 2, 3, 5, 20, 99, 100, 101, 500]: + dest = ByteArrayOutput() + stream_out = serializers.ChunkedStream(dest, buffer_length) + stream_out.write(original_bytes[:data_length]) + stream_out.close() + num_chunks = int(math.ceil(float(data_length) / buffer_length)) + # length for each chunk, and a final -1 at the very end + exp_size = (num_chunks + 1) * 4 + data_length + self.assertEqual(len(dest.buffer), exp_size) + dest_pos = 0 + data_pos = 0 + for chunk_idx in range(num_chunks): + chunk_length = read_int(dest.buffer[dest_pos:(dest_pos + 4)]) + if chunk_idx == num_chunks - 1: + exp_length = data_length % buffer_length + if exp_length == 0: + exp_length = buffer_length + else: + exp_length = buffer_length + self.assertEqual(chunk_length, exp_length) + dest_pos += 4 + dest_chunk = dest.buffer[dest_pos:dest_pos + chunk_length] + orig_chunk = original_bytes[data_pos:data_pos + chunk_length] + self.assertEqual(dest_chunk, orig_chunk) + dest_pos += chunk_length + data_pos += chunk_length + # ends with a -1 + self.assertEqual(dest.buffer[-4:], write_int(-1)) + + +class ByteArrayOutput(object): + def __init__(self): + self.buffer = bytearray() + + def write(self, b): + self.buffer += b + + def close(self): + pass + +if __name__ == '__main__': + from pyspark.test_serializers import * + if xmlrunner: + unittest.main(testRunner=xmlrunner.XMLTestRunner(output='target/test-reports'), verbosity=2) + else: + unittest.main(verbosity=2) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 8ac1df52fc59..050c2dd01836 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -372,9 +372,16 @@ def tearDown(self): class ReusedPySparkTestCase(unittest.TestCase): + @classmethod + def conf(cls): + """ + Override this in subclasses to supply a more specific conf + """ + return SparkConf() + @classmethod def setUpClass(cls): - cls.sc = SparkContext('local[4]', cls.__name__) + cls.sc = SparkContext('local[4]', cls.__name__, conf=cls.conf()) @classmethod def tearDownClass(cls): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index e934da4d2eb6..974344f01d92 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -324,16 +324,34 @@ def main(infile, outfile): importlib.invalidate_caches() # fetch names and values of broadcast variables + needs_broadcast_decryption_server = read_bool(infile) num_broadcast_variables = read_int(infile) + if needs_broadcast_decryption_server: + # read the decrypted data from a server in the jvm + port = read_int(infile) + auth_secret = utf8_deserializer.loads(infile) + (broadcast_sock_file, _) = local_connect_and_auth(port, auth_secret) + for _ in range(num_broadcast_variables): bid = read_long(infile) if bid >= 0: - path = utf8_deserializer.loads(infile) - _broadcastRegistry[bid] = Broadcast(path=path) + if needs_broadcast_decryption_server: + read_bid = read_long(broadcast_sock_file) + assert(read_bid == bid) + _broadcastRegistry[bid] = \ + Broadcast(sock_file=broadcast_sock_file) + else: + path = utf8_deserializer.loads(infile) + _broadcastRegistry[bid] = Broadcast(path=path) + else: bid = - bid - 1 _broadcastRegistry.pop(bid) + if needs_broadcast_decryption_server: + broadcast_sock_file.write(b'1') + broadcast_sock_file.close() + _accumulatorRegistry.clear() eval_type = read_int(infile) if eval_type == PythonEvalType.NON_UDF: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala index c0830e77b5a8..482e2bfeb709 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/python/PythonSQLUtils.scala @@ -17,6 +17,12 @@ package org.apache.spark.sql.api.python +import java.io.InputStream +import java.nio.channels.Channels + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.api.python.PythonRDDServer +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.catalyst.analysis.FunctionRegistry import org.apache.spark.sql.catalyst.expressions.ExpressionInfo @@ -33,19 +39,36 @@ private[sql] object PythonSQLUtils { } /** - * Python callable function to read a file in Arrow stream format and create a [[DataFrame]] + * Python callable function to read a file in Arrow stream format and create a [[RDD]] * using each serialized ArrowRecordBatch as a partition. - * - * @param sqlContext The active [[SQLContext]]. - * @param filename File to read the Arrow stream from. - * @param schemaString JSON Formatted Spark schema for Arrow batches. - * @return A new [[DataFrame]]. */ - def arrowReadStreamFromFile( - sqlContext: SQLContext, - filename: String, - schemaString: String): DataFrame = { - val jrdd = ArrowConverters.readArrowStreamFromFile(sqlContext, filename) - ArrowConverters.toDataFrame(jrdd, schemaString, sqlContext) + def readArrowStreamFromFile(sqlContext: SQLContext, filename: String): JavaRDD[Array[Byte]] = { + ArrowConverters.readArrowStreamFromFile(sqlContext, filename) + } + + /** + * Python callable function to read a file in Arrow stream format and create a [[DataFrame]] + * from an RDD. + */ + def toDataFrame( + arrowBatchRDD: JavaRDD[Array[Byte]], + schemaString: String, + sqlContext: SQLContext): DataFrame = { + ArrowConverters.toDataFrame(arrowBatchRDD, schemaString, sqlContext) } } + +/** + * Helper for making a dataframe from arrow data from data sent from python over a socket. This is + * used when encryption is enabled, and we don't want to write data to a file. + */ +private[sql] class ArrowRDDServer(sqlContext: SQLContext) extends PythonRDDServer { + + override protected def streamToRDD(input: InputStream): RDD[Array[Byte]] = { + // Create array to consume iterator so that we can safely close the inputStream + val batches = ArrowConverters.getBatchesFromStream(Channels.newChannel(input)).toArray + // Parallelize the record batches to create an RDD + JavaRDD.fromRDD(sqlContext.sparkContext.parallelize(batches, batches.length)) + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala index 1a48bc8398a6..2bf6a58b5565 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.execution.arrow import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, OutputStream} -import java.nio.channels.{Channels, SeekableByteChannel} +import java.nio.channels.{Channels, ReadableByteChannel} import scala.collection.JavaConverters._ @@ -31,6 +31,7 @@ import org.apache.arrow.vector.ipc.message.{ArrowRecordBatch, MessageSerializer} import org.apache.spark.TaskContext import org.apache.spark.api.java.JavaRDD import org.apache.spark.network.util.JavaUtils +import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types._ @@ -189,7 +190,7 @@ private[sql] object ArrowConverters { } /** - * Create a DataFrame from a JavaRDD of serialized ArrowRecordBatches. + * Create a DataFrame from an RDD of serialized ArrowRecordBatches. */ private[sql] def toDataFrame( arrowBatchRDD: JavaRDD[Array[Byte]], @@ -221,7 +222,7 @@ private[sql] object ArrowConverters { /** * Read an Arrow stream input and return an iterator of serialized ArrowRecordBatches. */ - private[sql] def getBatchesFromStream(in: SeekableByteChannel): Iterator[Array[Byte]] = { + private[sql] def getBatchesFromStream(in: ReadableByteChannel): Iterator[Array[Byte]] = { // Iterate over the serialized Arrow RecordBatch messages from a stream new Iterator[Array[Byte]] { @@ -271,7 +272,7 @@ private[sql] object ArrowConverters { } else { if (bodyLength > 0) { // Skip message body if not a RecordBatch - in.position(in.position() + bodyLength) + Channels.newInputStream(in).skip(bodyLength) } // Proceed to next message From 80e317b3a5cafb95a5b5a00d1d8b3b8c595985e2 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 6 Sep 2018 12:11:47 -0500 Subject: [PATCH 060/879] [PYSPARK][SQL] Updates to RowQueue Tested with updates to RowQueueSuite (cherry picked from commit 8f5a5a9e5b9f273443b2721f80c99dc7397ef4c0) --- .../spark/sql/execution/python/RowQueue.scala | 27 +++++++++++++----- .../sql/execution/python/RowQueueSuite.scala | 28 ++++++++++++++----- 2 files changed, 41 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala index e2fa6e7f504b..d2820ff335ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala @@ -21,9 +21,10 @@ import java.io._ import com.google.common.io.Closeables -import org.apache.spark.SparkException +import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.io.NioBufferedFileInputStream import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager} +import org.apache.spark.serializer.SerializerManager import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.memory.MemoryBlock @@ -108,9 +109,13 @@ private[python] abstract class InMemoryRowQueue(val page: MemoryBlock, numFields * A RowQueue that is backed by a file on disk. This queue will stop accepting new rows once any * reader has begun reading from the queue. */ -private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueue { - private var out = new DataOutputStream( - new BufferedOutputStream(new FileOutputStream(file.toString))) +private[python] case class DiskRowQueue( + file: File, + fields: Int, + serMgr: SerializerManager) extends RowQueue { + + private var out = new DataOutputStream(serMgr.wrapForEncryption( + new BufferedOutputStream(new FileOutputStream(file.toString)))) private var unreadBytes = 0L private var in: DataInputStream = _ @@ -131,7 +136,8 @@ private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueu if (out != null) { out.close() out = null - in = new DataInputStream(new NioBufferedFileInputStream(file)) + in = new DataInputStream(serMgr.wrapForEncryption( + new NioBufferedFileInputStream(file))) } if (unreadBytes > 0) { @@ -166,7 +172,8 @@ private[python] case class DiskRowQueue(file: File, fields: Int) extends RowQueu private[python] case class HybridRowQueue( memManager: TaskMemoryManager, tempDir: File, - numFields: Int) + numFields: Int, + serMgr: SerializerManager) extends MemoryConsumer(memManager) with RowQueue { // Each buffer should have at least one row @@ -212,7 +219,7 @@ private[python] case class HybridRowQueue( } private def createDiskQueue(): RowQueue = { - DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields) + DiskRowQueue(File.createTempFile("buffer", "", tempDir), numFields, serMgr) } private def createNewQueue(required: Long): RowQueue = { @@ -279,3 +286,9 @@ private[python] case class HybridRowQueue( } } } + +private[python] object HybridRowQueue { + def apply(taskMemoryMgr: TaskMemoryManager, file: File, fields: Int): HybridRowQueue = { + HybridRowQueue(taskMemoryMgr, file, fields, SparkEnv.get.serializerManager) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala index ffda33cf906c..1ec998632842 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/RowQueueSuite.scala @@ -20,12 +20,15 @@ package org.apache.spark.sql.execution.python import java.io.File import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.memory.{MemoryManager, TaskMemoryManager, TestMemoryManager} +import org.apache.spark.internal.config._ +import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} +import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.unsafe.memory.MemoryBlock import org.apache.spark.util.Utils -class RowQueueSuite extends SparkFunSuite { +class RowQueueSuite extends SparkFunSuite with EncryptionFunSuite { test("in-memory queue") { val page = MemoryBlock.fromLongArray(new Array[Long](1<<10)) @@ -53,10 +56,20 @@ class RowQueueSuite extends SparkFunSuite { queue.close() } - test("disk queue") { + private def createSerializerManager(conf: SparkConf): SerializerManager = { + val ioEncryptionKey = if (conf.get(IO_ENCRYPTION_ENABLED)) { + Some(CryptoStreamUtils.createKey(conf)) + } else { + None + } + new SerializerManager(new JavaSerializer(conf), conf, ioEncryptionKey) + } + + encryptionTest("disk queue") { conf => + val serManager = createSerializerManager(conf) val dir = Utils.createTempDir().getCanonicalFile dir.mkdirs() - val queue = DiskRowQueue(new File(dir, "buffer"), 1) + val queue = DiskRowQueue(new File(dir, "buffer"), 1, serManager) val row = new UnsafeRow(1) row.pointTo(new Array[Byte](16), 16) val n = 1000 @@ -81,11 +94,12 @@ class RowQueueSuite extends SparkFunSuite { queue.close() } - test("hybrid queue") { - val mem = new TestMemoryManager(new SparkConf()) + encryptionTest("hybrid queue") { conf => + val serManager = createSerializerManager(conf) + val mem = new TestMemoryManager(conf) mem.limit(4<<10) val taskM = new TaskMemoryManager(mem, 0) - val queue = HybridRowQueue(taskM, Utils.createTempDir().getCanonicalFile, 1) + val queue = HybridRowQueue(taskM, Utils.createTempDir().getCanonicalFile, 1, serManager) val row = new UnsafeRow(1) row.pointTo(new Array[Byte](16), 16) val n = (4<<10) / 16 * 3 From 7beb3417591b0f9c436d9175f3343ee79795d536 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 22 Aug 2018 16:38:28 -0500 Subject: [PATCH 061/879] [CORE] Updates to remote cache reads Covered by tests in DistributedSuite (cherry picked from commit a97001d21757ae214c86371141bd78a376200f66) --- .../spark/network/buffer/ManagedBuffer.java | 5 +- .../spark/network/shuffle/DownloadFile.java | 47 ++++++++++ ...eManager.java => DownloadFileManager.java} | 12 +-- .../shuffle/DownloadFileWritableChannel.java | 30 ++++++ .../shuffle/ExternalShuffleClient.java | 4 +- .../shuffle/OneForOneBlockFetcher.java | 28 ++---- .../spark/network/shuffle/ShuffleClient.java | 4 +- .../network/shuffle/SimpleDownloadFile.java | 91 +++++++++++++++++++ .../spark/network/BlockTransferService.scala | 6 +- .../netty/NettyBlockTransferService.scala | 4 +- .../apache/spark/storage/BlockManager.scala | 78 +++++++++++++--- .../org/apache/spark/storage/DiskStore.scala | 16 ++++ .../storage/ShuffleBlockFetcherIterator.scala | 21 +++-- .../spark/storage/BlockManagerSuite.scala | 8 +- .../ShuffleBlockFetcherIteratorSuite.scala | 6 +- 15 files changed, 298 insertions(+), 62 deletions(-) create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java rename common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/{TempFileManager.java => DownloadFileManager.java} (75%) create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java create mode 100644 common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/SimpleDownloadFile.java diff --git a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java index 1861f8d7fd8f..2d573f512437 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/buffer/ManagedBuffer.java @@ -36,7 +36,10 @@ */ public abstract class ManagedBuffer { - /** Number of bytes of the data. */ + /** + * Number of bytes of the data. If this buffer will decrypt for all of the views into the data, + * this is the size of the decrypted data. + */ public abstract long size(); /** diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java new file mode 100644 index 000000000000..633622b35175 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFile.java @@ -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. + */ + +package org.apache.spark.network.shuffle; + +import java.io.IOException; + +/** + * A handle on the file used when fetching remote data to disk. Used to ensure the lifecycle of + * writing the data, reading it back, and then cleaning it up is followed. Specific implementations + * may also handle encryption. The data can be read only via DownloadFileWritableChannel, + * which ensures data is not read until after the writer is closed. + */ +public interface DownloadFile { + /** + * Delete the file. + * + * @return true if and only if the file or directory is + * successfully deleted; false otherwise + */ + boolean delete(); + + /** + * A channel for writing data to the file. This special channel allows access to the data for + * reading, after the channel is closed, via {@link DownloadFileWritableChannel#closeAndRead()}. + */ + DownloadFileWritableChannel openForWriting() throws IOException; + + /** + * The path of the file, intended only for debug purposes. + */ + String path(); +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/TempFileManager.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileManager.java similarity index 75% rename from common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/TempFileManager.java rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileManager.java index 552364d274f1..c335a17ae1fe 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/TempFileManager.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileManager.java @@ -17,20 +17,20 @@ package org.apache.spark.network.shuffle; -import java.io.File; +import org.apache.spark.network.util.TransportConf; /** - * A manager to create temp block files to reduce the memory usage and also clean temp - * files when they won't be used any more. + * A manager to create temp block files used when fetching remote data to reduce the memory usage. + * It will clean files when they won't be used any more. */ -public interface TempFileManager { +public interface DownloadFileManager { /** Create a temp block file. */ - File createTempFile(); + DownloadFile createTempFile(TransportConf transportConf); /** * Register a temp file to clean up when it won't be used any more. Return whether the * file is registered successfully. If `false`, the caller should clean up the file by itself. */ - boolean registerTempFileToClean(File file); + boolean registerTempFileToClean(DownloadFile file); } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java new file mode 100644 index 000000000000..dbbbac43eb74 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/DownloadFileWritableChannel.java @@ -0,0 +1,30 @@ +/* + * 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.network.shuffle; + +import org.apache.spark.network.buffer.ManagedBuffer; + +import java.nio.channels.WritableByteChannel; + +/** + * A channel for writing data which is fetched to disk, which allows access to the written data only + * after the writer has been closed. Used with DownloadFile and DownloadFileManager. + */ +public interface DownloadFileWritableChannel extends WritableByteChannel { + ManagedBuffer closeAndRead(); +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java index 7ed0b6e93a7a..9a2cf0f95348 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -91,7 +91,7 @@ public void fetchBlocks( String execId, String[] blockIds, BlockFetchingListener listener, - TempFileManager tempFileManager) { + DownloadFileManager downloadFileManager) { checkInit(); logger.debug("External shuffle fetch from {}:{} (executor id {})", host, port, execId); try { @@ -99,7 +99,7 @@ public void fetchBlocks( (blockIds1, listener1) -> { TransportClient client = clientFactory.createClient(host, port); new OneForOneBlockFetcher(client, appId, execId, - blockIds1, listener1, conf, tempFileManager).start(); + blockIds1, listener1, conf, downloadFileManager).start(); }; int maxRetries = conf.maxIORetries(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index 0bc571874f07..30587023877c 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -17,18 +17,13 @@ package org.apache.spark.network.shuffle; -import java.io.File; -import java.io.FileOutputStream; import java.io.IOException; import java.nio.ByteBuffer; -import java.nio.channels.Channels; -import java.nio.channels.WritableByteChannel; import java.util.Arrays; import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import org.apache.spark.network.buffer.FileSegmentManagedBuffer; import org.apache.spark.network.buffer.ManagedBuffer; import org.apache.spark.network.client.ChunkReceivedCallback; import org.apache.spark.network.client.RpcResponseCallback; @@ -58,7 +53,7 @@ public class OneForOneBlockFetcher { private final BlockFetchingListener listener; private final ChunkReceivedCallback chunkCallback; private final TransportConf transportConf; - private final TempFileManager tempFileManager; + private final DownloadFileManager downloadFileManager; private StreamHandle streamHandle = null; @@ -79,14 +74,14 @@ public OneForOneBlockFetcher( String[] blockIds, BlockFetchingListener listener, TransportConf transportConf, - TempFileManager tempFileManager) { + DownloadFileManager downloadFileManager) { this.client = client; this.openMessage = new OpenBlocks(appId, execId, blockIds); this.blockIds = blockIds; this.listener = listener; this.chunkCallback = new ChunkCallback(); this.transportConf = transportConf; - this.tempFileManager = tempFileManager; + this.downloadFileManager = downloadFileManager; } /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ @@ -125,7 +120,7 @@ public void onSuccess(ByteBuffer response) { // Immediately request all chunks -- we expect that the total size of the request is // reasonable due to higher level chunking in [[ShuffleBlockFetcherIterator]]. for (int i = 0; i < streamHandle.numChunks; i++) { - if (tempFileManager != null) { + if (downloadFileManager != null) { client.stream(OneForOneStreamManager.genStreamChunkId(streamHandle.streamId, i), new DownloadCallback(i)); } else { @@ -159,13 +154,13 @@ private void failRemainingBlocks(String[] failedBlockIds, Throwable e) { private class DownloadCallback implements StreamCallback { - private WritableByteChannel channel = null; - private File targetFile = null; + private DownloadFileWritableChannel channel = null; + private DownloadFile targetFile = null; private int chunkIndex; DownloadCallback(int chunkIndex) throws IOException { - this.targetFile = tempFileManager.createTempFile(); - this.channel = Channels.newChannel(new FileOutputStream(targetFile)); + this.targetFile = downloadFileManager.createTempFile(transportConf); + this.channel = targetFile.openForWriting(); this.chunkIndex = chunkIndex; } @@ -178,11 +173,8 @@ public void onData(String streamId, ByteBuffer buf) throws IOException { @Override public void onComplete(String streamId) throws IOException { - channel.close(); - ManagedBuffer buffer = new FileSegmentManagedBuffer(transportConf, targetFile, 0, - targetFile.length()); - listener.onBlockFetchSuccess(blockIds[chunkIndex], buffer); - if (!tempFileManager.registerTempFileToClean(targetFile)) { + listener.onBlockFetchSuccess(blockIds[chunkIndex], channel.closeAndRead()); + if (!downloadFileManager.registerTempFileToClean(targetFile)) { targetFile.delete(); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java index 18b04fedcac5..62b99c40f61f 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java @@ -43,7 +43,7 @@ public void init(String appId) { } * @param execId the executor id. * @param blockIds block ids to fetch. * @param listener the listener to receive block fetching status. - * @param tempFileManager TempFileManager to create and clean temp files. + * @param downloadFileManager DownloadFileManager to create and clean temp files. * If it's not null, the remote blocks will be streamed * into temp shuffle files to reduce the memory usage, otherwise, * they will be kept in memory. @@ -54,7 +54,7 @@ public abstract void fetchBlocks( String execId, String[] blockIds, BlockFetchingListener listener, - TempFileManager tempFileManager); + DownloadFileManager downloadFileManager); /** * Get the shuffle MetricsSet from ShuffleClient, this will be used in MetricsSystem to diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/SimpleDownloadFile.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/SimpleDownloadFile.java new file mode 100644 index 000000000000..670612fd6f66 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/SimpleDownloadFile.java @@ -0,0 +1,91 @@ +/* + * 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.network.shuffle; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.channels.Channels; +import java.nio.channels.WritableByteChannel; + +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.buffer.ManagedBuffer; +import org.apache.spark.network.util.TransportConf; + +/** + * A DownloadFile that does not take any encryption settings into account for reading and + * writing data. + * + * This does *not* mean the data in the file is un-encrypted -- it could be that the data is + * already encrypted when its written, and subsequent layer is responsible for decrypting. + */ +public class SimpleDownloadFile implements DownloadFile { + + private final File file; + private final TransportConf transportConf; + + public SimpleDownloadFile(File file, TransportConf transportConf) { + this.file = file; + this.transportConf = transportConf; + } + + @Override + public boolean delete() { + return file.delete(); + } + + @Override + public DownloadFileWritableChannel openForWriting() throws IOException { + return new SimpleDownloadWritableChannel(); + } + + @Override + public String path() { + return file.getAbsolutePath(); + } + + private class SimpleDownloadWritableChannel implements DownloadFileWritableChannel { + + private final WritableByteChannel channel; + + SimpleDownloadWritableChannel() throws FileNotFoundException { + channel = Channels.newChannel(new FileOutputStream(file)); + } + + @Override + public ManagedBuffer closeAndRead() { + return new FileSegmentManagedBuffer(transportConf, file, 0, file.length()); + } + + @Override + public int write(ByteBuffer src) throws IOException { + return channel.write(src); + } + + @Override + public boolean isOpen() { + return channel.isOpen(); + } + + @Override + public void close() throws IOException { + channel.close(); + } + } +} diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index 1d8a266d0079..eef8c31e05ab 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -26,7 +26,7 @@ import scala.reflect.ClassTag import org.apache.spark.internal.Logging import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} -import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient, TempFileManager} +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ShuffleClient} import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.util.ThreadUtils @@ -68,7 +68,7 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo execId: String, blockIds: Array[String], listener: BlockFetchingListener, - tempFileManager: TempFileManager): Unit + tempFileManager: DownloadFileManager): Unit /** * Upload a single block to a remote node, available only after [[init]] is invoked. @@ -92,7 +92,7 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo port: Int, execId: String, blockId: String, - tempFileManager: TempFileManager): ManagedBuffer = { + tempFileManager: DownloadFileManager): ManagedBuffer = { // A monitor for the thread to wait on. val result = Promise[ManagedBuffer]() fetchBlocks(host, port, execId, Array(blockId), diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index 1905632a936d..dc55685b1e7b 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -33,7 +33,7 @@ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClientBootstrap, TransportClientFactory} import org.apache.spark.network.crypto.{AuthClientBootstrap, AuthServerBootstrap} import org.apache.spark.network.server._ -import org.apache.spark.network.shuffle.{BlockFetchingListener, OneForOneBlockFetcher, RetryingBlockFetcher, TempFileManager} +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, OneForOneBlockFetcher, RetryingBlockFetcher} import org.apache.spark.network.shuffle.protocol.{UploadBlock, UploadBlockStream} import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.JavaSerializer @@ -106,7 +106,7 @@ private[spark] class NettyBlockTransferService( execId: String, blockIds: Array[String], listener: BlockFetchingListener, - tempFileManager: TempFileManager): Unit = { + tempFileManager: DownloadFileManager): Unit = { logTrace(s"Fetch blocks from $host:$port (executor id $execId)") try { val blockFetchStarter = new RetryingBlockFetcher.BlockFetchStarter { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index f5c69ad241e3..22341467add5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -33,6 +33,7 @@ import scala.util.Random import scala.util.control.NonFatal import com.codahale.metrics.{MetricRegistry, MetricSet} +import com.google.common.io.CountingOutputStream import org.apache.spark._ import org.apache.spark.executor.{DataReadMethod, ShuffleWriteMetrics} @@ -43,8 +44,9 @@ import org.apache.spark.network._ import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.client.StreamCallbackWithID import org.apache.spark.network.netty.SparkTransportConf -import org.apache.spark.network.shuffle.{ExternalShuffleClient, TempFileManager} +import org.apache.spark.network.shuffle._ import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo +import org.apache.spark.network.util.TransportConf import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.serializer.{SerializerInstance, SerializerManager} @@ -213,11 +215,11 @@ private[spark] class BlockManager( private var blockReplicationPolicy: BlockReplicationPolicy = _ - // A TempFileManager used to track all the files of remote blocks which above the + // A DownloadFileManager used to track all the files of remote blocks which are above the // specified memory threshold. Files will be deleted automatically based on weak reference. // Exposed for test private[storage] val remoteBlockTempFileManager = - new BlockManager.RemoteBlockTempFileManager(this) + new BlockManager.RemoteBlockDownloadFileManager(this) private val maxRemoteBlockToMem = conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM) /** @@ -1664,23 +1666,28 @@ private[spark] object BlockManager { metricRegistry.registerAll(metricSet) } - class RemoteBlockTempFileManager(blockManager: BlockManager) - extends TempFileManager with Logging { + class RemoteBlockDownloadFileManager(blockManager: BlockManager) + extends DownloadFileManager with Logging { + // lazy because SparkEnv is set after this + lazy val encryptionKey = SparkEnv.get.securityManager.getIOEncryptionKey() - private class ReferenceWithCleanup(file: File, referenceQueue: JReferenceQueue[File]) - extends WeakReference[File](file, referenceQueue) { - private val filePath = file.getAbsolutePath + private class ReferenceWithCleanup( + file: DownloadFile, + referenceQueue: JReferenceQueue[DownloadFile] + ) extends WeakReference[DownloadFile](file, referenceQueue) { + + val filePath = file.path() def cleanUp(): Unit = { logDebug(s"Clean up file $filePath") - if (!new File(filePath).delete()) { + if (!file.delete()) { logDebug(s"Fail to delete file $filePath") } } } - private val referenceQueue = new JReferenceQueue[File] + private val referenceQueue = new JReferenceQueue[DownloadFile] private val referenceBuffer = Collections.newSetFromMap[ReferenceWithCleanup]( new ConcurrentHashMap) @@ -1692,11 +1699,21 @@ private[spark] object BlockManager { cleaningThread.setName("RemoteBlock-temp-file-clean-thread") cleaningThread.start() - override def createTempFile(): File = { - blockManager.diskBlockManager.createTempLocalBlock()._2 + override def createTempFile(transportConf: TransportConf): DownloadFile = { + val file = blockManager.diskBlockManager.createTempLocalBlock()._2 + encryptionKey match { + case Some(key) => + // encryption is enabled, so when we read the decrypted data off the network, we need to + // encrypt it when writing to disk. Note that the data may have been encrypted when it + // was cached on disk on the remote side, but it was already decrypted by now (see + // EncryptedBlockData). + new EncryptedDownloadFile(file, key) + case None => + new SimpleDownloadFile(file, transportConf) + } } - override def registerTempFileToClean(file: File): Boolean = { + override def registerTempFileToClean(file: DownloadFile): Boolean = { referenceBuffer.add(new ReferenceWithCleanup(file, referenceQueue)) } @@ -1724,4 +1741,39 @@ private[spark] object BlockManager { } } } + + /** + * A DownloadFile that encrypts data when it is written, and decrypts when it's read. + */ + private class EncryptedDownloadFile( + file: File, + key: Array[Byte]) extends DownloadFile { + + private val env = SparkEnv.get + + override def delete(): Boolean = file.delete() + + override def openForWriting(): DownloadFileWritableChannel = { + new EncryptedDownloadWritableChannel() + } + + override def path(): String = file.getAbsolutePath + + private class EncryptedDownloadWritableChannel extends DownloadFileWritableChannel { + private val countingOutput: CountingWritableChannel = new CountingWritableChannel( + Channels.newChannel(env.serializerManager.wrapForEncryption(new FileOutputStream(file)))) + + override def closeAndRead(): ManagedBuffer = { + countingOutput.close() + val size = countingOutput.getCount + new EncryptedManagedBuffer(new EncryptedBlockData(file, size, env.conf, key)) + } + + override def write(src: ByteBuffer): Int = countingOutput.write(src) + + override def isOpen: Boolean = countingOutput.isOpen() + + override def close(): Unit = countingOutput.close() + } + } } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index a820bc70b33b..d88bd710d1ea 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -30,6 +30,7 @@ import io.netty.channel.DefaultFileRegion import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.internal.{config, Logging} +import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.util.{AbstractFileRegion, JavaUtils} import org.apache.spark.security.CryptoStreamUtils import org.apache.spark.util.Utils @@ -260,7 +261,22 @@ private class EncryptedBlockData( throw e } } +} + +private class EncryptedManagedBuffer(val blockData: EncryptedBlockData) extends ManagedBuffer { + + // This is the size of the decrypted data + override def size(): Long = blockData.size + + override def nioByteBuffer(): ByteBuffer = blockData.toByteBuffer() + + override def convertToNetty(): AnyRef = blockData.toNetty() + + override def createInputStream(): InputStream = blockData.toInputStream() + + override def retain(): ManagedBuffer = this + override def release(): ManagedBuffer = this } private class ReadableChannelFileRegion(source: ReadableByteChannel, blockSize: Long) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 00d01dd28afb..e534c746433f 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.{File, InputStream, IOException} +import java.io.{InputStream, IOException} import java.nio.ByteBuffer import java.util.concurrent.LinkedBlockingQueue import javax.annotation.concurrent.GuardedBy @@ -28,7 +28,8 @@ import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} -import org.apache.spark.network.shuffle.{BlockFetchingListener, ShuffleClient, TempFileManager} +import org.apache.spark.network.shuffle._ +import org.apache.spark.network.util.TransportConf import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util.Utils import org.apache.spark.util.io.ChunkedByteBufferOutputStream @@ -71,7 +72,7 @@ final class ShuffleBlockFetcherIterator( maxBlocksInFlightPerAddress: Int, maxReqSizeShuffleToMem: Long, detectCorrupt: Boolean) - extends Iterator[(BlockId, InputStream)] with TempFileManager with Logging { + extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging { import ShuffleBlockFetcherIterator._ @@ -150,7 +151,7 @@ final class ShuffleBlockFetcherIterator( * deleted when cleanup. This is a layer of defensiveness against disk file leaks. */ @GuardedBy("this") - private[this] val shuffleFilesSet = mutable.HashSet[File]() + private[this] val shuffleFilesSet = mutable.HashSet[DownloadFile]() initialize() @@ -164,11 +165,15 @@ final class ShuffleBlockFetcherIterator( currentResult = null } - override def createTempFile(): File = { - blockManager.diskBlockManager.createTempLocalBlock()._2 + override def createTempFile(transportConf: TransportConf): DownloadFile = { + // we never need to do any encryption or decryption here, regardless of configs, because that + // is handled at another layer in the code. When encryption is enabled, shuffle data is written + // to disk encrypted in the first place, and sent over the network still encrypted. + new SimpleDownloadFile( + blockManager.diskBlockManager.createTempLocalBlock()._2, transportConf) } - override def registerTempFileToClean(file: File): Boolean = synchronized { + override def registerTempFileToClean(file: DownloadFile): Boolean = synchronized { if (isZombie) { false } else { @@ -204,7 +209,7 @@ final class ShuffleBlockFetcherIterator( } shuffleFilesSet.foreach { file => if (!file.delete()) { - logWarning("Failed to cleanup shuffle fetch temp file " + file.getAbsolutePath()) + logWarning("Failed to cleanup shuffle fetch temp file " + file.path()) } } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index dbee1f60d7af..32d6e8b94e1a 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -43,7 +43,7 @@ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap} -import org.apache.spark.network.shuffle.{BlockFetchingListener, TempFileManager} +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} import org.apache.spark.network.util.TransportConf import org.apache.spark.rpc.RpcEnv @@ -1437,7 +1437,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE class MockBlockTransferService(val maxFailures: Int) extends BlockTransferService { var numCalls = 0 - var tempFileManager: TempFileManager = null + var tempFileManager: DownloadFileManager = null override def init(blockDataManager: BlockDataManager): Unit = {} @@ -1447,7 +1447,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE execId: String, blockIds: Array[String], listener: BlockFetchingListener, - tempFileManager: TempFileManager): Unit = { + tempFileManager: DownloadFileManager): Unit = { listener.onBlockFetchSuccess("mockBlockId", new NioManagedBuffer(ByteBuffer.allocate(1))) } @@ -1474,7 +1474,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE port: Int, execId: String, blockId: String, - tempFileManager: TempFileManager): ManagedBuffer = { + tempFileManager: DownloadFileManager): ManagedBuffer = { numCalls += 1 this.tempFileManager = tempFileManager if (numCalls <= maxFailures) { diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index a2997dbd1b1a..b268195e09a5 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -33,7 +33,7 @@ import org.scalatest.PrivateMethodTester import org.apache.spark.{SparkFunSuite, TaskContext} import org.apache.spark.network._ import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} -import org.apache.spark.network.shuffle.{BlockFetchingListener, TempFileManager} +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} import org.apache.spark.network.util.LimitedInputStream import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util.Utils @@ -478,12 +478,12 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val remoteBlocks = Map[BlockId, ManagedBuffer]( ShuffleBlockId(0, 0, 0) -> createMockManagedBuffer()) val transfer = mock(classOf[BlockTransferService]) - var tempFileManager: TempFileManager = null + var tempFileManager: DownloadFileManager = null when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) .thenAnswer(new Answer[Unit] { override def answer(invocation: InvocationOnMock): Unit = { val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - tempFileManager = invocation.getArguments()(5).asInstanceOf[TempFileManager] + tempFileManager = invocation.getArguments()(5).asInstanceOf[DownloadFileManager] Future { listener.onBlockFetchSuccess( ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0))) From c40375190d8d6edaf52a20cbca74107f9e1908f6 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 18 Sep 2018 10:10:20 +0800 Subject: [PATCH 062/879] [SPARK-25443][BUILD] fix issues when building docs with release scripts in docker ## What changes were proposed in this pull request? These 2 changes are required to build the docs for Spark 2.4.0 RC1: 1. install `mkdocs` in the docker image 2. set locale to C.UTF-8. Otherwise jekyll fails to build the doc. ## How was this patch tested? tested manually when doing the 2.4.0 RC1 Closes #22438 from cloud-fan/infra. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit 0f1413e320bbf9804dac1b00d56f30bc20dc36a6) Signed-off-by: Wenchen Fan --- dev/create-release/release-build.sh | 5 ++--- dev/create-release/spark-rm/Dockerfile | 2 +- 2 files changed, 3 insertions(+), 4 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 098aa5745e34..4753c29b0387 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -76,9 +76,8 @@ for env in ASF_USERNAME GPG_PASSPHRASE GPG_KEY; do fi done -# Explicitly set locale in order to make `sort` output consistent across machines. -# See https://stackoverflow.com/questions/28881 for more details. -export LC_ALL=C +export LC_ALL=C.UTF-8 +export LANG=C.UTF-8 # Commit ref to checkout when building GIT_REF=${GIT_REF:-master} diff --git a/dev/create-release/spark-rm/Dockerfile b/dev/create-release/spark-rm/Dockerfile index 07ce320177f5..15f831cf06a6 100644 --- a/dev/create-release/spark-rm/Dockerfile +++ b/dev/create-release/spark-rm/Dockerfile @@ -73,7 +73,7 @@ RUN echo 'deb http://cran.cnr.Berkeley.edu/bin/linux/ubuntu xenial/' >> /etc/apt Rscript -e "install.packages(c('curl', 'xml2', 'httr', 'devtools', 'testthat', 'knitr', 'rmarkdown', 'roxygen2', 'e1071', 'survival'), repos='http://cran.us.r-project.org/')" && \ Rscript -e "devtools::install_github('jimhester/lintr')" && \ # Install tools needed to build the documentation. - $APT_INSTALL ruby2.3 ruby2.3-dev && \ + $APT_INSTALL ruby2.3 ruby2.3-dev mkdocs && \ gem install jekyll --no-rdoc --no-ri && \ gem install jekyll-redirect-from && \ gem install pygments.rb From ffd448bb0a42d3ac9bf0d7d8df543c53a03f2532 Mon Sep 17 00:00:00 2001 From: James Thompson Date: Mon, 17 Sep 2018 23:19:04 -0700 Subject: [PATCH 063/879] [SPARK-24151][SQL] Case insensitive resolution of CURRENT_DATE and CURRENT_TIMESTAMP ## What changes were proposed in this pull request? SPARK-22333 introduced a regression in the resolution of `CURRENT_DATE` and `CURRENT_TIMESTAMP`. Before that ticket, these 2 functions were resolved in a case insensitive way. After, this depends on the value of `spark.sql.caseSensitive`. The PR restores the previous behavior and makes their resolution case insensitive anyhow. The PR takes over #21217, therefore it closes #21217 and credit for this patch should be given to jamesthomp. ## How was this patch tested? added UT Closes #22440 from mgaido91/SPARK-24151. Lead-authored-by: James Thompson Co-authored-by: Marco Gaido Signed-off-by: gatorsmile (cherry picked from commit ba838fee001d553e30d2205337c1fa5ccbd57caf) Signed-off-by: gatorsmile --- docs/sql-programming-guide.md | 1 + .../spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 18 ++++++++++++++++++ 3 files changed, 20 insertions(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index e262987ab23d..f25415c0bc74 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1879,6 +1879,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see ## Upgrading From Spark SQL 2.3 to 2.4 + - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 580133dd971b..e3b17121bf35 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -1045,7 +1045,7 @@ class Analyzer( // support CURRENT_DATE and CURRENT_TIMESTAMP val literalFunctions = Seq(CurrentDate(), CurrentTimestamp()) val name = nameParts.head - val func = literalFunctions.find(e => resolver(e.prettyName, name)) + val func = literalFunctions.find(e => caseInsensitiveResolution(e.prettyName, name)) func.map(wrapper) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 3b3edac0a314..f9facbb71a4e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -32,6 +32,8 @@ import org.apache.spark.sql.catalyst.plans.{Cross, Inner} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -586,4 +588,20 @@ class AnalysisSuite extends AnalysisTest with Matchers { listRelation.select(MultiAlias(MultiAlias( PosExplode('list), Seq("first_pos", "first_val")), Seq("second_pos", "second_val")))) } + + test("SPARK-24151: CURRENT_DATE, CURRENT_TIMESTAMP should be case insensitive") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val input = Project(Seq( + UnresolvedAttribute("current_date"), + UnresolvedAttribute("CURRENT_DATE"), + UnresolvedAttribute("CURRENT_TIMESTAMP"), + UnresolvedAttribute("current_timestamp")), testRelation) + val expected = Project(Seq( + Alias(CurrentDate(), toPrettySQL(CurrentDate()))(), + Alias(CurrentDate(), toPrettySQL(CurrentDate()))(), + Alias(CurrentTimestamp(), toPrettySQL(CurrentTimestamp()))(), + Alias(CurrentTimestamp(), toPrettySQL(CurrentTimestamp()))()), testRelation).analyze + checkAnalysis(input, expected) + } + } } From cc3fbea8b6945a144d1b6426572fae34a7fe1343 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 18 Sep 2018 22:29:00 +0800 Subject: [PATCH 064/879] [SPARK-25445][BUILD] the release script should be able to publish a scala-2.12 build ## What changes were proposed in this pull request? update the package and publish steps, to support scala 2.12 ## How was this patch tested? manual test Closes #22441 from cloud-fan/scala. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit 1c0423b28705eb96237c0cb4e90f49305c64a997) Signed-off-by: Wenchen Fan --- dev/create-release/release-build.sh | 49 +++++++++++++++++++++-------- 1 file changed, 36 insertions(+), 13 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 4753c29b0387..4c90a772104f 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -111,13 +111,17 @@ fi # different versions of Scala are supported. BASE_PROFILES="-Pmesos -Pyarn" PUBLISH_SCALA_2_10=0 +PUBLISH_SCALA_2_12=0 SCALA_2_10_PROFILES="-Pscala-2.10" SCALA_2_11_PROFILES= -SCALA_2_12_PROFILES="-Pscala-2.12" +SCALA_2_12_PROFILES="-Pscala-2.12 -Pkafka-0-8" if [[ $SPARK_VERSION > "2.3" ]]; then BASE_PROFILES="$BASE_PROFILES -Pkubernetes -Pflume" SCALA_2_11_PROFILES="-Pkafka-0-8" + if [[ $SPARK_VERSION > "2.4" ]]; then + PUBLISH_SCALA_2_12=1 + fi else PUBLISH_SCALA_2_10=1 fi @@ -186,8 +190,17 @@ if [[ "$1" == "package" ]]; then # Updated for each binary build make_binary_release() { NAME=$1 - FLAGS="$MVN_EXTRA_OPTS -B $SCALA_2_11_PROFILES $BASE_RELEASE_PROFILES $2" - BUILD_PACKAGE=$3 + SCALA_VERSION=$2 + SCALA_PROFILES= + if [[ SCALA_VERSION == "2.10" ]]; then + SCALA_PROFILES="$SCALA_2_10_PROFILES" + elif [[ SCALA_VERSION == "2.12" ]]; then + SCALA_PROFILES="$SCALA_2_12_PROFILES" + else + SCALA_PROFILES="$SCALA_2_11_PROFILES" + fi + FLAGS="$MVN_EXTRA_OPTS -B $SCALA_PROFILES $BASE_RELEASE_PROFILES $3" + BUILD_PACKAGE=$4 # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds # share the same Zinc server. @@ -197,10 +210,11 @@ if [[ "$1" == "package" ]]; then cp -r spark spark-$SPARK_VERSION-bin-$NAME cd spark-$SPARK_VERSION-bin-$NAME - # TODO There should probably be a flag to make-distribution to allow 2.12 support - #if [[ $FLAGS == *scala-2.12* ]]; then - # ./dev/change-scala-version.sh 2.12 - #fi + if [[ SCALA_VERSION == "2.10" ]]; then + ./dev/change-scala-version.sh 2.10 + elif [[ SCALA_VERSION == "2.12" ]]; then + ./dev/change-scala-version.sh 2.12 + fi export ZINC_PORT=$ZINC_PORT echo "Creating distribution: $NAME ($FLAGS)" @@ -291,11 +305,20 @@ if [[ "$1" == "package" ]]; then for key in ${!BINARY_PKGS_ARGS[@]}; do args=${BINARY_PKGS_ARGS[$key]} extra=${BINARY_PKGS_EXTRA[$key]} - if ! make_binary_release "$key" "$args" "$extra"; then + if ! make_binary_release "$key" "2.11" "$args" "$extra"; then error "Failed to build $key package. Check logs for details." fi done + if [[ $PUBLISH_SCALA_2_12 = 1 ]]; then + key="without-hadoop-scala-2.12" + args="-Phadoop-provided" + extra="" + if ! make_binary_release "$key" "2.12" "$args" "$extra"; then + error "Failed to build $key package. Check logs for details." + fi + fi + rm -rf spark-$SPARK_VERSION-bin-*/ if ! is_dry_run; then @@ -414,15 +437,15 @@ if [[ "$1" == "publish-release" ]]; then -DskipTests $PUBLISH_PROFILES $SCALA_2_10_PROFILES clean install fi - #./dev/change-scala-version.sh 2.12 - #$MVN -DzincPort=$ZINC_PORT -Dmaven.repo.local=$tmp_repo \ - # -DskipTests $SCALA_2_12_PROFILES §$PUBLISH_PROFILES clean install + if ! is_dry_run && [[ $PUBLISH_SCALA_2_12 = 1 ]]; then + ./dev/change-scala-version.sh 2.12 + $MVN -DzincPort=$((ZINC_PORT + 2)) -Dmaven.repo.local=$tmp_repo -Dscala-2.12 \ + -DskipTests $PUBLISH_PROFILES $SCALA_2_12_PROFILES clean install + fi # Clean-up Zinc nailgun process $LSOF -P |grep $ZINC_PORT | grep LISTEN | awk '{ print $2; }' | xargs kill - #./dev/change-scala-version.sh 2.11 - pushd $tmp_repo/org/apache/spark # Remove any extra files generated during install From 8a2992ed4f80034a0adae3f27b49a3c68d8a5fde Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 18 Sep 2018 10:38:55 -0500 Subject: [PATCH 065/879] [SPARK-19550][DOC][FOLLOW-UP] Update tuning.md to use JDK8 ## What changes were proposed in this pull request? Update `tuning.md` and `rdd-programming-guide.md` to use JDK8. ## How was this patch tested? manual tests Closes #22446 from wangyum/java8. Authored-by: Yuming Wang Signed-off-by: Sean Owen (cherry picked from commit 182da81e9e75ac1658a39014beb90e60495bf544) Signed-off-by: Sean Owen --- docs/rdd-programming-guide.md | 4 ++-- docs/tuning.md | 8 ++++---- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md index d95b757f3685..005425754c64 100644 --- a/docs/rdd-programming-guide.md +++ b/docs/rdd-programming-guide.md @@ -859,7 +859,7 @@ We could also use `counts.sortByKey()`, for example, to sort the pairs alphabeti **Note:** when using custom objects as the key in key-value pair operations, you must be sure that a custom `equals()` method is accompanied with a matching `hashCode()` method. For full details, see the contract outlined in the [Object.hashCode() -documentation](http://docs.oracle.com/javase/7/docs/api/java/lang/Object.html#hashCode()). +documentation](https://docs.oracle.com/javase/8/docs/api/java/lang/Object.html#hashCode--). @@ -896,7 +896,7 @@ We could also use `counts.sortByKey()`, for example, to sort the pairs alphabeti **Note:** when using custom objects as the key in key-value pair operations, you must be sure that a custom `equals()` method is accompanied with a matching `hashCode()` method. For full details, see the contract outlined in the [Object.hashCode() -documentation](http://docs.oracle.com/javase/7/docs/api/java/lang/Object.html#hashCode()). +documentation](https://docs.oracle.com/javase/8/docs/api/java/lang/Object.html#hashCode--). diff --git a/docs/tuning.md b/docs/tuning.md index f60971aa2e0a..cd0f9cd08136 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -26,12 +26,12 @@ Often, this will be the first thing you should tune to optimize a Spark applicat Spark aims to strike a balance between convenience (allowing you to work with any Java type in your operations) and performance. It provides two serialization libraries: -* [Java serialization](http://docs.oracle.com/javase/6/docs/api/java/io/Serializable.html): +* [Java serialization](https://docs.oracle.com/javase/8/docs/api/java/io/Serializable.html): By default, Spark serializes objects using Java's `ObjectOutputStream` framework, and can work with any class you create that implements - [`java.io.Serializable`](http://docs.oracle.com/javase/6/docs/api/java/io/Serializable.html). + [`java.io.Serializable`](https://docs.oracle.com/javase/8/docs/api/java/io/Serializable.html). You can also control the performance of your serialization more closely by extending - [`java.io.Externalizable`](http://docs.oracle.com/javase/6/docs/api/java/io/Externalizable.html). + [`java.io.Externalizable`](https://docs.oracle.com/javase/8/docs/api/java/io/Externalizable.html). Java serialization is flexible but often quite slow, and leads to large serialized formats for many classes. * [Kryo serialization](https://github.com/EsotericSoftware/kryo): Spark can also use @@ -230,7 +230,7 @@ temporary objects created during task execution. Some steps which may be useful * Monitor how the frequency and time taken by garbage collection changes with the new settings. Our experience suggests that the effect of GC tuning depends on your application and the amount of memory available. -There are [many more tuning options](http://www.oracle.com/technetwork/java/javase/gc-tuning-6-140523.html) described online, +There are [many more tuning options](https://docs.oracle.com/javase/8/docs/technotes/guides/vm/gctuning/index.html) described online, but at a high level, managing how frequently full GC takes place can help in reducing the overhead. GC tuning flags for executors can be specified by setting `spark.executor.extraJavaOptions` in From 67f2cb6e0e9a0f19776eb6fca068dd4ca8e3fb24 Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Tue, 18 Sep 2018 11:43:35 -0700 Subject: [PATCH 066/879] [SPARK-25291][K8S] Fixing Flakiness of Executor Pod tests ## What changes were proposed in this pull request? Added fix to flakiness that was present in PySpark tests w.r.t Executors not being tested. Important fix to executorConf which was failing tests when executors *were* tested ## How was this patch tested? Unit and Integration tests Closes #22415 from ifilonenko/SPARK-25291. Authored-by: Ilan Filonenko Signed-off-by: Yinan Li (cherry picked from commit 123f0041d534f28e14343aafb4e5cec19dde14ad) Signed-off-by: Yinan Li --- .../k8s/integrationtest/KubernetesSuite.scala | 35 +++++++++++++------ .../KubernetesTestComponents.scala | 1 - .../integrationtest/SecretsTestsSuite.scala | 2 +- 3 files changed, 26 insertions(+), 12 deletions(-) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index 896a83a5badb..ba38f144d04c 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -23,7 +23,10 @@ import java.util.regex.Pattern import com.google.common.io.PatternFilenameFilter import io.fabric8.kubernetes.api.model.Pod +import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} +import io.fabric8.kubernetes.client.Watcher.Action import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, Tag} +import org.scalatest.Matchers import org.scalatest.concurrent.{Eventually, PatienceConfiguration} import org.scalatest.time.{Minutes, Seconds, Span} import scala.collection.JavaConverters._ @@ -31,10 +34,12 @@ import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.k8s.integrationtest.TestConfig._ import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory} +import org.apache.spark.internal.Logging private[spark] class KubernetesSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SecretsTestsSuite - with PythonTestsSuite with ClientModeTestsSuite { + with PythonTestsSuite with ClientModeTestsSuite + with Logging with Eventually with Matchers { import KubernetesSuite._ @@ -207,17 +212,28 @@ private[spark] class KubernetesSuite extends SparkFunSuite .getItems .get(0) driverPodChecker(driverPod) - - val executorPods = kubernetesTestComponents.kubernetesClient + val execPods = scala.collection.mutable.Map[String, Pod]() + val execWatcher = kubernetesTestComponents.kubernetesClient .pods() .withLabel("spark-app-locator", appLocator) .withLabel("spark-role", "executor") - .list() - .getItems - executorPods.asScala.foreach { pod => - executorPodChecker(pod) - } - + .watch(new Watcher[Pod] { + logInfo("Beginning watch of executors") + override def onClose(cause: KubernetesClientException): Unit = + logInfo("Ending watch of executors") + override def eventReceived(action: Watcher.Action, resource: Pod): Unit = { + val name = resource.getMetadata.getName + action match { + case Action.ADDED | Action.MODIFIED => + execPods(name) = resource + case Action.DELETED | Action.ERROR => + execPods.remove(name) + } + } + }) + Eventually.eventually(TIMEOUT, INTERVAL) { execPods.values.nonEmpty should be (true) } + execWatcher.close() + execPods.values.foreach(executorPodChecker(_)) Eventually.eventually(TIMEOUT, INTERVAL) { expectedLogOnCompletion.foreach { e => assert(kubernetesTestComponents.kubernetesClient @@ -228,7 +244,6 @@ private[spark] class KubernetesSuite extends SparkFunSuite } } } - protected def doBasicDriverPodCheck(driverPod: Pod): Unit = { assert(driverPod.getMetadata.getName === driverPodName) assert(driverPod.getSpec.getContainers.get(0).getImage === image) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index b602fdf39731..5615d6173eeb 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -62,7 +62,6 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl new SparkAppConf() .set("spark.master", s"k8s://${kubernetesClient.getMasterUrl}") .set("spark.kubernetes.namespace", namespace) - .set("spark.executor.memory", "500m") .set("spark.executor.cores", "1") .set("spark.executors.instances", "1") .set("spark.app.name", "spark-test-app") diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala index 7b05c1355ca2..b18a6aebda49 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.deploy.k8s.integrationtest import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{Pod, Secret, SecretBuilder} +import io.fabric8.kubernetes.api.model.{Pod, SecretBuilder} import org.apache.commons.codec.binary.Base64 import org.apache.commons.io.output.ByteArrayOutputStream import org.scalatest.concurrent.Eventually From 76514a015168de8d8b54b3abf6b835050eefd8c2 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 18 Sep 2018 16:33:37 -0500 Subject: [PATCH 067/879] [SPARK-25456][SQL][TEST] Fix PythonForeachWriterSuite PythonForeachWriterSuite was failing because RowQueue now needs to have a handle on a SparkEnv with a SerializerManager, so added a mock env with a serializer manager. Also fixed a typo in the `finally` that was hiding the real exception. Tested PythonForeachWriterSuite locally, full tests via jenkins. Closes #22452 from squito/SPARK-25456. Authored-by: Imran Rashid Signed-off-by: Imran Rashid (cherry picked from commit a6f37b0742d87d5c8ee3e134999d665e5719e822) Signed-off-by: Imran Rashid --- .../python/PythonForeachWriterSuite.scala | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala index 07e603477012..d02014c0dee5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/PythonForeachWriterSuite.scala @@ -19,17 +19,20 @@ package org.apache.spark.sql.execution.python import scala.collection.mutable.ArrayBuffer +import org.mockito.Mockito.when import org.scalatest.concurrent.Eventually +import org.scalatest.mockito.MockitoSugar import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.memory.{TaskMemoryManager, TestMemoryManager} +import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.python.PythonForeachWriter.UnsafeRowBuffer import org.apache.spark.sql.types.{DataType, IntegerType} import org.apache.spark.util.Utils -class PythonForeachWriterSuite extends SparkFunSuite with Eventually { +class PythonForeachWriterSuite extends SparkFunSuite with Eventually with MockitoSugar { testWithBuffer("UnsafeRowBuffer: iterator blocks when no data is available") { b => b.assertIteratorBlocked() @@ -75,7 +78,7 @@ class PythonForeachWriterSuite extends SparkFunSuite with Eventually { tester = new BufferTester(memBytes, sleepPerRowReadMs) f(tester) } finally { - if (tester == null) tester.close() + if (tester != null) tester.close() } } } @@ -83,7 +86,12 @@ class PythonForeachWriterSuite extends SparkFunSuite with Eventually { class BufferTester(memBytes: Long, sleepPerRowReadMs: Int) { private val buffer = { - val mem = new TestMemoryManager(new SparkConf()) + val mockEnv = mock[SparkEnv] + val conf = new SparkConf() + val serializerManager = new SerializerManager(new JavaSerializer(conf), conf, None) + when(mockEnv.serializerManager).thenReturn(serializerManager) + SparkEnv.set(mockEnv) + val mem = new TestMemoryManager(conf) mem.limit(memBytes) val taskM = new TaskMemoryManager(mem, 0) new UnsafeRowBuffer(taskM, Utils.createTempDir(), 1) From ba8560a96d39d6d5a3ccbb7e364e817603d8ca6c Mon Sep 17 00:00:00 2001 From: Santiago Saavedra Date: Tue, 18 Sep 2018 22:08:50 -0700 Subject: [PATCH 068/879] [SPARK-23200] Reset Kubernetes-specific config on Checkpoint restore Several configuration parameters related to Kubernetes need to be reset, as they are changed with each invokation of spark-submit and thus prevents recovery of Spark Streaming tasks. ## What changes were proposed in this pull request? When using the Kubernetes cluster-manager and spawning a Streaming workload, it is important to reset many spark.kubernetes.* properties that are generated by spark-submit but which would get rewritten when restoring a Checkpoint. This is so, because the spark-submit codepath creates Kubernetes resources, such as a ConfigMap, a Secret and other variables, which have an autogenerated name and the previous one will not resolve anymore. In short, this change enables checkpoint restoration for streaming workloads, and thus enables Spark Streaming workloads in Kubernetes, which were not possible to restore from a checkpoint before if the workload went down. ## How was this patch tested? This patch needs would benefit from testing in different k8s clusters. This is similar to the YARN related code for resetting a Spark Streaming workload, but for the Kubernetes scheduler. This PR removes the initcontainers properties that existed before because they are now removed in master. For a previous discussion, see the non-rebased work at: apache-spark-on-k8s#516 Closes #22392 from ssaavedra/fix-checkpointing-master. Authored-by: Santiago Saavedra Signed-off-by: Yinan Li (cherry picked from commit 497f00f62b3ddd1f40507fdfe10f30cd9effb6cf) Signed-off-by: Yinan Li --- .../main/scala/org/apache/spark/streaming/Checkpoint.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 3703a87cdb9a..a882558551e3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -54,6 +54,8 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) "spark.driver.bindAddress", "spark.driver.port", "spark.master", + "spark.kubernetes.driver.pod.name", + "spark.kubernetes.executor.podNamePrefix", "spark.yarn.jars", "spark.yarn.keytab", "spark.yarn.principal", @@ -64,6 +66,8 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) .remove("spark.driver.host") .remove("spark.driver.bindAddress") .remove("spark.driver.port") + .remove("spark.kubernetes.driver.pod.name") + .remove("spark.kubernetes.executor.podNamePrefix") val newReloadConf = new SparkConf(loadDefaults = true) propertiesToReload.foreach { prop => newReloadConf.getOption(prop).foreach { value => From 00ede120e6e60055c36f6fd4177d5756fbb465cc Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 18 Sep 2018 22:39:29 -0700 Subject: [PATCH 069/879] [SPARK-23173][SQL] rename spark.sql.fromJsonForceNullableSchema ## What changes were proposed in this pull request? `spark.sql.fromJsonForceNullableSchema` -> `spark.sql.function.fromJson.forceNullable` ## How was this patch tested? Made sure there are no more references to `spark.sql.fromJsonForceNullableSchema`. Closes #22459 from rxin/SPARK-23173. Authored-by: Reynold Xin Signed-off-by: gatorsmile (cherry picked from commit 6c7db7fd1ced1d143b1389d09990a620fc16be46) Signed-off-by: gatorsmile --- .../catalyst/expressions/jsonExpressions.scala | 4 ++-- .../org/apache/spark/sql/internal/SQLConf.scala | 16 ++++++++-------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index bd9090a07471..ade10ab044ae 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -517,12 +517,12 @@ case class JsonToStructs( timeZoneId: Option[String] = None) extends UnaryExpression with TimeZoneAwareExpression with CodegenFallback with ExpectsInputTypes { - val forceNullableSchema = SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA) + val forceNullableSchema: Boolean = SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA) // The JSON input data might be missing certain fields. We force the nullability // of the user-provided schema to avoid data corruptions. In particular, the parquet-mr encoder // can generate incorrect files if values are missing in columns declared as non-nullable. - val nullableSchema = if (forceNullableSchema) schema.asNullable else schema + val nullableSchema: DataType = if (forceNullableSchema) schema.asNullable else schema override def nullable: Boolean = true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 738d8fee891d..d98ea1073a27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -604,14 +604,6 @@ object SQLConf { .stringConf .createWithDefault("_corrupt_record") - val FROM_JSON_FORCE_NULLABLE_SCHEMA = buildConf("spark.sql.fromJsonForceNullableSchema") - .internal() - .doc("When true, force the output schema of the from_json() function to be nullable " + - "(including all the fields). Otherwise, the schema might not be compatible with" + - "actual data, which leads to curruptions.") - .booleanConf - .createWithDefault(true) - val BROADCAST_TIMEOUT = buildConf("spark.sql.broadcastTimeout") .doc("Timeout in seconds for the broadcast wait time in broadcast joins.") .timeConf(TimeUnit.SECONDS) @@ -1334,6 +1326,14 @@ object SQLConf { "When this conf is not set, the value from `spark.redaction.string.regex` is used.") .fallbackConf(org.apache.spark.internal.config.STRING_REDACTION_PATTERN) + val FROM_JSON_FORCE_NULLABLE_SCHEMA = buildConf("spark.sql.function.fromJson.forceNullable") + .internal() + .doc("When true, force the output schema of the from_json() function to be nullable " + + "(including all the fields). Otherwise, the schema might not be compatible with" + + "actual data, which leads to corruptions.") + .booleanConf + .createWithDefault(true) + val CONCAT_BINARY_AS_STRING = buildConf("spark.sql.function.concatBinaryAsString") .doc("When this option is set to false and all inputs are binary, `functions.concat` returns " + "an output as binary. Otherwise, it returns as a string. ") From f11f44548903bbab7ab764574d6bed326cf4cd8d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 18 Sep 2018 22:41:27 -0700 Subject: [PATCH 070/879] [SPARK-24626] Add statistics prefix to parallelFileListingInStatsComputation ## What changes were proposed in this pull request? To be more consistent with other statistics based configs. ## How was this patch tested? N/A - straightforward rename of config option. Used `git grep` to make sure there are no mention of it. Closes #22457 from rxin/SPARK-24626. Authored-by: Reynold Xin Signed-off-by: gatorsmile (cherry picked from commit 4193c7623b92765adaee539e723328ddc9048c09) Signed-off-by: gatorsmile --- docs/sql-programming-guide.md | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 63 +++++++++---------- 2 files changed, 32 insertions(+), 33 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index f25415c0bc74..2fa29a00e74c 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1896,7 +1896,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. - - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.parallelFileListingInStatsComputation.enabled` to `False`. + - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index d98ea1073a27..5e2ac02cd5c3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -249,22 +249,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val ENABLE_FALL_BACK_TO_HDFS_FOR_STATS = - buildConf("spark.sql.statistics.fallBackToHdfs") - .doc("If the table statistics are not available from table metadata enable fall back to hdfs." + - " This is useful in determining if a table is small enough to use auto broadcast joins.") - .booleanConf - .createWithDefault(false) - - val DEFAULT_SIZE_IN_BYTES = buildConf("spark.sql.defaultSizeInBytes") - .internal() - .doc("The default table size used in query planning. By default, it is set to Long.MaxValue " + - "which is larger than `spark.sql.autoBroadcastJoinThreshold` to be more conservative. " + - "That is to say by default the optimizer will not choose to broadcast a table unless it " + - "knows for sure its size is small enough.") - .longConf - .createWithDefault(Long.MaxValue) - val SHUFFLE_PARTITIONS = buildConf("spark.sql.shuffle.partitions") .doc("The default number of partitions to use when shuffling data for joins or aggregations.") .intConf @@ -1090,6 +1074,30 @@ object SQLConf { .internal() .stringConf + val PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION = + buildConf("spark.sql.statistics.parallelFileListingInStatsComputation.enabled") + .internal() + .doc("When true, SQL commands use parallel file listing, " + + "as opposed to single thread listing." + + "This usually speeds up commands that need to list many directories.") + .booleanConf + .createWithDefault(true) + + val ENABLE_FALL_BACK_TO_HDFS_FOR_STATS = buildConf("spark.sql.statistics.fallBackToHdfs") + .doc("If the table statistics are not available from table metadata enable fall back to hdfs." + + " This is useful in determining if a table is small enough to use auto broadcast joins.") + .booleanConf + .createWithDefault(false) + + val DEFAULT_SIZE_IN_BYTES = buildConf("spark.sql.defaultSizeInBytes") + .internal() + .doc("The default table size used in query planning. By default, it is set to Long.MaxValue " + + "which is larger than `spark.sql.autoBroadcastJoinThreshold` to be more conservative. " + + "That is to say by default the optimizer will not choose to broadcast a table unless it " + + "knows for sure its size is small enough.") + .longConf + .createWithDefault(Long.MaxValue) + val NDV_MAX_ERROR = buildConf("spark.sql.statistics.ndv.maxError") .internal() @@ -1533,15 +1541,6 @@ object SQLConf { "are performed before any UNION, EXCEPT and MINUS operations.") .booleanConf .createWithDefault(false) - - val PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION = - buildConf("spark.sql.parallelFileListingInStatsComputation.enabled") - .internal() - .doc("When true, SQL commands use parallel file listing, " + - "as opposed to single thread listing." + - "This usually speeds up commands that need to list many directories.") - .booleanConf - .createWithDefault(true) } /** @@ -1746,14 +1745,10 @@ class SQLConf extends Serializable with Logging { def advancedPartitionPredicatePushdownEnabled: Boolean = getConf(ADVANCED_PARTITION_PREDICATE_PUSHDOWN) - def fallBackToHdfsForStatsEnabled: Boolean = getConf(ENABLE_FALL_BACK_TO_HDFS_FOR_STATS) - def preferSortMergeJoin: Boolean = getConf(PREFER_SORTMERGEJOIN) def enableRadixSort: Boolean = getConf(RADIX_SORT_ENABLED) - def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES) - def isParquetSchemaMergingEnabled: Boolean = getConf(PARQUET_SCHEMA_MERGING_ENABLED) def isParquetSchemaRespectSummaries: Boolean = getConf(PARQUET_SCHEMA_RESPECT_SUMMARIES) @@ -1845,6 +1840,13 @@ class SQLConf extends Serializable with Logging { def sessionLocalTimeZone: String = getConf(SQLConf.SESSION_LOCAL_TIMEZONE) + def parallelFileListingInStatsComputation: Boolean = + getConf(SQLConf.PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION) + + def fallBackToHdfsForStatsEnabled: Boolean = getConf(ENABLE_FALL_BACK_TO_HDFS_FOR_STATS) + + def defaultSizeInBytes: Long = getConf(DEFAULT_SIZE_IN_BYTES) + def ndvMaxError: Double = getConf(NDV_MAX_ERROR) def histogramEnabled: Boolean = getConf(HISTOGRAM_ENABLED) @@ -1947,9 +1949,6 @@ class SQLConf extends Serializable with Logging { def setOpsPrecedenceEnforced: Boolean = getConf(SQLConf.LEGACY_SETOPS_PRECEDENCE_ENABLED) - def parallelFileListingInStatsComputation: Boolean = - getConf(SQLConf.PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION) - /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ From 538ae62e0cafc8180b04e4e5c74b79acee60d2b1 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 19 Sep 2018 18:30:46 +0800 Subject: [PATCH 071/879] [SPARK-25445][BUILD][FOLLOWUP] Resolve issues in release-build.sh for publishing scala-2.12 build ## What changes were proposed in this pull request? This is a follow up for #22441. 1. Remove flag "-Pkafka-0-8" for Scala 2.12 build. 2. Clean up the script, simpler logic. 3. Switch to Scala version to 2.11 before script exit. ## How was this patch tested? Manual test. Closes #22454 from gengliangwang/revise_release_build. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan (cherry picked from commit 5534a3a58e4025624fbad527dd129acb8025f25a) Signed-off-by: Wenchen Fan --- dev/create-release/release-build.sh | 38 ++++++++++++----------------- 1 file changed, 15 insertions(+), 23 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 4c90a772104f..cce5f8b6975c 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -111,21 +111,21 @@ fi # different versions of Scala are supported. BASE_PROFILES="-Pmesos -Pyarn" PUBLISH_SCALA_2_10=0 -PUBLISH_SCALA_2_12=0 SCALA_2_10_PROFILES="-Pscala-2.10" SCALA_2_11_PROFILES= -SCALA_2_12_PROFILES="-Pscala-2.12 -Pkafka-0-8" - if [[ $SPARK_VERSION > "2.3" ]]; then BASE_PROFILES="$BASE_PROFILES -Pkubernetes -Pflume" SCALA_2_11_PROFILES="-Pkafka-0-8" - if [[ $SPARK_VERSION > "2.4" ]]; then - PUBLISH_SCALA_2_12=1 - fi else PUBLISH_SCALA_2_10=1 fi +PUBLISH_SCALA_2_12=0 +SCALA_2_12_PROFILES="-Pscala-2.12" +if [[ $SPARK_VERSION > "2.4" ]]; then + PUBLISH_SCALA_2_12=1 +fi + # Hive-specific profiles for some builds HIVE_PROFILES="-Phive -Phive-thriftserver" # Profiles for publishing snapshots and release to Maven Central @@ -190,17 +190,9 @@ if [[ "$1" == "package" ]]; then # Updated for each binary build make_binary_release() { NAME=$1 - SCALA_VERSION=$2 - SCALA_PROFILES= - if [[ SCALA_VERSION == "2.10" ]]; then - SCALA_PROFILES="$SCALA_2_10_PROFILES" - elif [[ SCALA_VERSION == "2.12" ]]; then - SCALA_PROFILES="$SCALA_2_12_PROFILES" - else - SCALA_PROFILES="$SCALA_2_11_PROFILES" - fi - FLAGS="$MVN_EXTRA_OPTS -B $SCALA_PROFILES $BASE_RELEASE_PROFILES $3" - BUILD_PACKAGE=$4 + FLAGS="$MVN_EXTRA_OPTS -B $BASE_RELEASE_PROFILES $2" + BUILD_PACKAGE=$3 + SCALA_VERSION=$4 # We increment the Zinc port each time to avoid OOM's and other craziness if multiple builds # share the same Zinc server. @@ -210,10 +202,8 @@ if [[ "$1" == "package" ]]; then cp -r spark spark-$SPARK_VERSION-bin-$NAME cd spark-$SPARK_VERSION-bin-$NAME - if [[ SCALA_VERSION == "2.10" ]]; then - ./dev/change-scala-version.sh 2.10 - elif [[ SCALA_VERSION == "2.12" ]]; then - ./dev/change-scala-version.sh 2.12 + if [[ "$SCALA_VERSION" != "2.11" ]]; then + ./dev/change-scala-version.sh $SCALA_VERSION fi export ZINC_PORT=$ZINC_PORT @@ -305,7 +295,7 @@ if [[ "$1" == "package" ]]; then for key in ${!BINARY_PKGS_ARGS[@]}; do args=${BINARY_PKGS_ARGS[$key]} extra=${BINARY_PKGS_EXTRA[$key]} - if ! make_binary_release "$key" "2.11" "$args" "$extra"; then + if ! make_binary_release "$key" "$SCALA_2_11_PROFILES $args" "$extra" "2.11"; then error "Failed to build $key package. Check logs for details." fi done @@ -314,7 +304,7 @@ if [[ "$1" == "package" ]]; then key="without-hadoop-scala-2.12" args="-Phadoop-provided" extra="" - if ! make_binary_release "$key" "2.12" "$args" "$extra"; then + if ! make_binary_release "$key" "$SCALA_2_12_PROFILES $args" "$extra" "2.12"; then error "Failed to build $key package. Check logs for details." fi fi @@ -446,6 +436,8 @@ if [[ "$1" == "publish-release" ]]; then # Clean-up Zinc nailgun process $LSOF -P |grep $ZINC_PORT | grep LISTEN | awk '{ print $2; }' | xargs kill + ./dev/change-scala-version.sh 2.11 + pushd $tmp_repo/org/apache/spark # Remove any extra files generated during install From 9fefb47feab14b865978bdb8e6155a976de72416 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Wed, 19 Sep 2018 14:33:40 -0700 Subject: [PATCH 072/879] Revert "[SPARK-23173][SQL] rename spark.sql.fromJsonForceNullableSchema" This reverts commit 6c7db7fd1ced1d143b1389d09990a620fc16be46. (cherry picked from commit cb1b55cf771018f1560f6b173cdd7c6ca8061bc7) Signed-off-by: Dongjoon Hyun --- .../catalyst/expressions/jsonExpressions.scala | 4 ++-- .../org/apache/spark/sql/internal/SQLConf.scala | 16 ++++++++-------- 2 files changed, 10 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index ade10ab044ae..bd9090a07471 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -517,12 +517,12 @@ case class JsonToStructs( timeZoneId: Option[String] = None) extends UnaryExpression with TimeZoneAwareExpression with CodegenFallback with ExpectsInputTypes { - val forceNullableSchema: Boolean = SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA) + val forceNullableSchema = SQLConf.get.getConf(SQLConf.FROM_JSON_FORCE_NULLABLE_SCHEMA) // The JSON input data might be missing certain fields. We force the nullability // of the user-provided schema to avoid data corruptions. In particular, the parquet-mr encoder // can generate incorrect files if values are missing in columns declared as non-nullable. - val nullableSchema: DataType = if (forceNullableSchema) schema.asNullable else schema + val nullableSchema = if (forceNullableSchema) schema.asNullable else schema override def nullable: Boolean = true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 5e2ac02cd5c3..3e9cde4bc252 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -588,6 +588,14 @@ object SQLConf { .stringConf .createWithDefault("_corrupt_record") + val FROM_JSON_FORCE_NULLABLE_SCHEMA = buildConf("spark.sql.fromJsonForceNullableSchema") + .internal() + .doc("When true, force the output schema of the from_json() function to be nullable " + + "(including all the fields). Otherwise, the schema might not be compatible with" + + "actual data, which leads to curruptions.") + .booleanConf + .createWithDefault(true) + val BROADCAST_TIMEOUT = buildConf("spark.sql.broadcastTimeout") .doc("Timeout in seconds for the broadcast wait time in broadcast joins.") .timeConf(TimeUnit.SECONDS) @@ -1334,14 +1342,6 @@ object SQLConf { "When this conf is not set, the value from `spark.redaction.string.regex` is used.") .fallbackConf(org.apache.spark.internal.config.STRING_REDACTION_PATTERN) - val FROM_JSON_FORCE_NULLABLE_SCHEMA = buildConf("spark.sql.function.fromJson.forceNullable") - .internal() - .doc("When true, force the output schema of the from_json() function to be nullable " + - "(including all the fields). Otherwise, the schema might not be compatible with" + - "actual data, which leads to corruptions.") - .booleanConf - .createWithDefault(true) - val CONCAT_BINARY_AS_STRING = buildConf("spark.sql.function.concatBinaryAsString") .doc("When this option is set to false and all inputs are binary, `functions.concat` returns " + "an output as binary. Otherwise, it returns as a string. ") From 83a75a83cb24d20d4c2df5389bb8db34ad0335d9 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Wed, 19 Sep 2018 15:16:20 -0700 Subject: [PATCH 073/879] [SPARK-22666][ML][FOLLOW-UP] Improve testcase to tolerate different schema representation ## What changes were proposed in this pull request? Improve testcase "image datasource test: read non image" to tolerate different schema representation. Because file:/path and file:///path are both valid URI-ifications so in some environment the testcase will fail. ## How was this patch tested? Manual. Closes #22449 from WeichenXu123/image_url. Authored-by: WeichenXu Signed-off-by: Xiangrui Meng (cherry picked from commit 6f681d42964884d19bf22deb614550d712223117) Signed-off-by: Xiangrui Meng --- .../spark/ml/source/image/ImageFileFormatSuite.scala | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala index 1a6a8d67d8d6..38e25131df86 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.ml.source.image +import java.net.URI import java.nio.file.Paths import org.apache.spark.SparkFunSuite @@ -58,8 +59,14 @@ class ImageFileFormatSuite extends SparkFunSuite with MLlibTestSparkContext { .load(filePath) assert(df2.count() === 1) val result = df2.head() - assert(result === invalidImageRow( - Paths.get(filePath).toAbsolutePath().normalize().toUri().toString)) + + val resultOrigin = result.getStruct(0).getString(0) + // covert `origin` to `java.net.URI` object and then compare. + // because `file:/path` and `file:///path` are both valid URI-ifications + assert(new URI(resultOrigin) === Paths.get(filePath).toAbsolutePath().normalize().toUri()) + + // Compare other columns in the row to be the same with the `invalidImageRow` + assert(result === invalidImageRow(resultOrigin)) } test("image datasource partition test") { From 9031c784847353051bc0978f63ef4146ae9095ff Mon Sep 17 00:00:00 2001 From: Ilan Filonenko Date: Wed, 19 Sep 2018 15:37:56 -0700 Subject: [PATCH 074/879] [SPARK-25021][K8S][BACKPORT] Add spark.executor.pyspark.memory limit for K8S ## What changes were proposed in this pull request? Add spark.executor.pyspark.memory limit for K8S [BACKPORT] ## How was this patch tested? Unit and Integration tests Closes #22376 from ifilonenko/SPARK-25021-2.4. Authored-by: Ilan Filonenko Signed-off-by: Holden Karau --- dev/make-distribution.sh | 1 + docs/configuration.md | 2 +- .../org/apache/spark/deploy/k8s/Config.scala | 7 +++ .../features/BasicExecutorFeatureStep.scala | 14 +++++- .../bindings/JavaDriverFeatureStep.scala | 4 +- .../bindings/PythonDriverFeatureStep.scala | 4 +- .../bindings/RDriverFeatureStep.scala | 4 +- .../BasicDriverFeatureStepSuite.scala | 1 - .../BasicExecutorFeatureStepSuite.scala | 24 ++++++++++ .../bindings/JavaDriverFeatureStepSuite.scala | 1 - .../src/main/dockerfiles/spark/Dockerfile | 1 + .../dockerfiles/spark/bindings/R/Dockerfile | 2 +- .../spark/bindings/python/Dockerfile | 2 +- .../k8s/integrationtest/KubernetesSuite.scala | 33 +++++++++++++ .../integrationtest/PythonTestsSuite.scala | 34 +++++++++++--- .../tests}/py_container_checks.py | 0 .../integration-tests/tests}/pyfiles.py | 0 .../tests/worker_memory_check.py | 47 +++++++++++++++++++ 18 files changed, 165 insertions(+), 16 deletions(-) rename {examples/src/main/python => resource-managers/kubernetes/integration-tests/tests}/py_container_checks.py (100%) rename {examples/src/main/python => resource-managers/kubernetes/integration-tests/tests}/pyfiles.py (100%) create mode 100644 resource-managers/kubernetes/integration-tests/tests/worker_memory_check.py diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index 126d39dfe20e..668682fbb913 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -192,6 +192,7 @@ fi if [ -d "$SPARK_HOME"/resource-managers/kubernetes/core/target/ ]; then mkdir -p "$DISTDIR/kubernetes/" cp -a "$SPARK_HOME"/resource-managers/kubernetes/docker/src/main/dockerfiles "$DISTDIR/kubernetes/" + cp -a "$SPARK_HOME"/resource-managers/kubernetes/integration-tests/tests "$DISTDIR/kubernetes/" fi # Copy examples and dependencies diff --git a/docs/configuration.md b/docs/configuration.md index a3e59a05f7bd..782ccff66707 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -188,7 +188,7 @@ of the most common options to set are: unless otherwise specified. If set, PySpark memory for an executor will be limited to this amount. If not set, Spark will not limit Python's memory use and it is up to the application to avoid exceeding the overhead memory space - shared with other non-JVM processes. When PySpark is run in YARN, this memory + shared with other non-JVM processes. When PySpark is run in YARN or Kubernetes, this memory is added to executor resource requests. diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 1b582fe53624..c7338a721595 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -225,6 +225,13 @@ private[spark] object Config extends Logging { "Ensure that major Python version is either Python2 or Python3") .createWithDefault("2") + val APP_RESOURCE_TYPE = + ConfigBuilder("spark.kubernetes.resource.type") + .doc("This sets the resource type internally") + .internal() + .stringConf + .createOptional + val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = "spark.kubernetes.authenticate.submission" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index c37f713c56de..d89995ba5e4f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -24,7 +24,7 @@ import org.apache.spark.SparkException import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.internal.config.{EXECUTOR_CLASS_PATH, EXECUTOR_JAVA_OPTIONS, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD} +import org.apache.spark.internal.config.{EXECUTOR_CLASS_PATH, EXECUTOR_JAVA_OPTIONS, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD, PYSPARK_EXECUTOR_MEMORY} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -58,6 +58,16 @@ private[spark] class BasicExecutorFeatureStep( (kubernetesConf.get(MEMORY_OVERHEAD_FACTOR) * executorMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB + private val executorMemoryTotal = kubernetesConf.sparkConf + .getOption(APP_RESOURCE_TYPE.key).map{ res => + val additionalPySparkMemory = res match { + case "python" => + kubernetesConf.sparkConf + .get(PYSPARK_EXECUTOR_MEMORY).map(_.toInt).getOrElse(0) + case _ => 0 + } + executorMemoryWithOverhead + additionalPySparkMemory + }.getOrElse(executorMemoryWithOverhead) private val executorCores = kubernetesConf.sparkConf.getInt("spark.executor.cores", 1) private val executorCoresRequest = @@ -76,7 +86,7 @@ private[spark] class BasicExecutorFeatureStep( // executorId val hostname = name.substring(Math.max(0, name.length - 63)) val executorMemoryQuantity = new QuantityBuilder(false) - .withAmount(s"${executorMemoryWithOverhead}Mi") + .withAmount(s"${executorMemoryTotal}Mi") .build() val executorCpuQuantity = new QuantityBuilder(false) .withAmount(executorCoresRequest) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala index f52ec9fdc677..6f063b253cd7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.k8s.features.bindings import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.Config.APP_RESOURCE_TYPE import org.apache.spark.deploy.k8s.Constants.SPARK_CONF_PATH import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep import org.apache.spark.launcher.SparkLauncher @@ -38,7 +39,8 @@ private[spark] class JavaDriverFeatureStep( .build() SparkPod(pod.pod, withDriverArgs) } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty + override def getAdditionalPodSystemProperties(): Map[String, String] = + Map(APP_RESOURCE_TYPE.key -> "java") override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala index 406944a95338..cf0c03b22bd7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, HasMetadata} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s.Config.APP_RESOURCE_TYPE import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep @@ -68,7 +69,8 @@ private[spark] class PythonDriverFeatureStep( SparkPod(pod.pod, withPythonPrimaryContainer) } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty + override def getAdditionalPodSystemProperties(): Map[String, String] = + Map(APP_RESOURCE_TYPE.key -> "python") override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStep.scala index 11b09b399618..1a7ef52fefe7 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStep.scala @@ -21,6 +21,7 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, HasMetadata} import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s.Config.APP_RESOURCE_TYPE import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep @@ -54,7 +55,8 @@ private[spark] class RDriverFeatureStep( SparkPod(pod.pod, withRPrimaryContainer) } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty + override def getAdditionalPodSystemProperties(): Map[String, String] = + Map(APP_RESOURCE_TYPE.key -> "r") override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index d98e11355464..0968cce971c3 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -57,7 +57,6 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { MAIN_CLASS, APP_ARGS) - test("Check the pod respects all configurations from the user.") { val sparkConf = new SparkConf() .set(KUBERNETES_DRIVER_POD_NAME, "spark-driver-pod") diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index 95d373f79164..63b237b9dfe4 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -75,6 +75,7 @@ class BasicExecutorFeatureStepSuite .set("spark.driver.host", DRIVER_HOSTNAME) .set("spark.driver.port", DRIVER_PORT.toString) .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS.mkString(",")) + .set("spark.kubernetes.resource.type", "java") } test("basic executor pod has reasonable defaults") { @@ -161,6 +162,29 @@ class BasicExecutorFeatureStepSuite checkOwnerReferences(executor.pod, DRIVER_POD_UID) } + test("test executor pyspark memory") { + val conf = baseConf.clone() + conf.set("spark.kubernetes.resource.type", "python") + conf.set(org.apache.spark.internal.config.PYSPARK_EXECUTOR_MEMORY, 42L) + + val step = new BasicExecutorFeatureStep( + KubernetesConf( + conf, + KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), + RESOURCE_NAME_PREFIX, + APP_ID, + LABELS, + ANNOTATIONS, + Map.empty, + Map.empty, + Map.empty, + Nil, + Seq.empty[String])) + val executor = step.configurePod(SparkPod.initialPod()) + // This is checking that basic executor + executorMemory = 1408 + 42 = 1450 + assert(executor.container.getResources.getRequests.get("memory").getAmount === "1450Mi") + } + // There is always exactly one controller reference, and it points to the driver pod. private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { assert(executor.getMetadata.getOwnerReferences.size() === 1) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala index 18874afe6e53..bf552aeb8b90 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala @@ -56,6 +56,5 @@ class JavaDriverFeatureStepSuite extends SparkFunSuite { "--properties-file", SPARK_CONF_PATH, "--class", "test-class", "spark-internal", "5 7")) - } } diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index 071aa2020dd8..7ae57bf6e42d 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -43,6 +43,7 @@ COPY bin /opt/spark/bin COPY sbin /opt/spark/sbin COPY ${img_path}/spark/entrypoint.sh /opt/ COPY examples /opt/spark/examples +COPY kubernetes/tests /opt/spark/tests COPY data /opt/spark/data ENV SPARK_HOME /opt/spark diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile index e627883ba782..9f67422efeb3 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile @@ -19,10 +19,10 @@ ARG base_img FROM $base_img WORKDIR / RUN mkdir ${SPARK_HOME}/R -COPY R ${SPARK_HOME}/R RUN apk add --no-cache R R-dev +COPY R ${SPARK_HOME}/R ENV R_HOME /usr/lib/R WORKDIR /opt/spark/work-dir diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile index 72bb9620b45d..69b6efa6149a 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile @@ -19,7 +19,6 @@ ARG base_img FROM $base_img WORKDIR / RUN mkdir ${SPARK_HOME}/python -COPY python/lib ${SPARK_HOME}/python/lib # TODO: Investigate running both pip and pip3 via virtualenvs RUN apk add --no-cache python && \ apk add --no-cache python3 && \ @@ -33,6 +32,7 @@ RUN apk add --no-cache python && \ # Removed the .cache to save space rm -r /root/.cache +COPY python/lib ${SPARK_HOME}/python/lib ENV PYTHONPATH ${SPARK_HOME}/python/lib/pyspark.zip:${SPARK_HOME}/python/lib/py4j-*.zip WORKDIR /opt/spark/work-dir diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala index ba38f144d04c..e6840ce818c1 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesSuite.scala @@ -55,6 +55,17 @@ private[spark] class KubernetesSuite extends SparkFunSuite protected var containerLocalSparkDistroExamplesJar: String = _ protected var appLocator: String = _ + // Default memory limit is 1024M + 384M (minimum overhead constant) + private val baseMemory = s"${1024 + 384}Mi" + protected val memOverheadConstant = 0.8 + private val standardNonJVMMemory = s"${(1024 + 0.4*1024).toInt}Mi" + protected val additionalMemory = 200 + // 209715200 is 200Mi + protected val additionalMemoryInBytes = 209715200 + private val extraDriverTotalMemory = s"${(1024 + memOverheadConstant*1024).toInt}Mi" + private val extraExecTotalMemory = + s"${(1024 + memOverheadConstant*1024 + additionalMemory).toInt}Mi" + override def beforeAll(): Unit = { // The scalatest-maven-plugin gives system properties that are referenced but not set null // values. We need to remove the null-value properties before initializing the test backend. @@ -248,6 +259,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite assert(driverPod.getMetadata.getName === driverPodName) assert(driverPod.getSpec.getContainers.get(0).getImage === image) assert(driverPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-driver") + assert(driverPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount + === baseMemory) } @@ -255,28 +268,48 @@ private[spark] class KubernetesSuite extends SparkFunSuite assert(driverPod.getMetadata.getName === driverPodName) assert(driverPod.getSpec.getContainers.get(0).getImage === pyImage) assert(driverPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-driver") + assert(driverPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount + === standardNonJVMMemory) } protected def doBasicDriverRPodCheck(driverPod: Pod): Unit = { assert(driverPod.getMetadata.getName === driverPodName) assert(driverPod.getSpec.getContainers.get(0).getImage === rImage) assert(driverPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-driver") + assert(driverPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount + === standardNonJVMMemory) } protected def doBasicExecutorPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === image) assert(executorPod.getSpec.getContainers.get(0).getName === "executor") + assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount + === baseMemory) } protected def doBasicExecutorPyPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === pyImage) assert(executorPod.getSpec.getContainers.get(0).getName === "executor") + assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount + === standardNonJVMMemory) } protected def doBasicExecutorRPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === rImage) assert(executorPod.getSpec.getContainers.get(0).getName === "executor") + assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount + === standardNonJVMMemory) + } + + protected def doDriverMemoryCheck(driverPod: Pod): Unit = { + assert(driverPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount + === extraDriverTotalMemory) + } + + protected def doExecutorMemoryCheck(executorPod: Pod): Unit = { + assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount + === extraExecTotalMemory) } protected def checkCustomSettings(pod: Pod): Unit = { diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala index 1ebb30094dcd..06b73107ec23 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala @@ -23,9 +23,11 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => import PythonTestsSuite._ import KubernetesSuite.k8sTestTag + private val pySparkDockerImage = + s"${getTestImageRepo}/spark-py:${getTestImageTag}" test("Run PySpark on simple pi.py example", k8sTestTag) { sparkAppConf - .set("spark.kubernetes.container.image", s"${getTestImageRepo}/spark-py:${getTestImageTag}") + .set("spark.kubernetes.container.image", pySparkDockerImage) runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_PI, mainClass = "", @@ -39,7 +41,7 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => test("Run PySpark with Python2 to test a pyfiles example", k8sTestTag) { sparkAppConf - .set("spark.kubernetes.container.image", s"${getTestImageRepo}/spark-py:${getTestImageTag}") + .set("spark.kubernetes.container.image", pySparkDockerImage) .set("spark.kubernetes.pyspark.pythonVersion", "2") runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_FILES, @@ -57,7 +59,7 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => test("Run PySpark with Python3 to test a pyfiles example", k8sTestTag) { sparkAppConf - .set("spark.kubernetes.container.image", s"${getTestImageRepo}/spark-py:${getTestImageTag}") + .set("spark.kubernetes.container.image", pySparkDockerImage) .set("spark.kubernetes.pyspark.pythonVersion", "3") runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_FILES, @@ -72,12 +74,32 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => isJVM = false, pyFiles = Some(PYSPARK_CONTAINER_TESTS)) } + + test("Run PySpark with memory customization", k8sTestTag) { + sparkAppConf + .set("spark.kubernetes.container.image", pySparkDockerImage) + .set("spark.kubernetes.pyspark.pythonVersion", "3") + .set("spark.kubernetes.memoryOverheadFactor", s"$memOverheadConstant") + .set("spark.executor.pyspark.memory", s"${additionalMemory}m") + runSparkApplicationAndVerifyCompletion( + appResource = PYSPARK_MEMORY_CHECK, + mainClass = "", + expectedLogOnCompletion = Seq( + "PySpark Worker Memory Check is: True"), + appArgs = Array(s"$additionalMemoryInBytes"), + driverPodChecker = doDriverMemoryCheck, + executorPodChecker = doExecutorMemoryCheck, + appLocator = appLocator, + isJVM = false, + pyFiles = Some(PYSPARK_CONTAINER_TESTS)) + } } private[spark] object PythonTestsSuite { val CONTAINER_LOCAL_PYSPARK: String = "local:///opt/spark/examples/src/main/python/" val PYSPARK_PI: String = CONTAINER_LOCAL_PYSPARK + "pi.py" - val PYSPARK_FILES: String = CONTAINER_LOCAL_PYSPARK + "pyfiles.py" - val PYSPARK_CONTAINER_TESTS: String = CONTAINER_LOCAL_PYSPARK + "py_container_checks.py" + val TEST_LOCAL_PYSPARK: String = "local:///opt/spark/tests/" + val PYSPARK_FILES: String = TEST_LOCAL_PYSPARK + "pyfiles.py" + val PYSPARK_CONTAINER_TESTS: String = TEST_LOCAL_PYSPARK + "py_container_checks.py" + val PYSPARK_MEMORY_CHECK: String = TEST_LOCAL_PYSPARK + "worker_memory_check.py" } - diff --git a/examples/src/main/python/py_container_checks.py b/resource-managers/kubernetes/integration-tests/tests/py_container_checks.py similarity index 100% rename from examples/src/main/python/py_container_checks.py rename to resource-managers/kubernetes/integration-tests/tests/py_container_checks.py diff --git a/examples/src/main/python/pyfiles.py b/resource-managers/kubernetes/integration-tests/tests/pyfiles.py similarity index 100% rename from examples/src/main/python/pyfiles.py rename to resource-managers/kubernetes/integration-tests/tests/pyfiles.py diff --git a/resource-managers/kubernetes/integration-tests/tests/worker_memory_check.py b/resource-managers/kubernetes/integration-tests/tests/worker_memory_check.py new file mode 100644 index 000000000000..d312a29f388e --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/tests/worker_memory_check.py @@ -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. +# + +from __future__ import print_function + +import resource +import sys + +from pyspark.sql import SparkSession + + +if __name__ == "__main__": + """ + Usage: worker_memory_check [Memory_in_Mi] + """ + spark = SparkSession \ + .builder \ + .appName("PyMemoryTest") \ + .getOrCreate() + sc = spark.sparkContext + if len(sys.argv) < 2: + print("Usage: worker_memory_check [Memory_in_Mi]", file=sys.stderr) + sys.exit(-1) + + def f(x): + rLimit = resource.getrlimit(resource.RLIMIT_AS) + print("RLimit is " + str(rLimit)) + return rLimit + resourceValue = sc.parallelize([1]).map(f).collect()[0][0] + print("Resource Value is " + str(resourceValue)) + truthCheck = (resourceValue == int(sys.argv[1])) + print("PySpark Worker Memory Check is: " + str(truthCheck)) + spark.stop() From a9a8d3a4b92be89defd82d5f2eeb3f9af45c687d Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Wed, 19 Sep 2018 16:53:26 -0700 Subject: [PATCH 075/879] [SPARK-25425][SQL][BACKPORT-2.4] Extra options should override session options in DataSource V2 ## What changes were proposed in this pull request? In the PR, I propose overriding session options by extra options in DataSource V2. Extra options are more specific and set via `.option()`, and should overwrite more generic session options. ## How was this patch tested? Added tests for read and write paths. Closes #22474 from MaxGekk/session-options-2.4. Authored-by: Maxim Gekk Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/DataFrameReader.scala | 2 +- .../apache/spark/sql/DataFrameWriter.scala | 8 +++-- .../sql/sources/v2/DataSourceV2Suite.scala | 33 +++++++++++++++++++ .../sources/v2/SimpleWritableDataSource.scala | 7 +++- 4 files changed, 45 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 371ec7051879..27a1af27a644 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -202,7 +202,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { DataSourceOptions.PATHS_KEY -> objectMapper.writeValueAsString(paths.toArray) } Dataset.ofRows(sparkSession, DataSourceV2Relation.create( - ds, extraOptions.toMap ++ sessionOptions + pathsOption, + ds, sessionOptions ++ extraOptions.toMap + pathsOption, userSpecifiedSchema = userSpecifiedSchema)) } else { loadV1Source(paths: _*) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 4aeddfd96d17..80ade7c3e6ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -241,10 +241,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val source = cls.newInstance().asInstanceOf[DataSourceV2] source match { case ws: WriteSupport => - val options = extraOptions ++ - DataSourceV2Utils.extractSessionConfigs(source, df.sparkSession.sessionState.conf) + val sessionOptions = DataSourceV2Utils.extractSessionConfigs( + source, + df.sparkSession.sessionState.conf) + val options = sessionOptions ++ extraOptions + val relation = DataSourceV2Relation.create(source, options) - val relation = DataSourceV2Relation.create(source, options.toMap) if (mode == SaveMode.Append) { runCommand(df.sparkSession, "save") { AppendData.byName(relation, df.logicalPlan) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala index 12beca257a0a..bafde50bdc01 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.sources.v2 +import java.io.File import java.util.{ArrayList, List => JList} import test.org.apache.spark.sql.sources.v2._ @@ -322,6 +323,38 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { checkCanonicalizedOutput(df, 2, 2) checkCanonicalizedOutput(df.select('i), 2, 1) } + + test("SPARK-25425: extra options should override sessions options during reading") { + val prefix = "spark.datasource.userDefinedDataSource." + val optionName = "optionA" + withSQLConf(prefix + optionName -> "true") { + val df = spark + .read + .option(optionName, false) + .format(classOf[DataSourceV2WithSessionConfig].getName).load() + val options = df.queryExecution.optimizedPlan.collectFirst { + case d: DataSourceV2Relation => d.options + } + assert(options.get.get(optionName) == Some("false")) + } + } + + test("SPARK-25425: extra options should override sessions options during writing") { + withTempPath { path => + val sessionPath = path.getCanonicalPath + withSQLConf("spark.datasource.simpleWritableDataSource.path" -> sessionPath) { + withTempPath { file => + val optionPath = file.getCanonicalPath + val format = classOf[SimpleWritableDataSource].getName + + val df = Seq((1L, 2L)).toDF("i", "j") + df.write.format(format).option("path", optionPath).save() + assert(!new File(sessionPath).exists) + checkAnswer(spark.read.format(format).option("path", optionPath).load(), df) + } + } + } + } } class SimpleSinglePartitionSource extends DataSourceV2 with ReadSupport { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala index e1b8e9c44d72..654c62d8edc5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala @@ -38,10 +38,15 @@ import org.apache.spark.util.SerializableConfiguration * Each task writes data to `target/_temporary/jobId/$jobId-$partitionId-$attemptNumber`. * Each job moves files from `target/_temporary/jobId/` to `target`. */ -class SimpleWritableDataSource extends DataSourceV2 with ReadSupport with WriteSupport { +class SimpleWritableDataSource extends DataSourceV2 + with ReadSupport + with WriteSupport + with SessionConfigSupport { private val schema = new StructType().add("i", "long").add("j", "long") + override def keyPrefix: String = "simpleWritableDataSource" + class Reader(path: String, conf: Configuration) extends DataSourceReader { override def readSchema(): StructType = schema From 99ae693b3722db6e01825b8cf2c3f2ef74a65ddb Mon Sep 17 00:00:00 2001 From: Bryan Cutler Date: Thu, 20 Sep 2018 09:29:29 +0800 Subject: [PATCH 076/879] [SPARK-25471][PYTHON][TEST] Fix pyspark-sql test error when using Python 3.6 and Pandas 0.23 ## What changes were proposed in this pull request? Fix test that constructs a Pandas DataFrame by specifying the column order. Previously this test assumed the columns would be sorted alphabetically, however when using Python 3.6 with Pandas 0.23 or higher, the original column order is maintained. This causes the columns to get mixed up and the test errors. Manually tested with `python/run-tests` using Python 3.6.6 and Pandas 0.23.4 Closes #22477 from BryanCutler/pyspark-tests-py36-pd23-SPARK-25471. Authored-by: Bryan Cutler Signed-off-by: hyukjinkwon (cherry picked from commit 90e3955f384ca07bdf24faa6cdb60ded944cf0d8) Signed-off-by: hyukjinkwon --- python/pyspark/sql/tests.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 08d7cfadc084..603f994dc959 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -3266,7 +3266,7 @@ def test_create_dataframe_from_pandas_with_timestamp(self): import pandas as pd from datetime import datetime pdf = pd.DataFrame({"ts": [datetime(2017, 10, 31, 1, 1, 1)], - "d": [pd.Timestamp.now().date()]}) + "d": [pd.Timestamp.now().date()]}, columns=["d", "ts"]) # test types are inferred correctly without specifying schema df = self.spark.createDataFrame(pdf) self.assertTrue(isinstance(df.schema['ts'].dataType, TimestampType)) From 535bf1cc9e6b54df7059ac3109b8cba30057d040 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 19 Sep 2018 18:51:20 -0700 Subject: [PATCH 077/879] [SPARK-24157][SS][FOLLOWUP] Rename to spark.sql.streaming.noDataMicroBatches.enabled ## What changes were proposed in this pull request? This patch changes the config option `spark.sql.streaming.noDataMicroBatchesEnabled` to `spark.sql.streaming.noDataMicroBatches.enabled` to be more consistent with rest of the configs. Unfortunately there is one streaming config called `spark.sql.streaming.metricsEnabled`. For that one we should just use a fallback config and change it in a separate patch. ## How was this patch tested? Made sure no other references to this config are in the code base: ``` > git grep "noDataMicro" sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala: buildConf("spark.sql.streaming.noDataMicroBatches.enabled") ``` Closes #22476 from rxin/SPARK-24157. Authored-by: Reynold Xin Signed-off-by: Reynold Xin (cherry picked from commit 936c920347e196381b48bc3656ca81a06f2ff46d) Signed-off-by: Reynold Xin --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 3e9cde4bc252..8b82fe1eec50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1056,7 +1056,7 @@ object SQLConf { .createWithDefault(10000L) val STREAMING_NO_DATA_MICRO_BATCHES_ENABLED = - buildConf("spark.sql.streaming.noDataMicroBatchesEnabled") + buildConf("spark.sql.streaming.noDataMicroBatches.enabled") .doc( "Whether streaming micro-batch engine will execute batches without data " + "for eager state management for stateful streaming queries.") From 06efed29047041096910f300cb14e8cfff540efc Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Thu, 20 Sep 2018 10:10:20 +0800 Subject: [PATCH 078/879] [SPARK-24341][FOLLOWUP][DOCS] Add migration note for IN subqueries behavior ## What changes were proposed in this pull request? The PR updates the migration guide in order to explain the changes introduced in the behavior of the IN operator with subqueries, in particular, the improved handling of struct attributes in these situations. ## How was this patch tested? NA Closes #22469 from mgaido91/SPARK-24341_followup. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan (cherry picked from commit 8aae49afc7997aa1da61029409ef6d8ce0ab256a) Signed-off-by: Wenchen Fan --- docs/sql-programming-guide.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 2fa29a00e74c..c76f2e30e677 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1879,6 +1879,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see ## Upgrading From Spark SQL 2.3 to 2.4 + - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. From dfcff38394929970fee454c69864d0e10d59f8d4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 19 Sep 2018 21:23:35 -0700 Subject: [PATCH 079/879] [SPARK-4502][SQL] Rename to spark.sql.optimizer.nestedSchemaPruning.enabled ## What changes were proposed in this pull request? This patch adds an "optimizer" prefix to nested schema pruning. ## How was this patch tested? Should be covered by existing tests. Closes #22475 from rxin/SPARK-4502. Authored-by: Reynold Xin Signed-off-by: gatorsmile (cherry picked from commit 76399d75e23f2c7d6c2a1fb77a4387c5e15c809b) Signed-off-by: gatorsmile --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 8b82fe1eec50..c5901ee5c643 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1437,7 +1437,7 @@ object SQLConf { .createWithDefault(true) val NESTED_SCHEMA_PRUNING_ENABLED = - buildConf("spark.sql.nestedSchemaPruning.enabled") + buildConf("spark.sql.optimizer.nestedSchemaPruning.enabled") .internal() .doc("Prune nested fields from a logical relation's output which are unnecessary in " + "satisfying a query. This optimization allows columnar file format readers to avoid " + From e07042a3593199f5045e1476b6b324f7f0901143 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 20 Sep 2018 15:03:16 +0800 Subject: [PATCH 080/879] [MINOR][PYTHON][TEST] Use collect() instead of show() to make the output silent ## What changes were proposed in this pull request? This PR replace an effective `show()` to `collect()` to make the output silent. **Before:** ``` test_simple_udt_in_df (pyspark.sql.tests.SQLTests) ... +---+----------+ |key| val| +---+----------+ | 0|[0.0, 0.0]| | 1|[1.0, 1.0]| | 2|[2.0, 2.0]| | 0|[3.0, 3.0]| | 1|[4.0, 4.0]| | 2|[5.0, 5.0]| | 0|[6.0, 6.0]| | 1|[7.0, 7.0]| | 2|[8.0, 8.0]| | 0|[9.0, 9.0]| +---+----------+ ``` **After:** ``` test_simple_udt_in_df (pyspark.sql.tests.SQLTests) ... ok ``` ## How was this patch tested? Manually tested. Closes #22479 from HyukjinKwon/minor-udf-test. Authored-by: hyukjinkwon Signed-off-by: hyukjinkwon (cherry picked from commit 7ff5386ed934190344b2cda1069bde4bc68a3e63) Signed-off-by: hyukjinkwon --- python/pyspark/sql/tests.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 603f994dc959..8724bbc6ca7c 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1168,7 +1168,7 @@ def test_simple_udt_in_df(self): df = self.spark.createDataFrame( [(i % 3, PythonOnlyPoint(float(i), float(i))) for i in range(10)], schema=schema) - df.show() + df.collect() def test_nested_udt_in_df(self): schema = StructType().add("key", LongType()).add("val", ArrayType(PythonOnlyUDT())) From b3bdfd7f102eb79d111e096baa923926f6ccf7a2 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 20 Sep 2018 20:18:31 +0800 Subject: [PATCH 081/879] Revert [SPARK-19355][SPARK-25352] ## What changes were proposed in this pull request? This goes to revert sequential PRs based on some discussion and comments at https://github.com/apache/spark/pull/16677#issuecomment-422650759. #22344 #22330 #22239 #16677 ## How was this patch tested? Existing tests. Closes #22481 from viirya/revert-SPARK-19355-1. Authored-by: Liang-Chi Hsieh Signed-off-by: Wenchen Fan (cherry picked from commit 89671a27e783d77d4bfaec3d422cc8dd468ef04c) Signed-off-by: Wenchen Fan --- .../sort/BypassMergeSortShuffleWriter.java | 5 +- .../shuffle/sort/UnsafeShuffleWriter.java | 3 +- .../apache/spark/MapOutputStatistics.scala | 6 +- .../org/apache/spark/MapOutputTracker.scala | 10 +- .../apache/spark/scheduler/MapStatus.scala | 43 ++------ .../shuffle/sort/SortShuffleWriter.scala | 3 +- .../sort/UnsafeShuffleWriterSuite.java | 2 - .../apache/spark/MapOutputTrackerSuite.scala | 28 ++--- .../scala/org/apache/spark/ShuffleSuite.scala | 1 - .../spark/scheduler/DAGSchedulerSuite.scala | 10 +- .../spark/scheduler/MapStatusSuite.scala | 16 +-- .../serializer/KryoSerializerSuite.scala | 3 +- .../plans/physical/partitioning.scala | 14 --- .../apache/spark/sql/internal/SQLConf.scala | 9 -- .../spark/sql/execution/SparkStrategies.scala | 35 ++---- .../exchange/ShuffleExchangeExec.scala | 8 -- .../apache/spark/sql/execution/limit.scala | 104 +++--------------- .../test/resources/sql-tests/inputs/limit.sql | 2 - .../inputs/subquery/in-subquery/in-limit.sql | 5 +- .../resources/sql-tests/results/limit.sql.out | 92 +++++++--------- .../subquery/in-subquery/in-limit.sql.out | 56 ++++------ .../spark/sql/DataFrameAggregateSuite.scala | 12 +- .../org/apache/spark/sql/DataFrameSuite.scala | 22 +--- .../org/apache/spark/sql/SQLQuerySuite.scala | 11 +- .../execution/ExchangeCoordinatorSuite.scala | 6 +- .../spark/sql/execution/LimitSuite.scala | 81 -------------- .../spark/sql/execution/PlannerSuite.scala | 4 +- .../TakeOrderedAndProjectSuite.scala | 85 ++++++-------- .../execution/HiveCompatibilitySuite.scala | 4 - .../sql/hive/execution/PruningSuite.scala | 8 -- 30 files changed, 184 insertions(+), 504 deletions(-) delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/LimitSuite.scala diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index e3bd5496cf5b..323a5d3c5283 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -125,7 +125,7 @@ public void write(Iterator> records) throws IOException { if (!records.hasNext()) { partitionLengths = new long[numPartitions]; shuffleBlockResolver.writeIndexFileAndCommit(shuffleId, mapId, partitionLengths, null); - mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths, 0); + mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); return; } final SerializerInstance serInstance = serializer.newInstance(); @@ -167,8 +167,7 @@ public void write(Iterator> records) throws IOException { logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); } } - mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, writeMetrics.recordsWritten()); + mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } @VisibleForTesting diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 069e6d5f224d..4839d04522f1 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -248,8 +248,7 @@ void closeAndWriteOutput() throws IOException { logger.error("Error while deleting temp file {}", tmp.getAbsolutePath()); } } - mapStatus = MapStatus$.MODULE$.apply( - blockManager.shuffleServerId(), partitionLengths, writeMetrics.recordsWritten()); + mapStatus = MapStatus$.MODULE$.apply(blockManager.shuffleServerId(), partitionLengths); } @VisibleForTesting diff --git a/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala b/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala index ff85e11409e3..f8a6f1d0d8cb 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputStatistics.scala @@ -23,9 +23,5 @@ package org.apache.spark * @param shuffleId ID of the shuffle * @param bytesByPartitionId approximate number of output bytes for each map output partition * (may be inexact due to use of compressed map statuses) - * @param recordsByPartitionId number of output records for each map output partition */ -private[spark] class MapOutputStatistics( - val shuffleId: Int, - val bytesByPartitionId: Array[Long], - val recordsByPartitionId: Array[Long]) +private[spark] class MapOutputStatistics(val shuffleId: Int, val bytesByPartitionId: Array[Long]) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 41575ce4e6e3..1c4fa4bc6541 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -522,19 +522,16 @@ private[spark] class MapOutputTrackerMaster( def getStatistics(dep: ShuffleDependency[_, _, _]): MapOutputStatistics = { shuffleStatuses(dep.shuffleId).withMapStatuses { statuses => val totalSizes = new Array[Long](dep.partitioner.numPartitions) - val recordsByMapTask = new Array[Long](statuses.length) - val parallelAggThreshold = conf.get( SHUFFLE_MAP_OUTPUT_PARALLEL_AGGREGATION_THRESHOLD) val parallelism = math.min( Runtime.getRuntime.availableProcessors(), statuses.length.toLong * totalSizes.length / parallelAggThreshold + 1).toInt if (parallelism <= 1) { - statuses.zipWithIndex.foreach { case (s, index) => + for (s <- statuses) { for (i <- 0 until totalSizes.length) { totalSizes(i) += s.getSizeForBlock(i) } - recordsByMapTask(index) = s.numberOfOutput } } else { val threadPool = ThreadUtils.newDaemonFixedThreadPool(parallelism, "map-output-aggregate") @@ -551,11 +548,8 @@ private[spark] class MapOutputTrackerMaster( } finally { threadPool.shutdown() } - statuses.zipWithIndex.foreach { case (s, index) => - recordsByMapTask(index) = s.numberOfOutput - } } - new MapOutputStatistics(dep.shuffleId, totalSizes, recordsByMapTask) + new MapOutputStatistics(dep.shuffleId, totalSizes) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 7e1d75fe723d..659694dd189a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -31,8 +31,7 @@ import org.apache.spark.util.Utils /** * Result returned by a ShuffleMapTask to a scheduler. Includes the block manager address that the - * task ran on, the sizes of outputs for each reducer, and the number of outputs of the map task, - * for passing on to the reduce tasks. + * task ran on as well as the sizes of outputs for each reducer, for passing on to the reduce tasks. */ private[spark] sealed trait MapStatus { /** Location where this task was run. */ @@ -45,23 +44,18 @@ private[spark] sealed trait MapStatus { * necessary for correctness, since block fetchers are allowed to skip zero-size blocks. */ def getSizeForBlock(reduceId: Int): Long - - /** - * The number of outputs for the map task. - */ - def numberOfOutput: Long } private[spark] object MapStatus { - def apply(loc: BlockManagerId, uncompressedSizes: Array[Long], numOutput: Long): MapStatus = { + def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = { if (uncompressedSizes.length > Option(SparkEnv.get) .map(_.conf.get(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS)) .getOrElse(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS.defaultValue.get)) { - HighlyCompressedMapStatus(loc, uncompressedSizes, numOutput) + HighlyCompressedMapStatus(loc, uncompressedSizes) } else { - new CompressedMapStatus(loc, uncompressedSizes, numOutput) + new CompressedMapStatus(loc, uncompressedSizes) } } @@ -104,34 +98,29 @@ private[spark] object MapStatus { */ private[spark] class CompressedMapStatus( private[this] var loc: BlockManagerId, - private[this] var compressedSizes: Array[Byte], - private[this] var numOutput: Long) + private[this] var compressedSizes: Array[Byte]) extends MapStatus with Externalizable { - protected def this() = this(null, null.asInstanceOf[Array[Byte]], -1) // For deserialization only + protected def this() = this(null, null.asInstanceOf[Array[Byte]]) // For deserialization only - def this(loc: BlockManagerId, uncompressedSizes: Array[Long], numOutput: Long) { - this(loc, uncompressedSizes.map(MapStatus.compressSize), numOutput) + def this(loc: BlockManagerId, uncompressedSizes: Array[Long]) { + this(loc, uncompressedSizes.map(MapStatus.compressSize)) } override def location: BlockManagerId = loc - override def numberOfOutput: Long = numOutput - override def getSizeForBlock(reduceId: Int): Long = { MapStatus.decompressSize(compressedSizes(reduceId)) } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) - out.writeLong(numOutput) out.writeInt(compressedSizes.length) out.write(compressedSizes) } override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) - numOutput = in.readLong() val len = in.readInt() compressedSizes = new Array[Byte](len) in.readFully(compressedSizes) @@ -154,20 +143,17 @@ private[spark] class HighlyCompressedMapStatus private ( private[this] var numNonEmptyBlocks: Int, private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long, - private var hugeBlockSizes: Map[Int, Byte], - private[this] var numOutput: Long) + private var hugeBlockSizes: Map[Int, Byte]) extends MapStatus with Externalizable { // loc could be null when the default constructor is called during deserialization require(loc == null || avgSize > 0 || hugeBlockSizes.size > 0 || numNonEmptyBlocks == 0, "Average size can only be zero for map stages that produced no output") - protected def this() = this(null, -1, null, -1, null, -1) // For deserialization only + protected def this() = this(null, -1, null, -1, null) // For deserialization only override def location: BlockManagerId = loc - override def numberOfOutput: Long = numOutput - override def getSizeForBlock(reduceId: Int): Long = { assert(hugeBlockSizes != null) if (emptyBlocks.contains(reduceId)) { @@ -182,7 +168,6 @@ private[spark] class HighlyCompressedMapStatus private ( override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { loc.writeExternal(out) - out.writeLong(numOutput) emptyBlocks.writeExternal(out) out.writeLong(avgSize) out.writeInt(hugeBlockSizes.size) @@ -194,7 +179,6 @@ private[spark] class HighlyCompressedMapStatus private ( override def readExternal(in: ObjectInput): Unit = Utils.tryOrIOException { loc = BlockManagerId(in) - numOutput = in.readLong() emptyBlocks = new RoaringBitmap() emptyBlocks.readExternal(in) avgSize = in.readLong() @@ -210,10 +194,7 @@ private[spark] class HighlyCompressedMapStatus private ( } private[spark] object HighlyCompressedMapStatus { - def apply( - loc: BlockManagerId, - uncompressedSizes: Array[Long], - numOutput: Long): HighlyCompressedMapStatus = { + def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): HighlyCompressedMapStatus = { // We must keep track of which blocks are empty so that we don't report a zero-sized // block as being non-empty (or vice-versa) when using the average block size. var i = 0 @@ -254,6 +235,6 @@ private[spark] object HighlyCompressedMapStatus { emptyBlocks.trim() emptyBlocks.runOptimize() new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize, - hugeBlockSizesArray.toMap, numOutput) + hugeBlockSizesArray.toMap) } } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 91fc26762e53..274399b9cc1f 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -70,8 +70,7 @@ private[spark] class SortShuffleWriter[K, V, C]( val blockId = ShuffleBlockId(dep.shuffleId, mapId, IndexShuffleBlockResolver.NOOP_REDUCE_ID) val partitionLengths = sorter.writePartitionedFile(blockId, tmp) shuffleBlockResolver.writeIndexFileAndCommit(dep.shuffleId, mapId, partitionLengths, tmp) - mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths, - writeMetrics.recordsWritten) + mapStatus = MapStatus(blockManager.shuffleServerId, partitionLengths) } finally { if (tmp.exists() && !tmp.delete()) { logError(s"Error while deleting temp file ${tmp.getAbsolutePath}") diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index faa70f23b0ac..0d5c5ea7903e 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -233,7 +233,6 @@ public void writeEmptyIterator() throws Exception { writer.write(Iterators.emptyIterator()); final Option mapStatus = writer.stop(true); assertTrue(mapStatus.isDefined()); - assertEquals(0, mapStatus.get().numberOfOutput()); assertTrue(mergedOutputFile.exists()); assertArrayEquals(new long[NUM_PARTITITONS], partitionSizesInMergedFile); assertEquals(0, taskMetrics.shuffleWriteMetrics().recordsWritten()); @@ -253,7 +252,6 @@ public void writeWithoutSpilling() throws Exception { writer.write(dataToWrite.iterator()); final Option mapStatus = writer.stop(true); assertTrue(mapStatus.isDefined()); - assertEquals(NUM_PARTITITONS, mapStatus.get().numberOfOutput()); assertTrue(mergedOutputFile.exists()); long sumOfPartitionSizes = 0; diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index e79739692fe1..21f481d47724 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -62,9 +62,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) val size10000 = MapStatus.decompressSize(MapStatus.compressSize(10000L)) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(1000L, 10000L), 10)) + Array(1000L, 10000L))) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(10000L, 1000L), 10)) + Array(10000L, 1000L))) val statuses = tracker.getMapSizesByExecutorId(10, 0) assert(statuses.toSet === Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))), @@ -84,9 +84,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(compressedSize1000, compressedSize10000), 10)) + Array(compressedSize1000, compressedSize10000))) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(compressedSize10000, compressedSize1000), 10)) + Array(compressedSize10000, compressedSize1000))) assert(tracker.containsShuffle(10)) assert(tracker.getMapSizesByExecutorId(10, 0).nonEmpty) assert(0 == tracker.getNumCachedSerializedBroadcast) @@ -107,9 +107,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { val compressedSize1000 = MapStatus.compressSize(1000L) val compressedSize10000 = MapStatus.compressSize(10000L) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(compressedSize1000, compressedSize1000, compressedSize1000), 10)) + Array(compressedSize1000, compressedSize1000, compressedSize1000))) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(compressedSize10000, compressedSize1000, compressedSize1000), 10)) + Array(compressedSize10000, compressedSize1000, compressedSize1000))) assert(0 == tracker.getNumCachedSerializedBroadcast) // As if we had two simultaneous fetch failures @@ -145,7 +145,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) masterTracker.registerMapOutput(10, 0, MapStatus( - BlockManagerId("a", "hostA", 1000), Array(1000L), 10)) + BlockManagerId("a", "hostA", 1000), Array(1000L))) slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getMapSizesByExecutorId(10, 0).toSeq === Seq((BlockManagerId("a", "hostA", 1000), ArrayBuffer((ShuffleBlockId(10, 0, 0), size1000))))) @@ -182,7 +182,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { // Message size should be ~123B, and no exception should be thrown masterTracker.registerShuffle(10, 1) masterTracker.registerMapOutput(10, 0, MapStatus( - BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0), 0)) + BlockManagerId("88", "mph", 1000), Array.fill[Long](10)(0))) val senderAddress = RpcAddress("localhost", 12345) val rpcCallContext = mock(classOf[RpcCallContext]) when(rpcCallContext.senderAddress).thenReturn(senderAddress) @@ -216,11 +216,11 @@ class MapOutputTrackerSuite extends SparkFunSuite { // on hostB with output size 3 tracker.registerShuffle(10, 3) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(2L), 1)) + Array(2L))) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(2L), 1)) + Array(2L))) tracker.registerMapOutput(10, 2, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(3L), 1)) + Array(3L))) // When the threshold is 50%, only host A should be returned as a preferred location // as it has 4 out of 7 bytes of output. @@ -260,7 +260,7 @@ class MapOutputTrackerSuite extends SparkFunSuite { masterTracker.registerShuffle(20, 100) (0 until 100).foreach { i => masterTracker.registerMapOutput(20, i, new CompressedMapStatus( - BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0), 0)) + BlockManagerId("999", "mps", 1000), Array.fill[Long](4000000)(0))) } val senderAddress = RpcAddress("localhost", 12345) val rpcCallContext = mock(classOf[RpcCallContext]) @@ -309,9 +309,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { val size1000 = MapStatus.decompressSize(MapStatus.compressSize(1000L)) val size10000 = MapStatus.decompressSize(MapStatus.compressSize(10000L)) tracker.registerMapOutput(10, 0, MapStatus(BlockManagerId("a", "hostA", 1000), - Array(size0, size1000, size0, size10000), 1)) + Array(size0, size1000, size0, size10000))) tracker.registerMapOutput(10, 1, MapStatus(BlockManagerId("b", "hostB", 1000), - Array(size10000, size0, size1000, size0), 1)) + Array(size10000, size0, size1000, size0))) assert(tracker.containsShuffle(10)) assert(tracker.getMapSizesByExecutorId(10, 0, 4).toSeq === Seq( diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 456f97b535ef..b917469e4874 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -391,7 +391,6 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(mapOutput2.isDefined) assert(mapOutput1.get.location === mapOutput2.get.location) assert(mapOutput1.get.getSizeForBlock(0) === mapOutput1.get.getSizeForBlock(0)) - assert(mapOutput1.get.numberOfOutput === mapOutput2.get.numberOfOutput) // register one of the map outputs -- doesn't matter which one mapOutput1.foreach { case mapStatus => diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 4e87deb136df..c820f4899570 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -443,17 +443,17 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // map stage1 completes successfully, with one task on each executor complete(taskSets(0), Seq( (Success, - MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), 1)), + MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2))), (Success, - MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), 1)), + MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2))), (Success, makeMapStatus("hostB", 1)) )) // map stage2 completes successfully, with one task on each executor complete(taskSets(1), Seq( (Success, - MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2), 1)), + MapStatus(BlockManagerId("exec-hostA1", "hostA", 12345), Array.fill[Long](1)(2))), (Success, - MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2), 1)), + MapStatus(BlockManagerId("exec-hostA2", "hostA", 12345), Array.fill[Long](1)(2))), (Success, makeMapStatus("hostB", 1)) )) // make sure our test setup is correct @@ -2854,7 +2854,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi object DAGSchedulerSuite { def makeMapStatus(host: String, reduces: Int, sizes: Byte = 2): MapStatus = - MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes), 1) + MapStatus(makeBlockManagerId(host), Array.fill[Long](reduces)(sizes)) def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index 555e48bd28aa..354e6386fa60 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -60,7 +60,7 @@ class MapStatusSuite extends SparkFunSuite { stddev <- Seq(0.0, 0.01, 0.5, 1.0) ) { val sizes = Array.fill[Long](numSizes)(abs(round(Random.nextGaussian() * stddev)) + mean) - val status = MapStatus(BlockManagerId("a", "b", 10), sizes, 1) + val status = MapStatus(BlockManagerId("a", "b", 10), sizes) val status1 = compressAndDecompressMapStatus(status) for (i <- 0 until numSizes) { if (sizes(i) != 0) { @@ -74,7 +74,7 @@ class MapStatusSuite extends SparkFunSuite { test("large tasks should use " + classOf[HighlyCompressedMapStatus].getName) { val sizes = Array.fill[Long](2001)(150L) - val status = MapStatus(null, sizes, 1) + val status = MapStatus(null, sizes) assert(status.isInstanceOf[HighlyCompressedMapStatus]) assert(status.getSizeForBlock(10) === 150L) assert(status.getSizeForBlock(50) === 150L) @@ -86,7 +86,7 @@ class MapStatusSuite extends SparkFunSuite { val sizes = Array.tabulate[Long](3000) { i => i.toLong } val avg = sizes.sum / sizes.count(_ != 0) val loc = BlockManagerId("a", "b", 10) - val status = MapStatus(loc, sizes, 1) + val status = MapStatus(loc, sizes) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) @@ -108,7 +108,7 @@ class MapStatusSuite extends SparkFunSuite { val smallBlockSizes = sizes.filter(n => n > 0 && n < threshold) val avg = smallBlockSizes.sum / smallBlockSizes.length val loc = BlockManagerId("a", "b", 10) - val status = MapStatus(loc, sizes, 1) + val status = MapStatus(loc, sizes) val status1 = compressAndDecompressMapStatus(status) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) assert(status1.location == loc) @@ -164,7 +164,7 @@ class MapStatusSuite extends SparkFunSuite { SparkEnv.set(env) // Value of element in sizes is equal to the corresponding index. val sizes = (0L to 2000L).toArray - val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes, 1) + val status1 = MapStatus(BlockManagerId("exec-0", "host-0", 100), sizes) val arrayStream = new ByteArrayOutputStream(102400) val objectOutputStream = new ObjectOutputStream(arrayStream) assert(status1.isInstanceOf[HighlyCompressedMapStatus]) @@ -196,19 +196,19 @@ class MapStatusSuite extends SparkFunSuite { SparkEnv.set(env) val sizes = Array.fill[Long](500)(150L) // Test default value - val status = MapStatus(null, sizes, 1) + val status = MapStatus(null, sizes) assert(status.isInstanceOf[CompressedMapStatus]) // Test Non-positive values for (s <- -1 to 0) { assertThrows[IllegalArgumentException] { conf.set(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS, s) - val status = MapStatus(null, sizes, 1) + val status = MapStatus(null, sizes) } } // Test positive values Seq(1, 100, 499, 500, 501).foreach { s => conf.set(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS, s) - val status = MapStatus(null, sizes, 1) + val status = MapStatus(null, sizes) if(sizes.length > s) { assert(status.isInstanceOf[HighlyCompressedMapStatus]) } else { diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 36912441c03b..ac25bcef5434 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -345,8 +345,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val denseBlockSizes = new Array[Long](5000) val sparseBlockSizes = Array[Long](0L, 1L, 0L, 2L) Seq(denseBlockSizes, sparseBlockSizes).foreach { blockSizes => - ser.serialize( - HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes, 1)) + ser.serialize(HighlyCompressedMapStatus(BlockManagerId("exec-1", "host", 1234), blockSizes)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index cd28c733f361..cc1a5e835d9c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.plans.physical -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.types.{DataType, IntegerType} @@ -208,18 +206,6 @@ case object SinglePartition extends Partitioning { } } -/** - * Represents a partitioning where rows are only serialized/deserialized locally. The number - * of partitions are not changed and also the distribution of rows. This is mainly used to - * obtain some statistics of map tasks such as number of outputs. - */ -case class LocalPartitioning(childRDD: RDD[InternalRow]) extends Partitioning { - val numPartitions = childRDD.getNumPartitions - - // We will perform this partitioning no matter what the data distribution is. - override def satisfies0(required: Distribution): Boolean = false -} - /** * Represents a partitioning where rows are split up across partitions based on the hash * of `expressions`. All rows where `expressions` evaluate to the same values are guaranteed to be diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index c5901ee5c643..169649c993ff 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -235,13 +235,6 @@ object SQLConf { .intConf .createWithDefault(4) - val LIMIT_FLAT_GLOBAL_LIMIT = buildConf("spark.sql.limit.flatGlobalLimit") - .internal() - .doc("During global limit, try to evenly distribute limited rows across data " + - "partitions. If disabled, scanning data partitions sequentially until reaching limit number.") - .booleanConf - .createWithDefault(true) - val ADVANCED_PARTITION_PREDICATE_PUSHDOWN = buildConf("spark.sql.hive.advancedPartitionPredicatePushdown.enabled") .internal() @@ -1740,8 +1733,6 @@ class SQLConf extends Serializable with Logging { def limitScaleUpFactor: Int = getConf(LIMIT_SCALE_UP_FACTOR) - def limitFlatGlobalLimit: Boolean = getConf(LIMIT_FLAT_GLOBAL_LIMIT) - def advancedPartitionPredicatePushdownEnabled: Boolean = getConf(ADVANCED_PARTITION_PREDICATE_PUSHDOWN) 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 89442a70283f..dbc6db62bd82 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 @@ -66,35 +66,24 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { * Plans special cases of limit operators. */ object SpecialLimits extends Strategy { - private def decideTopRankNode(limit: Int, child: LogicalPlan): Seq[SparkPlan] = { - if (limit < conf.topKSortFallbackThreshold) { - child match { - case Sort(order, true, child) => - TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil - case Project(projectList, Sort(order, true, child)) => - TakeOrderedAndProjectExec(limit, order, projectList, planLater(child)) :: Nil - } - } else { - GlobalLimitExec(limit, - LocalLimitExec(limit, planLater(child)), - orderedLimit = true) :: Nil - } - } - override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case ReturnAnswer(rootPlan) => rootPlan match { - case Limit(IntegerLiteral(limit), s @ Sort(order, true, child)) => - decideTopRankNode(limit, s) - case Limit(IntegerLiteral(limit), p @ Project(projectList, Sort(order, true, child))) => - decideTopRankNode(limit, p) + case Limit(IntegerLiteral(limit), Sort(order, true, child)) + if limit < conf.topKSortFallbackThreshold => + TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil + case Limit(IntegerLiteral(limit), Project(projectList, Sort(order, true, child))) + if limit < conf.topKSortFallbackThreshold => + TakeOrderedAndProjectExec(limit, order, projectList, planLater(child)) :: Nil case Limit(IntegerLiteral(limit), child) => CollectLimitExec(limit, planLater(child)) :: Nil case other => planLater(other) :: Nil } - case Limit(IntegerLiteral(limit), s @ Sort(order, true, child)) => - decideTopRankNode(limit, s) - case Limit(IntegerLiteral(limit), p @ Project(projectList, Sort(order, true, child))) => - decideTopRankNode(limit, p) + case Limit(IntegerLiteral(limit), Sort(order, true, child)) + if limit < conf.topKSortFallbackThreshold => + TakeOrderedAndProjectExec(limit, order, child.output, planLater(child)) :: Nil + case Limit(IntegerLiteral(limit), Project(projectList, Sort(order, true, child))) + if limit < conf.topKSortFallbackThreshold => + TakeOrderedAndProjectExec(limit, order, projectList, planLater(child)) :: Nil case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index 9576605b1a21..aba94885f941 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -231,11 +231,6 @@ object ShuffleExchangeExec { override def numPartitions: Int = 1 override def getPartition(key: Any): Int = 0 } - case l: LocalPartitioning => - new Partitioner { - override def numPartitions: Int = l.numPartitions - override def getPartition(key: Any): Int = key.asInstanceOf[Int] - } case _ => sys.error(s"Exchange not implemented for $newPartitioning") // TODO: Handle BroadcastPartitioning. } @@ -252,9 +247,6 @@ object ShuffleExchangeExec { val projection = UnsafeProjection.create(h.partitionIdExpression :: Nil, outputAttributes) row => projection(row).getInt(0) case RangePartitioning(_, _) | SinglePartition => identity - case _: LocalPartitioning => - val partitionId = TaskContext.get().partitionId() - _ => partitionId case _ => sys.error(s"Exchange not implemented for $newPartitioning") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 1a09632f93ca..66bcda891373 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -47,16 +47,13 @@ case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode } /** - * Take the first `limit` elements of each child partition, but do not collect or shuffle them. + * Helper trait which defines methods that are shared by both + * [[LocalLimitExec]] and [[GlobalLimitExec]]. */ -case class LocalLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode with CodegenSupport { - +trait BaseLimitExec extends UnaryExecNode with CodegenSupport { + val limit: Int override def output: Seq[Attribute] = child.output - override def outputOrdering: Seq[SortOrder] = child.outputOrdering - - override def outputPartitioning: Partitioning = child.outputPartitioning - protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitions { iter => iter.take(limit) } @@ -96,96 +93,25 @@ case class LocalLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode wi } /** - * Take the `limit` elements of the child output. + * Take the first `limit` elements of each child partition, but do not collect or shuffle them. */ -case class GlobalLimitExec(limit: Int, child: SparkPlan, - orderedLimit: Boolean = false) extends UnaryExecNode { +case class LocalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { - override def output: Seq[Attribute] = child.output + override def outputOrdering: Seq[SortOrder] = child.outputOrdering override def outputPartitioning: Partitioning = child.outputPartitioning +} - override def outputOrdering: Seq[SortOrder] = child.outputOrdering +/** + * Take the first `limit` elements of the child's single output partition. + */ +case class GlobalLimitExec(limit: Int, child: SparkPlan) extends BaseLimitExec { - private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) + override def requiredChildDistribution: List[Distribution] = AllTuples :: Nil - protected override def doExecute(): RDD[InternalRow] = { - val childRDD = child.execute() - val partitioner = LocalPartitioning(childRDD) - val shuffleDependency = ShuffleExchangeExec.prepareShuffleDependency( - childRDD, child.output, partitioner, serializer) - val numberOfOutput: Seq[Long] = if (shuffleDependency.rdd.getNumPartitions != 0) { - // submitMapStage does not accept RDD with 0 partition. - // So, we will not submit this dependency. - val submittedStageFuture = sparkContext.submitMapStage(shuffleDependency) - submittedStageFuture.get().recordsByPartitionId.toSeq - } else { - Nil - } + override def outputPartitioning: Partitioning = child.outputPartitioning - // This is an optimization to evenly distribute limited rows across all partitions. - // When enabled, Spark goes to take rows at each partition repeatedly until reaching - // limit number. When disabled, Spark takes all rows at first partition, then rows - // at second partition ..., until reaching limit number. - // The optimization is disabled when it is needed to keep the original order of rows - // before global sort, e.g., select * from table order by col limit 10. - val flatGlobalLimit = sqlContext.conf.limitFlatGlobalLimit && !orderedLimit - - val shuffled = new ShuffledRowRDD(shuffleDependency) - - val sumOfOutput = numberOfOutput.sum - if (sumOfOutput <= limit) { - shuffled - } else if (!flatGlobalLimit) { - var numRowTaken = 0 - val takeAmounts = numberOfOutput.map { num => - if (numRowTaken + num < limit) { - numRowTaken += num.toInt - num.toInt - } else { - val toTake = limit - numRowTaken - numRowTaken += toTake - toTake - } - } - val broadMap = sparkContext.broadcast(takeAmounts) - shuffled.mapPartitionsWithIndexInternal { case (index, iter) => - iter.take(broadMap.value(index).toInt) - } - } else { - // We try to evenly require the asked limit number of rows across all child rdd's partitions. - var rowsNeedToTake: Long = limit - val takeAmountByPartition: Array[Long] = Array.fill[Long](numberOfOutput.length)(0L) - val remainingRowsByPartition: Array[Long] = Array(numberOfOutput: _*) - - while (rowsNeedToTake > 0) { - val nonEmptyParts = remainingRowsByPartition.count(_ > 0) - // If the rows needed to take are less the number of non-empty partitions, take one row from - // each non-empty partitions until we reach `limit` rows. - // Otherwise, evenly divide the needed rows to each non-empty partitions. - val takePerPart = math.max(1, rowsNeedToTake / nonEmptyParts) - remainingRowsByPartition.zipWithIndex.foreach { case (num, index) => - // In case `rowsNeedToTake` < `nonEmptyParts`, we may run out of `rowsNeedToTake` during - // the traversal, so we need to add this check. - if (rowsNeedToTake > 0 && num > 0) { - if (num >= takePerPart) { - rowsNeedToTake -= takePerPart - takeAmountByPartition(index) += takePerPart - remainingRowsByPartition(index) -= takePerPart - } else { - rowsNeedToTake -= num - takeAmountByPartition(index) += num - remainingRowsByPartition(index) -= num - } - } - } - } - val broadMap = sparkContext.broadcast(takeAmountByPartition) - shuffled.mapPartitionsWithIndexInternal { case (index, iter) => - iter.take(broadMap.value(index).toInt) - } - } - } + override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** diff --git a/sql/core/src/test/resources/sql-tests/inputs/limit.sql b/sql/core/src/test/resources/sql-tests/inputs/limit.sql index e33cd819f281..b4c73cf33e53 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/limit.sql @@ -1,5 +1,3 @@ --- Disable global limit parallel -set spark.sql.limit.flatGlobalLimit=false; -- limit on various data types SELECT * FROM testdata LIMIT 2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql index a862e0985b20..a40ee082ba3b 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/in-subquery/in-limit.sql @@ -1,9 +1,6 @@ -- A test suite for IN LIMIT in parent side, subquery, and both predicate subquery -- It includes correlated cases. --- Disable global limit optimization -set spark.sql.limit.flatGlobalLimit=false; - create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -100,4 +97,4 @@ WHERE t1d NOT IN (SELECT t2d LIMIT 1) GROUP BY t1b ORDER BY t1b NULLS last -LIMIT 1; +LIMIT 1; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/results/limit.sql.out b/sql/core/src/test/resources/sql-tests/results/limit.sql.out index 187f3bd6858f..02fe1de84f75 100644 --- a/sql/core/src/test/resources/sql-tests/results/limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/limit.sql.out @@ -1,134 +1,126 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 15 +-- Number of queries: 14 -- !query 0 -set spark.sql.limit.flatGlobalLimit=false --- !query 0 schema -struct --- !query 0 output -spark.sql.limit.flatGlobalLimit false - - --- !query 1 SELECT * FROM testdata LIMIT 2 --- !query 1 schema +-- !query 0 schema struct --- !query 1 output +-- !query 0 output 1 1 2 2 --- !query 2 +-- !query 1 SELECT * FROM arraydata LIMIT 2 --- !query 2 schema +-- !query 1 schema struct,nestedarraycol:array>> --- !query 2 output +-- !query 1 output [1,2,3] [[1,2,3]] [2,3,4] [[2,3,4]] --- !query 3 +-- !query 2 SELECT * FROM mapdata LIMIT 2 --- !query 3 schema +-- !query 2 schema struct> --- !query 3 output +-- !query 2 output {1:"a1",2:"b1",3:"c1",4:"d1",5:"e1"} {1:"a2",2:"b2",3:"c2",4:"d2"} --- !query 4 +-- !query 3 SELECT * FROM testdata LIMIT 2 + 1 --- !query 4 schema +-- !query 3 schema struct --- !query 4 output +-- !query 3 output 1 1 2 2 3 3 --- !query 5 +-- !query 4 SELECT * FROM testdata LIMIT CAST(1 AS int) --- !query 5 schema +-- !query 4 schema struct --- !query 5 output +-- !query 4 output 1 1 --- !query 6 +-- !query 5 SELECT * FROM testdata LIMIT -1 --- !query 6 schema +-- !query 5 schema struct<> --- !query 6 output +-- !query 5 output org.apache.spark.sql.AnalysisException The limit expression must be equal to or greater than 0, but got -1; --- !query 7 +-- !query 6 SELECT * FROM testData TABLESAMPLE (-1 ROWS) --- !query 7 schema +-- !query 6 schema struct<> --- !query 7 output +-- !query 6 output org.apache.spark.sql.AnalysisException The limit expression must be equal to or greater than 0, but got -1; --- !query 8 +-- !query 7 SELECT * FROM testdata LIMIT CAST(1 AS INT) --- !query 8 schema +-- !query 7 schema struct --- !query 8 output +-- !query 7 output 1 1 --- !query 9 +-- !query 8 SELECT * FROM testdata LIMIT CAST(NULL AS INT) --- !query 9 schema +-- !query 8 schema struct<> --- !query 9 output +-- !query 8 output org.apache.spark.sql.AnalysisException The evaluated limit expression must not be null, but got CAST(NULL AS INT); --- !query 10 +-- !query 9 SELECT * FROM testdata LIMIT key > 3 --- !query 10 schema +-- !query 9 schema struct<> --- !query 10 output +-- !query 9 output org.apache.spark.sql.AnalysisException The limit expression must evaluate to a constant value, but got (testdata.`key` > 3); --- !query 11 +-- !query 10 SELECT * FROM testdata LIMIT true --- !query 11 schema +-- !query 10 schema struct<> --- !query 11 output +-- !query 10 output org.apache.spark.sql.AnalysisException The limit expression must be integer type, but got boolean; --- !query 12 +-- !query 11 SELECT * FROM testdata LIMIT 'a' --- !query 12 schema +-- !query 11 schema struct<> --- !query 12 output +-- !query 11 output org.apache.spark.sql.AnalysisException The limit expression must be integer type, but got string; --- !query 13 +-- !query 12 SELECT * FROM (SELECT * FROM range(10) LIMIT 5) WHERE id > 3 --- !query 13 schema +-- !query 12 schema struct --- !query 13 output +-- !query 12 output 4 --- !query 14 +-- !query 13 SELECT * FROM testdata WHERE key < 3 LIMIT ALL --- !query 14 schema +-- !query 13 schema struct --- !query 14 output +-- !query 13 output 1 1 2 2 diff --git a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out index 9eb5b3383e73..71ca1f864947 100644 --- a/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/subquery/in-subquery/in-limit.sql.out @@ -1,16 +1,8 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 9 +-- Number of queries: 8 -- !query 0 -set spark.sql.limit.flatGlobalLimit=false --- !query 0 schema -struct --- !query 0 output -spark.sql.limit.flatGlobalLimit false - - --- !query 1 create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:00:00.000', date '2014-04-04'), ("val1b", 8S, 16, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -25,13 +17,13 @@ create temporary view t1 as select * from values ("val1a", 6S, 8, 10L, float(15.0), 20D, 20E2, timestamp '2014-04-04 01:02:00.001', date '2014-04-04'), ("val1e", 10S, null, 19L, float(17.0), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04') as t1(t1a, t1b, t1c, t1d, t1e, t1f, t1g, t1h, t1i) --- !query 1 schema +-- !query 0 schema struct<> --- !query 1 output +-- !query 0 output --- !query 2 +-- !query 1 create temporary view t2 as select * from values ("val2a", 6S, 12, 14L, float(15), 20D, 20E2, timestamp '2014-04-04 01:01:00.000', date '2014-04-04'), ("val1b", 10S, 12, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', date '2014-05-04'), @@ -47,13 +39,13 @@ create temporary view t2 as select * from values ("val1f", 19S, null, 19L, float(17), 25D, 26E2, timestamp '2014-10-04 01:01:00.000', date '2014-10-04'), ("val1b", null, 16, 19L, float(17), 25D, 26E2, timestamp '2014-05-04 01:01:00.000', null) as t2(t2a, t2b, t2c, t2d, t2e, t2f, t2g, t2h, t2i) --- !query 2 schema +-- !query 1 schema struct<> --- !query 2 output +-- !query 1 output --- !query 3 +-- !query 2 create temporary view t3 as select * from values ("val3a", 6S, 12, 110L, float(15), 20D, 20E2, timestamp '2014-04-04 01:02:00.000', date '2014-04-04'), ("val3a", 6S, 12, 10L, float(15), 20D, 20E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), @@ -68,27 +60,27 @@ create temporary view t3 as select * from values ("val3b", 8S, null, 719L, float(17), 25D, 26E2, timestamp '2014-05-04 01:02:00.000', date '2014-05-04'), ("val3b", 8S, null, 19L, float(17), 25D, 26E2, timestamp '2015-05-04 01:02:00.000', date '2015-05-04') as t3(t3a, t3b, t3c, t3d, t3e, t3f, t3g, t3h, t3i) --- !query 3 schema +-- !query 2 schema struct<> --- !query 3 output +-- !query 2 output --- !query 4 +-- !query 3 SELECT * FROM t1 WHERE t1a IN (SELECT t2a FROM t2 WHERE t1d = t2d) LIMIT 2 --- !query 4 schema +-- !query 3 schema struct --- !query 4 output +-- !query 3 output val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 --- !query 5 +-- !query 4 SELECT * FROM t1 WHERE t1c IN (SELECT t2c @@ -96,16 +88,16 @@ WHERE t1c IN (SELECT t2c WHERE t2b >= 8 LIMIT 2) LIMIT 4 --- !query 5 schema +-- !query 4 schema struct --- !query 5 output +-- !query 4 output val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 val1b 8 16 19 17.0 25.0 2600 2014-05-04 01:01:00 2014-05-04 val1c 8 16 19 17.0 25.0 2600 2014-05-04 01:02:00.001 2014-05-05 --- !query 6 +-- !query 5 SELECT Count(DISTINCT( t1a )), t1b FROM t1 @@ -116,29 +108,29 @@ WHERE t1d IN (SELECT t2d GROUP BY t1b ORDER BY t1b DESC NULLS FIRST LIMIT 1 --- !query 6 schema +-- !query 5 schema struct --- !query 6 output +-- !query 5 output 1 NULL --- !query 7 +-- !query 6 SELECT * FROM t1 WHERE t1b NOT IN (SELECT t2b FROM t2 WHERE t2b > 6 LIMIT 2) --- !query 7 schema +-- !query 6 schema struct --- !query 7 output +-- !query 6 output val1a 16 12 10 15.0 20.0 2000 2014-07-04 01:01:00 2014-07-04 val1a 16 12 21 15.0 20.0 2000 2014-06-04 01:02:00.001 2014-06-04 val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:00:00 2014-04-04 val1a 6 8 10 15.0 20.0 2000 2014-04-04 01:02:00.001 2014-04-04 --- !query 8 +-- !query 7 SELECT Count(DISTINCT( t1a )), t1b FROM t1 @@ -149,7 +141,7 @@ WHERE t1d NOT IN (SELECT t2d GROUP BY t1b ORDER BY t1b NULLS last LIMIT 1 --- !query 8 schema +-- !query 7 schema struct --- !query 8 output +-- !query 7 output 1 6 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index ed110f751645..d0106c44b7db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -557,13 +557,11 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { } test("SPARK-18004 limit + aggregates") { - withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "true") { - val df = Seq(("a", 1), ("b", 2), ("c", 1), ("d", 5)).toDF("id", "value").repartition(1) - val limit2Df = df.limit(2) - checkAnswer( - limit2Df.groupBy("id").count().select($"id"), - limit2Df.select($"id")) - } + val df = Seq(("a", 1), ("b", 2), ("c", 1), ("d", 5)).toDF("id", "value") + val limit2Df = df.limit(2) + checkAnswer( + limit2Df.groupBy("id").count().select($"id"), + limit2Df.select($"id")) } test("SPARK-17237 remove backticks in a pivot result schema") { 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 f001b138f4b8..279b7b8d49f5 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 @@ -31,7 +31,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.Uuid import org.apache.spark.sql.catalyst.plans.logical.{Filter, OneRowRelation, Union} -import org.apache.spark.sql.execution.{FilterExec, QueryExecution, TakeOrderedAndProjectExec, WholeStageCodegenExec} +import org.apache.spark.sql.execution.{FilterExec, QueryExecution, WholeStageCodegenExec} import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.functions._ @@ -2552,26 +2552,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } } - test("SPARK-25352: Ordered global limit when more than topKSortFallbackThreshold ") { - withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "true") { - val baseDf = spark.range(1000).toDF.repartition(3).sort("id") - - withSQLConf(SQLConf.TOP_K_SORT_FALLBACK_THRESHOLD.key -> "100") { - val expected = baseDf.limit(99) - val takeOrderedNode1 = expected.queryExecution.executedPlan - .find(_.isInstanceOf[TakeOrderedAndProjectExec]) - assert(takeOrderedNode1.isDefined) - - val result = baseDf.limit(100) - val takeOrderedNode2 = result.queryExecution.executedPlan - .find(_.isInstanceOf[TakeOrderedAndProjectExec]) - assert(takeOrderedNode2.isEmpty) - - checkAnswer(expected, result.collect().take(99)) - } - } - } - test("SPARK-25368 Incorrect predicate pushdown returns wrong result") { def check(newCol: Column, filter: Column, result: Seq[Row]): Unit = { val df1 = spark.createDataFrame(Seq( 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 01dc28d70184..8fcebb35a054 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 @@ -524,15 +524,6 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sortTest() } - test("limit for skew dataframe") { - // Create a skew dataframe. - val df = testData.repartition(100).union(testData).limit(50) - // Because `rdd` of dataframe will add a `DeserializeToObject` on top of `GlobalLimit`, - // the `GlobalLimit` will not be replaced with `CollectLimit`. So we can test if `GlobalLimit` - // work on skew partitions. - assert(df.rdd.count() == 50L) - } - test("CTE feature") { checkAnswer( sql("with q1 as (select * from testData limit 10) select * from q1"), @@ -1944,7 +1935,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { // TODO: support subexpression elimination in whole stage codegen withSQLConf("spark.sql.codegen.wholeStage" -> "false") { // select from a table to prevent constant folding. - val df = sql("SELECT a, b from testData2 order by a, b limit 1") + val df = sql("SELECT a, b from testData2 limit 1") checkAnswer(df, Row(1, 1)) checkAnswer(df.selectExpr("a + 1", "a + 1"), Row(2, 2)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala index 41de731d41f8..b736d43bfc6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala @@ -50,7 +50,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { expectedPartitionStartIndices: Array[Int]): Unit = { val mapOutputStatistics = bytesByPartitionIdArray.zipWithIndex.map { case (bytesByPartitionId, index) => - new MapOutputStatistics(index, bytesByPartitionId, Array[Long](1)) + new MapOutputStatistics(index, bytesByPartitionId) } val estimatedPartitionStartIndices = coordinator.estimatePartitionStartIndices(mapOutputStatistics) @@ -114,8 +114,8 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0, 0) val mapOutputStatistics = Array( - new MapOutputStatistics(0, bytesByPartitionId1, Array[Long](0)), - new MapOutputStatistics(1, bytesByPartitionId2, Array[Long](0))) + new MapOutputStatistics(0, bytesByPartitionId1), + new MapOutputStatistics(1, bytesByPartitionId2)) intercept[AssertionError](coordinator.estimatePartitionStartIndices(mapOutputStatistics)) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/LimitSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/LimitSuite.scala deleted file mode 100644 index a7840a5fcfae..000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/LimitSuite.scala +++ /dev/null @@ -1,81 +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.execution - -import scala.util.Random - -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext - - -class LimitSuite extends SparkPlanTest with SharedSQLContext { - - private var rand: Random = _ - private var seed: Long = 0 - - protected override def beforeAll(): Unit = { - super.beforeAll() - seed = System.currentTimeMillis() - rand = new Random(seed) - } - - test("Produce ordered global limit if more than topKSortFallbackThreshold") { - withSQLConf(SQLConf.TOP_K_SORT_FALLBACK_THRESHOLD.key -> "100") { - val df = LimitTest.generateRandomInputData(spark, rand).sort("a") - - val globalLimit = df.limit(99).queryExecution.executedPlan.collect { - case g: GlobalLimitExec => g - } - assert(globalLimit.size == 0) - - val topKSort = df.limit(99).queryExecution.executedPlan.collect { - case t: TakeOrderedAndProjectExec => t - } - assert(topKSort.size == 1) - - val orderedGlobalLimit = df.limit(100).queryExecution.executedPlan.collect { - case g: GlobalLimitExec => g - } - assert(orderedGlobalLimit.size == 1 && orderedGlobalLimit(0).orderedLimit == true) - } - } - - test("Ordered global limit") { - val baseDf = LimitTest.generateRandomInputData(spark, rand) - .select("a").repartition(3).sort("a") - - withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "true") { - val orderedGlobalLimit = GlobalLimitExec(3, baseDf.queryExecution.sparkPlan, - orderedLimit = true) - val orderedGlobalLimitResult = SparkPlanTest.executePlan(orderedGlobalLimit, spark.sqlContext) - .map(_.getInt(0)) - - val globalLimit = GlobalLimitExec(3, baseDf.queryExecution.sparkPlan, orderedLimit = false) - val globalLimitResult = SparkPlanTest.executePlan(globalLimit, spark.sqlContext) - .map(_.getInt(0)) - - // Global limit without order takes values at each partition sequentially. - // After global sort, the values in second partition must be larger than the values - // in first partition. - assert(orderedGlobalLimitResult(0) == globalLimitResult(0)) - assert(orderedGlobalLimitResult(1) < globalLimitResult(1)) - assert(orderedGlobalLimitResult(2) < globalLimitResult(2)) - } - } -} - diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index b10da6c70be1..e4e224df7607 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -262,7 +262,7 @@ class PlannerSuite extends SharedSQLContext { ).queryExecution.executedPlan.collect { case exchange: ShuffleExchangeExec => exchange }.length - assert(numExchanges === 3) + assert(numExchanges === 5) } { @@ -277,7 +277,7 @@ class PlannerSuite extends SharedSQLContext { ).queryExecution.executedPlan.collect { case exchange: ShuffleExchangeExec => exchange }.length - assert(numExchanges === 3) + assert(numExchanges === 5) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala index 9322204063af..7e317a4d8026 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TakeOrderedAndProjectSuite.scala @@ -19,10 +19,9 @@ package org.apache.spark.sql.execution import scala.util.Random -import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -38,6 +37,14 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { rand = new Random(seed) } + private def generateRandomInputData(): DataFrame = { + val schema = new StructType() + .add("a", IntegerType, nullable = false) + .add("b", IntegerType, nullable = false) + val inputData = Seq.fill(10000)(Row(rand.nextInt(), rand.nextInt())) + spark.createDataFrame(sparkContext.parallelize(Random.shuffle(inputData), 10), schema) + } + /** * Adds a no-op filter to the child plan in order to prevent executeCollect() from being * called directly on the child plan. @@ -48,62 +55,32 @@ class TakeOrderedAndProjectSuite extends SparkPlanTest with SharedSQLContext { val sortOrder = 'a.desc :: 'b.desc :: Nil test("TakeOrderedAndProject.doExecute without project") { - withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "false") { - withClue(s"seed = $seed") { - checkThatPlansAgree( - LimitTest.generateRandomInputData(spark, rand), - input => - noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, input.output, input)), - input => - GlobalLimitExec(limit, - LocalLimitExec(limit, - SortExec(sortOrder, true, input))), - sortAnswers = false) - } + withClue(s"seed = $seed") { + checkThatPlansAgree( + generateRandomInputData(), + input => + noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, input.output, input)), + input => + GlobalLimitExec(limit, + LocalLimitExec(limit, + SortExec(sortOrder, true, input))), + sortAnswers = false) } } test("TakeOrderedAndProject.doExecute with project") { - withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "false") { - withClue(s"seed = $seed") { - checkThatPlansAgree( - LimitTest.generateRandomInputData(spark, rand), - input => - noOpFilter( - TakeOrderedAndProjectExec(limit, sortOrder, Seq(input.output.last), input)), - input => - GlobalLimitExec(limit, - LocalLimitExec(limit, - ProjectExec(Seq(input.output.last), - SortExec(sortOrder, true, input)))), - sortAnswers = false) - } - } - } - - test("TakeOrderedAndProject.doExecute equals to ordered global limit") { - withSQLConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT.key -> "true") { - withClue(s"seed = $seed") { - checkThatPlansAgree( - LimitTest.generateRandomInputData(spark, rand), - input => - noOpFilter(TakeOrderedAndProjectExec(limit, sortOrder, input.output, input)), - input => - GlobalLimitExec(limit, - LocalLimitExec(limit, - SortExec(sortOrder, true, input)), orderedLimit = true), - sortAnswers = false) - } + withClue(s"seed = $seed") { + checkThatPlansAgree( + generateRandomInputData(), + input => + noOpFilter( + TakeOrderedAndProjectExec(limit, sortOrder, Seq(input.output.last), input)), + input => + GlobalLimitExec(limit, + LocalLimitExec(limit, + ProjectExec(Seq(input.output.last), + SortExec(sortOrder, true, input)))), + sortAnswers = false) } } } - -object LimitTest { - def generateRandomInputData(spark: SparkSession, rand: Random): DataFrame = { - val schema = new StructType() - .add("a", IntegerType, nullable = false) - .add("b", IntegerType, nullable = false) - val inputData = Seq.fill(10000)(Row(rand.nextInt(), rand.nextInt())) - spark.createDataFrame(spark.sparkContext.parallelize(Random.shuffle(inputData), 10), schema) - } -} 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 b9b2b7dbf38e..cebaad5b4ad9 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 @@ -40,7 +40,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { private val originalColumnBatchSize = TestHive.conf.columnBatchSize private val originalInMemoryPartitionPruning = TestHive.conf.inMemoryPartitionPruning private val originalCrossJoinEnabled = TestHive.conf.crossJoinEnabled - private val originalLimitFlatGlobalLimit = TestHive.conf.limitFlatGlobalLimit private val originalSessionLocalTimeZone = TestHive.conf.sessionLocalTimeZone def testCases: Seq[(String, File)] = { @@ -60,8 +59,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, true) // Ensures that cross joins are enabled so that we can test them TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, true) - // Ensure that limit operation returns rows in the same order as Hive - TestHive.setConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT, false) // Fix session local timezone to America/Los_Angeles for those timezone sensitive tests // (timestamp_*) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, "America/Los_Angeles") @@ -76,7 +73,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { TestHive.setConf(SQLConf.COLUMN_BATCH_SIZE, originalColumnBatchSize) TestHive.setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, originalInMemoryPartitionPruning) TestHive.setConf(SQLConf.CROSS_JOINS_ENABLED, originalCrossJoinEnabled) - TestHive.setConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT, originalLimitFlatGlobalLimit) TestHive.setConf(SQLConf.SESSION_LOCAL_TIMEZONE, originalSessionLocalTimeZone) // For debugging dump some statistics about how much time was spent in various optimizer rules diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 16541295eb45..cc592cf6ca62 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -22,29 +22,21 @@ import scala.collection.JavaConverters._ import org.scalatest.BeforeAndAfter import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} -import org.apache.spark.sql.internal.SQLConf /** * A set of test cases that validate partition and column pruning. */ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { - private val originalLimitFlatGlobalLimit = TestHive.conf.limitFlatGlobalLimit - override def beforeAll(): Unit = { super.beforeAll() TestHive.setCacheTables(false) - TestHive.setConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT, false) // Column/partition pruning is not implemented for `InMemoryColumnarTableScan` yet, // need to reset the environment to ensure all referenced tables in this suites are // not cached in-memory. Refer to https://issues.apache.org/jira/browse/SPARK-2283 // for details. TestHive.reset() } - override def afterAll() { - TestHive.setConf(SQLConf.LIMIT_FLAT_GLOBAL_LIMIT, originalLimitFlatGlobalLimit) - super.afterAll() - } // Column pruning tests From 78dd1d859b6b9976653bb33413a5c58906c9b627 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Thu, 20 Sep 2018 20:33:44 +0800 Subject: [PATCH 082/879] [SPARK-25417][SQL] ArrayContains function may return incorrect result when right expression is implicitly down casted ## What changes were proposed in this pull request? In ArrayContains, we currently cast the right hand side expression to match the element type of the left hand side Array. This may result in down casting and may return wrong result or questionable result. Example : ```SQL spark-sql> select array_contains(array(1), 1.34); true ``` ```SQL spark-sql> select array_contains(array(1), 'foo'); null ``` We should safely coerce both left and right hand side expressions. ## How was this patch tested? Added tests in DataFrameFunctionsSuite Closes #22408 from dilipbiswal/SPARK-25417. Authored-by: Dilip Biswal Signed-off-by: Wenchen Fan (cherry picked from commit 67f2c6a55425d0f38e26caaf7e0b665d978d0a68) Signed-off-by: Wenchen Fan --- docs/sql-programming-guide.md | 61 ++++++++++++++++++- python/pyspark/sql/tests.py | 3 +- .../expressions/collectionOperations.scala | 28 +++++---- .../spark/sql/DataFrameFunctionsSuite.scala | 51 ++++++++++++++++ 4 files changed, 128 insertions(+), 15 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index c76f2e30e677..d2e3ee3e7781 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1879,6 +1879,66 @@ working with timestamps in `pandas_udf`s to get the best performance, see ## Upgrading From Spark SQL 2.3 to 2.4 + - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. + + + + + + + + + + + + + + + + + + + + + + + + + +
    + Query + + Result Spark 2.3 or Prior + + Result Spark 2.4 + + Remarks +
    + SELECT
    array_contains(array(1), 1.34D);
    +
    + true + + false + + In Spark 2.4, left and right parameters are promoted to array(double) and double type respectively. +
    + SELECT
    array_contains(array(1), '1');
    +
    + true + + AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. + + Users can use explict cast +
    + SELECT
    array_contains(array(1), 'anystring');
    +
    + null + + AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. + + Users can use explict cast +
    + - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. @@ -1912,7 +1972,6 @@ working with timestamps in `pandas_udf`s to get the best performance, see - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: - + + + + +
    diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 8724bbc6ca7c..9fa1577681f0 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -1498,8 +1498,7 @@ def test_array_contains_function(self): from pyspark.sql.functions import array_contains df = self.spark.createDataFrame([(["1", "2", "3"],), ([],)], ['data']) - actual = df.select(array_contains(df.data, 1).alias('b')).collect() - # The value argument can be implicitly castable to the element's type of the array. + actual = df.select(array_contains(df.data, "1").alias('b')).collect() self.assertEqual([Row(b=True), Row(b=False)], actual) def test_between_function(self): diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index cc9edcfd41d0..e23ebef9643f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -1331,23 +1331,27 @@ case class ArrayContains(left: Expression, right: Expression) @transient private lazy val ordering: Ordering[Any] = TypeUtils.getInterpretedOrdering(right.dataType) - override def inputTypes: Seq[AbstractDataType] = right.dataType match { - case NullType => Seq.empty - case _ => left.dataType match { - case n @ ArrayType(element, _) => Seq(n, element) + override def inputTypes: Seq[AbstractDataType] = { + (left.dataType, right.dataType) match { + case (_, NullType) => Seq.empty + case (ArrayType(e1, hasNull), e2) => + TypeCoercion.findTightestCommonType(e1, e2) match { + case Some(dt) => Seq(ArrayType(dt, hasNull), dt) + case _ => Seq.empty + } case _ => Seq.empty } } override def checkInputDataTypes(): TypeCheckResult = { - if (right.dataType == NullType) { - TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as arguments") - } else if (!left.dataType.isInstanceOf[ArrayType] - || !left.dataType.asInstanceOf[ArrayType].elementType.sameType(right.dataType)) { - TypeCheckResult.TypeCheckFailure( - "Arguments must be an array followed by a value of same type as the array members") - } else { - TypeUtils.checkForOrderingExpr(right.dataType, s"function $prettyName") + (left.dataType, right.dataType) match { + case (_, NullType) => + TypeCheckResult.TypeCheckFailure("Null typed values cannot be used as arguments") + case (ArrayType(e1, _), e2) if e1.sameType(e2) => + TypeUtils.checkForOrderingExpr(e2, s"function $prettyName") + case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " + + s"been ${ArrayType.simpleString} followed by a value with same element type, but it's " + + s"[${left.dataType.catalogString}, ${right.dataType.catalogString}].") } } 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 121db442c77f..ad52fd01248e 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 @@ -26,6 +26,7 @@ import scala.util.Random import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.plans.logical.OneRowRelation import org.apache.spark.sql.catalyst.util.DateTimeTestUtils import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -735,6 +736,56 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { df.selectExpr("array_contains(array(1, null), array(1, null)[0])"), Seq(Row(true), Row(true)) ) + + checkAnswer( + OneRowRelation().selectExpr("array_contains(array(1), 1.23D)"), + Seq(Row(false)) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_contains(array(1), 1.0D)"), + Seq(Row(true)) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_contains(array(1.0D), 1)"), + Seq(Row(true)) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_contains(array(1.23D), 1)"), + Seq(Row(false)) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_contains(array(array(1)), array(1.0D))"), + Seq(Row(true)) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_contains(array(array(1)), array(1.23D))"), + Seq(Row(false)) + ) + + val e1 = intercept[AnalysisException] { + OneRowRelation().selectExpr("array_contains(array(1), .01234567890123456790123456780)") + } + val errorMsg1 = + s""" + |Input to function array_contains should have been array followed by a + |value with same element type, but it's [array, decimal(29,29)]. + """.stripMargin.replace("\n", " ").trim() + assert(e1.message.contains(errorMsg1)) + + val e2 = intercept[AnalysisException] { + OneRowRelation().selectExpr("array_contains(array(1), 'foo')") + } + val errorMsg2 = + s""" + |Input to function array_contains should have been array followed by a + |value with same element type, but it's [array, string]. + """.stripMargin.replace("\n", " ").trim() + assert(e2.message.contains(errorMsg2)) } test("arrays_overlap function") { From fc036729c948391288a653219b6f916b1b82f871 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 21 Sep 2018 00:41:42 +0800 Subject: [PATCH 083/879] [MINOR][PYTHON] Use a helper in `PythonUtils` instead of direct accessing Scala package ## What changes were proposed in this pull request? This PR proposes to use add a helper in `PythonUtils` instead of direct accessing Scala package. ## How was this patch tested? Jenkins tests. Closes #22483 from HyukjinKwon/minor-refactoring. Authored-by: hyukjinkwon Signed-off-by: hyukjinkwon (cherry picked from commit 88e7e87bd5c052e10f52d4bb97a9d78f5b524128) Signed-off-by: hyukjinkwon --- .../main/scala/org/apache/spark/api/python/PythonUtils.scala | 4 ++++ python/pyspark/context.py | 4 +--- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index 27a5e19f96a1..cdce371dfcbf 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -74,4 +74,8 @@ private[spark] object PythonUtils { def toScalaMap[K, V](jm: java.util.Map[K, V]): Map[K, V] = { jm.asScala.toMap } + + def getEncryptionEnabled(sc: JavaSparkContext): Boolean = { + sc.conf.get(org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED) + } } diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 2c92c29a1cc1..87255c40e330 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -192,9 +192,7 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, # If encryption is enabled, we need to setup a server in the jvm to read broadcast # data via a socket. # scala's mangled names w/ $ in them require special treatment. - encryption_conf = self._jvm.org.apache.spark.internal.config.__getattr__("package$")\ - .__getattr__("MODULE$").IO_ENCRYPTION_ENABLED() - self._encryption_enabled = self._jsc.sc().conf().get(encryption_conf) + self._encryption_enabled = self._jvm.PythonUtils.getEncryptionEnabled(self._jsc) self.pythonExec = os.environ.get("PYSPARK_PYTHON", 'python') self.pythonVer = "%d.%d" % sys.version_info[:2] From c67c597b604c1c22334913c81e69f6237639e42e Mon Sep 17 00:00:00 2001 From: maryannxue Date: Thu, 20 Sep 2018 10:00:28 -0700 Subject: [PATCH 084/879] [SPARK-25450][SQL] PushProjectThroughUnion rule uses the same exprId for project expressions in each Union child, causing mistakes in constant propagation ## What changes were proposed in this pull request? The problem was cause by the PushProjectThroughUnion rule, which, when creating new Project for each child of Union, uses the same exprId for expressions of the same position. This is wrong because, for each child of Union, the expressions are all independent, and it can lead to a wrong result if other rules like FoldablePropagation kicks in, taking two different expressions as the same. This fix is to create new expressions in the new Project for each child of Union. ## How was this patch tested? Added UT. Closes #22447 from maryannxue/push-project-thru-union-bug. Authored-by: maryannxue Signed-off-by: gatorsmile (cherry picked from commit 88446b6ad19371f15d06ef67052f6c1a8072c04a) Signed-off-by: gatorsmile --- .../sql/catalyst/optimizer/Optimizer.scala | 4 ++ .../PushProjectThroughUnionSuite.scala | 54 +++++++++++++++++++ 2 files changed, 58 insertions(+) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushProjectThroughUnionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index e4b4f1ecbe21..b489a6745dd3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -486,6 +486,10 @@ object PushProjectionThroughUnion extends Rule[LogicalPlan] with PredicateHelper private def pushToRight[A <: Expression](e: A, rewrites: AttributeMap[Attribute]) = { val result = e transform { case a: Attribute => rewrites(a) + } match { + // Make sure exprId is unique in each child of Union. + case Alias(child, alias) => Alias(child, alias)() + case other => other } // We must promise the compiler that we did not discard the names in the case of project diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushProjectThroughUnionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushProjectThroughUnionSuite.scala new file mode 100644 index 000000000000..294d29842b04 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PushProjectThroughUnionSuite.scala @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +class PushProjectThroughUnionSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("Optimizer Batch", FixedPoint(100), + PushProjectionThroughUnion, + FoldablePropagation) :: Nil + } + + test("SPARK-25450 PushProjectThroughUnion rule uses the same exprId for project expressions " + + "in each Union child, causing mistakes in constant propagation") { + val testRelation1 = LocalRelation('a.string, 'b.int, 'c.string) + val testRelation2 = LocalRelation('d.string, 'e.int, 'f.string) + val query = testRelation1 + .union(testRelation2.select("bar".as("d"), 'e, 'f)) + .select('a.as("n")) + .select('n, "dummy").analyze + val optimized = Optimize.execute(query) + + val expected = testRelation1 + .select('a.as("n")) + .select('n, "dummy") + .union(testRelation2 + .select("bar".as("d"), 'e, 'f) + .select("bar".as("n")) + .select("bar".as("n"), "dummy")).analyze + + comparePlans(optimized, expected) + } +} From 43c62e7974445cc2fa8574fd6bf2dbfa94195a0c Mon Sep 17 00:00:00 2001 From: Nihar Sheth Date: Thu, 20 Sep 2018 11:52:20 -0700 Subject: [PATCH 085/879] [SPARK-24918][CORE] Executor Plugin API ## What changes were proposed in this pull request? A continuation of squito's executor plugin task. By his request I took his code and added testing and moved the plugin initialization to a separate thread. Executor plugins now run on one separate thread, so the executor does not wait on them. Added testing. ## How was this patch tested? Added test cases that test using a sample plugin. Closes #22192 from NiharS/executorPlugin. Lead-authored-by: Nihar Sheth Co-authored-by: NiharS Signed-off-by: Marcelo Vanzin (cherry picked from commit 2f51e72356babac703cc20a531b4dcc7712f34af) Signed-off-by: Marcelo Vanzin --- .../java/org/apache/spark/ExecutorPlugin.java | 57 +++++++ .../org/apache/spark/executor/Executor.scala | 35 +++++ .../spark/internal/config/package.scala | 10 ++ .../scala/org/apache/spark/util/Utils.scala | 13 ++ .../org/apache/spark/ExecutorPluginSuite.java | 139 ++++++++++++++++++ 5 files changed, 254 insertions(+) create mode 100644 core/src/main/java/org/apache/spark/ExecutorPlugin.java create mode 100644 core/src/test/java/org/apache/spark/ExecutorPluginSuite.java diff --git a/core/src/main/java/org/apache/spark/ExecutorPlugin.java b/core/src/main/java/org/apache/spark/ExecutorPlugin.java new file mode 100644 index 000000000000..ec0b57f1a281 --- /dev/null +++ b/core/src/main/java/org/apache/spark/ExecutorPlugin.java @@ -0,0 +1,57 @@ +/* + * 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; + +import org.apache.spark.annotation.DeveloperApi; + +/** + * A plugin which can be automaticaly instantiated within each Spark executor. Users can specify + * plugins which should be created with the "spark.executor.plugins" configuration. An instance + * of each plugin will be created for every executor, including those created by dynamic allocation, + * before the executor starts running any tasks. + * + * The specific api exposed to the end users still considered to be very unstable. We will + * hopefully be able to keep compatability by providing default implementations for any methods + * added, but make no guarantees this will always be possible across all Spark releases. + * + * Spark does nothing to verify the plugin is doing legitimate things, or to manage the resources + * it uses. A plugin acquires the same privileges as the user running the task. A bad plugin + * could also intefere with task execution and make the executor fail in unexpected ways. + */ +@DeveloperApi +public interface ExecutorPlugin { + + /** + * Initialize the executor plugin. + * + *

    Each executor will, during its initialization, invoke this method on each + * plugin provided in the spark.executor.plugins configuration.

    + * + *

    Plugins should create threads in their implementation of this method for + * any polling, blocking, or intensive computation.

    + */ + default void init() {} + + /** + * Clean up and terminate this plugin. + * + *

    This function is called during the executor shutdown phase. The executor + * will wait for the plugin to terminate before continuing its own shutdown.

    + */ + default void shutdown() {} +} diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 86b19578037d..27f7ec8fda68 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -136,6 +136,29 @@ private[spark] class Executor( // for fetching remote cached RDD blocks, so need to make sure it uses the right classloader too. env.serializerManager.setDefaultClassLoader(replClassLoader) + private val executorPlugins: Seq[ExecutorPlugin] = { + val pluginNames = conf.get(EXECUTOR_PLUGINS) + if (pluginNames.nonEmpty) { + logDebug(s"Initializing the following plugins: ${pluginNames.mkString(", ")}") + + // Plugins need to load using a class loader that includes the executor's user classpath + val pluginList: Seq[ExecutorPlugin] = + Utils.withContextClassLoader(replClassLoader) { + val plugins = Utils.loadExtensions(classOf[ExecutorPlugin], pluginNames, conf) + plugins.foreach { plugin => + plugin.init() + logDebug(s"Successfully loaded plugin " + plugin.getClass().getCanonicalName()) + } + plugins + } + + logDebug("Finished initializing plugins") + pluginList + } else { + Nil + } + } + // Max size of direct result. If task result is bigger than this, we use the block manager // to send the result back. private val maxDirectResultSize = Math.min( @@ -219,6 +242,18 @@ private[spark] class Executor( heartbeater.shutdown() heartbeater.awaitTermination(10, TimeUnit.SECONDS) threadPool.shutdown() + + // Notify plugins that executor is shutting down so they can terminate cleanly + Utils.withContextClassLoader(replClassLoader) { + executorPlugins.foreach { plugin => + try { + plugin.shutdown() + } catch { + case e: Exception => + logWarning("Plugin " + plugin.getClass().getCanonicalName() + " shutdown failed", e) + } + } + } if (!isLocal) { env.stop() } diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 7f1eb1e52461..e7238192b8f1 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -618,4 +618,14 @@ package object config { .intConf .checkValue(v => v > 0, "The max failures should be a positive value.") .createWithDefault(40) + + private[spark] val EXECUTOR_PLUGINS = + ConfigBuilder("spark.executor.plugins") + .doc("Comma-separated list of class names for \"plugins\" implementing " + + "org.apache.spark.ExecutorPlugin. Plugins have the same privileges as any task " + + "in a Spark executor. They can also interfere with task execution and fail in " + + "unexpected ways. So be sure to only use this for trusted plugins.") + .stringConf + .toSequence + .createWithDefault(Nil) } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 14f68cd6f350..c8b148be8453 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -239,6 +239,19 @@ private[spark] object Utils extends Logging { // scalastyle:on classforname } + /** + * Run a segment of code using a different context class loader in the current thread + */ + def withContextClassLoader[T](ctxClassLoader: ClassLoader)(fn: => T): T = { + val oldClassLoader = Thread.currentThread().getContextClassLoader() + try { + Thread.currentThread().setContextClassLoader(ctxClassLoader) + fn + } finally { + Thread.currentThread().setContextClassLoader(oldClassLoader) + } + } + /** * Primitive often used when writing [[java.nio.ByteBuffer]] to [[java.io.DataOutput]] */ diff --git a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java new file mode 100644 index 000000000000..686eb28010c6 --- /dev/null +++ b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java @@ -0,0 +1,139 @@ +/* + * 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; + +import org.apache.spark.api.java.JavaSparkContext; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class ExecutorPluginSuite { + private static final String EXECUTOR_PLUGIN_CONF_NAME = "spark.executor.plugins"; + private static final String testBadPluginName = TestBadShutdownPlugin.class.getName(); + private static final String testPluginName = TestExecutorPlugin.class.getName(); + private static final String testSecondPluginName = TestSecondPlugin.class.getName(); + + // Static value modified by testing plugins to ensure plugins loaded correctly. + public static int numSuccessfulPlugins = 0; + + // Static value modified by testing plugins to verify plugins shut down properly. + public static int numSuccessfulTerminations = 0; + + private JavaSparkContext sc; + + @Before + public void setUp() { + sc = null; + numSuccessfulPlugins = 0; + numSuccessfulTerminations = 0; + } + + @After + public void tearDown() { + if (sc != null) { + sc.stop(); + sc = null; + } + } + + private SparkConf initializeSparkConf(String pluginNames) { + return new SparkConf() + .setMaster("local") + .setAppName("test") + .set(EXECUTOR_PLUGIN_CONF_NAME, pluginNames); + } + + @Test + public void testPluginClassDoesNotExist() { + SparkConf conf = initializeSparkConf("nonexistant.plugin"); + try { + sc = new JavaSparkContext(conf); + fail("No exception thrown for nonexistant plugin"); + } catch (Exception e) { + // We cannot catch ClassNotFoundException directly because Java doesn't think it'll be thrown + assertTrue(e.toString().startsWith("java.lang.ClassNotFoundException")); + } + } + + @Test + public void testAddPlugin() throws InterruptedException { + // Load the sample TestExecutorPlugin, which will change the value of numSuccessfulPlugins + SparkConf conf = initializeSparkConf(testPluginName); + sc = new JavaSparkContext(conf); + assertEquals(1, numSuccessfulPlugins); + sc.stop(); + sc = null; + assertEquals(1, numSuccessfulTerminations); + } + + @Test + public void testAddMultiplePlugins() throws InterruptedException { + // Load two plugins and verify they both execute. + SparkConf conf = initializeSparkConf(testPluginName + "," + testSecondPluginName); + sc = new JavaSparkContext(conf); + assertEquals(2, numSuccessfulPlugins); + sc.stop(); + sc = null; + assertEquals(2, numSuccessfulTerminations); + } + + @Test + public void testPluginShutdownWithException() { + // Verify an exception in one plugin shutdown does not affect the others + String pluginNames = testPluginName + "," + testBadPluginName + "," + testPluginName; + SparkConf conf = initializeSparkConf(pluginNames); + sc = new JavaSparkContext(conf); + assertEquals(3, numSuccessfulPlugins); + sc.stop(); + sc = null; + assertEquals(2, numSuccessfulTerminations); + } + + public static class TestExecutorPlugin implements ExecutorPlugin { + public void init() { + ExecutorPluginSuite.numSuccessfulPlugins++; + } + + public void shutdown() { + ExecutorPluginSuite.numSuccessfulTerminations++; + } + } + + public static class TestSecondPlugin implements ExecutorPlugin { + public void init() { + ExecutorPluginSuite.numSuccessfulPlugins++; + } + + public void shutdown() { + ExecutorPluginSuite.numSuccessfulTerminations++; + } + } + + public static class TestBadShutdownPlugin implements ExecutorPlugin { + public void init() { + ExecutorPluginSuite.numSuccessfulPlugins++; + } + + public void shutdown() { + throw new RuntimeException("This plugin will fail to cleanly shut down"); + } + } +} From 51f3659b7eada3722d6ab491885dd34df5475187 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 20 Sep 2018 17:41:24 -0700 Subject: [PATCH 086/879] [SPARK-24777][SQL] Add write benchmark for AVRO ## What changes were proposed in this pull request? Refactor `DataSourceWriteBenchmark` and add write benchmark for AVRO. ## How was this patch tested? Build and run the benchmark. Closes #22451 from gengliangwang/avroWriteBenchmark. Authored-by: Gengliang Wang Signed-off-by: gatorsmile (cherry picked from commit 950ab79957fc0cdc2dafac94765787e87ece9e74) Signed-off-by: gatorsmile --- .../benchmark/AvroWriteBenchmark.scala | 40 ++++++++++ .../BuiltInDataSourceWriteBenchmark.scala | 79 +++++++++++++++++++ .../benchmark/DataSourceWriteBenchmark.scala | 75 +++--------------- 3 files changed, 131 insertions(+), 63 deletions(-) create mode 100644 external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala diff --git a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala new file mode 100644 index 000000000000..df13b4a1c2d3 --- /dev/null +++ b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala @@ -0,0 +1,40 @@ +/* + * 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.benchmark + +/** + * Benchmark to measure Avro data sources write performance. + * Usage: + * 1. with spark-submit: bin/spark-submit --class + * 2. with sbt: build/sbt "avro/test:runMain " + */ +object AvroWriteBenchmark extends DataSourceWriteBenchmark { + def main(args: Array[String]): Unit = { + /* + Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz + Avro writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Output Single Int Column 2481 / 2499 6.3 157.8 1.0X + Output Single Double Column 2705 / 2710 5.8 172.0 0.9X + Output Int and String Column 5539 / 5639 2.8 352.2 0.4X + Output Partitions 4613 / 5004 3.4 293.3 0.5X + Output Buckets 5554 / 5561 2.8 353.1 0.4X + */ + runBenchmark("Avro") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala new file mode 100644 index 000000000000..2de516c19da9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala @@ -0,0 +1,79 @@ +/* + * 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.benchmark + +/** + * Benchmark to measure built-in data sources write performance. + * By default it measures 4 data source format: Parquet, ORC, JSON, CSV. Run it with spark-submit: + * spark-submit --class + * Or with sbt: + * build/sbt "sql/test:runMain " + * + * To measure specified formats, run it with arguments: + * spark-submit --class format1 [format2] [...] + * Or with sbt: + * build/sbt "sql/test:runMain format1 [format2] [...]" + */ +object BuiltInDataSourceWriteBenchmark extends DataSourceWriteBenchmark { + def main(args: Array[String]): Unit = { + val formats: Seq[String] = if (args.isEmpty) { + Seq("Parquet", "ORC", "JSON", "CSV") + } else { + args + } + + spark.conf.set("spark.sql.parquet.compression.codec", "snappy") + spark.conf.set("spark.sql.orc.compression.codec", "snappy") + /* + Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz + Parquet writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Output Single Int Column 1815 / 1932 8.7 115.4 1.0X + Output Single Double Column 1877 / 1878 8.4 119.3 1.0X + Output Int and String Column 6265 / 6543 2.5 398.3 0.3X + Output Partitions 4067 / 4457 3.9 258.6 0.4X + Output Buckets 5608 / 5820 2.8 356.6 0.3X + + ORC writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Output Single Int Column 1201 / 1239 13.1 76.3 1.0X + Output Single Double Column 1542 / 1600 10.2 98.0 0.8X + Output Int and String Column 6495 / 6580 2.4 412.9 0.2X + Output Partitions 3648 / 3842 4.3 231.9 0.3X + Output Buckets 5022 / 5145 3.1 319.3 0.2X + + JSON writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Output Single Int Column 1988 / 2093 7.9 126.4 1.0X + Output Single Double Column 2854 / 2911 5.5 181.4 0.7X + Output Int and String Column 6467 / 6653 2.4 411.1 0.3X + Output Partitions 4548 / 5055 3.5 289.1 0.4X + Output Buckets 5664 / 5765 2.8 360.1 0.4X + + CSV writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative + ------------------------------------------------------------------------------------------------ + Output Single Int Column 3025 / 3190 5.2 192.3 1.0X + Output Single Double Column 3575 / 3634 4.4 227.3 0.8X + Output Int and String Column 7313 / 7399 2.2 464.9 0.4X + Output Partitions 5105 / 5190 3.1 324.6 0.6X + Output Buckets 6986 / 6992 2.3 444.1 0.4X + */ + formats.foreach { format => + runBenchmark(format) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceWriteBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceWriteBenchmark.scala index 2d2cdebd067c..e3463d9e28ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceWriteBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceWriteBenchmark.scala @@ -21,25 +21,14 @@ import org.apache.spark.sql.SparkSession import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Benchmark -/** - * Benchmark to measure data source write performance. - * By default it measures 4 data source format: Parquet, ORC, JSON, CSV: - * spark-submit --class - * To measure specified formats, run it with arguments: - * spark-submit --class format1 [format2] [...] - */ -object DataSourceWriteBenchmark { +trait DataSourceWriteBenchmark { val conf = new SparkConf() .setAppName("DataSourceWriteBenchmark") .setIfMissing("spark.master", "local[1]") - .set("spark.sql.parquet.compression.codec", "snappy") - .set("spark.sql.orc.compression.codec", "snappy") + .set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") val spark = SparkSession.builder.config(conf).getOrCreate() - // Set default configs. Individual cases will change them if necessary. - spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - val tempTable = "temp" val numRows = 1024 * 1024 * 15 @@ -86,64 +75,24 @@ object DataSourceWriteBenchmark { } } - def main(args: Array[String]): Unit = { + def runBenchmark(format: String): Unit = { val tableInt = "tableInt" val tableDouble = "tableDouble" val tableIntString = "tableIntString" val tablePartition = "tablePartition" val tableBucket = "tableBucket" - val formats: Seq[String] = if (args.isEmpty) { - Seq("Parquet", "ORC", "JSON", "CSV") - } else { - args - } - /* - Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz - Parquet writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Output Single Int Column 1815 / 1932 8.7 115.4 1.0X - Output Single Double Column 1877 / 1878 8.4 119.3 1.0X - Output Int and String Column 6265 / 6543 2.5 398.3 0.3X - Output Partitions 4067 / 4457 3.9 258.6 0.4X - Output Buckets 5608 / 5820 2.8 356.6 0.3X - - ORC writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Output Single Int Column 1201 / 1239 13.1 76.3 1.0X - Output Single Double Column 1542 / 1600 10.2 98.0 0.8X - Output Int and String Column 6495 / 6580 2.4 412.9 0.2X - Output Partitions 3648 / 3842 4.3 231.9 0.3X - Output Buckets 5022 / 5145 3.1 319.3 0.2X - - JSON writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Output Single Int Column 1988 / 2093 7.9 126.4 1.0X - Output Single Double Column 2854 / 2911 5.5 181.4 0.7X - Output Int and String Column 6467 / 6653 2.4 411.1 0.3X - Output Partitions 4548 / 5055 3.5 289.1 0.4X - Output Buckets 5664 / 5765 2.8 360.1 0.4X - - CSV writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Output Single Int Column 3025 / 3190 5.2 192.3 1.0X - Output Single Double Column 3575 / 3634 4.4 227.3 0.8X - Output Int and String Column 7313 / 7399 2.2 464.9 0.4X - Output Partitions 5105 / 5190 3.1 324.6 0.6X - Output Buckets 6986 / 6992 2.3 444.1 0.4X - */ withTempTable(tempTable) { spark.range(numRows).createOrReplaceTempView(tempTable) - formats.foreach { format => - withTable(tableInt, tableDouble, tableIntString, tablePartition, tableBucket) { - val benchmark = new Benchmark(s"$format writer benchmark", numRows) - writeNumeric(tableInt, format, benchmark, "Int") - writeNumeric(tableDouble, format, benchmark, "Double") - writeIntString(tableIntString, format, benchmark) - writePartition(tablePartition, format, benchmark) - writeBucket(tableBucket, format, benchmark) - benchmark.run() - } + withTable(tableInt, tableDouble, tableIntString, tablePartition, tableBucket) { + val benchmark = new Benchmark(s"$format writer benchmark", numRows) + writeNumeric(tableInt, format, benchmark, "Int") + writeNumeric(tableDouble, format, benchmark, "Double") + writeIntString(tableIntString, format, benchmark) + writePartition(tablePartition, format, benchmark) + writeBucket(tableBucket, format, benchmark) + benchmark.run() } } } } + From 5d744499667fcd08825bca0ac6d5d90d6e110ebc Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 21 Sep 2018 10:39:45 +0800 Subject: [PATCH 087/879] Revert "[SPARK-23715][SQL] the input of to/from_utc_timestamp can not have timezone ## What changes were proposed in this pull request? This reverts commit 417ad92502e714da71552f64d0e1257d2fd5d3d0. We decided to keep the current behaviors unchanged and will consider whether we will deprecate the these functions in 3.0. For more details, see the discussion in https://issues.apache.org/jira/browse/SPARK-23715 ## How was this patch tested? The existing tests. Closes #22505 from gatorsmile/revertSpark-23715. Authored-by: gatorsmile Signed-off-by: Wenchen Fan (cherry picked from commit 5d25e154408f71d24c4829165a16014fdacdd209) Signed-off-by: Wenchen Fan --- docs/sql-programming-guide.md | 1 - .../sql/catalyst/analysis/TypeCoercion.scala | 30 +--- .../expressions/datetimeExpressions.scala | 42 ------ .../sql/catalyst/util/DateTimeUtils.scala | 22 +-- .../apache/spark/sql/internal/SQLConf.scala | 7 - .../catalyst/analysis/TypeCoercionSuite.scala | 12 +- .../resources/sql-tests/inputs/datetime.sql | 33 ----- .../sql-tests/results/datetime.sql.out | 135 +----------------- .../apache/spark/sql/DateFunctionsSuite.scala | 9 -- 9 files changed, 13 insertions(+), 278 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d2e3ee3e7781..71ca25a42d63 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1953,7 +1953,6 @@ working with timestamps in `pandas_udf`s to get the best performance, see - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. - - In version 2.3 and earlier, `to_utc_timestamp` and `from_utc_timestamp` respect the timezone in the input timestamp string, which breaks the assumption that the input timestamp is in a specific timezone. Therefore, these 2 functions can return unexpected results. In version 2.4 and later, this problem has been fixed. `to_utc_timestamp` and `from_utc_timestamp` will return null if the input timestamp string contains timezone. As an example, `from_utc_timestamp('2000-10-10 00:00:00', 'GMT+1')` will return `2000-10-10 01:00:00` in both Spark 2.3 and 2.4. However, `from_utc_timestamp('2000-10-10 00:00:00+00:00', 'GMT+1')`, assuming a local timezone of GMT+8, will return `2000-10-10 09:00:00` in Spark 2.3 but `null` in 2.4. For people who don't care about this problem and want to retain the previous behavior to keep their query unchanged, you can set `spark.sql.function.rejectTimezoneInString` to false. This option will be removed in Spark 3.0 and should only be used as a temporary workaround. - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 288b6358fbff..49d286f6cf12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -60,7 +60,7 @@ object TypeCoercion { IfCoercion :: StackCoercion :: Division :: - new ImplicitTypeCasts(conf) :: + ImplicitTypeCasts :: DateTimeOperations :: WindowFrameCoercion :: Nil @@ -841,33 +841,12 @@ object TypeCoercion { /** * Casts types according to the expected input types for [[Expression]]s. */ - class ImplicitTypeCasts(conf: SQLConf) extends TypeCoercionRule { - - private def rejectTzInString = conf.getConf(SQLConf.REJECT_TIMEZONE_IN_STRING) - + object ImplicitTypeCasts extends TypeCoercionRule { override protected def coerceTypes( plan: LogicalPlan): LogicalPlan = plan resolveExpressions { // Skip nodes who's children have not been resolved yet. case e if !e.childrenResolved => e - // Special rules for `from/to_utc_timestamp`. These 2 functions assume the input timestamp - // string is in a specific timezone, so the string itself should not contain timezone. - // TODO: We should move the type coercion logic to expressions instead of a central - // place to put all the rules. - case e: FromUTCTimestamp if e.left.dataType == StringType => - if (rejectTzInString) { - e.copy(left = StringToTimestampWithoutTimezone(e.left)) - } else { - e.copy(left = Cast(e.left, TimestampType)) - } - - case e: ToUTCTimestamp if e.left.dataType == StringType => - if (rejectTzInString) { - e.copy(left = StringToTimestampWithoutTimezone(e.left)) - } else { - e.copy(left = Cast(e.left, TimestampType)) - } - case b @ BinaryOperator(left, right) if left.dataType != right.dataType => findTightestCommonType(left.dataType, right.dataType).map { commonType => if (b.inputType.acceptsType(commonType)) { @@ -884,7 +863,7 @@ object TypeCoercion { case e: ImplicitCastInputTypes if e.inputTypes.nonEmpty => val children: Seq[Expression] = e.children.zip(e.inputTypes).map { case (in, expected) => // If we cannot do the implicit cast, just use the original input. - ImplicitTypeCasts.implicitCast(in, expected).getOrElse(in) + implicitCast(in, expected).getOrElse(in) } e.withNewChildren(children) @@ -900,9 +879,6 @@ object TypeCoercion { } e.withNewChildren(children) } - } - - object ImplicitTypeCasts { /** * Given an expected data type, try to cast the expression and return the cast expression. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index f95798d64db1..eb78e394f985 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1017,48 +1017,6 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S } } -/** - * A special expression used to convert the string input of `to/from_utc_timestamp` to timestamp, - * which requires the timestamp string to not have timezone information, otherwise null is returned. - */ -case class StringToTimestampWithoutTimezone(child: Expression, timeZoneId: Option[String] = None) - extends UnaryExpression with TimeZoneAwareExpression with ExpectsInputTypes { - - override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = - copy(timeZoneId = Option(timeZoneId)) - - override def inputTypes: Seq[AbstractDataType] = Seq(StringType) - override def dataType: DataType = TimestampType - override def nullable: Boolean = true - override def toString: String = child.toString - override def sql: String = child.sql - - override def nullSafeEval(input: Any): Any = { - DateTimeUtils.stringToTimestamp( - input.asInstanceOf[UTF8String], timeZone, rejectTzInString = true).orNull - } - - override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - val tz = ctx.addReferenceObj("timeZone", timeZone) - val longOpt = ctx.freshName("longOpt") - val eval = child.genCode(ctx) - val code = code""" - |${eval.code} - |${CodeGenerator.JAVA_BOOLEAN} ${ev.isNull} = true; - |${CodeGenerator.JAVA_LONG} ${ev.value} = ${CodeGenerator.defaultValue(TimestampType)}; - |if (!${eval.isNull}) { - | scala.Option $longOpt = $dtu.stringToTimestamp(${eval.value}, $tz, true); - | if ($longOpt.isDefined()) { - | ${ev.value} = ((Long) $longOpt.get()).longValue(); - | ${ev.isNull} = false; - | } - |} - """.stripMargin - ev.copy(code = code) - } -} - /** * Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders * that time as a timestamp in the given time zone. For example, 'GMT+1' would yield 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 02813d393979..81d7274607ac 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 @@ -300,28 +300,10 @@ object DateTimeUtils { * `T[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us]+[h]h:[m]m` */ def stringToTimestamp(s: UTF8String): Option[SQLTimestamp] = { - stringToTimestamp(s, defaultTimeZone(), rejectTzInString = false) + stringToTimestamp(s, defaultTimeZone()) } def stringToTimestamp(s: UTF8String, timeZone: TimeZone): Option[SQLTimestamp] = { - stringToTimestamp(s, timeZone, rejectTzInString = false) - } - - /** - * Converts a timestamp string to microseconds from the unix epoch, w.r.t. the given timezone. - * Returns None if the input string is not a valid timestamp format. - * - * @param s the input timestamp string. - * @param timeZone the timezone of the timestamp string, will be ignored if the timestamp string - * already contains timezone information and `forceTimezone` is false. - * @param rejectTzInString if true, rejects timezone in the input string, i.e., if the - * timestamp string contains timezone, like `2000-10-10 00:00:00+00:00`, - * return None. - */ - def stringToTimestamp( - s: UTF8String, - timeZone: TimeZone, - rejectTzInString: Boolean): Option[SQLTimestamp] = { if (s == null) { return None } @@ -439,8 +421,6 @@ object DateTimeUtils { return None } - if (tz.isDefined && rejectTzInString) return None - val c = if (tz.isEmpty) { Calendar.getInstance(timeZone) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 169649c993ff..e63dbba305b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1388,13 +1388,6 @@ object SQLConf { .stringConf .createWithDefault("") - val REJECT_TIMEZONE_IN_STRING = buildConf("spark.sql.function.rejectTimezoneInString") - .internal() - .doc("If true, `to_utc_timestamp` and `from_utc_timestamp` return null if the input string " + - "contains a timezone part, e.g. `2000-10-10 00:00:00+00:00`.") - .booleanConf - .createWithDefault(true) - object PartitionOverwriteMode extends Enumeration { val STATIC, DYNAMIC = Value } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 461eda4334bb..1602f4d04611 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -680,11 +680,11 @@ class TypeCoercionSuite extends AnalysisTest { test("cast NullType for expressions that implement ExpectsInputTypes") { import TypeCoercionSuite._ - ruleTest(new TypeCoercion.ImplicitTypeCasts(conf), + ruleTest(TypeCoercion.ImplicitTypeCasts, AnyTypeUnaryExpression(Literal.create(null, NullType)), AnyTypeUnaryExpression(Literal.create(null, NullType))) - ruleTest(new TypeCoercion.ImplicitTypeCasts(conf), + ruleTest(TypeCoercion.ImplicitTypeCasts, NumericTypeUnaryExpression(Literal.create(null, NullType)), NumericTypeUnaryExpression(Literal.create(null, DoubleType))) } @@ -692,11 +692,11 @@ class TypeCoercionSuite extends AnalysisTest { test("cast NullType for binary operators") { import TypeCoercionSuite._ - ruleTest(new TypeCoercion.ImplicitTypeCasts(conf), + ruleTest(TypeCoercion.ImplicitTypeCasts, AnyTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType)), AnyTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType))) - ruleTest(new TypeCoercion.ImplicitTypeCasts(conf), + ruleTest(TypeCoercion.ImplicitTypeCasts, NumericTypeBinaryOperator(Literal.create(null, NullType), Literal.create(null, NullType)), NumericTypeBinaryOperator(Literal.create(null, DoubleType), Literal.create(null, DoubleType))) } @@ -976,7 +976,7 @@ class TypeCoercionSuite extends AnalysisTest { } test("type coercion for CaseKeyWhen") { - ruleTest(new TypeCoercion.ImplicitTypeCasts(conf), + ruleTest(TypeCoercion.ImplicitTypeCasts, CaseKeyWhen(Literal(1.toShort), Seq(Literal(1), Literal("a"))), CaseKeyWhen(Cast(Literal(1.toShort), IntegerType), Seq(Literal(1), Literal("a"))) ) @@ -1436,7 +1436,7 @@ class TypeCoercionSuite extends AnalysisTest { } test("SPARK-17117 null type coercion in divide") { - val rules = Seq(FunctionArgumentConversion, Division, new ImplicitTypeCasts(conf)) + val rules = Seq(FunctionArgumentConversion, Division, ImplicitTypeCasts) val nullLit = Literal.create(null, NullType) ruleTest(rules, Divide(1L, nullLit), Divide(Cast(1L, DoubleType), Cast(nullLit, DoubleType))) ruleTest(rules, Divide(nullLit, 1L), Divide(Cast(nullLit, DoubleType), Cast(1L, DoubleType))) diff --git a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql index 4950a4b7a4e5..547c2bef02b2 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -27,36 +27,3 @@ select current_date = current_date(), current_timestamp = current_timestamp(), a select a, b from ttf2 order by a, current_date; select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), weekday('1582-10-15 13:10:15'); - -select from_utc_timestamp('2015-07-24 00:00:00', 'PST'); - -select from_utc_timestamp('2015-01-24 00:00:00', 'PST'); - -select from_utc_timestamp(null, 'PST'); - -select from_utc_timestamp('2015-07-24 00:00:00', null); - -select from_utc_timestamp(null, null); - -select from_utc_timestamp(cast(0 as timestamp), 'PST'); - -select from_utc_timestamp(cast('2015-01-24' as date), 'PST'); - -select to_utc_timestamp('2015-07-24 00:00:00', 'PST'); - -select to_utc_timestamp('2015-01-24 00:00:00', 'PST'); - -select to_utc_timestamp(null, 'PST'); - -select to_utc_timestamp('2015-07-24 00:00:00', null); - -select to_utc_timestamp(null, null); - -select to_utc_timestamp(cast(0 as timestamp), 'PST'); - -select to_utc_timestamp(cast('2015-01-24' as date), 'PST'); - --- SPARK-23715: the input of to/from_utc_timestamp can not have timezone -select from_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST'); - -select to_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST'); diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index 9eede305dbdc..4e1cfa6e48c1 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 26 +-- Number of queries: 10 -- !query 0 @@ -82,138 +82,9 @@ struct 1 2 2 3 - -- !query 9 select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), weekday('1582-10-15 13:10:15') --- !query 9 schema +-- !query 3 schema struct --- !query 9 output +-- !query 3 output 5 3 5 NULL 4 - - --- !query 10 -select from_utc_timestamp('2015-07-24 00:00:00', 'PST') --- !query 10 schema -struct --- !query 10 output -2015-07-23 17:00:00 - - --- !query 11 -select from_utc_timestamp('2015-01-24 00:00:00', 'PST') --- !query 11 schema -struct --- !query 11 output -2015-01-23 16:00:00 - - --- !query 12 -select from_utc_timestamp(null, 'PST') --- !query 12 schema -struct --- !query 12 output -NULL - - --- !query 13 -select from_utc_timestamp('2015-07-24 00:00:00', null) --- !query 13 schema -struct --- !query 13 output -NULL - - --- !query 14 -select from_utc_timestamp(null, null) --- !query 14 schema -struct --- !query 14 output -NULL - - --- !query 15 -select from_utc_timestamp(cast(0 as timestamp), 'PST') --- !query 15 schema -struct --- !query 15 output -1969-12-31 08:00:00 - - --- !query 16 -select from_utc_timestamp(cast('2015-01-24' as date), 'PST') --- !query 16 schema -struct --- !query 16 output -2015-01-23 16:00:00 - - --- !query 17 -select to_utc_timestamp('2015-07-24 00:00:00', 'PST') --- !query 17 schema -struct --- !query 17 output -2015-07-24 07:00:00 - - --- !query 18 -select to_utc_timestamp('2015-01-24 00:00:00', 'PST') --- !query 18 schema -struct --- !query 18 output -2015-01-24 08:00:00 - - --- !query 19 -select to_utc_timestamp(null, 'PST') --- !query 19 schema -struct --- !query 19 output -NULL - - --- !query 20 -select to_utc_timestamp('2015-07-24 00:00:00', null) --- !query 20 schema -struct --- !query 20 output -NULL - - --- !query 21 -select to_utc_timestamp(null, null) --- !query 21 schema -struct --- !query 21 output -NULL - - --- !query 22 -select to_utc_timestamp(cast(0 as timestamp), 'PST') --- !query 22 schema -struct --- !query 22 output -1970-01-01 00:00:00 - - --- !query 23 -select to_utc_timestamp(cast('2015-01-24' as date), 'PST') --- !query 23 schema -struct --- !query 23 output -2015-01-24 08:00:00 - - --- !query 24 -select from_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST') --- !query 24 schema -struct --- !query 24 output -NULL - - --- !query 25 -select to_utc_timestamp('2000-10-10 00:00:00+00:00', 'PST') --- !query 25 schema -struct --- !query 25 output -NULL diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala index 3af80b36ec42..c4ec7150c407 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala @@ -23,7 +23,6 @@ import java.util.Locale import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.unsafe.types.CalendarInterval @@ -730,12 +729,4 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { Row(Timestamp.valueOf("2015-07-24 07:00:00")), Row(Timestamp.valueOf("2015-07-24 22:00:00")))) } - - test("SPARK-23715: to/from_utc_timestamp can retain the previous behavior") { - withSQLConf(SQLConf.REJECT_TIMEZONE_IN_STRING.key -> "false") { - checkAnswer( - sql("SELECT from_utc_timestamp('2000-10-10 00:00:00+00:00', 'GMT+1')"), - Row(Timestamp.valueOf("2000-10-09 18:00:00"))) - } - } } From aff6aedf9f4d5d4a4c961f5138bff893b4b1cb08 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 21 Sep 2018 14:17:34 +0800 Subject: [PATCH 088/879] [SPARK-25384][SQL] Clarify fromJsonForceNullableSchema will be removed in Spark 3.0 See above. This should go into the 2.4 release. Closes #22509 from rxin/SPARK-25384. Authored-by: Reynold Xin Signed-off-by: Wenchen Fan (cherry picked from commit fb3276a54a2b7339e5e0fb62fb01cbefcc330c8b) Signed-off-by: Wenchen Fan --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index e63dbba305b7..4b8ce74bfb80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -585,7 +585,7 @@ object SQLConf { .internal() .doc("When true, force the output schema of the from_json() function to be nullable " + "(including all the fields). Otherwise, the schema might not be compatible with" + - "actual data, which leads to curruptions.") + "actual data, which leads to corruptions. This config will be removed in Spark 3.0.") .booleanConf .createWithDefault(true) From e42546259160396d3ddca5d66d0032d18a039df2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 21 Sep 2018 14:27:14 +0800 Subject: [PATCH 089/879] [SPARK-23549][SQL] Rename config spark.sql.legacy.compareDateTimestampInTimestamp ## What changes were proposed in this pull request? See title. Makes our legacy backward compatibility configs more consistent. ## How was this patch tested? Make sure all references have been updated: ``` > git grep compareDateTimestampInTimestamp docs/sql-programming-guide.md: - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala: // if conf.compareDateTimestampInTimestamp is true sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala: => if (conf.compareDateTimestampInTimestamp) Some(TimestampType) else Some(StringType) sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala: => if (conf.compareDateTimestampInTimestamp) Some(TimestampType) else Some(StringType) sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala: buildConf("spark.sql.legacy.compareDateTimestampInTimestamp") sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala: def compareDateTimestampInTimestamp : Boolean = getConf(COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP) sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala: "spark.sql.legacy.compareDateTimestampInTimestamp" -> convertToTS.toString) { ``` Closes #22508 from rxin/SPARK-23549. Authored-by: Reynold Xin Signed-off-by: Wenchen Fan (cherry picked from commit 411ecc365ea62aef7a29d8764e783e6a58dbb1d5) Signed-off-by: Wenchen Fan --- docs/sql-programming-guide.md | 2 +- .../apache/spark/sql/internal/SQLConf.scala | 35 +++++++++---------- .../catalyst/analysis/TypeCoercionSuite.scala | 2 +- 3 files changed, 19 insertions(+), 20 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 71ca25a42d63..d1c4204bad71 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1948,7 +1948,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, an column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. - - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.hive.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. + - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 4b8ce74bfb80..610f11b00168 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -557,16 +557,6 @@ object SQLConf { .checkValues(HiveCaseSensitiveInferenceMode.values.map(_.toString)) .createWithDefault(HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString) - val TYPECOERCION_COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP = - buildConf("spark.sql.typeCoercion.compareDateTimestampInTimestamp") - .internal() - .doc("When true (default), compare Date with Timestamp after converting both sides to " + - "Timestamp. This behavior is compatible with Hive 2.2 or later. See HIVE-15236. " + - "When false, restore the behavior prior to Spark 2.4. Compare Date with Timestamp after " + - "converting both sides to string. This config will be removed in spark 3.0") - .booleanConf - .createWithDefault(true) - val OPTIMIZER_METADATA_ONLY = buildConf("spark.sql.optimizer.metadataOnly") .doc("When true, enable the metadata-only query optimization that use the table's metadata " + "to produce the partition columns instead of table scans. It applies when all the columns " + @@ -1456,12 +1446,6 @@ object SQLConf { .booleanConf .createWithDefault(true) - val LEGACY_SIZE_OF_NULL = buildConf("spark.sql.legacy.sizeOfNull") - .doc("If it is set to true, size of null returns -1. This behavior was inherited from Hive. " + - "The size function returns null for null input if the flag is disabled.") - .booleanConf - .createWithDefault(true) - val REPL_EAGER_EVAL_ENABLED = buildConf("spark.sql.repl.eagerEval.enabled") .doc("Enables eager evaluation or not. When true, the top K rows of Dataset will be " + "displayed if and only if the REPL supports the eager evaluation. Currently, the " + @@ -1511,6 +1495,22 @@ object SQLConf { .checkValues((1 to 9).toSet + Deflater.DEFAULT_COMPRESSION) .createWithDefault(Deflater.DEFAULT_COMPRESSION) + val COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP = + buildConf("spark.sql.legacy.compareDateTimestampInTimestamp") + .internal() + .doc("When true (default), compare Date with Timestamp after converting both sides to " + + "Timestamp. This behavior is compatible with Hive 2.2 or later. See HIVE-15236. " + + "When false, restore the behavior prior to Spark 2.4. Compare Date with Timestamp after " + + "converting both sides to string. This config will be removed in Spark 3.0.") + .booleanConf + .createWithDefault(true) + + val LEGACY_SIZE_OF_NULL = buildConf("spark.sql.legacy.sizeOfNull") + .doc("If it is set to true, size of null returns -1. This behavior was inherited from Hive. " + + "The size function returns null for null input if the flag is disabled.") + .booleanConf + .createWithDefault(true) + val LEGACY_REPLACE_DATABRICKS_SPARK_AVRO_ENABLED = buildConf("spark.sql.legacy.replaceDatabricksSparkAvro.enabled") .doc("If it is set to true, the data source provider com.databricks.spark.avro is mapped " + @@ -1660,8 +1660,7 @@ class SQLConf extends Serializable with Logging { def caseSensitiveInferenceMode: HiveCaseSensitiveInferenceMode.Value = HiveCaseSensitiveInferenceMode.withName(getConf(HIVE_CASE_SENSITIVE_INFERENCE)) - def compareDateTimestampInTimestamp : Boolean = - getConf(TYPECOERCION_COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP) + def compareDateTimestampInTimestamp : Boolean = getConf(COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP) def gatherFastStats: Boolean = getConf(GATHER_FASTSTAT) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 1602f4d04611..0594673ecc92 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -1459,7 +1459,7 @@ class TypeCoercionSuite extends AnalysisTest { DoubleType))) Seq(true, false).foreach { convertToTS => withSQLConf( - "spark.sql.typeCoercion.compareDateTimestampInTimestamp" -> convertToTS.toString) { + "spark.sql.legacy.compareDateTimestampInTimestamp" -> convertToTS.toString) { val date0301 = Literal(java.sql.Date.valueOf("2017-03-01")) val timestamp0301000000 = Literal(Timestamp.valueOf("2017-03-01 00:00:00")) val timestamp0301000001 = Literal(Timestamp.valueOf("2017-03-01 00:00:01")) From 604828eda0930b933be39d5db7bdb1b29d499f32 Mon Sep 17 00:00:00 2001 From: Marek Novotny Date: Fri, 21 Sep 2018 18:16:54 +0900 Subject: [PATCH 090/879] [SPARK-25469][SQL] Eval methods of Concat, Reverse and ElementAt should use pattern matching only once ## What changes were proposed in this pull request? The PR proposes to avoid usage of pattern matching for each call of ```eval``` method within: - ```Concat``` - ```Reverse``` - ```ElementAt``` ## How was this patch tested? Run the existing tests for ```Concat```, ```Reverse``` and ```ElementAt``` expression classes. Closes #22471 from mn-mikke/SPARK-25470. Authored-by: Marek Novotny Signed-off-by: Takeshi Yamamuro (cherry picked from commit 2c9d8f56c71093faf152ca7136c5fcc4a7b2a95f) Signed-off-by: Takeshi Yamamuro --- .../expressions/collectionOperations.scala | 81 +++++++++++-------- 1 file changed, 48 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index e23ebef9643f..161adc9cc5ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -1268,11 +1268,15 @@ case class Reverse(child: Expression) extends UnaryExpression with ImplicitCastI override def dataType: DataType = child.dataType - @transient private lazy val elementType: DataType = dataType.asInstanceOf[ArrayType].elementType + override def nullSafeEval(input: Any): Any = doReverse(input) - override def nullSafeEval(input: Any): Any = input match { - case a: ArrayData => new GenericArrayData(a.toObjectArray(elementType).reverse) - case s: UTF8String => s.reverse() + @transient private lazy val doReverse: Any => Any = dataType match { + case ArrayType(elementType, _) => + input => { + val arrayData = input.asInstanceOf[ArrayData] + new GenericArrayData(arrayData.toObjectArray(elementType).reverse) + } + case StringType => _.asInstanceOf[UTF8String].reverse() } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -1294,6 +1298,7 @@ case class Reverse(child: Expression) extends UnaryExpression with ImplicitCastI val i = ctx.freshName("i") val j = ctx.freshName("j") + val elementType = dataType.asInstanceOf[ArrayType].elementType val initialization = CodeGenerator.createArrayData( arrayData, elementType, numElements, s" $prettyName failed.") val assignment = CodeGenerator.createArrayAssignment( @@ -2164,9 +2169,11 @@ case class ElementAt(left: Expression, right: Expression) extends GetMapValueUti override def nullable: Boolean = true - override def nullSafeEval(value: Any, ordinal: Any): Any = { - left.dataType match { - case _: ArrayType => + override def nullSafeEval(value: Any, ordinal: Any): Any = doElementAt(value, ordinal) + + @transient private lazy val doElementAt: (Any, Any) => Any = left.dataType match { + case _: ArrayType => + (value, ordinal) => { val array = value.asInstanceOf[ArrayData] val index = ordinal.asInstanceOf[Int] if (array.numElements() < math.abs(index)) { @@ -2185,9 +2192,9 @@ case class ElementAt(left: Expression, right: Expression) extends GetMapValueUti array.get(idx, dataType) } } - case _: MapType => - getValueEval(value, ordinal, mapKeyType, ordering) - } + } + case _: MapType => + (value, ordinal) => getValueEval(value, ordinal, mapKeyType, ordering) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -2278,33 +2285,41 @@ case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpressio override def foldable: Boolean = children.forall(_.foldable) - override def eval(input: InternalRow): Any = dataType match { + override def eval(input: InternalRow): Any = doConcat(input) + + @transient private lazy val doConcat: InternalRow => Any = dataType match { case BinaryType => - val inputs = children.map(_.eval(input).asInstanceOf[Array[Byte]]) - ByteArray.concat(inputs: _*) + input => { + val inputs = children.map(_.eval(input).asInstanceOf[Array[Byte]]) + ByteArray.concat(inputs: _*) + } case StringType => - val inputs = children.map(_.eval(input).asInstanceOf[UTF8String]) - UTF8String.concat(inputs : _*) + input => { + val inputs = children.map(_.eval(input).asInstanceOf[UTF8String]) + UTF8String.concat(inputs: _*) + } case ArrayType(elementType, _) => - val inputs = children.toStream.map(_.eval(input)) - if (inputs.contains(null)) { - null - } else { - val arrayData = inputs.map(_.asInstanceOf[ArrayData]) - val numberOfElements = arrayData.foldLeft(0L)((sum, ad) => sum + ad.numElements()) - if (numberOfElements > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { - throw new RuntimeException(s"Unsuccessful try to concat arrays with $numberOfElements" + - " elements due to exceeding the array size limit " + - ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH + ".") - } - val finalData = new Array[AnyRef](numberOfElements.toInt) - var position = 0 - for(ad <- arrayData) { - val arr = ad.toObjectArray(elementType) - Array.copy(arr, 0, finalData, position, arr.length) - position += arr.length + input => { + val inputs = children.toStream.map(_.eval(input)) + if (inputs.contains(null)) { + null + } else { + val arrayData = inputs.map(_.asInstanceOf[ArrayData]) + val numberOfElements = arrayData.foldLeft(0L)((sum, ad) => sum + ad.numElements()) + if (numberOfElements > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + throw new RuntimeException(s"Unsuccessful try to concat arrays with $numberOfElements" + + " elements due to exceeding the array size limit " + + ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH + ".") + } + val finalData = new Array[AnyRef](numberOfElements.toInt) + var position = 0 + for (ad <- arrayData) { + val arr = ad.toObjectArray(elementType) + Array.copy(arr, 0, finalData, position, arr.length) + position += arr.length + } + new GenericArrayData(finalData) } - new GenericArrayData(finalData) } } From ce66361125ae89f9d9535c325ad82213ee04d7ad Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 21 Sep 2018 09:45:41 -0700 Subject: [PATCH 091/879] [SPARK-19724][SQL] allowCreatingManagedTableUsingNonemptyLocation should have legacy prefix One more legacy config to go ... Closes #22515 from rxin/allowCreatingManagedTableUsingNonemptyLocation. Authored-by: Reynold Xin Signed-off-by: gatorsmile (cherry picked from commit 4a11209539130c6a075119bf87c5ad854d42978e) Signed-off-by: gatorsmile --- docs/sql-programming-guide.md | 2 +- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d1c4204bad71..b5302bb7dd62 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1949,7 +1949,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, an column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. - - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. + - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 610f11b00168..2788402140f3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1338,7 +1338,7 @@ object SQLConf { .createWithDefault(false) val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = - buildConf("spark.sql.allowCreatingManagedTableUsingNonemptyLocation") + buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") .internal() .doc("When this option is set to true, creating managed tables with nonempty location " + "is allowed. Otherwise, an analysis exception is thrown. ") From 138a63165ce90f8400e0a5c7503894662ead03c5 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Fri, 21 Sep 2018 13:05:24 -0700 Subject: [PATCH 092/879] [SPARK-25321][ML] Revert SPARK-14681 to avoid API breaking change ## What changes were proposed in this pull request? Revert SPARK-14681 to avoid API breaking change. PR [SPARK-14681] will break mleap. ## How was this patch tested? N/A Closes #22492 from WeichenXu123/revert_tree_change. Authored-by: WeichenXu Signed-off-by: Xiangrui Meng --- .../DecisionTreeClassifier.scala | 14 +- .../ml/classification/GBTClassifier.scala | 6 +- .../RandomForestClassifier.scala | 6 +- .../ml/regression/DecisionTreeRegressor.scala | 13 +- .../spark/ml/regression/GBTRegressor.scala | 6 +- .../ml/regression/RandomForestRegressor.scala | 6 +- .../scala/org/apache/spark/ml/tree/Node.scala | 247 ++++-------------- .../spark/ml/tree/impl/RandomForest.scala | 10 +- .../org/apache/spark/ml/tree/treeModels.scala | 36 +-- .../DecisionTreeClassifierSuite.scala | 31 +-- .../classification/GBTClassifierSuite.scala | 4 +- .../RandomForestClassifierSuite.scala | 5 +- .../DecisionTreeRegressorSuite.scala | 14 - .../ml/tree/impl/RandomForestSuite.scala | 22 +- .../apache/spark/ml/tree/impl/TreeTests.scala | 12 +- project/MimaExcludes.scala | 7 - 16 files changed, 107 insertions(+), 332 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 8a57bfc029d1..6648e78d8eaf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -168,7 +168,7 @@ object DecisionTreeClassifier extends DefaultParamsReadable[DecisionTreeClassifi @Since("1.4.0") class DecisionTreeClassificationModel private[ml] ( @Since("1.4.0")override val uid: String, - @Since("1.4.0")override val rootNode: ClassificationNode, + @Since("1.4.0")override val rootNode: Node, @Since("1.6.0")override val numFeatures: Int, @Since("1.5.0")override val numClasses: Int) extends ProbabilisticClassificationModel[Vector, DecisionTreeClassificationModel] @@ -181,7 +181,7 @@ class DecisionTreeClassificationModel private[ml] ( * Construct a decision tree classification model. * @param rootNode Root node of tree, with other nodes attached. */ - private[ml] def this(rootNode: ClassificationNode, numFeatures: Int, numClasses: Int) = + private[ml] def this(rootNode: Node, numFeatures: Int, numClasses: Int) = this(Identifiable.randomUID("dtc"), rootNode, numFeatures, numClasses) override def predict(features: Vector): Double = { @@ -279,9 +279,8 @@ object DecisionTreeClassificationModel extends MLReadable[DecisionTreeClassifica val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] val numClasses = (metadata.metadata \ "numClasses").extract[Int] - val root = loadTreeNodes(path, metadata, sparkSession, isClassification = true) - val model = new DecisionTreeClassificationModel(metadata.uid, - root.asInstanceOf[ClassificationNode], numFeatures, numClasses) + val root = loadTreeNodes(path, metadata, sparkSession) + val model = new DecisionTreeClassificationModel(metadata.uid, root, numFeatures, numClasses) metadata.getAndSetParams(model) model } @@ -296,10 +295,9 @@ object DecisionTreeClassificationModel extends MLReadable[DecisionTreeClassifica require(oldModel.algo == OldAlgo.Classification, s"Cannot convert non-classification DecisionTreeModel (old API) to" + s" DecisionTreeClassificationModel (new API). Algo is: ${oldModel.algo}") - val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures, isClassification = true) + val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures) val uid = if (parent != null) parent.uid else Identifiable.randomUID("dtc") // Can't infer number of features from old model, so default to -1 - new DecisionTreeClassificationModel(uid, - rootNode.asInstanceOf[ClassificationNode], numFeatures, -1) + new DecisionTreeClassificationModel(uid, rootNode, numFeatures, -1) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 33acd9914073..62cfa39746ff 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -412,14 +412,14 @@ object GBTClassificationModel extends MLReadable[GBTClassificationModel] { override def load(path: String): GBTClassificationModel = { implicit val format = DefaultFormats val (metadata: Metadata, treesData: Array[(Metadata, Node)], treeWeights: Array[Double]) = - EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName, false) + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) val numFeatures = (metadata.metadata \ numFeaturesKey).extract[Int] val numTrees = (metadata.metadata \ numTreesKey).extract[Int] val trees: Array[DecisionTreeRegressionModel] = treesData.map { case (treeMetadata, root) => - val tree = new DecisionTreeRegressionModel(treeMetadata.uid, - root.asInstanceOf[RegressionNode], numFeatures) + val tree = + new DecisionTreeRegressionModel(treeMetadata.uid, root, numFeatures) treeMetadata.getAndSetParams(tree) tree } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 94887ac346fe..57132381b647 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -313,15 +313,15 @@ object RandomForestClassificationModel extends MLReadable[RandomForestClassifica override def load(path: String): RandomForestClassificationModel = { implicit val format = DefaultFormats val (metadata: Metadata, treesData: Array[(Metadata, Node)], _) = - EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName, true) + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] val numClasses = (metadata.metadata \ "numClasses").extract[Int] val numTrees = (metadata.metadata \ "numTrees").extract[Int] val trees: Array[DecisionTreeClassificationModel] = treesData.map { case (treeMetadata, root) => - val tree = new DecisionTreeClassificationModel(treeMetadata.uid, - root.asInstanceOf[ClassificationNode], numFeatures, numClasses) + val tree = + new DecisionTreeClassificationModel(treeMetadata.uid, root, numFeatures, numClasses) treeMetadata.getAndSetParams(tree) tree } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index 018290f81842..6fa656275c1f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -160,7 +160,7 @@ object DecisionTreeRegressor extends DefaultParamsReadable[DecisionTreeRegressor @Since("1.4.0") class DecisionTreeRegressionModel private[ml] ( override val uid: String, - override val rootNode: RegressionNode, + override val rootNode: Node, override val numFeatures: Int) extends PredictionModel[Vector, DecisionTreeRegressionModel] with DecisionTreeModel with DecisionTreeRegressorParams with MLWritable with Serializable { @@ -175,7 +175,7 @@ class DecisionTreeRegressionModel private[ml] ( * Construct a decision tree regression model. * @param rootNode Root node of tree, with other nodes attached. */ - private[ml] def this(rootNode: RegressionNode, numFeatures: Int) = + private[ml] def this(rootNode: Node, numFeatures: Int) = this(Identifiable.randomUID("dtr"), rootNode, numFeatures) override def predict(features: Vector): Double = { @@ -279,9 +279,8 @@ object DecisionTreeRegressionModel extends MLReadable[DecisionTreeRegressionMode implicit val format = DefaultFormats val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] - val root = loadTreeNodes(path, metadata, sparkSession, isClassification = false) - val model = new DecisionTreeRegressionModel(metadata.uid, - root.asInstanceOf[RegressionNode], numFeatures) + val root = loadTreeNodes(path, metadata, sparkSession) + val model = new DecisionTreeRegressionModel(metadata.uid, root, numFeatures) metadata.getAndSetParams(model) model } @@ -296,8 +295,8 @@ object DecisionTreeRegressionModel extends MLReadable[DecisionTreeRegressionMode require(oldModel.algo == OldAlgo.Regression, s"Cannot convert non-regression DecisionTreeModel (old API) to" + s" DecisionTreeRegressionModel (new API). Algo is: ${oldModel.algo}") - val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures, isClassification = false) + val rootNode = Node.fromOld(oldModel.topNode, categoricalFeatures) val uid = if (parent != null) parent.uid else Identifiable.randomUID("dtr") - new DecisionTreeRegressionModel(uid, rootNode.asInstanceOf[RegressionNode], numFeatures) + new DecisionTreeRegressionModel(uid, rootNode, numFeatures) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 3305881b0ccc..07f88d8d5f84 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -338,15 +338,15 @@ object GBTRegressionModel extends MLReadable[GBTRegressionModel] { override def load(path: String): GBTRegressionModel = { implicit val format = DefaultFormats val (metadata: Metadata, treesData: Array[(Metadata, Node)], treeWeights: Array[Double]) = - EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName, false) + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] val numTrees = (metadata.metadata \ "numTrees").extract[Int] val trees: Array[DecisionTreeRegressionModel] = treesData.map { case (treeMetadata, root) => - val tree = new DecisionTreeRegressionModel(treeMetadata.uid, - root.asInstanceOf[RegressionNode], numFeatures) + val tree = + new DecisionTreeRegressionModel(treeMetadata.uid, root, numFeatures) treeMetadata.getAndSetParams(tree) tree } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 35875724b3cf..82bf66ff66d8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -271,13 +271,13 @@ object RandomForestRegressionModel extends MLReadable[RandomForestRegressionMode override def load(path: String): RandomForestRegressionModel = { implicit val format = DefaultFormats val (metadata: Metadata, treesData: Array[(Metadata, Node)], treeWeights: Array[Double]) = - EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName, false) + EnsembleModelReadWrite.loadImpl(path, sparkSession, className, treeClassName) val numFeatures = (metadata.metadata \ "numFeatures").extract[Int] val numTrees = (metadata.metadata \ "numTrees").extract[Int] val trees: Array[DecisionTreeRegressionModel] = treesData.map { case (treeMetadata, root) => - val tree = new DecisionTreeRegressionModel(treeMetadata.uid, - root.asInstanceOf[RegressionNode], numFeatures) + val tree = + new DecisionTreeRegressionModel(treeMetadata.uid, root, numFeatures) treeMetadata.getAndSetParams(tree) tree } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala index 0242bc76698d..d30be452a436 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala @@ -17,16 +17,14 @@ package org.apache.spark.ml.tree -import org.apache.spark.annotation.Since import org.apache.spark.ml.linalg.Vector import org.apache.spark.mllib.tree.impurity.ImpurityCalculator -import org.apache.spark.mllib.tree.model.{ImpurityStats, InformationGainStats => OldInformationGainStats, - Node => OldNode, Predict => OldPredict} +import org.apache.spark.mllib.tree.model.{ImpurityStats, InformationGainStats => OldInformationGainStats, Node => OldNode, Predict => OldPredict} /** * Decision tree node interface. */ -sealed trait Node extends Serializable { +sealed abstract class Node extends Serializable { // TODO: Add aggregate stats (once available). This will happen after we move the DecisionTree // code into the new API and deprecate the old API. SPARK-3727 @@ -86,86 +84,35 @@ private[ml] object Node { /** * Create a new Node from the old Node format, recursively creating child nodes as needed. */ - def fromOld( - oldNode: OldNode, - categoricalFeatures: Map[Int, Int], - isClassification: Boolean): Node = { + def fromOld(oldNode: OldNode, categoricalFeatures: Map[Int, Int]): Node = { if (oldNode.isLeaf) { // TODO: Once the implementation has been moved to this API, then include sufficient // statistics here. - if (isClassification) { - new ClassificationLeafNode(prediction = oldNode.predict.predict, - impurity = oldNode.impurity, impurityStats = null) - } else { - new RegressionLeafNode(prediction = oldNode.predict.predict, - impurity = oldNode.impurity, impurityStats = null) - } + new LeafNode(prediction = oldNode.predict.predict, + impurity = oldNode.impurity, impurityStats = null) } else { val gain = if (oldNode.stats.nonEmpty) { oldNode.stats.get.gain } else { 0.0 } - if (isClassification) { - new ClassificationInternalNode(prediction = oldNode.predict.predict, - impurity = oldNode.impurity, gain = gain, - leftChild = fromOld(oldNode.leftNode.get, categoricalFeatures, true) - .asInstanceOf[ClassificationNode], - rightChild = fromOld(oldNode.rightNode.get, categoricalFeatures, true) - .asInstanceOf[ClassificationNode], - split = Split.fromOld(oldNode.split.get, categoricalFeatures), impurityStats = null) - } else { - new RegressionInternalNode(prediction = oldNode.predict.predict, - impurity = oldNode.impurity, gain = gain, - leftChild = fromOld(oldNode.leftNode.get, categoricalFeatures, false) - .asInstanceOf[RegressionNode], - rightChild = fromOld(oldNode.rightNode.get, categoricalFeatures, false) - .asInstanceOf[RegressionNode], - split = Split.fromOld(oldNode.split.get, categoricalFeatures), impurityStats = null) - } + new InternalNode(prediction = oldNode.predict.predict, impurity = oldNode.impurity, + gain = gain, leftChild = fromOld(oldNode.leftNode.get, categoricalFeatures), + rightChild = fromOld(oldNode.rightNode.get, categoricalFeatures), + split = Split.fromOld(oldNode.split.get, categoricalFeatures), impurityStats = null) } } } -@Since("2.4.0") -sealed trait ClassificationNode extends Node { - - /** - * Get count of training examples for specified label in this node - * @param label label number in the range [0, numClasses) - */ - @Since("2.4.0") - def getLabelCount(label: Int): Double = { - require(label >= 0 && label < impurityStats.stats.length, - "label should be in the range between 0 (inclusive) " + - s"and ${impurityStats.stats.length} (exclusive).") - impurityStats.stats(label) - } -} - -@Since("2.4.0") -sealed trait RegressionNode extends Node { - - /** Number of training data points in this node */ - @Since("2.4.0") - def getCount: Double = impurityStats.stats(0) - - /** Sum over training data points of the labels in this node */ - @Since("2.4.0") - def getSum: Double = impurityStats.stats(1) - - /** Sum over training data points of the square of the labels in this node */ - @Since("2.4.0") - def getSumOfSquares: Double = impurityStats.stats(2) -} - -@Since("2.4.0") -sealed trait LeafNode extends Node { - - /** Prediction this node makes. */ - def prediction: Double - - def impurity: Double +/** + * Decision tree leaf node. + * @param prediction Prediction this node makes + * @param impurity Impurity measure at this node (for training data) + */ +class LeafNode private[ml] ( + override val prediction: Double, + override val impurity: Double, + override private[ml] val impurityStats: ImpurityCalculator) extends Node { override def toString: String = s"LeafNode(prediction = $prediction, impurity = $impurity)" @@ -188,58 +135,32 @@ sealed trait LeafNode extends Node { override private[ml] def maxSplitFeatureIndex(): Int = -1 -} - -/** - * Decision tree leaf node for classification. - */ -@Since("2.4.0") -class ClassificationLeafNode private[ml] ( - override val prediction: Double, - override val impurity: Double, - override private[ml] val impurityStats: ImpurityCalculator) - extends ClassificationNode with LeafNode { - override private[tree] def deepCopy(): Node = { - new ClassificationLeafNode(prediction, impurity, impurityStats) + new LeafNode(prediction, impurity, impurityStats) } } /** - * Decision tree leaf node for regression. + * Internal Decision Tree node. + * @param prediction Prediction this node would make if it were a leaf node + * @param impurity Impurity measure at this node (for training data) + * @param gain Information gain value. Values less than 0 indicate missing values; + * this quirk will be removed with future updates. + * @param leftChild Left-hand child node + * @param rightChild Right-hand child node + * @param split Information about the test used to split to the left or right child. */ -@Since("2.4.0") -class RegressionLeafNode private[ml] ( +class InternalNode private[ml] ( override val prediction: Double, override val impurity: Double, - override private[ml] val impurityStats: ImpurityCalculator) - extends RegressionNode with LeafNode { - - override private[tree] def deepCopy(): Node = { - new RegressionLeafNode(prediction, impurity, impurityStats) - } -} - -/** - * Internal Decision Tree node. - */ -@Since("2.4.0") -sealed trait InternalNode extends Node { - - /** - * Information gain value. Values less than 0 indicate missing values; - * this quirk will be removed with future updates. - */ - def gain: Double - - /** Left-hand child node */ - def leftChild: Node - - /** Right-hand child node */ - def rightChild: Node + val gain: Double, + val leftChild: Node, + val rightChild: Node, + val split: Split, + override private[ml] val impurityStats: ImpurityCalculator) extends Node { - /** Information about the test used to split to the left or right child. */ - def split: Split + // Note to developers: The constructor argument impurityStats should be reconsidered before we + // make the constructor public. We may be able to improve the representation. override def toString: String = { s"InternalNode(prediction = $prediction, impurity = $impurity, split = $split)" @@ -284,6 +205,11 @@ sealed trait InternalNode extends Node { math.max(split.featureIndex, math.max(leftChild.maxSplitFeatureIndex(), rightChild.maxSplitFeatureIndex())) } + + override private[tree] def deepCopy(): Node = { + new InternalNode(prediction, impurity, gain, leftChild.deepCopy(), rightChild.deepCopy(), + split, impurityStats) + } } private object InternalNode { @@ -314,57 +240,6 @@ private object InternalNode { } } -/** - * Internal Decision Tree node for regression. - */ -@Since("2.4.0") -class ClassificationInternalNode private[ml] ( - override val prediction: Double, - override val impurity: Double, - override val gain: Double, - override val leftChild: ClassificationNode, - override val rightChild: ClassificationNode, - override val split: Split, - override private[ml] val impurityStats: ImpurityCalculator) - extends ClassificationNode with InternalNode { - - // Note to developers: The constructor argument impurityStats should be reconsidered before we - // make the constructor public. We may be able to improve the representation. - - override private[tree] def deepCopy(): Node = { - new ClassificationInternalNode(prediction, impurity, gain, - leftChild.deepCopy().asInstanceOf[ClassificationNode], - rightChild.deepCopy().asInstanceOf[ClassificationNode], - split, impurityStats) - } -} - -/** - * Internal Decision Tree node for regression. - */ -@Since("2.4.0") -class RegressionInternalNode private[ml] ( - override val prediction: Double, - override val impurity: Double, - override val gain: Double, - override val leftChild: RegressionNode, - override val rightChild: RegressionNode, - override val split: Split, - override private[ml] val impurityStats: ImpurityCalculator) - extends RegressionNode with InternalNode { - - // Note to developers: The constructor argument impurityStats should be reconsidered before we - // make the constructor public. We may be able to improve the representation. - - override private[tree] def deepCopy(): Node = { - new RegressionInternalNode(prediction, impurity, gain, - leftChild.deepCopy().asInstanceOf[RegressionNode], - rightChild.deepCopy().asInstanceOf[RegressionNode], - split, impurityStats) - } -} - - /** * Version of a node used in learning. This uses vars so that we can modify nodes as we split the * tree by adding children, etc. @@ -390,52 +265,30 @@ private[tree] class LearningNode( var isLeaf: Boolean, var stats: ImpurityStats) extends Serializable { - def toNode(isClassification: Boolean): Node = toNode(isClassification, prune = true) - - def toClassificationNode(prune: Boolean = true): ClassificationNode = { - toNode(true, prune).asInstanceOf[ClassificationNode] - } - - def toRegressionNode(prune: Boolean = true): RegressionNode = { - toNode(false, prune).asInstanceOf[RegressionNode] - } + def toNode: Node = toNode(prune = true) /** * Convert this [[LearningNode]] to a regular [[Node]], and recurse on any children. */ - def toNode(isClassification: Boolean, prune: Boolean): Node = { + def toNode(prune: Boolean = true): Node = { if (!leftChild.isEmpty || !rightChild.isEmpty) { assert(leftChild.nonEmpty && rightChild.nonEmpty && split.nonEmpty && stats != null, "Unknown error during Decision Tree learning. Could not convert LearningNode to Node.") - (leftChild.get.toNode(isClassification, prune), - rightChild.get.toNode(isClassification, prune)) match { + (leftChild.get.toNode(prune), rightChild.get.toNode(prune)) match { case (l: LeafNode, r: LeafNode) if prune && l.prediction == r.prediction => - if (isClassification) { - new ClassificationLeafNode(l.prediction, stats.impurity, stats.impurityCalculator) - } else { - new RegressionLeafNode(l.prediction, stats.impurity, stats.impurityCalculator) - } + new LeafNode(l.prediction, stats.impurity, stats.impurityCalculator) case (l, r) => - if (isClassification) { - new ClassificationInternalNode(stats.impurityCalculator.predict, stats.impurity, - stats.gain, l.asInstanceOf[ClassificationNode], r.asInstanceOf[ClassificationNode], - split.get, stats.impurityCalculator) - } else { - new RegressionInternalNode(stats.impurityCalculator.predict, stats.impurity, stats.gain, - l.asInstanceOf[RegressionNode], r.asInstanceOf[RegressionNode], - split.get, stats.impurityCalculator) - } + new InternalNode(stats.impurityCalculator.predict, stats.impurity, stats.gain, + l, r, split.get, stats.impurityCalculator) } } else { - // Here we want to keep same behavior with the old mllib.DecisionTreeModel - val impurity = if (stats.valid) stats.impurity else -1.0 - if (isClassification) { - new ClassificationLeafNode(stats.impurityCalculator.predict, impurity, + if (stats.valid) { + new LeafNode(stats.impurityCalculator.predict, stats.impurity, stats.impurityCalculator) } else { - new RegressionLeafNode(stats.impurityCalculator.predict, impurity, - stats.impurityCalculator) + // Here we want to keep same behavior with the old mllib.DecisionTreeModel + new LeafNode(stats.impurityCalculator.predict, -1.0, stats.impurityCalculator) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 4cdd17266b77..822abd2d3522 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -226,23 +226,23 @@ private[spark] object RandomForest extends Logging with Serializable { case Some(uid) => if (strategy.algo == OldAlgo.Classification) { topNodes.map { rootNode => - new DecisionTreeClassificationModel(uid, rootNode.toClassificationNode(prune), - numFeatures, strategy.getNumClasses) + new DecisionTreeClassificationModel(uid, rootNode.toNode(prune), numFeatures, + strategy.getNumClasses) } } else { topNodes.map { rootNode => - new DecisionTreeRegressionModel(uid, rootNode.toRegressionNode(prune), numFeatures) + new DecisionTreeRegressionModel(uid, rootNode.toNode(prune), numFeatures) } } case None => if (strategy.algo == OldAlgo.Classification) { topNodes.map { rootNode => - new DecisionTreeClassificationModel(rootNode.toClassificationNode(prune), numFeatures, + new DecisionTreeClassificationModel(rootNode.toNode(prune), numFeatures, strategy.getNumClasses) } } else { topNodes.map(rootNode => - new DecisionTreeRegressionModel(rootNode.toRegressionNode(prune), numFeatures)) + new DecisionTreeRegressionModel(rootNode.toNode(prune), numFeatures)) } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala index f027b14f1d47..4aa4c3617e7f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -219,10 +219,8 @@ private[ml] object TreeEnsembleModel { importances.changeValue(feature, scaledGain, _ + scaledGain) computeFeatureImportance(n.leftChild, importances) computeFeatureImportance(n.rightChild, importances) - case _: LeafNode => + case n: LeafNode => // do nothing - case _ => - throw new IllegalArgumentException(s"Unknown node type: ${node.getClass.toString}") } } @@ -319,8 +317,6 @@ private[ml] object DecisionTreeModelReadWrite { (Seq(NodeData(id, node.prediction, node.impurity, node.impurityStats.stats, -1.0, -1, -1, SplitData(-1, Array.empty[Double], -1))), id) - case _ => - throw new IllegalArgumentException(s"Unknown node type: ${node.getClass.toString}") } } @@ -331,7 +327,7 @@ private[ml] object DecisionTreeModelReadWrite { def loadTreeNodes( path: String, metadata: DefaultParamsReader.Metadata, - sparkSession: SparkSession, isClassification: Boolean): Node = { + sparkSession: SparkSession): Node = { import sparkSession.implicits._ implicit val format = DefaultFormats @@ -343,7 +339,7 @@ private[ml] object DecisionTreeModelReadWrite { val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath).as[NodeData] - buildTreeFromNodes(data.collect(), impurityType, isClassification) + buildTreeFromNodes(data.collect(), impurityType) } /** @@ -352,8 +348,7 @@ private[ml] object DecisionTreeModelReadWrite { * @param impurityType Impurity type for this tree * @return Root node of reconstructed tree */ - def buildTreeFromNodes(data: Array[NodeData], impurityType: String, - isClassification: Boolean): Node = { + def buildTreeFromNodes(data: Array[NodeData], impurityType: String): Node = { // Load all nodes, sorted by ID. val nodes = data.sortBy(_.id) // Sanity checks; could remove @@ -369,21 +364,10 @@ private[ml] object DecisionTreeModelReadWrite { val node = if (n.leftChild != -1) { val leftChild = finalNodes(n.leftChild) val rightChild = finalNodes(n.rightChild) - if (isClassification) { - new ClassificationInternalNode(n.prediction, n.impurity, n.gain, - leftChild.asInstanceOf[ClassificationNode], rightChild.asInstanceOf[ClassificationNode], - n.split.getSplit, impurityStats) - } else { - new RegressionInternalNode(n.prediction, n.impurity, n.gain, - leftChild.asInstanceOf[RegressionNode], rightChild.asInstanceOf[RegressionNode], - n.split.getSplit, impurityStats) - } + new InternalNode(n.prediction, n.impurity, n.gain, leftChild, rightChild, + n.split.getSplit, impurityStats) } else { - if (isClassification) { - new ClassificationLeafNode(n.prediction, n.impurity, impurityStats) - } else { - new RegressionLeafNode(n.prediction, n.impurity, impurityStats) - } + new LeafNode(n.prediction, n.impurity, impurityStats) } finalNodes(n.id) = node } @@ -437,8 +421,7 @@ private[ml] object EnsembleModelReadWrite { path: String, sql: SparkSession, className: String, - treeClassName: String, - isClassification: Boolean): (Metadata, Array[(Metadata, Node)], Array[Double]) = { + treeClassName: String): (Metadata, Array[(Metadata, Node)], Array[Double]) = { import sql.implicits._ implicit val format = DefaultFormats val metadata = DefaultParamsReader.loadMetadata(path, sql.sparkContext, className) @@ -466,8 +449,7 @@ private[ml] object EnsembleModelReadWrite { val rootNodesRDD: RDD[(Int, Node)] = nodeData.rdd.map(d => (d.treeID, d.nodeData)).groupByKey().map { case (treeID: Int, nodeData: Iterable[NodeData]) => - treeID -> DecisionTreeModelReadWrite.buildTreeFromNodes( - nodeData.toArray, impurityType, isClassification) + treeID -> DecisionTreeModelReadWrite.buildTreeFromNodes(nodeData.toArray, impurityType) } val rootNodes: Array[Node] = rootNodesRDD.sortByKey().values.collect() (metadata, treesMetadata.zip(rootNodes), treesWeights) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala index d3dbb4e754d3..2930f4900d50 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.tree.ClassificationLeafNode +import org.apache.spark.ml.tree.LeafNode import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} @@ -61,8 +61,7 @@ class DecisionTreeClassifierSuite extends MLTest with DefaultReadWriteTest { test("params") { ParamsSuite.checkParams(new DecisionTreeClassifier) - val model = new DecisionTreeClassificationModel("dtc", - new ClassificationLeafNode(0.0, 0.0, null), 1, 2) + val model = new DecisionTreeClassificationModel("dtc", new LeafNode(0.0, 0.0, null), 1, 2) ParamsSuite.checkParams(model) } @@ -376,32 +375,6 @@ class DecisionTreeClassifierSuite extends MLTest with DefaultReadWriteTest { testDefaultReadWrite(model) } - - test("label/impurity stats") { - val arr = Array( - LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 0.0)))), - LabeledPoint(1.0, Vectors.sparse(2, Seq((1, 1.0)))), - LabeledPoint(0.0, Vectors.sparse(2, Seq((0, 1.0))))) - val rdd = sc.parallelize(arr) - val df = TreeTests.setMetadata(rdd, Map.empty[Int, Int], 2) - val dt1 = new DecisionTreeClassifier() - .setImpurity("entropy") - .setMaxDepth(2) - .setMinInstancesPerNode(2) - val model1 = dt1.fit(df) - - val rootNode1 = model1.rootNode - assert(Array(rootNode1.getLabelCount(0), rootNode1.getLabelCount(1)) === Array(2.0, 1.0)) - - val dt2 = new DecisionTreeClassifier() - .setImpurity("gini") - .setMaxDepth(2) - .setMinInstancesPerNode(2) - val model2 = dt2.fit(df) - - val rootNode2 = model2.rootNode - assert(Array(rootNode2.getLabelCount(0), rootNode2.getLabelCount(1)) === Array(2.0, 1.0)) - } } private[ml] object DecisionTreeClassifierSuite extends SparkFunSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index e6d2a8e2b900..304977634189 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.regression.DecisionTreeRegressionModel -import org.apache.spark.ml.tree.RegressionLeafNode +import org.apache.spark.ml.tree.LeafNode import org.apache.spark.ml.tree.impl.{GradientBoostedTrees, TreeTests} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ @@ -70,7 +70,7 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { test("params") { ParamsSuite.checkParams(new GBTClassifier) val model = new GBTClassificationModel("gbtc", - Array(new DecisionTreeRegressionModel("dtr", new RegressionLeafNode(0.0, 0.0, null), 1)), + Array(new DecisionTreeRegressionModel("dtr", new LeafNode(0.0, 0.0, null), 1)), Array(1.0), 1, 2) ParamsSuite.checkParams(model) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index 3062aa9f3d27..ba4a9cf08278 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.tree.ClassificationLeafNode +import org.apache.spark.ml.tree.LeafNode import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} @@ -71,8 +71,7 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { test("params") { ParamsSuite.checkParams(new RandomForestClassifier) val model = new RandomForestClassificationModel("rfc", - Array(new DecisionTreeClassificationModel("dtc", - new ClassificationLeafNode(0.0, 0.0, null), 1, 2)), 2, 2) + Array(new DecisionTreeClassificationModel("dtc", new LeafNode(0.0, 0.0, null), 1, 2)), 2, 2) ParamsSuite.checkParams(model) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala index 9ae27339b11d..29a438396516 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala @@ -191,20 +191,6 @@ class DecisionTreeRegressorSuite extends MLTest with DefaultReadWriteTest { TreeTests.allParamSettings ++ Map("maxDepth" -> 0), TreeTests.allParamSettings ++ Map("maxDepth" -> 0), checkModelData) } - - test("label/impurity stats") { - val categoricalFeatures = Map(0 -> 2, 1 -> 2) - val df = TreeTests.setMetadata(categoricalDataPointsRDD, categoricalFeatures, numClasses = 0) - val dtr = new DecisionTreeRegressor() - .setImpurity("variance") - .setMaxDepth(2) - .setMaxBins(8) - val model = dtr.fit(df) - val statInfo = model.rootNode - - assert(statInfo.getCount == 1000.0 && statInfo.getSum == 600.0 - && statInfo.getSumOfSquares == 600.0) - } } private[ml] object DecisionTreeRegressorSuite extends SparkFunSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala index 4dbbd75d2466..743dacf146fe 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala @@ -340,8 +340,8 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { assert(topNode.stats.impurity > 0.0) // set impurity and predict for child nodes - assert(topNode.leftChild.get.toNode(isClassification = true).prediction === 0.0) - assert(topNode.rightChild.get.toNode(isClassification = true).prediction === 1.0) + assert(topNode.leftChild.get.toNode.prediction === 0.0) + assert(topNode.rightChild.get.toNode.prediction === 1.0) assert(topNode.leftChild.get.stats.impurity === 0.0) assert(topNode.rightChild.get.stats.impurity === 0.0) } @@ -382,8 +382,8 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { assert(topNode.stats.impurity > 0.0) // set impurity and predict for child nodes - assert(topNode.leftChild.get.toNode(isClassification = true).prediction === 0.0) - assert(topNode.rightChild.get.toNode(isClassification = true).prediction === 1.0) + assert(topNode.leftChild.get.toNode.prediction === 0.0) + assert(topNode.rightChild.get.toNode.prediction === 1.0) assert(topNode.leftChild.get.stats.impurity === 0.0) assert(topNode.rightChild.get.stats.impurity === 0.0) } @@ -582,18 +582,18 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { left right */ val leftImp = new GiniCalculator(Array(3.0, 2.0, 1.0)) - val left = new ClassificationLeafNode(0.0, leftImp.calculate(), leftImp) + val left = new LeafNode(0.0, leftImp.calculate(), leftImp) val rightImp = new GiniCalculator(Array(1.0, 2.0, 5.0)) - val right = new ClassificationLeafNode(2.0, rightImp.calculate(), rightImp) + val right = new LeafNode(2.0, rightImp.calculate(), rightImp) - val parent = TreeTests.buildParentNode(left, right, new ContinuousSplit(0, 0.5), true) + val parent = TreeTests.buildParentNode(left, right, new ContinuousSplit(0, 0.5)) val parentImp = parent.impurityStats val left2Imp = new GiniCalculator(Array(1.0, 6.0, 1.0)) - val left2 = new ClassificationLeafNode(0.0, left2Imp.calculate(), left2Imp) + val left2 = new LeafNode(0.0, left2Imp.calculate(), left2Imp) - val grandParent = TreeTests.buildParentNode(left2, parent, new ContinuousSplit(1, 1.0), true) + val grandParent = TreeTests.buildParentNode(left2, parent, new ContinuousSplit(1, 1.0)) val grandImp = grandParent.impurityStats // Test feature importance computed at different subtrees. @@ -618,8 +618,8 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { // Forest consisting of (full tree) + (internal node with 2 leafs) val trees = Array(parent, grandParent).map { root => - new DecisionTreeClassificationModel(root.asInstanceOf[ClassificationNode], - numFeatures = 2, numClasses = 3).asInstanceOf[DecisionTreeModel] + new DecisionTreeClassificationModel(root, numFeatures = 2, numClasses = 3) + .asInstanceOf[DecisionTreeModel] } val importances: Vector = TreeEnsembleModel.featureImportances(trees, 2) val tree2norm = feature0importance + feature1importance diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala index 3f03d909d4a4..b6894b30b0c2 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala @@ -159,7 +159,7 @@ private[ml] object TreeTests extends SparkFunSuite { * @param split Split for parent node * @return Parent node with children attached */ - def buildParentNode(left: Node, right: Node, split: Split, isClassification: Boolean): Node = { + def buildParentNode(left: Node, right: Node, split: Split): Node = { val leftImp = left.impurityStats val rightImp = right.impurityStats val parentImp = leftImp.copy.add(rightImp) @@ -168,15 +168,7 @@ private[ml] object TreeTests extends SparkFunSuite { val gain = parentImp.calculate() - (leftWeight * leftImp.calculate() + rightWeight * rightImp.calculate()) val pred = parentImp.predict - if (isClassification) { - new ClassificationInternalNode(pred, parentImp.calculate(), gain, - left.asInstanceOf[ClassificationNode], right.asInstanceOf[ClassificationNode], - split, parentImp) - } else { - new RegressionInternalNode(pred, parentImp.calculate(), gain, - left.asInstanceOf[RegressionNode], right.asInstanceOf[RegressionNode], - split, parentImp) - } + new InternalNode(pred, parentImp.calculate(), gain, left, right, split, parentImp) } /** diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 7ff783da130a..b7e9cbc077c1 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -93,13 +93,6 @@ object MimaExcludes { ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.defaultParamMap"), ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.ml.param.Params.org$apache$spark$ml$param$Params$_setter_$defaultParamMap_="), - // [SPARK-14681][ML] Provide label/impurity stats for spark.ml decision tree nodes - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.LeafNode"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.InternalNode"), - ProblemFilters.exclude[IncompatibleTemplateDefProblem]("org.apache.spark.ml.tree.Node"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.classification.DecisionTreeClassificationModel.this"), - ProblemFilters.exclude[IncompatibleMethTypeProblem]("org.apache.spark.ml.regression.DecisionTreeRegressionModel.this"), - // [SPARK-7132][ML] Add fit with validation set to spark.ml GBT ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.getValidationIndicatorCol"), ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.org$apache$spark$ml$param$shared$HasValidationIndicatorCol$_setter_$validationIndicatorCol_="), From 1303eb5c8d976748ba3da23b66abb8eb6512ea5d Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Fri, 21 Sep 2018 13:08:01 -0700 Subject: [PATCH 093/879] [SPARK-25321][ML] Fix local LDA model constructor ## What changes were proposed in this pull request? change back the constructor to: ``` class LocalLDAModel private[ml] ( uid: String, vocabSize: Int, private[clustering] val oldLocalModel : OldLocalLDAModel, sparkSession: SparkSession) ``` Although it is marked `private[ml]`, it is used in `mleap` and the master change breaks `mleap` building. See mleap code [here](https://github.com/combust/mleap/blob/c7860af328d519cf56441b4a7cd8e6ec9d9fee59/mleap-spark/src/main/scala/org/apache/spark/ml/bundle/ops/clustering/LDAModelOp.scala#L57) ## How was this patch tested? Manual. Closes #22510 from WeichenXu123/LDA_fix. Authored-by: WeichenXu Signed-off-by: Xiangrui Meng (cherry picked from commit 40edab209bdefe793b59b650099cea026c244484) Signed-off-by: Xiangrui Meng --- .../src/main/scala/org/apache/spark/ml/clustering/LDA.scala | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala index 50867f776c52..84e73dc19a39 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/LDA.scala @@ -570,13 +570,11 @@ abstract class LDAModel private[ml] ( class LocalLDAModel private[ml] ( uid: String, vocabSize: Int, - private[clustering] val oldLocalModel_ : OldLocalLDAModel, + private[clustering] val oldLocalModel : OldLocalLDAModel, sparkSession: SparkSession) extends LDAModel(uid, vocabSize, sparkSession) { - override private[clustering] def oldLocalModel: OldLocalLDAModel = { - oldLocalModel_.setSeed(getSeed) - } + oldLocalModel.setSeed(getSeed) @Since("1.6.0") override def copy(extra: ParamMap): LocalLDAModel = { From c64e7506dabaccc60f8140aeae589053645f23a6 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sun, 23 Sep 2018 10:16:33 +0800 Subject: [PATCH 094/879] [MINOR][PYSPARK] Always Close the tempFile in _serialize_to_jvm ## What changes were proposed in this pull request? Always close the tempFile after `serializer.dump_stream(data, tempFile)` in _serialize_to_jvm ## How was this patch tested? N/A Closes #22523 from gatorsmile/fixMinor. Authored-by: gatorsmile Signed-off-by: hyukjinkwon --- python/pyspark/context.py | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 87255c40e330..0924d3d95f04 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -537,8 +537,10 @@ def _serialize_to_jvm(self, data, serializer, reader_func, createRDDServer): # parallelize from there. tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) try: - serializer.dump_stream(data, tempFile) - tempFile.close() + try: + serializer.dump_stream(data, tempFile) + finally: + tempFile.close() return reader_func(tempFile.name) finally: # we eagerily reads the file so we can delete right after. From 36e7c8fcc1aeff0b15deb1243bd9615a202d320f Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 24 Sep 2018 19:25:02 +0800 Subject: [PATCH 095/879] [SPARKR] Match pyspark features in SparkR communication protocol --- R/pkg/R/context.R | 43 +++++++++++++------ R/pkg/tests/fulltests/test_Serde.R | 32 ++++++++++++++ R/pkg/tests/fulltests/test_sparkSQL.R | 12 ------ .../scala/org/apache/spark/api/r/RRDD.scala | 33 +++++++++++++- .../scala/org/apache/spark/api/r/RUtils.scala | 4 ++ 5 files changed, 98 insertions(+), 26 deletions(-) diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index f168ca76b600..e99136723f65 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -167,18 +167,30 @@ parallelize <- function(sc, coll, numSlices = 1) { # 2-tuples of raws serializedSlices <- lapply(slices, serialize, connection = NULL) - # The PRC backend cannot handle arguments larger than 2GB (INT_MAX) + # The RPC backend cannot handle arguments larger than 2GB (INT_MAX) # If serialized data is safely less than that threshold we send it over the PRC channel. # Otherwise, we write it to a file and send the file name if (objectSize < sizeLimit) { jrdd <- callJStatic("org.apache.spark.api.r.RRDD", "createRDDFromArray", sc, serializedSlices) } else { - fileName <- writeToTempFile(serializedSlices) - jrdd <- tryCatch(callJStatic( - "org.apache.spark.api.r.RRDD", "createRDDFromFile", sc, fileName, as.integer(numSlices)), - finally = { - file.remove(fileName) - }) + if (callJStatic("org.apache.spark.api.r.RUtils", "getEncryptionEnabled", sc)) { + # the length of slices here is the parallelism to use in the jvm's sc.parallelize() + parallelism <- as.integer(numSlices) + jserver <- newJObject("org.apache.spark.api.r.RParallelizeServer", sc, parallelism) + authSecret <- callJMethod(jserver, "secret") + port <- callJMethod(jserver, "port") + conn <- socketConnection(port = port, blocking = TRUE, open = "wb", timeout = 1500) + doServerAuth(conn, authSecret) + writeToConnection(serializedSlices, conn) + jrdd <- callJMethod(jserver, "getResult") + } else { + fileName <- writeToTempFile(serializedSlices) + jrdd <- tryCatch(callJStatic( + "org.apache.spark.api.r.RRDD", "createRDDFromFile", sc, fileName, as.integer(numSlices)), + finally = { + file.remove(fileName) + }) + } } RDD(jrdd, "byte") @@ -194,14 +206,21 @@ getMaxAllocationLimit <- function(sc) { )) } +writeToConnection <- function(serializedSlices, conn) { + tryCatch({ + for (slice in serializedSlices) { + writeBin(as.integer(length(slice)), conn, endian = "big") + writeBin(slice, conn, endian = "big") + } + }, finally = { + close(conn) + }) +} + writeToTempFile <- function(serializedSlices) { fileName <- tempfile() conn <- file(fileName, "wb") - for (slice in serializedSlices) { - writeBin(as.integer(length(slice)), conn, endian = "big") - writeBin(slice, conn, endian = "big") - } - close(conn) + writeToConnection(serializedSlices, conn) fileName } diff --git a/R/pkg/tests/fulltests/test_Serde.R b/R/pkg/tests/fulltests/test_Serde.R index 3577929323b8..1525bdb2f5c8 100644 --- a/R/pkg/tests/fulltests/test_Serde.R +++ b/R/pkg/tests/fulltests/test_Serde.R @@ -124,3 +124,35 @@ test_that("SerDe of list of lists", { }) sparkR.session.stop() + +# Note that this test should be at the end of tests since the configruations used here are not +# specific to sessions, and the Spark context is restarted. +test_that("createDataFrame large objects", { + for (encryptionEnabled in list("true", "false")) { + # To simulate a large object scenario, we set spark.r.maxAllocationLimit to a smaller value + conf <- list(spark.r.maxAllocationLimit = "100", + spark.io.encryption.enabled = encryptionEnabled) + + suppressWarnings(sparkR.session(master = sparkRTestMaster, + sparkConfig = conf, + enableHiveSupport = FALSE)) + + sc <- getSparkContext() + actual <- callJStatic("org.apache.spark.api.r.RUtils", "getEncryptionEnabled", sc) + expected <- as.logical(encryptionEnabled) + expect_equal(actual, expected) + + tryCatch({ + # suppress warnings from dot in the field names. See also SPARK-21536. + df <- suppressWarnings(createDataFrame(iris, numPartitions = 3)) + expect_equal(getNumPartitions(df), 3) + expect_equal(dim(df), dim(iris)) + + df <- createDataFrame(cars, numPartitions = 3) + expect_equal(collect(df), cars) + }, + finally = { + sparkR.stop() + }) + } +}) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 0c4bdb31b027..ce3922fa0350 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -316,18 +316,6 @@ test_that("create DataFrame from RDD", { unsetHiveContext() }) -test_that("createDataFrame uses files for large objects", { - # To simulate a large file scenario, we set spark.r.maxAllocationLimit to a smaller value - conf <- callJMethod(sparkSession, "conf") - callJMethod(conf, "set", "spark.r.maxAllocationLimit", "100") - df <- suppressWarnings(createDataFrame(iris, numPartitions = 3)) - expect_equal(getNumPartitions(df), 3) - - # Resetting the conf back to default value - callJMethod(conf, "set", "spark.r.maxAllocationLimit", toString(.Machine$integer.max / 10)) - expect_equal(dim(df), dim(iris)) -}) - test_that("read/write csv as DataFrame", { if (windows_with_hadoop()) { csvPath <- tempfile(pattern = "sparkr-test", fileext = ".csv") diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala index 295355c7bf01..1dc61c7eef33 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -17,7 +17,9 @@ package org.apache.spark.api.r -import java.io.File +import java.io.{DataInputStream, File} +import java.net.Socket +import java.nio.charset.StandardCharsets.UTF_8 import java.util.{Map => JMap} import scala.collection.JavaConverters._ @@ -25,10 +27,11 @@ import scala.reflect.ClassTag import org.apache.spark._ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} -import org.apache.spark.api.python.PythonRDD +import org.apache.spark.api.python.{PythonRDD, PythonServer} import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD +import org.apache.spark.security.SocketAuthHelper private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( parent: RDD[T], @@ -163,3 +166,29 @@ private[r] object RRDD { PythonRDD.readRDDFromFile(jsc, fileName, parallelism) } } + +/** + * Helper for making RDD[Array[Byte]] from some R data, by reading the data from R + * over a socket. This is used in preference to writing data to a file when encryption is enabled. + */ +private[spark] class RParallelizeServer(sc: JavaSparkContext, parallelism: Int) + extends PythonServer[JavaRDD[Array[Byte]]]( + new RSocketAuthHelper(), "sparkr-parallelize-server") { + + override def handleConnection(sock: Socket): JavaRDD[Array[Byte]] = { + val in = sock.getInputStream() + PythonRDD.readRDDFromInputStream(sc.sc, in, parallelism) + } +} + +private[spark] class RSocketAuthHelper extends SocketAuthHelper(SparkEnv.get.conf) { + override protected def readUtf8(s: Socket): String = { + val din = new DataInputStream(s.getInputStream()) + val len = din.readInt() + val bytes = new Array[Byte](len) + din.readFully(bytes) + // The R code adds a null terminator to serialized strings, so ignore it here. + assert(bytes(bytes.length - 1) == 0) // sanity check. + new String(bytes, 0, bytes.length - 1, UTF_8) + } +} diff --git a/core/src/main/scala/org/apache/spark/api/r/RUtils.scala b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala index fdd8cf62f0e5..9bf35af1da92 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala @@ -21,6 +21,8 @@ import java.io.File import java.util.Arrays import org.apache.spark.{SparkEnv, SparkException} +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.api.python.PythonUtils private[spark] object RUtils { // Local path where R binary packages built from R source code contained in the spark @@ -104,4 +106,6 @@ private[spark] object RUtils { case e: Exception => false } } + + def getEncryptionEnabled(sc: JavaSparkContext): Boolean = PythonUtils.getEncryptionEnabled(sc) } From 13bc58d28a7399ee778cdf0bd27f61095a967b97 Mon Sep 17 00:00:00 2001 From: Stan Zhai Date: Mon, 24 Sep 2018 21:33:12 +0800 Subject: [PATCH 096/879] [SPARK-21318][SQL] Improve exception message thrown by `lookupFunction` ## What changes were proposed in this pull request? The function actually exists in current selected database, and it's failed to init during `lookupFunciton`, but the exception message is: ``` This function is neither a registered temporary function nor a permanent function registered in the database 'default'. ``` This is not conducive to positioning problems. This PR fix the problem. ## How was this patch tested? new test case + manual tests Closes #18544 from stanzhai/fix-udf-error-message. Authored-by: Stan Zhai Signed-off-by: Wenchen Fan (cherry picked from commit 804515f821086ea685815d3c8eff42d76b7d9e4e) Signed-off-by: Wenchen Fan --- .../catalog/SessionCatalogSuite.scala | 3 ++ .../spark/sql/hive/HiveSessionCatalog.scala | 10 +++--- .../spark/sql/hive/execution/UDAFEmpty.java | 32 +++++++++++++++++++ .../org/apache/spark/sql/hive/UDFSuite.scala | 16 ++++++++++ 4 files changed, 56 insertions(+), 5 deletions(-) create mode 100644 sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDAFEmpty.java diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 89fabd477406..19e8c0334689 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -1427,6 +1427,7 @@ abstract class SessionCatalogSuite extends AnalysisTest { Seq(true, false) foreach { caseSensitive => val conf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> caseSensitive) val catalog = new SessionCatalog(newBasicCatalog(), new SimpleFunctionRegistry, conf) + catalog.setCurrentDatabase("db1") try { val analyzer = new Analyzer(catalog, conf) @@ -1440,6 +1441,8 @@ abstract class SessionCatalogSuite extends AnalysisTest { } assert(cause.getMessage.contains("Undefined function: 'undefined_fn'")) + // SPARK-21318: the error message should contains the current database name + assert(cause.getMessage.contains("db1")) } finally { catalog.reset() } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index de41bb418181..405c0c8bfe66 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -131,14 +131,14 @@ private[sql] class HiveSessionCatalog( Try(super.lookupFunction(funcName, children)) match { case Success(expr) => expr case Failure(error) => - if (functionRegistry.functionExists(funcName)) { - // If the function actually exists in functionRegistry, it means that there is an - // error when we create the Expression using the given children. + if (super.functionExists(name)) { + // If the function exists (either in functionRegistry or externalCatalog), + // it means that there is an error when we create the Expression using the given children. // We need to throw the original exception. throw error } else { - // This function is not in functionRegistry, let's try to load it as a Hive's - // built-in function. + // This function does not exist (neither in functionRegistry or externalCatalog), + // let's try to load it as a Hive's built-in function. // Hive is case insensitive. val functionName = funcName.unquotedString.toLowerCase(Locale.ROOT) if (!hiveFunctions.contains(functionName)) { diff --git a/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDAFEmpty.java b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDAFEmpty.java new file mode 100644 index 000000000000..badc396688f5 --- /dev/null +++ b/sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDAFEmpty.java @@ -0,0 +1,32 @@ +/* + * 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.execution; + +import org.apache.hadoop.hive.ql.parse.SemanticException; +import org.apache.hadoop.hive.ql.udf.generic.AbstractGenericUDAFResolver; +import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFEvaluator; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfo; + +/** + * An empty UDAF that throws a semantic exception + */ +public class UDAFEmpty extends AbstractGenericUDAFResolver { + @Override + public GenericUDAFEvaluator getEvaluator(TypeInfo[] info) throws SemanticException { + throw new SemanticException("Can not get an evaluator of the empty UDAF"); + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala index a56c6f73989a..d567128e1a32 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/UDFSuite.scala @@ -193,4 +193,20 @@ class UDFSuite } } } + + test("SPARK-21318: The correct exception message should be thrown " + + "if a UDF/UDAF has already been registered") { + val functionName = "empty" + val functionClass = classOf[org.apache.spark.sql.hive.execution.UDAFEmpty].getCanonicalName + + withUserDefinedFunction(functionName -> false) { + sql(s"CREATE FUNCTION $functionName AS '$functionClass'") + + val e = intercept[AnalysisException] { + sql(s"SELECT $functionName(value) from $testTableName") + } + + assert(e.getMessage.contains("Can not get an evaluator of the empty UDAF")) + } + } } From 51d5378f8956e0abc1bae0ffdea78637011935a1 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Mon, 24 Sep 2018 21:37:51 +0800 Subject: [PATCH 097/879] [SPARK-25416][SQL] ArrayPosition function may return incorrect result when right expression is implicitly down casted ## What changes were proposed in this pull request? In ArrayPosition, we currently cast the right hand side expression to match the element type of the left hand side Array. This may result in down casting and may return wrong result or questionable result. Example : ```SQL spark-sql> select array_position(array(1), 1.34); 1 ``` ```SQL spark-sql> select array_position(array(1), 'foo'); null ``` We should safely coerce both left and right hand side expressions. ## How was this patch tested? Added tests in DataFrameFunctionsSuite Closes #22407 from dilipbiswal/SPARK-25416. Authored-by: Dilip Biswal Signed-off-by: Wenchen Fan (cherry picked from commit bb49661e192eed78a8a306deffd83c73bd4a9eff) Signed-off-by: Wenchen Fan --- .../expressions/collectionOperations.scala | 21 ++++--- .../spark/sql/DataFrameFunctionsSuite.scala | 57 +++++++++++++++++-- 2 files changed, 64 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 161adc9cc5ba..85bc1cdb4305 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -2071,18 +2071,23 @@ case class ArrayPosition(left: Expression, right: Expression) override def dataType: DataType = LongType override def inputTypes: Seq[AbstractDataType] = { - val elementType = left.dataType match { - case t: ArrayType => t.elementType - case _ => AnyDataType + (left.dataType, right.dataType) match { + case (ArrayType(e1, hasNull), e2) => + TypeCoercion.findTightestCommonType(e1, e2) match { + case Some(dt) => Seq(ArrayType(dt, hasNull), dt) + case _ => Seq.empty + } + case _ => Seq.empty } - Seq(ArrayType, elementType) } override def checkInputDataTypes(): TypeCheckResult = { - super.checkInputDataTypes() match { - case f: TypeCheckResult.TypeCheckFailure => f - case TypeCheckResult.TypeCheckSuccess => - TypeUtils.checkForOrderingExpr(right.dataType, s"function $prettyName") + (left.dataType, right.dataType) match { + case (ArrayType(e1, _), e2) if e1.sameType(e2) => + TypeUtils.checkForOrderingExpr(e2, s"function $prettyName") + case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " + + s"been ${ArrayType.simpleString} followed by a value with same element type, but it's " + + s"[${left.dataType.catalogString}, ${right.dataType.catalogString}].") } } 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 ad52fd01248e..fd71f2493561 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 @@ -1097,18 +1097,63 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { ) checkAnswer( - df.selectExpr("array_position(array(array(1), null)[0], 1)"), - Seq(Row(1L), Row(1L)) + OneRowRelation().selectExpr("array_position(array(1), 1.23D)"), + Seq(Row(0L)) ) + checkAnswer( - df.selectExpr("array_position(array(1, null), array(1, null)[0])"), - Seq(Row(1L), Row(1L)) + OneRowRelation().selectExpr("array_position(array(1), 1.0D)"), + Seq(Row(1L)) ) - val e = intercept[AnalysisException] { + checkAnswer( + OneRowRelation().selectExpr("array_position(array(1.D), 1)"), + Seq(Row(1L)) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_position(array(1.23D), 1)"), + Seq(Row(0L)) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_position(array(array(1)), array(1.0D))"), + Seq(Row(1L)) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_position(array(array(1)), array(1.23D))"), + Seq(Row(0L)) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_position(array(array(1), null)[0], 1)"), + Seq(Row(1L)) + ) + checkAnswer( + OneRowRelation().selectExpr("array_position(array(1, null), array(1, null)[0])"), + Seq(Row(1L)) + ) + + val e1 = intercept[AnalysisException] { Seq(("a string element", "a")).toDF().selectExpr("array_position(_1, _2)") } - assert(e.message.contains("argument 1 requires array type, however, '`_1`' is of string type")) + val errorMsg1 = + s""" + |Input to function array_position should have been array followed by a + |value with same element type, but it's [string, string]. + """.stripMargin.replace("\n", " ").trim() + assert(e1.message.contains(errorMsg1)) + + val e2 = intercept[AnalysisException] { + OneRowRelation().selectExpr("array_position(array(1), '1')") + } + val errorMsg2 = + s""" + |Input to function array_position should have been array followed by a + |value with same element type, but it's [array, string]. + """.stripMargin.replace("\n", " ").trim() + assert(e2.message.contains(errorMsg2)) } test("element_at function") { From ec384284eb427d7573bd94c707777e23e4137971 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 24 Sep 2018 08:49:19 -0700 Subject: [PATCH 098/879] [SPARK-25460][BRANCH-2.4][SS] DataSourceV2: SS sources do not respect SessionConfigSupport ## What changes were proposed in this pull request? This PR proposes to backport SPARK-25460 to branch-2.4: This PR proposes to respect `SessionConfigSupport` in SS datasources as well. Currently these are only respected in batch sources: https://github.com/apache/spark/blob/e06da95cd9423f55cdb154a2778b0bddf7be984c/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala#L198-L203 https://github.com/apache/spark/blob/e06da95cd9423f55cdb154a2778b0bddf7be984c/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala#L244-L249 If a developer makes a datasource V2 that supports both structured streaming and batch jobs, batch jobs respect a specific configuration, let's say, URL to connect and fetch data (which end users might not be aware of); however, structured streaming ends up with not supporting this (and should explicitly be set into options). ## How was this patch tested? Unit tests were added. Closes #22529 from HyukjinKwon/SPARK-25460-backport. Authored-by: hyukjinkwon Signed-off-by: Dongjoon Hyun --- .../sql/streaming/DataStreamReader.scala | 18 ++- .../sql/streaming/DataStreamWriter.scala | 16 ++- .../sources/StreamingDataSourceV2Suite.scala | 118 +++++++++++++++--- 3 files changed, 125 insertions(+), 27 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala index 7eb5db5ed0ee..a9cb5e8fc8e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamReader.scala @@ -26,6 +26,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} import org.apache.spark.sql.sources.StreamSourceProvider import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, MicroBatchReadSupport} @@ -158,7 +159,6 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo } val ds = DataSource.lookupDataSource(source, sparkSession.sqlContext.conf).newInstance() - val options = new DataSourceOptions(extraOptions.asJava) // We need to generate the V1 data source so we can pass it to the V2 relation as a shim. // We can't be sure at this point whether we'll actually want to use V2, since we don't know the // writer or whether the query is continuous. @@ -173,12 +173,16 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo } ds match { case s: MicroBatchReadSupport => + val sessionOptions = DataSourceV2Utils.extractSessionConfigs( + ds = s, conf = sparkSession.sessionState.conf) + val options = sessionOptions ++ extraOptions + val dataSourceOptions = new DataSourceOptions(options.asJava) var tempReader: MicroBatchReader = null val schema = try { tempReader = s.createMicroBatchReader( Optional.ofNullable(userSpecifiedSchema.orNull), Utils.createTempDir(namePrefix = s"temporaryReader").getCanonicalPath, - options) + dataSourceOptions) tempReader.readSchema() } finally { // Stop tempReader to avoid side-effect thing @@ -190,17 +194,21 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo Dataset.ofRows( sparkSession, StreamingRelationV2( - s, source, extraOptions.toMap, + s, source, options, schema.toAttributes, v1Relation)(sparkSession)) case s: ContinuousReadSupport => + val sessionOptions = DataSourceV2Utils.extractSessionConfigs( + ds = s, conf = sparkSession.sessionState.conf) + val options = sessionOptions ++ extraOptions + val dataSourceOptions = new DataSourceOptions(options.asJava) val tempReader = s.createContinuousReader( Optional.ofNullable(userSpecifiedSchema.orNull), Utils.createTempDir(namePrefix = s"temporaryReader").getCanonicalPath, - options) + dataSourceOptions) Dataset.ofRows( sparkSession, StreamingRelationV2( - s, source, extraOptions.toMap, + s, source, options, tempReader.readSchema().toAttributes, v1Relation)(sparkSession)) case _ => // Code path for data source v1. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 3b9a56ffdde4..735fd1751634 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -27,6 +27,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.execution.streaming.sources._ @@ -298,22 +299,27 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { } else { val ds = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) val disabledSources = df.sparkSession.sqlContext.conf.disabledV2StreamingWriters.split(",") + var options = extraOptions.toMap val sink = ds.newInstance() match { - case w: StreamWriteSupport if !disabledSources.contains(w.getClass.getCanonicalName) => w + case w: StreamWriteSupport if !disabledSources.contains(w.getClass.getCanonicalName) => + val sessionOptions = DataSourceV2Utils.extractSessionConfigs( + w, df.sparkSession.sessionState.conf) + options = sessionOptions ++ extraOptions + w case _ => val ds = DataSource( df.sparkSession, className = source, - options = extraOptions.toMap, + options = options, partitionColumns = normalizedParCols.getOrElse(Nil)) ds.createSink(outputMode) } df.sparkSession.sessionState.streamingQueryManager.startQuery( - extraOptions.get("queryName"), - extraOptions.get("checkpointLocation"), + options.get("queryName"), + options.get("checkpointLocation"), df, - extraOptions.toMap, + options, sink, outputMode, useTempCheckpointLocation = source == "console", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index 52b833a19c23..2565cd9c2365 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -26,11 +26,11 @@ import org.apache.spark.sql.execution.streaming.{RateStreamOffset, Sink, Streami import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} -import org.apache.spark.sql.sources.v2.{ContinuousReadSupport, DataSourceOptions, MicroBatchReadSupport, StreamWriteSupport} +import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.reader.InputPartition import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReader, MicroBatchReader, Offset, PartitionOffset} import org.apache.spark.sql.sources.v2.writer.streaming.StreamWriter -import org.apache.spark.sql.streaming.{OutputMode, StreamTest, Trigger} +import org.apache.spark.sql.streaming.{OutputMode, StreamingQuery, StreamTest, Trigger} import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -54,14 +54,20 @@ trait FakeMicroBatchReadSupport extends MicroBatchReadSupport { override def createMicroBatchReader( schema: Optional[StructType], checkpointLocation: String, - options: DataSourceOptions): MicroBatchReader = FakeReader() + options: DataSourceOptions): MicroBatchReader = { + LastReadOptions.options = options + FakeReader() + } } trait FakeContinuousReadSupport extends ContinuousReadSupport { override def createContinuousReader( schema: Optional[StructType], checkpointLocation: String, - options: DataSourceOptions): ContinuousReader = FakeReader() + options: DataSourceOptions): ContinuousReader = { + LastReadOptions.options = options + FakeReader() + } } trait FakeStreamWriteSupport extends StreamWriteSupport { @@ -70,16 +76,27 @@ trait FakeStreamWriteSupport extends StreamWriteSupport { schema: StructType, mode: OutputMode, options: DataSourceOptions): StreamWriter = { + LastWriteOptions.options = options throw new IllegalStateException("fake sink - cannot actually write") } } -class FakeReadMicroBatchOnly extends DataSourceRegister with FakeMicroBatchReadSupport { +class FakeReadMicroBatchOnly + extends DataSourceRegister + with FakeMicroBatchReadSupport + with SessionConfigSupport { override def shortName(): String = "fake-read-microbatch-only" + + override def keyPrefix: String = shortName() } -class FakeReadContinuousOnly extends DataSourceRegister with FakeContinuousReadSupport { +class FakeReadContinuousOnly + extends DataSourceRegister + with FakeContinuousReadSupport + with SessionConfigSupport { override def shortName(): String = "fake-read-continuous-only" + + override def keyPrefix: String = shortName() } class FakeReadBothModes extends DataSourceRegister @@ -91,8 +108,13 @@ class FakeReadNeitherMode extends DataSourceRegister { override def shortName(): String = "fake-read-neither-mode" } -class FakeWrite extends DataSourceRegister with FakeStreamWriteSupport { +class FakeWrite + extends DataSourceRegister + with FakeStreamWriteSupport + with SessionConfigSupport { override def shortName(): String = "fake-write-microbatch-continuous" + + override def keyPrefix: String = shortName() } class FakeNoWrite extends DataSourceRegister { @@ -120,6 +142,21 @@ class FakeWriteV1Fallback extends DataSourceRegister override def shortName(): String = "fake-write-v1-fallback" } +object LastReadOptions { + var options: DataSourceOptions = _ + + def clear(): Unit = { + options = null + } +} + +object LastWriteOptions { + var options: DataSourceOptions = _ + + def clear(): Unit = { + options = null + } +} class StreamingDataSourceV2Suite extends StreamTest { @@ -129,6 +166,11 @@ class StreamingDataSourceV2Suite extends StreamTest { spark.conf.set("spark.sql.streaming.checkpointLocation", fakeCheckpoint.getCanonicalPath) } + override def afterEach(): Unit = { + LastReadOptions.clear() + LastWriteOptions.clear() + } + val readFormats = Seq( "fake-read-microbatch-only", "fake-read-continuous-only", @@ -142,7 +184,14 @@ class StreamingDataSourceV2Suite extends StreamTest { Trigger.ProcessingTime(1000), Trigger.Continuous(1000)) - private def testPositiveCase(readFormat: String, writeFormat: String, trigger: Trigger) = { + private def testPositiveCase(readFormat: String, writeFormat: String, trigger: Trigger): Unit = { + testPositiveCaseWithQuery(readFormat, writeFormat, trigger)(() => _) + } + + private def testPositiveCaseWithQuery( + readFormat: String, + writeFormat: String, + trigger: Trigger)(check: StreamingQuery => Unit): Unit = { val query = spark.readStream .format(readFormat) .load() @@ -150,8 +199,8 @@ class StreamingDataSourceV2Suite extends StreamTest { .format(writeFormat) .trigger(trigger) .start() + check(query) query.stop() - query } private def testNegativeCase( @@ -187,19 +236,54 @@ class StreamingDataSourceV2Suite extends StreamTest { test("disabled v2 write") { // Ensure the V2 path works normally and generates a V2 sink.. - val v2Query = testPositiveCase( - "fake-read-microbatch-continuous", "fake-write-v1-fallback", Trigger.Once()) - assert(v2Query.asInstanceOf[StreamingQueryWrapper].streamingQuery.sink - .isInstanceOf[FakeWriteV1Fallback]) + testPositiveCaseWithQuery( + "fake-read-microbatch-continuous", "fake-write-v1-fallback", Trigger.Once()) { v2Query => + assert(v2Query.asInstanceOf[StreamingQueryWrapper].streamingQuery.sink + .isInstanceOf[FakeWriteV1Fallback]) + } // Ensure we create a V1 sink with the config. Note the config is a comma separated // list, including other fake entries. val fullSinkName = "org.apache.spark.sql.streaming.sources.FakeWriteV1Fallback" withSQLConf(SQLConf.DISABLED_V2_STREAMING_WRITERS.key -> s"a,b,c,test,$fullSinkName,d,e") { - val v1Query = testPositiveCase( - "fake-read-microbatch-continuous", "fake-write-v1-fallback", Trigger.Once()) - assert(v1Query.asInstanceOf[StreamingQueryWrapper].streamingQuery.sink - .isInstanceOf[FakeSink]) + testPositiveCaseWithQuery( + "fake-read-microbatch-continuous", "fake-write-v1-fallback", Trigger.Once()) { v1Query => + assert(v1Query.asInstanceOf[StreamingQueryWrapper].streamingQuery.sink + .isInstanceOf[FakeSink]) + } + } + } + + Seq( + Tuple2(classOf[FakeReadMicroBatchOnly], Trigger.Once()), + Tuple2(classOf[FakeReadContinuousOnly], Trigger.Continuous(1000)) + ).foreach { case (source, trigger) => + test(s"SPARK-25460: session options are respected in structured streaming sources - $source") { + // `keyPrefix` and `shortName` are the same in this test case + val readSource = source.newInstance().shortName() + val writeSource = "fake-write-microbatch-continuous" + + val readOptionName = "optionA" + withSQLConf(s"spark.datasource.$readSource.$readOptionName" -> "true") { + testPositiveCaseWithQuery(readSource, writeSource, trigger) { _ => + eventually(timeout(streamingTimeout)) { + // Write options should not be set. + assert(LastWriteOptions.options.getBoolean(readOptionName, false) == false) + assert(LastReadOptions.options.getBoolean(readOptionName, false) == true) + } + } + } + + val writeOptionName = "optionB" + withSQLConf(s"spark.datasource.$writeSource.$writeOptionName" -> "true") { + testPositiveCaseWithQuery(readSource, writeSource, trigger) { _ => + eventually(timeout(streamingTimeout)) { + // Read options should not be set. + assert(LastReadOptions.options.getBoolean(writeOptionName, false) == false) + assert(LastWriteOptions.options.getBoolean(writeOptionName, false) == true) + } + } + } } } From ffc081c8fd5093c25a2269338929ae301c28daea Mon Sep 17 00:00:00 2001 From: Shahid Date: Mon, 24 Sep 2018 14:17:42 -0700 Subject: [PATCH 099/879] [SPARK-25502][CORE][WEBUI] Empty Page when page number exceeds the reatinedTask size. ## What changes were proposed in this pull request? Test steps : 1) bin/spark-shell --conf spark.ui.retainedTasks=200 ``` val rdd = sc.parallelize(1 to 1000, 1000) rdd.count ``` Stage tab in the UI will display 10 pages with 100 tasks per page. But number of retained tasks is only 200. So, from the 3rd page onwards will display nothing. We have to calculate total pages based on the number of tasks need display in the UI. **Before fix:** ![empty_4](https://user-images.githubusercontent.com/23054875/45918251-b1650580-bea1-11e8-90d3-7e0d491981a2.jpg) **After fix:** ![empty_3](https://user-images.githubusercontent.com/23054875/45918257-c2ae1200-bea1-11e8-960f-dfbdb4a90ae7.jpg) ## How was this patch tested? Manually tested Closes #22526 from shahidki31/SPARK-25502. Authored-by: Shahid Signed-off-by: Marcelo Vanzin (cherry picked from commit 3ce2e008ec1bf70adc5a4b356e09a469e94af803) Signed-off-by: Marcelo Vanzin --- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 10 +++------- 1 file changed, 3 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 55eb98996266..fd6a298e577d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -117,7 +117,8 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We val localitySummary = store.localitySummary(stageData.stageId, stageData.attemptId) - val totalTasks = taskCount(stageData) + val totalTasks = stageData.numActiveTasks + stageData.numCompleteTasks + + stageData.numFailedTasks + stageData.numKilledTasks if (totalTasks == 0) { val content =
    @@ -685,7 +686,7 @@ private[ui] class TaskDataSource( private var _tasksToShow: Seq[TaskData] = null - override def dataSize: Int = taskCount(stage) + override def dataSize: Int = store.taskCount(stage.stageId, stage.attemptId).toInt override def sliceData(from: Int, to: Int): Seq[TaskData] = { if (_tasksToShow == null) { @@ -1051,9 +1052,4 @@ private[ui] object ApiHelper { (stage.map(_.name).getOrElse(""), stage.flatMap(_.description).getOrElse(job.name)) } - def taskCount(stageData: StageData): Int = { - stageData.numActiveTasks + stageData.numCompleteTasks + stageData.numFailedTasks + - stageData.numKilledTasks - } - } From e4c03e82278791fcc725600dc5b1f31741340139 Mon Sep 17 00:00:00 2001 From: Shahid Date: Mon, 24 Sep 2018 20:03:52 -0700 Subject: [PATCH 100/879] [SPARK-25503][CORE][WEBUI] Total task message in stage page is ambiguous ## What changes were proposed in this pull request? Test steps : 1) bin/spark-shell --conf spark.ui.retainedTasks=10 2) val rdd = sc.parallelize(1 to 1000, 1000) 3) rdd.count Stage page tab in the UI will display 10 tasks, but display message is wrong. It should reverse. **Before fix :** ![webui_1](https://user-images.githubusercontent.com/23054875/45917921-8926d800-be9c-11e8-8da5-3998d07e3ccc.jpg) **After fix** ![spark_web_ui2](https://user-images.githubusercontent.com/23054875/45917935-b4112c00-be9c-11e8-9d10-4fcc8e88568f.jpg) ## How was this patch tested? Manually tested Closes #22525 from shahidki31/SparkUI. Authored-by: Shahid Signed-off-by: Dongjoon Hyun (cherry picked from commit 615792da42b3ee3c5f623c869fada17a3aa92884) Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index fd6a298e577d..7428bbe6c559 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -133,7 +133,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We val totalTasksNumStr = if (totalTasks == storedTasks) { s"$totalTasks" } else { - s"$storedTasks, showing ${totalTasks}" + s"$totalTasks, showing $storedTasks" } val summary = From 4ca4ef7b9c44637dd30b934788d7831218553aea Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Tue, 25 Sep 2018 12:05:04 +0800 Subject: [PATCH 101/879] [SPARK-25519][SQL] ArrayRemove function may return incorrect result when right expression is implicitly downcasted. ## What changes were proposed in this pull request? In ArrayRemove, we currently cast the right hand side expression to match the element type of the left hand side Array. This may result in down casting and may return wrong result or questionable result. Example : ```SQL spark-sql> select array_remove(array(1,2,3), 1.23D); [2,3] ``` ```SQL spark-sql> select array_remove(array(1,2,3), 'foo'); NULL ``` We should safely coerce both left and right hand side expressions. ## How was this patch tested? Added tests in DataFrameFunctionsSuite Closes #22542 from dilipbiswal/SPARK-25519. Authored-by: Dilip Biswal Signed-off-by: Wenchen Fan (cherry picked from commit 7d8f5b62c57c9e2903edd305e8b9c5400652fdb0) Signed-off-by: Wenchen Fan --- .../expressions/collectionOperations.scala | 29 ++++++----- .../spark/sql/DataFrameFunctionsSuite.scala | 48 ++++++++++++++++++- 2 files changed, 63 insertions(+), 14 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 85bc1cdb4305..9cc7dbadd923 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -3088,11 +3088,24 @@ case class ArrayRemove(left: Expression, right: Expression) override def dataType: DataType = left.dataType override def inputTypes: Seq[AbstractDataType] = { - val elementType = left.dataType match { - case t: ArrayType => t.elementType - case _ => AnyDataType + (left.dataType, right.dataType) match { + case (ArrayType(e1, hasNull), e2) => + TypeCoercion.findTightestCommonType(e1, e2) match { + case Some(dt) => Seq(ArrayType(dt, hasNull), dt) + case _ => Seq.empty + } + case _ => Seq.empty + } + } + + override def checkInputDataTypes(): TypeCheckResult = { + (left.dataType, right.dataType) match { + case (ArrayType(e1, _), e2) if e1.sameType(e2) => + TypeUtils.checkForOrderingExpr(e2, s"function $prettyName") + case _ => TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " + + s"been ${ArrayType.simpleString} followed by a value with same element type, but it's " + + s"[${left.dataType.catalogString}, ${right.dataType.catalogString}].") } - Seq(ArrayType, elementType) } private def elementType: DataType = left.dataType.asInstanceOf[ArrayType].elementType @@ -3100,14 +3113,6 @@ case class ArrayRemove(left: Expression, right: Expression) @transient private lazy val ordering: Ordering[Any] = TypeUtils.getInterpretedOrdering(right.dataType) - override def checkInputDataTypes(): TypeCheckResult = { - super.checkInputDataTypes() match { - case f: TypeCheckResult.TypeCheckFailure => f - case TypeCheckResult.TypeCheckSuccess => - TypeUtils.checkForOrderingExpr(right.dataType, s"function $prettyName") - } - } - override def nullSafeEval(arr: Any, value: Any): Any = { val newArray = new Array[Any](arr.asInstanceOf[ArrayData].numElements()) var pos = 0 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 fd71f2493561..88dbae8c2135 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 @@ -1574,6 +1574,34 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Row(null)) ) + checkAnswer( + OneRowRelation().selectExpr("array_remove(array(1, 2), 1.23D)"), + Seq( + Row(Seq(1.0, 2.0)) + ) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_remove(array(1, 2), 1.0D)"), + Seq( + Row(Seq(2.0)) + ) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_remove(array(1.0D, 2.0D), 2)"), + Seq( + Row(Seq(1.0)) + ) + ) + + checkAnswer( + OneRowRelation().selectExpr("array_remove(array(1.1D, 1.2D), 1)"), + Seq( + Row(Seq(1.1, 1.2)) + ) + ) + checkAnswer( df.selectExpr("array_remove(a, 2)", "array_remove(b, \"a\")", "array_remove(c, \"\")"), @@ -1583,10 +1611,26 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Row(null, null, null)) ) - val e = intercept[AnalysisException] { + val e1 = intercept[AnalysisException] { Seq(("a string element", "a")).toDF().selectExpr("array_remove(_1, _2)") } - assert(e.message.contains("argument 1 requires array type, however, '`_1`' is of string type")) + val errorMsg1 = + s""" + |Input to function array_remove should have been array followed by a + |value with same element type, but it's [string, string]. + """.stripMargin.replace("\n", " ").trim() + assert(e1.message.contains(errorMsg1)) + + val e2 = intercept[AnalysisException] { + OneRowRelation().selectExpr("array_remove(array(1, 2), '1')") + } + + val errorMsg2 = + s""" + |Input to function array_remove should have been array followed by a + |value with same element type, but it's [array, string]. + """.stripMargin.replace("\n", " ").trim() + assert(e2.message.contains(errorMsg2)) } test("array_distinct functions") { From a709718dae495725af4e961b1e0f85bce5d34368 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 25 Sep 2018 20:13:07 +0800 Subject: [PATCH 102/879] [SPARK-23907][SQL] Revert regr_* functions entirely ## What changes were proposed in this pull request? This patch reverts entirely all the regr_* functions added in SPARK-23907. These were added by mgaido91 (and proposed by gatorsmile) to improve compatibility with other database systems, without any actual use cases. However, they are very rarely used, and in Spark there are much better ways to compute these functions, due to Spark's flexibility in exposing real programming APIs. I'm going through all the APIs added in Spark 2.4 and I think we should revert these. If there are strong enough demands and more use cases, we can add them back in the future pretty easily. ## How was this patch tested? Reverted test cases also. Closes #22541 from rxin/SPARK-23907. Authored-by: Reynold Xin Signed-off-by: hyukjinkwon (cherry picked from commit 9cbd001e2476cd06aa0bcfcc77a21a9077d5797a) Signed-off-by: hyukjinkwon --- .../catalyst/analysis/FunctionRegistry.scala | 9 - .../expressions/aggregate/regression.scala | 190 ------------------ .../sql-tests/inputs/udaf-regrfunctions.sql | 56 ------ .../results/udaf-regrfunctions.sql.out | 93 --------- 4 files changed, 348 deletions(-) delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/regression.scala delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/udaf-regrfunctions.sql delete mode 100644 sql/core/src/test/resources/sql-tests/results/udaf-regrfunctions.sql.out 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 77860e1584f4..695267ac130a 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 @@ -299,15 +299,6 @@ object FunctionRegistry { expression[CollectList]("collect_list"), expression[CollectSet]("collect_set"), expression[CountMinSketchAgg]("count_min_sketch"), - expression[RegrCount]("regr_count"), - expression[RegrSXX]("regr_sxx"), - expression[RegrSYY]("regr_syy"), - expression[RegrAvgX]("regr_avgx"), - expression[RegrAvgY]("regr_avgy"), - expression[RegrSXY]("regr_sxy"), - expression[RegrSlope]("regr_slope"), - expression[RegrR2]("regr_r2"), - expression[RegrIntercept]("regr_intercept"), // string functions expression[Ascii]("ascii"), diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/regression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/regression.scala deleted file mode 100644 index d8f4505588ff..000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/regression.scala +++ /dev/null @@ -1,190 +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.catalyst.expressions.aggregate - -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{AbstractDataType, DoubleType} - -/** - * Base trait for all regression functions. - */ -trait RegrLike extends AggregateFunction with ImplicitCastInputTypes { - def y: Expression - def x: Expression - - override def children: Seq[Expression] = Seq(y, x) - override def inputTypes: Seq[AbstractDataType] = Seq(DoubleType, DoubleType) - - protected def updateIfNotNull(exprs: Seq[Expression]): Seq[Expression] = { - assert(aggBufferAttributes.length == exprs.length) - val nullableChildren = children.filter(_.nullable) - if (nullableChildren.isEmpty) { - exprs - } else { - exprs.zip(aggBufferAttributes).map { case (e, a) => - If(nullableChildren.map(IsNull).reduce(Or), a, e) - } - } - } -} - - -@ExpressionDescription( - usage = "_FUNC_(y, x) - Returns the number of non-null pairs.", - since = "2.4.0") -case class RegrCount(y: Expression, x: Expression) - extends CountLike with RegrLike { - - override lazy val updateExpressions: Seq[Expression] = updateIfNotNull(Seq(count + 1L)) - - override def prettyName: String = "regr_count" -} - - -@ExpressionDescription( - usage = "_FUNC_(y, x) - Returns SUM(x*x)-SUM(x)*SUM(x)/N. Any pair with a NULL is ignored.", - since = "2.4.0") -case class RegrSXX(y: Expression, x: Expression) - extends CentralMomentAgg(x) with RegrLike { - - override protected def momentOrder = 2 - - override lazy val updateExpressions: Seq[Expression] = updateIfNotNull(updateExpressionsDef) - - override val evaluateExpression: Expression = { - If(n === Literal(0.0), Literal.create(null, DoubleType), m2) - } - - override def prettyName: String = "regr_sxx" -} - - -@ExpressionDescription( - usage = "_FUNC_(y, x) - Returns SUM(y*y)-SUM(y)*SUM(y)/N. Any pair with a NULL is ignored.", - since = "2.4.0") -case class RegrSYY(y: Expression, x: Expression) - extends CentralMomentAgg(y) with RegrLike { - - override protected def momentOrder = 2 - - override lazy val updateExpressions: Seq[Expression] = updateIfNotNull(updateExpressionsDef) - - override val evaluateExpression: Expression = { - If(n === Literal(0.0), Literal.create(null, DoubleType), m2) - } - - override def prettyName: String = "regr_syy" -} - - -@ExpressionDescription( - usage = "_FUNC_(y, x) - Returns the average of x. Any pair with a NULL is ignored.", - since = "2.4.0") -case class RegrAvgX(y: Expression, x: Expression) - extends AverageLike(x) with RegrLike { - - override lazy val updateExpressions: Seq[Expression] = updateIfNotNull(updateExpressionsDef) - - override def prettyName: String = "regr_avgx" -} - - -@ExpressionDescription( - usage = "_FUNC_(y, x) - Returns the average of y. Any pair with a NULL is ignored.", - since = "2.4.0") -case class RegrAvgY(y: Expression, x: Expression) - extends AverageLike(y) with RegrLike { - - override lazy val updateExpressions: Seq[Expression] = updateIfNotNull(updateExpressionsDef) - - override def prettyName: String = "regr_avgy" -} - -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(y, x) - Returns the covariance of y and x multiplied for the number of items in the dataset. Any pair with a NULL is ignored.", - since = "2.4.0") -// scalastyle:on line.size.limit -case class RegrSXY(y: Expression, x: Expression) - extends Covariance(y, x) with RegrLike { - - override lazy val updateExpressions: Seq[Expression] = updateIfNotNull(updateExpressionsDef) - - override val evaluateExpression: Expression = { - If(n === Literal(0.0), Literal.create(null, DoubleType), ck) - } - - override def prettyName: String = "regr_sxy" -} - - -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(y, x) - Returns the slope of the linear regression line. Any pair with a NULL is ignored.", - since = "2.4.0") -// scalastyle:on line.size.limit -case class RegrSlope(y: Expression, x: Expression) - extends PearsonCorrelation(y, x) with RegrLike { - - override lazy val updateExpressions: Seq[Expression] = updateIfNotNull(updateExpressionsDef) - - override val evaluateExpression: Expression = { - If(n < Literal(2.0) || yMk === Literal(0.0), Literal.create(null, DoubleType), ck / yMk) - } - - override def prettyName: String = "regr_slope" -} - - -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(y, x) - Returns the coefficient of determination (also called R-squared or goodness of fit) for the regression line. Any pair with a NULL is ignored.", - since = "2.4.0") -// scalastyle:on line.size.limit -case class RegrR2(y: Expression, x: Expression) - extends PearsonCorrelation(y, x) with RegrLike { - - override lazy val updateExpressions: Seq[Expression] = updateIfNotNull(updateExpressionsDef) - - override val evaluateExpression: Expression = { - If(n < Literal(2.0) || yMk === Literal(0.0), Literal.create(null, DoubleType), - If(xMk === Literal(0.0), Literal(1.0), ck * ck / yMk / xMk)) - } - - override def prettyName: String = "regr_r2" -} - - -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = "_FUNC_(y, x) - Returns the y-intercept of the linear regression line. Any pair with a NULL is ignored.", - since = "2.4.0") -// scalastyle:on line.size.limit -case class RegrIntercept(y: Expression, x: Expression) - extends PearsonCorrelation(y, x) with RegrLike { - - override lazy val updateExpressions: Seq[Expression] = updateIfNotNull(updateExpressionsDef) - - override val evaluateExpression: Expression = { - If(n === Literal(0.0) || yMk === Literal(0.0), Literal.create(null, DoubleType), - xAvg - (ck / yMk) * yAvg) - } - - override def prettyName: String = "regr_intercept" -} diff --git a/sql/core/src/test/resources/sql-tests/inputs/udaf-regrfunctions.sql b/sql/core/src/test/resources/sql-tests/inputs/udaf-regrfunctions.sql deleted file mode 100644 index 92c7e26e3add..000000000000 --- a/sql/core/src/test/resources/sql-tests/inputs/udaf-regrfunctions.sql +++ /dev/null @@ -1,56 +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. --- - -CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES - (101, 1, 1, 1), - (201, 2, 1, 1), - (301, 3, 1, 1), - (401, 4, 1, 11), - (501, 5, 1, null), - (601, 6, null, 1), - (701, 6, null, null), - (102, 1, 2, 2), - (202, 2, 1, 2), - (302, 3, 2, 1), - (402, 4, 2, 12), - (502, 5, 2, null), - (602, 6, null, 2), - (702, 6, null, null), - (103, 1, 3, 3), - (203, 2, 1, 3), - (303, 3, 3, 1), - (403, 4, 3, 13), - (503, 5, 3, null), - (603, 6, null, 3), - (703, 6, null, null), - (104, 1, 4, 4), - (204, 2, 1, 4), - (304, 3, 4, 1), - (404, 4, 4, 14), - (504, 5, 4, null), - (604, 6, null, 4), - (704, 6, null, null), - (800, 7, 1, 1) -as t1(id, px, y, x); - -select px, var_pop(x), var_pop(y), corr(y,x), covar_samp(y,x), covar_pop(y,x), regr_count(y,x), - regr_slope(y,x), regr_intercept(y,x), regr_r2(y,x), regr_sxx(y,x), regr_syy(y,x), regr_sxy(y,x), - regr_avgx(y,x), regr_avgy(y,x), regr_count(y,x) -from t1 group by px order by px; - - -select id, regr_count(y,x) over (partition by px) from t1 order by id; diff --git a/sql/core/src/test/resources/sql-tests/results/udaf-regrfunctions.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf-regrfunctions.sql.out deleted file mode 100644 index d7d009a64bf8..000000000000 --- a/sql/core/src/test/resources/sql-tests/results/udaf-regrfunctions.sql.out +++ /dev/null @@ -1,93 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- Number of queries: 3 - - --- !query 0 -CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES - (101, 1, 1, 1), - (201, 2, 1, 1), - (301, 3, 1, 1), - (401, 4, 1, 11), - (501, 5, 1, null), - (601, 6, null, 1), - (701, 6, null, null), - (102, 1, 2, 2), - (202, 2, 1, 2), - (302, 3, 2, 1), - (402, 4, 2, 12), - (502, 5, 2, null), - (602, 6, null, 2), - (702, 6, null, null), - (103, 1, 3, 3), - (203, 2, 1, 3), - (303, 3, 3, 1), - (403, 4, 3, 13), - (503, 5, 3, null), - (603, 6, null, 3), - (703, 6, null, null), - (104, 1, 4, 4), - (204, 2, 1, 4), - (304, 3, 4, 1), - (404, 4, 4, 14), - (504, 5, 4, null), - (604, 6, null, 4), - (704, 6, null, null), - (800, 7, 1, 1) -as t1(id, px, y, x) --- !query 0 schema -struct<> --- !query 0 output - - - --- !query 1 -select px, var_pop(x), var_pop(y), corr(y,x), covar_samp(y,x), covar_pop(y,x), regr_count(y,x), - regr_slope(y,x), regr_intercept(y,x), regr_r2(y,x), regr_sxx(y,x), regr_syy(y,x), regr_sxy(y,x), - regr_avgx(y,x), regr_avgy(y,x), regr_count(y,x) -from t1 group by px order by px --- !query 1 schema -struct --- !query 1 output -1 1.25 1.25 1.0 1.6666666666666667 1.25 4 1.0 0.0 1.0 5.0 5.0 5.0 2.5 2.5 4 -2 1.25 0.0 NULL 0.0 0.0 4 0.0 1.0 1.0 5.0 0.0 0.0 2.5 1.0 4 -3 0.0 1.25 NULL 0.0 0.0 4 NULL NULL NULL 0.0 5.0 0.0 1.0 2.5 4 -4 1.25 1.25 1.0 1.6666666666666667 1.25 4 1.0 -10.0 1.0 5.0 5.0 5.0 12.5 2.5 4 -5 NULL 1.25 NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL 0 -6 1.25 NULL NULL NULL NULL 0 NULL NULL NULL NULL NULL NULL NULL NULL 0 -7 0.0 0.0 NaN NaN 0.0 1 NULL NULL NULL 0.0 0.0 0.0 1.0 1.0 1 - - --- !query 2 -select id, regr_count(y,x) over (partition by px) from t1 order by id --- !query 2 schema -struct --- !query 2 output -101 4 -102 4 -103 4 -104 4 -201 4 -202 4 -203 4 -204 4 -301 4 -302 4 -303 4 -304 4 -401 4 -402 4 -403 4 -404 4 -501 0 -502 0 -503 0 -504 0 -601 0 -602 0 -603 0 -604 0 -701 0 -702 0 -703 0 -704 0 -800 1 From 544f86a69bba94dfcb241e41c799ed63ef4210fc Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 25 Sep 2018 11:42:27 -0700 Subject: [PATCH 103/879] [SPARK-25495][SS] FetchedData.reset should reset all fields ## What changes were proposed in this pull request? `FetchedData.reset` should reset `_nextOffsetInFetchedData` and `_offsetAfterPoll`. Otherwise it will cause inconsistent cached data and may make Kafka connector return wrong results. ## How was this patch tested? The new unit test. Closes #22507 from zsxwing/fix-kafka-reset. Lead-authored-by: Shixiong Zhu Co-authored-by: Shixiong Zhu Signed-off-by: Shixiong Zhu (cherry picked from commit 66d29870c09e6050dd846336e596faaa8b0d14ad) Signed-off-by: Shixiong Zhu --- .../sql/kafka010/KafkaDataConsumer.scala | 5 +- .../kafka010/KafkaMicroBatchSourceSuite.scala | 52 +++++++++++++++++++ 2 files changed, 56 insertions(+), 1 deletion(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala index ceb9e318b283..7b1314bc8c3c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala @@ -134,6 +134,8 @@ private[kafka010] case class InternalKafkaConsumer( /** Reset the internal pre-fetched data. */ def reset(): Unit = { _records = ju.Collections.emptyListIterator() + _nextOffsetInFetchedData = UNKNOWN_OFFSET + _offsetAfterPoll = UNKNOWN_OFFSET } /** @@ -361,8 +363,9 @@ private[kafka010] case class InternalKafkaConsumer( if (offset < fetchedData.offsetAfterPoll) { // Offsets in [offset, fetchedData.offsetAfterPoll) are invisible. Return a record to ask // the next call to start from `fetchedData.offsetAfterPoll`. + val nextOffsetToFetch = fetchedData.offsetAfterPoll fetchedData.reset() - return fetchedRecord.withRecord(null, fetchedData.offsetAfterPoll) + return fetchedRecord.withRecord(null, nextOffsetToFetch) } else { // Fetch records from Kafka and update `fetchedData`. fetchData(offset, pollTimeoutMs) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 65615fdb5b3e..e0b6d8cdd3d2 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -853,6 +853,58 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { ) } } + + test("SPARK-25495: FetchedData.reset should reset all fields") { + val topic = newTopic() + val topicPartition = new TopicPartition(topic, 0) + testUtils.createTopic(topic, partitions = 1) + + val ds = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("kafka.isolation.level", "read_committed") + .option("subscribe", topic) + .option("startingOffsets", "earliest") + .load() + .select($"value".as[String]) + + testUtils.withTranscationalProducer { producer => + producer.beginTransaction() + (0 to 3).foreach { i => + producer.send(new ProducerRecord[String, String](topic, i.toString)).get() + } + producer.commitTransaction() + } + testUtils.waitUntilOffsetAppears(topicPartition, 5) + + val q = ds.writeStream.foreachBatch { (ds, epochId) => + if (epochId == 0) { + // Send more message before the tasks of the current batch start reading the current batch + // data, so that the executors will prefetch messages in the next batch and drop them. In + // this case, if we forget to reset `FetchedData._nextOffsetInFetchedData` or + // `FetchedData._offsetAfterPoll` (See SPARK-25495), the next batch will see incorrect + // values and return wrong results hence fail the test. + testUtils.withTranscationalProducer { producer => + producer.beginTransaction() + (4 to 7).foreach { i => + producer.send(new ProducerRecord[String, String](topic, i.toString)).get() + } + producer.commitTransaction() + } + testUtils.waitUntilOffsetAppears(topicPartition, 10) + checkDatasetUnorderly(ds, (0 to 3).map(_.toString): _*) + } else { + checkDatasetUnorderly(ds, (4 to 7).map(_.toString): _*) + } + }.start() + try { + q.processAllAvailable() + } finally { + q.stop() + } + } } From f91247f812f87daa9fe4ec23b100f2310254df22 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 26 Sep 2018 08:45:27 +0800 Subject: [PATCH 104/879] [SPARK-25422][CORE] Don't memory map blocks streamed to disk. After data has been streamed to disk, the buffers are inserted into the memory store in some cases (eg., with broadcast blocks). But broadcast code also disposes of those buffers when the data has been read, to ensure that we don't leave mapped buffers using up memory, which then leads to garbage data in the memory store. ## How was this patch tested? Ran the old failing test in a loop. Full tests on jenkins Closes #22546 from squito/SPARK-25422-master. Authored-by: Imran Rashid Signed-off-by: Wenchen Fan (cherry picked from commit 9bb3a0c67bd851b09ff4701ef1d280e2a77d791b) Signed-off-by: Wenchen Fan --- .../apache/spark/storage/BlockManager.scala | 13 ++--- .../spark/util/io/ChunkedByteBuffer.scala | 47 ++++++++++--------- 2 files changed, 31 insertions(+), 29 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 22341467add5..0fe82ac0cedc 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -438,10 +438,8 @@ private[spark] class BlockManager( // stream. channel.close() // TODO SPARK-25035 Even if we're only going to write the data to disk after this, we end up - // using a lot of memory here. With encryption, we'll read the whole file into a regular - // byte buffer and OOM. Without encryption, we'll memory map the file and won't get a jvm - // OOM, but might get killed by the OS / cluster manager. We could at least read the tmp - // file as a stream in both cases. + // using a lot of memory here. We'll read the whole file into a regular + // byte buffer and OOM. We could at least read the tmp file as a stream. val buffer = securityManager.getIOEncryptionKey() match { case Some(key) => // we need to pass in the size of the unencrypted block @@ -453,7 +451,7 @@ private[spark] class BlockManager( new EncryptedBlockData(tmpFile, blockSize, conf, key).toChunkedByteBuffer(allocator) case None => - ChunkedByteBuffer.map(tmpFile, conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS).toInt) + ChunkedByteBuffer.fromFile(tmpFile, conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS).toInt) } putBytes(blockId, buffer, level)(classTag) tmpFile.delete() @@ -726,10 +724,9 @@ private[spark] class BlockManager( */ def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = { // TODO if we change this method to return the ManagedBuffer, then getRemoteValues - // could just use the inputStream on the temp file, rather than memory-mapping the file. + // could just use the inputStream on the temp file, rather than reading the file into memory. // Until then, replication can cause the process to use too much memory and get killed - // by the OS / cluster manager (not a java OOM, since it's a memory-mapped file) even though - // we've read the data to disk. + // even though we've read the data to disk. logDebug(s"Getting remote block $blockId") require(blockId != null, "BlockId is null") var runningFailureCount = 0 diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala index 39f050f6ca5a..4aa8d45ec740 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala @@ -19,17 +19,16 @@ package org.apache.spark.util.io import java.io.{File, FileInputStream, InputStream} import java.nio.ByteBuffer -import java.nio.channels.{FileChannel, WritableByteChannel} -import java.nio.file.StandardOpenOption - -import scala.collection.mutable.ListBuffer +import java.nio.channels.WritableByteChannel +import com.google.common.io.ByteStreams import com.google.common.primitives.UnsignedBytes +import org.apache.commons.io.IOUtils import org.apache.spark.SparkEnv import org.apache.spark.internal.config import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} -import org.apache.spark.network.util.ByteArrayWritableChannel +import org.apache.spark.network.util.{ByteArrayWritableChannel, LimitedInputStream} import org.apache.spark.storage.StorageUtils import org.apache.spark.util.Utils @@ -175,30 +174,36 @@ object ChunkedByteBuffer { def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): ChunkedByteBuffer = { data match { case f: FileSegmentManagedBuffer => - map(f.getFile, maxChunkSize, f.getOffset, f.getLength) + fromFile(f.getFile, maxChunkSize, f.getOffset, f.getLength) case other => new ChunkedByteBuffer(other.nioByteBuffer()) } } - def map(file: File, maxChunkSize: Int): ChunkedByteBuffer = { - map(file, maxChunkSize, 0, file.length()) + def fromFile(file: File, maxChunkSize: Int): ChunkedByteBuffer = { + fromFile(file, maxChunkSize, 0, file.length()) } - def map(file: File, maxChunkSize: Int, offset: Long, length: Long): ChunkedByteBuffer = { - Utils.tryWithResource(FileChannel.open(file.toPath, StandardOpenOption.READ)) { channel => - var remaining = length - var pos = offset - val chunks = new ListBuffer[ByteBuffer]() - while (remaining > 0) { - val chunkSize = math.min(remaining, maxChunkSize) - val chunk = channel.map(FileChannel.MapMode.READ_ONLY, pos, chunkSize) - pos += chunkSize - remaining -= chunkSize - chunks += chunk - } - new ChunkedByteBuffer(chunks.toArray) + private def fromFile( + file: File, + maxChunkSize: Int, + offset: Long, + length: Long): ChunkedByteBuffer = { + // We do *not* memory map the file, because we may end up putting this into the memory store, + // and spark currently is not expecting memory-mapped buffers in the memory store, it conflicts + // with other parts that manage the lifecyle of buffers and dispose them. See SPARK-25422. + val is = new FileInputStream(file) + ByteStreams.skipFully(is, offset) + val in = new LimitedInputStream(is, length) + val chunkSize = math.min(maxChunkSize, length).toInt + val out = new ChunkedByteBufferOutputStream(chunkSize, ByteBuffer.allocate _) + Utils.tryWithSafeFinally { + IOUtils.copy(in, out) + } { + in.close() + out.close() } + out.toChunkedByteBuffer } } From 3f203050ac764516e68fb43628bba0df5963e44d Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Wed, 26 Sep 2018 09:32:51 +0800 Subject: [PATCH 105/879] [SPARK-24324][PYTHON][FOLLOW-UP] Rename the Conf to spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName ## What changes were proposed in this pull request? Add the legacy prefix for spark.sql.execution.pandas.groupedMap.assignColumnsByPosition and rename it to spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName ## How was this patch tested? The existing tests. Closes #22540 from gatorsmile/renameAssignColumnsByPosition. Lead-authored-by: gatorsmile Co-authored-by: Hyukjin Kwon Signed-off-by: hyukjinkwon (cherry picked from commit 8c2edf46d0f89e5ec54968218d89f30a3f8190bc) Signed-off-by: hyukjinkwon --- python/pyspark/sql/tests.py | 3 ++- python/pyspark/worker.py | 7 ++++--- .../apache/spark/sql/internal/SQLConf.scala | 18 +++++++++--------- .../spark/sql/execution/arrow/ArrowUtils.scala | 9 +++------ 4 files changed, 18 insertions(+), 19 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 9fa1577681f0..cb186dee3456 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -5799,7 +5799,8 @@ def test_positional_assignment_conf(self): import pandas as pd from pyspark.sql.functions import pandas_udf, PandasUDFType - with self.sql_conf({"spark.sql.execution.pandas.groupedMap.assignColumnsByPosition": True}): + with self.sql_conf({ + "spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName": False}): @pandas_udf("a string, b float", PandasUDFType.GROUPED_MAP) def foo(_): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 974344f01d92..8c59f1f999f1 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -97,8 +97,9 @@ def verify_result_length(*a): def wrap_grouped_map_pandas_udf(f, return_type, argspec, runner_conf): - assign_cols_by_pos = runner_conf.get( - "spark.sql.execution.pandas.groupedMap.assignColumnsByPosition", False) + assign_cols_by_name = runner_conf.get( + "spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName", "true") + assign_cols_by_name = assign_cols_by_name.lower() == "true" def wrapped(key_series, value_series): import pandas as pd @@ -119,7 +120,7 @@ def wrapped(key_series, value_series): "Expected: {} Actual: {}".format(len(return_type), len(result.columns))) # Assign result columns by schema name if user labeled with strings, else use position - if not assign_cols_by_pos and any(isinstance(name, basestring) for name in result.columns): + if assign_cols_by_name and any(isinstance(name, basestring) for name in result.columns): return [(result[field.name], to_arrow_type(field.dataType)) for field in return_type] else: return [(result[result.columns[i]], to_arrow_type(field.dataType)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 2788402140f3..68daf9d6d330 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1275,15 +1275,15 @@ object SQLConf { .booleanConf .createWithDefault(true) - val PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_POSITION = - buildConf("spark.sql.execution.pandas.groupedMap.assignColumnsByPosition") + val PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME = + buildConf("spark.sql.legacy.execution.pandas.groupedMap.assignColumnsByName") .internal() - .doc("When true, a grouped map Pandas UDF will assign columns from the returned " + - "Pandas DataFrame based on position, regardless of column label type. When false, " + - "columns will be looked up by name if labeled with a string and fallback to use " + - "position if not. This configuration will be deprecated in future releases.") + .doc("When true, columns will be looked up by name if labeled with a string and fallback " + + "to use position if not. When false, a grouped map Pandas UDF will assign columns from " + + "the returned Pandas DataFrame based on position, regardless of column label type. " + + "This configuration will be deprecated in future releases.") .booleanConf - .createWithDefault(false) + .createWithDefault(true) val REPLACE_EXCEPT_WITH_FILTER = buildConf("spark.sql.optimizer.replaceExceptWithFilter") .internal() @@ -1884,8 +1884,8 @@ class SQLConf extends Serializable with Logging { def pandasRespectSessionTimeZone: Boolean = getConf(PANDAS_RESPECT_SESSION_LOCAL_TIMEZONE) - def pandasGroupedMapAssignColumnssByPosition: Boolean = - getConf(SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_POSITION) + def pandasGroupedMapAssignColumnsByName: Boolean = + getConf(SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME) def replaceExceptWithFilter: Boolean = getConf(REPLACE_EXCEPT_WITH_FILTER) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala index 533097ac399e..b1e8fb39ac9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala @@ -131,11 +131,8 @@ object ArrowUtils { } else { Nil } - val pandasColsByPosition = if (conf.pandasGroupedMapAssignColumnssByPosition) { - Seq(SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_POSITION.key -> "true") - } else { - Nil - } - Map(timeZoneConf ++ pandasColsByPosition: _*) + val pandasColsByName = Seq(SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key -> + conf.pandasGroupedMapAssignColumnsByName.toString) + Map(timeZoneConf ++ pandasColsByName: _*) } } From d44b863a2d58d3b57af1e8aa1550c6e925446032 Mon Sep 17 00:00:00 2001 From: seancxmao Date: Wed, 26 Sep 2018 22:14:14 +0800 Subject: [PATCH 106/879] [SPARK-20937][DOCS] Describe spark.sql.parquet.writeLegacyFormat property in Spark SQL, DataFrames and Datasets Guide ## What changes were proposed in this pull request? Describe spark.sql.parquet.writeLegacyFormat property in Spark SQL, DataFrames and Datasets Guide. ## How was this patch tested? N/A Closes #22453 from seancxmao/SPARK-20937. Authored-by: seancxmao Signed-off-by: hyukjinkwon (cherry picked from commit cf5c9c4b550c3a8ed59d7ef9404f2689ea763fa9) Signed-off-by: hyukjinkwon --- docs/sql-programming-guide.md | 11 +++++++++++ .../scala/org/apache/spark/sql/internal/SQLConf.scala | 7 +++++-- 2 files changed, 16 insertions(+), 2 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index b5302bb7dd62..2546064ae282 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1002,6 +1002,17 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession

    spark.sql.parquet.writeLegacyFormatfalse + If true, data will be written in a way of Spark 1.4 and earlier. For example, decimal values + will be written in Apache Parquet's fixed-length byte array format, which other systems such as + Apache Hive and Apache Impala use. If false, the newer format in Parquet will be used. For + example, decimals will be written in int-based format. If Parquet output is intended for use + with systems that do not support this newer format, set to true. +
    ## ORC Files diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 68daf9d6d330..bacd5e9f8517 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -431,8 +431,11 @@ object SQLConf { .createWithDefault(10) val PARQUET_WRITE_LEGACY_FORMAT = buildConf("spark.sql.parquet.writeLegacyFormat") - .doc("Whether to be compatible with the legacy Parquet format adopted by Spark 1.4 and prior " + - "versions, when converting Parquet schema to Spark SQL schema and vice versa.") + .doc("If true, data will be written in a way of Spark 1.4 and earlier. For example, decimal " + + "values will be written in Apache Parquet's fixed-length byte array format, which other " + + "systems such as Apache Hive and Apache Impala use. If false, the newer format in Parquet " + + "will be used. For example, decimals will be written in int-based format. If Parquet " + + "output is intended for use with systems that do not support this newer format, set to true.") .booleanConf .createWithDefault(false) From 99698279dfab35c44fbd607810ca2e220ada8d8a Mon Sep 17 00:00:00 2001 From: Rong Tang Date: Wed, 26 Sep 2018 10:37:17 -0500 Subject: [PATCH 107/879] [SPARK-25509][CORE] Windows doesn't support POSIX permissions ## What changes were proposed in this pull request? SHS V2 cannot enabled in Windows, because windows doesn't support POSIX permission. ## How was this patch tested? test case fails in windows without this fix. org.apache.spark.deploy.history.HistoryServerDiskManagerSuite test("leasing space") SHS V2 cannot run successfully in Windows without this fix. java.lang.UnsupportedOperationException: 'posix:permissions' not supported as initial attribute at sun.nio.fs.WindowsSecurityDescriptor.fromAttribute(WindowsSecurityDescriptor.java:358) Closes #22520 from jianjianjiao/FixWindowsPermssionsIssue. Authored-by: Rong Tang Signed-off-by: Sean Owen (cherry picked from commit a2ac5a72ccd2b14c8492d4a6da9e8b30f0f3c9b4) Signed-off-by: Sean Owen --- .../apache/spark/deploy/history/FsHistoryProvider.scala | 6 ++---- .../spark/deploy/history/HistoryServerDiskManager.scala | 7 ++----- 2 files changed, 4 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 44d23908146c..c23a659e76df 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -19,7 +19,6 @@ package org.apache.spark.deploy.history import java.io.{File, FileNotFoundException, IOException} import java.nio.file.Files -import java.nio.file.attribute.PosixFilePermissions import java.util.{Date, ServiceLoader} import java.util.concurrent.{ConcurrentHashMap, ExecutorService, Future, TimeUnit} import java.util.zip.{ZipEntry, ZipOutputStream} @@ -133,9 +132,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // Visible for testing. private[history] val listing: KVStore = storePath.map { path => - val perms = PosixFilePermissions.fromString("rwx------") - val dbPath = Files.createDirectories(new File(path, "listing.ldb").toPath(), - PosixFilePermissions.asFileAttribute(perms)).toFile() + val dbPath = Files.createDirectories(new File(path, "listing.ldb").toPath()).toFile() + Utils.chmod700(dbPath) val metadata = new FsHistoryProviderMetadata(CURRENT_LISTING_VERSION, AppStatusStore.CURRENT_VERSION, logDir.toString()) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala index c03a360b91ef..ad0dd23cb59c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala @@ -18,8 +18,6 @@ package org.apache.spark.deploy.history import java.io.File -import java.nio.file.Files -import java.nio.file.attribute.PosixFilePermissions import java.util.concurrent.atomic.AtomicLong import scala.collection.JavaConverters._ @@ -107,9 +105,8 @@ private class HistoryServerDiskManager( val needed = approximateSize(eventLogSize, isCompressed) makeRoom(needed) - val perms = PosixFilePermissions.fromString("rwx------") - val tmp = Files.createTempDirectory(tmpStoreDir.toPath(), "appstore", - PosixFilePermissions.asFileAttribute(perms)).toFile() + val tmp = Utils.createTempDir(tmpStoreDir.getPath(), "appstore") + Utils.chmod700(tmp) updateUsage(needed) val current = currentUsage.get() From dc60476133b76d8e80ac4d925d6e146cf65e9d2b Mon Sep 17 00:00:00 2001 From: Reza Safi Date: Wed, 26 Sep 2018 09:29:58 -0700 Subject: [PATCH 108/879] [SPARK-25318] Add exception handling when wrapping the input stream during the the fetch or stage retry in response to a corrupted block SPARK-4105 provided a solution to block corruption issue by retrying the fetch or the stage. In that solution there is a step that wraps the input stream with compression and/or encryption. This step is prone to exceptions, but in the current code there is no exception handling for this step and this has caused confusion for the user. The confusion was that after SPARK-4105 the user expects to see either a fetchFailed exception or a warning about a corrupted block. However an exception during wrapping can fail the job without any of those. This change adds exception handling for the wrapping step and also adds a fetch retry if we experience a corruption during the wrapping step. The reason for adding the retry is that usually user won't experience the same failure after rerunning the job and so it seems reasonable try to fetch and wrap one more time instead of failing. Closes #22325 from rezasafi/localcorruption. Authored-by: Reza Safi Signed-off-by: Marcelo Vanzin (cherry picked from commit bd2ae857d1c5f251056de38a7a40540986756b94) Signed-off-by: Marcelo Vanzin --- .../storage/ShuffleBlockFetcherIterator.scala | 50 +++++++++---------- 1 file changed, 25 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index e534c746433f..aecc2284a958 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -448,35 +448,35 @@ final class ShuffleBlockFetcherIterator( buf.release() throwFetchFailedException(blockId, address, e) } - - input = streamWrapper(blockId, in) - // Only copy the stream if it's wrapped by compression or encryption, also the size of - // block is small (the decompressed block is smaller than maxBytesInFlight) - if (detectCorrupt && !input.eq(in) && size < maxBytesInFlight / 3) { - val originalInput = input - val out = new ChunkedByteBufferOutputStream(64 * 1024, ByteBuffer.allocate) - try { + var isStreamCopied: Boolean = false + try { + input = streamWrapper(blockId, in) + // Only copy the stream if it's wrapped by compression or encryption, also the size of + // block is small (the decompressed block is smaller than maxBytesInFlight) + if (detectCorrupt && !input.eq(in) && size < maxBytesInFlight / 3) { + isStreamCopied = true + val out = new ChunkedByteBufferOutputStream(64 * 1024, ByteBuffer.allocate) // Decompress the whole block at once to detect any corruption, which could increase // the memory usage tne potential increase the chance of OOM. // TODO: manage the memory used here, and spill it into disk in case of OOM. - Utils.copyStream(input, out) - out.close() + Utils.copyStream(input, out, closeStreams = true) input = out.toChunkedByteBuffer.toInputStream(dispose = true) - } catch { - case e: IOException => - buf.release() - if (buf.isInstanceOf[FileSegmentManagedBuffer] - || corruptedBlocks.contains(blockId)) { - throwFetchFailedException(blockId, address, e) - } else { - logWarning(s"got an corrupted block $blockId from $address, fetch again", e) - corruptedBlocks += blockId - fetchRequests += FetchRequest(address, Array((blockId, size))) - result = null - } - } finally { - // TODO: release the buf here to free memory earlier - originalInput.close() + } + } catch { + case e: IOException => + buf.release() + if (buf.isInstanceOf[FileSegmentManagedBuffer] + || corruptedBlocks.contains(blockId)) { + throwFetchFailedException(blockId, address, e) + } else { + logWarning(s"got an corrupted block $blockId from $address, fetch again", e) + corruptedBlocks += blockId + fetchRequests += FetchRequest(address, Array((blockId, size))) + result = null + } + } finally { + // TODO: release the buf here to free memory earlier + if (isStreamCopied) { in.close() } } From 8d172007968dc4f1d4a091ccb9e16cd785c0a363 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 26 Sep 2018 10:15:16 -0700 Subject: [PATCH 109/879] [SPARK-24519][CORE] Compute SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS only once ## What changes were proposed in this pull request? Previously SPARK-24519 created a modifiable config SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS. However, the config is being parsed for every creation of MapStatus, which could be very expensive. Another problem with the previous approach is that it created the illusion that this can be changed dynamically at runtime, which was not true. This PR changes it so the config is computed only once. ## How was this patch tested? Removed a test case that's no longer valid. Closes #22521 from rxin/SPARK-24519. Authored-by: Reynold Xin Signed-off-by: Dongjoon Hyun (cherry picked from commit e702fb1d5218d062fcb8e618b92dad7958eb4062) Signed-off-by: Dongjoon Hyun --- .../apache/spark/scheduler/MapStatus.scala | 12 ++++++-- .../spark/scheduler/MapStatusSuite.scala | 28 ------------------- 2 files changed, 9 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 659694dd189a..0e221edf3965 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -49,10 +49,16 @@ private[spark] sealed trait MapStatus { private[spark] object MapStatus { + /** + * Min partition number to use [[HighlyCompressedMapStatus]]. A bit ugly here because in test + * code we can't assume SparkEnv.get exists. + */ + private lazy val minPartitionsToUseHighlyCompressMapStatus = Option(SparkEnv.get) + .map(_.conf.get(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS)) + .getOrElse(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS.defaultValue.get) + def apply(loc: BlockManagerId, uncompressedSizes: Array[Long]): MapStatus = { - if (uncompressedSizes.length > Option(SparkEnv.get) - .map(_.conf.get(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS)) - .getOrElse(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS.defaultValue.get)) { + if (uncompressedSizes.length > minPartitionsToUseHighlyCompressMapStatus) { HighlyCompressedMapStatus(loc, uncompressedSizes) } else { new CompressedMapStatus(loc, uncompressedSizes) diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index 354e6386fa60..2155a0f2b6c2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -188,32 +188,4 @@ class MapStatusSuite extends SparkFunSuite { assert(count === 3000) } } - - test("SPARK-24519: HighlyCompressedMapStatus has configurable threshold") { - val conf = new SparkConf() - val env = mock(classOf[SparkEnv]) - doReturn(conf).when(env).conf - SparkEnv.set(env) - val sizes = Array.fill[Long](500)(150L) - // Test default value - val status = MapStatus(null, sizes) - assert(status.isInstanceOf[CompressedMapStatus]) - // Test Non-positive values - for (s <- -1 to 0) { - assertThrows[IllegalArgumentException] { - conf.set(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS, s) - val status = MapStatus(null, sizes) - } - } - // Test positive values - Seq(1, 100, 499, 500, 501).foreach { s => - conf.set(config.SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS, s) - val status = MapStatus(null, sizes) - if(sizes.length > s) { - assert(status.isInstanceOf[HighlyCompressedMapStatus]) - } else { - assert(status.isInstanceOf[CompressedMapStatus]) - } - } - } } From 2ff91f213d1312357d36fa2012387a131389077e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 26 Sep 2018 17:47:05 -0700 Subject: [PATCH 110/879] [SPARK-25454][SQL] add a new config for picking minimum precision for integral literals ## What changes were proposed in this pull request? https://github.com/apache/spark/pull/20023 proposed to allow precision lose during decimal operations, to reduce the possibilities of overflow. This is a behavior change and is protected by the DECIMAL_OPERATIONS_ALLOW_PREC_LOSS config. However, that PR introduced another behavior change: pick a minimum precision for integral literals, which is not protected by a config. This PR add a new config for it: `spark.sql.literal.pickMinimumPrecision`. This can allow users to work around issue in SPARK-25454, which is caused by a long-standing bug of negative scale. ## How was this patch tested? a new test Closes #22494 from cloud-fan/decimal. Authored-by: Wenchen Fan Signed-off-by: gatorsmile (cherry picked from commit d0990e3dfee752a6460a6360e1a773138364d774) Signed-off-by: gatorsmile --- .../sql/catalyst/analysis/DecimalPrecision.scala | 10 ++++++---- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 11 +++++++++++ .../scala/org/apache/spark/sql/SQLQuerySuite.scala | 7 +++++++ 3 files changed, 24 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala index e511f8064e28..82692334544e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala @@ -290,11 +290,13 @@ object DecimalPrecision extends TypeCoercionRule { // potentially loosing 11 digits of the fractional part. Using only the precision needed // by the Literal, instead, the result would be DECIMAL(38 + 1 + 1, 18), which would // become DECIMAL(38, 16), safely having a much lower precision loss. - case (l: Literal, r) if r.dataType.isInstanceOf[DecimalType] - && l.dataType.isInstanceOf[IntegralType] => + case (l: Literal, r) if r.dataType.isInstanceOf[DecimalType] && + l.dataType.isInstanceOf[IntegralType] && + SQLConf.get.literalPickMinimumPrecision => b.makeCopy(Array(Cast(l, DecimalType.fromLiteral(l)), r)) - case (l, r: Literal) if l.dataType.isInstanceOf[DecimalType] - && r.dataType.isInstanceOf[IntegralType] => + case (l, r: Literal) if l.dataType.isInstanceOf[DecimalType] && + r.dataType.isInstanceOf[IntegralType] && + SQLConf.get.literalPickMinimumPrecision => b.makeCopy(Array(l, Cast(r, DecimalType.fromLiteral(r)))) // Promote integers inside a binary expression with fixed-precision decimals to decimals, // and fixed-precision decimals in an expression with floats / doubles to doubles diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index bacd5e9f8517..99d2f562082e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1311,6 +1311,15 @@ object SQLConf { .booleanConf .createWithDefault(true) + val LITERAL_PICK_MINIMUM_PRECISION = + buildConf("spark.sql.legacy.literal.pickMinimumPrecision") + .internal() + .doc("When integral literal is used in decimal operations, pick a minimum precision " + + "required by the literal if this config is true, to make the resulting precision and/or " + + "scale smaller. This can reduce the possibility of precision lose and/or overflow.") + .booleanConf + .createWithDefault(true) + val SQL_OPTIONS_REDACTION_PATTERN = buildConf("spark.sql.redaction.options.regex") .doc("Regex to decide which keys in a Spark SQL command's options map contain sensitive " + @@ -1894,6 +1903,8 @@ class SQLConf extends Serializable with Logging { def decimalOperationsAllowPrecisionLoss: Boolean = getConf(DECIMAL_OPERATIONS_ALLOW_PREC_LOSS) + def literalPickMinimumPrecision: Boolean = getConf(LITERAL_PICK_MINIMUM_PRECISION) + def continuousStreamingExecutorQueueSize: Int = getConf(CONTINUOUS_STREAMING_EXECUTOR_QUEUE_SIZE) def continuousStreamingExecutorPollIntervalMs: Long = 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 8fcebb35a054..631ab1b7ece7 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 @@ -2849,6 +2849,13 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val result = ds.flatMap(_.bar).distinct result.rdd.isEmpty } + + test("SPARK-25454: decimal division with negative scale") { + // TODO: completely fix this issue even when LITERAL_PRECISE_PRECISION is true. + withSQLConf(SQLConf.LITERAL_PICK_MINIMUM_PRECISION.key -> "false") { + checkAnswer(sql("select 26393499451 / (1e6 * 1000)"), Row(BigDecimal("26.3934994510000"))) + } + } } case class Foo(bar: Option[String]) From 7656358adc39eb8eb881368ab5a066fbf86149c8 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 27 Sep 2018 09:51:20 +0800 Subject: [PATCH 111/879] [SPARK-25540][SQL][PYSPARK] Make HiveContext in PySpark behave as the same as Scala. ## What changes were proposed in this pull request? In Scala, `HiveContext` sets a config `spark.sql.catalogImplementation` of the given `SparkContext` and then passes to `SparkSession.builder`. The `HiveContext` in PySpark should behave as the same as Scala. ## How was this patch tested? Existing tests. Closes #22552 from ueshin/issues/SPARK-25540/hive_context. Authored-by: Takuya UESHIN Signed-off-by: Wenchen Fan (cherry picked from commit c3c45cbd76d91d591d98cf8411fcfd30079f5969) Signed-off-by: Wenchen Fan --- python/pyspark/sql/context.py | 3 ++- python/pyspark/sql/session.py | 19 ++++++++++++++----- 2 files changed, 16 insertions(+), 6 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 9c094dd9a903..1938965a7e17 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -485,7 +485,8 @@ def __init__(self, sparkContext, jhiveContext=None): "SparkSession.builder.enableHiveSupport().getOrCreate() instead.", DeprecationWarning) if jhiveContext is None: - sparkSession = SparkSession.builder.enableHiveSupport().getOrCreate() + sparkContext._conf.set("spark.sql.catalogImplementation", "hive") + sparkSession = SparkSession.builder._sparkContext(sparkContext).getOrCreate() else: sparkSession = SparkSession(sparkContext, jhiveContext.sparkSession()) SQLContext.__init__(self, sparkContext, sparkSession, jhiveContext) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index 51a38ebfd19f..a5e287257731 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -83,6 +83,7 @@ class Builder(object): _lock = RLock() _options = {} + _sc = None @since(2.0) def config(self, key=None, value=None, conf=None): @@ -139,6 +140,11 @@ def enableHiveSupport(self): """ return self.config("spark.sql.catalogImplementation", "hive") + def _sparkContext(self, sc): + with self._lock: + self._sc = sc + return self + @since(2.0) def getOrCreate(self): """Gets an existing :class:`SparkSession` or, if there is no existing one, creates a @@ -167,11 +173,14 @@ def getOrCreate(self): from pyspark.conf import SparkConf session = SparkSession._instantiatedSession if session is None or session._sc._jsc is None: - sparkConf = SparkConf() - for key, value in self._options.items(): - sparkConf.set(key, value) - sc = SparkContext.getOrCreate(sparkConf) - # This SparkContext may be an existing one. + if self._sc is not None: + sc = self._sc + else: + sparkConf = SparkConf() + for key, value in self._options.items(): + sparkConf.set(key, value) + sc = SparkContext.getOrCreate(sparkConf) + # This SparkContext may be an existing one. for key, value in self._options.items(): # we need to propagate the confs # before we create the SparkSession. Otherwise, confs like From f12769e73a81f5a333c4ff91813ad698ffb16eec Mon Sep 17 00:00:00 2001 From: Shahid Date: Wed, 26 Sep 2018 21:10:39 -0700 Subject: [PATCH 112/879] [SPARK-25536][CORE] metric value for METRIC_OUTPUT_RECORDS_WRITTEN is incorrect ## What changes were proposed in this pull request? changed metric value of METRIC_OUTPUT_RECORDS_WRITTEN from 'task.metrics.inputMetrics.recordsRead' to 'task.metrics.outputMetrics.recordsWritten'. This bug was introduced in SPARK-22190. https://github.com/apache/spark/pull/19426 ## How was this patch tested? Existing tests Closes #22555 from shahidki31/SPARK-25536. Authored-by: Shahid Signed-off-by: Dongjoon Hyun (cherry picked from commit 5def10e61e49dba85f4d8b39c92bda15137990a2) Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/executor/Executor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 27f7ec8fda68..97dfcc482b17 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -499,7 +499,7 @@ private[spark] class Executor( executorSource.METRIC_OUTPUT_BYTES_WRITTEN .inc(task.metrics.outputMetrics.bytesWritten) executorSource.METRIC_OUTPUT_RECORDS_WRITTEN - .inc(task.metrics.inputMetrics.recordsRead) + .inc(task.metrics.outputMetrics.recordsWritten) executorSource.METRIC_RESULT_SIZE.inc(task.metrics.resultSize) executorSource.METRIC_DISK_BYTES_SPILLED.inc(task.metrics.diskBytesSpilled) executorSource.METRIC_MEMORY_BYTES_SPILLED.inc(task.metrics.memoryBytesSpilled) From 01c000b522d4b2e0ebbe9617be591e5de3b59dea Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 27 Sep 2018 12:38:14 +0800 Subject: [PATCH 113/879] Revert "[SPARK-25540][SQL][PYSPARK] Make HiveContext in PySpark behave as the same as Scala." This reverts commit 7656358adc39eb8eb881368ab5a066fbf86149c8. --- python/pyspark/sql/context.py | 3 +-- python/pyspark/sql/session.py | 19 +++++-------------- 2 files changed, 6 insertions(+), 16 deletions(-) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 1938965a7e17..9c094dd9a903 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -485,8 +485,7 @@ def __init__(self, sparkContext, jhiveContext=None): "SparkSession.builder.enableHiveSupport().getOrCreate() instead.", DeprecationWarning) if jhiveContext is None: - sparkContext._conf.set("spark.sql.catalogImplementation", "hive") - sparkSession = SparkSession.builder._sparkContext(sparkContext).getOrCreate() + sparkSession = SparkSession.builder.enableHiveSupport().getOrCreate() else: sparkSession = SparkSession(sparkContext, jhiveContext.sparkSession()) SQLContext.__init__(self, sparkContext, sparkSession, jhiveContext) diff --git a/python/pyspark/sql/session.py b/python/pyspark/sql/session.py index a5e287257731..51a38ebfd19f 100644 --- a/python/pyspark/sql/session.py +++ b/python/pyspark/sql/session.py @@ -83,7 +83,6 @@ class Builder(object): _lock = RLock() _options = {} - _sc = None @since(2.0) def config(self, key=None, value=None, conf=None): @@ -140,11 +139,6 @@ def enableHiveSupport(self): """ return self.config("spark.sql.catalogImplementation", "hive") - def _sparkContext(self, sc): - with self._lock: - self._sc = sc - return self - @since(2.0) def getOrCreate(self): """Gets an existing :class:`SparkSession` or, if there is no existing one, creates a @@ -173,14 +167,11 @@ def getOrCreate(self): from pyspark.conf import SparkConf session = SparkSession._instantiatedSession if session is None or session._sc._jsc is None: - if self._sc is not None: - sc = self._sc - else: - sparkConf = SparkConf() - for key, value in self._options.items(): - sparkConf.set(key, value) - sc = SparkContext.getOrCreate(sparkConf) - # This SparkContext may be an existing one. + sparkConf = SparkConf() + for key, value in self._options.items(): + sparkConf.set(key, value) + sc = SparkContext.getOrCreate(sparkConf) + # This SparkContext may be an existing one. for key, value in self._options.items(): # we need to propagate the confs # before we create the SparkSession. Otherwise, confs like From 0cf4c5bbec920490e902dca961d226e172a0c15b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E7=8E=8B=E5=B0=8F=E5=88=9A?= Date: Thu, 27 Sep 2018 00:02:05 -0500 Subject: [PATCH 114/879] [SPARK-25468][WEBUI] Highlight current page index in the spark UI MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? This PR is highlight current page index in the spark UI and history server UI, https://issues.apache.org/jira/browse/SPARK-25468 I have add the following code in webui.css ``` .paginate_button.active>a { color: #999999; text-decoration: underline; } ``` ## How was this patch tested? Manual tests for Chrome, Firefox and Safari Before modifying: ![image](https://user-images.githubusercontent.com/10048468/45914897-01ca6c00-be7e-11e8-8e31-47d45db0c3bf.png) After modifying: ![image](https://user-images.githubusercontent.com/10048468/45913987-7e564e00-be70-11e8-9c16-de17e2c63308.png) Closes #22516 from Adamyuanyuan/spark-adam-25468. Lead-authored-by: 王小刚 Co-authored-by: Adam Wang Signed-off-by: Sean Owen (cherry picked from commit 8b727994edd27104d49c6d690f93c6858fb9e1fc) Signed-off-by: Sean Owen --- core/src/main/resources/org/apache/spark/ui/static/webui.css | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 935d9b1aec61..4b060b0f4e53 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -251,4 +251,9 @@ a.expandbutton { .table-cell-width-limited td { max-width: 600px; +} + +.paginate_button.active > a { + color: #999999; + text-decoration: underline; } \ No newline at end of file From 0b4e58187b787cc7a6d57a2a9d467934ece24252 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 27 Sep 2018 15:02:20 +0800 Subject: [PATCH 115/879] [SPARK-23715][SQL][DOC] improve document for from/to_utc_timestamp ## What changes were proposed in this pull request? We have an agreement that the behavior of `from/to_utc_timestamp` is corrected, although the function itself doesn't make much sense in Spark: https://issues.apache.org/jira/browse/SPARK-23715 This PR improves the document. ## How was this patch tested? N/A Closes #22543 from cloud-fan/doc. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit ff876137faba1802b66ecd483ba15f6ccd83ffc5) Signed-off-by: Wenchen Fan --- R/pkg/R/functions.R | 26 ++++++++++++---- python/pyspark/sql/functions.py | 30 +++++++++++++++---- .../expressions/datetimeExpressions.scala | 30 +++++++++++++++---- 3 files changed, 68 insertions(+), 18 deletions(-) diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 572dee50127b..63bd427acb1a 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -2203,9 +2203,16 @@ setMethod("from_json", signature(x = "Column", schema = "characterOrstructType") }) #' @details -#' \code{from_utc_timestamp}: Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a -#' time in UTC, and renders that time as a timestamp in the given time zone. For example, 'GMT+1' -#' would yield '2017-07-14 03:40:00.0'. +#' \code{from_utc_timestamp}: This is a common function for databases supporting TIMESTAMP WITHOUT +#' TIMEZONE. This function takes a timestamp which is timezone-agnostic, and interprets it as a +#' timestamp in UTC, and renders that timestamp as a timestamp in the given time zone. +#' However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not +#' timezone-agnostic. So in Spark this function just shift the timestamp value from UTC timezone to +#' the given timezone. +#' This function may return confusing result if the input is a string with timezone, e.g. +#' (\code{2018-03-13T06:18:23+00:00}). The reason is that, Spark firstly cast the string to +#' timestamp according to the timezone in the string, and finally display the result by converting +#' the timestamp to string according to the session local timezone. #' #' @rdname column_datetime_diff_functions #' @@ -2261,9 +2268,16 @@ setMethod("next_day", signature(y = "Column", x = "character"), }) #' @details -#' \code{to_utc_timestamp}: Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a -#' time in the given time zone, and renders that time as a timestamp in UTC. For example, 'GMT+1' -#' would yield '2017-07-14 01:40:00.0'. +#' \code{to_utc_timestamp}: This is a common function for databases supporting TIMESTAMP WITHOUT +#' TIMEZONE. This function takes a timestamp which is timezone-agnostic, and interprets it as a +#' timestamp in the given timezone, and renders that timestamp as a timestamp in UTC. +#' However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not +#' timezone-agnostic. So in Spark this function just shift the timestamp value from the given +#' timezone to UTC timezone. +#' This function may return confusing result if the input is a string with timezone, e.g. +#' (\code{2018-03-13T06:18:23+00:00}). The reason is that, Spark firstly cast the string to +#' timestamp according to the timezone in the string, and finally display the result by converting +#' the timestamp to string according to the session local timezone. #' #' @rdname column_datetime_diff_functions #' @aliases to_utc_timestamp to_utc_timestamp,Column,character-method diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 6da5237d18de..8c541798bcc0 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -1283,9 +1283,18 @@ def unix_timestamp(timestamp=None, format='yyyy-MM-dd HH:mm:ss'): @since(1.5) def from_utc_timestamp(timestamp, tz): """ - Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders - that time as a timestamp in the given time zone. For example, 'GMT+1' would yield - '2017-07-14 03:40:00.0'. + This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function + takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in UTC, and + renders that timestamp as a timestamp in the given time zone. + + However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not + timezone-agnostic. So in Spark this function just shift the timestamp value from UTC timezone to + the given timezone. + + This function may return confusing result if the input is a string with timezone, e.g. + '2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp + according to the timezone in the string, and finally display the result by converting the + timestamp to string according to the session local timezone. :param timestamp: the column that contains timestamps :param tz: a string that has the ID of timezone, e.g. "GMT", "America/Los_Angeles", etc @@ -1308,9 +1317,18 @@ def from_utc_timestamp(timestamp, tz): @since(1.5) def to_utc_timestamp(timestamp, tz): """ - Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in the given time - zone, and renders that time as a timestamp in UTC. For example, 'GMT+1' would yield - '2017-07-14 01:40:00.0'. + This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function + takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in the given + timezone, and renders that timestamp as a timestamp in UTC. + + However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not + timezone-agnostic. So in Spark this function just shift the timestamp value from the given + timezone to UTC timezone. + + This function may return confusing result if the input is a string with timezone, e.g. + '2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp + according to the timezone in the string, and finally display the result by converting the + timestamp to string according to the session local timezone. :param timestamp: the column that contains timestamps :param tz: a string that has the ID of timezone, e.g. "GMT", "America/Los_Angeles", etc diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala index eb78e394f985..45e17ae235a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -1018,9 +1018,18 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S } /** - * Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in UTC, and renders - * that time as a timestamp in the given time zone. For example, 'GMT+1' would yield - * '2017-07-14 03:40:00.0'. + * This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function + * takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in UTC, and + * renders that timestamp as a timestamp in the given time zone. + * + * However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not + * timezone-agnostic. So in Spark this function just shift the timestamp value from UTC timezone to + * the given timezone. + * + * This function may return confusing result if the input is a string with timezone, e.g. + * '2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp + * according to the timezone in the string, and finally display the result by converting the + * timestamp to string according to the session local timezone. */ // scalastyle:off line.size.limit @ExpressionDescription( @@ -1215,9 +1224,18 @@ case class MonthsBetween( } /** - * Given a timestamp like '2017-07-14 02:40:00.0', interprets it as a time in the given time zone, - * and renders that time as a timestamp in UTC. For example, 'GMT+1' would yield - * '2017-07-14 01:40:00.0'. + * This is a common function for databases supporting TIMESTAMP WITHOUT TIMEZONE. This function + * takes a timestamp which is timezone-agnostic, and interprets it as a timestamp in the given + * timezone, and renders that timestamp as a timestamp in UTC. + * + * However, timestamp in Spark represents number of microseconds from the Unix epoch, which is not + * timezone-agnostic. So in Spark this function just shift the timestamp value from the given + * timezone to UTC timezone. + * + * This function may return confusing result if the input is a string with timezone, e.g. + * '2018-03-13T06:18:23+00:00'. The reason is that, Spark firstly cast the string to timestamp + * according to the timezone in the string, and finally display the result by converting the + * timestamp to string according to the session local timezone. */ // scalastyle:off line.size.limit @ExpressionDescription( From 53eb8585452b8637c8264c5ceb8d5fe28e7ae936 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 27 Sep 2018 15:13:18 +0800 Subject: [PATCH 116/879] [SPARK-25314][SQL] Fix Python UDF accessing attributes from both side of join in join conditions ## What changes were proposed in this pull request? Thanks for bahchis reporting this. It is more like a follow up work for #16581, this PR fix the scenario of Python UDF accessing attributes from both side of join in join condition. ## How was this patch tested? Add regression tests in PySpark and `BatchEvalPythonExecSuite`. Closes #22326 from xuanyuanking/SPARK-25314. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan (cherry picked from commit 2a8cbfddba2a59d144b32910c68c22d0199093fe) Signed-off-by: Wenchen Fan --- python/pyspark/sql/tests.py | 64 +++++++++++++++++++ .../sql/catalyst/optimizer/Optimizer.scala | 8 ++- .../spark/sql/catalyst/optimizer/joins.scala | 49 ++++++++++++++ 3 files changed, 119 insertions(+), 2 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index cb186dee3456..dece1dabea2c 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -552,6 +552,70 @@ def test_udf_in_filter_on_top_of_join(self): df = left.crossJoin(right).filter(f("a", "b")) self.assertEqual(df.collect(), [Row(a=1, b=1)]) + def test_udf_in_join_condition(self): + # regression test for SPARK-25314 + from pyspark.sql.functions import udf + left = self.spark.createDataFrame([Row(a=1)]) + right = self.spark.createDataFrame([Row(b=1)]) + f = udf(lambda a, b: a == b, BooleanType()) + df = left.join(right, f("a", "b")) + with self.assertRaisesRegexp(AnalysisException, 'Detected implicit cartesian product'): + df.collect() + with self.sql_conf({"spark.sql.crossJoin.enabled": True}): + self.assertEqual(df.collect(), [Row(a=1, b=1)]) + + def test_udf_in_left_semi_join_condition(self): + # regression test for SPARK-25314 + from pyspark.sql.functions import udf + left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) + right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1)]) + f = udf(lambda a, b: a == b, BooleanType()) + df = left.join(right, f("a", "b"), "leftsemi") + with self.assertRaisesRegexp(AnalysisException, 'Detected implicit cartesian product'): + df.collect() + with self.sql_conf({"spark.sql.crossJoin.enabled": True}): + self.assertEqual(df.collect(), [Row(a=1, a1=1, a2=1)]) + + def test_udf_and_common_filter_in_join_condition(self): + # regression test for SPARK-25314 + # test the complex scenario with both udf and common filter + from pyspark.sql.functions import udf + left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) + right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1), Row(b=1, b1=3, b2=1)]) + f = udf(lambda a, b: a == b, BooleanType()) + df = left.join(right, [f("a", "b"), left.a1 == right.b1]) + # do not need spark.sql.crossJoin.enabled=true for udf is not the only join condition. + self.assertEqual(df.collect(), [Row(a=1, a1=1, a2=1, b=1, b1=1, b2=1)]) + + def test_udf_and_common_filter_in_left_semi_join_condition(self): + # regression test for SPARK-25314 + # test the complex scenario with both udf and common filter + from pyspark.sql.functions import udf + left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) + right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1), Row(b=1, b1=3, b2=1)]) + f = udf(lambda a, b: a == b, BooleanType()) + df = left.join(right, [f("a", "b"), left.a1 == right.b1], "left_semi") + # do not need spark.sql.crossJoin.enabled=true for udf is not the only join condition. + self.assertEqual(df.collect(), [Row(a=1, a1=1, a2=1)]) + + def test_udf_not_supported_in_join_condition(self): + # regression test for SPARK-25314 + # test python udf is not supported in join type besides left_semi and inner join. + from pyspark.sql.functions import udf + left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) + right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1), Row(b=1, b1=3, b2=1)]) + f = udf(lambda a, b: a == b, BooleanType()) + + def runWithJoinType(join_type, type_string): + with self.assertRaisesRegexp( + AnalysisException, + 'Using PythonUDF.*%s is not supported.' % type_string): + left.join(right, [f("a", "b"), left.a1 == right.b1], join_type).collect() + runWithJoinType("full", "FullOuter") + runWithJoinType("left", "LeftOuter") + runWithJoinType("right", "RightOuter") + runWithJoinType("leftanti", "LeftAnti") + def test_udf_without_arguments(self): self.spark.catalog.registerFunction("foo", lambda: "bar") [row] = self.spark.sql("SELECT foo()").collect() diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index b489a6745dd3..079cd358348e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -165,7 +165,10 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) Batch("LocalRelation", fixedPoint, ConvertToLocalRelation, PropagateEmptyRelation) :+ - // The following batch should be executed after batch "Join Reorder" and "LocalRelation". + Batch("Extract PythonUDF From JoinCondition", Once, + PullOutPythonUDFInJoinCondition) :+ + // The following batch should be executed after batch "Join Reorder" "LocalRelation" and + // "Extract PythonUDF From JoinCondition". Batch("Check Cartesian Products", Once, CheckCartesianProducts) :+ Batch("RewriteSubquery", Once, @@ -202,7 +205,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) ReplaceDistinctWithAggregate.ruleName :: PullupCorrelatedPredicates.ruleName :: RewriteCorrelatedScalarSubquery.ruleName :: - RewritePredicateSubquery.ruleName :: Nil + RewritePredicateSubquery.ruleName :: + PullOutPythonUDFInJoinCondition.ruleName :: Nil /** * Optimize all the subqueries inside expression. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index edbeaf273fd6..7149edee0173 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.annotation.tailrec +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ExtractFiltersAndInnerJoins import org.apache.spark.sql.catalyst.plans._ @@ -152,3 +153,51 @@ object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { if (j.joinType == newJoinType) f else Filter(condition, j.copy(joinType = newJoinType)) } } + +/** + * PythonUDF in join condition can not be evaluated, this rule will detect the PythonUDF + * and pull them out from join condition. For python udf accessing attributes from only one side, + * they are pushed down by operation push down rules. If not (e.g. user disables filter push + * down rules), we need to pull them out in this rule too. + */ +object PullOutPythonUDFInJoinCondition extends Rule[LogicalPlan] with PredicateHelper { + def hasPythonUDF(expression: Expression): Boolean = { + expression.collectFirst { case udf: PythonUDF => udf }.isDefined + } + + override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + case j @ Join(_, _, joinType, condition) + if condition.isDefined && hasPythonUDF(condition.get) => + if (!joinType.isInstanceOf[InnerLike] && joinType != LeftSemi) { + // The current strategy only support InnerLike and LeftSemi join because for other type, + // it breaks SQL semantic if we run the join condition as a filter after join. If we pass + // the plan here, it'll still get a an invalid PythonUDF RuntimeException with message + // `requires attributes from more than one child`, we throw firstly here for better + // readable information. + throw new AnalysisException("Using PythonUDF in join condition of join type" + + s" $joinType is not supported.") + } + // If condition expression contains python udf, it will be moved out from + // the new join conditions. + val (udf, rest) = + splitConjunctivePredicates(condition.get).partition(hasPythonUDF) + val newCondition = if (rest.isEmpty) { + logWarning(s"The join condition:$condition of the join plan contains PythonUDF only," + + s" it will be moved out and the join plan will be turned to cross join.") + None + } else { + Some(rest.reduceLeft(And)) + } + val newJoin = j.copy(condition = newCondition) + joinType match { + case _: InnerLike => Filter(udf.reduceLeft(And), newJoin) + case LeftSemi => + Project( + j.left.output.map(_.toAttribute), + Filter(udf.reduceLeft(And), newJoin.copy(joinType = Inner))) + case _ => + throw new AnalysisException("Using PythonUDF in join condition of join type" + + s" $joinType is not supported.") + } + } +} From 3c78ea2589e1e2f3824ae6fa273eceaee3934391 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Thu, 27 Sep 2018 15:04:59 +0800 Subject: [PATCH 117/879] [SPARK-25522][SQL] Improve type promotion for input arguments of elementAt function ## What changes were proposed in this pull request? In ElementAt, when first argument is MapType, we should coerce the key type and the second argument based on findTightestCommonType. This is not happening currently. We may produce wrong output as we will incorrectly downcast the right hand side double expression to int. ```SQL spark-sql> select element_at(map(1,"one", 2, "two"), 2.2); two ``` Also, when the first argument is ArrayType, the second argument should be an integer type or a smaller integral type that can be safely casted to an integer type. Currently we may do an unsafe cast. In the following case, we should fail with an error as 2.2 is not a integer index. But instead we down cast it to int currently and return a result instead. ```SQL spark-sql> select element_at(array(1,2), 1.24D); 1 ``` This PR also supports implicit cast between two MapTypes. I have followed similar logic that exists today to do implicit casts between two array types. ## How was this patch tested? Added new tests in DataFrameFunctionSuite, TypeCoercionSuite. Closes #22544 from dilipbiswal/SPARK-25522. Authored-by: Dilip Biswal Signed-off-by: Wenchen Fan (cherry picked from commit d03e0af80d7659f12821cc2442efaeaee94d3985) Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/TypeCoercion.scala | 19 +++++ .../spark/sql/catalyst/expressions/Cast.scala | 2 +- .../expressions/collectionOperations.scala | 37 ++++++--- .../catalyst/analysis/TypeCoercionSuite.scala | 43 +++++++++-- .../spark/sql/DataFrameFunctionsSuite.scala | 75 ++++++++++++++++++- 5 files changed, 154 insertions(+), 22 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 49d286f6cf12..72ac80e0a0a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -950,6 +950,25 @@ object TypeCoercion { if !Cast.forceNullable(fromType, toType) => implicitCast(fromType, toType).map(ArrayType(_, false)).orNull + // Implicit cast between Map types. + // Follows the same semantics of implicit casting between two array types. + // Refer to documentation above. Make sure that both key and values + // can not be null after the implicit cast operation by calling forceNullable + // method. + case (MapType(fromKeyType, fromValueType, fn), MapType(toKeyType, toValueType, tn)) + if !Cast.forceNullable(fromKeyType, toKeyType) && Cast.resolvableNullability(fn, tn) => + if (Cast.forceNullable(fromValueType, toValueType) && !tn) { + null + } else { + val newKeyType = implicitCast(fromKeyType, toKeyType).orNull + val newValueType = implicitCast(fromValueType, toValueType).orNull + if (newKeyType != null && newValueType != null) { + MapType(newKeyType, newValueType, tn) + } else { + null + } + } + case _ => null } Option(ret) 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 8f777997bf61..ee463bf5eb6a 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 @@ -183,7 +183,7 @@ object Cast { case _ => false } - private def resolvableNullability(from: Boolean, to: Boolean) = !from || to + def resolvableNullability(from: Boolean, to: Boolean): Boolean = !from || to } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index 9cc7dbadd923..b24d7486f345 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -2154,21 +2154,34 @@ case class ElementAt(left: Expression, right: Expression) extends GetMapValueUti } override def inputTypes: Seq[AbstractDataType] = { - Seq(TypeCollection(ArrayType, MapType), - left.dataType match { - case _: ArrayType => IntegerType - case _: MapType => mapKeyType - case _ => AnyDataType // no match for a wrong 'left' expression type - } - ) + (left.dataType, right.dataType) match { + case (arr: ArrayType, e2: IntegralType) if (e2 != LongType) => + Seq(arr, IntegerType) + case (MapType(keyType, valueType, hasNull), e2) => + TypeCoercion.findTightestCommonType(keyType, e2) match { + case Some(dt) => Seq(MapType(dt, valueType, hasNull), dt) + case _ => Seq.empty + } + case (l, r) => Seq.empty + + } } override def checkInputDataTypes(): TypeCheckResult = { - super.checkInputDataTypes() match { - case f: TypeCheckResult.TypeCheckFailure => f - case TypeCheckResult.TypeCheckSuccess if left.dataType.isInstanceOf[MapType] => - TypeUtils.checkForOrderingExpr(mapKeyType, s"function $prettyName") - case TypeCheckResult.TypeCheckSuccess => TypeCheckResult.TypeCheckSuccess + (left.dataType, right.dataType) match { + case (_: ArrayType, e2) if e2 != IntegerType => + TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " + + s"been ${ArrayType.simpleString} followed by a ${IntegerType.simpleString}, but it's " + + s"[${left.dataType.catalogString}, ${right.dataType.catalogString}].") + case (MapType(e1, _, _), e2) if (!e2.sameType(e1)) => + TypeCheckResult.TypeCheckFailure(s"Input to function $prettyName should have " + + s"been ${MapType.simpleString} followed by a value of same key type, but it's " + + s"[${left.dataType.catalogString}, ${right.dataType.catalogString}].") + case (e1, _) if (!e1.isInstanceOf[MapType] && !e1.isInstanceOf[ArrayType]) => + TypeCheckResult.TypeCheckFailure(s"The first argument to function $prettyName should " + + s"have been ${ArrayType.simpleString} or ${MapType.simpleString} type, but its " + + s"${left.dataType.catalogString} type.") + case _ => TypeCheckResult.TypeCheckSuccess } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala index 0594673ecc92..0eba1c537d67 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercionSuite.scala @@ -257,12 +257,43 @@ class TypeCoercionSuite extends AnalysisTest { shouldNotCast(checkedType, IntegralType) } - test("implicit type cast - MapType(StringType, StringType)") { - val checkedType = MapType(StringType, StringType) - checkTypeCasting(checkedType, castableTypes = Seq(checkedType)) - shouldNotCast(checkedType, DecimalType) - shouldNotCast(checkedType, NumericType) - shouldNotCast(checkedType, IntegralType) + test("implicit type cast between two Map types") { + val sourceType = MapType(IntegerType, IntegerType, true) + val castableTypes = numericTypes ++ Seq(StringType).filter(!Cast.forceNullable(IntegerType, _)) + val targetTypes = numericTypes.filter(!Cast.forceNullable(IntegerType, _)).map { t => + MapType(t, sourceType.valueType, valueContainsNull = true) + } + val nonCastableTargetTypes = allTypes.filterNot(castableTypes.contains(_)).map {t => + MapType(t, sourceType.valueType, valueContainsNull = true) + } + + // Tests that its possible to setup implicit casts between two map types when + // source map's key type is integer and the target map's key type are either Byte, Short, + // Long, Double, Float, Decimal(38, 18) or String. + targetTypes.foreach { targetType => + shouldCast(sourceType, targetType, targetType) + } + + // Tests that its not possible to setup implicit casts between two map types when + // source map's key type is integer and the target map's key type are either Binary, + // Boolean, Date, Timestamp, Array, Struct, CaleandarIntervalType or NullType + nonCastableTargetTypes.foreach { targetType => + shouldNotCast(sourceType, targetType) + } + + // Tests that its not possible to cast from nullable map type to not nullable map type. + val targetNotNullableTypes = allTypes.filterNot(_ == IntegerType).map { t => + MapType(t, sourceType.valueType, valueContainsNull = false) + } + val sourceMapExprWithValueNull = + CreateMap(Seq(Literal.default(sourceType.keyType), + Literal.create(null, sourceType.valueType))) + targetNotNullableTypes.foreach { targetType => + val castDefault = + TypeCoercion.ImplicitTypeCasts.implicitCast(sourceMapExprWithValueNull, targetType) + assert(castDefault.isEmpty, + s"Should not be able to cast $sourceType to $targetType, but got $castDefault") + } } test("implicit type cast - StructType().add(\"a1\", StringType)") { 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 88dbae8c2135..60ebc5e6cc09 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 @@ -1211,11 +1211,80 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Seq(Row("3"), Row(""), Row(null)) ) - val e = intercept[AnalysisException] { + val e1 = intercept[AnalysisException] { Seq(("a string element", 1)).toDF().selectExpr("element_at(_1, _2)") } - assert(e.message.contains( - "argument 1 requires (array or map) type, however, '`_1`' is of string type")) + val errorMsg1 = + s""" + |The first argument to function element_at should have been array or map type, but + |its string type. + """.stripMargin.replace("\n", " ").trim() + assert(e1.message.contains(errorMsg1)) + + checkAnswer( + OneRowRelation().selectExpr("element_at(array(2, 1), 2S)"), + Seq(Row(1)) + ) + + checkAnswer( + OneRowRelation().selectExpr("element_at(array('a', 'b'), 1Y)"), + Seq(Row("a")) + ) + + checkAnswer( + OneRowRelation().selectExpr("element_at(array(1, 2, 3), 3)"), + Seq(Row(3)) + ) + + val e2 = intercept[AnalysisException] { + OneRowRelation().selectExpr("element_at(array('a', 'b'), 1L)") + } + val errorMsg2 = + s""" + |Input to function element_at should have been array followed by a int, but it's + |[array, bigint]. + """.stripMargin.replace("\n", " ").trim() + assert(e2.message.contains(errorMsg2)) + + checkAnswer( + OneRowRelation().selectExpr("element_at(map(1, 'a', 2, 'b'), 2Y)"), + Seq(Row("b")) + ) + + checkAnswer( + OneRowRelation().selectExpr("element_at(map(1, 'a', 2, 'b'), 1S)"), + Seq(Row("a")) + ) + + checkAnswer( + OneRowRelation().selectExpr("element_at(map(1, 'a', 2, 'b'), 2)"), + Seq(Row("b")) + ) + + checkAnswer( + OneRowRelation().selectExpr("element_at(map(1, 'a', 2, 'b'), 2L)"), + Seq(Row("b")) + ) + + checkAnswer( + OneRowRelation().selectExpr("element_at(map(1, 'a', 2, 'b'), 1.0D)"), + Seq(Row("a")) + ) + + checkAnswer( + OneRowRelation().selectExpr("element_at(map(1, 'a', 2, 'b'), 1.23D)"), + Seq(Row(null)) + ) + + val e3 = intercept[AnalysisException] { + OneRowRelation().selectExpr("element_at(map(1, 'a', 2, 'b'), '1')") + } + val errorMsg3 = + s""" + |Input to function element_at should have been map followed by a value of same + |key type, but it's [map, string]. + """.stripMargin.replace("\n", " ").trim() + assert(e3.message.contains(errorMsg3)) } test("array_union functions") { From 42f25f309e91c8cde1814e3720099ac1e64783da Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 27 Sep 2018 14:30:59 +0000 Subject: [PATCH 118/879] Preparing Spark release v2.4.0-rc2 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 714b6f18b19d..f52d785e05cd 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.0 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index ee0de73f21d5..63ab510eb683 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b89e0fed2a66..b10e11849a74 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 3105a8182afc..74c6d232b36a 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f85248b53080..fbdc979d120d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 3350990c0461..53d1b2b58f1f 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 9a2cb90a2f30..98145481adc7 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 9b927a9ba525..f0c8ebe25e63 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 718875f88e74..c635cd3b9394 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 583a2fff7638..ed3efffa180b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 72473771c2e9..7f9a91e2fba0 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1-SNAPSHOT -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.0 +SPARK_VERSION_SHORT: 2.4.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 737ebfd7900c..c7b8354b0968 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 7c6b78ac2b31..a91c13362caa 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 6343a15c43a8..db239f23e1de 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 2f4886832b75..cf19fed1bef5 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9404e2a52d90..da5dba82f3cf 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 73dbc8d99449..e591ce802256 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 912d82b764e5..e8b13cdc2bef 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 8454f5c108dc..c3d3b888656b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 05f7cf870af8..468ace0ff8d7 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 0e935ac112af..db8fda66c3cd 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 7977389718cb..aa1e1267f57b 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7193df69900d..7d07e18f04a2 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 4b318b00ff5b..f545f212b2f0 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6fbb1b7db6aa..6b388ffcf18b 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 9a42d9a69967..fc369c73edc0 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 702f60ab9f00..f4cb94147d9e 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 62322da714bd..22f60efae9af 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index c8e45b27c78f..b17e67d78b98 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 6201e0dd3c9a..bf0d406da9ed 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 47b8772a8676..dc434b7bb1c5 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 87282141fa64..53e3424394ba 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1.dev0" +__version__ = "2.4.0" diff --git a/repl/pom.xml b/repl/pom.xml index d09548348eb5..4532f0b8686e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index a9d9c8de9959..47fe3f2765b2 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 793167103179..f59895e9a1bd 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index e6457289b859..0eacbbfac2b9 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index e70711ce7257..2afdc15be446 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 65cf33f32972..ef1d012d2f80 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c327b85ee866..f4be074642ae 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d9d742444e7e..e5c1065451f7 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c428be67aa3d..b522bfeac130 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 37e96b807653..8e71adabb3ec 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index dd00c2f6cd2c..c00e1b851ba0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml From 659ecb54ae0ec4c9a0e9513b295fb67d2146ba9c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 27 Sep 2018 14:31:03 +0000 Subject: [PATCH 119/879] Preparing development version 2.4.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index f52d785e05cd..714b6f18b19d 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.0 +Version: 2.4.1 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 63ab510eb683..ee0de73f21d5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b10e11849a74..b89e0fed2a66 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 74c6d232b36a..3105a8182afc 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index fbdc979d120d..f85248b53080 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 53d1b2b58f1f..3350990c0461 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 98145481adc7..9a2cb90a2f30 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index f0c8ebe25e63..9b927a9ba525 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index c635cd3b9394..718875f88e74 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ed3efffa180b..583a2fff7638 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7f9a91e2fba0..72473771c2e9 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.0 -SPARK_VERSION_SHORT: 2.4.0 +SPARK_VERSION: 2.4.1-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index c7b8354b0968..737ebfd7900c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index a91c13362caa..7c6b78ac2b31 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index db239f23e1de..6343a15c43a8 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index cf19fed1bef5..2f4886832b75 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index da5dba82f3cf..9404e2a52d90 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index e591ce802256..73dbc8d99449 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index e8b13cdc2bef..912d82b764e5 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index c3d3b888656b..8454f5c108dc 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 468ace0ff8d7..05f7cf870af8 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index db8fda66c3cd..0e935ac112af 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index aa1e1267f57b..7977389718cb 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7d07e18f04a2..7193df69900d 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f545f212b2f0..4b318b00ff5b 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6b388ffcf18b..6fbb1b7db6aa 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index fc369c73edc0..9a42d9a69967 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index f4cb94147d9e..702f60ab9f00 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 22f60efae9af..62322da714bd 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index b17e67d78b98..c8e45b27c78f 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bf0d406da9ed..6201e0dd3c9a 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index dc434b7bb1c5..47b8772a8676 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 53e3424394ba..87282141fa64 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.0" +__version__ = "2.4.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 4532f0b8686e..d09548348eb5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 47fe3f2765b2..a9d9c8de9959 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index f59895e9a1bd..793167103179 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 0eacbbfac2b9..e6457289b859 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 2afdc15be446..e70711ce7257 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ef1d012d2f80..65cf33f32972 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f4be074642ae..c327b85ee866 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index e5c1065451f7..d9d742444e7e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b522bfeac130..c428be67aa3d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 8e71adabb3ec..37e96b807653 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c00e1b851ba0..dd00c2f6cd2c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml From 0256f8a0973c2fc8815fa710670dbe68317335b5 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 27 Sep 2018 09:26:50 -0700 Subject: [PATCH 120/879] [SPARK-25546][CORE] Don't cache value of EVENT_LOG_CALLSITE_LONG_FORM. Caching the value of that config means different instances of SparkEnv will always use whatever was the first value to be read. It also breaks tests that use RDDInfo outside of the scope of a SparkContext. Since this is not a performance sensitive area, there's no advantage in caching the config value. Closes #22558 from vanzin/SPARK-25546. Authored-by: Marcelo Vanzin Signed-off-by: Dongjoon Hyun (cherry picked from commit 5fd22d05363dd8c0e1b10f3822ccb71eb42f6db9) Signed-off-by: Dongjoon Hyun --- core/src/main/scala/org/apache/spark/storage/RDDInfo.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala index 19f86569c1e3..917cfab1c699 100644 --- a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -55,11 +55,13 @@ class RDDInfo( } private[spark] object RDDInfo { - private val callsiteLongForm = SparkEnv.get.conf.get(EVENT_LOG_CALLSITE_LONG_FORM) - def fromRdd(rdd: RDD[_]): RDDInfo = { val rddName = Option(rdd.name).getOrElse(Utils.getFormattedClassName(rdd)) val parentIds = rdd.dependencies.map(_.rdd.id) + val callsiteLongForm = Option(SparkEnv.get) + .map(_.conf.get(EVENT_LOG_CALLSITE_LONG_FORM)) + .getOrElse(false) + val callSite = if (callsiteLongForm) { rdd.creationSite.longForm } else { From a43a082e0a9fe123e5705f4b0f73483c10c1ad9e Mon Sep 17 00:00:00 2001 From: Shahid Date: Wed, 26 Sep 2018 10:47:49 -0700 Subject: [PATCH 121/879] [SPARK-25533][CORE][WEBUI] AppSummary should hold the information about succeeded Jobs and completed stages only Currently, In the spark UI, when there are failed jobs or failed stages, display message for the completed jobs and completed stages are not consistent with the previous versions of spark. Reason is because, AppSummary holds the information about all the jobs and stages. But, In the below code, it checks against the completedJobs and completedStages. So, AppSummary should hold only successful jobs and stages. https://github.com/apache/spark/blob/66d29870c09e6050dd846336e596faaa8b0d14ad/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala#L306 https://github.com/apache/spark/blob/66d29870c09e6050dd846336e596faaa8b0d14ad/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala#L119 So, we should keep only completed jobs and stage information in the AppSummary, to make it consistent with Spark2.2 Test steps: bin/spark-shell ``` sc.parallelize(1 to 5, 5).collect() sc.parallelize(1 to 5, 2).map{ x => throw new RuntimeException("Fail")}.collect() ``` **Before fix:** ![screenshot from 2018-09-26 03-24-53](https://user-images.githubusercontent.com/23054875/46045669-f60bcd80-c13b-11e8-9aa6-a2e5a2038dba.png) ![screenshot from 2018-09-26 03-25-08](https://user-images.githubusercontent.com/23054875/46045699-0ae86100-c13c-11e8-94e5-ad35944c7615.png) **After fix:** ![screenshot from 2018-09-26 03-16-14](https://user-images.githubusercontent.com/23054875/46045636-d83e6880-c13b-11e8-98df-f49d15c18958.png) ![screenshot from 2018-09-26 03-16-28](https://user-images.githubusercontent.com/23054875/46045645-e1c7d080-c13b-11e8-8c9c-d32e1f663356.png) Closes #22549 from shahidki31/SPARK-25533. Authored-by: Shahid Signed-off-by: Marcelo Vanzin (cherry picked from commit 5ee21661834e837d414bc20591982a092c0aece3) Signed-off-by: Marcelo Vanzin --- .../apache/spark/status/AppStatusListener.scala | 14 ++++++++------ 1 file changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index b9f604b6b80d..513c929e703f 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -374,10 +374,11 @@ private[spark] class AppStatusListener( job.completionTime = if (event.time > 0) Some(new Date(event.time)) else None update(job, now, last = true) + if (job.status == JobExecutionStatus.SUCCEEDED) { + appSummary = new AppSummary(appSummary.numCompletedJobs + 1, appSummary.numCompletedStages) + kvstore.write(appSummary) + } } - - appSummary = new AppSummary(appSummary.numCompletedJobs + 1, appSummary.numCompletedStages) - kvstore.write(appSummary) } override def onStageSubmitted(event: SparkListenerStageSubmitted): Unit = { @@ -639,10 +640,11 @@ private[spark] class AppStatusListener( if (removeStage) { liveStages.remove((event.stageInfo.stageId, event.stageInfo.attemptNumber)) } + if (stage.status == v1.StageStatus.COMPLETE) { + appSummary = new AppSummary(appSummary.numCompletedJobs, appSummary.numCompletedStages + 1) + kvstore.write(appSummary) + } } - - appSummary = new AppSummary(appSummary.numCompletedJobs, appSummary.numCompletedStages + 1) - kvstore.write(appSummary) } private def removeBlackListedStageFrom(exec: LiveExecutor, stageId: Int, now: Long) = { From b2a1e2f8d47876658debe95bffd9a848ccaa5819 Mon Sep 17 00:00:00 2001 From: maryannxue Date: Fri, 28 Sep 2018 00:09:06 -0700 Subject: [PATCH 122/879] [SPARK-25505][SQL] The output order of grouping columns in Pivot is different from the input order ## What changes were proposed in this pull request? The grouping columns from a Pivot query are inferred as "input columns - pivot columns - pivot aggregate columns", where input columns are the output of the child relation of Pivot. The grouping columns will be the leading columns in the pivot output and they should preserve the same order as specified by the input. For example, ``` SELECT * FROM ( SELECT course, earnings, "a" as a, "z" as z, "b" as b, "y" as y, "c" as c, "x" as x, "d" as d, "w" as w FROM courseSales ) PIVOT ( sum(earnings) FOR course IN ('dotNET', 'Java') ) ``` The output columns should be "a, z, b, y, c, x, d, w, ..." but now it is "a, b, c, d, w, x, y, z, ..." The fix is to use the child plan's `output` instead of `outputSet` so that the order can be preserved. ## How was this patch tested? Added UT. Closes #22519 from maryannxue/spark-25505. Authored-by: maryannxue Signed-off-by: gatorsmile (cherry picked from commit e120a38c0cdfb569c9151bef4d53e98175da2b25) Signed-off-by: gatorsmile --- .../spark/sql/catalyst/analysis/Analyzer.scala | 7 +++++-- .../test/resources/sql-tests/inputs/pivot.sql | 10 ++++++++++ .../resources/sql-tests/results/pivot.sql.out | 17 ++++++++++++++++- 3 files changed, 31 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index e3b17121bf35..d303b435477f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -554,8 +554,11 @@ class Analyzer( Cast(value, pivotColumn.dataType, Some(conf.sessionLocalTimeZone)).eval(EmptyRow) } // Group-by expressions coming from SQL are implicit and need to be deduced. - val groupByExprs = groupByExprsOpt.getOrElse( - (child.outputSet -- aggregates.flatMap(_.references) -- pivotColumn.references).toSeq) + val groupByExprs = groupByExprsOpt.getOrElse { + val pivotColAndAggRefs = + (pivotColumn.references ++ aggregates.flatMap(_.references)).toSet + child.output.filterNot(pivotColAndAggRefs.contains) + } val singleAgg = aggregates.size == 1 def outputName(value: Expression, aggregate: Expression): String = { val stringValue = value match { diff --git a/sql/core/src/test/resources/sql-tests/inputs/pivot.sql b/sql/core/src/test/resources/sql-tests/inputs/pivot.sql index 1f607b334dc1..81547ab46ce0 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pivot.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pivot.sql @@ -287,3 +287,13 @@ PIVOT ( sum(earnings) FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) ); + +-- grouping columns output in the same order as input +SELECT * FROM ( + SELECT course, earnings, "a" as a, "z" as z, "b" as b, "y" as y, "c" as c, "x" as x, "d" as d, "w" as w + FROM courseSales +) +PIVOT ( + sum(earnings) + FOR course IN ('dotNET', 'Java') +); diff --git a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out index 2dd92930f92a..487883a7f384 100644 --- a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 31 +-- Number of queries: 32 -- !query 0 @@ -476,3 +476,18 @@ struct<> -- !query 30 output org.apache.spark.sql.AnalysisException Invalid pivot column 'named_struct(course, course#x, m, m#x)'. Pivot columns must be comparable.; + + +-- !query 31 +SELECT * FROM ( + SELECT course, earnings, "a" as a, "z" as z, "b" as b, "y" as y, "c" as c, "x" as x, "d" as d, "w" as w + FROM courseSales +) +PIVOT ( + sum(earnings) + FOR course IN ('dotNET', 'Java') +) +-- !query 31 schema +struct +-- !query 31 output +a z b y c x d w 63000 50000 From 81391c274eb371dbf4cfed0afca47806f6fcfd00 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 28 Sep 2018 14:10:24 -0700 Subject: [PATCH 123/879] [SPARK-23285][DOC][FOLLOWUP] Fix missing markup tag ## What changes were proposed in this pull request? This adds a missing markup tag. This should go to `master/branch-2.4`. ## How was this patch tested? Manual via `SKIP_API=1 jekyll build`. Closes #22585 from dongjoon-hyun/SPARK-23285. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 0b33f08683a41f6f3a6ec02c327010c0722cc1d1) Signed-off-by: Dongjoon Hyun --- docs/running-on-kubernetes.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index c83dad6df1e7..fc7c9a5450e6 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -678,6 +678,7 @@ specific to Spark on Kubernetes. Example values include 0.1, 500m, 1.5, 5, etc., with the definition of cpu units documented in [CPU units](https://kubernetes.io/docs/tasks/configure-pod-container/assign-cpu-resource/#cpu-units). This is distinct from spark.executor.cores: it is only used and takes precedence over spark.executor.cores for specifying the executor pod cpu request if set. Task parallelism, e.g., number of tasks an executor can run concurrently is not affected by this. + spark.kubernetes.executor.limit.cores From 7614313c9443712553332962d62dfe5aacc7ed34 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 28 Sep 2018 14:29:56 -0700 Subject: [PATCH 124/879] [SPARK-25542][CORE][TEST] Move flaky test in OpenHashMapSuite to OpenHashSetSuite and make it against OpenHashSet ## What changes were proposed in this pull request? The specified test in OpenHashMapSuite to test large items is somehow flaky to throw OOM. By considering the original work #6763 that added this test, the test can be against OpenHashSetSuite. And by doing this should be to save memory because OpenHashMap allocates two more arrays when growing the map/set. ## How was this patch tested? Existing tests. Closes #22569 from viirya/SPARK-25542. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun (cherry picked from commit b7d80349b0e367d78cab238e62c2ec353f0f12b3) Signed-off-by: Dongjoon Hyun --- .../spark/util/collection/OpenHashMapSuite.scala | 10 ---------- .../spark/util/collection/OpenHashSetSuite.scala | 13 +++++++++++++ 2 files changed, 13 insertions(+), 10 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala index 151235dd0fb9..68bcc5e5a509 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashMapSuite.scala @@ -185,16 +185,6 @@ class OpenHashMapSuite extends SparkFunSuite with Matchers { assert(map.contains(null)) } - test("support for more than 12M items") { - val cnt = 12000000 // 12M - val map = new OpenHashMap[Int, Int](cnt) - for (i <- 0 until cnt) { - map(i) = 1 - } - val numInvalidValues = map.iterator.count(_._2 == 0) - assertResult(0)(numInvalidValues) - } - test("distinguish between the 0/0.0/0L and null") { val specializedMap1 = new OpenHashMap[String, Long] specializedMap1("a") = null.asInstanceOf[Long] diff --git a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala index b887f937a9da..44d2118d7794 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/OpenHashSetSuite.scala @@ -255,4 +255,17 @@ class OpenHashSetSuite extends SparkFunSuite with Matchers { val set = new OpenHashSet[Long](0) assert(set.size === 0) } + + test("support for more than 12M items") { + val cnt = 12000000 // 12M + val set = new OpenHashSet[Int](cnt) + for (i <- 0 until cnt) { + set.add(i) + assert(set.contains(i)) + + val pos1 = set.getPos(i) + val pos2 = set.addWithoutResize(i) & OpenHashSet.POSITION_MASK + assert(pos1 == pos2) + } + } } From ec2c17abf43d304fab26dde3ae624f553cdbd32e Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 29 Sep 2018 11:43:58 +0800 Subject: [PATCH 125/879] [SPARK-25570][SQL][TEST] Replace 2.3.1 with 2.3.2 in HiveExternalCatalogVersionsSuite ## What changes were proposed in this pull request? This PR aims to prevent test slowdowns at `HiveExternalCatalogVersionsSuite` by using the latest Apache Spark 2.3.2 link because the Apache mirrors will remove the old Spark 2.3.1 binaries eventually. `HiveExternalCatalogVersionsSuite` will not fail because [SPARK-24813](https://issues.apache.org/jira/browse/SPARK-24813) implements a fallback logic. However, it will cause many trials and fallbacks in all builds over `branch-2.3/branch-2.4/master`. We had better fix this issue. ## How was this patch tested? Pass the Jenkins with the updated version. Closes #22587 from dongjoon-hyun/SPARK-25570. Authored-by: Dongjoon Hyun Signed-off-by: hyukjinkwon (cherry picked from commit 1e437835e96c4417117f44c29eba5ebc0112926f) Signed-off-by: hyukjinkwon --- .../spark/sql/hive/HiveExternalCatalogVersionsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index 25df3339e62f..46b66c14d0fb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -203,7 +203,7 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { object PROCESS_TABLES extends QueryTest with SQLTestUtils { // Tests the latest version of every release line. - val testingVersions = Seq("2.1.3", "2.2.2", "2.3.1") + val testingVersions = Seq("2.1.3", "2.2.2", "2.3.2") protected var spark: SparkSession = _ From a14306b1d5a135cff0441c1c953032d0c6a51c47 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 29 Sep 2018 18:18:37 +0800 Subject: [PATCH 126/879] [SPARK-25262][DOC][FOLLOWUP] Fix link tags in html table ## What changes were proposed in this pull request? Markdown links are not working inside html table. We should use html link tag. ## How was this patch tested? Verified in IntelliJ IDEA's markdown editor and online markdown editor. Closes #22588 from viirya/SPARK-25262-followup. Authored-by: Liang-Chi Hsieh Signed-off-by: hyukjinkwon (cherry picked from commit dcb9a97f3e16d4645529ac619c3197fcba1c9806) Signed-off-by: hyukjinkwon --- docs/running-on-kubernetes.md | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index fc7c9a5450e6..f19aa412389f 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -667,15 +667,15 @@ specific to Spark on Kubernetes. spark.kubernetes.driver.limit.cores (none) - Specify a hard cpu [limit](https://kubernetes.io/docs/concepts/configuration/manage-compute-resources-container/#resource-requests-and-limits-of-pod-and-container) for the driver pod. + Specify a hard cpu limit for the driver pod. spark.kubernetes.executor.request.cores (none) - Specify the cpu request for each executor pod. Values conform to the Kubernetes [convention](https://kubernetes.io/docs/concepts/configuration/manage-compute-resources-container/#meaning-of-cpu). - Example values include 0.1, 500m, 1.5, 5, etc., with the definition of cpu units documented in [CPU units](https://kubernetes.io/docs/tasks/configure-pod-container/assign-cpu-resource/#cpu-units). + Specify the cpu request for each executor pod. Values conform to the Kubernetes convention. + Example values include 0.1, 500m, 1.5, 5, etc., with the definition of cpu units documented in CPU units. This is distinct from spark.executor.cores: it is only used and takes precedence over spark.executor.cores for specifying the executor pod cpu request if set. Task parallelism, e.g., number of tasks an executor can run concurrently is not affected by this. @@ -684,7 +684,7 @@ specific to Spark on Kubernetes. spark.kubernetes.executor.limit.cores (none) - Specify a hard cpu [limit](https://kubernetes.io/docs/concepts/configuration/manage-compute-resources-container/#resource-requests-and-limits-of-pod-and-container) for each executor pod launched for the Spark Application. + Specify a hard cpu limit for each executor pod launched for the Spark Application. From fef30271bc4a77e7f94b240cd8d7fef71f47935e Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Sat, 29 Sep 2018 14:48:32 -0700 Subject: [PATCH 127/879] [SPARK-25572][SPARKR] test only if not cran ## What changes were proposed in this pull request? CRAN doesn't seem to respect the system requirements as running tests - we have seen cases where SparkR is run on Java 10, which unfortunately Spark does not start on. For 2.4, lets attempt skipping all tests ## How was this patch tested? manual, jenkins, appveyor Author: Felix Cheung Closes #22589 from felixcheung/ralltests. (cherry picked from commit f4b138082ff91be74b0f5bbe19cdb90dd9e5f131) Signed-off-by: Felix Cheung --- R/pkg/tests/run-all.R | 83 +++++++++++++++++++++++-------------------- 1 file changed, 44 insertions(+), 39 deletions(-) diff --git a/R/pkg/tests/run-all.R b/R/pkg/tests/run-all.R index 94d75188fb94..1e9641855888 100644 --- a/R/pkg/tests/run-all.R +++ b/R/pkg/tests/run-all.R @@ -18,50 +18,55 @@ library(testthat) library(SparkR) -# Turn all warnings into errors -options("warn" = 2) +# SPARK-25572 +if (identical(Sys.getenv("NOT_CRAN"), "true")) { -if (.Platform$OS.type == "windows") { - Sys.setenv(TZ = "GMT") -} + # Turn all warnings into errors + options("warn" = 2) -# Setup global test environment -# Install Spark first to set SPARK_HOME + if (.Platform$OS.type == "windows") { + Sys.setenv(TZ = "GMT") + } -# NOTE(shivaram): We set overwrite to handle any old tar.gz files or directories left behind on -# CRAN machines. For Jenkins we should already have SPARK_HOME set. -install.spark(overwrite = TRUE) + # Setup global test environment + # Install Spark first to set SPARK_HOME -sparkRDir <- file.path(Sys.getenv("SPARK_HOME"), "R") -sparkRWhitelistSQLDirs <- c("spark-warehouse", "metastore_db") -invisible(lapply(sparkRWhitelistSQLDirs, - function(x) { unlink(file.path(sparkRDir, x), recursive = TRUE, force = TRUE)})) -sparkRFilesBefore <- list.files(path = sparkRDir, all.files = TRUE) + # NOTE(shivaram): We set overwrite to handle any old tar.gz files or directories left behind on + # CRAN machines. For Jenkins we should already have SPARK_HOME set. + install.spark(overwrite = TRUE) -sparkRTestMaster <- "local[1]" -sparkRTestConfig <- list() -if (identical(Sys.getenv("NOT_CRAN"), "true")) { - sparkRTestMaster <- "" -} else { - # Disable hsperfdata on CRAN - old_java_opt <- Sys.getenv("_JAVA_OPTIONS") - Sys.setenv("_JAVA_OPTIONS" = paste("-XX:-UsePerfData", old_java_opt)) - tmpDir <- tempdir() - tmpArg <- paste0("-Djava.io.tmpdir=", tmpDir) - sparkRTestConfig <- list(spark.driver.extraJavaOptions = tmpArg, - spark.executor.extraJavaOptions = tmpArg) -} + sparkRDir <- file.path(Sys.getenv("SPARK_HOME"), "R") + sparkRWhitelistSQLDirs <- c("spark-warehouse", "metastore_db") + invisible(lapply(sparkRWhitelistSQLDirs, + function(x) { unlink(file.path(sparkRDir, x), recursive = TRUE, force = TRUE)})) + sparkRFilesBefore <- list.files(path = sparkRDir, all.files = TRUE) -test_package("SparkR") + sparkRTestMaster <- "local[1]" + sparkRTestConfig <- list() + if (identical(Sys.getenv("NOT_CRAN"), "true")) { + sparkRTestMaster <- "" + } else { + # Disable hsperfdata on CRAN + old_java_opt <- Sys.getenv("_JAVA_OPTIONS") + Sys.setenv("_JAVA_OPTIONS" = paste("-XX:-UsePerfData", old_java_opt)) + tmpDir <- tempdir() + tmpArg <- paste0("-Djava.io.tmpdir=", tmpDir) + sparkRTestConfig <- list(spark.driver.extraJavaOptions = tmpArg, + spark.executor.extraJavaOptions = tmpArg) + } -if (identical(Sys.getenv("NOT_CRAN"), "true")) { - # set random seed for predictable results. mostly for base's sample() in tree and classification - set.seed(42) - # for testthat 1.0.2 later, change reporter from "summary" to default_reporter() - testthat:::run_tests("SparkR", - file.path(sparkRDir, "pkg", "tests", "fulltests"), - NULL, - "summary") -} + test_package("SparkR") + + if (identical(Sys.getenv("NOT_CRAN"), "true")) { + # set random seed for predictable results. mostly for base's sample() in tree and classification + set.seed(42) + # for testthat 1.0.2 later, change reporter from "summary" to default_reporter() + testthat:::run_tests("SparkR", + file.path(sparkRDir, "pkg", "tests", "fulltests"), + NULL, + "summary") + } -SparkR:::uninstallDownloadedSpark() + SparkR:::uninstallDownloadedSpark() + +} From 6f510c69b4a78d77b3fc58813bdbdf3f2e8566bc Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Sat, 29 Sep 2018 18:10:04 -0700 Subject: [PATCH 128/879] [SPARK-25568][CORE] Continue to update the remaining accumulators when failing to update one accumulator ## What changes were proposed in this pull request? Since we don't fail a job when `AccumulatorV2.merge` fails, we should try to update the remaining accumulators so that they can still report correct values. ## How was this patch tested? The new unit test. Closes #22586 from zsxwing/SPARK-25568. Authored-by: Shixiong Zhu Signed-off-by: gatorsmile (cherry picked from commit b6b8a6632e2b6e5482aaf4bfa093700752a9df80) Signed-off-by: gatorsmile --- .../apache/spark/scheduler/DAGScheduler.scala | 20 +++++++++++++------ .../spark/scheduler/DAGSchedulerSuite.scala | 20 +++++++++++++++++++ docs/rdd-programming-guide.md | 4 ++++ 3 files changed, 38 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 50c91da8b13d..036fa38ce240 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1242,9 +1242,10 @@ private[spark] class DAGScheduler( private def updateAccumulators(event: CompletionEvent): Unit = { val task = event.task val stage = stageIdToStage(task.stageId) - try { - event.accumUpdates.foreach { updates => - val id = updates.id + + event.accumUpdates.foreach { updates => + val id = updates.id + try { // Find the corresponding accumulator on the driver and update it val acc: AccumulatorV2[Any, Any] = AccumulatorContext.get(id) match { case Some(accum) => accum.asInstanceOf[AccumulatorV2[Any, Any]] @@ -1258,10 +1259,17 @@ private[spark] class DAGScheduler( event.taskInfo.setAccumulables( acc.toInfo(Some(updates.value), Some(acc.value)) +: event.taskInfo.accumulables) } + } catch { + case NonFatal(e) => + // Log the class name to make it easy to find the bad implementation + val accumClassName = AccumulatorContext.get(id) match { + case Some(accum) => accum.getClass.getName + case None => "Unknown class" + } + logError( + s"Failed to update accumulator $id ($accumClassName) for task ${task.partitionId}", + e) } - } catch { - case NonFatal(e) => - logError(s"Failed to update accumulators for task ${task.partitionId}", e) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index c820f4899570..e4bf0abce947 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -1877,6 +1877,26 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(sc.parallelize(1 to 10, 2).count() === 10) } + test("misbehaved accumulator should not impact other accumulators") { + val bad = new LongAccumulator { + override def merge(other: AccumulatorV2[java.lang.Long, java.lang.Long]): Unit = { + throw new DAGSchedulerSuiteDummyException + } + } + sc.register(bad, "bad") + val good = sc.longAccumulator("good") + + sc.parallelize(1 to 10, 2).foreach { item => + bad.add(1) + good.add(1) + } + + // This is to ensure the `bad` accumulator did fail to update its value + assert(bad.value == 0L) + // Should be able to update the "good" accumulator + assert(good.value == 10L) + } + /** * The job will be failed on first task throwing a DAGSchedulerSuiteDummyException. * Any subsequent task WILL throw a legitimate java.lang.UnsupportedOperationException. diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md index 005425754c64..9a07d6ca24b6 100644 --- a/docs/rdd-programming-guide.md +++ b/docs/rdd-programming-guide.md @@ -1465,6 +1465,10 @@ jsc.sc().register(myVectorAcc, "MyVectorAcc1"); Note that, when programmers define their own type of AccumulatorV2, the resulting type can be different than that of the elements added. +*Warning*: When a Spark task finishes, Spark will try to merge the accumulated updates in this task to an accumulator. +If it fails, Spark will ignore the failure and still mark the task successful and continue to run other tasks. Hence, +a buggy accumulator will not impact a Spark job, but it may not get updated correctly although a Spark job is successful. +
    From 8e6fb473b01591da44c7e72bf60d19cbba38fa48 Mon Sep 17 00:00:00 2001 From: Darcy Shen Date: Sun, 30 Sep 2018 09:00:23 -0500 Subject: [PATCH 129/879] [CORE][MINOR] Fix obvious error and compiling for Scala 2.12.7 ## What changes were proposed in this pull request? Fix an obvious error. ## How was this patch tested? Existing tests. Closes #22577 from sadhen/minor_fix. Authored-by: Darcy Shen Signed-off-by: Sean Owen (cherry picked from commit 40e6ed89405828ff312eca0abd43cfba4b9185b2) Signed-off-by: Sean Owen --- .../org/apache/spark/status/api/v1/OneApplicationResource.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala index 32100c570453..1f4082cac8f7 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala @@ -175,7 +175,7 @@ private[v1] class OneApplicationAttemptResource extends AbstractApplicationResou def getAttempt(): ApplicationAttemptInfo = { uiRoot.getApplicationInfo(appId) .flatMap { app => - app.attempts.filter(_.attemptId == attemptId).headOption + app.attempts.find(_.attemptId.contains(attemptId)) } .getOrElse { throw new NotFoundException(s"unknown app $appId, attempt $attemptId") From c886f050b51862bd8cdb41bc13f4dfedebdbcd31 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 30 Sep 2018 14:28:20 -0700 Subject: [PATCH 130/879] [SPARK-25543][K8S] Print debug message iff execIdsRemovedInThisRound is not empty. ## What changes were proposed in this pull request? Spurious logs like /sec. 2018-09-26 09:33:57 DEBUG ExecutorPodsLifecycleManager:58 - Removed executors with ids from Spark that were either found to be deleted or non-existent in the cluster. 2018-09-26 09:33:58 DEBUG ExecutorPodsLifecycleManager:58 - Removed executors with ids from Spark that were either found to be deleted or non-existent in the cluster. 2018-09-26 09:33:59 DEBUG ExecutorPodsLifecycleManager:58 - Removed executors with ids from Spark that were either found to be deleted or non-existent in the cluster. 2018-09-26 09:34:00 DEBUG ExecutorPodsLifecycleManager:58 - Removed executors with ids from Spark that were either found to be deleted or non-existent in the cluster. The fix is easy, first check if there are any removed executors, before producing the log message. ## How was this patch tested? Tested by manually deploying to a minikube cluster. Closes #22565 from ScrapCodes/spark-25543/k8s/debug-log-spurious-warning. Authored-by: Prashant Sharma Signed-off-by: Dongjoon Hyun (cherry picked from commit 4da541a5d23b039eb549dd849cf121bdc8676e59) Signed-off-by: Dongjoon Hyun --- .../cluster/k8s/ExecutorPodsLifecycleManager.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala index b28d93990313..14814635732d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala @@ -99,8 +99,11 @@ private[spark] class ExecutorPodsLifecycleManager( } } } - logDebug(s"Removed executors with ids ${execIdsRemovedInThisRound.mkString(",")}" + - s" from Spark that were either found to be deleted or non-existent in the cluster.") + + if (execIdsRemovedInThisRound.nonEmpty) { + logDebug(s"Removed executors with ids ${execIdsRemovedInThisRound.mkString(",")}" + + s" from Spark that were either found to be deleted or non-existent in the cluster.") + } } private def onFinalNonDeletedState( From 7b1094b54c3810b4c0b02ba14d282f44be0813c3 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Sun, 30 Sep 2018 22:08:04 -0700 Subject: [PATCH 131/879] [SPARK-25505][SQL][FOLLOWUP] Fix for attributes cosmetically different in Pivot clause ## What changes were proposed in this pull request? #22519 introduced a bug when the attributes in the pivot clause are cosmetically different from the output ones (eg. different case). In particular, the problem is that the PR used a `Set[Attribute]` instead of an `AttributeSet`. ## How was this patch tested? added UT Closes #22582 from mgaido91/SPARK-25505_followup. Authored-by: Marco Gaido Signed-off-by: gatorsmile (cherry picked from commit fb8f4c05657595e089b6812d97dbfee246fce06f) Signed-off-by: gatorsmile --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 3 +-- sql/core/src/test/resources/sql-tests/inputs/pivot.sql | 5 +++-- sql/core/src/test/resources/sql-tests/results/pivot.sql.out | 4 ++-- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d303b435477f..fdb68dd38f21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -555,8 +555,7 @@ class Analyzer( } // Group-by expressions coming from SQL are implicit and need to be deduced. val groupByExprs = groupByExprsOpt.getOrElse { - val pivotColAndAggRefs = - (pivotColumn.references ++ aggregates.flatMap(_.references)).toSet + val pivotColAndAggRefs = pivotColumn.references ++ AttributeSet(aggregates) child.output.filterNot(pivotColAndAggRefs.contains) } val singleAgg = aggregates.size == 1 diff --git a/sql/core/src/test/resources/sql-tests/inputs/pivot.sql b/sql/core/src/test/resources/sql-tests/inputs/pivot.sql index 81547ab46ce0..c2ecd97e2b02 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/pivot.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/pivot.sql @@ -289,11 +289,12 @@ PIVOT ( ); -- grouping columns output in the same order as input +-- correctly handle pivot columns with different cases SELECT * FROM ( SELECT course, earnings, "a" as a, "z" as z, "b" as b, "y" as y, "c" as c, "x" as x, "d" as d, "w" as w FROM courseSales ) PIVOT ( - sum(earnings) - FOR course IN ('dotNET', 'Java') + sum(Earnings) + FOR Course IN ('dotNET', 'Java') ); diff --git a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out index 487883a7f384..595ce1f8efcd 100644 --- a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out @@ -484,8 +484,8 @@ SELECT * FROM ( FROM courseSales ) PIVOT ( - sum(earnings) - FOR course IN ('dotNET', 'Java') + sum(Earnings) + FOR Course IN ('dotNET', 'Java') ) -- !query 31 schema struct From 82990e5efba3693cdaf02f325ca677cb5f7425fc Mon Sep 17 00:00:00 2001 From: seancxmao Date: Sun, 30 Sep 2018 22:49:14 -0700 Subject: [PATCH 132/879] [SPARK-25453][SQL][TEST][.FFFFFFFFF] OracleIntegrationSuite IllegalArgumentException: Timestamp format must be yyyy-mm-dd hh:mm:ss ## What changes were proposed in this pull request? This PR aims to fix the failed test of `OracleIntegrationSuite`. ## How was this patch tested? Existing integration tests. Closes #22461 from seancxmao/SPARK-25453. Authored-by: seancxmao Signed-off-by: gatorsmile (cherry picked from commit 21f0b73dbcd94f9eea8cbc06a024b0e899edaf4c) Signed-off-by: gatorsmile --- docs/sql-programming-guide.md | 2 +- .../apache/spark/sql/jdbc/OracleIntegrationSuite.scala | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 2546064ae282..d525405e548b 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1500,7 +1500,7 @@ See the [Apache Avro Data Source Guide](avro-data-source-guide.html). * The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java's DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs. * Some databases, such as H2, convert all names to upper case. You'll need to use upper case to refer to those names in Spark SQL. - + * Users can specify vendor-specific JDBC connection properties in the data source options to do special treatment. For example, `spark.read.format("jdbc").option("url", oracleJdbcUrl).option("oracle.jdbc.mapDateToTimestamp", "false")`. `oracle.jdbc.mapDateToTimestamp` defaults to true, users often need to disable this flag to avoid Oracle date being resolved as timestamp. # Performance Tuning diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala index 09a2cd83aed6..70d294d0ca65 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/OracleIntegrationSuite.scala @@ -442,6 +442,12 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo .option("lowerBound", "2018-07-06") .option("upperBound", "2018-07-20") .option("numPartitions", 3) + // oracle.jdbc.mapDateToTimestamp defaults to true. If this flag is not disabled, column d + // (Oracle DATE) will be resolved as Catalyst Timestamp, which will fail bound evaluation of + // the partition column. E.g. 2018-07-06 cannot be evaluated as Timestamp, and the error + // message says: Timestamp format must be yyyy-mm-dd hh:mm:ss[.fffffffff]. + .option("oracle.jdbc.mapDateToTimestamp", "false") + .option("sessionInitStatement", "ALTER SESSION SET NLS_DATE_FORMAT = 'YYYY-MM-DD'") .load() df1.logicalPlan match { @@ -462,6 +468,9 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo .option("lowerBound", "2018-07-04 03:30:00.0") .option("upperBound", "2018-07-27 14:11:05.0") .option("numPartitions", 2) + .option("oracle.jdbc.mapDateToTimestamp", "false") + .option("sessionInitStatement", + "ALTER SESSION SET NLS_TIMESTAMP_FORMAT = 'YYYY-MM-DD HH24:MI:SS.FF'") .load() df2.logicalPlan match { From 426c2bd35937add1a26e77d2f2879f0e3f0c2f45 Mon Sep 17 00:00:00 2001 From: Aleksandr Koriagin Date: Mon, 1 Oct 2018 17:18:45 +0800 Subject: [PATCH 133/879] [SPARK-23401][PYTHON][TESTS] Add more data types for PandasUDFTests ## What changes were proposed in this pull request? Add more data types for Pandas UDF Tests for PySpark SQL ## How was this patch tested? manual tests Closes #22568 from AlexanderKoryagin/new_types_for_pandas_udf_tests. Lead-authored-by: Aleksandr Koriagin Co-authored-by: hyukjinkwon Co-authored-by: Alexander Koryagin Signed-off-by: hyukjinkwon (cherry picked from commit 30f5d0f2ddfe56266ea81e4255f9b4f373dab237) Signed-off-by: hyukjinkwon --- python/pyspark/sql/tests.py | 107 ++++++++++++++++++++++++++---------- 1 file changed, 79 insertions(+), 28 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index dece1dabea2c..690035a0e7df 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -5478,32 +5478,81 @@ def data(self): .withColumn("v", explode(col('vs'))).drop('vs') def test_supported_types(self): - from pyspark.sql.functions import pandas_udf, PandasUDFType, array, col - df = self.data.withColumn("arr", array(col("id"))) + from decimal import Decimal + from distutils.version import LooseVersion + import pyarrow as pa + from pyspark.sql.functions import pandas_udf, PandasUDFType - # Different forms of group map pandas UDF, results of these are the same + values = [ + 1, 2, 3, + 4, 5, 1.1, + 2.2, Decimal(1.123), + [1, 2, 2], True, 'hello' + ] + output_fields = [ + ('id', IntegerType()), ('byte', ByteType()), ('short', ShortType()), + ('int', IntegerType()), ('long', LongType()), ('float', FloatType()), + ('double', DoubleType()), ('decim', DecimalType(10, 3)), + ('array', ArrayType(IntegerType())), ('bool', BooleanType()), ('str', StringType()) + ] - output_schema = StructType( - [StructField('id', LongType()), - StructField('v', IntegerType()), - StructField('arr', ArrayType(LongType())), - StructField('v1', DoubleType()), - StructField('v2', LongType())]) + # TODO: Add BinaryType to variables above once minimum pyarrow version is 0.10.0 + if LooseVersion(pa.__version__) >= LooseVersion("0.10.0"): + values.append(bytearray([0x01, 0x02])) + output_fields.append(('bin', BinaryType())) + output_schema = StructType([StructField(*x) for x in output_fields]) + df = self.spark.createDataFrame([values], schema=output_schema) + + # Different forms of group map pandas UDF, results of these are the same udf1 = pandas_udf( - lambda pdf: pdf.assign(v1=pdf.v * pdf.id * 1.0, v2=pdf.v + pdf.id), + lambda pdf: pdf.assign( + byte=pdf.byte * 2, + short=pdf.short * 2, + int=pdf.int * 2, + long=pdf.long * 2, + float=pdf.float * 2, + double=pdf.double * 2, + decim=pdf.decim * 2, + bool=False if pdf.bool else True, + str=pdf.str + 'there', + array=pdf.array, + ), output_schema, PandasUDFType.GROUPED_MAP ) udf2 = pandas_udf( - lambda _, pdf: pdf.assign(v1=pdf.v * pdf.id * 1.0, v2=pdf.v + pdf.id), + lambda _, pdf: pdf.assign( + byte=pdf.byte * 2, + short=pdf.short * 2, + int=pdf.int * 2, + long=pdf.long * 2, + float=pdf.float * 2, + double=pdf.double * 2, + decim=pdf.decim * 2, + bool=False if pdf.bool else True, + str=pdf.str + 'there', + array=pdf.array, + ), output_schema, PandasUDFType.GROUPED_MAP ) udf3 = pandas_udf( - lambda key, pdf: pdf.assign(id=key[0], v1=pdf.v * pdf.id * 1.0, v2=pdf.v + pdf.id), + lambda key, pdf: pdf.assign( + id=key[0], + byte=pdf.byte * 2, + short=pdf.short * 2, + int=pdf.int * 2, + long=pdf.long * 2, + float=pdf.float * 2, + double=pdf.double * 2, + decim=pdf.decim * 2, + bool=False if pdf.bool else True, + str=pdf.str + 'there', + array=pdf.array, + ), output_schema, PandasUDFType.GROUPED_MAP ) @@ -5667,24 +5716,26 @@ def test_wrong_args(self): pandas_udf(lambda x, y: x, DoubleType(), PandasUDFType.SCALAR)) def test_unsupported_types(self): + from distutils.version import LooseVersion + import pyarrow as pa from pyspark.sql.functions import pandas_udf, PandasUDFType - schema = StructType( - [StructField("id", LongType(), True), - StructField("map", MapType(StringType(), IntegerType()), True)]) - with QuietTest(self.sc): - with self.assertRaisesRegexp( - NotImplementedError, - 'Invalid returnType.*grouped map Pandas UDF.*MapType'): - pandas_udf(lambda x: x, schema, PandasUDFType.GROUPED_MAP) - schema = StructType( - [StructField("id", LongType(), True), - StructField("arr_ts", ArrayType(TimestampType()), True)]) - with QuietTest(self.sc): - with self.assertRaisesRegexp( - NotImplementedError, - 'Invalid returnType.*grouped map Pandas UDF.*ArrayType.*TimestampType'): - pandas_udf(lambda x: x, schema, PandasUDFType.GROUPED_MAP) + common_err_msg = 'Invalid returnType.*grouped map Pandas UDF.*' + unsupported_types = [ + StructField('map', MapType(StringType(), IntegerType())), + StructField('arr_ts', ArrayType(TimestampType())), + StructField('null', NullType()), + ] + + # TODO: Remove this if-statement once minimum pyarrow version is 0.10.0 + if LooseVersion(pa.__version__) < LooseVersion("0.10.0"): + unsupported_types.append(StructField('bin', BinaryType())) + + for unsupported_type in unsupported_types: + schema = StructType([StructField('id', LongType(), True), unsupported_type]) + with QuietTest(self.sc): + with self.assertRaisesRegexp(NotImplementedError, common_err_msg): + pandas_udf(lambda x: x, schema, PandasUDFType.GROUPED_MAP) # Regression test for SPARK-23314 def test_timestamp_dst(self): From ad7b3f6bacccf429c8aeda7ff3b25507aac314f7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 1 Oct 2018 21:35:12 -0500 Subject: [PATCH 134/879] [SPARK-25578][BUILD] Update to Scala 2.12.7 ## What changes were proposed in this pull request? Update to Scala 2.12.7. See https://issues.apache.org/jira/browse/SPARK-25578 for why. ## How was this patch tested? Existing tests. Closes #22600 from srowen/SPARK-25578. Authored-by: Sean Owen Signed-off-by: Sean Owen (cherry picked from commit 5114db5781967c1e8046296905d97560187479fb) Signed-off-by: Sean Owen --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 47b8772a8676..28804be0301c 100644 --- a/pom.xml +++ b/pom.xml @@ -2760,7 +2760,7 @@ scala-2.12 - 2.12.6 + 2.12.7 2.12 From ea4068a0af036f9e266f64eac558aaa984f08a93 Mon Sep 17 00:00:00 2001 From: Shahid Date: Tue, 2 Oct 2018 08:05:09 -0700 Subject: [PATCH 135/879] [SPARK-25583][DOC] Add history-server related configuration in the documentation. ## What changes were proposed in this pull request? Add history-server related configuration in the documentation. Some of the history server related configurations were missing in the documentation.Like, 'spark.history.store.maxDiskUsage', 'spark.ui.liveUpdate.period' etc. ## How was this patch tested? ![screenshot from 2018-10-01 20-58-26](https://user-images.githubusercontent.com/23054875/46298568-04833a80-c5bd-11e8-95b8-54c9d6582fd2.png) ![screenshot from 2018-10-01 20-59-31](https://user-images.githubusercontent.com/23054875/46298591-11a02980-c5bd-11e8-93d0-892afdfd4f9a.png) ![screenshot from 2018-10-01 20-59-45](https://user-images.githubusercontent.com/23054875/46298601-1533b080-c5bd-11e8-9689-e9b39882a7b5.png) Closes #22601 from shahidki31/historyConf. Authored-by: Shahid Signed-off-by: Dongjoon Hyun (cherry picked from commit 71876633f3af706408355b5fb561b58dbc593360) Signed-off-by: Dongjoon Hyun --- docs/configuration.md | 16 ++++++++++++++++ docs/monitoring.md | 25 +++++++++++++++++++++++++ 2 files changed, 41 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index 782ccff66707..55773937d4d7 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -793,6 +793,13 @@ Apart from these, the following properties are also available, and may be useful Buffer size to use when writing to output streams, in KiB unless otherwise specified. + + spark.ui.dagGraph.retainedRootRDDs + Int.MaxValue + + How many DAG graph nodes the Spark UI and status APIs remember before garbage collecting. + + spark.ui.enabled true @@ -807,6 +814,15 @@ Apart from these, the following properties are also available, and may be useful Allows jobs and stages to be killed from the web UI. + + spark.ui.liveUpdate.period + 100ms + + How often to update live entities. -1 means "never update" when replaying applications, + meaning only the last write will happen. For live applications, this avoids a few + operations that we can live without when rapidly processing incoming task events. + + spark.ui.port 4040 diff --git a/docs/monitoring.md b/docs/monitoring.md index f6d52ef4597e..69bf3082f0f2 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -185,6 +185,23 @@ Security options for the Spark History Server are covered more detail in the Job history files older than this will be deleted when the filesystem history cleaner runs. + + spark.history.fs.endEventReparseChunkSize + 1m + + How many bytes to parse at the end of log files looking for the end event. + This is used to speed up generation of application listings by skipping unnecessary + parts of event log files. It can be disabled by setting this config to 0. + + + + spark.history.fs.inProgressOptimization.enabled + true + + Enable optimized handling of in-progress logs. This option may leave finished + applications that fail to rename their event logs listed as in-progress. + + spark.history.fs.numReplayThreads 25% of available cores @@ -192,6 +209,14 @@ Security options for the Spark History Server are covered more detail in the Number of threads that will be used by history server to process event logs. + + spark.history.store.maxDiskUsage + 10g + + Maximum disk usage for the local directory where the cache application history information + are stored. + + spark.history.store.path (none) From 443d12dbbe40e932978a9a1a811128da8afba89b Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Wed, 3 Oct 2018 07:28:34 -0700 Subject: [PATCH 136/879] [SPARK-25538][SQL] Zero-out all bytes when writing decimal ## What changes were proposed in this pull request? In #20850 when writing non-null decimals, instead of zero-ing all the 16 allocated bytes, we zero-out only the padding bytes. Since we always allocate 16 bytes, if the number of bytes needed for a decimal is lower than 9, then this means that the bytes between 8 and 16 are not zero-ed. I see 2 solutions here: - we can zero-out all the bytes in advance as it was done before #20850 (safer solution IMHO); - we can allocate only the needed bytes (may be a bit more efficient in terms of memory used, but I have not investigated the feasibility of this option). Hence I propose here the first solution in order to fix the correctness issue. We can eventually switch to the second if we think is more efficient later. ## How was this patch tested? Running the test attached in the JIRA + added UT Closes #22602 from mgaido91/SPARK-25582. Authored-by: Marco Gaido Signed-off-by: Dongjoon Hyun (cherry picked from commit d7ae36a810bfcbedfe7360eb2cdbbc3ca970e4d0) Signed-off-by: Dongjoon Hyun --- .../expressions/codegen/UnsafeRowWriter.java | 10 ++-- .../codegen/UnsafeRowWriterSuite.scala | 53 +++++++++++++++++++ 2 files changed, 57 insertions(+), 6 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java index 71c49d8ed017..3960d6d52047 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java @@ -185,13 +185,13 @@ public void write(int ordinal, Decimal input, int precision, int scale) { // grow the global buffer before writing data. holder.grow(16); + // always zero-out the 16-byte buffer + Platform.putLong(getBuffer(), cursor(), 0L); + Platform.putLong(getBuffer(), cursor() + 8, 0L); + // Make sure Decimal object has the same scale as DecimalType. // Note that we may pass in null Decimal object to set null for it. if (input == null || !input.changePrecision(precision, scale)) { - // zero-out the bytes - Platform.putLong(getBuffer(), cursor(), 0L); - Platform.putLong(getBuffer(), cursor() + 8, 0L); - BitSetMethods.set(getBuffer(), startingOffset, ordinal); // keep the offset for future update setOffsetAndSize(ordinal, 0); @@ -200,8 +200,6 @@ public void write(int ordinal, Decimal input, int precision, int scale) { final int numBytes = bytes.length; assert numBytes <= 16; - zeroOutPaddingBytes(numBytes); - // Write the bytes to the variable length portion. Platform.copyMemory( bytes, Platform.BYTE_ARRAY_OFFSET, getBuffer(), cursor(), numBytes); diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala new file mode 100644 index 000000000000..fb651b76fc16 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.codegen + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types.Decimal + +class UnsafeRowWriterSuite extends SparkFunSuite { + + def checkDecimalSizeInBytes(decimal: Decimal, numBytes: Int): Unit = { + assert(decimal.toJavaBigDecimal.unscaledValue().toByteArray.length == numBytes) + } + + test("SPARK-25538: zero-out all bits for decimals") { + val decimal1 = Decimal(0.431) + decimal1.changePrecision(38, 18) + checkDecimalSizeInBytes(decimal1, 8) + + val decimal2 = Decimal(123456789.1232456789) + decimal2.changePrecision(38, 18) + checkDecimalSizeInBytes(decimal2, 11) + // On an UnsafeRowWriter we write decimal2 first and then decimal1 + val unsafeRowWriter1 = new UnsafeRowWriter(1) + unsafeRowWriter1.resetRowWriter() + unsafeRowWriter1.write(0, decimal2, decimal2.precision, decimal2.scale) + unsafeRowWriter1.reset() + unsafeRowWriter1.write(0, decimal1, decimal1.precision, decimal1.scale) + val res1 = unsafeRowWriter1.getRow + // On a second UnsafeRowWriter we write directly decimal1 + val unsafeRowWriter2 = new UnsafeRowWriter(1) + unsafeRowWriter2.resetRowWriter() + unsafeRowWriter2.write(0, decimal1, decimal1.precision, decimal1.scale) + val res2 = unsafeRowWriter2.getRow + // The two rows should be the equal + assert(res1 == res2) + } + +} From 0763b758de55fd14d7da4832d01b5713e582b257 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 4 Oct 2018 09:36:23 +0800 Subject: [PATCH 137/879] [SPARK-25601][PYTHON] Register Grouped aggregate UDF Vectorized UDFs for SQL Statement ## What changes were proposed in this pull request? This PR proposes to register Grouped aggregate UDF Vectorized UDFs for SQL Statement, for instance: ```python from pyspark.sql.functions import pandas_udf, PandasUDFType pandas_udf("integer", PandasUDFType.GROUPED_AGG) def sum_udf(v): return v.sum() spark.udf.register("sum_udf", sum_udf) q = "SELECT v2, sum_udf(v1) FROM VALUES (3, 0), (2, 0), (1, 1) tbl(v1, v2) GROUP BY v2" spark.sql(q).show() ``` ``` +---+-----------+ | v2|sum_udf(v1)| +---+-----------+ | 1| 1| | 0| 5| +---+-----------+ ``` ## How was this patch tested? Manual test and unit test. Closes #22620 from HyukjinKwon/SPARK-25601. Authored-by: hyukjinkwon Signed-off-by: hyukjinkwon --- python/pyspark/sql/tests.py | 20 ++++++++++++++++++-- python/pyspark/sql/udf.py | 15 +++++++++++++-- 2 files changed, 31 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 690035a0e7df..e9910321c26d 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -5595,8 +5595,9 @@ def test_register_grouped_map_udf(self): foo_udf = pandas_udf(lambda x: x, "id long", PandasUDFType.GROUPED_MAP) with QuietTest(self.sc): - with self.assertRaisesRegexp(ValueError, 'f must be either SQL_BATCHED_UDF or ' - 'SQL_SCALAR_PANDAS_UDF'): + with self.assertRaisesRegexp( + ValueError, + 'f.*SQL_BATCHED_UDF.*SQL_SCALAR_PANDAS_UDF.*SQL_GROUPED_AGG_PANDAS_UDF.*'): self.spark.catalog.registerFunction("foo_udf", foo_udf) def test_decorator(self): @@ -6412,6 +6413,21 @@ def test_invalid_args(self): 'mixture.*aggregate function.*group aggregate pandas UDF'): df.groupby(df.id).agg(mean_udf(df.v), mean(df.v)).collect() + def test_register_vectorized_udf_basic(self): + from pyspark.sql.functions import pandas_udf + from pyspark.rdd import PythonEvalType + + sum_pandas_udf = pandas_udf( + lambda v: v.sum(), "integer", PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF) + + self.assertEqual(sum_pandas_udf.evalType, PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF) + group_agg_pandas_udf = self.spark.udf.register("sum_pandas_udf", sum_pandas_udf) + self.assertEqual(group_agg_pandas_udf.evalType, PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF) + q = "SELECT sum_pandas_udf(v1) FROM VALUES (3, 0), (2, 0), (1, 1) tbl(v1, v2) GROUP BY v2" + actual = sorted(map(lambda r: r[0], self.spark.sql(q).collect())) + expected = [1, 5] + self.assertEqual(actual, expected) + @unittest.skipIf( not _have_pandas or not _have_pyarrow, diff --git a/python/pyspark/sql/udf.py b/python/pyspark/sql/udf.py index 9dbe49b831ce..58f4e0dff5ee 100644 --- a/python/pyspark/sql/udf.py +++ b/python/pyspark/sql/udf.py @@ -298,6 +298,15 @@ def register(self, name, f, returnType=None): >>> spark.sql("SELECT add_one(id) FROM range(3)").collect() # doctest: +SKIP [Row(add_one(id)=1), Row(add_one(id)=2), Row(add_one(id)=3)] + >>> @pandas_udf("integer", PandasUDFType.GROUPED_AGG) # doctest: +SKIP + ... def sum_udf(v): + ... return v.sum() + ... + >>> _ = spark.udf.register("sum_udf", sum_udf) # doctest: +SKIP + >>> q = "SELECT sum_udf(v1) FROM VALUES (3, 0), (2, 0), (1, 1) tbl(v1, v2) GROUP BY v2" + >>> spark.sql(q).collect() # doctest: +SKIP + [Row(sum_udf(v1)=1), Row(sum_udf(v1)=5)] + .. note:: Registration for a user-defined function (case 2.) was added from Spark 2.3.0. """ @@ -310,9 +319,11 @@ def register(self, name, f, returnType=None): "Invalid returnType: data type can not be specified when f is" "a user-defined function, but got %s." % returnType) if f.evalType not in [PythonEvalType.SQL_BATCHED_UDF, - PythonEvalType.SQL_SCALAR_PANDAS_UDF]: + PythonEvalType.SQL_SCALAR_PANDAS_UDF, + PythonEvalType.SQL_GROUPED_AGG_PANDAS_UDF]: raise ValueError( - "Invalid f: f must be either SQL_BATCHED_UDF or SQL_SCALAR_PANDAS_UDF") + "Invalid f: f must be SQL_BATCHED_UDF, SQL_SCALAR_PANDAS_UDF or " + "SQL_GROUPED_AGG_PANDAS_UDF") register_udf = UserDefinedFunction(f.func, returnType=f.returnType, name=name, evalType=f.evalType, deterministic=f.deterministic) From c9bb83a7d66f52f1b9d95257f11f24fe72546e29 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 4 Oct 2018 20:15:21 +0800 Subject: [PATCH 138/879] [SPARK-25602][SQL] SparkPlan.getByteArrayRdd should not consume the input when not necessary ## What changes were proposed in this pull request? In `SparkPlan.getByteArrayRdd`, we should only call `it.hasNext` when the limit is not hit, as `iter.hasNext` may produce one row and buffer it, and cause wrong metrics. ## How was this patch tested? new tests Closes #22621 from cloud-fan/range. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit 71c24aad36ae6b3f50447a019bf893490dcf1cf4) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/SparkPlan.scala | 4 +- .../execution/metric/SQLMetricsSuite.scala | 55 ++++++++++++++++++- 2 files changed, 57 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 1f97993e2045..a84e5efc8fce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -250,7 +250,9 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ val codec = CompressionCodec.createCodec(SparkEnv.get.conf) val bos = new ByteArrayOutputStream() val out = new DataOutputStream(codec.compressedOutputStream(bos)) - while (iter.hasNext && (n < 0 || count < n)) { + // `iter.hasNext` may produce one row and buffer it, we should only call it when the limit is + // not hit. + while ((n < 0 || count < n) && iter.hasNext) { val row = iter.next().asInstanceOf[UnsafeRow] out.writeInt(row.getSizeInBytes) row.writeToStream(out, buffer) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index d45eb0c27a6b..085a44548848 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -24,7 +24,7 @@ import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.LocalRelation -import org.apache.spark.sql.execution.ui.SQLAppStatusStore +import org.apache.spark.sql.execution.{FilterExec, RangeExec, SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -517,4 +517,57 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared test("writing data out metrics with dynamic partition: parquet") { testMetricsDynamicPartition("parquet", "parquet", "t1") } + + test("SPARK-25602: SparkPlan.getByteArrayRdd should not consume the input when not necessary") { + def checkFilterAndRangeMetrics( + df: DataFrame, + filterNumOutputs: Int, + rangeNumOutputs: Int): Unit = { + var filter: FilterExec = null + var range: RangeExec = null + val collectFilterAndRange: SparkPlan => Unit = { + case f: FilterExec => + assert(filter == null, "the query should only have one Filter") + filter = f + case r: RangeExec => + assert(range == null, "the query should only have one Range") + range = r + case _ => + } + if (SQLConf.get.wholeStageEnabled) { + df.queryExecution.executedPlan.foreach { + case w: WholeStageCodegenExec => + w.child.foreach(collectFilterAndRange) + case _ => + } + } else { + df.queryExecution.executedPlan.foreach(collectFilterAndRange) + } + + assert(filter != null && range != null, "the query doesn't have Filter and Range") + assert(filter.metrics("numOutputRows").value == filterNumOutputs) + assert(range.metrics("numOutputRows").value == rangeNumOutputs) + } + + val df = spark.range(0, 3000, 1, 2).toDF().filter('id % 3 === 0) + val df2 = df.limit(2) + Seq(true, false).foreach { wholeStageEnabled => + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> wholeStageEnabled.toString) { + df.collect() + checkFilterAndRangeMetrics(df, filterNumOutputs = 1000, rangeNumOutputs = 3000) + + df.queryExecution.executedPlan.foreach(_.resetMetrics()) + // For each partition, we get 2 rows. Then the Filter should produce 2 rows per-partition, + // and Range should produce 1000 rows (one batch) per-partition. Totally Filter produces + // 4 rows, and Range produces 2000 rows. + df.queryExecution.toRdd.mapPartitions(_.take(2)).collect() + checkFilterAndRangeMetrics(df, filterNumOutputs = 4, rangeNumOutputs = 2000) + + // Top-most limit will call `CollectLimitExec.executeCollect`, which will only run the first + // task, so totally the Filter produces 2 rows, and Range produces 1000 rows (one batch). + df2.collect() + checkFilterAndRangeMetrics(df2, filterNumOutputs = 2, rangeNumOutputs = 1000) + } + } + } } From 2c700ee30d7fe7c7fdc7dbfe697ef5f41bd17215 Mon Sep 17 00:00:00 2001 From: s71955 Date: Fri, 5 Oct 2018 13:09:16 +0800 Subject: [PATCH 139/879] [SPARK-25521][SQL] Job id showing null in the logs when insert into command Job is finished. ## What changes were proposed in this pull request? ``As part of insert command in FileFormatWriter, a job context is created for handling the write operation , While initializing the job context using setupJob() API in HadoopMapReduceCommitProtocol , we set the jobid in the Jobcontext configuration.In FileFormatWriter since we are directly getting the jobId from the map reduce JobContext the job id will come as null while adding the log. As a solution we shall get the jobID from the configuration of the map reduce Jobcontext.`` ## How was this patch tested? Manually, verified the logs after the changes. ![spark-25521 1](https://user-images.githubusercontent.com/12999161/46164933-e95ab700-c2ac-11e8-88e9-49fa5100b872.PNG) Closes #22572 from sujith71955/master_log_issue. Authored-by: s71955 Signed-off-by: Wenchen Fan (cherry picked from commit 459700727fadf3f35a211eab2ffc8d68a4a1c39a) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/datasources/FileFormatWriter.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 7c6ab4bc922f..774fe38f5c2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -183,15 +183,15 @@ object FileFormatWriter extends Logging { val commitMsgs = ret.map(_.commitMsg) committer.commitJob(job, commitMsgs) - logInfo(s"Job ${job.getJobID} committed.") + logInfo(s"Write Job ${description.uuid} committed.") processStats(description.statsTrackers, ret.map(_.summary.stats)) - logInfo(s"Finished processing stats for job ${job.getJobID}.") + logInfo(s"Finished processing stats for write job ${description.uuid}.") // return a set of all the partition paths that were updated during this job ret.map(_.summary.updatedPartitions).reduceOption(_ ++ _).getOrElse(Set.empty) } catch { case cause: Throwable => - logError(s"Aborting job ${job.getJobID}.", cause) + logError(s"Aborting job ${description.uuid}.", cause) committer.abortJob(job) throw new SparkException("Job aborted.", cause) } From 0a70afdc08d76f84c59ec50f2f92144f54271602 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 5 Oct 2018 10:45:15 -0700 Subject: [PATCH 140/879] [SPARK-25644][SS] Fix java foreachBatch in DataStreamWriter ## What changes were proposed in this pull request? The java `foreachBatch` API in `DataStreamWriter` should accept `java.lang.Long` rather `scala.Long`. ## How was this patch tested? New java test. Closes #22633 from zsxwing/fix-java-foreachbatch. Authored-by: Shixiong Zhu Signed-off-by: Shixiong Zhu --- .../sql/streaming/DataStreamWriter.scala | 2 +- .../JavaDataStreamReaderWriterSuite.java | 89 +++++++++++++++++++ 2 files changed, 90 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 735fd1751634..4eb2918657fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -379,7 +379,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * @since 2.4.0 */ @InterfaceStability.Evolving - def foreachBatch(function: VoidFunction2[Dataset[T], Long]): DataStreamWriter[T] = { + def foreachBatch(function: VoidFunction2[Dataset[T], java.lang.Long]): DataStreamWriter[T] = { foreachBatch((batchDs: Dataset[T], batchId: Long) => function.call(batchDs, batchId)) } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java new file mode 100644 index 000000000000..48cdb2642d83 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/streaming/JavaDataStreamReaderWriterSuite.java @@ -0,0 +1,89 @@ +/* + * 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 test.org.apache.spark.sql.streaming; + +import java.io.File; + +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.function.VoidFunction2; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.ForeachWriter; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.streaming.StreamingQuery; +import org.apache.spark.sql.test.TestSparkSession; +import org.apache.spark.util.Utils; + +public class JavaDataStreamReaderWriterSuite { + private SparkSession spark; + private String input; + + @Before + public void setUp() { + spark = new TestSparkSession(); + input = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "input").toString(); + } + + @After + public void tearDown() { + try { + Utils.deleteRecursively(new File(input)); + } finally { + spark.stop(); + spark = null; + } + } + + @Test + public void testForeachBatchAPI() { + StreamingQuery query = spark + .readStream() + .textFile(input) + .writeStream() + .foreachBatch(new VoidFunction2, Long>() { + @Override + public void call(Dataset v1, Long v2) throws Exception {} + }) + .start(); + query.stop(); + } + + @Test + public void testForeachAPI() { + StreamingQuery query = spark + .readStream() + .textFile(input) + .writeStream() + .foreach(new ForeachWriter() { + @Override + public boolean open(long partitionId, long epochId) { + return true; + } + + @Override + public void process(String value) {} + + @Override + public void close(Throwable errorOrNull) {} + }) + .start(); + query.stop(); + } +} From a2991d23348bd1f4ecc33e5c762ccd12bb65f5cd Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Fri, 5 Oct 2018 21:15:16 -0700 Subject: [PATCH 141/879] [SPARK-25646][K8S] Fix docker-image-tool.sh on dev build. The docker file was referencing a path that only existed in the distribution tarball; it needs to be parameterized so that the right path can be used in a dev build. Tested on local dev build. Closes #22634 from vanzin/SPARK-25646. Authored-by: Marcelo Vanzin Signed-off-by: Dongjoon Hyun (cherry picked from commit 58287a39864db463eeef17d1152d664be021d9ef) Signed-off-by: Dongjoon Hyun --- bin/docker-image-tool.sh | 2 ++ .../kubernetes/docker/src/main/dockerfiles/spark/Dockerfile | 3 ++- 2 files changed, 4 insertions(+), 1 deletion(-) diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index d6371051ef7f..228494de6d5a 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -54,6 +54,8 @@ function build { img_path=$IMG_PATH --build-arg spark_jars=assembly/target/scala-$SPARK_SCALA_VERSION/jars + --build-arg + k8s_tests=resource-managers/kubernetes/integration-tests/tests ) else # Not passed as an argument to docker, but used to validate the Spark directory. diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index 7ae57bf6e42d..1c4dcd547687 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -19,6 +19,7 @@ FROM openjdk:8-alpine ARG spark_jars=jars ARG img_path=kubernetes/dockerfiles +ARG k8s_tests=kubernetes/tests # Before building the docker image, first build and make a Spark distribution following # the instructions in http://spark.apache.org/docs/latest/building-spark.html. @@ -43,7 +44,7 @@ COPY bin /opt/spark/bin COPY sbin /opt/spark/sbin COPY ${img_path}/spark/entrypoint.sh /opt/ COPY examples /opt/spark/examples -COPY kubernetes/tests /opt/spark/tests +COPY ${k8s_tests} /opt/spark/tests COPY data /opt/spark/data ENV SPARK_HOME /opt/spark From 48e2e6fcc3617f021b55c2e2be0cda39cad89711 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 6 Oct 2018 09:40:42 -0700 Subject: [PATCH 142/879] [SPARK-25644][SS][FOLLOWUP][BUILD] Fix Scala 2.12 build error due to foreachBatch ## What changes were proposed in this pull request? This PR fixes the Scala-2.12 build error due to ambiguity in `foreachBatch` test cases. - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-master-test-maven-hadoop-2.7-ubuntu-scala-2.12/428/console ```scala [error] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-ubuntu-scala-2.12/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala:102: ambiguous reference to overloaded definition, [error] both method foreachBatch in class DataStreamWriter of type (function: org.apache.spark.api.java.function.VoidFunction2[org.apache.spark.sql.Dataset[Int],Long])org.apache.spark.sql.streaming.DataStreamWriter[Int] [error] and method foreachBatch in class DataStreamWriter of type (function: (org.apache.spark.sql.Dataset[Int], Long) => Unit)org.apache.spark.sql.streaming.DataStreamWriter[Int] [error] match argument types ((org.apache.spark.sql.Dataset[Int], Any) => Unit) [error] ds.writeStream.foreachBatch((_, _) => {}).trigger(Trigger.Continuous("1 second")).start() [error] ^ [error] /home/jenkins/workspace/spark-master-test-maven-hadoop-2.7-ubuntu-scala-2.12/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala:106: ambiguous reference to overloaded definition, [error] both method foreachBatch in class DataStreamWriter of type (function: org.apache.spark.api.java.function.VoidFunction2[org.apache.spark.sql.Dataset[Int],Long])org.apache.spark.sql.streaming.DataStreamWriter[Int] [error] and method foreachBatch in class DataStreamWriter of type (function: (org.apache.spark.sql.Dataset[Int], Long) => Unit)org.apache.spark.sql.streaming.DataStreamWriter[Int] [error] match argument types ((org.apache.spark.sql.Dataset[Int], Any) => Unit) [error] ds.writeStream.foreachBatch((_, _) => {}).partitionBy("value").start() [error] ^ ``` ## How was this patch tested? Manual. Since this failure occurs in Scala-2.12 profile and test cases, Jenkins will not test this. We need to build with Scala-2.12 and run the tests. Closes #22649 from dongjoon-hyun/SPARK-SCALA212. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 9cbf105ab1256d65f027115ba5505842ce8fffe3) Signed-off-by: Dongjoon Hyun --- .../spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala | 4 ++-- .../execution/streaming/sources/ForeachBatchSinkSuite.scala | 5 +++-- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index e0b6d8cdd3d2..d89e45e1e77f 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.TopicPartition import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ -import org.apache.spark.sql.{ForeachWriter, SparkSession} +import org.apache.spark.sql.{Dataset, ForeachWriter, SparkSession} import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution @@ -879,7 +879,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { } testUtils.waitUntilOffsetAppears(topicPartition, 5) - val q = ds.writeStream.foreachBatch { (ds, epochId) => + val q = ds.writeStream.foreachBatch { (ds: Dataset[String], epochId: Long) => if (epochId == 0) { // Send more message before the tasks of the current batch start reading the current batch // data, so that the executors will prefetch messages in the next batch and drop them. In diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala index 71dff443e883..3e9ccb0f705d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/ForeachBatchSinkSuite.scala @@ -99,11 +99,12 @@ class ForeachBatchSinkSuite extends StreamTest { } assert(ex1.getMessage.contains("foreachBatch function cannot be null")) val ex2 = intercept[AnalysisException] { - ds.writeStream.foreachBatch((_, _) => {}).trigger(Trigger.Continuous("1 second")).start() + ds.writeStream.foreachBatch((_: Dataset[Int], _: Long) => {}) + .trigger(Trigger.Continuous("1 second")).start() } assert(ex2.getMessage.contains("'foreachBatch' is not supported with continuous trigger")) val ex3 = intercept[AnalysisException] { - ds.writeStream.foreachBatch((_, _) => {}).partitionBy("value").start() + ds.writeStream.foreachBatch((_: Dataset[Int], _: Long) => {}).partitionBy("value").start() } assert(ex3.getMessage.contains("'foreachBatch' does not support partitioning")) } From c8b94099a88763e6609966ef0dbf160ecbfe33d8 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 6 Oct 2018 15:49:41 -0700 Subject: [PATCH 143/879] [SPARK-25671] Build external/spark-ganglia-lgpl in Jenkins Test ## What changes were proposed in this pull request? Currently, we do not build external/spark-ganglia-lgpl in Jenkins tests when the code is changed. ## How was this patch tested? N/A Closes #22658 from gatorsmile/buildGanglia. Authored-by: gatorsmile Signed-off-by: gatorsmile (cherry picked from commit 8bb242902760535d12c6c40c5d8481a98fdc11e0) Signed-off-by: gatorsmile --- dev/sparktestsupport/modules.py | 10 ++++++++++ 1 file changed, 10 insertions(+) diff --git a/dev/sparktestsupport/modules.py b/dev/sparktestsupport/modules.py index e267fbfa623b..e7ac063e234e 100644 --- a/dev/sparktestsupport/modules.py +++ b/dev/sparktestsupport/modules.py @@ -557,6 +557,16 @@ def __hash__(self): sbt_test_goals=["kubernetes/test"] ) + +spark_ganglia_lgpl = Module( + name="spark-ganglia-lgpl", + dependencies=[], + build_profile_flags=["-Pspark-ganglia-lgpl"], + source_file_regexes=[ + "external/spark-ganglia-lgpl", + ] +) + # The root module is a dummy module which is used to run all of the tests. # No other modules should directly depend on this module. root = Module( From 4214ddd34514351a58cf6a0254f33c6d5c8fd924 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 8 Oct 2018 15:07:06 +0800 Subject: [PATCH 144/879] [SPARK-25673][BUILD] Remove Travis CI which enables Java lint check ## What changes were proposed in this pull request? https://github.com/apache/spark/pull/12980 added Travis CI file mainly for linter because we disabled Java lint check in Jenkins. It's enabled as of https://github.com/apache/spark/pull/21399 and now SBT runs it. Looks we can now remove the file added before. ## How was this patch tested? N/A Closes #22665 Closes #22667 from HyukjinKwon/SPARK-25673. Authored-by: hyukjinkwon Signed-off-by: hyukjinkwon (cherry picked from commit 219922422003e59cc8b3bece60778536759fa669) Signed-off-by: hyukjinkwon --- .travis.yml | 50 -------------------------------------------------- 1 file changed, 50 deletions(-) delete mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml deleted file mode 100644 index 05b94adeeb93..000000000000 --- a/.travis.yml +++ /dev/null @@ -1,50 +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. - -# Spark provides this Travis CI configuration file to help contributors -# check Scala/Java style conformance and JDK7/8 compilation easily -# during their preparing pull requests. -# - Scalastyle is executed during `maven install` implicitly. -# - Java Checkstyle is executed by `lint-java`. -# See the related discussion here. -# https://github.com/apache/spark/pull/12980 - -# 1. Choose OS (Ubuntu 14.04.3 LTS Server Edition 64bit, ~2 CORE, 7.5GB RAM) -sudo: required -dist: trusty - -# 2. Choose language and target JDKs for parallel builds. -language: java -jdk: - - oraclejdk8 - -# 3. Setup cache directory for SBT and Maven. -cache: - directories: - - $HOME/.sbt - - $HOME/.m2 - -# 4. Turn off notifications. -notifications: - email: false - -# 5. Run maven install before running lint-java. -install: - - export MAVEN_SKIP_RC=1 - - build/mvn -T 4 -q -DskipTests -Pkubernetes -Pmesos -Pyarn -Pkinesis-asl -Phive -Phive-thriftserver install - -# 6. Run lint-java. -script: - - dev/lint-java From 692ddb3f92ad6ee5ceca2f5ee4ea67d636c32d88 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 8 Oct 2018 15:18:08 +0800 Subject: [PATCH 145/879] [SPARK-25591][PYSPARK][SQL] Avoid overwriting deserialized accumulator ## What changes were proposed in this pull request? If we use accumulators in more than one UDFs, it is possible to overwrite deserialized accumulators and its values. We should check if an accumulator was deserialized before overwriting it in accumulator registry. ## How was this patch tested? Added test. Closes #22635 from viirya/SPARK-25591. Authored-by: Liang-Chi Hsieh Signed-off-by: hyukjinkwon (cherry picked from commit cb90617f894fd51a092710271823ec7d1cd3a668) Signed-off-by: hyukjinkwon --- python/pyspark/accumulators.py | 12 ++++++++---- python/pyspark/sql/tests.py | 25 +++++++++++++++++++++++++ 2 files changed, 33 insertions(+), 4 deletions(-) diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 30ad04297c68..00ec094e7e3b 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -109,10 +109,14 @@ def _deserialize_accumulator(aid, zero_value, accum_param): from pyspark.accumulators import _accumulatorRegistry - accum = Accumulator(aid, zero_value, accum_param) - accum._deserialized = True - _accumulatorRegistry[aid] = accum - return accum + # If this certain accumulator was deserialized, don't overwrite it. + if aid in _accumulatorRegistry: + return _accumulatorRegistry[aid] + else: + accum = Accumulator(aid, zero_value, accum_param) + accum._deserialized = True + _accumulatorRegistry[aid] = accum + return accum class Accumulator(object): diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index e9910321c26d..b05de54773eb 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -3556,6 +3556,31 @@ def test_repr_behaviors(self): self.assertEquals(None, df._repr_html_()) self.assertEquals(expected, df.__repr__()) + # SPARK-25591 + def test_same_accumulator_in_udfs(self): + from pyspark.sql.functions import udf + + data_schema = StructType([StructField("a", IntegerType(), True), + StructField("b", IntegerType(), True)]) + data = self.spark.createDataFrame([[1, 2]], schema=data_schema) + + test_accum = self.sc.accumulator(0) + + def first_udf(x): + test_accum.add(1) + return x + + def second_udf(x): + test_accum.add(100) + return x + + func_udf = udf(first_udf, IntegerType()) + func_udf2 = udf(second_udf, IntegerType()) + data = data.withColumn("out1", func_udf(data["a"])) + data = data.withColumn("out2", func_udf2(data["b"])) + data.collect() + self.assertEqual(test_accum.value, 101) + class HiveSparkSubmitTests(SparkSubmitTests): From 193ce77fccf54cfdacdc011db13655c28e524458 Mon Sep 17 00:00:00 2001 From: shivusondur Date: Mon, 8 Oct 2018 15:43:08 +0800 Subject: [PATCH 146/879] [SPARK-25677][DOC] spark.io.compression.codec = org.apache.spark.io.ZstdCompressionCodec throwing IllegalArgumentException Exception ## What changes were proposed in this pull request? Documentation is updated with proper classname org.apache.spark.io.ZStdCompressionCodec ## How was this patch tested? we used the spark.io.compression.codec = org.apache.spark.io.ZStdCompressionCodec and verified the logs. Closes #22669 from shivusondur/CompressionIssue. Authored-by: shivusondur Signed-off-by: hyukjinkwon (cherry picked from commit 1a6815cd9f421a106f8d96a36a53042a00f02386) Signed-off-by: hyukjinkwon --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 55773937d4d7..613e214783d5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -954,7 +954,7 @@ Apart from these, the following properties are also available, and may be useful org.apache.spark.io.LZ4CompressionCodec, org.apache.spark.io.LZFCompressionCodec, org.apache.spark.io.SnappyCompressionCodec, - and org.apache.spark.io.ZstdCompressionCodec. + and org.apache.spark.io.ZStdCompressionCodec. From 4baa4d42acf2400537462bddff811a7644bb49a3 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 8 Oct 2018 14:32:04 -0700 Subject: [PATCH 147/879] [SPARK-25639][DOCS] Added docs for foreachBatch, python foreach and multiple watermarks ## What changes were proposed in this pull request? Added - Python foreach - Scala, Java and Python foreachBatch - Multiple watermark policy - The semantics of what changes are allowed to the streaming between restarts. ## How was this patch tested? No tests Closes #22627 from tdas/SPARK-25639. Authored-by: Tathagata Das Signed-off-by: Tathagata Das (cherry picked from commit f9935a3f85f46deef2cb7b213c1c02c8ff627a8c) Signed-off-by: Tathagata Das --- .../structured-streaming-programming-guide.md | 323 +++++++++++++++++- 1 file changed, 312 insertions(+), 11 deletions(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 73de1892977a..b6e427735e74 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -1560,6 +1560,35 @@ streamingDf <- dropDuplicates(streamingDf, "guid", "eventTime")
    +### Policy for handling multiple watermarks +A streaming query can have multiple input streams that are unioned or joined together. +Each of the input streams can have a different threshold of late data that needs to +be tolerated for stateful operations. You specify these thresholds using +``withWatermarks("eventTime", delay)`` on each of the input streams. For example, consider +a query with stream-stream joins between `inputStream1` and `inputStream2`. + + inputStream1.withWatermark("eventTime1", "1 hour") + .join( + inputStream2.withWatermark("eventTime2", "2 hours"), + joinCondition) + +While executing the query, Structured Streaming individually tracks the maximum +event time seen in each input stream, calculates watermarks based on the corresponding delay, +and chooses a single global watermark with them to be used for stateful operations. By default, +the minimum is chosen as the global watermark because it ensures that no data is +accidentally dropped as too late if one of the streams falls behind the others +(for example, one of the streams stop receiving data due to upstream failures). In other words, +the global watermark will safely move at the pace of the slowest stream and the query output will +be delayed accordingly. + +However, in some cases, you may want to get faster results even if it means dropping data from the +slowest stream. Since Spark 2.4, you can set the multiple watermark policy to choose +the maximum value as the global watermark by setting the SQL configuration +``spark.sql.streaming.multipleWatermarkPolicy`` to ``max`` (default is ``min``). +This lets the global watermark move at the pace of the fastest stream. +However, as a side effect, data from the slower streams will be aggressively dropped. Hence, use +this configuration judiciously. + ### Arbitrary Stateful Operations Many usecases require more advanced stateful operations than aggregations. For example, in many usecases, you have to track sessions from data streams of events. For doing such sessionization, you will have to save arbitrary types of data as state, and perform arbitrary operations on the state using the data stream events in every trigger. Since Spark 2.2, this can be done using the operation `mapGroupsWithState` and the more powerful operation `flatMapGroupsWithState`. Both operations allow you to apply user-defined code on grouped Datasets to update user-defined state. For more concrete details, take a look at the API documentation ([Scala](api/scala/index.html#org.apache.spark.sql.streaming.GroupState)/[Java](api/java/org/apache/spark/sql/streaming/GroupState.html)) and the examples ([Scala]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala)/[Java]({{site.SPARK_GITHUB_URL}}/blob/v{{site.SPARK_VERSION_SHORT}}/examples/src/main/java/org/apache/spark/examples/sql/streaming/JavaStructuredSessionization.java)). @@ -1799,8 +1828,16 @@ Here are the details of all the sinks in Spark. Append, Update, Complete None Depends on ForeachWriter implementation - More details in the next section + More details in the next section + + ForeachBatch Sink + Append, Update, Complete + None + Depends on the implementation + More details in the next section + + Console Sink Append, Update, Complete @@ -1989,22 +2026,214 @@ head(sql("select * from aggregates")) -##### Using Foreach -The `foreach` operation allows arbitrary operations to be computed on the output data. As of Spark 2.1, this is available only for Scala and Java. To use this, you will have to implement the interface `ForeachWriter` -([Scala](api/scala/index.html#org.apache.spark.sql.ForeachWriter)/[Java](api/java/org/apache/spark/sql/ForeachWriter.html) docs), -which has methods that get called whenever there is a sequence of rows generated as output after a trigger. Note the following important points. +##### Using Foreach and ForeachBatch +The `foreach` and `foreachBatch` operations allow you to apply arbitrary operations and writing +logic on the output of a streaming query. They have slightly different use cases - while `foreach` +allows custom write logic on every row, `foreachBatch` allows arbitrary operations +and custom logic on the output of each micro-batch. Let's understand their usages in more detail. + +###### ForeachBatch +`foreachBatch(...)` allows you to specify a function that is executed on +the output data of every micro-batch of a streaming query. Since Spark 2.4, this is supported in Scala, Java and Python. +It takes two parameters: a DataFrame or Dataset that has the output data of a micro-batch and the unique ID of the micro-batch. + +
    +
    + +{% highlight scala %} +streamingDF.writeStream.foreachBatch { (batchDF: DataFrame, batchId: Long) => + // Transform and write batchDF +}.start() +{% endhighlight %} + +
    +
    + +{% highlight java %} +streamingDatasetOfString.writeStream().foreachBatch( + new VoidFunction2, Long> { + public void call(Dataset dataset, Long batchId) { + // Transform and write batchDF + } + } +).start(); +{% endhighlight %} + +
    +
    + +{% highlight python %} +def foreach_batch_function(df, epoch_id): + # Transform and write batchDF + pass + +streamingDF.writeStream.foreachBatch(foreach_batch_function).start() +{% endhighlight %} + +
    +
    +R is not yet supported. +
    +
    + +With `foreachBatch`, you can do the following. + +- **Reuse existing batch data sources** - For many storage systems, there may not be a streaming sink available yet, + but there may already exist a data writer for batch queries. Using `foreachBatch`, you can use the batch + data writers on the output of each micro-batch. +- **Write to multiple locations** - If you want to write the output of a streaming query to multiple locations, + then you can simply write the output DataFrame/Dataset multiple times. However, each attempt to write can + cause the output data to be recomputed (including possible re-reading of the input data). To avoid recomputations, + you should cache the output DataFrame/Dataset, write it to multiple locations, and then uncache it. Here is an outline. + + streamingDF.writeStream.foreachBatch { (batchDF: DataFrame, batchId: Long) => + batchDF.persist() + batchDF.write.format(...).save(...) // location 1 + batchDF.write.format(...).save(...) // location 2 + batchDF.unpersist() + } + +- **Apply additional DataFrame operations** - Many DataFrame and Dataset operations are not supported + in streaming DataFrames because Spark does not support generating incremental plans in those cases. + Using `foreachBatch`, you can apply some of these operations on each micro-batch output. However, you will have to reason about the end-to-end semantics of doing that operation yourself. + +**Note:** +- By default, `foreachBatch` provides only at-least-once write guarantees. However, you can use the + batchId provided to the function as way to deduplicate the output and get an exactly-once guarantee. +- `foreachBatch` does not work with the continuous processing mode as it fundamentally relies on the + micro-batch execution of a streaming query. If you write data in the continuous mode, use `foreach` instead. + + +###### Foreach +If `foreachBatch` is not an option (for example, corresponding batch data writer does not exist, or +continuous processing mode), then you can express you custom writer logic using `foreach`. +Specifically, you can express the data writing logic by dividing it into three methods: `open`, `process`, and `close`. +Since Spark 2.4, `foreach` is available in Scala, Java and Python. + +
    +
    + +In Scala, you have to extend the class `ForeachWriter` ([docs](api/scala/index.html#org.apache.spark.sql.ForeachWriter)). + +{% highlight scala %} +streamingDatasetOfString.writeStream.foreach( + new ForeachWriter[String] { + + def open(partitionId: Long, version: Long): Boolean = { + // Open connection + } + + def process(record: String): Unit = { + // Write string to connection + } + + def close(errorOrNull: Throwable): Unit = { + // Close the connection + } + } +).start() +{% endhighlight %} + +
    +
    + +In Java, you have to extend the class `ForeachWriter` ([docs](api/java/org/apache/spark/sql/ForeachWriter.html)). +{% highlight java %} +streamingDatasetOfString.writeStream().foreach( + new ForeachWriter[String] { + + @Override public boolean open(long partitionId, long version) { + // Open connection + } + + @Override public void process(String record) { + // Write string to connection + } + + @Override public void close(Throwable errorOrNull) { + // Close the connection + } + } +).start(); + +{% endhighlight %} + +
    +
    + +In Python, you can invoke foreach in two ways: in a function or in an object. +The function offers a simple way to express your processing logic but does not allow you to +deduplicate generated data when failures cause reprocessing of some input data. +For that situation you must specify the processing logic in an object. + +1. The function takes a row as input. + + {% highlight python %} + def process_row(row): + # Write row to storage + pass + + query = streamingDF.writeStream.foreach(process_row).start() + {% endhighlight %} + +2. The object has a process method and optional open and close methods: + + {% highlight python %} + class ForeachWriter: + def open(self, partition_id, epoch_id): + # Open connection. This method is optional in Python. + pass + + def process(self, row): + # Write row to connection. This method is NOT optional in Python. + pass + + def close(self, error): + # Close the connection. This method in optional in Python. + pass + + query = streamingDF.writeStream.foreach(ForeachWriter()).start() + {% endhighlight %} + +
    +
    +R is not yet supported. +
    +
    + + +**Execution semantics** +When the streaming query is started, Spark calls the function or the object’s methods in the following way: + +- A single copy of this object is responsible for all the data generated by a single task in a query. + In other words, one instance is responsible for processing one partition of the data generated in a distributed manner. + +- This object must be serializable, because each task will get a fresh serialized-deserialized copy + of the provided object. Hence, it is strongly recommended that any initialization for writing data + (for example. opening a connection or starting a transaction) is done after the open() method has + been called, which signifies that the task is ready to generate data. + +- The lifecycle of the methods are as follows: + + - For each partition with partition_id: -- The writer must be serializable, as it will be serialized and sent to the executors for execution. + - For each batch/epoch of streaming data with epoch_id: -- All the three methods, `open`, `process` and `close` will be called on the executors. + - Method open(partitionId, epochId) is called. -- The writer must do all the initialization (e.g. opening connections, starting a transaction, etc.) only when the `open` method is called. Be aware that, if there is any initialization in the class as soon as the object is created, then that initialization will happen in the driver (because that is where the instance is being created), which may not be what you intend. + - If open(...) returns true, for each row in the partition and batch/epoch, method process(row) is called. -- `version` and `partition` are two parameters in `open` that uniquely represent a set of rows that needs to be pushed out. `version` is a monotonically increasing id that increases with every trigger. `partition` is an id that represents a partition of the output, since the output is distributed and will be processed on multiple executors. + - Method close(error) is called with error (if any) seen while processing rows. -- `open` can use the `version` and `partition` to choose whether it needs to write the sequence of rows. Accordingly, it can return `true` (proceed with writing), or `false` (no need to write). If `false` is returned, then `process` will not be called on any row. For example, after a partial failure, some of the output partitions of the failed trigger may have already been committed to a database. Based on metadata stored in the database, the writer can identify partitions that have already been committed and accordingly return false to skip committing them again. +- The close() method (if it exists) is called if an open() method exists and returns successfully (irrespective of the return value), except if the JVM or Python process crashes in the middle. -- Whenever `open` is called, `close` will also be called (unless the JVM exits due to some error). This is true even if `open` returns false. If there is any error in processing and writing the data, `close` will be called with the error. It is your responsibility to clean up state (e.g. connections, transactions, etc.) that have been created in `open` such that there are no resource leaks. +- **Note:** The partitionId and epochId in the open() method can be used to deduplicate generated data + when failures cause reprocessing of some input data. This depends on the execution mode of the query. + If the streaming query is being executed in the micro-batch mode, then every partition represented + by a unique tuple (partition_id, epoch_id) is guaranteed to have the same data. + Hence, (partition_id, epoch_id) can be used to deduplicate and/or transactionally commit + data and achieve exactly-once guarantees. However, if the streaming query is being executed + in the continuous mode, then this guarantee does not hold and therefore should not be used for deduplication. #### Triggers The trigger settings of a streaming query defines the timing of streaming data processing, whether @@ -2709,6 +2938,78 @@ write.stream(aggDF, "memory", outputMode = "complete", checkpointLocation = "pat + +## Recovery Semantics after Changes in a Streaming Query +There are limitations on what changes in a streaming query are allowed between restarts from the +same checkpoint location. Here are a few kinds of changes that are either not allowed, or +the effect of the change is not well-defined. For all of them: + +- The term *allowed* means you can do the specified change but whether the semantics of its effect + is well-defined depends on the query and the change. + +- The term *not allowed* means you should not do the specified change as the restarted query is likely + to fail with unpredictable errors. `sdf` represents a streaming DataFrame/Dataset + generated with sparkSession.readStream. + +**Types of changes** + +- *Changes in the number or type (i.e. different source) of input sources*: This is not allowed. + +- *Changes in the parameters of input sources*: Whether this is allowed and whether the semantics + of the change are well-defined depends on the source and the query. Here are a few examples. + + - Addition/deletion/modification of rate limits is allowed: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "topic").option("maxOffsetsPerTrigger", ...)` + + - Changes to subscribed topics/files is generally not allowed as the results are unpredictable: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "newTopic")` + +- *Changes in the type of output sink*: Changes between a few specific combinations of sinks + are allowed. This needs to be verified on a case-by-case basis. Here are a few examples. + + - File sink to Kafka sink is allowed. Kafka will see only the new data. + + - Kafka sink to file sink is not allowed. + + - Kafka sink changed to foreach, or vice versa is allowed. + +- *Changes in the parameters of output sink*: Whether this is allowed and whether the semantics of + the change are well-defined depends on the sink and the query. Here are a few examples. + + - Changes to output directory of a file sink is not allowed: `sdf.writeStream.format("parquet").option("path", "/somePath")` to `sdf.writeStream.format("parquet").option("path", "/anotherPath")` + + - Changes to output topic is allowed: `sdf.writeStream.format("kafka").option("topic", "someTopic")` to `sdf.writeStream.format("kafka").option("topic", "anotherTopic")` + + - Changes to the user-defined foreach sink (that is, the `ForeachWriter` code) is allowed, but the semantics of the change depends on the code. + +- *Changes in projection / filter / map-like operations**: Some cases are allowed. For example: + + - Addition / deletion of filters is allowed: `sdf.selectExpr("a")` to `sdf.where(...).selectExpr("a").filter(...)`. + + - Changes in projections with same output schema is allowed: `sdf.selectExpr("stringColumn AS json").writeStream` to `sdf.selectExpr("anotherStringColumn AS json").writeStream` + + - Changes in projections with different output schema are conditionally allowed: `sdf.selectExpr("a").writeStream` to `sdf.selectExpr("b").writeStream` is allowed only if the output sink allows the schema change from `"a"` to `"b"`. + +- *Changes in stateful operations*: Some operations in streaming queries need to maintain + state data in order to continuously update the result. Structured Streaming automatically checkpoints + the state data to fault-tolerant storage (for example, HDFS, AWS S3, Azure Blob storage) and restores it after restart. + However, this assumes that the schema of the state data remains same across restarts. This means that + *any changes (that is, additions, deletions, or schema modifications) to the stateful operations of a streaming query are not allowed between restarts*. + Here is the list of stateful operations whose schema should not be changed between restarts in order to ensure state recovery: + + - *Streaming aggregation*: For example, `sdf.groupBy("a").agg(...)`. Any change in number or type of grouping keys or aggregates is not allowed. + + - *Streaming deduplication*: For example, `sdf.dropDuplicates("a")`. Any change in number or type of grouping keys or aggregates is not allowed. + + - *Stream-stream join*: For example, `sdf1.join(sdf2, ...)` (i.e. both inputs are generated with `sparkSession.readStream`). Changes + in the schema or equi-joining columns are not allowed. Changes in join type (outer or inner) not allowed. Other changes in the join condition are ill-defined. + + - *Arbitrary stateful operation*: For example, `sdf.groupByKey(...).mapGroupsWithState(...)` or `sdf.groupByKey(...).flatMapGroupsWithState(...)`. + Any change to the schema of the user-defined state and the type of timeout is not allowed. + Any change within the user-defined state-mapping function are allowed, but the semantic effect of the change depends on the user-defined logic. + If you really want to support state schema changes, then you can explicitly encode/decode your complex state data + structures into bytes using an encoding/decoding scheme that supports schema migration. For example, + if you save your state as Avro-encoded bytes, then you are free to change the Avro-state-schema between query + restarts as the binary state will always be restored successfully. + # Continuous Processing ## [Experimental] {:.no_toc} From 404c840393086290cf975652f596b4768aa5d4eb Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 9 Oct 2018 14:35:00 +0800 Subject: [PATCH 148/879] [SPARK-25669][SQL] Check CSV header only when it exists ## What changes were proposed in this pull request? Currently the first row of dataset of CSV strings is compared to field names of user specified or inferred schema independently of presence of CSV header. It causes false-positive error messages. For example, parsing `"1,2"` outputs the error: ```java java.lang.IllegalArgumentException: CSV header does not conform to the schema. Header: 1, 2 Schema: _c0, _c1 Expected: _c0 but found: 1 ``` In the PR, I propose: - Checking CSV header only when it exists - Filter header from the input dataset only if it exists ## How was this patch tested? Added a test to `CSVSuite` which reproduces the issue. Closes #22656 from MaxGekk/inferred-header-check. Authored-by: Maxim Gekk Signed-off-by: hyukjinkwon (cherry picked from commit 46fe40838aa682a7073dd6f1373518b0c8498a94) Signed-off-by: hyukjinkwon --- .../main/scala/org/apache/spark/sql/DataFrameReader.scala | 7 +++++-- .../spark/sql/execution/datasources/csv/CSVSuite.scala | 6 ++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 27a1af27a644..869c584aed90 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -505,7 +505,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { val actualSchema = StructType(schema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) - val linesWithoutHeader: RDD[String] = maybeFirstLine.map { firstLine => + val linesWithoutHeader = if (parsedOptions.headerFlag && maybeFirstLine.isDefined) { + val firstLine = maybeFirstLine.get val parser = new CsvParser(parsedOptions.asParserSettings) val columnNames = parser.parseLine(firstLine) CSVDataSource.checkHeaderColumnNames( @@ -515,7 +516,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { parsedOptions.enforceSchema, sparkSession.sessionState.conf.caseSensitiveAnalysis) filteredLines.rdd.mapPartitions(CSVUtils.filterHeaderLine(_, firstLine, parsedOptions)) - }.getOrElse(filteredLines.rdd) + } else { + filteredLines.rdd + } val parsed = linesWithoutHeader.mapPartitions { iter => val rawParser = new UnivocityParser(actualSchema, parsedOptions) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index f70df0bcecde..5d4746cf90b3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -1820,4 +1820,10 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te checkAnswer(spark.read.option("multiLine", true).schema(schema).csv(input), Row(null)) assert(spark.read.csv(input).collect().toSet == Set(Row())) } + + test("field names of inferred schema shouldn't compare to the first row") { + val input = Seq("1,2").toDS() + val df = spark.read.option("enforceSchema", false).csv(input) + checkAnswer(df, Row("1", "2")) + } } From 8e4a99bd201b9204fec52580f19ae70a229ed94e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 10 Oct 2018 13:26:12 +0000 Subject: [PATCH 149/879] Preparing Spark release v2.4.0-rc3 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 714b6f18b19d..f52d785e05cd 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.0 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index ee0de73f21d5..63ab510eb683 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b89e0fed2a66..b10e11849a74 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 3105a8182afc..74c6d232b36a 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f85248b53080..fbdc979d120d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 3350990c0461..53d1b2b58f1f 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 9a2cb90a2f30..98145481adc7 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 9b927a9ba525..f0c8ebe25e63 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 718875f88e74..c635cd3b9394 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 583a2fff7638..ed3efffa180b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 72473771c2e9..7f9a91e2fba0 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1-SNAPSHOT -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.0 +SPARK_VERSION_SHORT: 2.4.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 737ebfd7900c..c7b8354b0968 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 7c6b78ac2b31..a91c13362caa 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 6343a15c43a8..db239f23e1de 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 2f4886832b75..cf19fed1bef5 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9404e2a52d90..da5dba82f3cf 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 73dbc8d99449..e591ce802256 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 912d82b764e5..e8b13cdc2bef 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 8454f5c108dc..c3d3b888656b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 05f7cf870af8..468ace0ff8d7 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 0e935ac112af..db8fda66c3cd 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 7977389718cb..aa1e1267f57b 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7193df69900d..7d07e18f04a2 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 4b318b00ff5b..f545f212b2f0 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6fbb1b7db6aa..6b388ffcf18b 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 9a42d9a69967..fc369c73edc0 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 702f60ab9f00..f4cb94147d9e 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 62322da714bd..22f60efae9af 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index c8e45b27c78f..b17e67d78b98 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 6201e0dd3c9a..bf0d406da9ed 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 28804be0301c..4cc619e697e4 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 87282141fa64..53e3424394ba 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1.dev0" +__version__ = "2.4.0" diff --git a/repl/pom.xml b/repl/pom.xml index d09548348eb5..4532f0b8686e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index a9d9c8de9959..47fe3f2765b2 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 793167103179..f59895e9a1bd 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index e6457289b859..0eacbbfac2b9 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index e70711ce7257..2afdc15be446 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 65cf33f32972..ef1d012d2f80 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c327b85ee866..f4be074642ae 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d9d742444e7e..e5c1065451f7 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c428be67aa3d..b522bfeac130 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 37e96b807653..8e71adabb3ec 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index dd00c2f6cd2c..c00e1b851ba0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml From 71b8739fe0f6d63775ee799e5867295ff6637c8c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 10 Oct 2018 13:26:16 +0000 Subject: [PATCH 150/879] Preparing development version 2.4.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index f52d785e05cd..714b6f18b19d 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.0 +Version: 2.4.1 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 63ab510eb683..ee0de73f21d5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b10e11849a74..b89e0fed2a66 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 74c6d232b36a..3105a8182afc 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index fbdc979d120d..f85248b53080 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 53d1b2b58f1f..3350990c0461 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 98145481adc7..9a2cb90a2f30 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index f0c8ebe25e63..9b927a9ba525 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index c635cd3b9394..718875f88e74 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ed3efffa180b..583a2fff7638 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7f9a91e2fba0..72473771c2e9 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.0 -SPARK_VERSION_SHORT: 2.4.0 +SPARK_VERSION: 2.4.1-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index c7b8354b0968..737ebfd7900c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index a91c13362caa..7c6b78ac2b31 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index db239f23e1de..6343a15c43a8 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index cf19fed1bef5..2f4886832b75 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index da5dba82f3cf..9404e2a52d90 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index e591ce802256..73dbc8d99449 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index e8b13cdc2bef..912d82b764e5 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index c3d3b888656b..8454f5c108dc 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 468ace0ff8d7..05f7cf870af8 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index db8fda66c3cd..0e935ac112af 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index aa1e1267f57b..7977389718cb 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7d07e18f04a2..7193df69900d 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f545f212b2f0..4b318b00ff5b 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6b388ffcf18b..6fbb1b7db6aa 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index fc369c73edc0..9a42d9a69967 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index f4cb94147d9e..702f60ab9f00 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 22f60efae9af..62322da714bd 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index b17e67d78b98..c8e45b27c78f 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bf0d406da9ed..6201e0dd3c9a 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4cc619e697e4..28804be0301c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 53e3424394ba..87282141fa64 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.0" +__version__ = "2.4.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 4532f0b8686e..d09548348eb5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 47fe3f2765b2..a9d9c8de9959 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index f59895e9a1bd..793167103179 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 0eacbbfac2b9..e6457289b859 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 2afdc15be446..e70711ce7257 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ef1d012d2f80..65cf33f32972 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f4be074642ae..c327b85ee866 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index e5c1065451f7..d9d742444e7e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b522bfeac130..c428be67aa3d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 8e71adabb3ec..37e96b807653 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c00e1b851ba0..dd00c2f6cd2c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml From cd40655965072051dfae65eabd979edff0e4d398 Mon Sep 17 00:00:00 2001 From: Devaraj K Date: Wed, 10 Oct 2018 09:24:36 -0700 Subject: [PATCH 151/879] [SPARK-25636][CORE] spark-submit cuts off the failure reason when there is an error connecting to master ## What changes were proposed in this pull request? Cause of the error is wrapped with SparkException, now finding the cause from the wrapped exception and throwing the cause instead of the wrapped exception. ## How was this patch tested? Verified it manually by checking the cause of the error, it gives the error as shown below. ### Without the PR change ``` [apache-spark]$ ./bin/spark-submit --verbose --master spark://****** .... Error: Exception thrown in awaitResult: Run with --help for usage help or --verbose for debug output ``` ### With the PR change ``` [apache-spark]$ ./bin/spark-submit --verbose --master spark://****** .... Exception in thread "main" org.apache.spark.SparkException: Exception thrown in awaitResult: at org.apache.spark.util.ThreadUtils$.awaitResult(ThreadUtils.scala:226) at org.apache.spark.rpc.RpcTimeout.awaitResult(RpcTimeout.scala:75) .... at org.apache.spark.deploy.SparkSubmit.main(SparkSubmit.scala) Caused by: java.io.IOException: Failed to connect to devaraj-pc1/10.3.66.65:7077 at org.apache.spark.network.client.TransportClientFactory.createClient(TransportClientFactory.java:245) .... at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) Caused by: io.netty.channel.AbstractChannel$AnnotatedConnectException: Connection refused: devaraj-pc1/10.3.66.65:7077 at sun.nio.ch.SocketChannelImpl.checkConnect(Native Method) .... at io.netty.util.concurrent.DefaultThreadFactory$DefaultRunnableDecorator.run(DefaultThreadFactory.java:138) ... 1 more Caused by: java.net.ConnectException: Connection refused ... 11 more ``` Closes #22623 from devaraj-kavali/SPARK-25636. Authored-by: Devaraj K Signed-off-by: Marcelo Vanzin (cherry picked from commit 8a7872dc254710f9b29fdfdb2915a949ef606871) Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/deploy/SparkSubmit.scala | 2 -- .../apache/spark/deploy/SparkSubmitSuite.scala | 17 +++++++++++------ 2 files changed, 11 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index cf902db8709e..1d32d964dc9d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -925,8 +925,6 @@ object SparkSubmit extends CommandLineUtils with Logging { } catch { case e: SparkUserAppException => exitFn(e.exitCode) - case e: SparkException => - printErrorAndExit(e.getMessage()) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 9eae3605d073..652c36ffa6e7 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -74,20 +74,25 @@ trait TestPrematureExit { @volatile var exitedCleanly = false mainObject.exitFn = (_) => exitedCleanly = true + @volatile var exception: Exception = null val thread = new Thread { override def run() = try { mainObject.main(input) } catch { - // If exceptions occur after the "exit" has happened, fine to ignore them. - // These represent code paths not reachable during normal execution. - case e: Exception => if (!exitedCleanly) throw e + // Capture the exception to check whether the exception contains searchString or not + case e: Exception => exception = e } } thread.start() thread.join() - val joined = printStream.lineBuffer.mkString("\n") - if (!joined.contains(searchString)) { - fail(s"Search string '$searchString' not found in $joined") + if (exitedCleanly) { + val joined = printStream.lineBuffer.mkString("\n") + assert(joined.contains(searchString)) + } else { + assert(exception != null) + if (!exception.getMessage.contains(searchString)) { + throw exception + } } } } From e80ab130e2327949224d433a14ba1e33310ee24c Mon Sep 17 00:00:00 2001 From: liuxian Date: Thu, 11 Oct 2018 14:24:15 -0700 Subject: [PATCH 152/879] [SPARK-25674][SQL] If the records are incremented by more than 1 at a time,the number of bytes might rarely ever get updated MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? If the records are incremented by more than 1 at a time,the number of bytes might rarely ever get updated,because it might skip over the count that is an exact multiple of UPDATE_INPUT_METRICS_INTERVAL_RECORDS. This PR just checks whether the increment causes the value to exceed a higher multiple of UPDATE_INPUT_METRICS_INTERVAL_RECORDS. ## How was this patch tested? existed unit tests Closes #22594 from 10110346/inputMetrics. Authored-by: liuxian Signed-off-by: Sean Owen (cherry picked from commit 69f5e9cce14632a1f912c3632243a4e20b275365) Signed-off-by: Sean Owen --- .../apache/spark/sql/execution/datasources/FileScanRDD.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index 345c9d82ca0e..dd3c154259c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -104,12 +104,15 @@ class FileScanRDD( val nextElement = currentIterator.next() // TODO: we should have a better separation of row based and batch based scan, so that we // don't need to run this `if` for every record. + val preNumRecordsRead = inputMetrics.recordsRead if (nextElement.isInstanceOf[ColumnarBatch]) { inputMetrics.incRecordsRead(nextElement.asInstanceOf[ColumnarBatch].numRows()) } else { inputMetrics.incRecordsRead(1) } - if (inputMetrics.recordsRead % SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + // The records may be incremented by more than 1 at a time. + if (preNumRecordsRead / SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS != + inputMetrics.recordsRead / SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS) { updateBytesRead() } nextElement From 1961f8e62f2c6d546d42d37423bfad2f55e75e6a Mon Sep 17 00:00:00 2001 From: maryannxue Date: Thu, 11 Oct 2018 20:45:08 -0700 Subject: [PATCH 153/879] [SPARK-25690][SQL] Analyzer rule HandleNullInputsForUDF does not stabilize and can be applied infinitely ## What changes were proposed in this pull request? The HandleNullInputsForUDF rule can generate new If node infinitely, thus causing problems like match of SQL cache missed. This was fixed in SPARK-24891 and was then broken by SPARK-25044. The unit test in `AnalysisSuite` added in SPARK-24891 should have failed but didn't because it wasn't properly updated after the `ScalaUDF` constructor signature change. So this PR also updates the test accordingly based on the new `ScalaUDF` constructor. ## How was this patch tested? Updated the original UT. This should be justified as the original UT became invalid after SPARK-25044. Closes #22701 from maryannxue/spark-25690. Authored-by: maryannxue Signed-off-by: gatorsmile (cherry picked from commit 368513048198efcee8c9a35678b608be0cb9ad48) Signed-off-by: gatorsmile --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 4 +++- .../apache/spark/sql/catalyst/analysis/AnalysisSuite.scala | 4 ++-- 2 files changed, 5 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index fdb68dd38f21..9c0975eecd44 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2164,8 +2164,10 @@ class Analyzer( // TODO: skip null handling for not-nullable primitive inputs after we can completely // trust the `nullable` information. + val needsNullCheck = (nullable: Boolean, expr: Expression) => + nullable && !expr.isInstanceOf[KnownNotNull] val inputsNullCheck = nullableTypes.zip(inputs) - .filter { case (nullable, _) => !nullable } + .filter { case (nullableType, expr) => needsNullCheck(!nullableType, expr) } .map { case (_, expr) => IsNull(expr) } .reduceLeftOption[Expression]((e1, e2) => Or(e1, e2)) // Once we add an `If` check above the udf, it is safe to mark those checked inputs diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index f9facbb71a4e..cf76c92b093b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -351,8 +351,8 @@ class AnalysisSuite extends AnalysisTest with Matchers { test("SPARK-24891 Fix HandleNullInputsForUDF rule") { val a = testRelation.output(0) val func = (x: Int, y: Int) => x + y - val udf1 = ScalaUDF(func, IntegerType, a :: a :: Nil) - val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil) + val udf1 = ScalaUDF(func, IntegerType, a :: a :: Nil, nullableTypes = false :: false :: Nil) + val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil, nullableTypes = false :: false :: Nil) val plan = Project(Alias(udf2, "")() :: Nil, testRelation) comparePlans(plan.analyze, plan.analyze.analyze) } From 3dba5d41f1a66ae5eb08404d103284110c45a351 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 12 Oct 2018 00:24:06 -0700 Subject: [PATCH 154/879] [SPARK-25708][SQL] HAVING without GROUP BY means global aggregate According to the SQL standard, when a query contains `HAVING`, it indicates an aggregate operator. For more details please refer to https://blog.jooq.org/2014/12/04/do-you-really-understand-sqls-group-by-and-having-clauses/ However, in Spark SQL parser, we treat HAVING as a normal filter when there is no GROUP BY, which breaks SQL semantic and lead to wrong result. This PR fixes the parser. new test Closes #22696 from cloud-fan/having. Authored-by: Wenchen Fan Signed-off-by: gatorsmile (cherry picked from commit 78e133141ce8131c60181f947346802864b0951a) Signed-off-by: gatorsmile --- docs/sql-programming-guide.md | 1 + .../sql/catalyst/parser/AstBuilder.scala | 41 +++++++++++++------ .../apache/spark/sql/internal/SQLConf.scala | 8 ++++ .../sql/catalyst/parser/PlanParserSuite.scala | 2 +- .../resources/sql-tests/inputs/group-by.sql | 7 ++++ .../sql-tests/results/group-by.sql.out | 27 +++++++++++- .../sql/hive/execution/HiveQuerySuite.scala | 4 -- 7 files changed, 71 insertions(+), 19 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index d525405e548b..e45e50da9fe0 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1971,6 +1971,7 @@ working with timestamps in `pandas_udf`s to get the best performance, see - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. + - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. ## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index 7bc1f63e3054..c6d21058e6f1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -394,6 +394,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging Filter(expression(ctx), plan) } + def withHaving(ctx: BooleanExpressionContext, plan: LogicalPlan): LogicalPlan = { + // Note that we add a cast to non-predicate expressions. If the expression itself is + // already boolean, the optimizer will get rid of the unnecessary cast. + val predicate = expression(ctx) match { + case p: Predicate => p + case e => Cast(e, BooleanType) + } + Filter(predicate, plan) + } + + // Expressions. val expressions = Option(namedExpressionSeq).toSeq .flatMap(_.namedExpression.asScala) @@ -446,30 +457,34 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case e: NamedExpression => e case e: Expression => UnresolvedAlias(e) } - val withProject = if (aggregation != null) { - withAggregation(aggregation, namedExpressions, withFilter) - } else if (namedExpressions.nonEmpty) { + + def createProject() = if (namedExpressions.nonEmpty) { Project(namedExpressions, withFilter) } else { withFilter } - // Having - val withHaving = withProject.optional(having) { - // Note that we add a cast to non-predicate expressions. If the expression itself is - // already boolean, the optimizer will get rid of the unnecessary cast. - val predicate = expression(having) match { - case p: Predicate => p - case e => Cast(e, BooleanType) + val withProject = if (aggregation == null && having != null) { + if (conf.getConf(SQLConf.LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE)) { + // If the legacy conf is set, treat HAVING without GROUP BY as WHERE. + withHaving(having, createProject()) + } else { + // According to SQL standard, HAVING without GROUP BY means global aggregate. + withHaving(having, Aggregate(Nil, namedExpressions, withFilter)) } - Filter(predicate, withProject) + } else if (aggregation != null) { + val aggregate = withAggregation(aggregation, namedExpressions, withFilter) + aggregate.optionalMap(having)(withHaving) + } else { + // When hitting this branch, `having` must be null. + createProject() } // Distinct val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) { - Distinct(withHaving) + Distinct(withProject) } else { - withHaving + withProject } // Window diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 99d2f562082e..05264d33e440 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -1539,6 +1539,14 @@ object SQLConf { "are performed before any UNION, EXCEPT and MINUS operations.") .booleanConf .createWithDefault(false) + + val LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE = + buildConf("spark.sql.legacy.parser.havingWithoutGroupByAsWhere") + .internal() + .doc("If it is set to true, the parser will treat HAVING without GROUP BY as a normal " + + "WHERE, which does not follow SQL standard.") + .booleanConf + .createWithDefault(false) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala index 422bf97e30e7..f5da90f7cf0c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/PlanParserSuite.scala @@ -108,7 +108,7 @@ class PlanParserSuite extends AnalysisTest { assertEqual("select a, b from db.c where x < 1", table("db", "c").where('x < 1).select('a, 'b)) assertEqual( "select a, b from db.c having x < 1", - table("db", "c").select('a, 'b).where('x < 1)) + table("db", "c").groupBy()('a, 'b).where('x < 1)) assertEqual("select distinct a, b from db.c", Distinct(table("db", "c").select('a, 'b))) assertEqual("select all a, b from db.c", table("db", "c").select('a, 'b)) assertEqual("select from tbl", OneRowRelation().select('from.as("tbl"))) diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 2c18d6aaabdb..433db7152743 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -73,3 +73,10 @@ where b.z != b.z; -- SPARK-24369 multiple distinct aggregations having the same argument set SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) FROM (VALUES (1, 1), (2, 2), (2, 2)) t(x, y); + +-- SPARK-25708 HAVING without GROUP BY means global aggregate +SELECT 1 FROM range(10) HAVING true; + +SELECT 1 FROM range(10) HAVING MAX(id) > 0; + +SELECT id FROM range(10) HAVING id > 0; diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index 581aa1754ce1..f9d1ee8a6bcd 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 27 +-- Number of queries: 30 -- !query 0 @@ -250,3 +250,28 @@ SELECT corr(DISTINCT x, y), corr(DISTINCT y, x), count(*) struct -- !query 26 output 1.0 1.0 3 + + +-- !query 27 +SELECT 1 FROM range(10) HAVING true +-- !query 27 schema +struct<1:int> +-- !query 27 output +1 + + +-- !query 28 +SELECT 1 FROM range(10) HAVING MAX(id) > 0 +-- !query 28 schema +struct<1:int> +-- !query 28 output +1 + + +-- !query 29 +SELECT id FROM range(10) HAVING id > 0 +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.AnalysisException +grouping expressions sequence is empty, and '`id`' is not an aggregate function. Wrap '()' in windowing function(s) or wrap '`id`' in first() (or first_value) if you don't care which value you get.; diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index b9c32e789a41..a5cff35abf37 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -740,10 +740,6 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd sql("select key, count(*) c from src group by key having c").collect() } - test("SPARK-2225: turn HAVING without GROUP BY into a simple filter") { - assert(sql("select key from src having key > 490").collect().size < 100) - } - test("union/except/intersect") { assertResult(Array(Row(1), Row(1))) { sql("select 1 as a union all select 1 as a").collect() From bb211cf27bd1e106da430fff144a3a579cce026d Mon Sep 17 00:00:00 2001 From: Shahid Date: Fri, 12 Oct 2018 12:57:09 -0500 Subject: [PATCH 155/879] [SPARK-25697][CORE] When zstd compression enabled, InProgress application is throwing Error in the history webui ## What changes were proposed in this pull request? When we enable event log compression and compression codec as 'zstd', we are unable to open the webui of the running application from the history server page. The reason is that, Replay listener was unable to read from the zstd compressed eventlog due to the zstd frame was not finished yet. This causes truncated error while reading the eventLog. So, when we try to open the WebUI from the History server page, it throws "truncated error ", and we never able to open running application in the webui, when we enable zstd compression. In this PR, when the IO excpetion happens, and if it is a running application, we log the error, "Failed to read Spark event log: evetLogDirAppName.inprogress", instead of throwing exception. ## How was this patch tested? Test steps: 1)spark.eventLog.compress = true 2)spark.io.compression.codec = zstd 3)restart history server 4) launch bin/spark-shell 5) run some queries 6) Open history server page 7) click on the application **Before fix:** ![screenshot from 2018-10-10 23-52-12](https://user-images.githubusercontent.com/23054875/46757387-9b4fa580-cce7-11e8-96ad-8938400483ed.png) ![screenshot from 2018-10-10 23-52-28](https://user-images.githubusercontent.com/23054875/46757393-a0145980-cce7-11e8-8cb0-44b583dde648.png) **After fix:** ![screenshot from 2018-10-10 23-43-49](https://user-images.githubusercontent.com/23054875/46756971-6858e200-cce6-11e8-946c-0bffebb2cfba.png) ![screenshot from 2018-10-10 23-44-05](https://user-images.githubusercontent.com/23054875/46756981-6d1d9600-cce6-11e8-95ea-ff8339a2fdfd.png) (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #22689 from shahidki31/SPARK-25697. Authored-by: Shahid Signed-off-by: Sean Owen (cherry picked from commit 8e039a75548e91b0a8799d9d72c6797b066ddd62) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/scheduler/ReplayListenerBus.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index 226c23733c87..4c6b0c1227b1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -118,6 +118,8 @@ private[spark] class ReplayListenerBus extends SparkListenerBus with Logging { case e: HaltReplayException => // Just stop replay. case _: EOFException if maybeTruncated => + case _: IOException if maybeTruncated => + logWarning(s"Failed to read Spark event log: $sourceName") case ioe: IOException => throw ioe case e: Exception => From 1a335444e6ba4124bd0f7f351f097c0bdb46ae85 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Fri, 12 Oct 2018 12:04:00 -0700 Subject: [PATCH 156/879] [SPARK-25660][SQL] Fix for the backward slash as CSV fields delimiter ## What changes were proposed in this pull request? The PR addresses the exception raised on accessing chars out of delimiter string. In particular, the backward slash `\` as the CSV fields delimiter causes the following exception on reading `abc\1`: ```Scala String index out of range: 1 java.lang.StringIndexOutOfBoundsException: String index out of range: 1 at java.lang.String.charAt(String.java:658) ``` because `str.charAt(1)` tries to access a char out of `str` in `CSVUtils.toChar` ## How was this patch tested? Added tests for empty string and string containing the backward slash to `CSVUtilsSuite`. Besides of that I added an end-to-end test to check how the backward slash is handled in reading CSV string with it. Closes #22654 from MaxGekk/csv-slash-delim. Authored-by: Maxim Gekk Signed-off-by: gatorsmile (cherry picked from commit c7eadb5e6652468f9d5cd714c112ba1de187eea8) Signed-off-by: gatorsmile --- .../execution/datasources/csv/CSVUtils.scala | 36 ++++++++++--------- .../execution/datasources/csv/CSVSuite.scala | 10 ++++++ .../datasources/csv/CSVUtilsSuite.scala | 14 ++++++++ 3 files changed, 43 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala index 7ce65fa89b02..b367b3d0ac64 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala @@ -97,23 +97,25 @@ object CSVUtils { */ @throws[IllegalArgumentException] def toChar(str: String): Char = { - if (str.charAt(0) == '\\') { - str.charAt(1) - match { - case 't' => '\t' - case 'r' => '\r' - case 'b' => '\b' - case 'f' => '\f' - case '\"' => '\"' // In case user changes quote char and uses \" as delimiter in options - case '\'' => '\'' - case 'u' if str == """\u0000""" => '\u0000' - case _ => - throw new IllegalArgumentException(s"Unsupported special character for delimiter: $str") - } - } else if (str.length == 1) { - str.charAt(0) - } else { - throw new IllegalArgumentException(s"Delimiter cannot be more than one character: $str") + (str: Seq[Char]) match { + case Seq() => throw new IllegalArgumentException("Delimiter cannot be empty string") + case Seq('\\') => throw new IllegalArgumentException("Single backslash is prohibited." + + " It has special meaning as beginning of an escape sequence." + + " To get the backslash character, pass a string with two backslashes as the delimiter.") + case Seq(c) => c + case Seq('\\', 't') => '\t' + case Seq('\\', 'r') => '\r' + case Seq('\\', 'b') => '\b' + case Seq('\\', 'f') => '\f' + // In case user changes quote char and uses \" as delimiter in options + case Seq('\\', '\"') => '\"' + case Seq('\\', '\'') => '\'' + case Seq('\\', '\\') => '\\' + case _ if str == """\u0000""" => '\u0000' + case Seq('\\', _) => + throw new IllegalArgumentException(s"Unsupported special character for delimiter: $str") + case _ => + throw new IllegalArgumentException(s"Delimiter cannot be more than one character: $str") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala index 5d4746cf90b3..d59035b716cf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVSuite.scala @@ -1826,4 +1826,14 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te val df = spark.read.option("enforceSchema", false).csv(input) checkAnswer(df, Row("1", "2")) } + + test("using the backward slash as the delimiter") { + val input = Seq("""abc\1""").toDS() + val delimiter = """\\""" + checkAnswer(spark.read.option("delimiter", delimiter).csv(input), Row("abc", "1")) + checkAnswer(spark.read.option("inferSchema", true).option("delimiter", delimiter).csv(input), + Row("abc", 1)) + val schema = new StructType().add("a", StringType).add("b", IntegerType) + checkAnswer(spark.read.schema(schema).option("delimiter", delimiter).csv(input), Row("abc", 1)) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtilsSuite.scala index 221e44ce2cff..60fcbd2ff008 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtilsSuite.scala @@ -28,6 +28,7 @@ class CSVUtilsSuite extends SparkFunSuite { assert(CSVUtils.toChar("""\"""") === '\"') assert(CSVUtils.toChar("""\'""") === '\'') assert(CSVUtils.toChar("""\u0000""") === '\u0000') + assert(CSVUtils.toChar("""\\""") === '\\') } test("Does not accept delimiter larger than one character") { @@ -44,4 +45,17 @@ class CSVUtilsSuite extends SparkFunSuite { assert(exception.getMessage.contains("Unsupported special character for delimiter")) } + test("string with one backward slash is prohibited") { + val exception = intercept[IllegalArgumentException]{ + CSVUtils.toChar("""\""") + } + assert(exception.getMessage.contains("Single backslash is prohibited")) + } + + test("output proper error message for empty string") { + val exception = intercept[IllegalArgumentException]{ + CSVUtils.toChar("") + } + assert(exception.getMessage.contains("Delimiter cannot be empty string")) + } } From 0f58b989d4ebc8218880f8a4a32dc8189e7fbb43 Mon Sep 17 00:00:00 2001 From: Mathieu St-Louis Date: Fri, 12 Oct 2018 14:09:10 -0500 Subject: [PATCH 157/879] [STREAMING][DOC] Fix typo & formatting for JavaDoc ## What changes were proposed in this pull request? - Fixed typo for function outputMode - OutputMode.Complete(), changed `these is some updates` to `there are some updates` - Replaced hyphenized list by HTML unordered list tags in comments to fix the Javadoc documentation. Current render from most recent [Spark API Docs](https://spark.apache.org/docs/2.3.1/api/java/org/apache/spark/sql/streaming/DataStreamWriter.html): #### outputMode(OutputMode) - List formatted as a prose. ![image](https://user-images.githubusercontent.com/2295469/46250648-11086700-c3f4-11e8-8a5a-d88b079c165d.png) #### outputMode(String) - List formatted as a prose. ![image](https://user-images.githubusercontent.com/2295469/46250651-24b3cd80-c3f4-11e8-9dac-ae37599afbce.png) #### partitionBy(String*) - List formatted as a prose. ![image](https://user-images.githubusercontent.com/2295469/46250655-36957080-c3f4-11e8-990b-47bd612d3c51.png) ## How was this patch tested? This PR contains a document patch ergo no functional testing is required. Closes #22593 from niofire/fix-typo-datastreamwriter. Authored-by: Mathieu St-Louis Signed-off-by: Sean Owen (cherry picked from commit 4e141a416082cb978396ffbd6bf529b168652b9d) Signed-off-by: Sean Owen --- .../scala/org/apache/spark/sql/Column.scala | 14 +++---- .../apache/spark/sql/DataFrameWriter.scala | 27 +++++++----- .../org/apache/spark/sql/ForeachWriter.scala | 7 +++- .../spark/sql/SparkSessionExtensions.scala | 15 ++++--- .../sql/streaming/DataStreamWriter.scala | 41 +++++++++++-------- 5 files changed, 61 insertions(+), 43 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index ae27690f2e5b..a046127c3edb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -199,13 +199,13 @@ class Column(val expr: Expression) extends Logging { /** * Extracts a value or values from a complex type. * The following types of extraction are supported: - * - * - Given an Array, an integer ordinal can be used to retrieve a single value. - * - Given a Map, a key of the correct type can be used to retrieve an individual value. - * - Given a Struct, a string fieldName can be used to extract that field. - * - Given an Array of Structs, a string fieldName can be used to extract filed - * of every struct in that array, and return an Array of fields - * + *
      + *
    • Given an Array, an integer ordinal can be used to retrieve a single value.
    • + *
    • Given a Map, a key of the correct type can be used to retrieve an individual value.
    • + *
    • Given a Struct, a string fieldName can be used to extract that field.
    • + *
    • Given an Array of Structs, a string fieldName can be used to extract filed + * of every struct in that array, and return an Array of fields.
    • + *
    * @group expr_ops * @since 1.4.0 */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index 80ade7c3e6ea..c1e2f496682d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -47,10 +47,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Specifies the behavior when data or table already exists. Options include: - * - `SaveMode.Overwrite`: overwrite the existing data. - * - `SaveMode.Append`: append the data. - * - `SaveMode.Ignore`: ignore the operation (i.e. no-op). - * - `SaveMode.ErrorIfExists`: default option, throw an exception at runtime. + *
      + *
    • `SaveMode.Overwrite`: overwrite the existing data.
    • + *
    • `SaveMode.Append`: append the data.
    • + *
    • `SaveMode.Ignore`: ignore the operation (i.e. no-op).
    • + *
    • `SaveMode.ErrorIfExists`: default option, throw an exception at runtime.
    • + *
    * * @since 1.4.0 */ @@ -61,10 +63,12 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { /** * Specifies the behavior when data or table already exists. Options include: - * - `overwrite`: overwrite the existing data. - * - `append`: append the data. - * - `ignore`: ignore the operation (i.e. no-op). - * - `error` or `errorifexists`: default option, throw an exception at runtime. + *
      + *
    • `overwrite`: overwrite the existing data.
    • + *
    • `append`: append the data.
    • + *
    • `ignore`: ignore the operation (i.e. no-op).
    • + *
    • `error` or `errorifexists`: default option, throw an exception at runtime.
    • + *
    * * @since 1.4.0 */ @@ -163,9 +167,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * Partitions the output by the given columns on the file system. If specified, the output is * laid out on the file system similar to Hive's partitioning scheme. As an example, when we * partition a dataset by year and then month, the directory layout would look like: - * - * - year=2016/month=01/ - * - year=2016/month=02/ + *
      + *
    • year=2016/month=01/
    • + *
    • year=2016/month=02/
    • + *
    * * Partitioning is one of the most widely used techniques to optimize physical data layout. * It provides a coarse-grained index for skipping unnecessary data reads when queries have diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala index b21c50af1843..52b8c839643e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala @@ -130,8 +130,11 @@ abstract class ForeachWriter[T] extends Serializable { * Called when stopping to process one partition of new data in the executor side. This is * guaranteed to be called either `open` returns `true` or `false`. However, * `close` won't be called in the following cases: - * - JVM crashes without throwing a `Throwable` - * - `open` throws a `Throwable`. + * + *
      + *
    • JVM crashes without throwing a `Throwable`
    • + *
    • `open` throws a `Throwable`.
    • + *
    * * @param errorOrNull the error thrown during processing data or null if there was no error. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala index f99c108161f9..6b02ac2ded8d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala @@ -30,12 +30,15 @@ import org.apache.spark.sql.catalyst.rules.Rule * regarding binary compatibility and source compatibility of methods here. * * This current provides the following extension points: - * - Analyzer Rules. - * - Check Analysis Rules - * - Optimizer Rules. - * - Planning Strategies. - * - Customized Parser. - * - (External) Catalog listeners. + * + *
      + *
    • Analyzer Rules.
    • + *
    • Check Analysis Rules.
    • + *
    • Optimizer Rules.
    • + *
    • Planning Strategies.
    • + *
    • Customized Parser.
    • + *
    • (External) Catalog listeners.
    • + *
    * * The extensions can be used by calling withExtension on the [[SparkSession.Builder]], for * example: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala index 4eb2918657fd..ec7eb1d3decc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/DataStreamWriter.scala @@ -46,14 +46,16 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { /** * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. - * - `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be - * written to the sink - * - `OutputMode.Complete()`: all the rows in the streaming DataFrame/Dataset will be written - * to the sink every time these is some updates - * - `OutputMode.Update()`: only the rows that were updated in the streaming DataFrame/Dataset - * will be written to the sink every time there are some updates. If - * the query doesn't contain aggregations, it will be equivalent to - * `OutputMode.Append()` mode. + *
      + *
    • `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be + * written to the sink.
    • + *
    • `OutputMode.Complete()`: all the rows in the streaming DataFrame/Dataset will be written + * to the sink every time there are some updates.
    • + *
    • `OutputMode.Update()`: only the rows that were updated in the streaming + * DataFrame/Dataset will be written to the sink every time there are some updates. + * If the query doesn't contain aggregations, it will be equivalent to + * `OutputMode.Append()` mode.
    • + *
    * * @since 2.0.0 */ @@ -64,13 +66,16 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { /** * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. - * - `append`: only the new rows in the streaming DataFrame/Dataset will be written to - * the sink - * - `complete`: all the rows in the streaming DataFrame/Dataset will be written to the sink - * every time these is some updates - * - `update`: only the rows that were updated in the streaming DataFrame/Dataset will - * be written to the sink every time there are some updates. If the query doesn't - * contain aggregations, it will be equivalent to `append` mode. + *
      + *
    • `append`: only the new rows in the streaming DataFrame/Dataset will be written to + * the sink.
    • + *
    • `complete`: all the rows in the streaming DataFrame/Dataset will be written to the sink + * every time there are some updates.
    • + *
    • `update`: only the rows that were updated in the streaming DataFrame/Dataset will + * be written to the sink every time there are some updates. If the query doesn't + * contain aggregations, it will be equivalent to `append` mode.
    • + *
    + * * @since 2.0.0 */ def outputMode(outputMode: String): DataStreamWriter[T] = { @@ -131,8 +136,10 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * laid out on the file system similar to Hive's partitioning scheme. As an example, when we * partition a dataset by year and then month, the directory layout would look like: * - * - year=2016/month=01/ - * - year=2016/month=02/ + *
      + *
    • year=2016/month=01/
    • + *
    • year=2016/month=02/
    • + *
    * * Partitioning is one of the most widely used techniques to optimize physical data layout. * It provides a coarse-grained index for skipping unnecessary data reads when queries have From 5554a33f2809495d78d396339e87fde311427328 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 12 Oct 2018 21:02:38 -0700 Subject: [PATCH 158/879] [SPARK-25714] Fix Null Handling in the Optimizer rule BooleanSimplification ## What changes were proposed in this pull request? ```Scala val df1 = Seq(("abc", 1), (null, 3)).toDF("col1", "col2") df1.write.mode(SaveMode.Overwrite).parquet("/tmp/test1") val df2 = spark.read.parquet("/tmp/test1") df2.filter("col1 = 'abc' OR (col1 != 'abc' AND col2 == 3)").show() ``` Before the PR, it returns both rows. After the fix, it returns `Row ("abc", 1))`. This is to fix the bug in NULL handling in BooleanSimplification. This is a bug introduced in Spark 1.6 release. ## How was this patch tested? Added test cases Closes #22702 from gatorsmile/fixBooleanSimplify2. Authored-by: gatorsmile Signed-off-by: gatorsmile (cherry picked from commit c9ba59d38e2be17b802156b49d374a726e66c6b9) Signed-off-by: gatorsmile --- .../sql/catalyst/expressions/predicates.scala | 35 ++++++ .../sql/catalyst/optimizer/expressions.scala | 34 ++++-- .../BooleanSimplificationSuite.scala | 111 ++++++++++++++---- .../org/apache/spark/sql/DataFrameSuite.scala | 10 ++ 4 files changed, 157 insertions(+), 33 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 149bd79278a5..7f21a628c0bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -129,6 +129,13 @@ case class Not(child: Expression) override def inputTypes: Seq[DataType] = Seq(BooleanType) + // +---------+-----------+ + // | CHILD | NOT CHILD | + // +---------+-----------+ + // | TRUE | FALSE | + // | FALSE | TRUE | + // | UNKNOWN | UNKNOWN | + // +---------+-----------+ protected override def nullSafeEval(input: Any): Any = !input.asInstanceOf[Boolean] override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -406,6 +413,13 @@ case class And(left: Expression, right: Expression) extends BinaryOperator with override def sqlOperator: String = "AND" + // +---------+---------+---------+---------+ + // | AND | TRUE | FALSE | UNKNOWN | + // +---------+---------+---------+---------+ + // | TRUE | TRUE | FALSE | UNKNOWN | + // | FALSE | FALSE | FALSE | FALSE | + // | UNKNOWN | UNKNOWN | FALSE | UNKNOWN | + // +---------+---------+---------+---------+ override def eval(input: InternalRow): Any = { val input1 = left.eval(input) if (input1 == false) { @@ -469,6 +483,13 @@ case class Or(left: Expression, right: Expression) extends BinaryOperator with P override def sqlOperator: String = "OR" + // +---------+---------+---------+---------+ + // | OR | TRUE | FALSE | UNKNOWN | + // +---------+---------+---------+---------+ + // | TRUE | TRUE | TRUE | TRUE | + // | FALSE | TRUE | FALSE | UNKNOWN | + // | UNKNOWN | TRUE | UNKNOWN | UNKNOWN | + // +---------+---------+---------+---------+ override def eval(input: InternalRow): Any = { val input1 = left.eval(input) if (input1 == true) { @@ -592,6 +613,13 @@ case class EqualTo(left: Expression, right: Expression) override def symbol: String = "=" + // +---------+---------+---------+---------+ + // | = | TRUE | FALSE | UNKNOWN | + // +---------+---------+---------+---------+ + // | TRUE | TRUE | FALSE | UNKNOWN | + // | FALSE | FALSE | TRUE | UNKNOWN | + // | UNKNOWN | UNKNOWN | UNKNOWN | UNKNOWN | + // +---------+---------+---------+---------+ protected override def nullSafeEval(left: Any, right: Any): Any = ordering.equiv(left, right) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -629,6 +657,13 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp override def nullable: Boolean = false + // +---------+---------+---------+---------+ + // | <=> | TRUE | FALSE | UNKNOWN | + // +---------+---------+---------+---------+ + // | TRUE | TRUE | FALSE | UNKNOWN | + // | FALSE | FALSE | TRUE | UNKNOWN | + // | UNKNOWN | UNKNOWN | UNKNOWN | TRUE | + // +---------+---------+---------+---------+ override def eval(input: InternalRow): Any = { val input1 = left.eval(input) val input2 = right.eval(input) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index f8037588fa71..8459043b3651 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -276,15 +276,31 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { case a And b if a.semanticEquals(b) => a case a Or b if a.semanticEquals(b) => a - case a And (b Or c) if Not(a).semanticEquals(b) => And(a, c) - case a And (b Or c) if Not(a).semanticEquals(c) => And(a, b) - case (a Or b) And c if a.semanticEquals(Not(c)) => And(b, c) - case (a Or b) And c if b.semanticEquals(Not(c)) => And(a, c) - - case a Or (b And c) if Not(a).semanticEquals(b) => Or(a, c) - case a Or (b And c) if Not(a).semanticEquals(c) => Or(a, b) - case (a And b) Or c if a.semanticEquals(Not(c)) => Or(b, c) - case (a And b) Or c if b.semanticEquals(Not(c)) => Or(a, c) + // The following optimization is applicable only when the operands are nullable, + // since the three-value logic of AND and OR are different in NULL handling. + // See the chart: + // +---------+---------+---------+---------+ + // | p | q | p OR q | p AND q | + // +---------+---------+---------+---------+ + // | TRUE | TRUE | TRUE | TRUE | + // | TRUE | FALSE | TRUE | FALSE | + // | TRUE | UNKNOWN | TRUE | UNKNOWN | + // | FALSE | TRUE | TRUE | FALSE | + // | FALSE | FALSE | FALSE | FALSE | + // | FALSE | UNKNOWN | UNKNOWN | FALSE | + // | UNKNOWN | TRUE | TRUE | UNKNOWN | + // | UNKNOWN | FALSE | UNKNOWN | FALSE | + // | UNKNOWN | UNKNOWN | UNKNOWN | UNKNOWN | + // +---------+---------+---------+---------+ + case a And (b Or c) if !a.nullable && Not(a).semanticEquals(b) => And(a, c) + case a And (b Or c) if !a.nullable && Not(a).semanticEquals(c) => And(a, b) + case (a Or b) And c if !a.nullable && a.semanticEquals(Not(c)) => And(b, c) + case (a Or b) And c if !b.nullable && b.semanticEquals(Not(c)) => And(a, c) + + case a Or (b And c) if !a.nullable && Not(a).semanticEquals(b) => Or(a, c) + case a Or (b And c) if !a.nullable && Not(a).semanticEquals(c) => Or(a, b) + case (a And b) Or c if !a.nullable && a.semanticEquals(Not(c)) => Or(b, c) + case (a And b) Or c if !b.nullable && b.semanticEquals(Not(c)) => Or(a, c) // Common factor elimination for conjunction case and @ (left And right) => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 6cd1108eef33..a0de5f693095 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.BooleanType -class BooleanSimplificationSuite extends PlanTest with PredicateHelper { +class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with PredicateHelper { object Optimize extends RuleExecutor[LogicalPlan] { val batches = @@ -71,6 +71,14 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { comparePlans(actual, correctAnswer) } + private def checkConditionInNotNullableRelation( + input: Expression, expected: Expression): Unit = { + val plan = testNotNullableRelationWithData.where(input).analyze + val actual = Optimize.execute(plan) + val correctAnswer = testNotNullableRelationWithData.where(expected).analyze + comparePlans(actual, correctAnswer) + } + private def checkConditionInNotNullableRelation( input: Expression, expected: LogicalPlan): Unit = { val plan = testNotNullableRelationWithData.where(input).analyze @@ -119,42 +127,55 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { 'a === 'b || 'b > 3 && 'a > 3 && 'a < 5) } - test("e && (!e || f)") { - checkCondition('e && (!'e || 'f ), 'e && 'f) + test("e && (!e || f) - not nullable") { + checkConditionInNotNullableRelation('e && (!'e || 'f ), 'e && 'f) - checkCondition('e && ('f || !'e ), 'e && 'f) + checkConditionInNotNullableRelation('e && ('f || !'e ), 'e && 'f) - checkCondition((!'e || 'f ) && 'e, 'f && 'e) + checkConditionInNotNullableRelation((!'e || 'f ) && 'e, 'f && 'e) - checkCondition(('f || !'e ) && 'e, 'f && 'e) + checkConditionInNotNullableRelation(('f || !'e ) && 'e, 'f && 'e) } - test("a < 1 && (!(a < 1) || f)") { - checkCondition('a < 1 && (!('a < 1) || 'f), ('a < 1) && 'f) - checkCondition('a < 1 && ('f || !('a < 1)), ('a < 1) && 'f) + test("e && (!e || f) - nullable") { + Seq ('e && (!'e || 'f ), + 'e && ('f || !'e ), + (!'e || 'f ) && 'e, + ('f || !'e ) && 'e, + 'e || (!'e && 'f), + 'e || ('f && !'e), + ('e && 'f) || !'e, + ('f && 'e) || !'e).foreach { expr => + checkCondition(expr, expr) + } + } - checkCondition('a <= 1 && (!('a <= 1) || 'f), ('a <= 1) && 'f) - checkCondition('a <= 1 && ('f || !('a <= 1)), ('a <= 1) && 'f) + test("a < 1 && (!(a < 1) || f) - not nullable") { + checkConditionInNotNullableRelation('a < 1 && (!('a < 1) || 'f), ('a < 1) && 'f) + checkConditionInNotNullableRelation('a < 1 && ('f || !('a < 1)), ('a < 1) && 'f) - checkCondition('a > 1 && (!('a > 1) || 'f), ('a > 1) && 'f) - checkCondition('a > 1 && ('f || !('a > 1)), ('a > 1) && 'f) + checkConditionInNotNullableRelation('a <= 1 && (!('a <= 1) || 'f), ('a <= 1) && 'f) + checkConditionInNotNullableRelation('a <= 1 && ('f || !('a <= 1)), ('a <= 1) && 'f) - checkCondition('a >= 1 && (!('a >= 1) || 'f), ('a >= 1) && 'f) - checkCondition('a >= 1 && ('f || !('a >= 1)), ('a >= 1) && 'f) + checkConditionInNotNullableRelation('a > 1 && (!('a > 1) || 'f), ('a > 1) && 'f) + checkConditionInNotNullableRelation('a > 1 && ('f || !('a > 1)), ('a > 1) && 'f) + + checkConditionInNotNullableRelation('a >= 1 && (!('a >= 1) || 'f), ('a >= 1) && 'f) + checkConditionInNotNullableRelation('a >= 1 && ('f || !('a >= 1)), ('a >= 1) && 'f) } - test("a < 1 && ((a >= 1) || f)") { - checkCondition('a < 1 && ('a >= 1 || 'f ), ('a < 1) && 'f) - checkCondition('a < 1 && ('f || 'a >= 1), ('a < 1) && 'f) + test("a < 1 && ((a >= 1) || f) - not nullable") { + checkConditionInNotNullableRelation('a < 1 && ('a >= 1 || 'f ), ('a < 1) && 'f) + checkConditionInNotNullableRelation('a < 1 && ('f || 'a >= 1), ('a < 1) && 'f) - checkCondition('a <= 1 && ('a > 1 || 'f ), ('a <= 1) && 'f) - checkCondition('a <= 1 && ('f || 'a > 1), ('a <= 1) && 'f) + checkConditionInNotNullableRelation('a <= 1 && ('a > 1 || 'f ), ('a <= 1) && 'f) + checkConditionInNotNullableRelation('a <= 1 && ('f || 'a > 1), ('a <= 1) && 'f) - checkCondition('a > 1 && (('a <= 1) || 'f), ('a > 1) && 'f) - checkCondition('a > 1 && ('f || ('a <= 1)), ('a > 1) && 'f) + checkConditionInNotNullableRelation('a > 1 && (('a <= 1) || 'f), ('a > 1) && 'f) + checkConditionInNotNullableRelation('a > 1 && ('f || ('a <= 1)), ('a > 1) && 'f) - checkCondition('a >= 1 && (('a < 1) || 'f), ('a >= 1) && 'f) - checkCondition('a >= 1 && ('f || ('a < 1)), ('a >= 1) && 'f) + checkConditionInNotNullableRelation('a >= 1 && (('a < 1) || 'f), ('a >= 1) && 'f) + checkConditionInNotNullableRelation('a >= 1 && ('f || ('a < 1)), ('a >= 1) && 'f) } test("DeMorgan's law") { @@ -217,4 +238,46 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { checkCondition('e || !'f, testRelationWithData.where('e || !'f).analyze) checkCondition(!'f || 'e, testRelationWithData.where(!'f || 'e).analyze) } + + protected def assertEquivalent(e1: Expression, e2: Expression): Unit = { + val correctAnswer = Project(Alias(e2, "out")() :: Nil, OneRowRelation()).analyze + val actual = Optimize.execute(Project(Alias(e1, "out")() :: Nil, OneRowRelation()).analyze) + comparePlans(actual, correctAnswer) + } + + test("filter reduction - positive cases") { + val fields = Seq( + 'col1NotNULL.boolean.notNull, + 'col2NotNULL.boolean.notNull + ) + val Seq(col1NotNULL, col2NotNULL) = fields.zipWithIndex.map { case (f, i) => f.at(i) } + + val exprs = Seq( + // actual expressions of the transformations: original -> transformed + (col1NotNULL && (!col1NotNULL || col2NotNULL)) -> (col1NotNULL && col2NotNULL), + (col1NotNULL && (col2NotNULL || !col1NotNULL)) -> (col1NotNULL && col2NotNULL), + ((!col1NotNULL || col2NotNULL) && col1NotNULL) -> (col2NotNULL && col1NotNULL), + ((col2NotNULL || !col1NotNULL) && col1NotNULL) -> (col2NotNULL && col1NotNULL), + + (col1NotNULL || (!col1NotNULL && col2NotNULL)) -> (col1NotNULL || col2NotNULL), + (col1NotNULL || (col2NotNULL && !col1NotNULL)) -> (col1NotNULL || col2NotNULL), + ((!col1NotNULL && col2NotNULL) || col1NotNULL) -> (col2NotNULL || col1NotNULL), + ((col2NotNULL && !col1NotNULL) || col1NotNULL) -> (col2NotNULL || col1NotNULL) + ) + + // check plans + for ((originalExpr, expectedExpr) <- exprs) { + assertEquivalent(originalExpr, expectedExpr) + } + + // check evaluation + val binaryBooleanValues = Seq(true, false) + for (col1NotNULLVal <- binaryBooleanValues; + col2NotNULLVal <- binaryBooleanValues; + (originalExpr, expectedExpr) <- exprs) { + val inputRow = create_row(col1NotNULLVal, col2NotNULLVal) + val optimizedVal = evaluateWithoutCodegen(expectedExpr, inputRow) + checkEvaluation(originalExpr, optimizedVal, inputRow) + } + } } 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 279b7b8d49f5..4a7bd2f5b065 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 @@ -30,6 +30,7 @@ import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.Uuid +import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation import org.apache.spark.sql.catalyst.plans.logical.{Filter, OneRowRelation, Union} import org.apache.spark.sql.execution.{FilterExec, QueryExecution, WholeStageCodegenExec} import org.apache.spark.sql.execution.aggregate.HashAggregateExec @@ -2579,4 +2580,13 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer(df.where("(NOT a) OR a"), Seq.empty) } + + test("SPARK-25714 Null handling in BooleanSimplification") { + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> ConvertToLocalRelation.ruleName) { + val df = Seq(("abc", 1), (null, 3)).toDF("col1", "col2") + checkAnswer( + df.filter("col1 = 'abc' OR (col1 != 'abc' AND col2 == 3)"), + Row ("abc", 1)) + } + } } From 765cbcace23da888a1ca6d0e76b8bc03f4a8e883 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 12 Oct 2018 23:03:06 -0700 Subject: [PATCH 159/879] [MINOR] Fix code comment in BooleanSimplification. --- .../org/apache/spark/sql/catalyst/optimizer/expressions.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 8459043b3651..3c4b284b307f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -276,7 +276,7 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { case a And b if a.semanticEquals(b) => a case a Or b if a.semanticEquals(b) => a - // The following optimization is applicable only when the operands are nullable, + // The following optimization is applicable only when the operands are not nullable, // since the three-value logic of AND and OR are different in NULL handling. // See the chart: // +---------+---------+---------+---------+ From 663481933833667086daf7f266fb86085dc65252 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Sat, 13 Oct 2018 14:49:38 +0800 Subject: [PATCH 160/879] [SPARK-25718][SQL] Detect recursive reference in Avro schema and throw exception ## What changes were proposed in this pull request? Avro schema allows recursive reference, e.g. the schema for linked-list in https://avro.apache.org/docs/1.8.2/spec.html#schema_record ``` { "type": "record", "name": "LongList", "aliases": ["LinkedLongs"], // old name for this "fields" : [ {"name": "value", "type": "long"}, // each element has a long {"name": "next", "type": ["null", "LongList"]} // optional next element ] } ``` In current Spark SQL, it is impossible to convert the schema as `StructType` . Run `SchemaConverters.toSqlType(avroSchema)` and we will get stack overflow exception. We should detect the recursive reference and throw exception for it. ## How was this patch tested? New unit test case. Closes #22709 from gengliangwang/avroRecursiveRef. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan (cherry picked from commit 2eaf0587883ac3c65e77d01ffbb39f64c6152f87) Signed-off-by: Wenchen Fan --- .../spark/sql/avro/SchemaConverters.scala | 26 ++++++-- .../org/apache/spark/sql/avro/AvroSuite.scala | 65 +++++++++++++++++++ 2 files changed, 84 insertions(+), 7 deletions(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala index bd1576587d7f..64127af73881 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala @@ -43,6 +43,10 @@ object SchemaConverters { * This function takes an avro schema and returns a sql schema. */ def toSqlType(avroSchema: Schema): SchemaType = { + toSqlTypeHelper(avroSchema, Set.empty) + } + + def toSqlTypeHelper(avroSchema: Schema, existingRecordNames: Set[String]): SchemaType = { avroSchema.getType match { case INT => avroSchema.getLogicalType match { case _: Date => SchemaType(DateType, nullable = false) @@ -67,21 +71,28 @@ object SchemaConverters { case ENUM => SchemaType(StringType, nullable = false) case RECORD => + if (existingRecordNames.contains(avroSchema.getFullName)) { + throw new IncompatibleSchemaException(s""" + |Found recursive reference in Avro schema, which can not be processed by Spark: + |${avroSchema.toString(true)} + """.stripMargin) + } + val newRecordNames = existingRecordNames + avroSchema.getFullName val fields = avroSchema.getFields.asScala.map { f => - val schemaType = toSqlType(f.schema()) + val schemaType = toSqlTypeHelper(f.schema(), newRecordNames) StructField(f.name, schemaType.dataType, schemaType.nullable) } SchemaType(StructType(fields), nullable = false) case ARRAY => - val schemaType = toSqlType(avroSchema.getElementType) + val schemaType = toSqlTypeHelper(avroSchema.getElementType, existingRecordNames) SchemaType( ArrayType(schemaType.dataType, containsNull = schemaType.nullable), nullable = false) case MAP => - val schemaType = toSqlType(avroSchema.getValueType) + val schemaType = toSqlTypeHelper(avroSchema.getValueType, existingRecordNames) SchemaType( MapType(StringType, schemaType.dataType, valueContainsNull = schemaType.nullable), nullable = false) @@ -91,13 +102,14 @@ object SchemaConverters { // In case of a union with null, eliminate it and make a recursive call val remainingUnionTypes = avroSchema.getTypes.asScala.filterNot(_.getType == NULL) if (remainingUnionTypes.size == 1) { - toSqlType(remainingUnionTypes.head).copy(nullable = true) + toSqlTypeHelper(remainingUnionTypes.head, existingRecordNames).copy(nullable = true) } else { - toSqlType(Schema.createUnion(remainingUnionTypes.asJava)).copy(nullable = true) + toSqlTypeHelper(Schema.createUnion(remainingUnionTypes.asJava), existingRecordNames) + .copy(nullable = true) } } else avroSchema.getTypes.asScala.map(_.getType) match { case Seq(t1) => - toSqlType(avroSchema.getTypes.get(0)) + toSqlTypeHelper(avroSchema.getTypes.get(0), existingRecordNames) case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) => SchemaType(LongType, nullable = false) case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) => @@ -107,7 +119,7 @@ object SchemaConverters { // This is consistent with the behavior when converting between Avro and Parquet. val fields = avroSchema.getTypes.asScala.zipWithIndex.map { case (s, i) => - val schemaType = toSqlType(s) + val schemaType = toSqlTypeHelper(s, existingRecordNames) // All fields are nullable because only one of them is set at a time StructField(s"member$i", schemaType.dataType, nullable = true) } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala index 9ad4388414ea..8b088b359eb9 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroSuite.scala @@ -1266,4 +1266,69 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { checkCodec(df, path, "xz") } } + + private def checkSchemaWithRecursiveLoop(avroSchema: String): Unit = { + val message = intercept[IncompatibleSchemaException] { + SchemaConverters.toSqlType(new Schema.Parser().parse(avroSchema)) + }.getMessage + + assert(message.contains("Found recursive reference in Avro schema")) + } + + test("Detect recursive loop") { + checkSchemaWithRecursiveLoop(""" + |{ + | "type": "record", + | "name": "LongList", + | "fields" : [ + | {"name": "value", "type": "long"}, // each element has a long + | {"name": "next", "type": ["null", "LongList"]} // optional next element + | ] + |} + """.stripMargin) + + checkSchemaWithRecursiveLoop(""" + |{ + | "type": "record", + | "name": "LongList", + | "fields": [ + | { + | "name": "value", + | "type": { + | "type": "record", + | "name": "foo", + | "fields": [ + | { + | "name": "parent", + | "type": "LongList" + | } + | ] + | } + | } + | ] + |} + """.stripMargin) + + checkSchemaWithRecursiveLoop(""" + |{ + | "type": "record", + | "name": "LongList", + | "fields" : [ + | {"name": "value", "type": "long"}, + | {"name": "array", "type": {"type": "array", "items": "LongList"}} + | ] + |} + """.stripMargin) + + checkSchemaWithRecursiveLoop(""" + |{ + | "type": "record", + | "name": "LongList", + | "fields" : [ + | {"name": "value", "type": "long"}, + | {"name": "map", "type": {"type": "map", "values": "LongList"}} + | ] + |} + """.stripMargin) + } } From c4efcf1ddde9478bf9d62f63aada3dcfce07c66c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 13 Oct 2018 16:43:10 -0700 Subject: [PATCH 161/879] [SPARK-25714][SQL][FOLLOWUP] improve the comment inside BooleanSimplification rule ## What changes were proposed in this pull request? improve the code comment added in https://github.com/apache/spark/pull/22702/files ## How was this patch tested? N/A Closes #22711 from cloud-fan/minor. Authored-by: Wenchen Fan Signed-off-by: gatorsmile (cherry picked from commit b73f76beb3c33feef0cb451726da50740ffed689) Signed-off-by: gatorsmile --- .../sql/catalyst/optimizer/expressions.scala | 24 ++++++++++++------- 1 file changed, 15 insertions(+), 9 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index 3c4b284b307f..468a950fb108 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -276,31 +276,37 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { case a And b if a.semanticEquals(b) => a case a Or b if a.semanticEquals(b) => a - // The following optimization is applicable only when the operands are not nullable, + // The following optimizations are applicable only when the operands are not nullable, // since the three-value logic of AND and OR are different in NULL handling. // See the chart: // +---------+---------+---------+---------+ - // | p | q | p OR q | p AND q | + // | operand | operand | OR | AND | // +---------+---------+---------+---------+ // | TRUE | TRUE | TRUE | TRUE | // | TRUE | FALSE | TRUE | FALSE | - // | TRUE | UNKNOWN | TRUE | UNKNOWN | - // | FALSE | TRUE | TRUE | FALSE | // | FALSE | FALSE | FALSE | FALSE | - // | FALSE | UNKNOWN | UNKNOWN | FALSE | // | UNKNOWN | TRUE | TRUE | UNKNOWN | // | UNKNOWN | FALSE | UNKNOWN | FALSE | // | UNKNOWN | UNKNOWN | UNKNOWN | UNKNOWN | // +---------+---------+---------+---------+ + + // (NULL And (NULL Or FALSE)) = NULL, but (NULL And FALSE) = FALSE. Thus, a can't be nullable. case a And (b Or c) if !a.nullable && Not(a).semanticEquals(b) => And(a, c) + // (NULL And (FALSE Or NULL)) = NULL, but (NULL And FALSE) = FALSE. Thus, a can't be nullable. case a And (b Or c) if !a.nullable && Not(a).semanticEquals(c) => And(a, b) - case (a Or b) And c if !a.nullable && a.semanticEquals(Not(c)) => And(b, c) - case (a Or b) And c if !b.nullable && b.semanticEquals(Not(c)) => And(a, c) + // ((NULL Or FALSE) And NULL) = NULL, but (FALSE And NULL) = FALSE. Thus, c can't be nullable. + case (a Or b) And c if !c.nullable && a.semanticEquals(Not(c)) => And(b, c) + // ((FALSE Or NULL) And NULL) = NULL, but (FALSE And NULL) = FALSE. Thus, c can't be nullable. + case (a Or b) And c if !c.nullable && b.semanticEquals(Not(c)) => And(a, c) + // (NULL Or (NULL And TRUE)) = NULL, but (NULL Or TRUE) = TRUE. Thus, a can't be nullable. case a Or (b And c) if !a.nullable && Not(a).semanticEquals(b) => Or(a, c) + // (NULL Or (TRUE And NULL)) = NULL, but (NULL Or TRUE) = TRUE. Thus, a can't be nullable. case a Or (b And c) if !a.nullable && Not(a).semanticEquals(c) => Or(a, b) - case (a And b) Or c if !a.nullable && a.semanticEquals(Not(c)) => Or(b, c) - case (a And b) Or c if !b.nullable && b.semanticEquals(Not(c)) => Or(a, c) + // ((NULL And TRUE) Or NULL) = NULL, but (TRUE Or NULL) = TRUE. Thus, c can't be nullable. + case (a And b) Or c if !c.nullable && a.semanticEquals(Not(c)) => Or(b, c) + // ((TRUE And NULL) Or NULL) = NULL, but (TRUE Or NULL) = TRUE. Thus, c can't be nullable. + case (a And b) Or c if !c.nullable && b.semanticEquals(Not(c)) => Or(a, c) // Common factor elimination for conjunction case and @ (left And right) => From 883ca3f990e7533938ab5b4d24b1b35f9da0768a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 13 Oct 2018 18:01:28 -0700 Subject: [PATCH 162/879] [SPARK-25726][SQL][TEST] Fix flaky test in SaveIntoDataSourceCommandSuite ## What changes were proposed in this pull request? [SPARK-22479](https://github.com/apache/spark/pull/19708/files#diff-5c22ac5160d3c9d81225c5dd86265d27R31) adds a test case which sometimes fails because the used password string `123` matches `41230802`. This PR aims to fix the flakiness. - https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/97343/consoleFull ```scala SaveIntoDataSourceCommandSuite: - simpleString is redacted *** FAILED *** "SaveIntoDataSourceCommand .org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider41230802, Map(password -> *********(redacted), url -> *********(redacted), driver -> mydriver), ErrorIfExists +- Range (0, 1, step=1, splits=Some(2)) " contained "123" (SaveIntoDataSourceCommandSuite.scala:42) ``` ## How was this patch tested? Pass the Jenkins with the updated test case Closes #22716 from dongjoon-hyun/SPARK-25726. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 6bbceb9fefe815d18001c6dd84f9ea2883d17a88) Signed-off-by: Dongjoon Hyun --- .../execution/datasources/SaveIntoDataSourceCommandSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala index a1da3ec43eae..8b06b175a210 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala @@ -25,7 +25,7 @@ class SaveIntoDataSourceCommandSuite extends SharedSQLContext { test("simpleString is redacted") { val URL = "connection.url" - val PASS = "123" + val PASS = "mypassword" val DRIVER = "mydriver" val dataSource = DataSource( From 3e776d73b87b8dd7cda603e409dbc4eb258748bb Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Sat, 13 Oct 2018 22:10:17 -0700 Subject: [PATCH 163/879] [SPARK-25727][SQL] Add outputOrdering to otherCopyArgs in InMemoryRelation ## What changes were proposed in this pull request? Add `outputOrdering ` to `otherCopyArgs` in InMemoryRelation so that this field will be copied when we doing the tree transformation. ``` val data = Seq(100).toDF("count").cache() data.queryExecution.optimizedPlan.toJSON ``` The above code can generate the following error: ``` assertion failed: InMemoryRelation fields: output, cacheBuilder, statsOfPlanToCache, outputOrdering, values: List(count#178), CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) Project [value#176 AS count#178] +- LocalTableScan [value#176] ,None), Statistics(sizeInBytes=12.0 B, hints=none) java.lang.AssertionError: assertion failed: InMemoryRelation fields: output, cacheBuilder, statsOfPlanToCache, outputOrdering, values: List(count#178), CachedRDDBuilder(true,10000,StorageLevel(disk, memory, deserialized, 1 replicas),*(1) Project [value#176 AS count#178] +- LocalTableScan [value#176] ,None), Statistics(sizeInBytes=12.0 B, hints=none) at scala.Predef$.assert(Predef.scala:170) at org.apache.spark.sql.catalyst.trees.TreeNode.jsonFields(TreeNode.scala:611) at org.apache.spark.sql.catalyst.trees.TreeNode.org$apache$spark$sql$catalyst$trees$TreeNode$$collectJsonValue$1(TreeNode.scala:599) at org.apache.spark.sql.catalyst.trees.TreeNode.jsonValue(TreeNode.scala:604) at org.apache.spark.sql.catalyst.trees.TreeNode.toJSON(TreeNode.scala:590) ``` ## How was this patch tested? Added a test Closes #22715 from gatorsmile/copyArgs1. Authored-by: gatorsmile Signed-off-by: Dongjoon Hyun (cherry picked from commit 6c3f2c6a6aa69f80de5504961cfd61b9a61ea7ce) Signed-off-by: Dongjoon Hyun --- .../spark/sql/execution/columnar/InMemoryRelation.scala | 2 +- .../sql/execution/columnar/InMemoryColumnarQuerySuite.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 1a8fbaca53f5..b752b771f43a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -206,7 +206,7 @@ case class InMemoryRelation( outputOrdering).asInstanceOf[this.type] } - override protected def otherCopyArgs: Seq[AnyRef] = Seq(statsOfPlanToCache) + override protected def otherCopyArgs: Seq[AnyRef] = Seq(statsOfPlanToCache, outputOrdering) override def simpleString: String = s"InMemoryRelation [${Utils.truncatedString(output, ", ")}], ${cacheBuilder.storageLevel}" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index efc2f20a907f..b1b23e443987 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -488,6 +488,12 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } } + test("SPARK-25727 - otherCopyArgs in InMemoryRelation does not include outputOrdering") { + val data = Seq(100).toDF("count").cache() + val json = data.queryExecution.optimizedPlan.toJSON + assert(json.contains("outputOrdering") && json.contains("statsOfPlanToCache")) + } + test("SPARK-22673: InMemoryRelation should utilize existing stats of the plan to be cached") { // This test case depends on the size of parquet in statistics. withSQLConf( From b6e4aca0be7f3b863c326063a3c02aa8a1c266a3 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Mon, 15 Oct 2018 10:46:10 +0800 Subject: [PATCH 164/879] [SPARK-25700][SQL][BRANCH-2.4] Partially revert append mode support in Data Source V2 ## What changes were proposed in this pull request? This PR proposes to partially revert https://github.com/apache/spark/commit/5fef6e3513d6023a837c427d183006d153c7102b so that it does make a readsupport and read schema when it writes in branch 2-4 since it's too breaking change. https://github.com/apache/spark/commit/5fef6e3513d6023a837c427d183006d153c7102b happened to create a readsupport in write path, which ended up with reading schema from readsupport at write path. For instance, this breaks `spark.range(1).format("source").write.save("non-existent-path")` case since there's no way to read the schema from "non-existent-path". See also https://github.com/apache/spark/pull/22009#discussion_r223982672 See also https://github.com/apache/spark/pull/22688 See also http://apache-spark-developers-list.1001551.n3.nabble.com/Possible-bug-in-DatasourceV2-td25343.html ## How was this patch tested? Unit test and manual tests. Closes #22697 from HyukjinKwon/append-revert-2.4. Authored-by: hyukjinkwon Signed-off-by: Wenchen Fan --- .../apache/spark/sql/DataFrameWriter.scala | 20 +++++--------- .../sql/sources/v2/DataSourceV2Suite.scala | 27 +++++++++++++++++++ .../sources/v2/SimpleWritableDataSource.scala | 5 ++-- 3 files changed, 35 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala index c1e2f496682d..b77dfd9ed9e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala @@ -250,22 +250,14 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { source, df.sparkSession.sessionState.conf) val options = sessionOptions ++ extraOptions - val relation = DataSourceV2Relation.create(source, options) - if (mode == SaveMode.Append) { - runCommand(df.sparkSession, "save") { - AppendData.byName(relation, df.logicalPlan) - } + val writer = ws.createWriter( + UUID.randomUUID.toString, df.logicalPlan.output.toStructType, mode, + new DataSourceOptions(options.asJava)) - } else { - val writer = ws.createWriter( - UUID.randomUUID.toString, df.logicalPlan.output.toStructType, mode, - new DataSourceOptions(options.asJava)) - - if (writer.isPresent) { - runCommand(df.sparkSession, "save") { - WriteToDataSourceV2(writer.get, df.logicalPlan) - } + if (writer.isPresent) { + runCommand(df.sparkSession, "save") { + WriteToDataSourceV2(writer.get, df.logicalPlan) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala index bafde50bdc01..2367bdd16952 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala @@ -355,6 +355,22 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { } } } + + test("SPARK-25700: do not read schema when writing") { + withTempPath { file => + val cls = classOf[SimpleWriteOnlyDataSource] + val path = file.getCanonicalPath + val df = spark.range(5).select('id as 'i, -'id as 'j) + try { + df.write.format(cls.getName).option("path", path).mode("error").save() + df.write.format(cls.getName).option("path", path).mode("overwrite").save() + df.write.format(cls.getName).option("path", path).mode("ignore").save() + df.write.format(cls.getName).option("path", path).mode("append").save() + } catch { + case e: SchemaReadAttemptException => fail("Schema read was attempted.", e) + } + } + } } class SimpleSinglePartitionSource extends DataSourceV2 with ReadSupport { @@ -594,3 +610,14 @@ class SpecificInputPartitionReader(i: Array[Int], j: Array[Int]) override def close(): Unit = {} } + +class SchemaReadAttemptException(m: String) extends RuntimeException(m) + +class SimpleWriteOnlyDataSource extends SimpleWritableDataSource { + override def fullSchema(): StructType = { + // This is a bit hacky since this source implements read support but throws + // during schema retrieval. Might have to rewrite but it's done + // such so for minimised changes. + throw new SchemaReadAttemptException("read is not supported") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala index 654c62d8edc5..4cf02595cd96 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala @@ -43,12 +43,12 @@ class SimpleWritableDataSource extends DataSourceV2 with WriteSupport with SessionConfigSupport { - private val schema = new StructType().add("i", "long").add("j", "long") + protected def fullSchema() = new StructType().add("i", "long").add("j", "long") override def keyPrefix: String = "simpleWritableDataSource" class Reader(path: String, conf: Configuration) extends DataSourceReader { - override def readSchema(): StructType = schema + override def readSchema(): StructType = SimpleWritableDataSource.this.fullSchema() override def planInputPartitions(): JList[InputPartition[InternalRow]] = { val dataPath = new Path(path) @@ -113,7 +113,6 @@ class SimpleWritableDataSource extends DataSourceV2 schema: StructType, mode: SaveMode, options: DataSourceOptions): Optional[DataSourceWriter] = { - assert(DataType.equalsStructurally(schema.asNullable, this.schema.asNullable)) assert(!SparkContext.getActive.get.conf.getBoolean("spark.speculation", false)) val path = new Path(options.get("path").get()) From d64b355886b2dd3486729d51bac1a17abba3f64a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Mon, 15 Oct 2018 18:34:30 -0700 Subject: [PATCH 165/879] [SPARK-25738][SQL] Fix LOAD DATA INPATH for hdfs port ## What changes were proposed in this pull request? LOAD DATA INPATH didn't work if the defaultFS included a port for hdfs. Handling this just requires a small change to use the correct URI constructor. ## How was this patch tested? Added a unit test, ran all tests via jenkins Closes #22733 from squito/SPARK-25738. Authored-by: Imran Rashid Signed-off-by: Marcelo Vanzin (cherry picked from commit fdaa99897ac8755938d031896ae0eefb46ce7107) Signed-off-by: Marcelo Vanzin --- .../apache/spark/sql/execution/command/tables.scala | 11 +++++++---- .../spark/sql/hive/execution/SQLQuerySuite.scala | 8 ++++++++ 2 files changed, 15 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 2eca1c40a5b3..64831e5089a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -306,7 +306,8 @@ case class LoadDataCommand( val loadPath = { if (isLocal) { val localFS = FileContext.getLocalFSFileContext() - makeQualified(FsConstants.LOCAL_FS_URI, localFS.getWorkingDirectory(), new Path(path)) + LoadDataCommand.makeQualified(FsConstants.LOCAL_FS_URI, localFS.getWorkingDirectory(), + new Path(path)) } else { val loadPath = new Path(path) // Follow Hive's behavior: @@ -323,7 +324,7 @@ case class LoadDataCommand( // by considering the wild card scenario in mind.as per old logic query param is // been considered while creating URI instance and if path contains wild card char '?' // the remaining charecters after '?' will be removed while forming URI instance - makeQualified(defaultFS, uriPath, loadPath) + LoadDataCommand.makeQualified(defaultFS, uriPath, loadPath) } } val fs = loadPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) @@ -363,7 +364,9 @@ case class LoadDataCommand( CommandUtils.updateTableStats(sparkSession, targetTable) Seq.empty[Row] } +} +object LoadDataCommand { /** * Returns a qualified path object. Method ported from org.apache.hadoop.fs.Path class. * @@ -372,7 +375,7 @@ case class LoadDataCommand( * @param path Path instance based on the path string specified by the user. * @return qualified path object */ - private def makeQualified(defaultUri: URI, workingDir: Path, path: Path): Path = { + private[sql] def makeQualified(defaultUri: URI, workingDir: Path, path: Path): Path = { val pathUri = if (path.isAbsolute()) path.toUri() else new Path(workingDir, path).toUri() if (pathUri.getScheme == null || pathUri.getAuthority == null && defaultUri.getAuthority != null) { @@ -383,7 +386,7 @@ case class LoadDataCommand( pathUri.getAuthority } try { - val newUri = new URI(scheme, authority, pathUri.getPath, pathUri.getFragment) + val newUri = new URI(scheme, authority, pathUri.getPath, null, pathUri.getFragment) new Path(newUri) } catch { case e: URISyntaxException => 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 e49aea267026..dfcde8cc0d39 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import java.io.File +import java.net.URI import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import java.util.{Locale, Set} @@ -32,6 +33,7 @@ import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, Functio import org.apache.spark.sql.catalyst.catalog.{CatalogTableType, CatalogUtils, HiveTableRelation} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} +import org.apache.spark.sql.execution.command.LoadDataCommand import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} @@ -1985,6 +1987,12 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("SPARK-25738: defaultFs can have a port") { + val defaultURI = new URI("hdfs://fizz.buzz.com:8020") + val r = LoadDataCommand.makeQualified(defaultURI, new Path("/foo/bar"), new Path("/flim/flam")) + assert(r === new Path("hdfs://fizz.buzz.com:8020/flim/flam")) + } + test("Insert overwrite with partition") { withTable("tableWithPartition") { sql( From 8bc7ab03dd417f0f9cca509df4566d621e633305 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Tue, 16 Oct 2018 08:58:29 +0800 Subject: [PATCH 166/879] [SPARK-25674][FOLLOW-UP] Update the stats for each ColumnarBatch ## What changes were proposed in this pull request? This PR is a follow-up of https://github.com/apache/spark/pull/22594 . This alternative can avoid the unneeded computation in the hot code path. - For row-based scan, we keep the original way. - For the columnar scan, we just need to update the stats after each batch. ## How was this patch tested? N/A Closes #22731 from gatorsmile/udpateStatsFileScanRDD. Authored-by: gatorsmile Signed-off-by: Wenchen Fan (cherry picked from commit 4cee191c04f14d7272347e4b29201763c6cfb6bf) Signed-off-by: Sean Owen --- .../sql/execution/datasources/FileScanRDD.scala | 15 ++++++++------- 1 file changed, 8 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index dd3c154259c7..ffea33c08ef9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -85,7 +85,7 @@ class FileScanRDD( // If we do a coalesce, however, we are likely to compute multiple partitions in the same // task and in the same thread, in which case we need to avoid override values written by // previous partitions (SPARK-13071). - private def updateBytesRead(): Unit = { + private def incTaskInputMetricsBytesRead(): Unit = { inputMetrics.setBytesRead(existingBytesRead + getBytesReadCallback()) } @@ -106,15 +106,16 @@ class FileScanRDD( // don't need to run this `if` for every record. val preNumRecordsRead = inputMetrics.recordsRead if (nextElement.isInstanceOf[ColumnarBatch]) { + incTaskInputMetricsBytesRead() inputMetrics.incRecordsRead(nextElement.asInstanceOf[ColumnarBatch].numRows()) } else { + // too costly to update every record + if (inputMetrics.recordsRead % + SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + incTaskInputMetricsBytesRead() + } inputMetrics.incRecordsRead(1) } - // The records may be incremented by more than 1 at a time. - if (preNumRecordsRead / SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS != - inputMetrics.recordsRead / SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS) { - updateBytesRead() - } nextElement } @@ -201,7 +202,7 @@ class FileScanRDD( } override def close(): Unit = { - updateBytesRead() + incTaskInputMetricsBytesRead() InputFileBlockHolder.unset() } } From 77156f8c81720ec7364b386a95ef1b30713fe55c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 16 Oct 2018 15:13:01 +0800 Subject: [PATCH 167/879] [SPARK-25736][SQL][TEST] add tests to verify the behavior of multi-column count ## What changes were proposed in this pull request? AFAIK multi-column count is not widely supported by the mainstream databases(postgres doesn't support), and the SQL standard doesn't define it clearly, as near as I can tell. Since Spark supports it, we should clearly document the current behavior and add tests to verify it. ## How was this patch tested? N/A Closes #22728 from cloud-fan/doc. Authored-by: Wenchen Fan Signed-off-by: hyukjinkwon (cherry picked from commit e028fd3aed9e5e4c478f307f0a467b54b73ff0d5) Signed-off-by: hyukjinkwon --- .../expressions/aggregate/Count.scala | 2 +- .../test/resources/sql-tests/inputs/count.sql | 27 +++++++++ .../resources/sql-tests/results/count.sql.out | 55 +++++++++++++++++++ 3 files changed, 83 insertions(+), 1 deletion(-) create mode 100644 sql/core/src/test/resources/sql-tests/inputs/count.sql create mode 100644 sql/core/src/test/resources/sql-tests/results/count.sql.out diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala index 40582d0abd76..8cab8e485699 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala @@ -52,7 +52,7 @@ abstract class CountLike extends DeclarativeAggregate { usage = """ _FUNC_(*) - Returns the total number of retrieved rows, including rows containing null. - _FUNC_(expr) - Returns the number of rows for which the supplied expression is non-null. + _FUNC_(expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are all non-null. _FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-null. """) diff --git a/sql/core/src/test/resources/sql-tests/inputs/count.sql b/sql/core/src/test/resources/sql-tests/inputs/count.sql new file mode 100644 index 000000000000..9f9ee4a873d4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/count.sql @@ -0,0 +1,27 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b); + +-- count with single expression +SELECT + count(*), count(1), count(null), count(a), count(b), count(a + b), count((a, b)) +FROM testData; + +-- distinct count with single expression +SELECT + count(DISTINCT 1), + count(DISTINCT null), + count(DISTINCT a), + count(DISTINCT b), + count(DISTINCT (a + b)), + count(DISTINCT (a, b)) +FROM testData; + +-- count with multiple expressions +SELECT count(a, b), count(b, a), count(testData.*) FROM testData; + +-- distinct count with multiple expressions +SELECT + count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*) +FROM testData; diff --git a/sql/core/src/test/resources/sql-tests/results/count.sql.out b/sql/core/src/test/resources/sql-tests/results/count.sql.out new file mode 100644 index 000000000000..b8a86d4c4459 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/count.sql.out @@ -0,0 +1,55 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT + count(*), count(1), count(null), count(a), count(b), count(a + b), count((a, b)) +FROM testData +-- !query 1 schema +struct +-- !query 1 output +7 7 0 5 5 4 7 + + +-- !query 2 +SELECT + count(DISTINCT 1), + count(DISTINCT null), + count(DISTINCT a), + count(DISTINCT b), + count(DISTINCT (a + b)), + count(DISTINCT (a, b)) +FROM testData +-- !query 2 schema +struct +-- !query 2 output +1 0 2 2 2 6 + + +-- !query 3 +SELECT count(a, b), count(b, a), count(testData.*) FROM testData +-- !query 3 schema +struct +-- !query 3 output +4 4 4 + + +-- !query 4 +SELECT + count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*) +FROM testData +-- !query 4 schema +struct +-- !query 4 output +3 3 3 3 From 144cb949d597e6cd0e662f2320e983cb6903ecfb Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 16 Oct 2018 20:30:23 +0800 Subject: [PATCH 168/879] [SPARK-25579][SQL] Use quoted attribute names if needed in pushed ORC predicates ## What changes were proposed in this pull request? This PR aims to fix an ORC performance regression at Spark 2.4.0 RCs from Spark 2.3.2. Currently, for column names with `.`, the pushed predicates are ignored. **Test Data** ```scala scala> val df = spark.range(Int.MaxValue).sample(0.2).toDF("col.with.dot") scala> df.write.mode("overwrite").orc("/tmp/orc") ``` **Spark 2.3.2** ```scala scala> spark.sql("set spark.sql.orc.impl=native") scala> spark.sql("set spark.sql.orc.filterPushdown=true") scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show) +------------+ |col.with.dot| +------------+ | 5| | 7| | 8| +------------+ Time taken: 1542 ms scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show) +------------+ |col.with.dot| +------------+ | 5| | 7| | 8| +------------+ Time taken: 152 ms ``` **Spark 2.4.0 RC3** ```scala scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show) +------------+ |col.with.dot| +------------+ | 5| | 7| | 8| +------------+ Time taken: 4074 ms scala> spark.time(spark.read.orc("/tmp/orc").where("`col.with.dot` < 10").show) +------------+ |col.with.dot| +------------+ | 5| | 7| | 8| +------------+ Time taken: 1771 ms ``` ## How was this patch tested? Pass the Jenkins with a newly added test case. Closes #22597 from dongjoon-hyun/SPARK-25579. Authored-by: Dongjoon Hyun Signed-off-by: hyukjinkwon (cherry picked from commit 2c664edc060a41340eb374fd44b5d32c3c06a15c) Signed-off-by: hyukjinkwon --- .../datasources/orc/OrcFilters.scala | 37 ++++++++++++++----- .../datasources/orc/OrcQuerySuite.scala | 28 ++++---------- .../execution/datasources/orc/OrcTest.scala | 10 +++++ 3 files changed, 46 insertions(+), 29 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala index dbafc468c6c4..5b93a60a80ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -67,6 +67,16 @@ private[sql] object OrcFilters { } } + // Since ORC 1.5.0 (ORC-323), we need to quote for column names with `.` characters + // in order to distinguish predicate pushdown for nested columns. + private def quoteAttributeNameIfNeeded(name: String) : String = { + if (!name.contains("`") && name.contains(".")) { + s"`$name`" + } else { + name + } + } + /** * Create ORC filter as a SearchArgument instance. */ @@ -178,38 +188,47 @@ private[sql] object OrcFilters { // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().equals(attribute, getType(attribute), castedValue).end()) + Some(builder.startAnd().equals(quotedName, getType(attribute), castedValue).end()) case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().nullSafeEquals(attribute, getType(attribute), castedValue).end()) + Some(builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end()) case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThan(attribute, getType(attribute), castedValue).end()) + Some(builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end()) case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThanEquals(attribute, getType(attribute), castedValue).end()) + Some(builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end()) case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThanEquals(attribute, getType(attribute), castedValue).end()) + Some(builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end()) case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThan(attribute, getType(attribute), castedValue).end()) + Some(builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end()) case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startAnd().isNull(attribute, getType(attribute)).end()) + val quotedName = quoteAttributeNameIfNeeded(attribute) + Some(builder.startAnd().isNull(quotedName, getType(attribute)).end()) case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startNot().isNull(attribute, getType(attribute)).end()) + val quotedName = quoteAttributeNameIfNeeded(attribute) + Some(builder.startNot().isNull(quotedName, getType(attribute)).end()) case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) - Some(builder.startAnd().in(attribute, getType(attribute), + Some(builder.startAnd().in(quotedName, getType(attribute), castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) case _ => None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala index e9dccbf2e261..998b7b31dcd6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcQuerySuite.scala @@ -445,16 +445,7 @@ abstract class OrcQueryTest extends OrcTest { test("Support for pushing down filters for decimal types") { withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { val data = (0 until 10).map(i => Tuple1(BigDecimal.valueOf(i))) - withTempPath { file => - // It needs to repartition data so that we can have several ORC files - // in order to skip stripes in ORC. - spark.createDataFrame(data).toDF("a").repartition(10) - .write.orc(file.getCanonicalPath) - val df = spark.read.orc(file.getCanonicalPath).where("a == 2") - val actual = stripSparkFilter(df).count() - - assert(actual < 10) - } + checkPredicatePushDown(spark.createDataFrame(data).toDF("a"), 10, "a == 2") } } @@ -465,16 +456,7 @@ abstract class OrcQueryTest extends OrcTest { val milliseconds = Timestamp.valueOf(timeString).getTime + i * 3600 Tuple1(new Timestamp(milliseconds)) } - withTempPath { file => - // It needs to repartition data so that we can have several ORC files - // in order to skip stripes in ORC. - spark.createDataFrame(data).toDF("a").repartition(10) - .write.orc(file.getCanonicalPath) - val df = spark.read.orc(file.getCanonicalPath).where(s"a == '$timeString'") - val actual = stripSparkFilter(df).count() - - assert(actual < 10) - } + checkPredicatePushDown(spark.createDataFrame(data).toDF("a"), 10, s"a == '$timeString'") } } @@ -674,6 +656,12 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { } } + test("SPARK-25579 ORC PPD should support column names with dot") { + withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { + checkPredicatePushDown(spark.range(10).toDF("col.dot"), 10, "`col.dot` == 2") + } + } + test("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") { withSQLConf(SQLConf.ORC_IMPLEMENTATION.key -> "hive") { val e = intercept[AnalysisException] { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala index 38b34a03e3e4..a35c536038c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcTest.scala @@ -106,4 +106,14 @@ abstract class OrcTest extends QueryTest with SQLTestUtils with BeforeAndAfterAl df: DataFrame, path: File): Unit = { df.write.mode(SaveMode.Overwrite).orc(path.getCanonicalPath) } + + protected def checkPredicatePushDown(df: DataFrame, numRows: Int, predicate: String): Unit = { + withTempPath { file => + // It needs to repartition data so that we can have several ORC files + // in order to skip stripes in ORC. + df.repartition(numRows).write.orc(file.getCanonicalPath) + val actual = stripSparkFilter(spark.read.orc(file.getCanonicalPath).where(predicate)).count() + assert(actual < numRows) + } + } } From 3591bd2293f49ac8023166597704ad1bd21dabe9 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E5=BD=AD=E7=81=BF00244106?= <00244106@zte.intra> Date: Wed, 17 Oct 2018 12:45:13 +0800 Subject: [PATCH 169/879] [SQL][CATALYST][MINOR] update some error comments MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? this PR correct some comment error: 1. change from "as low a possible" to "as low as possible" in RewriteDistinctAggregates.scala 2. delete redundant word “with” in HiveTableScanExec’s doExecute() method ## How was this patch tested? Existing unit tests. Closes #22694 from CarolinePeng/update_comment. Authored-by: 彭灿00244106 <00244106@zte.intra> Signed-off-by: hyukjinkwon (cherry picked from commit e9332f600eb4f275b3bff368863a68c2a4349182) Signed-off-by: hyukjinkwon --- .../sql/catalyst/optimizer/RewriteDistinctAggregates.scala | 4 ++-- .../apache/spark/sql/hive/execution/HiveTableScanExec.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala index 4448ace7105a..b9468007cac6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala @@ -95,7 +95,7 @@ import org.apache.spark.sql.types.IntegerType * * This rule duplicates the input data by two or more times (# distinct groups + an optional * non-distinct group). This will put quite a bit of memory pressure of the used aggregate and - * exchange operators. Keeping the number of distinct groups as low a possible should be priority, + * exchange operators. Keeping the number of distinct groups as low as possible should be priority, * we could improve this in the current rule by applying more advanced expression canonicalization * techniques. */ @@ -241,7 +241,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { groupByAttrs ++ distinctAggChildAttrs ++ Seq(gid) ++ regularAggChildAttrMap.map(_._2), a.child) - // Construct the first aggregate operator. This de-duplicates the all the children of + // Construct the first aggregate operator. This de-duplicates all the children of // distinct operators, and applies the regular aggregate operators. val firstAggregateGroupBy = groupByAttrs ++ distinctAggChildAttrs :+ gid val firstAggregate = Aggregate( diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index b3795b443040..92c6632ad786 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -182,7 +182,7 @@ case class HiveTableScanExec( protected override def doExecute(): RDD[InternalRow] = { // Using dummyCallSite, as getCallSite can turn out to be expensive with - // with multiple partitions. + // multiple partitions. val rdd = if (!relation.isPartitioned) { Utils.withDummyCallSite(sqlContext.sparkContext) { hadoopReader.makeRDDForTable(hiveQlTable) From 362103b951f9b7bdb816e2543edc410c9d66df86 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 17 Oct 2018 06:52:00 -0500 Subject: [PATCH 170/879] [SPARK-25754][DOC] Change CDN for MathJax ## What changes were proposed in this pull request? Currently when we open our doc site: https://spark.apache.org/docs/latest/index.html , there is one warning ![image](https://user-images.githubusercontent.com/1097932/47065926-2b757980-d217-11e8-868f-02ce73f513ae.png) This PR is to change the CDN as per the migration tips: https://www.mathjax.org/cdn-shutting-down/ This is very very trivial. But it would be good to follow the suggestion from MathJax team and remove the warning, in case one day the original CDN is no longer available. ## How was this patch tested? Manual check. Closes #22753 from gengliangwang/migrateMathJax. Authored-by: Gengliang Wang Signed-off-by: Sean Owen (cherry picked from commit 2ab4473bed44828cee5a47759b5c36fc81dd5d71) Signed-off-by: Sean Owen --- docs/_layouts/global.html | 3 ++- docs/js/api-docs.js | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 88d549c3f101..77edebeffdc2 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -184,7 +184,8 @@

    {{ page.title }}

    }); }; script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') + - 'cdn.mathjax.org/mathjax/latest/MathJax.js?config=TeX-AMS-MML_HTMLorMML'; + 'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' + + '?config=TeX-AMS-MML_HTMLorMML'; d.getElementsByTagName('head')[0].appendChild(script); }(document)); diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js index 13514e11b933..287fce61540e 100644 --- a/docs/js/api-docs.js +++ b/docs/js/api-docs.js @@ -58,6 +58,7 @@ $(document).ready(function() { }); }; script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') + - 'cdn.mathjax.org/mathjax/latest/MathJax.js?config=TeX-AMS-MML_HTMLorMML'; + 'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' + + '?config=TeX-AMS-MML_HTMLorMML'; document.getElementsByTagName('head')[0].appendChild(script); }); From b698bd46f09bf7121faaeee45843c32ec053b0c9 Mon Sep 17 00:00:00 2001 From: Vladimir Kuriatkov Date: Wed, 17 Oct 2018 22:13:05 +0800 Subject: [PATCH 171/879] [SPARK-21402][SQL] Fix java array of structs deserialization When deserializing values of ArrayType with struct elements in java beans, fields of structs get mixed up. I suggest using struct data types retrieved from resolved input data instead of inferring them from java beans. ## What changes were proposed in this pull request? MapObjects expression is used to map array elements to java beans. Struct type of elements is inferred from java bean structure and ends up with mixed up field order. I used UnresolvedMapObjects instead of MapObjects, which allows to provide element type for MapObjects during analysis based on the resolved input data, not on the java bean. ## How was this patch tested? Added a test case. Built complete project on travis. michalsenkyr cloud-fan marmbrus liancheng Closes #22708 from vofque/SPARK-21402. Lead-authored-by: Vladimir Kuriatkov Co-authored-by: Vladimir Kuriatkov Signed-off-by: Wenchen Fan (cherry picked from commit e5b8136f47a947356e74c8d4bf9d03139f455a2f) Signed-off-by: Wenchen Fan --- .../sql/catalyst/JavaTypeInference.scala | 3 +- .../spark/sql/JavaBeanWithArraySuite.java | 154 ++++++++++++++++++ .../test-data/with-array-fields.json | 3 + 3 files changed, 158 insertions(+), 2 deletions(-) create mode 100644 sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanWithArraySuite.java create mode 100644 sql/core/src/test/resources/test-data/with-array-fields.json diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 3ecc137c8cd7..7a226d72f597 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -271,10 +271,9 @@ object JavaTypeInference { case c if listType.isAssignableFrom(typeToken) => val et = elementType(typeToken) - MapObjects( + UnresolvedMapObjects( p => deserializerFor(et, Some(p)), getPath, - inferDataType(et)._1, customCollectionCls = Some(c)) case _ if mapType.isAssignableFrom(typeToken) => diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanWithArraySuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanWithArraySuite.java new file mode 100644 index 000000000000..70dd11067253 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanWithArraySuite.java @@ -0,0 +1,154 @@ +/* + * 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 test.org.apache.spark.sql; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Encoder; +import org.apache.spark.sql.Encoders; +import org.apache.spark.sql.test.TestSparkSession; + +public class JavaBeanWithArraySuite { + + private static final List RECORDS = new ArrayList<>(); + + static { + RECORDS.add(new Record(1, Arrays.asList(new Interval(111, 211), new Interval(121, 221)))); + RECORDS.add(new Record(2, Arrays.asList(new Interval(112, 212), new Interval(122, 222)))); + RECORDS.add(new Record(3, Arrays.asList(new Interval(113, 213), new Interval(123, 223)))); + } + + private TestSparkSession spark; + + @Before + public void setUp() { + spark = new TestSparkSession(); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + @Test + public void testBeanWithArrayFieldDeserialization() { + + Encoder encoder = Encoders.bean(Record.class); + + Dataset dataset = spark + .read() + .format("json") + .schema("id int, intervals array>") + .load("src/test/resources/test-data/with-array-fields.json") + .as(encoder); + + List records = dataset.collectAsList(); + Assert.assertEquals(records, RECORDS); + } + + public static class Record { + + private int id; + private List intervals; + + public Record() { } + + Record(int id, List intervals) { + this.id = id; + this.intervals = intervals; + } + + public int getId() { + return id; + } + + public void setId(int id) { + this.id = id; + } + + public List getIntervals() { + return intervals; + } + + public void setIntervals(List intervals) { + this.intervals = intervals; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof Record)) return false; + Record other = (Record) obj; + return (other.id == this.id) && other.intervals.equals(this.intervals); + } + + @Override + public String toString() { + return String.format("{ id: %d, intervals: %s }", id, intervals); + } + } + + public static class Interval { + + private long startTime; + private long endTime; + + public Interval() { } + + Interval(long startTime, long endTime) { + this.startTime = startTime; + this.endTime = endTime; + } + + public long getStartTime() { + return startTime; + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + } + + public long getEndTime() { + return endTime; + } + + public void setEndTime(long endTime) { + this.endTime = endTime; + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof Interval)) return false; + Interval other = (Interval) obj; + return (other.startTime == this.startTime) && (other.endTime == this.endTime); + } + + @Override + public String toString() { + return String.format("[%d,%d]", startTime, endTime); + } + } +} diff --git a/sql/core/src/test/resources/test-data/with-array-fields.json b/sql/core/src/test/resources/test-data/with-array-fields.json new file mode 100644 index 000000000000..ff3674af2fbc --- /dev/null +++ b/sql/core/src/test/resources/test-data/with-array-fields.json @@ -0,0 +1,3 @@ +{ "id": 1, "intervals": [{ "startTime": 111, "endTime": 211 }, { "startTime": 121, "endTime": 221 }]} +{ "id": 2, "intervals": [{ "startTime": 112, "endTime": 212 }, { "startTime": 122, "endTime": 222 }]} +{ "id": 3, "intervals": [{ "startTime": 113, "endTime": 213 }, { "startTime": 123, "endTime": 223 }]} \ No newline at end of file From ac9a6f08a19e1896cb6aa123b567cc5c2ffe9067 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 17 Oct 2018 09:51:41 -0500 Subject: [PATCH 172/879] [SPARK-25741][WEBUI] Long URLs are not rendered properly in web UI ## What changes were proposed in this pull request? When the URL for description column in the table of job/stage page is long, WebUI doesn't render it properly. ![beforefix](https://user-images.githubusercontent.com/1097932/47009242-9323ba00-d16e-11e8-8262-0848d814442a.jpeg) Both job and stage page are using the class `name-link` for the description URL, so change the style of `a.name-link` to fix it. ## How was this patch tested? Manual test on my local: ![afterfix](https://user-images.githubusercontent.com/1097932/47009269-a46cc680-d16e-11e8-9ff5-0318a20db634.jpeg) Closes #22744 from gengliangwang/fixUILink. Authored-by: Gengliang Wang Signed-off-by: Sean Owen (cherry picked from commit 1901f06211661c19d70f231db235cca3cdb1f2dd) Signed-off-by: Sean Owen --- core/src/main/resources/org/apache/spark/ui/static/webui.css | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 4b060b0f4e53..266eeec55576 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -88,6 +88,10 @@ a.kill-link { float: right; } +a.name-link { + word-wrap: break-word; +} + span.expand-details { font-size: 10pt; cursor: pointer; From 71a6a9ce8558913bc410918c14b6799be9baaeb3 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Thu, 18 Oct 2018 10:32:25 -0700 Subject: [PATCH 173/879] [SPARK-25758][ML] Deprecate computeCost on BisectingKMeans ## What changes were proposed in this pull request? The PR proposes to deprecate the `computeCost` method on `BisectingKMeans` in favor of the adoption of `ClusteringEvaluator` in order to evaluate the clustering. ## How was this patch tested? NA Closes #22756 from mgaido91/SPARK-25758. Authored-by: Marco Gaido Signed-off-by: Dongjoon Hyun (cherry picked from commit c2962546d9a5900a5628a31b83d2c4b22c3a7936) Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/ml/clustering/BisectingKMeans.scala | 5 +++++ python/pyspark/ml/clustering.py | 6 ++++++ 2 files changed, 11 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 5cb16cc76588..2243d99364af 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -125,8 +125,13 @@ class BisectingKMeansModel private[ml] ( /** * Computes the sum of squared distances between the input points and their corresponding cluster * centers. + * + * @deprecated This method is deprecated and will be removed in 3.0.0. Use ClusteringEvaluator + * instead. You can also get the cost on the training dataset in the summary. */ @Since("2.0.0") + @deprecated("This method is deprecated and will be removed in 3.0.0. Use ClusteringEvaluator " + + "instead. You can also get the cost on the training dataset in the summary.", "2.4.0") def computeCost(dataset: Dataset[_]): Double = { SchemaUtils.validateVectorCompatibleColumn(dataset.schema, getFeaturesCol) val data = DatasetUtils.columnToOldVector(dataset, getFeaturesCol) diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 5ef4e765ea4e..11eb12405d1b 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -540,7 +540,13 @@ def computeCost(self, dataset): """ Computes the sum of squared distances between the input points and their corresponding cluster centers. + + ..note:: Deprecated in 2.4.0. It will be removed in 3.0.0. Use ClusteringEvaluator instead. + You can also get the cost on the training dataset in the summary. """ + warnings.warn("Deprecated in 2.4.0. It will be removed in 3.0.0. Use ClusteringEvaluator " + "instead. You can also get the cost on the training dataset in the summary.", + DeprecationWarning) return self._call_java("computeCost", dataset) @property From 71535516419831242fa7fc9177e8f5fdd3c6112b Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Thu, 18 Oct 2018 11:59:06 -0700 Subject: [PATCH 174/879] [SPARK-24499][SQL][DOC] Split the page of sql-programming-guide.html to multiple separate pages 1. Split the main page of sql-programming-guide into 7 parts: - Getting Started - Data Sources - Performance Turing - Distributed SQL Engine - PySpark Usage Guide for Pandas with Apache Arrow - Migration Guide - Reference 2. Add left menu for sql-programming-guide, keep first level index for each part in the menu. ![image](https://user-images.githubusercontent.com/4833765/47016859-6332e180-d183-11e8-92e8-ce62518a83c4.png) Local test with jekyll build/serve. Closes #22746 from xuanyuanking/SPARK-24499. Authored-by: Yuanjian Li Signed-off-by: gatorsmile (cherry picked from commit 987f386588de7311b066cf0f62f0eed64d4aa7d7) Signed-off-by: gatorsmile --- docs/_data/menu-sql.yaml | 81 + docs/_includes/nav-left-wrapper-sql.html | 6 + docs/_includes/nav-left.html | 3 +- docs/_layouts/global.html | 8 +- docs/ml-pipeline.md | 2 +- docs/sparkr.md | 6 +- ...urce-guide.md => sql-data-sources-avro.md} | 0 docs/sql-data-sources-hive-tables.md | 166 + docs/sql-data-sources-jdbc.md | 223 ++ docs/sql-data-sources-json.md | 81 + docs/sql-data-sources-load-save-functions.md | 283 ++ docs/sql-data-sources-orc.md | 26 + docs/sql-data-sources-parquet.md | 321 ++ docs/sql-data-sources-troubleshooting.md | 9 + docs/sql-data-sources.md | 42 + docs/sql-distributed-sql-engine.md | 84 + docs/sql-getting-started.md | 369 ++ .../sql-migration-guide-hive-compatibility.md | 137 + docs/sql-migration-guide-upgrade.md | 516 +++ docs/sql-migration-guide.md | 23 + docs/sql-performance-turing.md | 151 + docs/sql-programming-guide.md | 3119 +---------------- docs/sql-pyspark-pandas-with-arrow.md | 166 + docs/sql-reference.md | 641 ++++ .../structured-streaming-programming-guide.md | 2 +- 25 files changed, 3343 insertions(+), 3122 deletions(-) create mode 100644 docs/_data/menu-sql.yaml create mode 100644 docs/_includes/nav-left-wrapper-sql.html rename docs/{avro-data-source-guide.md => sql-data-sources-avro.md} (100%) create mode 100644 docs/sql-data-sources-hive-tables.md create mode 100644 docs/sql-data-sources-jdbc.md create mode 100644 docs/sql-data-sources-json.md create mode 100644 docs/sql-data-sources-load-save-functions.md create mode 100644 docs/sql-data-sources-orc.md create mode 100644 docs/sql-data-sources-parquet.md create mode 100644 docs/sql-data-sources-troubleshooting.md create mode 100644 docs/sql-data-sources.md create mode 100644 docs/sql-distributed-sql-engine.md create mode 100644 docs/sql-getting-started.md create mode 100644 docs/sql-migration-guide-hive-compatibility.md create mode 100644 docs/sql-migration-guide-upgrade.md create mode 100644 docs/sql-migration-guide.md create mode 100644 docs/sql-performance-turing.md create mode 100644 docs/sql-pyspark-pandas-with-arrow.md create mode 100644 docs/sql-reference.md diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml new file mode 100644 index 000000000000..671876323269 --- /dev/null +++ b/docs/_data/menu-sql.yaml @@ -0,0 +1,81 @@ +- text: Getting Started + url: sql-getting-started.html + subitems: + - text: "Starting Point: SparkSession" + url: sql-getting-started.html#starting-point-sparksession + - text: Creating DataFrames + url: sql-getting-started.html#creating-dataframes + - text: Untyped Dataset Operations (DataFrame operations) + url: sql-getting-started.html#untyped-dataset-operations-aka-dataframe-operations + - text: Running SQL Queries Programmatically + url: sql-getting-started.html#running-sql-queries-programmatically + - text: Global Temporary View + url: sql-getting-started.html#global-temporary-view + - text: Creating Datasets + url: sql-getting-started.html#creating-datasets + - text: Interoperating with RDDs + url: sql-getting-started.html#interoperating-with-rdds + - text: Aggregations + url: sql-getting-started.html#aggregations +- text: Data Sources + url: sql-data-sources.html + subitems: + - text: "Generic Load/Save Functions" + url: sql-data-sources-load-save-functions.html + - text: Parquet Files + url: sql-data-sources-parquet.html + - text: ORC Files + url: sql-data-sources-orc.html + - text: JSON Files + url: sql-data-sources-json.html + - text: Hive Tables + url: sql-data-sources-hive-tables.html + - text: JDBC To Other Databases + url: sql-data-sources-jdbc.html + - text: Avro Files + url: sql-data-sources-avro.html + - text: Troubleshooting + url: sql-data-sources-troubleshooting.html +- text: Performance Turing + url: sql-performance-turing.html + subitems: + - text: Caching Data In Memory + url: sql-performance-turing.html#caching-data-in-memory + - text: Other Configuration Options + url: sql-performance-turing.html#other-configuration-options + - text: Broadcast Hint for SQL Queries + url: sql-performance-turing.html#broadcast-hint-for-sql-queries +- text: Distributed SQL Engine + url: sql-distributed-sql-engine.html + subitems: + - text: "Running the Thrift JDBC/ODBC server" + url: sql-distributed-sql-engine.html#running-the-thrift-jdbcodbc-server + - text: Running the Spark SQL CLI + url: sql-distributed-sql-engine.html#running-the-spark-sql-cli +- text: PySpark Usage Guide for Pandas with Apache Arrow + url: sql-pyspark-pandas-with-arrow.html + subitems: + - text: Apache Arrow in Spark + url: sql-pyspark-pandas-with-arrow.html#apache-arrow-in-spark + - text: "Enabling for Conversion to/from Pandas" + url: sql-pyspark-pandas-with-arrow.html#enabling-for-conversion-tofrom-pandas + - text: "Pandas UDFs (a.k.a. Vectorized UDFs)" + url: sql-pyspark-pandas-with-arrow.html#pandas-udfs-aka-vectorized-udfs + - text: Usage Notes + url: sql-pyspark-pandas-with-arrow.html#usage-notes +- text: Migration Guide + url: sql-migration-guide.html + subitems: + - text: Spark SQL Upgrading Guide + url: sql-migration-guide-upgrade.html + - text: Compatibility with Apache Hive + url: sql-migration-guide-hive-compatibility.html +- text: Reference + url: sql-reference.html + subitems: + - text: Data Types + url: sql-reference.html#data-types + - text: NaN Semantics + url: sql-reference.html#nan-semantics + - text: Arithmetic operations + url: sql-reference.html#arithmetic-operations diff --git a/docs/_includes/nav-left-wrapper-sql.html b/docs/_includes/nav-left-wrapper-sql.html new file mode 100644 index 000000000000..edc4cf4514d0 --- /dev/null +++ b/docs/_includes/nav-left-wrapper-sql.html @@ -0,0 +1,6 @@ +
    +
    +

    Spark SQL Guide

    + {% include nav-left.html nav=include.nav-sql %} +
    +
    \ No newline at end of file diff --git a/docs/_includes/nav-left.html b/docs/_includes/nav-left.html index 73176f413255..19d68fd19163 100644 --- a/docs/_includes/nav-left.html +++ b/docs/_includes/nav-left.html @@ -10,7 +10,8 @@ {% endif %} - {% if item.subitems and navurl contains item.url %} + {% assign tag = item.url | remove: ".html" %} + {% if item.subitems and navurl contains tag %} {% include nav-left.html nav=item.subitems %} {% endif %} {% endfor %} diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 77edebeffdc2..cbe4306799d4 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -126,8 +126,12 @@
    - {% if page.url contains "/ml" %} - {% include nav-left-wrapper-ml.html nav-mllib=site.data.menu-mllib nav-ml=site.data.menu-ml %} + {% if page.url contains "/ml" or page.url contains "/sql" %} + {% if page.url contains "/ml" %} + {% include nav-left-wrapper-ml.html nav-mllib=site.data.menu-mllib nav-ml=site.data.menu-ml %} + {% else %} + {% include nav-left-wrapper-sql.html nav-sql=site.data.menu-sql %} + {% endif %}
    diff --git a/docs/ml-pipeline.md b/docs/ml-pipeline.md index e22e9003c30f..8c01ccb94c75 100644 --- a/docs/ml-pipeline.md +++ b/docs/ml-pipeline.md @@ -57,7 +57,7 @@ E.g., a learning algorithm is an `Estimator` which trains on a `DataFrame` and p Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data. This API adopts the `DataFrame` from Spark SQL in order to support a variety of data types. -`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#data-types) for a list of supported types. +`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-reference.html#data-types) for a list of supported types. In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [`Vector`](mllib-data-types.html#local-vector) types. A `DataFrame` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. diff --git a/docs/sparkr.md b/docs/sparkr.md index b4248e8bb21d..d9963b751f37 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -104,7 +104,7 @@ The following Spark driver properties can be set in `sparkConfig` with `sparkR.s
    ## Creating SparkDataFrames -With a `SparkSession`, applications can create `SparkDataFrame`s from a local R data frame, from a [Hive table](sql-programming-guide.html#hive-tables), or from other [data sources](sql-programming-guide.html#data-sources). +With a `SparkSession`, applications can create `SparkDataFrame`s from a local R data frame, from a [Hive table](sql-data-sources-hive-tables.html), or from other [data sources](sql-data-sources.html). ### From local data frames The simplest way to create a data frame is to convert a local R data frame into a SparkDataFrame. Specifically, we can use `as.DataFrame` or `createDataFrame` and pass in the local R data frame to create a SparkDataFrame. As an example, the following creates a `SparkDataFrame` based using the `faithful` dataset from R. @@ -125,7 +125,7 @@ head(df) ### From Data Sources -SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. +SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-data-sources-load-save-functions.html#manually-specifying-options) that are available for the built-in data sources. The general method for creating SparkDataFrames from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active SparkSession will be used automatically. SparkR supports reading JSON, CSV and Parquet files natively, and through packages available from sources like [Third Party Projects](https://spark.apache.org/third-party-projects.html), you can find data source connectors for popular file formats like Avro. These packages can either be added by @@ -180,7 +180,7 @@ write.df(people, path = "people.parquet", source = "parquet", mode = "overwrite" ### From Hive tables -You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details can be found in the [SQL programming guide](sql-programming-guide.html#starting-point-sparksession). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`). +You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details can be found in the [SQL programming guide](sql-getting-started.html#starting-point-sparksession). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`).
    {% highlight r %} diff --git a/docs/avro-data-source-guide.md b/docs/sql-data-sources-avro.md similarity index 100% rename from docs/avro-data-source-guide.md rename to docs/sql-data-sources-avro.md diff --git a/docs/sql-data-sources-hive-tables.md b/docs/sql-data-sources-hive-tables.md new file mode 100644 index 000000000000..687e6f8e0a7c --- /dev/null +++ b/docs/sql-data-sources-hive-tables.md @@ -0,0 +1,166 @@ +--- +layout: global +title: Hive Tables +displayTitle: Hive Tables +--- + +* Table of contents +{:toc} + +Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). +However, since Hive has a large number of dependencies, these dependencies are not included in the +default Spark distribution. If Hive dependencies can be found on the classpath, Spark will load them +automatically. Note that these Hive dependencies must also be present on all of the worker nodes, as +they will need access to the Hive serialization and deserialization libraries (SerDes) in order to +access data stored in Hive. + +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (for security configuration), +and `hdfs-site.xml` (for HDFS configuration) file in `conf/`. + +When working with Hive, one must instantiate `SparkSession` with Hive support, including +connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. +Users who do not have an existing Hive deployment can still enable Hive support. When not configured +by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and +creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory +`spark-warehouse` in the current directory that the Spark application is started. Note that +the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. +Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. +You may need to grant write privilege to the user who starts the Spark application. + +
    + +
    +{% include_example spark_hive scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %} +
    + +
    +{% include_example spark_hive java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java %} +
    + +
    +{% include_example spark_hive python/sql/hive.py %} +
    + +
    + +When working with Hive one must instantiate `SparkSession` with Hive support. This +adds support for finding tables in the MetaStore and writing queries using HiveQL. + +{% include_example spark_hive r/RSparkSQLExample.R %} + +
    +
    + +### Specifying storage format for Hive tables + +When you create a Hive table, you need to define how this table should read/write data from/to file system, +i.e. the "input format" and "output format". You also need to define how this table should deserialize the data +to rows, or serialize rows to data, i.e. the "serde". The following options can be used to specify the storage +format("serde", "input format", "output format"), e.g. `CREATE TABLE src(id int) USING hive OPTIONS(fileFormat 'parquet')`. +By default, we will read the table files as plain text. Note that, Hive storage handler is not supported yet when +creating table, you can create a table using storage handler at Hive side, and use Spark SQL to read it. + + + + + + + + + + + + + + + + + + + + + + +
    Property NameMeaning
    fileFormat + A fileFormat is kind of a package of storage format specifications, including "serde", "input format" and + "output format". Currently we support 6 fileFormats: 'sequencefile', 'rcfile', 'orc', 'parquet', 'textfile' and 'avro'. +
    inputFormat, outputFormat + These 2 options specify the name of a corresponding `InputFormat` and `OutputFormat` class as a string literal, + e.g. `org.apache.hadoop.hive.ql.io.orc.OrcInputFormat`. These 2 options must be appeared in pair, and you can not + specify them if you already specified the `fileFormat` option. +
    serde + This option specifies the name of a serde class. When the `fileFormat` option is specified, do not specify this option + if the given `fileFormat` already include the information of serde. Currently "sequencefile", "textfile" and "rcfile" + don't include the serde information and you can use this option with these 3 fileFormats. +
    fieldDelim, escapeDelim, collectionDelim, mapkeyDelim, lineDelim + These options can only be used with "textfile" fileFormat. They define how to read delimited files into rows. +
    + +All other properties defined with `OPTIONS` will be regarded as Hive serde properties. + +### Interacting with Different Versions of Hive Metastore + +One of the most important pieces of Spark SQL's Hive support is interaction with Hive metastore, +which enables Spark SQL to access metadata of Hive tables. Starting from Spark 1.4.0, a single binary +build of Spark SQL can be used to query different versions of Hive metastores, using the configuration described below. +Note that independent of the version of Hive that is being used to talk to the metastore, internally Spark SQL +will compile against Hive 1.2.1 and use those classes for internal execution (serdes, UDFs, UDAFs, etc). + +The following options can be used to configure the version of Hive that is used to retrieve metadata: + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.hive.metastore.version1.2.1 + Version of the Hive metastore. Available + options are 0.12.0 through 2.3.3. +
    spark.sql.hive.metastore.jarsbuiltin + Location of the jars that should be used to instantiate the HiveMetastoreClient. This + property can be one of three options: +
      +
    1. builtin
    2. + Use Hive 1.2.1, which is bundled with the Spark assembly when -Phive is + enabled. When this option is chosen, spark.sql.hive.metastore.version must be + either 1.2.1 or not defined. +
    3. maven
    4. + Use Hive jars of specified version downloaded from Maven repositories. This configuration + is not generally recommended for production deployments. +
    5. A classpath in the standard format for the JVM. This classpath must include all of Hive + and its dependencies, including the correct version of Hadoop. These jars only need to be + present on the driver, but if you are running in yarn cluster mode then you must ensure + they are packaged with your application.
    6. +
    +
    spark.sql.hive.metastore.sharedPrefixescom.mysql.jdbc,
    org.postgresql,
    com.microsoft.sqlserver,
    oracle.jdbc
    +

    + A comma-separated list of class prefixes that should be loaded using the classloader that is + shared between Spark SQL and a specific version of Hive. An example of classes that should + be shared is JDBC drivers that are needed to talk to the metastore. Other classes that need + to be shared are those that interact with classes that are already shared. For example, + custom appenders that are used by log4j. +

    +
    spark.sql.hive.metastore.barrierPrefixes(empty) +

    + A comma separated list of class prefixes that should explicitly be reloaded for each version + of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a + prefix that typically would be shared (i.e. org.apache.spark.*). +

    +
    diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md new file mode 100644 index 000000000000..057e8217241a --- /dev/null +++ b/docs/sql-data-sources-jdbc.md @@ -0,0 +1,223 @@ +--- +layout: global +title: JDBC To Other Databases +displayTitle: JDBC To Other Databases +--- + +* Table of contents +{:toc} + +Spark SQL also includes a data source that can read data from other databases using JDBC. This +functionality should be preferred over using [JdbcRDD](api/scala/index.html#org.apache.spark.rdd.JdbcRDD). +This is because the results are returned +as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources. +The JDBC data source is also easier to use from Java or Python as it does not require the user to +provide a ClassTag. +(Note that this is different than the Spark SQL JDBC server, which allows other applications to +run queries using Spark SQL). + +To get started you will need to include the JDBC driver for your particular database on the +spark classpath. For example, to connect to postgres from the Spark Shell you would run the +following command: + +{% highlight bash %} +bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jar +{% endhighlight %} + +Tables from the remote database can be loaded as a DataFrame or Spark SQL temporary view using +the Data Sources API. Users can specify the JDBC connection properties in the data source options. +user and password are normally provided as connection properties for +logging into the data sources. In addition to the connection properties, Spark also supports +the following case-insensitive options: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameMeaning
    url + The JDBC URL to connect to. The source-specific connection properties may be specified in the URL. e.g., jdbc:postgresql://localhost/test?user=fred&password=secret +
    dbtable + The JDBC table that should be read from or written into. Note that when using it in the read + path anything that is valid in a FROM clause of a SQL query can be used. + For example, instead of a full table you could also use a subquery in parentheses. It is not + allowed to specify `dbtable` and `query` options at the same time. +
    query + A query that will be used to read data into Spark. The specified query will be parenthesized and used + as a subquery in the FROM clause. Spark will also assign an alias to the subquery clause. + As an example, spark will issue a query of the following form to the JDBC Source.

    + SELECT <columns> FROM (<user_specified_query>) spark_gen_alias

    + Below are couple of restrictions while using this option.
    +
      +
    1. It is not allowed to specify `dbtable` and `query` options at the same time.
    2. +
    3. It is not allowed to specify `query` and `partitionColumn` options at the same time. When specifying + `partitionColumn` option is required, the subquery can be specified using `dbtable` option instead and + partition columns can be qualified using the subquery alias provided as part of `dbtable`.
      + Example:
      + + spark.read.format("jdbc")
      +    .option("dbtable", "(select c1, c2 from t1) as subq")
      +    .option("partitionColumn", "subq.c1"
      +    .load() +
    4. +
    +
    driver + The class name of the JDBC driver to use to connect to this URL. +
    partitionColumn, lowerBound, upperBound + These options must all be specified if any of them is specified. In addition, + numPartitions must be specified. They describe how to partition the table when + reading in parallel from multiple workers. + partitionColumn must be a numeric, date, or timestamp column from the table in question. + Notice that lowerBound and upperBound are just used to decide the + partition stride, not for filtering the rows in table. So all rows in the table will be + partitioned and returned. This option applies only to reading. +
    numPartitions + The maximum number of partitions that can be used for parallelism in table reading and + writing. This also determines the maximum number of concurrent JDBC connections. + If the number of partitions to write exceeds this limit, we decrease it to this limit by + calling coalesce(numPartitions) before writing. +
    queryTimeout + The number of seconds the driver will wait for a Statement object to execute to the given + number of seconds. Zero means there is no limit. In the write path, this option depends on + how JDBC drivers implement the API setQueryTimeout, e.g., the h2 JDBC driver + checks the timeout of each query instead of an entire JDBC batch. + It defaults to 0. +
    fetchsize + The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). This option applies only to reading. +
    batchsize + The JDBC batch size, which determines how many rows to insert per round trip. This can help performance on JDBC drivers. This option applies only to writing. It defaults to 1000. +
    isolationLevel + The transaction isolation level, which applies to current connection. It can be one of NONE, READ_COMMITTED, READ_UNCOMMITTED, REPEATABLE_READ, or SERIALIZABLE, corresponding to standard transaction isolation levels defined by JDBC's Connection object, with default of READ_UNCOMMITTED. This option applies only to writing. Please refer the documentation in java.sql.Connection. +
    sessionInitStatement + After each database session is opened to the remote DB and before starting to read data, this option executes a custom SQL statement (or a PL/SQL block). Use this to implement session initialization code. Example: option("sessionInitStatement", """BEGIN execute immediate 'alter session set "_serial_direct_read"=true'; END;""") +
    truncate + This is a JDBC writer related option. When SaveMode.Overwrite is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g., indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to false. This option applies only to writing. +
    cascadeTruncate + This is a JDBC writer related option. If enabled and supported by the JDBC database (PostgreSQL and Oracle at the moment), this options allows execution of a TRUNCATE TABLE t CASCADE (in the case of PostgreSQL a TRUNCATE TABLE ONLY t CASCADE is executed to prevent inadvertently truncating descendant tables). This will affect other tables, and thus should be used with care. This option applies only to writing. It defaults to the default cascading truncate behaviour of the JDBC database in question, specified in the isCascadeTruncate in each JDBCDialect. +
    createTableOptions + This is a JDBC writer related option. If specified, this option allows setting of database-specific table and partition options when creating a table (e.g., CREATE TABLE t (name string) ENGINE=InnoDB.). This option applies only to writing. +
    createTableColumnTypes + The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: "name CHAR(64), comments VARCHAR(1024)"). The specified types should be valid spark sql data types. This option applies only to writing. +
    customSchema + The custom schema to use for reading data from JDBC connectors. For example, "id DECIMAL(38, 0), name STRING". You can also specify partial fields, and the others use the default type mapping. For example, "id DECIMAL(38, 0)". The column names should be identical to the corresponding column names of JDBC table. Users can specify the corresponding data types of Spark SQL instead of using the defaults. This option applies only to reading. +
    pushDownPredicate + The option to enable or disable predicate push-down into the JDBC data source. The default value is true, in which case Spark will push down filters to the JDBC data source as much as possible. Otherwise, if set to false, no filter will be pushed down to the JDBC data source and thus all filters will be handled by Spark. Predicate push-down is usually turned off when the predicate filtering is performed faster by Spark than by the JDBC data source. +
    + +
    + +
    +{% include_example jdbc_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example jdbc_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example jdbc_dataset python/sql/datasource.py %} +
    + +
    +{% include_example jdbc_dataset r/RSparkSQLExample.R %} +
    + +
    + +{% highlight sql %} + +CREATE TEMPORARY VIEW jdbcTable +USING org.apache.spark.sql.jdbc +OPTIONS ( + url "jdbc:postgresql:dbserver", + dbtable "schema.tablename", + user 'username', + password 'password' +) + +INSERT INTO TABLE jdbcTable +SELECT * FROM resultTable +{% endhighlight %} + +
    +
    diff --git a/docs/sql-data-sources-json.md b/docs/sql-data-sources-json.md new file mode 100644 index 000000000000..f84336b5716d --- /dev/null +++ b/docs/sql-data-sources-json.md @@ -0,0 +1,81 @@ +--- +layout: global +title: JSON Files +displayTitle: JSON Files +--- + +
    + +
    +Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset[Row]`. +This conversion can be done using `SparkSession.read.json()` on either a `Dataset[String]`, +or a JSON file. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set the `multiLine` option to `true`. + +{% include_example json_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset`. +This conversion can be done using `SparkSession.read().json()` on either a `Dataset`, +or a JSON file. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set the `multiLine` option to `true`. + +{% include_example json_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. +This conversion can be done using `SparkSession.read.json` on a JSON file. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set the `multiLine` parameter to `True`. + +{% include_example json_dataset python/sql/datasource.py %} +
    + +
    +Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. using +the `read.json()` function, which loads data from a directory of JSON files where each line of the +files is a JSON object. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set a named parameter `multiLine` to `TRUE`. + +{% include_example json_dataset r/RSparkSQLExample.R %} + +
    + +
    + +{% highlight sql %} + +CREATE TEMPORARY VIEW jsonTable +USING org.apache.spark.sql.json +OPTIONS ( + path "examples/src/main/resources/people.json" +) + +SELECT * FROM jsonTable + +{% endhighlight %} + +
    + +
    \ No newline at end of file diff --git a/docs/sql-data-sources-load-save-functions.md b/docs/sql-data-sources-load-save-functions.md new file mode 100644 index 000000000000..e1dd0a3f54e4 --- /dev/null +++ b/docs/sql-data-sources-load-save-functions.md @@ -0,0 +1,283 @@ +--- +layout: global +title: Generic Load/Save Functions +displayTitle: Generic Load/Save Functions +--- + +* Table of contents +{:toc} + + +In the simplest form, the default data source (`parquet` unless otherwise configured by +`spark.sql.sources.default`) will be used for all operations. + + +
    +
    +{% include_example generic_load_save_functions scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example generic_load_save_functions java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    + +{% include_example generic_load_save_functions python/sql/datasource.py %} +
    + +
    + +{% include_example generic_load_save_functions r/RSparkSQLExample.R %} + +
    +
    + +### Manually Specifying Options + +You can also manually specify the data source that will be used along with any extra options +that you would like to pass to the data source. Data sources are specified by their fully qualified +name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use their short +names (`json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`). DataFrames loaded from any data +source type can be converted into other types using this syntax. + +To load a JSON file you can use: + +
    +
    +{% include_example manual_load_options scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example manual_load_options java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example manual_load_options python/sql/datasource.py %} +
    + +
    +{% include_example manual_load_options r/RSparkSQLExample.R %} +
    +
    + +To load a CSV file you can use: + +
    +
    +{% include_example manual_load_options_csv scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example manual_load_options_csv java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example manual_load_options_csv python/sql/datasource.py %} +
    + +
    +{% include_example manual_load_options_csv r/RSparkSQLExample.R %} + +
    +
    + +### Run SQL on files directly + +Instead of using read API to load a file into DataFrame and query it, you can also query that +file directly with SQL. + +
    +
    +{% include_example direct_sql scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example direct_sql java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example direct_sql python/sql/datasource.py %} +
    + +
    +{% include_example direct_sql r/RSparkSQLExample.R %} + +
    +
    + +### Save Modes + +Save operations can optionally take a `SaveMode`, that specifies how to handle existing data if +present. It is important to realize that these save modes do not utilize any locking and are not +atomic. Additionally, when performing an `Overwrite`, the data will be deleted before writing out the +new data. + + + + + + + + + + + + + + + + + + + + + + + +
    Scala/JavaAny LanguageMeaning
    SaveMode.ErrorIfExists (default)"error" or "errorifexists" (default) + When saving a DataFrame to a data source, if data already exists, + an exception is expected to be thrown. +
    SaveMode.Append"append" + When saving a DataFrame to a data source, if data/table already exists, + contents of the DataFrame are expected to be appended to existing data. +
    SaveMode.Overwrite"overwrite" + Overwrite mode means that when saving a DataFrame to a data source, + if data/table already exists, existing data is expected to be overwritten by the contents of + the DataFrame. +
    SaveMode.Ignore"ignore" + Ignore mode means that when saving a DataFrame to a data source, if data already exists, + the save operation is expected not to save the contents of the DataFrame and not to + change the existing data. This is similar to a CREATE TABLE IF NOT EXISTS in SQL. +
    + +### Saving to Persistent Tables + +`DataFrames` can also be saved as persistent tables into Hive metastore using the `saveAsTable` +command. Notice that an existing Hive deployment is not necessary to use this feature. Spark will create a +default local Hive metastore (using Derby) for you. Unlike the `createOrReplaceTempView` command, +`saveAsTable` will materialize the contents of the DataFrame and create a pointer to the data in the +Hive metastore. Persistent tables will still exist even after your Spark program has restarted, as +long as you maintain your connection to the same metastore. A DataFrame for a persistent table can +be created by calling the `table` method on a `SparkSession` with the name of the table. + +For file-based data source, e.g. text, parquet, json, etc. you can specify a custom table path via the +`path` option, e.g. `df.write.option("path", "/some/path").saveAsTable("t")`. When the table is dropped, +the custom table path will not be removed and the table data is still there. If no custom table path is +specified, Spark will write data to a default table path under the warehouse directory. When the table is +dropped, the default table path will be removed too. + +Starting from Spark 2.1, persistent datasource tables have per-partition metadata stored in the Hive metastore. This brings several benefits: + +- Since the metastore can return only necessary partitions for a query, discovering all the partitions on the first query to the table is no longer needed. +- Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. + +Note that partition information is not gathered by default when creating external datasource tables (those with a `path` option). To sync the partition information in the metastore, you can invoke `MSCK REPAIR TABLE`. + +### Bucketing, Sorting and Partitioning + +For file-based data source, it is also possible to bucket and sort or partition the output. +Bucketing and sorting are applicable only to persistent tables: + +
    + +
    +{% include_example write_sorting_and_bucketing scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example write_sorting_and_bucketing java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example write_sorting_and_bucketing python/sql/datasource.py %} +
    + +
    + +{% highlight sql %} + +CREATE TABLE users_bucketed_by_name( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING parquet +CLUSTERED BY(name) INTO 42 BUCKETS; + +{% endhighlight %} + +
    + +
    + +while partitioning can be used with both `save` and `saveAsTable` when using the Dataset APIs. + + +
    + +
    +{% include_example write_partitioning scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example write_partitioning java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example write_partitioning python/sql/datasource.py %} +
    + +
    + +{% highlight sql %} + +CREATE TABLE users_by_favorite_color( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING csv PARTITIONED BY(favorite_color); + +{% endhighlight %} + +
    + +
    + +It is possible to use both partitioning and bucketing for a single table: + +
    + +
    +{% include_example write_partition_and_bucket scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example write_partition_and_bucket java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example write_partition_and_bucket python/sql/datasource.py %} +
    + +
    + +{% highlight sql %} + +CREATE TABLE users_bucketed_and_partitioned( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING parquet +PARTITIONED BY (favorite_color) +CLUSTERED BY(name) SORTED BY (favorite_numbers) INTO 42 BUCKETS; + +{% endhighlight %} + +
    + +
    + +`partitionBy` creates a directory structure as described in the [Partition Discovery](sql-data-sources-parquet.html#partition-discovery) section. +Thus, it has limited applicability to columns with high cardinality. In contrast + `bucketBy` distributes +data across a fixed number of buckets and can be used when a number of unique values is unbounded. diff --git a/docs/sql-data-sources-orc.md b/docs/sql-data-sources-orc.md new file mode 100644 index 000000000000..ef07d2fa2e53 --- /dev/null +++ b/docs/sql-data-sources-orc.md @@ -0,0 +1,26 @@ +--- +layout: global +title: ORC Files +displayTitle: ORC Files +--- + +Since Spark 2.3, Spark supports a vectorized ORC reader with a new ORC file format for ORC files. +To do that, the following configurations are newly added. The vectorized reader is used for the +native ORC tables (e.g., the ones created using the clause `USING ORC`) when `spark.sql.orc.impl` +is set to `native` and `spark.sql.orc.enableVectorizedReader` is set to `true`. For the Hive ORC +serde tables (e.g., the ones created using the clause `USING HIVE OPTIONS (fileFormat 'ORC')`), +the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also set to `true`. + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.orc.implnativeThe name of ORC implementation. It can be one of native and hive. native means the native ORC support that is built on Apache ORC 1.4. `hive` means the ORC library in Hive 1.2.1.
    spark.sql.orc.enableVectorizedReadertrueEnables vectorized orc decoding in native implementation. If false, a new non-vectorized ORC reader is used in native implementation. For hive implementation, this is ignored.
    diff --git a/docs/sql-data-sources-parquet.md b/docs/sql-data-sources-parquet.md new file mode 100644 index 000000000000..4fed3eaf83e5 --- /dev/null +++ b/docs/sql-data-sources-parquet.md @@ -0,0 +1,321 @@ +--- +layout: global +title: Parquet Files +displayTitle: Parquet Files +--- + +* Table of contents +{:toc} + +[Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. +Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema +of the original data. When writing Parquet files, all columns are automatically converted to be nullable for +compatibility reasons. + +### Loading Data Programmatically + +Using the data from the above example: + +
    + +
    +{% include_example basic_parquet_example scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example basic_parquet_example java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    + +{% include_example basic_parquet_example python/sql/datasource.py %} +
    + +
    + +{% include_example basic_parquet_example r/RSparkSQLExample.R %} + +
    + +
    + +{% highlight sql %} + +CREATE TEMPORARY VIEW parquetTable +USING org.apache.spark.sql.parquet +OPTIONS ( + path "examples/src/main/resources/people.parquet" +) + +SELECT * FROM parquetTable + +{% endhighlight %} + +
    + +
    + +### Partition Discovery + +Table partitioning is a common optimization approach used in systems like Hive. In a partitioned +table, data are usually stored in different directories, with partitioning column values encoded in +the path of each partition directory. All built-in file sources (including Text/CSV/JSON/ORC/Parquet) +are able to discover and infer partitioning information automatically. +For example, we can store all our previously used +population data into a partitioned table using the following directory structure, with two extra +columns, `gender` and `country` as partitioning columns: + +{% highlight text %} + +path +└── to + └── table + ├── gender=male + │   ├── ... + │   │ + │   ├── country=US + │   │   └── data.parquet + │   ├── country=CN + │   │   └── data.parquet + │   └── ... + └── gender=female +    ├── ... +    │ +    ├── country=US +    │   └── data.parquet +    ├── country=CN +    │   └── data.parquet +    └── ... + +{% endhighlight %} + +By passing `path/to/table` to either `SparkSession.read.parquet` or `SparkSession.read.load`, Spark SQL +will automatically extract the partitioning information from the paths. +Now the schema of the returned DataFrame becomes: + +{% highlight text %} + +root +|-- name: string (nullable = true) +|-- age: long (nullable = true) +|-- gender: string (nullable = true) +|-- country: string (nullable = true) + +{% endhighlight %} + +Notice that the data types of the partitioning columns are automatically inferred. Currently, +numeric data types, date, timestamp and string type are supported. Sometimes users may not want +to automatically infer the data types of the partitioning columns. For these use cases, the +automatic type inference can be configured by +`spark.sql.sources.partitionColumnTypeInference.enabled`, which is default to `true`. When type +inference is disabled, string type will be used for the partitioning columns. + +Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths +by default. For the above example, if users pass `path/to/table/gender=male` to either +`SparkSession.read.parquet` or `SparkSession.read.load`, `gender` will not be considered as a +partitioning column. If users need to specify the base path that partition discovery +should start with, they can set `basePath` in the data source options. For example, +when `path/to/table/gender=male` is the path of the data and +users set `basePath` to `path/to/table/`, `gender` will be a partitioning column. + +### Schema Merging + +Like Protocol Buffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with +a simple schema, and gradually add more columns to the schema as needed. In this way, users may end +up with multiple Parquet files with different but mutually compatible schemas. The Parquet data +source is now able to automatically detect this case and merge schemas of all these files. + +Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we +turned it off by default starting from 1.5.0. You may enable it by + +1. setting data source option `mergeSchema` to `true` when reading Parquet files (as shown in the + examples below), or +2. setting the global SQL option `spark.sql.parquet.mergeSchema` to `true`. + +
    + +
    +{% include_example schema_merging scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example schema_merging java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    + +{% include_example schema_merging python/sql/datasource.py %} +
    + +
    + +{% include_example schema_merging r/RSparkSQLExample.R %} + +
    + +
    + +### Hive metastore Parquet table conversion + +When reading from and writing to Hive metastore Parquet tables, Spark SQL will try to use its own +Parquet support instead of Hive SerDe for better performance. This behavior is controlled by the +`spark.sql.hive.convertMetastoreParquet` configuration, and is turned on by default. + +#### Hive/Parquet Schema Reconciliation + +There are two key differences between Hive and Parquet from the perspective of table schema +processing. + +1. Hive is case insensitive, while Parquet is not +1. Hive considers all columns nullable, while nullability in Parquet is significant + +Due to this reason, we must reconcile Hive metastore schema with Parquet schema when converting a +Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are: + +1. Fields that have the same name in both schema must have the same data type regardless of + nullability. The reconciled field should have the data type of the Parquet side, so that + nullability is respected. + +1. The reconciled schema contains exactly those fields defined in Hive metastore schema. + + - Any fields that only appear in the Parquet schema are dropped in the reconciled schema. + - Any fields that only appear in the Hive metastore schema are added as nullable field in the + reconciled schema. + +#### Metadata Refreshing + +Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table +conversion is enabled, metadata of those converted tables are also cached. If these tables are +updated by Hive or other external tools, you need to refresh them manually to ensure consistent +metadata. + +
    + +
    + +{% highlight scala %} +// spark is an existing SparkSession +spark.catalog.refreshTable("my_table") +{% endhighlight %} + +
    + +
    + +{% highlight java %} +// spark is an existing SparkSession +spark.catalog().refreshTable("my_table"); +{% endhighlight %} + +
    + +
    + +{% highlight python %} +# spark is an existing SparkSession +spark.catalog.refreshTable("my_table") +{% endhighlight %} + +
    + +
    + +{% highlight r %} +refreshTable("my_table") +{% endhighlight %} + +
    + +
    + +{% highlight sql %} +REFRESH TABLE my_table; +{% endhighlight %} + +
    + +
    + +### Configuration + +Configuration of Parquet can be done using the `setConf` method on `SparkSession` or by running +`SET key=value` commands using SQL. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.parquet.binaryAsStringfalse + Some other Parquet-producing systems, in particular Impala, Hive, and older versions of Spark SQL, do + not differentiate between binary data and strings when writing out the Parquet schema. This + flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. +
    spark.sql.parquet.int96AsTimestamptrue + Some Parquet-producing systems, in particular Impala and Hive, store Timestamp into INT96. This + flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems. +
    spark.sql.parquet.compression.codecsnappy + Sets the compression codec used when writing Parquet files. If either `compression` or + `parquet.compression` is specified in the table-specific options/properties, the precedence would be + `compression`, `parquet.compression`, `spark.sql.parquet.compression.codec`. Acceptable values include: + none, uncompressed, snappy, gzip, lzo, brotli, lz4, zstd. + Note that `zstd` requires `ZStandardCodec` to be installed before Hadoop 2.9.0, `brotli` requires + `BrotliCodec` to be installed. +
    spark.sql.parquet.filterPushdowntrueEnables Parquet filter push-down optimization when set to true.
    spark.sql.hive.convertMetastoreParquettrue + When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of the built in + support. +
    spark.sql.parquet.mergeSchemafalse +

    + When true, the Parquet data source merges schemas collected from all data files, otherwise the + schema is picked from the summary file or a random data file if no summary file is available. +

    +
    spark.sql.optimizer.metadataOnlytrue +

    + When true, enable the metadata-only query optimization that use the table's metadata to + produce the partition columns instead of table scans. It applies when all the columns scanned + are partition columns and the query has an aggregate operator that satisfies distinct + semantics. +

    +
    spark.sql.parquet.writeLegacyFormatfalse + If true, data will be written in a way of Spark 1.4 and earlier. For example, decimal values + will be written in Apache Parquet's fixed-length byte array format, which other systems such as + Apache Hive and Apache Impala use. If false, the newer format in Parquet will be used. For + example, decimals will be written in int-based format. If Parquet output is intended for use + with systems that do not support this newer format, set to true. +
    diff --git a/docs/sql-data-sources-troubleshooting.md b/docs/sql-data-sources-troubleshooting.md new file mode 100644 index 000000000000..5775eb8b5c95 --- /dev/null +++ b/docs/sql-data-sources-troubleshooting.md @@ -0,0 +1,9 @@ +--- +layout: global +title: Troubleshooting +displayTitle: Troubleshooting +--- + + * The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java's DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs. + * Some databases, such as H2, convert all names to upper case. You'll need to use upper case to refer to those names in Spark SQL. + * Users can specify vendor-specific JDBC connection properties in the data source options to do special treatment. For example, `spark.read.format("jdbc").option("url", oracleJdbcUrl).option("oracle.jdbc.mapDateToTimestamp", "false")`. `oracle.jdbc.mapDateToTimestamp` defaults to true, users often need to disable this flag to avoid Oracle date being resolved as timestamp. diff --git a/docs/sql-data-sources.md b/docs/sql-data-sources.md new file mode 100644 index 000000000000..aa607ec23a56 --- /dev/null +++ b/docs/sql-data-sources.md @@ -0,0 +1,42 @@ +--- +layout: global +title: Data Sources +displayTitle: Data Sources +--- + + +Spark SQL supports operating on a variety of data sources through the DataFrame interface. +A DataFrame can be operated on using relational transformations and can also be used to create a temporary view. +Registering a DataFrame as a temporary view allows you to run SQL queries over its data. This section +describes the general methods for loading and saving data using the Spark Data Sources and then +goes into specific options that are available for the built-in data sources. + + +* [Generic Load/Save Functions](sql-data-sources-load-save-functions.html) + * [Manually Specifying Options](sql-data-sources-load-save-functions.html#manually-specifying-options) + * [Run SQL on files directly](sql-data-sources-load-save-functions.html#run-sql-on-files-directly) + * [Save Modes](sql-data-sources-load-save-functions.html#save-modes) + * [Saving to Persistent Tables](sql-data-sources-load-save-functions.html#run-sql-on-files-directly) + * [Bucketing, Sorting and Partitioning](sql-data-sources-load-save-functions.html#run-sql-on-files-directly) +* [Parquet Files](sql-data-sources-parquet.html) + * [Loading Data Programmatically](sql-data-sources-parquet.html#loading-data-programmatically) + * [Partition Discovery](sql-data-sources-parquet.html#partition-discovery) + * [Schema Merging](sql-data-sources-parquet.html#schema-merging) + * [Hive metastore Parquet table conversion](sql-data-sources-parquet.html#hive-metastore-parquet-table-conversion) + * [Configuration](sql-data-sources-parquet.html#configuration) +* [ORC Files](sql-data-sources-orc.html) +* [JSON Files](sql-data-sources-json.html) +* [Hive Tables](sql-data-sources-hive-tables.html) + * [Specifying storage format for Hive tables](sql-data-sources-hive-tables.html#specifying-storage-format-for-hive-tables) + * [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore) +* [JDBC To Other Databases](sql-data-sources-jdbc.html) +* [Avro Files](sql-data-sources-avro.html) + * [Deploying](sql-data-sources-avro.html#deploying) + * [Load and Save Functions](sql-data-sources-avro.html#load-and-save-functions) + * [to_avro() and from_avro()](sql-data-sources-avro.html#to_avro-and-from_avro) + * [Data Source Option](sql-data-sources-avro.html#data-source-option) + * [Configuration](sql-data-sources-avro.html#configuration) + * [Compatibility with Databricks spark-avro](sql-data-sources-avro.html#compatibility-with-databricks-spark-avro) + * [Supported types for Avro -> Spark SQL conversion](sql-data-sources-avro.html#supported-types-for-avro---spark-sql-conversion) + * [Supported types for Spark SQL -> Avro conversion](sql-data-sources-avro.html#supported-types-for-spark-sql---avro-conversion) +* [Troubleshooting](sql-data-sources-troubleshooting.html) diff --git a/docs/sql-distributed-sql-engine.md b/docs/sql-distributed-sql-engine.md new file mode 100644 index 000000000000..66d6fdaf90a0 --- /dev/null +++ b/docs/sql-distributed-sql-engine.md @@ -0,0 +1,84 @@ +--- +layout: global +title: Distributed SQL Engine +displayTitle: Distributed SQL Engine +--- + +* Table of contents +{:toc} + +Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. +In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, +without the need to write any code. + +## Running the Thrift JDBC/ODBC server + +The Thrift JDBC/ODBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) +in Hive 1.2.1. You can test the JDBC server with the beeline script that comes with either Spark or Hive 1.2.1. + +To start the JDBC/ODBC server, run the following in the Spark directory: + + ./sbin/start-thriftserver.sh + +This script accepts all `bin/spark-submit` command line options, plus a `--hiveconf` option to +specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` for a complete list of +all available options. By default, the server listens on localhost:10000. You may override this +behaviour via either environment variables, i.e.: + +{% highlight bash %} +export HIVE_SERVER2_THRIFT_PORT= +export HIVE_SERVER2_THRIFT_BIND_HOST= +./sbin/start-thriftserver.sh \ + --master \ + ... +{% endhighlight %} + +or system properties: + +{% highlight bash %} +./sbin/start-thriftserver.sh \ + --hiveconf hive.server2.thrift.port= \ + --hiveconf hive.server2.thrift.bind.host= \ + --master + ... +{% endhighlight %} + +Now you can use beeline to test the Thrift JDBC/ODBC server: + + ./bin/beeline + +Connect to the JDBC/ODBC server in beeline with: + + beeline> !connect jdbc:hive2://localhost:10000 + +Beeline will ask you for a username and password. In non-secure mode, simply enter the username on +your machine and a blank password. For secure mode, please follow the instructions given in the +[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients). + +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. + +You may also use the beeline script that comes with Hive. + +Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. +Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: + + hive.server2.transport.mode - Set this to value: http + hive.server2.thrift.http.port - HTTP port number to listen on; default is 10001 + hive.server2.http.endpoint - HTTP endpoint; default is cliservice + +To test, use beeline to connect to the JDBC/ODBC server in http mode with: + + beeline> !connect jdbc:hive2://:/?hive.server2.transport.mode=http;hive.server2.thrift.http.path= + + +## Running the Spark SQL CLI + +The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute +queries input from the command line. Note that the Spark SQL CLI cannot talk to the Thrift JDBC server. + +To start the Spark SQL CLI, run the following in the Spark directory: + + ./bin/spark-sql + +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. +You may run `./bin/spark-sql --help` for a complete list of all available options. diff --git a/docs/sql-getting-started.md b/docs/sql-getting-started.md new file mode 100644 index 000000000000..88512205894a --- /dev/null +++ b/docs/sql-getting-started.md @@ -0,0 +1,369 @@ +--- +layout: global +title: Getting Started +displayTitle: Getting Started +--- + +* Table of contents +{:toc} + +## Starting Point: SparkSession + +
    +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: + +{% include_example init_session scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: + +{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder`: + +{% include_example init_session python/sql/basic.py %} +
    + +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/R/sparkR.session.html) class. To initialize a basic `SparkSession`, just call `sparkR.session()`: + +{% include_example init_session r/RSparkSQLExample.R %} + +Note that when invoked for the first time, `sparkR.session()` initializes a global `SparkSession` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SparkSession` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SparkSession` instance around. +
    +
    + +`SparkSession` in Spark 2.0 provides builtin support for Hive features including the ability to +write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. +To use these features, you do not need to have an existing Hive setup. + +## Creating DataFrames + +
    +
    +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](sql-data-sources.html). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](sql-data-sources.html). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](sql-data-sources.html). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df python/sql/basic.py %} +
    + +
    +With a `SparkSession`, applications can create DataFrames from a local R data.frame, +from a Hive table, or from [Spark data sources](sql-data-sources.html). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df r/RSparkSQLExample.R %} + +
    +
    + + +## Untyped Dataset Operations (aka DataFrame Operations) + +DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/SparkDataFrame.html). + +As mentioned above, in Spark 2.0, DataFrames are just Dataset of `Row`s in Scala and Java API. These operations are also referred as "untyped transformations" in contrast to "typed transformations" come with strongly typed Scala/Java Datasets. + +Here we include some basic examples of structured data processing using Datasets: + +
    +
    +{% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} + +For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). + +In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$). +
    + +
    + +{% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} + +For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/java/org/apache/spark/sql/Dataset.html). + +In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html). +
    + +
    +In Python, it's possible to access a DataFrame's columns either by attribute +(`df.age`) or by indexing (`df['age']`). While the former is convenient for +interactive data exploration, users are highly encouraged to use the +latter form, which is future proof and won't break with column names that +are also attributes on the DataFrame class. + +{% include_example untyped_ops python/sql/basic.py %} +For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/python/pyspark.sql.html#pyspark.sql.DataFrame). + +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/python/pyspark.sql.html#module-pyspark.sql.functions). + +
    + +
    + +{% include_example untyped_ops r/RSparkSQLExample.R %} + +For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html). + +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/R/SparkDataFrame.html). + +
    + +
    + +## Running SQL Queries Programmatically + +
    +
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + +{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `Dataset`. + +{% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + +{% include_example run_sql python/sql/basic.py %} +
    + +
    +The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. + +{% include_example run_sql r/RSparkSQLExample.R %} + +
    +
    + + +## Global Temporary View + +Temporary views in Spark SQL are session-scoped and will disappear if the session that creates it +terminates. If you want to have a temporary view that is shared among all sessions and keep alive +until the Spark application terminates, you can create a global temporary view. Global temporary +view is tied to a system preserved database `global_temp`, and we must use the qualified name to +refer it, e.g. `SELECT * FROM global_temp.view1`. + +
    +
    +{% include_example global_temp_view scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +{% include_example global_temp_view java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    +{% include_example global_temp_view python/sql/basic.py %} +
    + +
    + +{% highlight sql %} + +CREATE GLOBAL TEMPORARY VIEW temp_view AS SELECT a + 1, b * 2 FROM tbl + +SELECT * FROM global_temp.temp_view + +{% endhighlight %} + +
    +
    + + +## Creating Datasets + +Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use +a specialized [Encoder](api/scala/index.html#org.apache.spark.sql.Encoder) to serialize the objects +for processing or transmitting over the network. While both encoders and standard serialization are +responsible for turning an object into bytes, encoders are code generated dynamically and use a format +that allows Spark to perform many operations like filtering, sorting and hashing without deserializing +the bytes back into an object. + +
    +
    +{% include_example create_ds scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +{% include_example create_ds java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    +
    + +## Interoperating with RDDs + +Spark SQL supports two different methods for converting existing RDDs into Datasets. The first +method uses reflection to infer the schema of an RDD that contains specific types of objects. This +reflection-based approach leads to more concise code and works well when you already know the schema +while writing your Spark application. + +The second method for creating Datasets is through a programmatic interface that allows you to +construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows +you to construct Datasets when the columns and their types are not known until runtime. + +### Inferring the Schema Using Reflection +
    + +
    + +The Scala interface for Spark SQL supports automatically converting an RDD containing case classes +to a DataFrame. The case class +defines the schema of the table. The names of the arguments to the case class are read using +reflection and become the names of the columns. Case classes can also be nested or contain complex +types such as `Seq`s or `Array`s. This RDD can be implicitly converted to a DataFrame and then be +registered as a table. Tables can be used in subsequent SQL statements. + +{% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    + +Spark SQL supports automatically converting an RDD of +[JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) into a DataFrame. +The `BeanInfo`, obtained using reflection, defines the schema of the table. Currently, Spark SQL +does not support JavaBeans that contain `Map` field(s). Nested JavaBeans and `List` or `Array` +fields are supported though. You can create a JavaBean by creating a class that implements +Serializable and has getters and setters for all of its fields. + +{% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    + +Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of +key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, +and the types are inferred by sampling the whole dataset, similar to the inference that is performed on JSON files. + +{% include_example schema_inferring python/sql/basic.py %} +
    + +
    + +### Programmatically Specifying the Schema + +
    + +
    + +When case classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed +and fields will be projected differently for different users), +a `DataFrame` can be created programmatically with three steps. + +1. Create an RDD of `Row`s from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +`Row`s in the RDD created in Step 1. +3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided +by `SparkSession`. + +For example: + +{% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    + +When JavaBean classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and +fields will be projected differently for different users), +a `Dataset` can be created programmatically with three steps. + +1. Create an RDD of `Row`s from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +`Row`s in the RDD created in Step 1. +3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided +by `SparkSession`. + +For example: + +{% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    + +When a dictionary of kwargs cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and +fields will be projected differently for different users), +a `DataFrame` can be created programmatically with three steps. + +1. Create an RDD of tuples or lists from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +tuples or lists in the RDD created in the step 1. +3. Apply the schema to the RDD via `createDataFrame` method provided by `SparkSession`. + +For example: + +{% include_example programmatic_schema python/sql/basic.py %} +
    + +
    + +## Aggregations + +The [built-in DataFrames functions](api/scala/index.html#org.apache.spark.sql.functions$) provide common +aggregations such as `count()`, `countDistinct()`, `avg()`, `max()`, `min()`, etc. +While those functions are designed for DataFrames, Spark SQL also has type-safe versions for some of them in +[Scala](api/scala/index.html#org.apache.spark.sql.expressions.scalalang.typed$) and +[Java](api/java/org/apache/spark/sql/expressions/javalang/typed.html) to work with strongly typed Datasets. +Moreover, users are not limited to the predefined aggregate functions and can create their own. + +### Untyped User-Defined Aggregate Functions +Users have to extend the [UserDefinedAggregateFunction](api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction) +abstract class to implement a custom untyped aggregate function. For example, a user-defined average +can look like: + +
    +
    +{% include_example untyped_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%} +
    +
    +{% include_example untyped_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%} +
    +
    + +### Type-Safe User-Defined Aggregate Functions + +User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) abstract class. +For example, a type-safe user-defined average can look like: + +
    +
    +{% include_example typed_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%} +
    +
    +{% include_example typed_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%} +
    +
    diff --git a/docs/sql-migration-guide-hive-compatibility.md b/docs/sql-migration-guide-hive-compatibility.md new file mode 100644 index 000000000000..0234ea28bb33 --- /dev/null +++ b/docs/sql-migration-guide-hive-compatibility.md @@ -0,0 +1,137 @@ +--- +layout: global +title: Compatibility with Apache Hive +displayTitle: Compatibility with Apache Hive +--- + +* Table of contents +{:toc} + +Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. +Currently, Hive SerDes and UDFs are based on Hive 1.2.1, +and Spark SQL can be connected to different versions of Hive Metastore +(from 0.12.0 to 2.3.3. Also see [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore)). + +#### Deploying in Existing Hive Warehouses + +The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive +installations. You do not need to modify your existing Hive Metastore or change the data placement +or partitioning of your tables. + +### Supported Hive Features + +Spark SQL supports the vast majority of Hive features, such as: + +* Hive query statements, including: + * `SELECT` + * `GROUP BY` + * `ORDER BY` + * `CLUSTER BY` + * `SORT BY` +* All Hive operators, including: + * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) + * Logical operators (`AND`, `&&`, `OR`, `||`, etc) + * Complex type constructors + * Mathematical functions (`sign`, `ln`, `cos`, etc) + * String functions (`instr`, `length`, `printf`, etc) +* User defined functions (UDF) +* User defined aggregation functions (UDAF) +* User defined serialization formats (SerDes) +* Window functions +* Joins + * `JOIN` + * `{LEFT|RIGHT|FULL} OUTER JOIN` + * `LEFT SEMI JOIN` + * `CROSS JOIN` +* Unions +* Sub-queries + * `SELECT col FROM ( SELECT a + b AS col from t1) t2` +* Sampling +* Explain +* Partitioned tables including dynamic partition insertion +* View +* All Hive DDL Functions, including: + * `CREATE TABLE` + * `CREATE TABLE AS SELECT` + * `ALTER TABLE` +* Most Hive Data types, including: + * `TINYINT` + * `SMALLINT` + * `INT` + * `BIGINT` + * `BOOLEAN` + * `FLOAT` + * `DOUBLE` + * `STRING` + * `BINARY` + * `TIMESTAMP` + * `DATE` + * `ARRAY<>` + * `MAP<>` + * `STRUCT<>` + +### Unsupported Hive Functionality + +Below is a list of Hive features that we don't support yet. Most of these features are rarely used +in Hive deployments. + +**Major Hive Features** + +* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL + doesn't support buckets yet. + + +**Esoteric Hive Features** + +* `UNION` type +* Unique join +* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at + the moment and only supports populating the sizeInBytes field of the hive metastore. + +**Hive Input/Output Formats** + +* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. +* Hadoop archive + +**Hive Optimizations** + +A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are +less important due to Spark SQL's in-memory computational model. Others are slotted for future +releases of Spark SQL. + +* Block-level bitmap indexes and virtual columns (used to build indexes) +* Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you + need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". +* Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still + launches tasks to compute the result. +* Skew data flag: Spark SQL does not follow the skew data flags in Hive. +* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. +* Merge multiple small files for query results: if the result output contains multiple small files, + Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS + metadata. Spark SQL does not support that. + +**Hive UDF/UDTF/UDAF** + +Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs: + +* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically + include additional resources required by this UDF. +* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses + a deprecated interface `initialize(ObjectInspector[])` only. +* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize + functions with `MapredContext`, which is inapplicable to Spark. +* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources. + Spark SQL does not call this function when tasks finish. +* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation. + Spark SQL currently does not support the reuse of aggregation. +* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating + an aggregate over a fixed window. + +### Incompatible Hive UDF + +Below are the scenarios in which Hive and Spark generate different results: + +* `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN. +* `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. +* `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md new file mode 100644 index 000000000000..3476aa83c9b2 --- /dev/null +++ b/docs/sql-migration-guide-upgrade.md @@ -0,0 +1,516 @@ +--- +layout: global +title: Spark SQL Upgrading Guide +displayTitle: Spark SQL Upgrading Guide +--- + +* Table of contents +{:toc} + +## Upgrading From Spark SQL 2.3 to 2.4 + + - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. + + + + + + + + + + + + + + + + + + + + + + + + + +
    + Query + + Result Spark 2.3 or Prior + + Result Spark 2.4 + + Remarks +
    + SELECT
    array_contains(array(1), 1.34D);
    +
    + true + + false + + In Spark 2.4, left and right parameters are promoted to array(double) and double type respectively. +
    + SELECT
    array_contains(array(1), '1');
    +
    + true + + AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. + + Users can use explicit cast +
    + SELECT
    array_contains(array(1), 'anystring');
    +
    + null + + AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. + + Users can use explicit cast +
    + + - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. + - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. + - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. + - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. + - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. + - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. + - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. + - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, a column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. + - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. + - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. + - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. + - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. + - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. + - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. + - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. + - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. + - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. + - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. + - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. + - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. + - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. + - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. + +## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above + + - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. + +## Upgrading From Spark SQL 2.2 to 2.3 + + - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. + - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. + - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. + - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + InputA \ InputB + + NullType + + IntegerType + + LongType + + DecimalType(38,0)* + + DoubleType + + DateType + + TimestampType + + StringType +
    + NullType + NullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
    + IntegerType + IntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
    + LongType + LongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
    + DecimalType(38,0)* + DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
    + DoubleType + DoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
    + DateType + DateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
    + TimestampType + TimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
    + StringType + StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
    + + Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. + - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. + - In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration `spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See [SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details. + - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. + - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](sql-performance-turing.html#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). + - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. + - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. + + - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive's new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes + - The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`). + - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. + - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible. + - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. + - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. + + - When creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 2.3, the builder comes to not update the configurations. If you want to update them, you need to update them prior to creating a `SparkSession`. + +## Upgrading From Spark SQL 2.1 to 2.2 + + - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. + + - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). + +## Upgrading From Spark SQL 2.0 to 2.1 + + - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. + - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. + - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. + - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. + - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. + - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. + +## Upgrading From Spark SQL 1.6 to 2.0 + + - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and + `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. + + - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for + `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed + transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., + `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in + Python and R is not a language feature, the concept of Dataset does not apply to these languages’ + APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the + single-node data frame notion in these languages. + + - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` + - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` + - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` + + - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. + - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` + in order to prevent accidental dropping the existing data in the user-provided locations. + That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. + Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. + Note that this is different from the Hive behavior. + - As a result, `DROP TABLE` statements on those tables will not remove the data. + + - `spark.sql.parquet.cacheMetadata` is no longer used. + See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details. + +## Upgrading From Spark SQL 1.5 to 1.6 + + - From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC + connection owns a copy of their own SQL configuration and temporary function registry. Cached + tables are still shared though. If you prefer to run the Thrift server in the old single-session + mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add + this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`: + + {% highlight bash %} + ./sbin/start-thriftserver.sh \ + --conf spark.sql.hive.thriftServer.singleSession=true \ + ... + {% endhighlight %} + - Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns + of the same name of a DataFrame. + + - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This + change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType + from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for + details. + +## Upgrading From Spark SQL 1.4 to 1.5 + + - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with + code generation for expression evaluation. These features can both be disabled by setting + `spark.sql.tungsten.enabled` to `false`. + - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting + `spark.sql.parquet.mergeSchema` to `true`. + - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or + access nested values. For example `df['table.column.nestedField']`. However, this means that if + your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). + - In-memory columnar storage partition pruning is on by default. It can be disabled by setting + `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. + - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum + precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now + used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. + - Timestamps are now stored at a precision of 1us, rather than 1ns + - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains + unchanged. + - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). + - JSON data source will not automatically load new files that are created by other applications + (i.e. files that are not inserted to the dataset through Spark SQL). + For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), + users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method + to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate + the DataFrame and the new DataFrame will include new files. + - DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name. + +## Upgrading from Spark SQL 1.3 to 1.4 + +#### DataFrame data reader/writer interface + +Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) +and writing data out (`DataFrame.write`), +and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). + +See the API docs for `SQLContext.read` ( + Scala, + Java, + Python +) and `DataFrame.write` ( + Scala, + Java, + Python +) more information. + + +#### DataFrame.groupBy retains grouping columns + +Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the +grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. + +
    +
    +{% highlight scala %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg($"department", max("age"), sum("expense")) + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")) + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    +{% highlight java %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg(col("department"), max("age"), sum("expense")); + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")); + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false"); + +{% endhighlight %} +
    + +
    +{% highlight python %} + +import pyspark.sql.functions as func + +# In 1.3.x, in order for the grouping column "department" to show up, +# it must be included explicitly as part of the agg function call. +df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) + +# In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(func.max("age"), func.sum("expense")) + +# Revert to 1.3.x behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    + + +#### Behavior change on DataFrame.withColumn + +Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added +as a new column with its specified name in the result DataFrame even if there may be any existing +columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different +name from names of all existing columns or replacing existing columns of the same name. + +Note that this change is only for Scala API, not for PySpark and SparkR. + + +## Upgrading from Spark SQL 1.0-1.2 to 1.3 + +In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the +available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other +releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked +as unstable (i.e., DeveloperAPI or Experimental). + +#### Rename of SchemaRDD to DataFrame + +The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has +been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD +directly, but instead provide most of the functionality that RDDs provide though their own +implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. + +In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for +some use cases. It is still recommended that users update their code to use `DataFrame` instead. +Java and Python users will need to update their code. + +#### Unification of the Java and Scala APIs + +Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) +that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users +of either language should use `SQLContext` and `DataFrame`. In general these classes try to +use types that are usable from both languages (i.e. `Array` instead of language-specific collections). +In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading +is used instead. + +Additionally, the Java specific types API has been removed. Users of both Scala and Java should +use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. + + +#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) + +Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought +all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit +conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. +Users should now write `import sqlContext.implicits._`. + +Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e., +case classes or tuples) with a method `toDF`, instead of applying automatically. + +When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import +`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: +`import org.apache.spark.sql.functions._`. + +#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) + +Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users +should instead import the classes in `org.apache.spark.sql.types` + +#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) + +Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been +moved into the udf object in `SQLContext`. + +
    +
    +{% highlight scala %} + +sqlContext.udf.register("strLen", (s: String) => s.length()) + +{% endhighlight %} +
    + +
    +{% highlight java %} + +sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType); + +{% endhighlight %} +
    + +
    + +Python UDF registration is unchanged. + +#### Python DataTypes No Longer Singletons + +When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of +referencing a singleton. diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md new file mode 100644 index 000000000000..71d83e8a5570 --- /dev/null +++ b/docs/sql-migration-guide.md @@ -0,0 +1,23 @@ +--- +layout: global +title: Migration Guide +displayTitle: Migration Guide +--- + +* [Spark SQL Upgrading Guide](sql-migration-guide-upgrade.html) + * [Upgrading From Spark SQL 2.4 to 3.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-24-to-30) + * [Upgrading From Spark SQL 2.3 to 2.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-23-to-24) + * [Upgrading From Spark SQL 2.3.0 to 2.3.1 and above](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-230-to-231-and-above) + * [Upgrading From Spark SQL 2.2 to 2.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-22-to-23) + * [Upgrading From Spark SQL 2.1 to 2.2](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-21-to-22) + * [Upgrading From Spark SQL 2.0 to 2.1](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-20-to-21) + * [Upgrading From Spark SQL 1.6 to 2.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-16-to-20) + * [Upgrading From Spark SQL 1.5 to 1.6](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-15-to-16) + * [Upgrading From Spark SQL 1.4 to 1.5](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-14-to-15) + * [Upgrading from Spark SQL 1.3 to 1.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-13-to-14) + * [Upgrading from Spark SQL 1.0-1.2 to 1.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-10-12-to-13) +* [Compatibility with Apache Hive](sql-migration-guide-hive-compatibility.html) + * [Deploying in Existing Hive Warehouses](sql-migration-guide-hive-compatibility.html#deploying-in-existing-hive-warehouses) + * [Supported Hive Features](sql-migration-guide-hive-compatibility.html#supported-hive-features) + * [Unsupported Hive Functionality](sql-migration-guide-hive-compatibility.html#unsupported-hive-functionality) + * [Incompatible Hive UDF](sql-migration-guide-hive-compatibility.html#incompatible-hive-udf) diff --git a/docs/sql-performance-turing.md b/docs/sql-performance-turing.md new file mode 100644 index 000000000000..7c7c4a815545 --- /dev/null +++ b/docs/sql-performance-turing.md @@ -0,0 +1,151 @@ +--- +layout: global +title: Performance Tuning +displayTitle: Performance Tuning +--- + +* Table of contents +{:toc} + +For some workloads, it is possible to improve performance by either caching data in memory, or by +turning on some experimental options. + +## Caching Data In Memory + +Spark SQL can cache tables using an in-memory columnar format by calling `spark.catalog.cacheTable("tableName")` or `dataFrame.cache()`. +Then Spark SQL will scan only required columns and will automatically tune compression to minimize +memory usage and GC pressure. You can call `spark.catalog.uncacheTable("tableName")` to remove the table from memory. + +Configuration of in-memory caching can be done using the `setConf` method on `SparkSession` or by running +`SET key=value` commands using SQL. + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.inMemoryColumnarStorage.compressedtrue + When set to true Spark SQL will automatically select a compression codec for each column based + on statistics of the data. +
    spark.sql.inMemoryColumnarStorage.batchSize10000 + Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization + and compression, but risk OOMs when caching data. +
    + +## Other Configuration Options + +The following options can also be used to tune the performance of query execution. It is possible +that these options will be deprecated in future release as more optimizations are performed automatically. + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.files.maxPartitionBytes134217728 (128 MB) + The maximum number of bytes to pack into a single partition when reading files. +
    spark.sql.files.openCostInBytes4194304 (4 MB) + The estimated cost to open a file, measured by the number of bytes could be scanned in the same + time. This is used when putting multiple files into a partition. It is better to over-estimated, + then the partitions with small files will be faster than partitions with bigger files (which is + scheduled first). +
    spark.sql.broadcastTimeout300 +

    + Timeout in seconds for the broadcast wait time in broadcast joins +

    +
    spark.sql.autoBroadcastJoinThreshold10485760 (10 MB) + Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when + performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently + statistics are only supported for Hive Metastore tables where the command + ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been run. +
    spark.sql.shuffle.partitions200 + Configures the number of partitions to use when shuffling data for joins or aggregations. +
    + +## Broadcast Hint for SQL Queries + +The `BROADCAST` hint guides Spark to broadcast each specified table when joining them with another table or view. +When Spark deciding the join methods, the broadcast hash join (i.e., BHJ) is preferred, +even if the statistics is above the configuration `spark.sql.autoBroadcastJoinThreshold`. +When both sides of a join are specified, Spark broadcasts the one having the lower statistics. +Note Spark does not guarantee BHJ is always chosen, since not all cases (e.g. full outer join) +support BHJ. When the broadcast nested loop join is selected, we still respect the hint. + +
    + +
    + +{% highlight scala %} +import org.apache.spark.sql.functions.broadcast +broadcast(spark.table("src")).join(spark.table("records"), "key").show() +{% endhighlight %} + +
    + +
    + +{% highlight java %} +import static org.apache.spark.sql.functions.broadcast; +broadcast(spark.table("src")).join(spark.table("records"), "key").show(); +{% endhighlight %} + +
    + +
    + +{% highlight python %} +from pyspark.sql.functions import broadcast +broadcast(spark.table("src")).join(spark.table("records"), "key").show() +{% endhighlight %} + +
    + +
    + +{% highlight r %} +src <- sql("SELECT * FROM src") +records <- sql("SELECT * FROM records") +head(join(broadcast(src), records, src$key == records$key)) +{% endhighlight %} + +
    + +
    + +{% highlight sql %} +-- We accept BROADCAST, BROADCASTJOIN and MAPJOIN for broadcast hint +SELECT /*+ BROADCAST(r) */ * FROM records r JOIN src s ON r.key = s.key +{% endhighlight %} + +
    +
    diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index e45e50da9fe0..42b00c9c8368 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -4,11 +4,6 @@ displayTitle: Spark SQL, DataFrames and Datasets Guide title: Spark SQL and DataFrames --- -* This will become a table of contents (this text will be scraped). -{:toc} - -# Overview - Spark SQL is a Spark module for structured data processing. Unlike the basic Spark RDD API, the interfaces provided by Spark SQL provide Spark with more information about the structure of both the data and the computation being performed. Internally, Spark SQL uses this extra information to perform extra optimizations. There are several ways to @@ -24,17 +19,17 @@ the `spark-shell`, `pyspark` shell, or `sparkR` shell. One use of Spark SQL is to execute SQL queries. Spark SQL can also be used to read data from an existing Hive installation. For more on how to -configure this feature, please refer to the [Hive Tables](#hive-tables) section. When running +configure this feature, please refer to the [Hive Tables](sql-data-sources-hive-tables.html) section. When running SQL from within another programming language the results will be returned as a [Dataset/DataFrame](#datasets-and-dataframes). -You can also interact with the SQL interface using the [command-line](#running-the-spark-sql-cli) -or over [JDBC/ODBC](#running-the-thrift-jdbcodbc-server). +You can also interact with the SQL interface using the [command-line](sql-distributed-sql-engine.html#running-the-spark-sql-cli) +or over [JDBC/ODBC](#sql-distributed-sql-engine.html#running-the-thrift-jdbcodbc-server). ## Datasets and DataFrames A Dataset is a distributed collection of data. Dataset is a new interface added in Spark 1.6 that provides the benefits of RDDs (strong typing, ability to use powerful lambda functions) with the benefits of Spark SQL's optimized -execution engine. A Dataset can be [constructed](#creating-datasets) from JVM objects and then +execution engine. A Dataset can be [constructed](sql-getting-started.html#creating-datasets) from JVM objects and then manipulated using functional transformations (`map`, `flatMap`, `filter`, etc.). The Dataset API is available in [Scala][scala-datasets] and [Java][java-datasets]. Python does not have the support for the Dataset API. But due to Python's dynamic nature, @@ -43,7 +38,7 @@ many of the benefits of the Dataset API are already available (i.e. you can acce A DataFrame is a *Dataset* organized into named columns. It is conceptually equivalent to a table in a relational database or a data frame in R/Python, but with richer -optimizations under the hood. DataFrames can be constructed from a wide array of [sources](#data-sources) such +optimizations under the hood. DataFrames can be constructed from a wide array of [sources](sql-data-sources.html) such as: structured data files, tables in Hive, external databases, or existing RDDs. The DataFrame API is available in Scala, Java, [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame), and [R](api/R/index.html). @@ -55,3107 +50,3 @@ While, in [Java API][java-datasets], users need to use `Dataset` to represe [java-datasets]: api/java/index.html?org/apache/spark/sql/Dataset.html Throughout this document, we will often refer to Scala/Java Datasets of `Row`s as DataFrames. - -# Getting Started - -## Starting Point: SparkSession - -
    -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: - -{% include_example init_session scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: - -{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder`: - -{% include_example init_session python/sql/basic.py %} -
    - -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/R/sparkR.session.html) class. To initialize a basic `SparkSession`, just call `sparkR.session()`: - -{% include_example init_session r/RSparkSQLExample.R %} - -Note that when invoked for the first time, `sparkR.session()` initializes a global `SparkSession` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SparkSession` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SparkSession` instance around. -
    -
    - -`SparkSession` in Spark 2.0 provides builtin support for Hive features including the ability to -write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. -To use these features, you do not need to have an existing Hive setup. - -## Creating DataFrames - -
    -
    -With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), -from a Hive table, or from [Spark data sources](#data-sources). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), -from a Hive table, or from [Spark data sources](#data-sources). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    -With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), -from a Hive table, or from [Spark data sources](#data-sources). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df python/sql/basic.py %} -
    - -
    -With a `SparkSession`, applications can create DataFrames from a local R data.frame, -from a Hive table, or from [Spark data sources](#data-sources). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df r/RSparkSQLExample.R %} - -
    -
    - - -## Untyped Dataset Operations (aka DataFrame Operations) - -DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/SparkDataFrame.html). - -As mentioned above, in Spark 2.0, DataFrames are just Dataset of `Row`s in Scala and Java API. These operations are also referred as "untyped transformations" in contrast to "typed transformations" come with strongly typed Scala/Java Datasets. - -Here we include some basic examples of structured data processing using Datasets: - -
    -
    -{% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} - -For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). - -In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$). -
    - -
    - -{% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} - -For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/java/org/apache/spark/sql/Dataset.html). - -In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html). -
    - -
    -In Python, it's possible to access a DataFrame's columns either by attribute -(`df.age`) or by indexing (`df['age']`). While the former is convenient for -interactive data exploration, users are highly encouraged to use the -latter form, which is future proof and won't break with column names that -are also attributes on the DataFrame class. - -{% include_example untyped_ops python/sql/basic.py %} -For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/python/pyspark.sql.html#pyspark.sql.DataFrame). - -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/python/pyspark.sql.html#module-pyspark.sql.functions). - -
    - -
    - -{% include_example untyped_ops r/RSparkSQLExample.R %} - -For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html). - -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/R/SparkDataFrame.html). - -
    - -
    - -## Running SQL Queries Programmatically - -
    -
    -The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. - -{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `Dataset`. - -{% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    -The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. - -{% include_example run_sql python/sql/basic.py %} -
    - -
    -The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. - -{% include_example run_sql r/RSparkSQLExample.R %} - -
    -
    - - -## Global Temporary View - -Temporary views in Spark SQL are session-scoped and will disappear if the session that creates it -terminates. If you want to have a temporary view that is shared among all sessions and keep alive -until the Spark application terminates, you can create a global temporary view. Global temporary -view is tied to a system preserved database `global_temp`, and we must use the qualified name to -refer it, e.g. `SELECT * FROM global_temp.view1`. - -
    -
    -{% include_example global_temp_view scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -{% include_example global_temp_view java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    -{% include_example global_temp_view python/sql/basic.py %} -
    - -
    - -{% highlight sql %} - -CREATE GLOBAL TEMPORARY VIEW temp_view AS SELECT a + 1, b * 2 FROM tbl - -SELECT * FROM global_temp.temp_view - -{% endhighlight %} - -
    -
    - - -## Creating Datasets - -Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use -a specialized [Encoder](api/scala/index.html#org.apache.spark.sql.Encoder) to serialize the objects -for processing or transmitting over the network. While both encoders and standard serialization are -responsible for turning an object into bytes, encoders are code generated dynamically and use a format -that allows Spark to perform many operations like filtering, sorting and hashing without deserializing -the bytes back into an object. - -
    -
    -{% include_example create_ds scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -{% include_example create_ds java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    -
    - -## Interoperating with RDDs - -Spark SQL supports two different methods for converting existing RDDs into Datasets. The first -method uses reflection to infer the schema of an RDD that contains specific types of objects. This -reflection-based approach leads to more concise code and works well when you already know the schema -while writing your Spark application. - -The second method for creating Datasets is through a programmatic interface that allows you to -construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows -you to construct Datasets when the columns and their types are not known until runtime. - -### Inferring the Schema Using Reflection -
    - -
    - -The Scala interface for Spark SQL supports automatically converting an RDD containing case classes -to a DataFrame. The case class -defines the schema of the table. The names of the arguments to the case class are read using -reflection and become the names of the columns. Case classes can also be nested or contain complex -types such as `Seq`s or `Array`s. This RDD can be implicitly converted to a DataFrame and then be -registered as a table. Tables can be used in subsequent SQL statements. - -{% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    - -Spark SQL supports automatically converting an RDD of -[JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) into a DataFrame. -The `BeanInfo`, obtained using reflection, defines the schema of the table. Currently, Spark SQL -does not support JavaBeans that contain `Map` field(s). Nested JavaBeans and `List` or `Array` -fields are supported though. You can create a JavaBean by creating a class that implements -Serializable and has getters and setters for all of its fields. - -{% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    - -Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of -key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, -and the types are inferred by sampling the whole dataset, similar to the inference that is performed on JSON files. - -{% include_example schema_inferring python/sql/basic.py %} -
    - -
    - -### Programmatically Specifying the Schema - -
    - -
    - -When case classes cannot be defined ahead of time (for example, -the structure of records is encoded in a string, or a text dataset will be parsed -and fields will be projected differently for different users), -a `DataFrame` can be created programmatically with three steps. - -1. Create an RDD of `Row`s from the original RDD; -2. Create the schema represented by a `StructType` matching the structure of -`Row`s in the RDD created in Step 1. -3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided -by `SparkSession`. - -For example: - -{% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    - -When JavaBean classes cannot be defined ahead of time (for example, -the structure of records is encoded in a string, or a text dataset will be parsed and -fields will be projected differently for different users), -a `Dataset` can be created programmatically with three steps. - -1. Create an RDD of `Row`s from the original RDD; -2. Create the schema represented by a `StructType` matching the structure of -`Row`s in the RDD created in Step 1. -3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided -by `SparkSession`. - -For example: - -{% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    - -When a dictionary of kwargs cannot be defined ahead of time (for example, -the structure of records is encoded in a string, or a text dataset will be parsed and -fields will be projected differently for different users), -a `DataFrame` can be created programmatically with three steps. - -1. Create an RDD of tuples or lists from the original RDD; -2. Create the schema represented by a `StructType` matching the structure of -tuples or lists in the RDD created in the step 1. -3. Apply the schema to the RDD via `createDataFrame` method provided by `SparkSession`. - -For example: - -{% include_example programmatic_schema python/sql/basic.py %} -
    - -
    - -## Aggregations - -The [built-in DataFrames functions](api/scala/index.html#org.apache.spark.sql.functions$) provide common -aggregations such as `count()`, `countDistinct()`, `avg()`, `max()`, `min()`, etc. -While those functions are designed for DataFrames, Spark SQL also has type-safe versions for some of them in -[Scala](api/scala/index.html#org.apache.spark.sql.expressions.scalalang.typed$) and -[Java](api/java/org/apache/spark/sql/expressions/javalang/typed.html) to work with strongly typed Datasets. -Moreover, users are not limited to the predefined aggregate functions and can create their own. - -### Untyped User-Defined Aggregate Functions -Users have to extend the [UserDefinedAggregateFunction](api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction) -abstract class to implement a custom untyped aggregate function. For example, a user-defined average -can look like: - -
    -
    -{% include_example untyped_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%} -
    -
    -{% include_example untyped_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%} -
    -
    - -### Type-Safe User-Defined Aggregate Functions - -User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) abstract class. -For example, a type-safe user-defined average can look like: - -
    -
    -{% include_example typed_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%} -
    -
    -{% include_example typed_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%} -
    -
    - -# Data Sources - -Spark SQL supports operating on a variety of data sources through the DataFrame interface. -A DataFrame can be operated on using relational transformations and can also be used to create a temporary view. -Registering a DataFrame as a temporary view allows you to run SQL queries over its data. This section -describes the general methods for loading and saving data using the Spark Data Sources and then -goes into specific options that are available for the built-in data sources. - -## Generic Load/Save Functions - -In the simplest form, the default data source (`parquet` unless otherwise configured by -`spark.sql.sources.default`) will be used for all operations. - -
    -
    -{% include_example generic_load_save_functions scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example generic_load_save_functions java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    - -{% include_example generic_load_save_functions python/sql/datasource.py %} -
    - -
    - -{% include_example generic_load_save_functions r/RSparkSQLExample.R %} - -
    -
    - -### Manually Specifying Options - -You can also manually specify the data source that will be used along with any extra options -that you would like to pass to the data source. Data sources are specified by their fully qualified -name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use their short -names (`json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`). DataFrames loaded from any data -source type can be converted into other types using this syntax. - -To load a JSON file you can use: - -
    -
    -{% include_example manual_load_options scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example manual_load_options java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example manual_load_options python/sql/datasource.py %} -
    - -
    -{% include_example manual_load_options r/RSparkSQLExample.R %} -
    -
    - -To load a CSV file you can use: - -
    -
    -{% include_example manual_load_options_csv scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example manual_load_options_csv java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example manual_load_options_csv python/sql/datasource.py %} -
    - -
    -{% include_example manual_load_options_csv r/RSparkSQLExample.R %} - -
    -
    - -### Run SQL on files directly - -Instead of using read API to load a file into DataFrame and query it, you can also query that -file directly with SQL. - -
    -
    -{% include_example direct_sql scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example direct_sql java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example direct_sql python/sql/datasource.py %} -
    - -
    -{% include_example direct_sql r/RSparkSQLExample.R %} - -
    -
    - -### Save Modes - -Save operations can optionally take a `SaveMode`, that specifies how to handle existing data if -present. It is important to realize that these save modes do not utilize any locking and are not -atomic. Additionally, when performing an `Overwrite`, the data will be deleted before writing out the -new data. - - - - - - - - - - - - - - - - - - - - - - - -
    Scala/JavaAny LanguageMeaning
    SaveMode.ErrorIfExists (default)"error" or "errorifexists" (default) - When saving a DataFrame to a data source, if data already exists, - an exception is expected to be thrown. -
    SaveMode.Append"append" - When saving a DataFrame to a data source, if data/table already exists, - contents of the DataFrame are expected to be appended to existing data. -
    SaveMode.Overwrite"overwrite" - Overwrite mode means that when saving a DataFrame to a data source, - if data/table already exists, existing data is expected to be overwritten by the contents of - the DataFrame. -
    SaveMode.Ignore"ignore" - Ignore mode means that when saving a DataFrame to a data source, if data already exists, - the save operation is expected to not save the contents of the DataFrame and to not - change the existing data. This is similar to a CREATE TABLE IF NOT EXISTS in SQL. -
    - -### Saving to Persistent Tables - -`DataFrames` can also be saved as persistent tables into Hive metastore using the `saveAsTable` -command. Notice that an existing Hive deployment is not necessary to use this feature. Spark will create a -default local Hive metastore (using Derby) for you. Unlike the `createOrReplaceTempView` command, -`saveAsTable` will materialize the contents of the DataFrame and create a pointer to the data in the -Hive metastore. Persistent tables will still exist even after your Spark program has restarted, as -long as you maintain your connection to the same metastore. A DataFrame for a persistent table can -be created by calling the `table` method on a `SparkSession` with the name of the table. - -For file-based data source, e.g. text, parquet, json, etc. you can specify a custom table path via the -`path` option, e.g. `df.write.option("path", "/some/path").saveAsTable("t")`. When the table is dropped, -the custom table path will not be removed and the table data is still there. If no custom table path is -specified, Spark will write data to a default table path under the warehouse directory. When the table is -dropped, the default table path will be removed too. - -Starting from Spark 2.1, persistent datasource tables have per-partition metadata stored in the Hive metastore. This brings several benefits: - -- Since the metastore can return only necessary partitions for a query, discovering all the partitions on the first query to the table is no longer needed. -- Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. - -Note that partition information is not gathered by default when creating external datasource tables (those with a `path` option). To sync the partition information in the metastore, you can invoke `MSCK REPAIR TABLE`. - -### Bucketing, Sorting and Partitioning - -For file-based data source, it is also possible to bucket and sort or partition the output. -Bucketing and sorting are applicable only to persistent tables: - -
    - -
    -{% include_example write_sorting_and_bucketing scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example write_sorting_and_bucketing java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example write_sorting_and_bucketing python/sql/datasource.py %} -
    - -
    - -{% highlight sql %} - -CREATE TABLE users_bucketed_by_name( - name STRING, - favorite_color STRING, - favorite_numbers array -) USING parquet -CLUSTERED BY(name) INTO 42 BUCKETS; - -{% endhighlight %} - -
    - -
    - -while partitioning can be used with both `save` and `saveAsTable` when using the Dataset APIs. - - -
    - -
    -{% include_example write_partitioning scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example write_partitioning java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example write_partitioning python/sql/datasource.py %} -
    - -
    - -{% highlight sql %} - -CREATE TABLE users_by_favorite_color( - name STRING, - favorite_color STRING, - favorite_numbers array -) USING csv PARTITIONED BY(favorite_color); - -{% endhighlight %} - -
    - -
    - -It is possible to use both partitioning and bucketing for a single table: - -
    - -
    -{% include_example write_partition_and_bucket scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example write_partition_and_bucket java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example write_partition_and_bucket python/sql/datasource.py %} -
    - -
    - -{% highlight sql %} - -CREATE TABLE users_bucketed_and_partitioned( - name STRING, - favorite_color STRING, - favorite_numbers array -) USING parquet -PARTITIONED BY (favorite_color) -CLUSTERED BY(name) SORTED BY (favorite_numbers) INTO 42 BUCKETS; - -{% endhighlight %} - -
    - -
    - -`partitionBy` creates a directory structure as described in the [Partition Discovery](#partition-discovery) section. -Thus, it has limited applicability to columns with high cardinality. In contrast - `bucketBy` distributes -data across a fixed number of buckets and can be used when a number of unique values is unbounded. - -## Parquet Files - -[Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. -Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema -of the original data. When writing Parquet files, all columns are automatically converted to be nullable for -compatibility reasons. - -### Loading Data Programmatically - -Using the data from the above example: - -
    - -
    -{% include_example basic_parquet_example scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example basic_parquet_example java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    - -{% include_example basic_parquet_example python/sql/datasource.py %} -
    - -
    - -{% include_example basic_parquet_example r/RSparkSQLExample.R %} - -
    - -
    - -{% highlight sql %} - -CREATE TEMPORARY VIEW parquetTable -USING org.apache.spark.sql.parquet -OPTIONS ( - path "examples/src/main/resources/people.parquet" -) - -SELECT * FROM parquetTable - -{% endhighlight %} - -
    - -
    - -### Partition Discovery - -Table partitioning is a common optimization approach used in systems like Hive. In a partitioned -table, data are usually stored in different directories, with partitioning column values encoded in -the path of each partition directory. All built-in file sources (including Text/CSV/JSON/ORC/Parquet) -are able to discover and infer partitioning information automatically. -For example, we can store all our previously used -population data into a partitioned table using the following directory structure, with two extra -columns, `gender` and `country` as partitioning columns: - -{% highlight text %} - -path -└── to - └── table - ├── gender=male - │   ├── ... - │   │ - │   ├── country=US - │   │   └── data.parquet - │   ├── country=CN - │   │   └── data.parquet - │   └── ... - └── gender=female -    ├── ... -    │ -    ├── country=US -    │   └── data.parquet -    ├── country=CN -    │   └── data.parquet -    └── ... - -{% endhighlight %} - -By passing `path/to/table` to either `SparkSession.read.parquet` or `SparkSession.read.load`, Spark SQL -will automatically extract the partitioning information from the paths. -Now the schema of the returned DataFrame becomes: - -{% highlight text %} - -root -|-- name: string (nullable = true) -|-- age: long (nullable = true) -|-- gender: string (nullable = true) -|-- country: string (nullable = true) - -{% endhighlight %} - -Notice that the data types of the partitioning columns are automatically inferred. Currently, -numeric data types, date, timestamp and string type are supported. Sometimes users may not want -to automatically infer the data types of the partitioning columns. For these use cases, the -automatic type inference can be configured by -`spark.sql.sources.partitionColumnTypeInference.enabled`, which is default to `true`. When type -inference is disabled, string type will be used for the partitioning columns. - -Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths -by default. For the above example, if users pass `path/to/table/gender=male` to either -`SparkSession.read.parquet` or `SparkSession.read.load`, `gender` will not be considered as a -partitioning column. If users need to specify the base path that partition discovery -should start with, they can set `basePath` in the data source options. For example, -when `path/to/table/gender=male` is the path of the data and -users set `basePath` to `path/to/table/`, `gender` will be a partitioning column. - -### Schema Merging - -Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with -a simple schema, and gradually add more columns to the schema as needed. In this way, users may end -up with multiple Parquet files with different but mutually compatible schemas. The Parquet data -source is now able to automatically detect this case and merge schemas of all these files. - -Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we -turned it off by default starting from 1.5.0. You may enable it by - -1. setting data source option `mergeSchema` to `true` when reading Parquet files (as shown in the - examples below), or -2. setting the global SQL option `spark.sql.parquet.mergeSchema` to `true`. - -
    - -
    -{% include_example schema_merging scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example schema_merging java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    - -{% include_example schema_merging python/sql/datasource.py %} -
    - -
    - -{% include_example schema_merging r/RSparkSQLExample.R %} - -
    - -
    - -### Hive metastore Parquet table conversion - -When reading from and writing to Hive metastore Parquet tables, Spark SQL will try to use its own -Parquet support instead of Hive SerDe for better performance. This behavior is controlled by the -`spark.sql.hive.convertMetastoreParquet` configuration, and is turned on by default. - -#### Hive/Parquet Schema Reconciliation - -There are two key differences between Hive and Parquet from the perspective of table schema -processing. - -1. Hive is case insensitive, while Parquet is not -1. Hive considers all columns nullable, while nullability in Parquet is significant - -Due to this reason, we must reconcile Hive metastore schema with Parquet schema when converting a -Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are: - -1. Fields that have the same name in both schema must have the same data type regardless of - nullability. The reconciled field should have the data type of the Parquet side, so that - nullability is respected. - -1. The reconciled schema contains exactly those fields defined in Hive metastore schema. - - - Any fields that only appear in the Parquet schema are dropped in the reconciled schema. - - Any fields that only appear in the Hive metastore schema are added as nullable field in the - reconciled schema. - -#### Metadata Refreshing - -Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table -conversion is enabled, metadata of those converted tables are also cached. If these tables are -updated by Hive or other external tools, you need to refresh them manually to ensure consistent -metadata. - -
    - -
    - -{% highlight scala %} -// spark is an existing SparkSession -spark.catalog.refreshTable("my_table") -{% endhighlight %} - -
    - -
    - -{% highlight java %} -// spark is an existing SparkSession -spark.catalog().refreshTable("my_table"); -{% endhighlight %} - -
    - -
    - -{% highlight python %} -# spark is an existing SparkSession -spark.catalog.refreshTable("my_table") -{% endhighlight %} - -
    - -
    - -{% highlight r %} -refreshTable("my_table") -{% endhighlight %} - -
    - -
    - -{% highlight sql %} -REFRESH TABLE my_table; -{% endhighlight %} - -
    - -
    - -### Configuration - -Configuration of Parquet can be done using the `setConf` method on `SparkSession` or by running -`SET key=value` commands using SQL. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.parquet.binaryAsStringfalse - Some other Parquet-producing systems, in particular Impala, Hive, and older versions of Spark SQL, do - not differentiate between binary data and strings when writing out the Parquet schema. This - flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems. -
    spark.sql.parquet.int96AsTimestamptrue - Some Parquet-producing systems, in particular Impala and Hive, store Timestamp into INT96. This - flag tells Spark SQL to interpret INT96 data as a timestamp to provide compatibility with these systems. -
    spark.sql.parquet.compression.codecsnappy - Sets the compression codec used when writing Parquet files. If either `compression` or - `parquet.compression` is specified in the table-specific options/properties, the precedence would be - `compression`, `parquet.compression`, `spark.sql.parquet.compression.codec`. Acceptable values include: - none, uncompressed, snappy, gzip, lzo, brotli, lz4, zstd. -
    spark.sql.parquet.filterPushdowntrueEnables Parquet filter push-down optimization when set to true.
    spark.sql.hive.convertMetastoreParquettrue - When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of the built in - support. -
    spark.sql.parquet.mergeSchemafalse -

    - When true, the Parquet data source merges schemas collected from all data files, otherwise the - schema is picked from the summary file or a random data file if no summary file is available. -

    -
    spark.sql.optimizer.metadataOnlytrue -

    - When true, enable the metadata-only query optimization that use the table's metadata to - produce the partition columns instead of table scans. It applies when all the columns scanned - are partition columns and the query has an aggregate operator that satisfies distinct - semantics. -

    -
    spark.sql.parquet.writeLegacyFormatfalse - If true, data will be written in a way of Spark 1.4 and earlier. For example, decimal values - will be written in Apache Parquet's fixed-length byte array format, which other systems such as - Apache Hive and Apache Impala use. If false, the newer format in Parquet will be used. For - example, decimals will be written in int-based format. If Parquet output is intended for use - with systems that do not support this newer format, set to true. -
    - -## ORC Files - -Since Spark 2.3, Spark supports a vectorized ORC reader with a new ORC file format for ORC files. -To do that, the following configurations are newly added. The vectorized reader is used for the -native ORC tables (e.g., the ones created using the clause `USING ORC`) when `spark.sql.orc.impl` -is set to `native` and `spark.sql.orc.enableVectorizedReader` is set to `true`. For the Hive ORC -serde tables (e.g., the ones created using the clause `USING HIVE OPTIONS (fileFormat 'ORC')`), -the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also set to `true`. - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.orc.implnativeThe name of ORC implementation. It can be one of native and hive. native means the native ORC support that is built on Apache ORC 1.4. `hive` means the ORC library in Hive 1.2.1.
    spark.sql.orc.enableVectorizedReadertrueEnables vectorized orc decoding in native implementation. If false, a new non-vectorized ORC reader is used in native implementation. For hive implementation, this is ignored.
    - -## JSON Datasets -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset[Row]`. -This conversion can be done using `SparkSession.read.json()` on either a `Dataset[String]`, -or a JSON file. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set the `multiLine` option to `true`. - -{% include_example json_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset`. -This conversion can be done using `SparkSession.read().json()` on either a `Dataset`, -or a JSON file. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set the `multiLine` option to `true`. - -{% include_example json_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using `SparkSession.read.json` on a JSON file. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set the `multiLine` parameter to `True`. - -{% include_example json_dataset python/sql/datasource.py %} -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. using -the `read.json()` function, which loads data from a directory of JSON files where each line of the -files is a JSON object. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set a named parameter `multiLine` to `TRUE`. - -{% include_example json_dataset r/RSparkSQLExample.R %} - -
    - -
    - -{% highlight sql %} - -CREATE TEMPORARY VIEW jsonTable -USING org.apache.spark.sql.json -OPTIONS ( - path "examples/src/main/resources/people.json" -) - -SELECT * FROM jsonTable - -{% endhighlight %} - -
    - -
    - -## Hive Tables - -Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). -However, since Hive has a large number of dependencies, these dependencies are not included in the -default Spark distribution. If Hive dependencies can be found on the classpath, Spark will load them -automatically. Note that these Hive dependencies must also be present on all of the worker nodes, as -they will need access to the Hive serialization and deserialization libraries (SerDes) in order to -access data stored in Hive. - -Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (for security configuration), -and `hdfs-site.xml` (for HDFS configuration) file in `conf/`. - -When working with Hive, one must instantiate `SparkSession` with Hive support, including -connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. -Users who do not have an existing Hive deployment can still enable Hive support. When not configured -by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and -creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the Spark application is started. Note that -the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. -Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the Spark application. - -
    - -
    -{% include_example spark_hive scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %} -
    - -
    -{% include_example spark_hive java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java %} -
    - -
    -{% include_example spark_hive python/sql/hive.py %} -
    - -
    - -When working with Hive one must instantiate `SparkSession` with Hive support. This -adds support for finding tables in the MetaStore and writing queries using HiveQL. - -{% include_example spark_hive r/RSparkSQLExample.R %} - -
    -
    - -### Specifying storage format for Hive tables - -When you create a Hive table, you need to define how this table should read/write data from/to file system, -i.e. the "input format" and "output format". You also need to define how this table should deserialize the data -to rows, or serialize rows to data, i.e. the "serde". The following options can be used to specify the storage -format("serde", "input format", "output format"), e.g. `CREATE TABLE src(id int) USING hive OPTIONS(fileFormat 'parquet')`. -By default, we will read the table files as plain text. Note that, Hive storage handler is not supported yet when -creating table, you can create a table using storage handler at Hive side, and use Spark SQL to read it. - - - - - - - - - - - - - - - - - - - - - - -
    Property NameMeaning
    fileFormat - A fileFormat is kind of a package of storage format specifications, including "serde", "input format" and - "output format". Currently we support 6 fileFormats: 'sequencefile', 'rcfile', 'orc', 'parquet', 'textfile' and 'avro'. -
    inputFormat, outputFormat - These 2 options specify the name of a corresponding `InputFormat` and `OutputFormat` class as a string literal, - e.g. `org.apache.hadoop.hive.ql.io.orc.OrcInputFormat`. These 2 options must be appeared in pair, and you can not - specify them if you already specified the `fileFormat` option. -
    serde - This option specifies the name of a serde class. When the `fileFormat` option is specified, do not specify this option - if the given `fileFormat` already include the information of serde. Currently "sequencefile", "textfile" and "rcfile" - don't include the serde information and you can use this option with these 3 fileFormats. -
    fieldDelim, escapeDelim, collectionDelim, mapkeyDelim, lineDelim - These options can only be used with "textfile" fileFormat. They define how to read delimited files into rows. -
    - -All other properties defined with `OPTIONS` will be regarded as Hive serde properties. - -### Interacting with Different Versions of Hive Metastore - -One of the most important pieces of Spark SQL's Hive support is interaction with Hive metastore, -which enables Spark SQL to access metadata of Hive tables. Starting from Spark 1.4.0, a single binary -build of Spark SQL can be used to query different versions of Hive metastores, using the configuration described below. -Note that independent of the version of Hive that is being used to talk to the metastore, internally Spark SQL -will compile against Hive 1.2.1 and use those classes for internal execution (serdes, UDFs, UDAFs, etc). - -The following options can be used to configure the version of Hive that is used to retrieve metadata: - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.hive.metastore.version1.2.1 - Version of the Hive metastore. Available - options are 0.12.0 through 2.3.3. -
    spark.sql.hive.metastore.jarsbuiltin - Location of the jars that should be used to instantiate the HiveMetastoreClient. This - property can be one of three options: -
      -
    1. builtin
    2. - Use Hive 1.2.1, which is bundled with the Spark assembly when -Phive is - enabled. When this option is chosen, spark.sql.hive.metastore.version must be - either 1.2.1 or not defined. -
    3. maven
    4. - Use Hive jars of specified version downloaded from Maven repositories. This configuration - is not generally recommended for production deployments. -
    5. A classpath in the standard format for the JVM. This classpath must include all of Hive - and its dependencies, including the correct version of Hadoop. These jars only need to be - present on the driver, but if you are running in yarn cluster mode then you must ensure - they are packaged with your application.
    6. -
    -
    spark.sql.hive.metastore.sharedPrefixescom.mysql.jdbc,
    org.postgresql,
    com.microsoft.sqlserver,
    oracle.jdbc
    -

    - A comma-separated list of class prefixes that should be loaded using the classloader that is - shared between Spark SQL and a specific version of Hive. An example of classes that should - be shared is JDBC drivers that are needed to talk to the metastore. Other classes that need - to be shared are those that interact with classes that are already shared. For example, - custom appenders that are used by log4j. -

    -
    spark.sql.hive.metastore.barrierPrefixes(empty) -

    - A comma separated list of class prefixes that should explicitly be reloaded for each version - of Hive that Spark SQL is communicating with. For example, Hive UDFs that are declared in a - prefix that typically would be shared (i.e. org.apache.spark.*). -

    -
    - - -## JDBC To Other Databases - -Spark SQL also includes a data source that can read data from other databases using JDBC. This -functionality should be preferred over using [JdbcRDD](api/scala/index.html#org.apache.spark.rdd.JdbcRDD). -This is because the results are returned -as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources. -The JDBC data source is also easier to use from Java or Python as it does not require the user to -provide a ClassTag. -(Note that this is different than the Spark SQL JDBC server, which allows other applications to -run queries using Spark SQL). - -To get started you will need to include the JDBC driver for your particular database on the -spark classpath. For example, to connect to postgres from the Spark Shell you would run the -following command: - -{% highlight bash %} -bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jar -{% endhighlight %} - -Tables from the remote database can be loaded as a DataFrame or Spark SQL temporary view using -the Data Sources API. Users can specify the JDBC connection properties in the data source options. -user and password are normally provided as connection properties for -logging into the data sources. In addition to the connection properties, Spark also supports -the following case-insensitive options: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameMeaning
    url - The JDBC URL to connect to. The source-specific connection properties may be specified in the URL. e.g., jdbc:postgresql://localhost/test?user=fred&password=secret -
    dbtable - The JDBC table that should be read from or written into. Note that when using it in the read - path anything that is valid in a FROM clause of a SQL query can be used. - For example, instead of a full table you could also use a subquery in parentheses. It is not - allowed to specify `dbtable` and `query` options at the same time. -
    query - A query that will be used to read data into Spark. The specified query will be parenthesized and used - as a subquery in the FROM clause. Spark will also assign an alias to the subquery clause. - As an example, spark will issue a query of the following form to the JDBC Source.

    - SELECT <columns> FROM (<user_specified_query>) spark_gen_alias

    - Below are couple of restrictions while using this option.
    -
      -
    1. It is not allowed to specify `dbtable` and `query` options at the same time.
    2. -
    3. It is not allowed to spcify `query` and `partitionColumn` options at the same time. When specifying - `partitionColumn` option is required, the subquery can be specified using `dbtable` option instead and - partition columns can be qualified using the subquery alias provided as part of `dbtable`.
      - Example:
      - - spark.read.format("jdbc")
      -    .option("dbtable", "(select c1, c2 from t1) as subq")
      -    .option("partitionColumn", "subq.c1"
      -    .load() -
    4. -
    -
    driver - The class name of the JDBC driver to use to connect to this URL. -
    partitionColumn, lowerBound, upperBound - These options must all be specified if any of them is specified. In addition, - numPartitions must be specified. They describe how to partition the table when - reading in parallel from multiple workers. - partitionColumn must be a numeric, date, or timestamp column from the table in question. - Notice that lowerBound and upperBound are just used to decide the - partition stride, not for filtering the rows in table. So all rows in the table will be - partitioned and returned. This option applies only to reading. -
    numPartitions - The maximum number of partitions that can be used for parallelism in table reading and - writing. This also determines the maximum number of concurrent JDBC connections. - If the number of partitions to write exceeds this limit, we decrease it to this limit by - calling coalesce(numPartitions) before writing. -
    queryTimeout - The number of seconds the driver will wait for a Statement object to execute to the given - number of seconds. Zero means there is no limit. In the write path, this option depends on - how JDBC drivers implement the API setQueryTimeout, e.g., the h2 JDBC driver - checks the timeout of each query instead of an entire JDBC batch. - It defaults to 0. -
    fetchsize - The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). This option applies only to reading. -
    batchsize - The JDBC batch size, which determines how many rows to insert per round trip. This can help performance on JDBC drivers. This option applies only to writing. It defaults to 1000. -
    isolationLevel - The transaction isolation level, which applies to current connection. It can be one of NONE, READ_COMMITTED, READ_UNCOMMITTED, REPEATABLE_READ, or SERIALIZABLE, corresponding to standard transaction isolation levels defined by JDBC's Connection object, with default of READ_UNCOMMITTED. This option applies only to writing. Please refer the documentation in java.sql.Connection. -
    sessionInitStatement - After each database session is opened to the remote DB and before starting to read data, this option executes a custom SQL statement (or a PL/SQL block). Use this to implement session initialization code. Example: option("sessionInitStatement", """BEGIN execute immediate 'alter session set "_serial_direct_read"=true'; END;""") -
    truncate - This is a JDBC writer related option. When SaveMode.Overwrite is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g., indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to false. This option applies only to writing. -
    cascadeTruncate - This is a JDBC writer related option. If enabled and supported by the JDBC database (PostgreSQL and Oracle at the moment), this options allows execution of a TRUNCATE TABLE t CASCADE (in the case of PostgreSQL a TRUNCATE TABLE ONLY t CASCADE is executed to prevent inadvertently truncating descendant tables). This will affect other tables, and thus should be used with care. This option applies only to writing. It defaults to the default cascading truncate behaviour of the JDBC database in question, specified in the isCascadeTruncate in each JDBCDialect. -
    createTableOptions - This is a JDBC writer related option. If specified, this option allows setting of database-specific table and partition options when creating a table (e.g., CREATE TABLE t (name string) ENGINE=InnoDB.). This option applies only to writing. -
    createTableColumnTypes - The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: "name CHAR(64), comments VARCHAR(1024)"). The specified types should be valid spark sql data types. This option applies only to writing. -
    customSchema - The custom schema to use for reading data from JDBC connectors. For example, "id DECIMAL(38, 0), name STRING". You can also specify partial fields, and the others use the default type mapping. For example, "id DECIMAL(38, 0)". The column names should be identical to the corresponding column names of JDBC table. Users can specify the corresponding data types of Spark SQL instead of using the defaults. This option applies only to reading. -
    pushDownPredicate - The option to enable or disable predicate push-down into the JDBC data source. The default value is true, in which case Spark will push down filters to the JDBC data source as much as possible. Otherwise, if set to false, no filter will be pushed down to the JDBC data source and thus all filters will be handled by Spark. Predicate push-down is usually turned off when the predicate filtering is performed faster by Spark than by the JDBC data source. -
    - -
    - -
    -{% include_example jdbc_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example jdbc_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example jdbc_dataset python/sql/datasource.py %} -
    - -
    -{% include_example jdbc_dataset r/RSparkSQLExample.R %} -
    - -
    - -{% highlight sql %} - -CREATE TEMPORARY VIEW jdbcTable -USING org.apache.spark.sql.jdbc -OPTIONS ( - url "jdbc:postgresql:dbserver", - dbtable "schema.tablename", - user 'username', - password 'password' -) - -INSERT INTO TABLE jdbcTable -SELECT * FROM resultTable -{% endhighlight %} - -
    -
    - -## Avro Files -See the [Apache Avro Data Source Guide](avro-data-source-guide.html). - -## Troubleshooting - - * The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java's DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs. - * Some databases, such as H2, convert all names to upper case. You'll need to use upper case to refer to those names in Spark SQL. - * Users can specify vendor-specific JDBC connection properties in the data source options to do special treatment. For example, `spark.read.format("jdbc").option("url", oracleJdbcUrl).option("oracle.jdbc.mapDateToTimestamp", "false")`. `oracle.jdbc.mapDateToTimestamp` defaults to true, users often need to disable this flag to avoid Oracle date being resolved as timestamp. - -# Performance Tuning - -For some workloads, it is possible to improve performance by either caching data in memory, or by -turning on some experimental options. - -## Caching Data In Memory - -Spark SQL can cache tables using an in-memory columnar format by calling `spark.catalog.cacheTable("tableName")` or `dataFrame.cache()`. -Then Spark SQL will scan only required columns and will automatically tune compression to minimize -memory usage and GC pressure. You can call `spark.catalog.uncacheTable("tableName")` to remove the table from memory. - -Configuration of in-memory caching can be done using the `setConf` method on `SparkSession` or by running -`SET key=value` commands using SQL. - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.inMemoryColumnarStorage.compressedtrue - When set to true Spark SQL will automatically select a compression codec for each column based - on statistics of the data. -
    spark.sql.inMemoryColumnarStorage.batchSize10000 - Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization - and compression, but risk OOMs when caching data. -
    - -## Other Configuration Options - -The following options can also be used to tune the performance of query execution. It is possible -that these options will be deprecated in future release as more optimizations are performed automatically. - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.files.maxPartitionBytes134217728 (128 MB) - The maximum number of bytes to pack into a single partition when reading files. -
    spark.sql.files.openCostInBytes4194304 (4 MB) - The estimated cost to open a file, measured by the number of bytes could be scanned in the same - time. This is used when putting multiple files into a partition. It is better to over estimated, - then the partitions with small files will be faster than partitions with bigger files (which is - scheduled first). -
    spark.sql.broadcastTimeout300 -

    - Timeout in seconds for the broadcast wait time in broadcast joins -

    -
    spark.sql.autoBroadcastJoinThreshold10485760 (10 MB) - Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when - performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently - statistics are only supported for Hive Metastore tables where the command - ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been run. -
    spark.sql.shuffle.partitions200 - Configures the number of partitions to use when shuffling data for joins or aggregations. -
    - -## Broadcast Hint for SQL Queries - -The `BROADCAST` hint guides Spark to broadcast each specified table when joining them with another table or view. -When Spark deciding the join methods, the broadcast hash join (i.e., BHJ) is preferred, -even if the statistics is above the configuration `spark.sql.autoBroadcastJoinThreshold`. -When both sides of a join are specified, Spark broadcasts the one having the lower statistics. -Note Spark does not guarantee BHJ is always chosen, since not all cases (e.g. full outer join) -support BHJ. When the broadcast nested loop join is selected, we still respect the hint. - -
    - -
    - -{% highlight scala %} -import org.apache.spark.sql.functions.broadcast -broadcast(spark.table("src")).join(spark.table("records"), "key").show() -{% endhighlight %} - -
    - -
    - -{% highlight java %} -import static org.apache.spark.sql.functions.broadcast; -broadcast(spark.table("src")).join(spark.table("records"), "key").show(); -{% endhighlight %} - -
    - -
    - -{% highlight python %} -from pyspark.sql.functions import broadcast -broadcast(spark.table("src")).join(spark.table("records"), "key").show() -{% endhighlight %} - -
    - -
    - -{% highlight r %} -src <- sql("SELECT * FROM src") -records <- sql("SELECT * FROM records") -head(join(broadcast(src), records, src$key == records$key)) -{% endhighlight %} - -
    - -
    - -{% highlight sql %} --- We accept BROADCAST, BROADCASTJOIN and MAPJOIN for broadcast hint -SELECT /*+ BROADCAST(r) */ * FROM records r JOIN src s ON r.key = s.key -{% endhighlight %} - -
    -
    - -# Distributed SQL Engine - -Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. -In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, -without the need to write any code. - -## Running the Thrift JDBC/ODBC server - -The Thrift JDBC/ODBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) -in Hive 1.2.1 You can test the JDBC server with the beeline script that comes with either Spark or Hive 1.2.1. - -To start the JDBC/ODBC server, run the following in the Spark directory: - - ./sbin/start-thriftserver.sh - -This script accepts all `bin/spark-submit` command line options, plus a `--hiveconf` option to -specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` for a complete list of -all available options. By default, the server listens on localhost:10000. You may override this -behaviour via either environment variables, i.e.: - -{% highlight bash %} -export HIVE_SERVER2_THRIFT_PORT= -export HIVE_SERVER2_THRIFT_BIND_HOST= -./sbin/start-thriftserver.sh \ - --master \ - ... -{% endhighlight %} - -or system properties: - -{% highlight bash %} -./sbin/start-thriftserver.sh \ - --hiveconf hive.server2.thrift.port= \ - --hiveconf hive.server2.thrift.bind.host= \ - --master - ... -{% endhighlight %} - -Now you can use beeline to test the Thrift JDBC/ODBC server: - - ./bin/beeline - -Connect to the JDBC/ODBC server in beeline with: - - beeline> !connect jdbc:hive2://localhost:10000 - -Beeline will ask you for a username and password. In non-secure mode, simply enter the username on -your machine and a blank password. For secure mode, please follow the instructions given in the -[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients). - -Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. - -You may also use the beeline script that comes with Hive. - -Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. -Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: - - hive.server2.transport.mode - Set this to value: http - hive.server2.thrift.http.port - HTTP port number to listen on; default is 10001 - hive.server2.http.endpoint - HTTP endpoint; default is cliservice - -To test, use beeline to connect to the JDBC/ODBC server in http mode with: - - beeline> !connect jdbc:hive2://:/?hive.server2.transport.mode=http;hive.server2.thrift.http.path= - - -## Running the Spark SQL CLI - -The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute -queries input from the command line. Note that the Spark SQL CLI cannot talk to the Thrift JDBC server. - -To start the Spark SQL CLI, run the following in the Spark directory: - - ./bin/spark-sql - -Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. -You may run `./bin/spark-sql --help` for a complete list of all available -options. - -# PySpark Usage Guide for Pandas with Apache Arrow - -## Apache Arrow in Spark - -Apache Arrow is an in-memory columnar data format that is used in Spark to efficiently transfer -data between JVM and Python processes. This currently is most beneficial to Python users that -work with Pandas/NumPy data. Its usage is not automatic and might require some minor -changes to configuration or code to take full advantage and ensure compatibility. This guide will -give a high-level description of how to use Arrow in Spark and highlight any differences when -working with Arrow-enabled data. - -### Ensure PyArrow Installed - -If you install PySpark using pip, then PyArrow can be brought in as an extra dependency of the -SQL module with the command `pip install pyspark[sql]`. Otherwise, you must ensure that PyArrow -is installed and available on all cluster nodes. The current supported version is 0.8.0. -You can install using pip or conda from the conda-forge channel. See PyArrow -[installation](https://arrow.apache.org/docs/python/install.html) for details. - -## Enabling for Conversion to/from Pandas - -Arrow is available as an optimization when converting a Spark DataFrame to a Pandas DataFrame -using the call `toPandas()` and when creating a Spark DataFrame from a Pandas DataFrame with -`createDataFrame(pandas_df)`. To use Arrow when executing these calls, users need to first set -the Spark configuration 'spark.sql.execution.arrow.enabled' to 'true'. This is disabled by default. - -In addition, optimizations enabled by 'spark.sql.execution.arrow.enabled' could fallback automatically -to non-Arrow optimization implementation if an error occurs before the actual computation within Spark. -This can be controlled by 'spark.sql.execution.arrow.fallback.enabled'. - -
    -
    -{% include_example dataframe_with_arrow python/sql/arrow.py %} -
    -
    - -Using the above optimizations with Arrow will produce the same results as when Arrow is not -enabled. Note that even with Arrow, `toPandas()` results in the collection of all records in the -DataFrame to the driver program and should be done on a small subset of the data. Not all Spark -data types are currently supported and an error can be raised if a column has an unsupported type, -see [Supported SQL Types](#supported-sql-types). If an error occurs during `createDataFrame()`, -Spark will fall back to create the DataFrame without Arrow. - -## Pandas UDFs (a.k.a. Vectorized UDFs) - -Pandas UDFs are user defined functions that are executed by Spark using Arrow to transfer data and -Pandas to work with the data. A Pandas UDF is defined using the keyword `pandas_udf` as a decorator -or to wrap the function, no additional configuration is required. Currently, there are two types of -Pandas UDF: Scalar and Grouped Map. - -### Scalar - -Scalar Pandas UDFs are used for vectorizing scalar operations. They can be used with functions such -as `select` and `withColumn`. The Python function should take `pandas.Series` as inputs and return -a `pandas.Series` of the same length. Internally, Spark will execute a Pandas UDF by splitting -columns into batches and calling the function for each batch as a subset of the data, then -concatenating the results together. - -The following example shows how to create a scalar Pandas UDF that computes the product of 2 columns. - -
    -
    -{% include_example scalar_pandas_udf python/sql/arrow.py %} -
    -
    - -### Grouped Map -Grouped map Pandas UDFs are used with `groupBy().apply()` which implements the "split-apply-combine" pattern. -Split-apply-combine consists of three steps: -* Split the data into groups by using `DataFrame.groupBy`. -* Apply a function on each group. The input and output of the function are both `pandas.DataFrame`. The - input data contains all the rows and columns for each group. -* Combine the results into a new `DataFrame`. - -To use `groupBy().apply()`, the user needs to define the following: -* A Python function that defines the computation for each group. -* A `StructType` object or a string that defines the schema of the output `DataFrame`. - -The column labels of the returned `pandas.DataFrame` must either match the field names in the -defined output schema if specified as strings, or match the field data types by position if not -strings, e.g. integer indices. See [pandas.DataFrame](https://pandas.pydata.org/pandas-docs/stable/generated/pandas.DataFrame.html#pandas.DataFrame) -on how to label columns when constructing a `pandas.DataFrame`. - -Note that all data for a group will be loaded into memory before the function is applied. This can -lead to out of memory exceptions, especially if the group sizes are skewed. The configuration for -[maxRecordsPerBatch](#setting-arrow-batch-size) is not applied on groups and it is up to the user -to ensure that the grouped data will fit into the available memory. - -The following example shows how to use `groupby().apply()` to subtract the mean from each value in the group. - -
    -
    -{% include_example grouped_map_pandas_udf python/sql/arrow.py %} -
    -
    - -For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) and -[`pyspark.sql.GroupedData.apply`](api/python/pyspark.sql.html#pyspark.sql.GroupedData.apply). - -### Grouped Aggregate - -Grouped aggregate Pandas UDFs are similar to Spark aggregate functions. Grouped aggregate Pandas UDFs are used with `groupBy().agg()` and -[`pyspark.sql.Window`](api/python/pyspark.sql.html#pyspark.sql.Window). It defines an aggregation from one or more `pandas.Series` -to a scalar value, where each `pandas.Series` represents a column within the group or window. - -Note that this type of UDF does not support partial aggregation and all data for a group or window will be loaded into memory. Also, -only unbounded window is supported with Grouped aggregate Pandas UDFs currently. - -The following example shows how to use this type of UDF to compute mean with groupBy and window operations: - -
    -
    -{% include_example grouped_agg_pandas_udf python/sql/arrow.py %} -
    -
    - -For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) - -## Usage Notes - -### Supported SQL Types - -Currently, all Spark SQL data types are supported by Arrow-based conversion except `BinaryType`, `MapType`, -`ArrayType` of `TimestampType`, and nested `StructType`. - -### Setting Arrow Batch Size - -Data partitions in Spark are converted into Arrow record batches, which can temporarily lead to -high memory usage in the JVM. To avoid possible out of memory exceptions, the size of the Arrow -record batches can be adjusted by setting the conf "spark.sql.execution.arrow.maxRecordsPerBatch" -to an integer that will determine the maximum number of rows for each batch. The default value is -10,000 records per batch. If the number of columns is large, the value should be adjusted -accordingly. Using this limit, each data partition will be made into 1 or more record batches for -processing. - -### Timestamp with Time Zone Semantics - -Spark internally stores timestamps as UTC values, and timestamp data that is brought in without -a specified time zone is converted as local time to UTC with microsecond resolution. When timestamp -data is exported or displayed in Spark, the session time zone is used to localize the timestamp -values. The session time zone is set with the configuration 'spark.sql.session.timeZone' and will -default to the JVM system local time zone if not set. Pandas uses a `datetime64` type with nanosecond -resolution, `datetime64[ns]`, with optional time zone on a per-column basis. - -When timestamp data is transferred from Spark to Pandas it will be converted to nanoseconds -and each column will be converted to the Spark session time zone then localized to that time -zone, which removes the time zone and displays values as local time. This will occur -when calling `toPandas()` or `pandas_udf` with timestamp columns. - -When timestamp data is transferred from Pandas to Spark, it will be converted to UTC microseconds. This -occurs when calling `createDataFrame` with a Pandas DataFrame or when returning a timestamp from a -`pandas_udf`. These conversions are done automatically to ensure Spark will have data in the -expected format, so it is not necessary to do any of these conversions yourself. Any nanosecond -values will be truncated. - -Note that a standard UDF (non-Pandas) will load timestamp data as Python datetime objects, which is -different than a Pandas timestamp. It is recommended to use Pandas time series functionality when -working with timestamps in `pandas_udf`s to get the best performance, see -[here](https://pandas.pydata.org/pandas-docs/stable/timeseries.html) for details. - -# Migration Guide - -## Upgrading From Spark SQL 2.3 to 2.4 - - - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. - - - - - - - - - - - - - - - - - - - - - - - - - -
    - Query - - Result Spark 2.3 or Prior - - Result Spark 2.4 - - Remarks -
    - SELECT
    array_contains(array(1), 1.34D);
    -
    - true - - false - - In Spark 2.4, left and right parameters are promoted to array(double) and double type respectively. -
    - SELECT
    array_contains(array(1), '1');
    -
    - true - - AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. - - Users can use explict cast -
    - SELECT
    array_contains(array(1), 'anystring');
    -
    - null - - AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. - - Users can use explict cast -
    - - - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. - - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. - - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. - - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. - - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. - - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. - - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. - - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, an column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. - - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. - - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. - - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. - - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. - - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. - - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. - - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. - - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. - - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. - - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. - - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. - - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. - - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. - - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. - -## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above - - - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. - -## Upgrading From Spark SQL 2.2 to 2.3 - - - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. - - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. - - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. - - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - InputA \ InputB - - NullType - - IntegerType - - LongType - - DecimalType(38,0)* - - DoubleType - - DateType - - TimestampType - - StringType -
    - NullType - NullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
    - IntegerType - IntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
    - LongType - LongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
    - DecimalType(38,0)* - DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
    - DoubleType - DoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
    - DateType - DateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
    - TimestampType - TimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
    - StringType - StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
    - - Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. - - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. - - In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration `spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See [SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details. - - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. - - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). - - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. - - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. - - - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive's new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes - - The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`). - - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. - - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible. - - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. - - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. - -## Upgrading From Spark SQL 2.1 to 2.2 - - - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. - - - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). - -## Upgrading From Spark SQL 2.0 to 2.1 - - - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. - - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. - - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. - - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. - - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. - - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. - -## Upgrading From Spark SQL 1.6 to 2.0 - - - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and - `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. - - - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for - `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed - transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., - `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in - Python and R is not a language feature, the concept of Dataset does not apply to these languages’ - APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the - single-node data frame notion in these languages. - - - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` - - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` - - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` - - - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. - - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` - in order to prevent accidental dropping the existing data in the user-provided locations. - That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. - Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. - Note that this is different from the Hive behavior. - - As a result, `DROP TABLE` statements on those tables will not remove the data. - - - `spark.sql.parquet.cacheMetadata` is no longer used. - See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details. - -## Upgrading From Spark SQL 1.5 to 1.6 - - - From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC - connection owns a copy of their own SQL configuration and temporary function registry. Cached - tables are still shared though. If you prefer to run the Thrift server in the old single-session - mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add - this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`: - - {% highlight bash %} - ./sbin/start-thriftserver.sh \ - --conf spark.sql.hive.thriftServer.singleSession=true \ - ... - {% endhighlight %} - - Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns - of the same name of a DataFrame. - - - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This - change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType - from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for - details. - -## Upgrading From Spark SQL 1.4 to 1.5 - - - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with - code generation for expression evaluation. These features can both be disabled by setting - `spark.sql.tungsten.enabled` to `false`. - - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting - `spark.sql.parquet.mergeSchema` to `true`. - - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or - access nested values. For example `df['table.column.nestedField']`. However, this means that if - your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). - - In-memory columnar storage partition pruning is on by default. It can be disabled by setting - `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. - - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum - precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now - used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. - - Timestamps are now stored at a precision of 1us, rather than 1ns - - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains - unchanged. - - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). - - JSON data source will not automatically load new files that are created by other applications - (i.e. files that are not inserted to the dataset through Spark SQL). - For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), - users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method - to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate - the DataFrame and the new DataFrame will include new files. - - DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name. - -## Upgrading from Spark SQL 1.3 to 1.4 - -#### DataFrame data reader/writer interface - -Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) -and writing data out (`DataFrame.write`), -and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). - -See the API docs for `SQLContext.read` ( - Scala, - Java, - Python -) and `DataFrame.write` ( - Scala, - Java, - Python -) more information. - - -#### DataFrame.groupBy retains grouping columns - -Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the -grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. - -
    -
    -{% highlight scala %} - -// In 1.3.x, in order for the grouping column "department" to show up, -// it must be included explicitly as part of the agg function call. -df.groupBy("department").agg($"department", max("age"), sum("expense")) - -// In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(max("age"), sum("expense")) - -// Revert to 1.3 behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false") - -{% endhighlight %} -
    - -
    -{% highlight java %} - -// In 1.3.x, in order for the grouping column "department" to show up, -// it must be included explicitly as part of the agg function call. -df.groupBy("department").agg(col("department"), max("age"), sum("expense")); - -// In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(max("age"), sum("expense")); - -// Revert to 1.3 behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false"); - -{% endhighlight %} -
    - -
    -{% highlight python %} - -import pyspark.sql.functions as func - -# In 1.3.x, in order for the grouping column "department" to show up, -# it must be included explicitly as part of the agg function call. -df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) - -# In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(func.max("age"), func.sum("expense")) - -# Revert to 1.3.x behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false") - -{% endhighlight %} -
    - -
    - - -#### Behavior change on DataFrame.withColumn - -Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added -as a new column with its specified name in the result DataFrame even if there may be any existing -columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different -name from names of all existing columns or replacing existing columns of the same name. - -Note that this change is only for Scala API, not for PySpark and SparkR. - - -## Upgrading from Spark SQL 1.0-1.2 to 1.3 - -In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the -available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other -releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked -as unstable (i.e., DeveloperAPI or Experimental). - -#### Rename of SchemaRDD to DataFrame - -The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has -been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD -directly, but instead provide most of the functionality that RDDs provide though their own -implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. - -In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for -some use cases. It is still recommended that users update their code to use `DataFrame` instead. -Java and Python users will need to update their code. - -#### Unification of the Java and Scala APIs - -Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) -that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users -of either language should use `SQLContext` and `DataFrame`. In general these classes try to -use types that are usable from both languages (i.e. `Array` instead of language-specific collections). -In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading -is used instead. - -Additionally, the Java specific types API has been removed. Users of both Scala and Java should -use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. - - -#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) - -Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought -all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit -conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. -Users should now write `import sqlContext.implicits._`. - -Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e., -case classes or tuples) with a method `toDF`, instead of applying automatically. - -When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import -`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: -`import org.apache.spark.sql.functions._`. - -#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) - -Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users -should instead import the classes in `org.apache.spark.sql.types` - -#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) - -Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been -moved into the udf object in `SQLContext`. - -
    -
    -{% highlight scala %} - -sqlContext.udf.register("strLen", (s: String) => s.length()) - -{% endhighlight %} -
    - -
    -{% highlight java %} - -sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType); - -{% endhighlight %} -
    - -
    - -Python UDF registration is unchanged. - -#### Python DataTypes No Longer Singletons - -When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of -referencing a singleton. - -## Compatibility with Apache Hive - -Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. -Currently, Hive SerDes and UDFs are based on Hive 1.2.1, -and Spark SQL can be connected to different versions of Hive Metastore -(from 0.12.0 to 2.3.3. Also see [Interacting with Different Versions of Hive Metastore](#interacting-with-different-versions-of-hive-metastore)). - -#### Deploying in Existing Hive Warehouses - -The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive -installations. You do not need to modify your existing Hive Metastore or change the data placement -or partitioning of your tables. - -### Supported Hive Features - -Spark SQL supports the vast majority of Hive features, such as: - -* Hive query statements, including: - * `SELECT` - * `GROUP BY` - * `ORDER BY` - * `CLUSTER BY` - * `SORT BY` -* All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) - * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) - * Logical operators (`AND`, `&&`, `OR`, `||`, etc) - * Complex type constructors - * Mathematical functions (`sign`, `ln`, `cos`, etc) - * String functions (`instr`, `length`, `printf`, etc) -* User defined functions (UDF) -* User defined aggregation functions (UDAF) -* User defined serialization formats (SerDes) -* Window functions -* Joins - * `JOIN` - * `{LEFT|RIGHT|FULL} OUTER JOIN` - * `LEFT SEMI JOIN` - * `CROSS JOIN` -* Unions -* Sub-queries - * `SELECT col FROM ( SELECT a + b AS col from t1) t2` -* Sampling -* Explain -* Partitioned tables including dynamic partition insertion -* View -* All Hive DDL Functions, including: - * `CREATE TABLE` - * `CREATE TABLE AS SELECT` - * `ALTER TABLE` -* Most Hive Data types, including: - * `TINYINT` - * `SMALLINT` - * `INT` - * `BIGINT` - * `BOOLEAN` - * `FLOAT` - * `DOUBLE` - * `STRING` - * `BINARY` - * `TIMESTAMP` - * `DATE` - * `ARRAY<>` - * `MAP<>` - * `STRUCT<>` - -### Unsupported Hive Functionality - -Below is a list of Hive features that we don't support yet. Most of these features are rarely used -in Hive deployments. - -**Major Hive Features** - -* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL - doesn't support buckets yet. - - -**Esoteric Hive Features** - -* `UNION` type -* Unique join -* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at - the moment and only supports populating the sizeInBytes field of the hive metastore. - -**Hive Input/Output Formats** - -* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. -* Hadoop archive - -**Hive Optimizations** - -A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are -less important due to Spark SQL's in-memory computational model. Others are slotted for future -releases of Spark SQL. - -* Block-level bitmap indexes and virtual columns (used to build indexes) -* Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you - need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". -* Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still - launches tasks to compute the result. -* Skew data flag: Spark SQL does not follow the skew data flags in Hive. -* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. -* Merge multiple small files for query results: if the result output contains multiple small files, - Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS - metadata. Spark SQL does not support that. - -**Hive UDF/UDTF/UDAF** - -Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs: - -* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically - include additional resources required by this UDF. -* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses - a deprecated interface `initialize(ObjectInspector[])` only. -* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize - functions with `MapredContext`, which is inapplicable to Spark. -* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources. - Spark SQL does not call this function when tasks finish. -* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation. - Spark SQL currently does not support the reuse of aggregation. -* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating - an aggregate over a fixed window. - -### Incompatible Hive UDF - -Below are the scenarios in which Hive and Spark generate different results: - -* `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN. -* `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. -* `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. - -# Reference - -## Data Types - -Spark SQL and DataFrames support the following data types: - -* Numeric types - - `ByteType`: Represents 1-byte signed integer numbers. - The range of numbers is from `-128` to `127`. - - `ShortType`: Represents 2-byte signed integer numbers. - The range of numbers is from `-32768` to `32767`. - - `IntegerType`: Represents 4-byte signed integer numbers. - The range of numbers is from `-2147483648` to `2147483647`. - - `LongType`: Represents 8-byte signed integer numbers. - The range of numbers is from `-9223372036854775808` to `9223372036854775807`. - - `FloatType`: Represents 4-byte single-precision floating point numbers. - - `DoubleType`: Represents 8-byte double-precision floating point numbers. - - `DecimalType`: Represents arbitrary-precision signed decimal numbers. Backed internally by `java.math.BigDecimal`. A `BigDecimal` consists of an arbitrary precision integer unscaled value and a 32-bit integer scale. -* String type - - `StringType`: Represents character string values. -* Binary type - - `BinaryType`: Represents byte sequence values. -* Boolean type - - `BooleanType`: Represents boolean values. -* Datetime type - - `TimestampType`: Represents values comprising values of fields year, month, day, - hour, minute, and second. - - `DateType`: Represents values comprising values of fields year, month, day. -* Complex types - - `ArrayType(elementType, containsNull)`: Represents values comprising a sequence of - elements with the type of `elementType`. `containsNull` is used to indicate if - elements in a `ArrayType` value can have `null` values. - - `MapType(keyType, valueType, valueContainsNull)`: - Represents values comprising a set of key-value pairs. The data type of keys are - described by `keyType` and the data type of values are described by `valueType`. - For a `MapType` value, keys are not allowed to have `null` values. `valueContainsNull` - is used to indicate if values of a `MapType` value can have `null` values. - - `StructType(fields)`: Represents values with the structure described by - a sequence of `StructField`s (`fields`). - * `StructField(name, dataType, nullable)`: Represents a field in a `StructType`. - The name of a field is indicated by `name`. The data type of a field is indicated - by `dataType`. `nullable` is used to indicate if values of this fields can have - `null` values. - -
    -
    - -All data types of Spark SQL are located in the package `org.apache.spark.sql.types`. -You can access them by doing - -{% include_example data_types scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in ScalaAPI to access or create a data type
    ByteType Byte - ByteType -
    ShortType Short - ShortType -
    IntegerType Int - IntegerType -
    LongType Long - LongType -
    FloatType Float - FloatType -
    DoubleType Double - DoubleType -
    DecimalType java.math.BigDecimal - DecimalType -
    StringType String - StringType -
    BinaryType Array[Byte] - BinaryType -
    BooleanType Boolean - BooleanType -
    TimestampType java.sql.Timestamp - TimestampType -
    DateType java.sql.Date - DateType -
    ArrayType scala.collection.Seq - ArrayType(elementType, [containsNull])
    - Note: The default value of containsNull is true. -
    MapType scala.collection.Map - MapType(keyType, valueType, [valueContainsNull])
    - Note: The default value of valueContainsNull is true. -
    StructType org.apache.spark.sql.Row - StructType(fields)
    - Note: fields is a Seq of StructFields. Also, two fields with the same - name are not allowed. -
    StructField The value type in Scala of the data type of this field - (For example, Int for a StructField with the data type IntegerType) - StructField(name, dataType, [nullable])
    - Note: The default value of nullable is true. -
    - -
    - -
    - -All data types of Spark SQL are located in the package of -`org.apache.spark.sql.types`. To access or create a data type, -please use factory methods provided in -`org.apache.spark.sql.types.DataTypes`. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in JavaAPI to access or create a data type
    ByteType byte or Byte - DataTypes.ByteType -
    ShortType short or Short - DataTypes.ShortType -
    IntegerType int or Integer - DataTypes.IntegerType -
    LongType long or Long - DataTypes.LongType -
    FloatType float or Float - DataTypes.FloatType -
    DoubleType double or Double - DataTypes.DoubleType -
    DecimalType java.math.BigDecimal - DataTypes.createDecimalType()
    - DataTypes.createDecimalType(precision, scale). -
    StringType String - DataTypes.StringType -
    BinaryType byte[] - DataTypes.BinaryType -
    BooleanType boolean or Boolean - DataTypes.BooleanType -
    TimestampType java.sql.Timestamp - DataTypes.TimestampType -
    DateType java.sql.Date - DataTypes.DateType -
    ArrayType java.util.List - DataTypes.createArrayType(elementType)
    - Note: The value of containsNull will be true
    - DataTypes.createArrayType(elementType, containsNull). -
    MapType java.util.Map - DataTypes.createMapType(keyType, valueType)
    - Note: The value of valueContainsNull will be true.
    - DataTypes.createMapType(keyType, valueType, valueContainsNull)
    -
    StructType org.apache.spark.sql.Row - DataTypes.createStructType(fields)
    - Note: fields is a List or an array of StructFields. - Also, two fields with the same name are not allowed. -
    StructField The value type in Java of the data type of this field - (For example, int for a StructField with the data type IntegerType) - DataTypes.createStructField(name, dataType, nullable) -
    - -
    - -
    - -All data types of Spark SQL are located in the package of `pyspark.sql.types`. -You can access them by doing -{% highlight python %} -from pyspark.sql.types import * -{% endhighlight %} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in PythonAPI to access or create a data type
    ByteType - int or long
    - Note: Numbers will be converted to 1-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -128 to 127. -
    - ByteType() -
    ShortType - int or long
    - Note: Numbers will be converted to 2-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -32768 to 32767. -
    - ShortType() -
    IntegerType int or long - IntegerType() -
    LongType - long
    - Note: Numbers will be converted to 8-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of - -9223372036854775808 to 9223372036854775807. - Otherwise, please convert data to decimal.Decimal and use DecimalType. -
    - LongType() -
    FloatType - float
    - Note: Numbers will be converted to 4-byte single-precision floating - point numbers at runtime. -
    - FloatType() -
    DoubleType float - DoubleType() -
    DecimalType decimal.Decimal - DecimalType() -
    StringType string - StringType() -
    BinaryType bytearray - BinaryType() -
    BooleanType bool - BooleanType() -
    TimestampType datetime.datetime - TimestampType() -
    DateType datetime.date - DateType() -
    ArrayType list, tuple, or array - ArrayType(elementType, [containsNull])
    - Note: The default value of containsNull is True. -
    MapType dict - MapType(keyType, valueType, [valueContainsNull])
    - Note: The default value of valueContainsNull is True. -
    StructType list or tuple - StructType(fields)
    - Note: fields is a Seq of StructFields. Also, two fields with the same - name are not allowed. -
    StructField The value type in Python of the data type of this field - (For example, Int for a StructField with the data type IntegerType) - StructField(name, dataType, [nullable])
    - Note: The default value of nullable is True. -
    - -
    - -
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in RAPI to access or create a data type
    ByteType - integer
    - Note: Numbers will be converted to 1-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -128 to 127. -
    - "byte" -
    ShortType - integer
    - Note: Numbers will be converted to 2-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -32768 to 32767. -
    - "short" -
    IntegerType integer - "integer" -
    LongType - integer
    - Note: Numbers will be converted to 8-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of - -9223372036854775808 to 9223372036854775807. - Otherwise, please convert data to decimal.Decimal and use DecimalType. -
    - "long" -
    FloatType - numeric
    - Note: Numbers will be converted to 4-byte single-precision floating - point numbers at runtime. -
    - "float" -
    DoubleType numeric - "double" -
    DecimalType Not supported - Not supported -
    StringType character - "string" -
    BinaryType raw - "binary" -
    BooleanType logical - "bool" -
    TimestampType POSIXct - "timestamp" -
    DateType Date - "date" -
    ArrayType vector or list - list(type="array", elementType=elementType, containsNull=[containsNull])
    - Note: The default value of containsNull is TRUE. -
    MapType environment - list(type="map", keyType=keyType, valueType=valueType, valueContainsNull=[valueContainsNull])
    - Note: The default value of valueContainsNull is TRUE. -
    StructType named list - list(type="struct", fields=fields)
    - Note: fields is a Seq of StructFields. Also, two fields with the same - name are not allowed. -
    StructField The value type in R of the data type of this field - (For example, integer for a StructField with the data type IntegerType) - list(name=name, type=dataType, nullable=[nullable])
    - Note: The default value of nullable is TRUE. -
    - -
    - -
    - -## NaN Semantics - -There is specially handling for not-a-number (NaN) when dealing with `float` or `double` types that -does not exactly match standard floating point semantics. -Specifically: - - - NaN = NaN returns true. - - In aggregations, all NaN values are grouped together. - - NaN is treated as a normal value in join keys. - - NaN values go last when in ascending order, larger than any other numeric value. - - ## Arithmetic operations - -Operations performed on numeric types (with the exception of `decimal`) are not checked for overflow. -This means that in case an operation causes an overflow, the result is the same that the same operation -returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable, -the result is a negative number). diff --git a/docs/sql-pyspark-pandas-with-arrow.md b/docs/sql-pyspark-pandas-with-arrow.md new file mode 100644 index 000000000000..e8e9f55bd12b --- /dev/null +++ b/docs/sql-pyspark-pandas-with-arrow.md @@ -0,0 +1,166 @@ +--- +layout: global +title: PySpark Usage Guide for Pandas with Apache Arrow +displayTitle: PySpark Usage Guide for Pandas with Apache Arrow +--- + +* Table of contents +{:toc} + +## Apache Arrow in Spark + +Apache Arrow is an in-memory columnar data format that is used in Spark to efficiently transfer +data between JVM and Python processes. This currently is most beneficial to Python users that +work with Pandas/NumPy data. Its usage is not automatic and might require some minor +changes to configuration or code to take full advantage and ensure compatibility. This guide will +give a high-level description of how to use Arrow in Spark and highlight any differences when +working with Arrow-enabled data. + +### Ensure PyArrow Installed + +If you install PySpark using pip, then PyArrow can be brought in as an extra dependency of the +SQL module with the command `pip install pyspark[sql]`. Otherwise, you must ensure that PyArrow +is installed and available on all cluster nodes. The current supported version is 0.8.0. +You can install using pip or conda from the conda-forge channel. See PyArrow +[installation](https://arrow.apache.org/docs/python/install.html) for details. + +## Enabling for Conversion to/from Pandas + +Arrow is available as an optimization when converting a Spark DataFrame to a Pandas DataFrame +using the call `toPandas()` and when creating a Spark DataFrame from a Pandas DataFrame with +`createDataFrame(pandas_df)`. To use Arrow when executing these calls, users need to first set +the Spark configuration 'spark.sql.execution.arrow.enabled' to 'true'. This is disabled by default. + +In addition, optimizations enabled by 'spark.sql.execution.arrow.enabled' could fallback automatically +to non-Arrow optimization implementation if an error occurs before the actual computation within Spark. +This can be controlled by 'spark.sql.execution.arrow.fallback.enabled'. + +
    +
    +{% include_example dataframe_with_arrow python/sql/arrow.py %} +
    +
    + +Using the above optimizations with Arrow will produce the same results as when Arrow is not +enabled. Note that even with Arrow, `toPandas()` results in the collection of all records in the +DataFrame to the driver program and should be done on a small subset of the data. Not all Spark +data types are currently supported and an error can be raised if a column has an unsupported type, +see [Supported SQL Types](#supported-sql-types). If an error occurs during `createDataFrame()`, +Spark will fall back to create the DataFrame without Arrow. + +## Pandas UDFs (a.k.a. Vectorized UDFs) + +Pandas UDFs are user defined functions that are executed by Spark using Arrow to transfer data and +Pandas to work with the data. A Pandas UDF is defined using the keyword `pandas_udf` as a decorator +or to wrap the function, no additional configuration is required. Currently, there are two types of +Pandas UDF: Scalar and Grouped Map. + +### Scalar + +Scalar Pandas UDFs are used for vectorizing scalar operations. They can be used with functions such +as `select` and `withColumn`. The Python function should take `pandas.Series` as inputs and return +a `pandas.Series` of the same length. Internally, Spark will execute a Pandas UDF by splitting +columns into batches and calling the function for each batch as a subset of the data, then +concatenating the results together. + +The following example shows how to create a scalar Pandas UDF that computes the product of 2 columns. + +
    +
    +{% include_example scalar_pandas_udf python/sql/arrow.py %} +
    +
    + +### Grouped Map +Grouped map Pandas UDFs are used with `groupBy().apply()` which implements the "split-apply-combine" pattern. +Split-apply-combine consists of three steps: +* Split the data into groups by using `DataFrame.groupBy`. +* Apply a function on each group. The input and output of the function are both `pandas.DataFrame`. The + input data contains all the rows and columns for each group. +* Combine the results into a new `DataFrame`. + +To use `groupBy().apply()`, the user needs to define the following: +* A Python function that defines the computation for each group. +* A `StructType` object or a string that defines the schema of the output `DataFrame`. + +The column labels of the returned `pandas.DataFrame` must either match the field names in the +defined output schema if specified as strings, or match the field data types by position if not +strings, e.g. integer indices. See [pandas.DataFrame](https://pandas.pydata.org/pandas-docs/stable/generated/pandas.DataFrame.html#pandas.DataFrame) +on how to label columns when constructing a `pandas.DataFrame`. + +Note that all data for a group will be loaded into memory before the function is applied. This can +lead to out of memory exceptions, especially if the group sizes are skewed. The configuration for +[maxRecordsPerBatch](#setting-arrow-batch-size) is not applied on groups and it is up to the user +to ensure that the grouped data will fit into the available memory. + +The following example shows how to use `groupby().apply()` to subtract the mean from each value in the group. + +
    +
    +{% include_example grouped_map_pandas_udf python/sql/arrow.py %} +
    +
    + +For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) and +[`pyspark.sql.GroupedData.apply`](api/python/pyspark.sql.html#pyspark.sql.GroupedData.apply). + +### Grouped Aggregate + +Grouped aggregate Pandas UDFs are similar to Spark aggregate functions. Grouped aggregate Pandas UDFs are used with `groupBy().agg()` and +[`pyspark.sql.Window`](api/python/pyspark.sql.html#pyspark.sql.Window). It defines an aggregation from one or more `pandas.Series` +to a scalar value, where each `pandas.Series` represents a column within the group or window. + +Note that this type of UDF does not support partial aggregation and all data for a group or window will be loaded into memory. Also, +only unbounded window is supported with Grouped aggregate Pandas UDFs currently. + +The following example shows how to use this type of UDF to compute mean with groupBy and window operations: + +
    +
    +{% include_example grouped_agg_pandas_udf python/sql/arrow.py %} +
    +
    + +For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) + +## Usage Notes + +### Supported SQL Types + +Currently, all Spark SQL data types are supported by Arrow-based conversion except `BinaryType`, `MapType`, +`ArrayType` of `TimestampType`, and nested `StructType`. + +### Setting Arrow Batch Size + +Data partitions in Spark are converted into Arrow record batches, which can temporarily lead to +high memory usage in the JVM. To avoid possible out of memory exceptions, the size of the Arrow +record batches can be adjusted by setting the conf "spark.sql.execution.arrow.maxRecordsPerBatch" +to an integer that will determine the maximum number of rows for each batch. The default value is +10,000 records per batch. If the number of columns is large, the value should be adjusted +accordingly. Using this limit, each data partition will be made into 1 or more record batches for +processing. + +### Timestamp with Time Zone Semantics + +Spark internally stores timestamps as UTC values, and timestamp data that is brought in without +a specified time zone is converted as local time to UTC with microsecond resolution. When timestamp +data is exported or displayed in Spark, the session time zone is used to localize the timestamp +values. The session time zone is set with the configuration 'spark.sql.session.timeZone' and will +default to the JVM system local time zone if not set. Pandas uses a `datetime64` type with nanosecond +resolution, `datetime64[ns]`, with optional time zone on a per-column basis. + +When timestamp data is transferred from Spark to Pandas it will be converted to nanoseconds +and each column will be converted to the Spark session time zone then localized to that time +zone, which removes the time zone and displays values as local time. This will occur +when calling `toPandas()` or `pandas_udf` with timestamp columns. + +When timestamp data is transferred from Pandas to Spark, it will be converted to UTC microseconds. This +occurs when calling `createDataFrame` with a Pandas DataFrame or when returning a timestamp from a +`pandas_udf`. These conversions are done automatically to ensure Spark will have data in the +expected format, so it is not necessary to do any of these conversions yourself. Any nanosecond +values will be truncated. + +Note that a standard UDF (non-Pandas) will load timestamp data as Python datetime objects, which is +different than a Pandas timestamp. It is recommended to use Pandas time series functionality when +working with timestamps in `pandas_udf`s to get the best performance, see +[here](https://pandas.pydata.org/pandas-docs/stable/timeseries.html) for details. diff --git a/docs/sql-reference.md b/docs/sql-reference.md new file mode 100644 index 000000000000..9e4239b6bad2 --- /dev/null +++ b/docs/sql-reference.md @@ -0,0 +1,641 @@ +--- +layout: global +title: Reference +displayTitle: Reference +--- + +* Table of contents +{:toc} + +## Data Types + +Spark SQL and DataFrames support the following data types: + +* Numeric types + - `ByteType`: Represents 1-byte signed integer numbers. + The range of numbers is from `-128` to `127`. + - `ShortType`: Represents 2-byte signed integer numbers. + The range of numbers is from `-32768` to `32767`. + - `IntegerType`: Represents 4-byte signed integer numbers. + The range of numbers is from `-2147483648` to `2147483647`. + - `LongType`: Represents 8-byte signed integer numbers. + The range of numbers is from `-9223372036854775808` to `9223372036854775807`. + - `FloatType`: Represents 4-byte single-precision floating point numbers. + - `DoubleType`: Represents 8-byte double-precision floating point numbers. + - `DecimalType`: Represents arbitrary-precision signed decimal numbers. Backed internally by `java.math.BigDecimal`. A `BigDecimal` consists of an arbitrary precision integer unscaled value and a 32-bit integer scale. +* String type + - `StringType`: Represents character string values. +* Binary type + - `BinaryType`: Represents byte sequence values. +* Boolean type + - `BooleanType`: Represents boolean values. +* Datetime type + - `TimestampType`: Represents values comprising values of fields year, month, day, + hour, minute, and second. + - `DateType`: Represents values comprising values of fields year, month, day. +* Complex types + - `ArrayType(elementType, containsNull)`: Represents values comprising a sequence of + elements with the type of `elementType`. `containsNull` is used to indicate if + elements in a `ArrayType` value can have `null` values. + - `MapType(keyType, valueType, valueContainsNull)`: + Represents values comprising a set of key-value pairs. The data type of keys are + described by `keyType` and the data type of values are described by `valueType`. + For a `MapType` value, keys are not allowed to have `null` values. `valueContainsNull` + is used to indicate if values of a `MapType` value can have `null` values. + - `StructType(fields)`: Represents values with the structure described by + a sequence of `StructField`s (`fields`). + * `StructField(name, dataType, nullable)`: Represents a field in a `StructType`. + The name of a field is indicated by `name`. The data type of a field is indicated + by `dataType`. `nullable` is used to indicate if values of this fields can have + `null` values. + +
    +
    + +All data types of Spark SQL are located in the package `org.apache.spark.sql.types`. +You can access them by doing + +{% include_example data_types scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in ScalaAPI to access or create a data type
    ByteType Byte + ByteType +
    ShortType Short + ShortType +
    IntegerType Int + IntegerType +
    LongType Long + LongType +
    FloatType Float + FloatType +
    DoubleType Double + DoubleType +
    DecimalType java.math.BigDecimal + DecimalType +
    StringType String + StringType +
    BinaryType Array[Byte] + BinaryType +
    BooleanType Boolean + BooleanType +
    TimestampType java.sql.Timestamp + TimestampType +
    DateType java.sql.Date + DateType +
    ArrayType scala.collection.Seq + ArrayType(elementType, [containsNull])
    + Note: The default value of containsNull is true. +
    MapType scala.collection.Map + MapType(keyType, valueType, [valueContainsNull])
    + Note: The default value of valueContainsNull is true. +
    StructType org.apache.spark.sql.Row + StructType(fields)
    + Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
    StructField The value type in Scala of the data type of this field + (For example, Int for a StructField with the data type IntegerType) + StructField(name, dataType, [nullable])
    + Note: The default value of nullable is true. +
    + +
    + +
    + +All data types of Spark SQL are located in the package of +`org.apache.spark.sql.types`. To access or create a data type, +please use factory methods provided in +`org.apache.spark.sql.types.DataTypes`. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in JavaAPI to access or create a data type
    ByteType byte or Byte + DataTypes.ByteType +
    ShortType short or Short + DataTypes.ShortType +
    IntegerType int or Integer + DataTypes.IntegerType +
    LongType long or Long + DataTypes.LongType +
    FloatType float or Float + DataTypes.FloatType +
    DoubleType double or Double + DataTypes.DoubleType +
    DecimalType java.math.BigDecimal + DataTypes.createDecimalType()
    + DataTypes.createDecimalType(precision, scale). +
    StringType String + DataTypes.StringType +
    BinaryType byte[] + DataTypes.BinaryType +
    BooleanType boolean or Boolean + DataTypes.BooleanType +
    TimestampType java.sql.Timestamp + DataTypes.TimestampType +
    DateType java.sql.Date + DataTypes.DateType +
    ArrayType java.util.List + DataTypes.createArrayType(elementType)
    + Note: The value of containsNull will be true
    + DataTypes.createArrayType(elementType, containsNull). +
    MapType java.util.Map + DataTypes.createMapType(keyType, valueType)
    + Note: The value of valueContainsNull will be true.
    + DataTypes.createMapType(keyType, valueType, valueContainsNull)
    +
    StructType org.apache.spark.sql.Row + DataTypes.createStructType(fields)
    + Note: fields is a List or an array of StructFields. + Also, two fields with the same name are not allowed. +
    StructField The value type in Java of the data type of this field + (For example, int for a StructField with the data type IntegerType) + DataTypes.createStructField(name, dataType, nullable) +
    + +
    + +
    + +All data types of Spark SQL are located in the package of `pyspark.sql.types`. +You can access them by doing +{% highlight python %} +from pyspark.sql.types import * +{% endhighlight %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in PythonAPI to access or create a data type
    ByteType + int or long
    + Note: Numbers will be converted to 1-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -128 to 127. +
    + ByteType() +
    ShortType + int or long
    + Note: Numbers will be converted to 2-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -32768 to 32767. +
    + ShortType() +
    IntegerType int or long + IntegerType() +
    LongType + long
    + Note: Numbers will be converted to 8-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of + -9223372036854775808 to 9223372036854775807. + Otherwise, please convert data to decimal.Decimal and use DecimalType. +
    + LongType() +
    FloatType + float
    + Note: Numbers will be converted to 4-byte single-precision floating + point numbers at runtime. +
    + FloatType() +
    DoubleType float + DoubleType() +
    DecimalType decimal.Decimal + DecimalType() +
    StringType string + StringType() +
    BinaryType bytearray + BinaryType() +
    BooleanType bool + BooleanType() +
    TimestampType datetime.datetime + TimestampType() +
    DateType datetime.date + DateType() +
    ArrayType list, tuple, or array + ArrayType(elementType, [containsNull])
    + Note: The default value of containsNull is True. +
    MapType dict + MapType(keyType, valueType, [valueContainsNull])
    + Note: The default value of valueContainsNull is True. +
    StructType list or tuple + StructType(fields)
    + Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
    StructField The value type in Python of the data type of this field + (For example, Int for a StructField with the data type IntegerType) + StructField(name, dataType, [nullable])
    + Note: The default value of nullable is True. +
    + +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in RAPI to access or create a data type
    ByteType + integer
    + Note: Numbers will be converted to 1-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -128 to 127. +
    + "byte" +
    ShortType + integer
    + Note: Numbers will be converted to 2-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -32768 to 32767. +
    + "short" +
    IntegerType integer + "integer" +
    LongType + integer
    + Note: Numbers will be converted to 8-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of + -9223372036854775808 to 9223372036854775807. + Otherwise, please convert data to decimal.Decimal and use DecimalType. +
    + "long" +
    FloatType + numeric
    + Note: Numbers will be converted to 4-byte single-precision floating + point numbers at runtime. +
    + "float" +
    DoubleType numeric + "double" +
    DecimalType Not supported + Not supported +
    StringType character + "string" +
    BinaryType raw + "binary" +
    BooleanType logical + "bool" +
    TimestampType POSIXct + "timestamp" +
    DateType Date + "date" +
    ArrayType vector or list + list(type="array", elementType=elementType, containsNull=[containsNull])
    + Note: The default value of containsNull is TRUE. +
    MapType environment + list(type="map", keyType=keyType, valueType=valueType, valueContainsNull=[valueContainsNull])
    + Note: The default value of valueContainsNull is TRUE. +
    StructType named list + list(type="struct", fields=fields)
    + Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
    StructField The value type in R of the data type of this field + (For example, integer for a StructField with the data type IntegerType) + list(name=name, type=dataType, nullable=[nullable])
    + Note: The default value of nullable is TRUE. +
    + +
    + +
    + +## NaN Semantics + +There is specially handling for not-a-number (NaN) when dealing with `float` or `double` types that +does not exactly match standard floating point semantics. +Specifically: + + - NaN = NaN returns true. + - In aggregations, all NaN values are grouped together. + - NaN is treated as a normal value in join keys. + - NaN values go last when in ascending order, larger than any other numeric value. + +## Arithmetic operations + +Operations performed on numeric types (with the exception of `decimal`) are not checked for overflow. +This means that in case an operation causes an overflow, the result is the same that the same operation +returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable, +the result is a negative number). diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index b6e427735e74..3678bfb8609a 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -536,7 +536,7 @@ Here are the details of all the sources in Spark. href="api/R/read.stream.html">R). E.g. for "parquet" format options see DataStreamReader.parquet().

    - In addition, there are session configurations that affect certain file-formats. See the SQL Programming Guide for more details. E.g., for "parquet", see Parquet configuration section. + In addition, there are session configurations that affect certain file-formats. See the SQL Programming Guide for more details. E.g., for "parquet", see Parquet configuration section. Yes Supports glob paths, but does not support multiple comma-separated paths/globs. From fd5b247262761271ac36d67fe66f7814acc664a9 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Thu, 18 Oct 2018 13:51:13 -0700 Subject: [PATCH 175/879] [SPARK-24499][DOC][FOLLOW-UP] Split the page of sql-programming-guide.html to multiple separate pages ## What changes were proposed in this pull request? Forgot to clean remove the link for `Upgrading From Spark SQL 2.4 to 3.0` when merging to 2.4 ## How was this patch tested? N/A Closes #22769 from gatorsmile/test2.4. Authored-by: gatorsmile Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide.md | 1 - 1 file changed, 1 deletion(-) diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md index 71d83e8a5570..a3fc52ca6d94 100644 --- a/docs/sql-migration-guide.md +++ b/docs/sql-migration-guide.md @@ -5,7 +5,6 @@ displayTitle: Migration Guide --- * [Spark SQL Upgrading Guide](sql-migration-guide-upgrade.html) - * [Upgrading From Spark SQL 2.4 to 3.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-24-to-30) * [Upgrading From Spark SQL 2.3 to 2.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-23-to-24) * [Upgrading From Spark SQL 2.3.0 to 2.3.1 and above](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-230-to-231-and-above) * [Upgrading From Spark SQL 2.2 to 2.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-22-to-23) From 36307b1e4b42ce22b07e7a3fc2679c4b5e7c34c8 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Fri, 19 Oct 2018 09:33:46 +0800 Subject: [PATCH 176/879] [SPARK-25764][ML][EXAMPLES] Update BisectingKMeans example to use ClusteringEvaluator ## What changes were proposed in this pull request? The PR updates the examples for `BisectingKMeans` so that they don't use the deprecated method `computeCost` (see SPARK-25758). ## How was this patch tested? running examples Closes #22763 from mgaido91/SPARK-25764. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan (cherry picked from commit d0ecff28545ac81f5ba7ac06957ced65b6e3ebcd) Signed-off-by: Wenchen Fan --- .../examples/ml/JavaBisectingKMeansExample.java | 12 +++++++++--- .../src/main/python/ml/bisecting_k_means_example.py | 12 +++++++++--- .../spark/examples/ml/BisectingKMeansExample.scala | 12 +++++++++--- 3 files changed, 27 insertions(+), 9 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java index 8c82aaaacca3..f517dc314b2b 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -20,6 +20,7 @@ // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans; import org.apache.spark.ml.clustering.BisectingKMeansModel; +import org.apache.spark.ml.evaluation.ClusteringEvaluator; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -50,9 +51,14 @@ public static void main(String[] args) { BisectingKMeans bkm = new BisectingKMeans().setK(2).setSeed(1); BisectingKMeansModel model = bkm.fit(dataset); - // Evaluate clustering. - double cost = model.computeCost(dataset); - System.out.println("Within Set Sum of Squared Errors = " + cost); + // Make predictions + Dataset predictions = model.transform(dataset); + + // Evaluate clustering by computing Silhouette score + ClusteringEvaluator evaluator = new ClusteringEvaluator(); + + double silhouette = evaluator.evaluate(predictions); + System.out.println("Silhouette with squared euclidean distance = " + silhouette); // Shows the result. System.out.println("Cluster Centers: "); diff --git a/examples/src/main/python/ml/bisecting_k_means_example.py b/examples/src/main/python/ml/bisecting_k_means_example.py index 7842d2009e23..82adb338b5d9 100644 --- a/examples/src/main/python/ml/bisecting_k_means_example.py +++ b/examples/src/main/python/ml/bisecting_k_means_example.py @@ -24,6 +24,7 @@ # $example on$ from pyspark.ml.clustering import BisectingKMeans +from pyspark.ml.evaluation import ClusteringEvaluator # $example off$ from pyspark.sql import SparkSession @@ -41,9 +42,14 @@ bkm = BisectingKMeans().setK(2).setSeed(1) model = bkm.fit(dataset) - # Evaluate clustering. - cost = model.computeCost(dataset) - print("Within Set Sum of Squared Errors = " + str(cost)) + # Make predictions + predictions = model.transform(dataset) + + # Evaluate clustering by computing Silhouette score + evaluator = ClusteringEvaluator() + + silhouette = evaluator.evaluate(predictions) + print("Silhouette with squared euclidean distance = " + str(silhouette)) # Shows the result. print("Cluster Centers: ") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala index 5f8f2c99cbaf..14e13df02733 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala @@ -21,6 +21,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans +import org.apache.spark.ml.evaluation.ClusteringEvaluator // $example off$ import org.apache.spark.sql.SparkSession @@ -48,9 +49,14 @@ object BisectingKMeansExample { val bkm = new BisectingKMeans().setK(2).setSeed(1) val model = bkm.fit(dataset) - // Evaluate clustering. - val cost = model.computeCost(dataset) - println(s"Within Set Sum of Squared Errors = $cost") + // Make predictions + val predictions = model.transform(dataset) + + // Evaluate clustering by computing Silhouette score + val evaluator = new ClusteringEvaluator() + + val silhouette = evaluator.evaluate(predictions) + println(s"Silhouette with squared euclidean distance = $silhouette") // Shows the result. println("Cluster Centers: ") From 9ed2e42044a1105a1c8b5868dbb320b1b477bcf0 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 19 Oct 2018 13:55:27 +0800 Subject: [PATCH 177/879] [MINOR][DOC] Spacing items in migration guide for readability and consistency ## What changes were proposed in this pull request? Currently, migration guide has no space between each item which looks too compact and hard to read. Some of items already had some spaces between them in the migration guide. This PR suggest to format them consistently for readability. Before: ![screen shot 2018-10-18 at 10 00 04 am](https://user-images.githubusercontent.com/6477701/47126768-9e84fb80-d2bc-11e8-9211-84703486c553.png) After: ![screen shot 2018-10-18 at 9 53 55 am](https://user-images.githubusercontent.com/6477701/47126708-4fd76180-d2bc-11e8-9aa5-546f0622ca20.png) ## How was this patch tested? Manually tested: Closes #22761 from HyukjinKwon/minor-migration-doc. Authored-by: hyukjinkwon Signed-off-by: hyukjinkwon (cherry picked from commit c8f7691c64a28174a54e8faa159b50a3836a7225) Signed-off-by: hyukjinkwon --- docs/sql-migration-guide-upgrade.md | 54 +++++++++++++++++++++++++++++ 1 file changed, 54 insertions(+) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 3476aa83c9b2..062e07bd23c5 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -70,26 +70,47 @@ displayTitle: Spark SQL Upgrading Guide - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. + - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. + - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. + - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. + - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. + - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. + - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. + - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, a column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. + - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. + - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. + - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. + - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. + - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. + - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. + - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. + - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. + - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. + - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. + - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. + - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. + - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. + - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. ## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above @@ -99,8 +120,11 @@ displayTitle: Spark SQL Upgrading Guide ## Upgrading From Spark SQL 2.2 to 2.3 - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. + - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. + - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. + - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: @@ -239,18 +263,29 @@ displayTitle: Spark SQL Upgrading Guide
    Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. + - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. + - In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration `spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See [SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details. + - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. + - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](sql-performance-turing.html#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). + - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. + - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive's new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes + - The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`). + - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. + - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible. + - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. + - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. - When creating a `SparkSession` with `SparkSession.builder.getOrCreate()`, if there is an existing `SparkContext`, the builder was trying to update the `SparkConf` of the existing `SparkContext` with configurations specified to the builder, but the `SparkContext` is shared by all `SparkSession`s, so we should not update them. Since 2.3, the builder comes to not update the configurations. If you want to update them, you need to update them prior to creating a `SparkSession`. @@ -264,15 +299,20 @@ displayTitle: Spark SQL Upgrading Guide ## Upgrading From Spark SQL 2.0 to 2.1 - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. + - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. + - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. + - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. + - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. ## Upgrading From Spark SQL 1.6 to 2.0 - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and + `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for @@ -284,15 +324,19 @@ displayTitle: Spark SQL Upgrading Guide single-node data frame notion in these languages. - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` + - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` + - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. + - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` in order to prevent accidental dropping the existing data in the user-provided locations. That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. Note that this is different from the Hive behavior. + - As a result, `DROP TABLE` statements on those tables will not remove the data. - `spark.sql.parquet.cacheMetadata` is no longer used. @@ -311,6 +355,7 @@ displayTitle: Spark SQL Upgrading Guide --conf spark.sql.hive.thriftServer.singleSession=true \ ... {% endhighlight %} + - Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns of the same name of a DataFrame. @@ -324,26 +369,35 @@ displayTitle: Spark SQL Upgrading Guide - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with code generation for expression evaluation. These features can both be disabled by setting `spark.sql.tungsten.enabled` to `false`. + - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting `spark.sql.parquet.mergeSchema` to `true`. + - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or access nested values. For example `df['table.column.nestedField']`. However, this means that if your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). + - In-memory columnar storage partition pruning is on by default. It can be disabled by setting `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. + - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. + - Timestamps are now stored at a precision of 1us, rather than 1ns + - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains unchanged. + - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). + - JSON data source will not automatically load new files that are created by other applications (i.e. files that are not inserted to the dataset through Spark SQL). For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate the DataFrame and the new DataFrame will include new files. + - DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name. ## Upgrading from Spark SQL 1.3 to 1.4 From df60d9f3469022866de2f41939a38e7e5d02dc1b Mon Sep 17 00:00:00 2001 From: maryannxue Date: Fri, 19 Oct 2018 21:03:59 +0800 Subject: [PATCH 178/879] [SPARK-25044][FOLLOW-UP] Change ScalaUDF constructor signature ## What changes were proposed in this pull request? This is a follow-up PR for #22259. The extra field added in `ScalaUDF` with the original PR was declared optional, but should be indeed required, otherwise callers of `ScalaUDF`'s constructor could ignore this new field and cause the result to be incorrect. This PR makes the new field required and changes its name to `handleNullForInputs`. #22259 breaks the previous behavior for null-handling of primitive-type input parameters. For example, for `val f = udf({(x: Int, y: Any) => x})`, `f(null, "str")` should return `null` but would return `0` after #22259. In this PR, all UDF methods except `def udf(f: AnyRef, dataType: DataType): UserDefinedFunction` have been restored with the original behavior. The only exception is documented in the Spark SQL migration guide. In addition, now that we have this extra field indicating if a null-test should be applied on the corresponding input value, we can also make use of this flag to avoid the rule `HandleNullInputsForUDF` being applied infinitely. ## How was this patch tested? Added UT in UDFSuite Passed affected existing UTs: AnalysisSuite UDFSuite Closes #22732 from maryannxue/spark-25044-followup. Lead-authored-by: maryannxue Co-authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit e8167768cfebfdb11acd8e0a06fe34ca43c14648) Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/ScalaReflection.scala | 22 +- .../sql/catalyst/analysis/Analyzer.scala | 51 ++-- .../sql/catalyst/expressions/ScalaUDF.scala | 14 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 18 +- .../catalyst/expressions/ScalaUDFSuite.scala | 9 +- .../sql/catalyst/trees/TreeNodeSuite.scala | 2 +- .../apache/spark/sql/UDFRegistration.scala | 218 ++++++++++-------- .../datasources/FileFormatDataWriter.scala | 3 +- .../sql/expressions/UserDefinedFunction.scala | 24 +- .../org/apache/spark/sql/functions.scala | 54 ++--- .../scala/org/apache/spark/sql/UDFSuite.scala | 24 ++ 11 files changed, 257 insertions(+), 182 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 0238d57de244..c27180e2a6b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -19,8 +19,11 @@ package org.apache.spark.sql.catalyst import java.lang.reflect.Constructor +import scala.util.Properties + import org.apache.commons.lang3.reflect.ConstructorUtils +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ @@ -879,7 +882,7 @@ object ScalaReflection extends ScalaReflection { * Support for generating catalyst schemas for scala objects. Note that unlike its companion * object, this trait able to work in both the runtime and the compile time (macro) universe. */ -trait ScalaReflection { +trait ScalaReflection extends Logging { /** The universe we work in (runtime or macro) */ val universe: scala.reflect.api.Universe @@ -932,6 +935,23 @@ trait ScalaReflection { tpe.dealias.erasure.typeSymbol.asClass.fullName } + /** + * Returns the nullability of the input parameter types of the scala function object. + * + * Note that this only works with Scala 2.11, and the information returned may be inaccurate if + * used with a different Scala version. + */ + def getParameterTypeNullability(func: AnyRef): Seq[Boolean] = { + if (!Properties.versionString.contains("2.11")) { + logWarning(s"Scala ${Properties.versionString} cannot get type nullability correctly via " + + "reflection, thus Spark cannot add proper input null check for UDF. To avoid this " + + "problem, use the typed UDF interfaces instead.") + } + val methods = func.getClass.getMethods.filter(m => m.getName == "apply" && !m.isBridge) + assert(methods.length == 1) + methods.head.getParameterTypes.map(!_.isPrimitive) + } + /** * Returns the parameter names and types for the primary constructor of this type. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 9c0975eecd44..4a83067bd896 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -2151,36 +2151,27 @@ class Analyzer( case p => p transformExpressionsUp { - case udf@ScalaUDF(func, _, inputs, _, _, _, _, nullableTypes) => - if (nullableTypes.isEmpty) { - // If no nullability info is available, do nothing. No fields will be specially - // checked for null in the plan. If nullability info is incorrect, the results - // of the UDF could be wrong. - udf - } else { - // Otherwise, add special handling of null for fields that can't accept null. - // The result of operations like this, when passed null, is generally to return null. - assert(nullableTypes.length == inputs.length) - - // TODO: skip null handling for not-nullable primitive inputs after we can completely - // trust the `nullable` information. - val needsNullCheck = (nullable: Boolean, expr: Expression) => - nullable && !expr.isInstanceOf[KnownNotNull] - val inputsNullCheck = nullableTypes.zip(inputs) - .filter { case (nullableType, expr) => needsNullCheck(!nullableType, expr) } - .map { case (_, expr) => IsNull(expr) } - .reduceLeftOption[Expression]((e1, e2) => Or(e1, e2)) - // Once we add an `If` check above the udf, it is safe to mark those checked inputs - // as not nullable (i.e., wrap them with `KnownNotNull`), because the null-returning - // branch of `If` will be called if any of these checked inputs is null. Thus we can - // prevent this rule from being applied repeatedly. - val newInputs = nullableTypes.zip(inputs).map { case (nullable, expr) => - if (nullable) expr else KnownNotNull(expr) - } - inputsNullCheck - .map(If(_, Literal.create(null, udf.dataType), udf.copy(children = newInputs))) - .getOrElse(udf) - } + case udf @ ScalaUDF(_, _, inputs, inputsNullSafe, _, _, _, _) + if inputsNullSafe.contains(false) => + // Otherwise, add special handling of null for fields that can't accept null. + // The result of operations like this, when passed null, is generally to return null. + assert(inputsNullSafe.length == inputs.length) + + // TODO: skip null handling for not-nullable primitive inputs after we can completely + // trust the `nullable` information. + val inputsNullCheck = inputsNullSafe.zip(inputs) + .filter { case (nullSafe, _) => !nullSafe } + .map { case (_, expr) => IsNull(expr) } + .reduceLeftOption[Expression]((e1, e2) => Or(e1, e2)) + // Once we add an `If` check above the udf, it is safe to mark those checked inputs + // as null-safe (i.e., set `inputsNullSafe` all `true`), because the null-returning + // branch of `If` will be called if any of these checked inputs is null. Thus we can + // prevent this rule from being applied repeatedly. + val newInputsNullSafe = inputsNullSafe.map(_ => true) + inputsNullCheck + .map(If(_, Literal.create(null, udf.dataType), + udf.copy(inputsNullSafe = newInputsNullSafe))) + .getOrElse(udf) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala index 8954fe8a58e6..fae90caebf96 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDF.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.types.DataType @@ -31,6 +31,9 @@ import org.apache.spark.sql.types.DataType * null. Use boxed type or [[Option]] if you wanna do the null-handling yourself. * @param dataType Return type of function. * @param children The input expressions of this UDF. + * @param inputsNullSafe Whether the inputs are of non-primitive types or not nullable. Null values + * of Scala primitive types will be converted to the type's default value and + * lead to wrong results, thus need special handling before calling the UDF. * @param inputTypes The expected input types of this UDF, used to perform type coercion. If we do * not want to perform coercion, simply use "Nil". Note that it would've been * better to use Option of Seq[DataType] so we can use "None" as the case for no @@ -39,17 +42,16 @@ import org.apache.spark.sql.types.DataType * @param nullable True if the UDF can return null value. * @param udfDeterministic True if the UDF is deterministic. Deterministic UDF returns same result * each time it is invoked with a particular input. - * @param nullableTypes which of the inputTypes are nullable (i.e. not primitive) */ case class ScalaUDF( function: AnyRef, dataType: DataType, children: Seq[Expression], + inputsNullSafe: Seq[Boolean], inputTypes: Seq[DataType] = Nil, udfName: Option[String] = None, nullable: Boolean = true, - udfDeterministic: Boolean = true, - nullableTypes: Seq[Boolean] = Nil) + udfDeterministic: Boolean = true) extends Expression with ImplicitCastInputTypes with NonSQLExpression with UserDefinedExpression { // The constructor for SPARK 2.1 and 2.2 @@ -60,8 +62,8 @@ case class ScalaUDF( inputTypes: Seq[DataType], udfName: Option[String]) = { this( - function, dataType, children, inputTypes, udfName, nullable = true, - udfDeterministic = true, nullableTypes = Nil) + function, dataType, children, ScalaReflection.getParameterTypeNullability(function), + inputTypes, udfName, nullable = true, udfDeterministic = true) } override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index cf76c92b093b..d8cb6f7caa99 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -314,24 +314,24 @@ class AnalysisSuite extends AnalysisTest with Matchers { } // non-primitive parameters do not need special null handling - val udf1 = ScalaUDF((s: String) => "x", StringType, string :: Nil) + val udf1 = ScalaUDF((s: String) => "x", StringType, string :: Nil, true :: Nil) val expected1 = udf1 checkUDF(udf1, expected1) // only primitive parameter needs special null handling val udf2 = ScalaUDF((s: String, d: Double) => "x", StringType, string :: double :: Nil, - nullableTypes = true :: false :: Nil) + true :: false :: Nil) val expected2 = - If(IsNull(double), nullResult, udf2.copy(children = string :: KnownNotNull(double) :: Nil)) + If(IsNull(double), nullResult, udf2.copy(inputsNullSafe = true :: true :: Nil)) checkUDF(udf2, expected2) // special null handling should apply to all primitive parameters val udf3 = ScalaUDF((s: Short, d: Double) => "x", StringType, short :: double :: Nil, - nullableTypes = false :: false :: Nil) + false :: false :: Nil) val expected3 = If( IsNull(short) || IsNull(double), nullResult, - udf3.copy(children = KnownNotNull(short) :: KnownNotNull(double) :: Nil)) + udf3.copy(inputsNullSafe = true :: true :: Nil)) checkUDF(udf3, expected3) // we can skip special null handling for primitive parameters that are not nullable @@ -340,19 +340,19 @@ class AnalysisSuite extends AnalysisTest with Matchers { (s: Short, d: Double) => "x", StringType, short :: double.withNullability(false) :: Nil, - nullableTypes = false :: false :: Nil) + false :: false :: Nil) val expected4 = If( IsNull(short), nullResult, - udf4.copy(children = KnownNotNull(short) :: double.withNullability(false) :: Nil)) + udf4.copy(inputsNullSafe = true :: true :: Nil)) // checkUDF(udf4, expected4) } test("SPARK-24891 Fix HandleNullInputsForUDF rule") { val a = testRelation.output(0) val func = (x: Int, y: Int) => x + y - val udf1 = ScalaUDF(func, IntegerType, a :: a :: Nil, nullableTypes = false :: false :: Nil) - val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil, nullableTypes = false :: false :: Nil) + val udf1 = ScalaUDF(func, IntegerType, a :: a :: Nil, false :: false :: Nil) + val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil, false :: false :: Nil) val plan = Project(Alias(udf2, "")() :: Nil, testRelation) comparePlans(plan.analyze, plan.analyze.analyze) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala index e083ae008924..467cfd5598ff 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ScalaUDFSuite.scala @@ -26,10 +26,10 @@ import org.apache.spark.sql.types.{IntegerType, StringType} class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { test("basic") { - val intUdf = ScalaUDF((i: Int) => i + 1, IntegerType, Literal(1) :: Nil) + val intUdf = ScalaUDF((i: Int) => i + 1, IntegerType, Literal(1) :: Nil, true :: Nil) checkEvaluation(intUdf, 2) - val stringUdf = ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil) + val stringUdf = ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, true :: Nil) checkEvaluation(stringUdf, "ax") } @@ -37,7 +37,8 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { val udf = ScalaUDF( (s: String) => s.toLowerCase(Locale.ROOT), StringType, - Literal.create(null, StringType) :: Nil) + Literal.create(null, StringType) :: Nil, + true :: Nil) val e1 = intercept[SparkException](udf.eval()) assert(e1.getMessage.contains("Failed to execute user defined function")) @@ -50,7 +51,7 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { test("SPARK-22695: ScalaUDF should not use global variables") { val ctx = new CodegenContext - ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil).genCode(ctx) + ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, true :: Nil).genCode(ctx) assert(ctx.inlinedMutableStates.isEmpty) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index b7092f4c42d4..64aa1ee39046 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -564,7 +564,7 @@ class TreeNodeSuite extends SparkFunSuite { } test("toJSON should not throws java.lang.StackOverflowError") { - val udf = ScalaUDF(SelfReferenceUDF(), BooleanType, Seq("col1".attr)) + val udf = ScalaUDF(SelfReferenceUDF(), BooleanType, Seq("col1".attr), true :: Nil) // Should not throw java.lang.StackOverflowError udf.toJSON } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala index c37ba0c60c3d..aa3a6c3bf122 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UDFRegistration.scala @@ -113,7 +113,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends (0 to 22).foreach { x => val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"}) val typeTags = (1 to x).map(i => s"A$i: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _) - val inputSchemas = (1 to x).foldRight("Nil")((i, s) => {s"ScalaReflection.schemaFor[A$i] :: $s"}) + val inputSchemas = (1 to x).foldRight("Nil")((i, s) => {s"Try(ScalaReflection.schemaFor[A$i]).toOption :: $s"}) println(s""" |/** | * Registers a deterministic Scala closure of $x arguments as user-defined function (UDF). @@ -122,10 +122,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends | */ |def register[$typeTags](name: String, func: Function$x[$types]): UserDefinedFunction = { | val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - | val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try($inputSchemas).toOption + | val inputSchemas: Seq[Option[ScalaReflection.Schema]] = $inputSchemas | def builder(e: Seq[Expression]) = if (e.length == $x) { - | ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - | udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + | ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + | if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + | Some(name), nullable, udfDeterministic = true) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $x; Found: " + e.length) @@ -151,7 +152,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { | val func = f$anyCast.call($anyParams) | def builder(e: Seq[Expression]) = if (e.length == $i) { - | ScalaUDF($funcCall, returnType, e, udfName = Some(name)) + | ScalaUDF($funcCall, returnType, e, e.map(_ => true), udfName = Some(name)) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $i; Found: " + e.length) @@ -168,10 +169,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag](name: String, func: Function0[RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Nil def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) @@ -188,10 +190,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag](name: String, func: Function1[A1, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) @@ -208,10 +211,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag](name: String, func: Function2[A1, A2, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) @@ -228,10 +232,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](name: String, func: Function3[A1, A2, A3, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) @@ -248,10 +253,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](name: String, func: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) @@ -268,10 +274,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](name: String, func: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) @@ -288,10 +295,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](name: String, func: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) @@ -308,10 +316,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](name: String, func: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) @@ -328,10 +337,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](name: String, func: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) @@ -348,10 +358,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](name: String, func: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) @@ -368,10 +379,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](name: String, func: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) @@ -388,10 +400,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag](name: String, func: Function11[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) @@ -408,10 +421,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag](name: String, func: Function12[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) @@ -428,10 +442,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag](name: String, func: Function13[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) @@ -448,10 +463,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag](name: String, func: Function14[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) @@ -468,10 +484,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag](name: String, func: Function15[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) @@ -488,10 +505,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag](name: String, func: Function16[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) @@ -508,10 +526,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag](name: String, func: Function17[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) @@ -528,10 +547,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag](name: String, func: Function18[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) @@ -548,10 +568,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag](name: String, func: Function19[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) @@ -568,10 +589,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag](name: String, func: Function20[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: ScalaReflection.schemaFor[A20] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) @@ -588,10 +610,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag](name: String, func: Function21[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: ScalaReflection.schemaFor[A20] :: ScalaReflection.schemaFor[A21] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Try(ScalaReflection.schemaFor[A21]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) @@ -608,10 +631,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ def register[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag, A11: TypeTag, A12: TypeTag, A13: TypeTag, A14: TypeTag, A15: TypeTag, A16: TypeTag, A17: TypeTag, A18: TypeTag, A19: TypeTag, A20: TypeTag, A21: TypeTag, A22: TypeTag](name: String, func: Function22[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, A11, A12, A13, A14, A15, A16, A17, A18, A19, A20, A21, A22, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: ScalaReflection.schemaFor[A20] :: ScalaReflection.schemaFor[A21] :: ScalaReflection.schemaFor[A22] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Try(ScalaReflection.schemaFor[A2]).toOption :: Try(ScalaReflection.schemaFor[A3]).toOption :: Try(ScalaReflection.schemaFor[A4]).toOption :: Try(ScalaReflection.schemaFor[A5]).toOption :: Try(ScalaReflection.schemaFor[A6]).toOption :: Try(ScalaReflection.schemaFor[A7]).toOption :: Try(ScalaReflection.schemaFor[A8]).toOption :: Try(ScalaReflection.schemaFor[A9]).toOption :: Try(ScalaReflection.schemaFor[A10]).toOption :: Try(ScalaReflection.schemaFor[A11]).toOption :: Try(ScalaReflection.schemaFor[A12]).toOption :: Try(ScalaReflection.schemaFor[A13]).toOption :: Try(ScalaReflection.schemaFor[A14]).toOption :: Try(ScalaReflection.schemaFor[A15]).toOption :: Try(ScalaReflection.schemaFor[A16]).toOption :: Try(ScalaReflection.schemaFor[A17]).toOption :: Try(ScalaReflection.schemaFor[A18]).toOption :: Try(ScalaReflection.schemaFor[A19]).toOption :: Try(ScalaReflection.schemaFor[A20]).toOption :: Try(ScalaReflection.schemaFor[A21]).toOption :: Try(ScalaReflection.schemaFor[A22]).toOption :: Nil def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.nullable).getOrElse(true)), + if (inputSchemas.contains(None)) Nil else inputSchemas.map(_.get.dataType), + Some(name), nullable, udfDeterministic = true) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) @@ -719,7 +743,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF0[_], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF0[Any]].call() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(() => func, returnType, e, udfName = Some(name)) + ScalaUDF(() => func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) @@ -734,7 +758,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF1[_, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) @@ -749,7 +773,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF2[_, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) @@ -764,7 +788,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF3[_, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) @@ -779,7 +803,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF4[_, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) @@ -794,7 +818,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF5[_, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) @@ -809,7 +833,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF6[_, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) @@ -824,7 +848,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) @@ -839,7 +863,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) @@ -854,7 +878,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) @@ -869,7 +893,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) @@ -884,7 +908,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF11[_, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF11[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) @@ -899,7 +923,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF12[_, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF12[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) @@ -914,7 +938,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF13[_, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF13[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) @@ -929,7 +953,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF14[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF14[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) @@ -944,7 +968,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF15[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF15[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) @@ -959,7 +983,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF16[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF16[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) @@ -974,7 +998,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF17[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF17[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) @@ -989,7 +1013,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF18[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF18[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) @@ -1004,7 +1028,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF19[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF19[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) @@ -1019,7 +1043,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF20[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF20[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) @@ -1034,7 +1058,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF21[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF21[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) @@ -1049,7 +1073,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends def register(name: String, f: UDF22[_, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _, _], returnType: DataType): Unit = { val func = f.asInstanceOf[UDF22[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => true), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala index 6499328e89ce..10733810b641 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatDataWriter.scala @@ -179,7 +179,8 @@ class DynamicPartitionDataWriter( val partitionName = ScalaUDF( ExternalCatalogUtils.getPartitionPathString _, StringType, - Seq(Literal(c.name), Cast(c, StringType, Option(description.timeZoneId)))) + Seq(Literal(c.name), Cast(c, StringType, Option(description.timeZoneId))), + Seq(true, true)) if (i == 0) Seq(partitionName) else Seq(Literal(Path.SEPARATOR), partitionName) }) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala index 697757f8a73c..eb956c4b3e88 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/UserDefinedFunction.scala @@ -73,19 +73,24 @@ case class UserDefinedFunction protected[sql] ( */ @scala.annotation.varargs def apply(exprs: Column*): Column = { - if (inputTypes.isDefined && nullableTypes.isDefined) { - require(inputTypes.get.length == nullableTypes.get.length) + // TODO: make sure this class is only instantiated through `SparkUserDefinedFunction.create()` + // and `nullableTypes` is always set. + if (nullableTypes.isEmpty) { + nullableTypes = Some(ScalaReflection.getParameterTypeNullability(f)) + } + if (inputTypes.isDefined) { + assert(inputTypes.get.length == nullableTypes.get.length) } Column(ScalaUDF( f, dataType, exprs.map(_.expr), + nullableTypes.get, inputTypes.getOrElse(Nil), udfName = _nameOption, nullable = _nullable, - udfDeterministic = _deterministic, - nullableTypes = nullableTypes.getOrElse(Nil))) + udfDeterministic = _deterministic)) } private def copyAll(): UserDefinedFunction = { @@ -146,9 +151,14 @@ private[sql] object SparkUserDefinedFunction { def create( f: AnyRef, dataType: DataType, - inputSchemas: Option[Seq[ScalaReflection.Schema]]): UserDefinedFunction = { - val udf = new UserDefinedFunction(f, dataType, inputSchemas.map(_.map(_.dataType))) - udf.nullableTypes = inputSchemas.map(_.map(_.nullable)) + inputSchemas: Seq[Option[ScalaReflection.Schema]]): UserDefinedFunction = { + val inputTypes = if (inputSchemas.contains(None)) { + None + } else { + Some(inputSchemas.map(_.get.dataType)) + } + val udf = new UserDefinedFunction(f, dataType, inputTypes) + udf.nullableTypes = Some(inputSchemas.map(_.map(_.nullable).getOrElse(true))) udf } } 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 10b67d7a1ca5..6a43ce160efe 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 @@ -3819,7 +3819,7 @@ object functions { (0 to 10).foreach { x => val types = (1 to x).foldRight("RT")((i, s) => {s"A$i, $s"}) val typeTags = (1 to x).map(i => s"A$i: TypeTag").foldLeft("RT: TypeTag")(_ + ", " + _) - val inputSchemas = (1 to x).foldRight("Nil")((i, s) => {s"ScalaReflection.schemaFor(typeTag[A$i]) :: $s"}) + val inputSchemas = (1 to x).foldRight("Nil")((i, s) => {s"Try(ScalaReflection.schemaFor(typeTag[A$i])).toOption :: $s"}) println(s""" |/** | * Defines a Scala closure of $x arguments as user-defined function (UDF). @@ -3832,7 +3832,7 @@ object functions { | */ |def udf[$typeTags](f: Function$x[$types]): UserDefinedFunction = { | val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - | val inputSchemas = Try($inputTypes).toOption + | val inputSchemas = $inputSchemas | val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) | if (nullable) udf else udf.asNonNullable() |}""".stripMargin) @@ -3856,7 +3856,7 @@ object functions { | */ |def udf(f: UDF$i[$extTypeArgs], returnType: DataType): UserDefinedFunction = { | val func = f$anyCast.call($anyParams) - | SparkUserDefinedFunction.create($funcCall, returnType, inputSchemas = None) + | SparkUserDefinedFunction.create($funcCall, returnType, inputSchemas = Seq.fill($i)(None)) |}""".stripMargin) } @@ -3877,7 +3877,7 @@ object functions { */ def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(Nil).toOption + val inputSchemas = Nil val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3893,7 +3893,7 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag](f: Function1[A1, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1]) :: Nil).toOption + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Nil val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3909,7 +3909,7 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag](f: Function2[A1, A2, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1]) :: ScalaReflection.schemaFor(typeTag[A2]) :: Nil).toOption + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Nil val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3925,7 +3925,7 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag](f: Function3[A1, A2, A3, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: Nil).toOption + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Nil val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3941,7 +3941,7 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag](f: Function4[A1, A2, A3, A4, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: Nil).toOption + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Nil val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3957,7 +3957,7 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag](f: Function5[A1, A2, A3, A4, A5, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: Nil).toOption + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Nil val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3973,7 +3973,7 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag](f: Function6[A1, A2, A3, A4, A5, A6, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: ScalaReflection.schemaFor(typeTag[A6]) :: Nil).toOption + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Nil val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3989,7 +3989,7 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag](f: Function7[A1, A2, A3, A4, A5, A6, A7, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: ScalaReflection.schemaFor(typeTag[A6]) :: ScalaReflection.schemaFor(typeTag[A7]) :: Nil).toOption + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Nil val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -4005,7 +4005,7 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag](f: Function8[A1, A2, A3, A4, A5, A6, A7, A8, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: ScalaReflection.schemaFor(typeTag[A6]) :: ScalaReflection.schemaFor(typeTag[A7]) :: ScalaReflection.schemaFor(typeTag[A8]) :: Nil).toOption + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A8])).toOption :: Nil val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -4021,7 +4021,7 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag](f: Function9[A1, A2, A3, A4, A5, A6, A7, A8, A9, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: ScalaReflection.schemaFor(typeTag[A6]) :: ScalaReflection.schemaFor(typeTag[A7]) :: ScalaReflection.schemaFor(typeTag[A8]) :: ScalaReflection.schemaFor(typeTag[A9]) :: Nil).toOption + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A8])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A9])).toOption :: Nil val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -4037,7 +4037,7 @@ object functions { */ def udf[RT: TypeTag, A1: TypeTag, A2: TypeTag, A3: TypeTag, A4: TypeTag, A5: TypeTag, A6: TypeTag, A7: TypeTag, A8: TypeTag, A9: TypeTag, A10: TypeTag](f: Function10[A1, A2, A3, A4, A5, A6, A7, A8, A9, A10, RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: ScalaReflection.schemaFor(typeTag[A6]) :: ScalaReflection.schemaFor(typeTag[A7]) :: ScalaReflection.schemaFor(typeTag[A8]) :: ScalaReflection.schemaFor(typeTag[A9]) :: ScalaReflection.schemaFor(typeTag[A10]) :: Nil).toOption + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A3])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A4])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A5])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A6])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A7])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A8])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A9])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A10])).toOption :: Nil val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -4057,7 +4057,7 @@ object functions { */ def udf(f: UDF0[_], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF0[Any]].call() - SparkUserDefinedFunction.create(() => func, returnType, inputSchemas = None) + SparkUserDefinedFunction.create(() => func, returnType, inputSchemas = Seq.fill(0)(None)) } /** @@ -4071,7 +4071,7 @@ object functions { */ def udf(f: UDF1[_, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(1)(None)) } /** @@ -4085,7 +4085,7 @@ object functions { */ def udf(f: UDF2[_, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(2)(None)) } /** @@ -4099,7 +4099,7 @@ object functions { */ def udf(f: UDF3[_, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(3)(None)) } /** @@ -4113,7 +4113,7 @@ object functions { */ def udf(f: UDF4[_, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF4[Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(4)(None)) } /** @@ -4127,7 +4127,7 @@ object functions { */ def udf(f: UDF5[_, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF5[Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(5)(None)) } /** @@ -4141,7 +4141,7 @@ object functions { */ def udf(f: UDF6[_, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF6[Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(6)(None)) } /** @@ -4155,7 +4155,7 @@ object functions { */ def udf(f: UDF7[_, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF7[Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(7)(None)) } /** @@ -4169,7 +4169,7 @@ object functions { */ def udf(f: UDF8[_, _, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF8[Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(8)(None)) } /** @@ -4183,7 +4183,7 @@ object functions { */ def udf(f: UDF9[_, _, _, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF9[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(9)(None)) } /** @@ -4197,7 +4197,7 @@ object functions { */ def udf(f: UDF10[_, _, _, _, _, _, _, _, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF10[Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any]].call(_: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction.create(func, returnType, inputSchemas = Seq.fill(10)(None)) } // scalastyle:on parameter.number @@ -4216,7 +4216,9 @@ object functions { * @since 2.0.0 */ def udf(f: AnyRef, dataType: DataType): UserDefinedFunction = { - SparkUserDefinedFunction.create(f, dataType, inputSchemas = None) + // TODO: should call SparkUserDefinedFunction.create() instead but inputSchemas is currently + // unavailable. We may need to create type-safe overloaded versions of udf() methods. + new UserDefinedFunction(f, dataType, inputTypes = None) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala index 30dca9497ddd..f8ed21bbf7c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -393,4 +393,28 @@ class UDFSuite extends QueryTest with SharedSQLContext { checkAnswer(df, Seq(Row("12"), Row("24"), Row("3null"), Row(null))) } } + + test("SPARK-25044 Verify null input handling for primitive types - with udf()") { + val udf1 = udf((x: Long, y: Any) => x * 2 + (if (y == null) 1 else 0)) + val df = spark.range(0, 3).toDF("a") + .withColumn("b", udf1($"a", lit(null))) + .withColumn("c", udf1(lit(null), $"a")) + + checkAnswer( + df, + Seq( + Row(0, 1, null), + Row(1, 3, null), + Row(2, 5, null))) + } + + test("SPARK-25044 Verify null input handling for primitive types - with udf.register") { + withTable("t") { + Seq((null, new Integer(1), "x"), ("M", null, "y"), ("N", new Integer(3), null)) + .toDF("a", "b", "c").write.format("json").saveAsTable("t") + spark.udf.register("f", (a: String, b: Int, c: Any) => a + b + c) + val df = spark.sql("SELECT f(a, b, c) FROM t") + checkAnswer(df, Seq(Row("null1x"), Row(null), Row("N3null"))) + } + } } From 6a06b8ccef57017172666cd004d5eb6be994d19e Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Fri, 19 Oct 2018 21:17:14 +0800 Subject: [PATCH 179/879] [SPARK-25768][SQL] fix constant argument expecting UDAFs ## What changes were proposed in this pull request? Without this PR some UDAFs like `GenericUDAFPercentileApprox` can throw an exception because expecting a constant parameter (object inspector) as a particular argument. The exception is thrown because `toPrettySQL` call in `ResolveAliases` analyzer rule transforms a `Literal` parameter to a `PrettyAttribute` which is then transformed to an `ObjectInspector` instead of a `ConstantObjectInspector`. The exception comes from `getEvaluator` method of `GenericUDAFPercentileApprox` that actually shouldn't be called during `toPrettySQL` transformation. The reason why it is called are the non lazy fields in `HiveUDAFFunction`. This PR makes all fields of `HiveUDAFFunction` lazy. ## How was this patch tested? added new UT Closes #22766 from peter-toth/SPARK-25768. Authored-by: Peter Toth Signed-off-by: Wenchen Fan (cherry picked from commit f38594fc561208e17af80d17acf8da362b91fca4) Signed-off-by: Wenchen Fan --- .../org/apache/spark/sql/hive/hiveUDFs.scala | 53 ++++++++++--------- .../sql/hive/execution/HiveUDFSuite.scala | 14 +++++ 2 files changed, 42 insertions(+), 25 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index 68af99ea272a..4a8450901e3a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -340,39 +340,40 @@ private[hive] case class HiveUDAFFunction( resolver.getEvaluator(parameterInfo) } - // The UDAF evaluator used to consume raw input rows and produce partial aggregation results. - @transient - private lazy val partial1ModeEvaluator = newEvaluator() + private case class HiveEvaluator( + evaluator: GenericUDAFEvaluator, + objectInspector: ObjectInspector) + // The UDAF evaluator used to consume raw input rows and produce partial aggregation results. // Hive `ObjectInspector` used to inspect partial aggregation results. @transient - private val partialResultInspector = partial1ModeEvaluator.init( - GenericUDAFEvaluator.Mode.PARTIAL1, - inputInspectors - ) + private lazy val partial1HiveEvaluator = { + val evaluator = newEvaluator() + HiveEvaluator(evaluator, evaluator.init(GenericUDAFEvaluator.Mode.PARTIAL1, inputInspectors)) + } // The UDAF evaluator used to merge partial aggregation results. @transient private lazy val partial2ModeEvaluator = { val evaluator = newEvaluator() - evaluator.init(GenericUDAFEvaluator.Mode.PARTIAL2, Array(partialResultInspector)) + evaluator.init(GenericUDAFEvaluator.Mode.PARTIAL2, Array(partial1HiveEvaluator.objectInspector)) evaluator } // Spark SQL data type of partial aggregation results @transient - private lazy val partialResultDataType = inspectorToDataType(partialResultInspector) + private lazy val partialResultDataType = + inspectorToDataType(partial1HiveEvaluator.objectInspector) // The UDAF evaluator used to compute the final result from a partial aggregation result objects. - @transient - private lazy val finalModeEvaluator = newEvaluator() - // Hive `ObjectInspector` used to inspect the final aggregation result object. @transient - private val returnInspector = finalModeEvaluator.init( - GenericUDAFEvaluator.Mode.FINAL, - Array(partialResultInspector) - ) + private lazy val finalHiveEvaluator = { + val evaluator = newEvaluator() + HiveEvaluator( + evaluator, + evaluator.init(GenericUDAFEvaluator.Mode.FINAL, Array(partial1HiveEvaluator.objectInspector))) + } // Wrapper functions used to wrap Spark SQL input arguments into Hive specific format. @transient @@ -381,7 +382,7 @@ private[hive] case class HiveUDAFFunction( // Unwrapper function used to unwrap final aggregation result objects returned by Hive UDAFs into // Spark SQL specific format. @transient - private lazy val resultUnwrapper = unwrapperFor(returnInspector) + private lazy val resultUnwrapper = unwrapperFor(finalHiveEvaluator.objectInspector) @transient private lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length) @@ -391,7 +392,7 @@ private[hive] case class HiveUDAFFunction( override def nullable: Boolean = true - override lazy val dataType: DataType = inspectorToDataType(returnInspector) + override lazy val dataType: DataType = inspectorToDataType(finalHiveEvaluator.objectInspector) override def prettyName: String = name @@ -401,13 +402,13 @@ private[hive] case class HiveUDAFFunction( } override def createAggregationBuffer(): AggregationBuffer = - partial1ModeEvaluator.getNewAggregationBuffer + partial1HiveEvaluator.evaluator.getNewAggregationBuffer @transient private lazy val inputProjection = UnsafeProjection.create(children) override def update(buffer: AggregationBuffer, input: InternalRow): AggregationBuffer = { - partial1ModeEvaluator.iterate( + partial1HiveEvaluator.evaluator.iterate( buffer, wrap(inputProjection(input), inputWrappers, cached, inputDataTypes)) buffer } @@ -417,12 +418,12 @@ private[hive] case class HiveUDAFFunction( // buffer in the 3rd format mentioned in the ScalaDoc of this class. Originally, Hive converts // this `AggregationBuffer`s into this format before shuffling partial aggregation results, and // calls `GenericUDAFEvaluator.terminatePartial()` to do the conversion. - partial2ModeEvaluator.merge(buffer, partial1ModeEvaluator.terminatePartial(input)) + partial2ModeEvaluator.merge(buffer, partial1HiveEvaluator.evaluator.terminatePartial(input)) buffer } override def eval(buffer: AggregationBuffer): Any = { - resultUnwrapper(finalModeEvaluator.terminate(buffer)) + resultUnwrapper(finalHiveEvaluator.evaluator.terminate(buffer)) } override def serialize(buffer: AggregationBuffer): Array[Byte] = { @@ -439,9 +440,10 @@ private[hive] case class HiveUDAFFunction( // Helper class used to de/serialize Hive UDAF `AggregationBuffer` objects private class AggregationBufferSerDe { - private val partialResultUnwrapper = unwrapperFor(partialResultInspector) + private val partialResultUnwrapper = unwrapperFor(partial1HiveEvaluator.objectInspector) - private val partialResultWrapper = wrapperFor(partialResultInspector, partialResultDataType) + private val partialResultWrapper = + wrapperFor(partial1HiveEvaluator.objectInspector, partialResultDataType) private val projection = UnsafeProjection.create(Array(partialResultDataType)) @@ -451,7 +453,8 @@ private[hive] case class HiveUDAFFunction( // `GenericUDAFEvaluator.terminatePartial()` converts an `AggregationBuffer` into an object // that can be inspected by the `ObjectInspector` returned by `GenericUDAFEvaluator.init()`. // Then we can unwrap it to a Spark SQL value. - mutableRow.update(0, partialResultUnwrapper(partial1ModeEvaluator.terminatePartial(buffer))) + mutableRow.update(0, partialResultUnwrapper( + partial1HiveEvaluator.evaluator.terminatePartial(buffer))) val unsafeRow = projection(mutableRow) val bytes = ByteBuffer.allocate(unsafeRow.getSizeInBytes) unsafeRow.writeTo(bytes) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index 6198d4963df3..a6fc744cc8b5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -638,6 +638,20 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { Row(3) :: Row(3) :: Nil) } } + + test("SPARK-25768 constant argument expecting Hive UDF") { + withTempView("inputTable") { + spark.range(10).createOrReplaceTempView("inputTable") + withUserDefinedFunction("testGenericUDAFPercentileApprox" -> false) { + val numFunc = spark.catalog.listFunctions().count() + sql(s"CREATE FUNCTION testGenericUDAFPercentileApprox AS '" + + s"${classOf[GenericUDAFPercentileApprox].getName}'") + checkAnswer( + sql("SELECT testGenericUDAFPercentileApprox(id, 0.5) FROM inputTable"), + Seq(Row(4.0))) + } + } + } } class TestPair(x: Int, y: Int) extends Writable with Serializable { From 8926c4a6237ab059875aa7502d6417317d58381a Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 19 Oct 2018 21:34:35 +0800 Subject: [PATCH 180/879] fix security issue of zinc --- build/mvn | 31 ++++++++++++++++++++++++------- 1 file changed, 24 insertions(+), 7 deletions(-) diff --git a/build/mvn b/build/mvn index 2487b81abb4e..0289ef3c5dc5 100755 --- a/build/mvn +++ b/build/mvn @@ -139,8 +139,17 @@ if [ "$1" == "--force" ]; then shift fi +if [ "$1" == "--zinc" ]; then + echo "Using zinc for incremental compilation. Be sure you are aware of the implications of " + echo "running this server process on your machine" + USE_ZINC=1 + shift +fi + # Install the proper version of Scala, Zinc and Maven for the build -install_zinc +if [ -n "${USE_ZINC}" ]; then + install_zinc +fi install_scala install_mvn @@ -149,12 +158,15 @@ cd "${_CALLING_DIR}" # Now that zinc is ensured to be installed, check its status and, if its # not running or just installed, start it -if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`"${ZINC_BIN}" -status -port ${ZINC_PORT}`" ]; then - export ZINC_OPTS=${ZINC_OPTS:-"$_COMPILE_JVM_OPTS"} - "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} - "${ZINC_BIN}" -start -port ${ZINC_PORT} \ - -scala-compiler "${SCALA_COMPILER}" \ - -scala-library "${SCALA_LIBRARY}" &>/dev/null +if [ -n "${USE_ZINC}" ]; then + if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`"${ZINC_BIN}" -status -port ${ZINC_PORT}`" ]; then + export ZINC_OPTS=${ZINC_OPTS:-"$_COMPILE_JVM_OPTS"} + "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} + "${ZINC_BIN}" -start -port ${ZINC_PORT} -server 127.0.0.1 \ + -idle-timeout 30m \ + -scala-compiler "${SCALA_COMPILER}" \ + -scala-library "${SCALA_LIBRARY}" &>/dev/null + fi fi # Set any `mvn` options if not already present @@ -164,3 +176,8 @@ echo "Using \`mvn\` from path: $MVN_BIN" 1>&2 # Last, call the `mvn` command as usual "${MVN_BIN}" -DzincPort=${ZINC_PORT} "$@" + +if [ -n "${USE_ZINC}" ]; then + # Try to shut down zinc explicitly + "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} +fi From 1ff8dd424041f8dd525d1ec97828d9663f1f10ea Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 19 Oct 2018 14:22:00 +0000 Subject: [PATCH 181/879] Preparing Spark release v2.4.0-rc4 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 714b6f18b19d..f52d785e05cd 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.0 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index ee0de73f21d5..63ab510eb683 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b89e0fed2a66..b10e11849a74 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 3105a8182afc..74c6d232b36a 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f85248b53080..fbdc979d120d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 3350990c0461..53d1b2b58f1f 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 9a2cb90a2f30..98145481adc7 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 9b927a9ba525..f0c8ebe25e63 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 718875f88e74..c635cd3b9394 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 583a2fff7638..ed3efffa180b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 72473771c2e9..7f9a91e2fba0 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1-SNAPSHOT -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.0 +SPARK_VERSION_SHORT: 2.4.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 737ebfd7900c..c7b8354b0968 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 7c6b78ac2b31..a91c13362caa 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 6343a15c43a8..db239f23e1de 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 2f4886832b75..cf19fed1bef5 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9404e2a52d90..da5dba82f3cf 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 73dbc8d99449..e591ce802256 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 912d82b764e5..e8b13cdc2bef 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 8454f5c108dc..c3d3b888656b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 05f7cf870af8..468ace0ff8d7 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 0e935ac112af..db8fda66c3cd 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 7977389718cb..aa1e1267f57b 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7193df69900d..7d07e18f04a2 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 4b318b00ff5b..f545f212b2f0 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6fbb1b7db6aa..6b388ffcf18b 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 9a42d9a69967..fc369c73edc0 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 702f60ab9f00..f4cb94147d9e 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 62322da714bd..22f60efae9af 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index c8e45b27c78f..b17e67d78b98 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 6201e0dd3c9a..bf0d406da9ed 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 28804be0301c..4cc619e697e4 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 87282141fa64..53e3424394ba 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1.dev0" +__version__ = "2.4.0" diff --git a/repl/pom.xml b/repl/pom.xml index d09548348eb5..4532f0b8686e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index a9d9c8de9959..47fe3f2765b2 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 793167103179..f59895e9a1bd 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index e6457289b859..0eacbbfac2b9 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index e70711ce7257..2afdc15be446 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 65cf33f32972..ef1d012d2f80 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c327b85ee866..f4be074642ae 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d9d742444e7e..e5c1065451f7 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c428be67aa3d..b522bfeac130 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 37e96b807653..8e71adabb3ec 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index dd00c2f6cd2c..c00e1b851ba0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml From 9c0c6d4d5039267be35e564d8d6712318d557317 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 19 Oct 2018 14:22:04 +0000 Subject: [PATCH 182/879] Preparing development version 2.4.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index f52d785e05cd..714b6f18b19d 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.0 +Version: 2.4.1 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 63ab510eb683..ee0de73f21d5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b10e11849a74..b89e0fed2a66 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 74c6d232b36a..3105a8182afc 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index fbdc979d120d..f85248b53080 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 53d1b2b58f1f..3350990c0461 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 98145481adc7..9a2cb90a2f30 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index f0c8ebe25e63..9b927a9ba525 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index c635cd3b9394..718875f88e74 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ed3efffa180b..583a2fff7638 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7f9a91e2fba0..72473771c2e9 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.0 -SPARK_VERSION_SHORT: 2.4.0 +SPARK_VERSION: 2.4.1-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index c7b8354b0968..737ebfd7900c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index a91c13362caa..7c6b78ac2b31 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index db239f23e1de..6343a15c43a8 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index cf19fed1bef5..2f4886832b75 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index da5dba82f3cf..9404e2a52d90 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index e591ce802256..73dbc8d99449 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index e8b13cdc2bef..912d82b764e5 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index c3d3b888656b..8454f5c108dc 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 468ace0ff8d7..05f7cf870af8 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index db8fda66c3cd..0e935ac112af 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index aa1e1267f57b..7977389718cb 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7d07e18f04a2..7193df69900d 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f545f212b2f0..4b318b00ff5b 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6b388ffcf18b..6fbb1b7db6aa 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index fc369c73edc0..9a42d9a69967 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index f4cb94147d9e..702f60ab9f00 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 22f60efae9af..62322da714bd 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index b17e67d78b98..c8e45b27c78f 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bf0d406da9ed..6201e0dd3c9a 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4cc619e697e4..28804be0301c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 53e3424394ba..87282141fa64 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.0" +__version__ = "2.4.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 4532f0b8686e..d09548348eb5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 47fe3f2765b2..a9d9c8de9959 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index f59895e9a1bd..793167103179 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 0eacbbfac2b9..e6457289b859 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 2afdc15be446..e70711ce7257 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ef1d012d2f80..65cf33f32972 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f4be074642ae..c327b85ee866 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index e5c1065451f7..d9d742444e7e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b522bfeac130..c428be67aa3d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 8e71adabb3ec..37e96b807653 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c00e1b851ba0..dd00c2f6cd2c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml From 1001d2314275c902da519725da266a23b537e33a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 19 Oct 2018 12:52:41 -0500 Subject: [PATCH 183/879] [SPARK-25704][CORE] Allocate a bit less than Int.MaxValue JVMs don't you allocate arrays of length exactly Int.MaxValue, so leave a little extra room. This is necessary when reading blocks >2GB off the network (for remote reads or for cache replication). Unit tests via jenkins, ran a test with blocks over 2gb on a cluster Closes #22705 from squito/SPARK-25704. Authored-by: Imran Rashid Signed-off-by: Imran Rashid --- .../org/apache/spark/storage/BlockManager.scala | 6 ++---- .../apache/spark/util/io/ChunkedByteBuffer.scala | 16 +++++++++------- 2 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 0fe82ac0cedc..c01a45315191 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -133,8 +133,6 @@ private[spark] class BlockManager( private[spark] val externalShuffleServiceEnabled = conf.get(config.SHUFFLE_SERVICE_ENABLED) - private val chunkSize = - conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", Int.MaxValue.toString).toInt private val remoteReadNioBufferConversion = conf.getBoolean("spark.network.remoteReadNioBufferConversion", false) @@ -451,7 +449,7 @@ private[spark] class BlockManager( new EncryptedBlockData(tmpFile, blockSize, conf, key).toChunkedByteBuffer(allocator) case None => - ChunkedByteBuffer.fromFile(tmpFile, conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS).toInt) + ChunkedByteBuffer.fromFile(tmpFile) } putBytes(blockId, buffer, level)(classTag) tmpFile.delete() @@ -797,7 +795,7 @@ private[spark] class BlockManager( if (remoteReadNioBufferConversion) { return Some(new ChunkedByteBuffer(data.nioByteBuffer())) } else { - return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize)) + return Some(ChunkedByteBuffer.fromManagedBuffer(data)) } } logDebug(s"The value of block $blockId is null") diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala index 4aa8d45ec740..9547cb49bbee 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala @@ -30,6 +30,7 @@ import org.apache.spark.internal.config import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.util.{ByteArrayWritableChannel, LimitedInputStream} import org.apache.spark.storage.StorageUtils +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils /** @@ -169,24 +170,25 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { } -object ChunkedByteBuffer { +private[spark] object ChunkedByteBuffer { + + // TODO eliminate this method if we switch BlockManager to getting InputStreams - def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): ChunkedByteBuffer = { + def fromManagedBuffer(data: ManagedBuffer): ChunkedByteBuffer = { data match { case f: FileSegmentManagedBuffer => - fromFile(f.getFile, maxChunkSize, f.getOffset, f.getLength) + fromFile(f.getFile, f.getOffset, f.getLength) case other => new ChunkedByteBuffer(other.nioByteBuffer()) } } - def fromFile(file: File, maxChunkSize: Int): ChunkedByteBuffer = { - fromFile(file, maxChunkSize, 0, file.length()) + def fromFile(file: File): ChunkedByteBuffer = { + fromFile(file, 0, file.length()) } private def fromFile( file: File, - maxChunkSize: Int, offset: Long, length: Long): ChunkedByteBuffer = { // We do *not* memory map the file, because we may end up putting this into the memory store, @@ -195,7 +197,7 @@ object ChunkedByteBuffer { val is = new FileInputStream(file) ByteStreams.skipFully(is, offset) val in = new LimitedInputStream(is, length) - val chunkSize = math.min(maxChunkSize, length).toInt + val chunkSize = math.min(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, length).toInt val out = new ChunkedByteBufferOutputStream(chunkSize, ByteBuffer.allocate _) Utils.tryWithSafeFinally { IOUtils.copy(in, out) From 432697c7b58785ca8439717fa748a72224cf0859 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 19 Oct 2018 14:57:52 -0700 Subject: [PATCH 184/879] Revert "[SPARK-25758][ML] Deprecate computeCost on BisectingKMeans" This reverts commit c2962546d9a5900a5628a31b83d2c4b22c3a7936. --- .../org/apache/spark/ml/clustering/BisectingKMeans.scala | 5 ----- python/pyspark/ml/clustering.py | 6 ------ 2 files changed, 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 2243d99364af..5cb16cc76588 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -125,13 +125,8 @@ class BisectingKMeansModel private[ml] ( /** * Computes the sum of squared distances between the input points and their corresponding cluster * centers. - * - * @deprecated This method is deprecated and will be removed in 3.0.0. Use ClusteringEvaluator - * instead. You can also get the cost on the training dataset in the summary. */ @Since("2.0.0") - @deprecated("This method is deprecated and will be removed in 3.0.0. Use ClusteringEvaluator " + - "instead. You can also get the cost on the training dataset in the summary.", "2.4.0") def computeCost(dataset: Dataset[_]): Double = { SchemaUtils.validateVectorCompatibleColumn(dataset.schema, getFeaturesCol) val data = DatasetUtils.columnToOldVector(dataset, getFeaturesCol) diff --git a/python/pyspark/ml/clustering.py b/python/pyspark/ml/clustering.py index 11eb12405d1b..5ef4e765ea4e 100644 --- a/python/pyspark/ml/clustering.py +++ b/python/pyspark/ml/clustering.py @@ -540,13 +540,7 @@ def computeCost(self, dataset): """ Computes the sum of squared distances between the input points and their corresponding cluster centers. - - ..note:: Deprecated in 2.4.0. It will be removed in 3.0.0. Use ClusteringEvaluator instead. - You can also get the cost on the training dataset in the summary. """ - warnings.warn("Deprecated in 2.4.0. It will be removed in 3.0.0. Use ClusteringEvaluator " - "instead. You can also get the cost on the training dataset in the summary.", - DeprecationWarning) return self._call_java("computeCost", dataset) @property From e3a60b0d63eec58adbb1aabb9640b049e40be3bf Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 20 Oct 2018 09:30:12 +0800 Subject: [PATCH 185/879] Revert "[SPARK-25764][ML][EXAMPLES] Update BisectingKMeans example to use ClusteringEvaluator" This reverts commit 36307b1e4b42ce22b07e7a3fc2679c4b5e7c34c8. --- .../examples/ml/JavaBisectingKMeansExample.java | 12 +++--------- .../src/main/python/ml/bisecting_k_means_example.py | 12 +++--------- .../spark/examples/ml/BisectingKMeansExample.scala | 12 +++--------- 3 files changed, 9 insertions(+), 27 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java index f517dc314b2b..8c82aaaacca3 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -20,7 +20,6 @@ // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans; import org.apache.spark.ml.clustering.BisectingKMeansModel; -import org.apache.spark.ml.evaluation.ClusteringEvaluator; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -51,14 +50,9 @@ public static void main(String[] args) { BisectingKMeans bkm = new BisectingKMeans().setK(2).setSeed(1); BisectingKMeansModel model = bkm.fit(dataset); - // Make predictions - Dataset predictions = model.transform(dataset); - - // Evaluate clustering by computing Silhouette score - ClusteringEvaluator evaluator = new ClusteringEvaluator(); - - double silhouette = evaluator.evaluate(predictions); - System.out.println("Silhouette with squared euclidean distance = " + silhouette); + // Evaluate clustering. + double cost = model.computeCost(dataset); + System.out.println("Within Set Sum of Squared Errors = " + cost); // Shows the result. System.out.println("Cluster Centers: "); diff --git a/examples/src/main/python/ml/bisecting_k_means_example.py b/examples/src/main/python/ml/bisecting_k_means_example.py index 82adb338b5d9..7842d2009e23 100644 --- a/examples/src/main/python/ml/bisecting_k_means_example.py +++ b/examples/src/main/python/ml/bisecting_k_means_example.py @@ -24,7 +24,6 @@ # $example on$ from pyspark.ml.clustering import BisectingKMeans -from pyspark.ml.evaluation import ClusteringEvaluator # $example off$ from pyspark.sql import SparkSession @@ -42,14 +41,9 @@ bkm = BisectingKMeans().setK(2).setSeed(1) model = bkm.fit(dataset) - # Make predictions - predictions = model.transform(dataset) - - # Evaluate clustering by computing Silhouette score - evaluator = ClusteringEvaluator() - - silhouette = evaluator.evaluate(predictions) - print("Silhouette with squared euclidean distance = " + str(silhouette)) + # Evaluate clustering. + cost = model.computeCost(dataset) + print("Within Set Sum of Squared Errors = " + str(cost)) # Shows the result. print("Cluster Centers: ") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala index 14e13df02733..5f8f2c99cbaf 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala @@ -21,7 +21,6 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans -import org.apache.spark.ml.evaluation.ClusteringEvaluator // $example off$ import org.apache.spark.sql.SparkSession @@ -49,14 +48,9 @@ object BisectingKMeansExample { val bkm = new BisectingKMeans().setK(2).setSeed(1) val model = bkm.fit(dataset) - // Make predictions - val predictions = model.transform(dataset) - - // Evaluate clustering by computing Silhouette score - val evaluator = new ClusteringEvaluator() - - val silhouette = evaluator.evaluate(predictions) - println(s"Silhouette with squared euclidean distance = $silhouette") + // Evaluate clustering. + val cost = model.computeCost(dataset) + println(s"Within Set Sum of Squared Errors = $cost") // Shows the result. println("Cluster Centers: ") From d6a02c5687b7fd566be493010c7efa246dbe410b Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Fri, 19 Oct 2018 23:55:19 -0700 Subject: [PATCH 186/879] [SPARK-24499][SQL][DOC][FOLLOWUP] Fix some broken links ## What changes were proposed in this pull request? Fix some broken links in the new document. I have clicked through all the links. Hopefully i haven't missed any :-) ## How was this patch tested? Built using jekyll and verified the links. Closes #22772 from dilipbiswal/doc_check. Authored-by: Dilip Biswal Signed-off-by: gatorsmile (cherry picked from commit ed9d0aac905136375444c1e00a2a9a0822b264aa) Signed-off-by: gatorsmile --- docs/sql-data-sources.md | 4 ++-- docs/sql-programming-guide.md | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/sql-data-sources.md b/docs/sql-data-sources.md index aa607ec23a56..636636af6263 100644 --- a/docs/sql-data-sources.md +++ b/docs/sql-data-sources.md @@ -16,8 +16,8 @@ goes into specific options that are available for the built-in data sources. * [Manually Specifying Options](sql-data-sources-load-save-functions.html#manually-specifying-options) * [Run SQL on files directly](sql-data-sources-load-save-functions.html#run-sql-on-files-directly) * [Save Modes](sql-data-sources-load-save-functions.html#save-modes) - * [Saving to Persistent Tables](sql-data-sources-load-save-functions.html#run-sql-on-files-directly) - * [Bucketing, Sorting and Partitioning](sql-data-sources-load-save-functions.html#run-sql-on-files-directly) + * [Saving to Persistent Tables](sql-data-sources-load-save-functions.html#saving-to-persistent-tables) + * [Bucketing, Sorting and Partitioning](sql-data-sources-load-save-functions.html#bucketing-sorting-and-partitioning) * [Parquet Files](sql-data-sources-parquet.html) * [Loading Data Programmatically](sql-data-sources-parquet.html#loading-data-programmatically) * [Partition Discovery](sql-data-sources-parquet.html#partition-discovery) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 42b00c9c8368..eca8915dfa97 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -22,7 +22,7 @@ Spark SQL can also be used to read data from an existing Hive installation. For configure this feature, please refer to the [Hive Tables](sql-data-sources-hive-tables.html) section. When running SQL from within another programming language the results will be returned as a [Dataset/DataFrame](#datasets-and-dataframes). You can also interact with the SQL interface using the [command-line](sql-distributed-sql-engine.html#running-the-spark-sql-cli) -or over [JDBC/ODBC](#sql-distributed-sql-engine.html#running-the-thrift-jdbcodbc-server). +or over [JDBC/ODBC](sql-distributed-sql-engine.html#running-the-thrift-jdbcodbc-server). ## Datasets and DataFrames From 869242c6b8008c30b7e527760df48d7cb8df4593 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 19 Oct 2018 23:56:40 -0700 Subject: [PATCH 187/879] [MINOR][DOC] Update the building doc to use Maven 3.5.4 and Java 8 only ## What changes were proposed in this pull request? Since we didn't test Java 9 ~ 11 up to now in the community, fix the document to describe Java 8 only. ## How was this patch tested? N/A (This is a document only change.) Closes #22781 from dongjoon-hyun/SPARK-JDK-DOC. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit fc9ba9dcc6ad47fbd05f093b94e7e13580000d5f) Signed-off-by: Dongjoon Hyun --- docs/building-spark.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index 1501f0bb8454..7b9697c5726b 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -12,7 +12,7 @@ redirect_from: "building-with-maven.html" ## Apache Maven The Maven-based build is the build of reference for Apache Spark. -Building Spark using Maven requires Maven 3.3.9 or newer and Java 8+. +Building Spark using Maven requires Maven 3.5.4 and Java 8. Note that support for Java 7 was removed as of Spark 2.2.0. ### Setting up Maven's Memory Usage From 0239277dd3e56e355101381da80d6538a7297db9 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Sat, 20 Oct 2018 10:32:09 -0700 Subject: [PATCH 188/879] [DOC][MINOR] Fix minor error in the code of graphx guide ## What changes were proposed in this pull request? Fix minor error in the code "sketch of pregel implementation" of GraphX guide. This fixed error relates to `[SPARK-12995][GraphX] Remove deprecate APIs from Pregel` ## How was this patch tested? N/A Closes #22780 from WeichenXu123/minor_doc_update1. Authored-by: WeichenXu Signed-off-by: Dongjoon Hyun (cherry picked from commit 3b4f35f568eb3844d2a789c8a409bc705477df6b) Signed-off-by: Dongjoon Hyun --- docs/graphx-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 35293348e3f3..cb96fd773aa5 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -726,7 +726,7 @@ class GraphOps[VD, ED] { var g = mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache() // compute the messages - var messages = g.mapReduceTriplets(sendMsg, mergeMsg) + var messages = GraphXUtils.mapReduceTriplets(g, sendMsg, mergeMsg) var activeMessages = messages.count() // Loop until no messages remain or maxIterations is achieved var i = 0 From c21d7e1bb958a0cfa4cba34a688d594466088c9e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 19 Oct 2018 23:54:15 +0800 Subject: [PATCH 189/879] fix security issue of zinc(simplier version) --- build/mvn | 33 +++++++++-------------------- dev/create-release/release-build.sh | 6 ------ dev/run-tests.py | 10 --------- 3 files changed, 10 insertions(+), 39 deletions(-) diff --git a/build/mvn b/build/mvn index 0289ef3c5dc5..b60ea644b262 100755 --- a/build/mvn +++ b/build/mvn @@ -139,17 +139,8 @@ if [ "$1" == "--force" ]; then shift fi -if [ "$1" == "--zinc" ]; then - echo "Using zinc for incremental compilation. Be sure you are aware of the implications of " - echo "running this server process on your machine" - USE_ZINC=1 - shift -fi - # Install the proper version of Scala, Zinc and Maven for the build -if [ -n "${USE_ZINC}" ]; then - install_zinc -fi +install_zinc install_scala install_mvn @@ -158,15 +149,13 @@ cd "${_CALLING_DIR}" # Now that zinc is ensured to be installed, check its status and, if its # not running or just installed, start it -if [ -n "${USE_ZINC}" ]; then - if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`"${ZINC_BIN}" -status -port ${ZINC_PORT}`" ]; then - export ZINC_OPTS=${ZINC_OPTS:-"$_COMPILE_JVM_OPTS"} - "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} - "${ZINC_BIN}" -start -port ${ZINC_PORT} -server 127.0.0.1 \ - -idle-timeout 30m \ - -scala-compiler "${SCALA_COMPILER}" \ - -scala-library "${SCALA_LIBRARY}" &>/dev/null - fi +if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`"${ZINC_BIN}" -status -port ${ZINC_PORT}`" ]; then + export ZINC_OPTS=${ZINC_OPTS:-"$_COMPILE_JVM_OPTS"} + "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} + "${ZINC_BIN}" -start -port ${ZINC_PORT} \ + -server 127.0.0.1 -idle-timeout 30m \ + -scala-compiler "${SCALA_COMPILER}" \ + -scala-library "${SCALA_LIBRARY}" &>/dev/null fi # Set any `mvn` options if not already present @@ -177,7 +166,5 @@ echo "Using \`mvn\` from path: $MVN_BIN" 1>&2 # Last, call the `mvn` command as usual "${MVN_BIN}" -DzincPort=${ZINC_PORT} "$@" -if [ -n "${USE_ZINC}" ]; then - # Try to shut down zinc explicitly - "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} -fi +# Try to shut down zinc explicitly +"${ZINC_BIN}" -shutdown -port ${ZINC_PORT} diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index cce5f8b6975c..02c4193364cc 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -386,9 +386,6 @@ if [[ "$1" == "publish-snapshot" ]]; then #$MVN -DzincPort=$ZINC_PORT --settings $tmp_settings \ # -DskipTests $SCALA_2_12_PROFILES $PUBLISH_PROFILES clean deploy - # Clean-up Zinc nailgun process - $LSOF -P |grep $ZINC_PORT | grep LISTEN | awk '{ print $2; }' | xargs kill - rm $tmp_settings cd .. exit 0 @@ -433,9 +430,6 @@ if [[ "$1" == "publish-release" ]]; then -DskipTests $PUBLISH_PROFILES $SCALA_2_12_PROFILES clean install fi - # Clean-up Zinc nailgun process - $LSOF -P |grep $ZINC_PORT | grep LISTEN | awk '{ print $2; }' | xargs kill - ./dev/change-scala-version.sh 2.11 pushd $tmp_repo/org/apache/spark diff --git a/dev/run-tests.py b/dev/run-tests.py index f534637b80d6..65e6f29c5db5 100755 --- a/dev/run-tests.py +++ b/dev/run-tests.py @@ -249,15 +249,6 @@ def get_zinc_port(): return random.randrange(3030, 4030) -def kill_zinc_on_port(zinc_port): - """ - Kill the Zinc process running on the given port, if one exists. - """ - cmd = "%s -P |grep %s | grep LISTEN | awk '{ print $2; }' | xargs kill" - lsof_exe = which("lsof") - subprocess.check_call(cmd % (lsof_exe if lsof_exe else "/usr/sbin/lsof", zinc_port), shell=True) - - def exec_maven(mvn_args=()): """Will call Maven in the current directory with the list of mvn_args passed in and returns the subprocess for any further processing""" @@ -267,7 +258,6 @@ def exec_maven(mvn_args=()): zinc_flag = "-DzincPort=%s" % zinc_port flags = [os.path.join(SPARK_HOME, "build", "mvn"), "--force", zinc_flag] run_cmd(flags + mvn_args) - kill_zinc_on_port(zinc_port) def exec_sbt(sbt_args=()): From e69e2bfa486d8d3b9d203b96ca9c0f37c2b6cabe Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 22 Oct 2018 14:50:51 +0000 Subject: [PATCH 190/879] Preparing Spark release v2.4.0-rc4 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 714b6f18b19d..f52d785e05cd 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.0 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index ee0de73f21d5..63ab510eb683 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b89e0fed2a66..b10e11849a74 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 3105a8182afc..74c6d232b36a 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f85248b53080..fbdc979d120d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 3350990c0461..53d1b2b58f1f 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 9a2cb90a2f30..98145481adc7 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 9b927a9ba525..f0c8ebe25e63 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 718875f88e74..c635cd3b9394 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 583a2fff7638..ed3efffa180b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 72473771c2e9..7f9a91e2fba0 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1-SNAPSHOT -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.0 +SPARK_VERSION_SHORT: 2.4.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 737ebfd7900c..c7b8354b0968 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 7c6b78ac2b31..a91c13362caa 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 6343a15c43a8..db239f23e1de 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 2f4886832b75..cf19fed1bef5 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9404e2a52d90..da5dba82f3cf 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 73dbc8d99449..e591ce802256 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 912d82b764e5..e8b13cdc2bef 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 8454f5c108dc..c3d3b888656b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 05f7cf870af8..468ace0ff8d7 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 0e935ac112af..db8fda66c3cd 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 7977389718cb..aa1e1267f57b 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7193df69900d..7d07e18f04a2 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 4b318b00ff5b..f545f212b2f0 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6fbb1b7db6aa..6b388ffcf18b 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 9a42d9a69967..fc369c73edc0 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 702f60ab9f00..f4cb94147d9e 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 62322da714bd..22f60efae9af 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index c8e45b27c78f..b17e67d78b98 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 6201e0dd3c9a..bf0d406da9ed 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 28804be0301c..4cc619e697e4 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 87282141fa64..53e3424394ba 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1.dev0" +__version__ = "2.4.0" diff --git a/repl/pom.xml b/repl/pom.xml index d09548348eb5..4532f0b8686e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index a9d9c8de9959..47fe3f2765b2 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 793167103179..f59895e9a1bd 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index e6457289b859..0eacbbfac2b9 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index e70711ce7257..2afdc15be446 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 65cf33f32972..ef1d012d2f80 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c327b85ee866..f4be074642ae 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d9d742444e7e..e5c1065451f7 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c428be67aa3d..b522bfeac130 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 37e96b807653..8e71adabb3ec 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index dd00c2f6cd2c..c00e1b851ba0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml From f33d888a2600589534383b39507ffb47404700b1 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 22 Oct 2018 14:50:55 +0000 Subject: [PATCH 191/879] Preparing development version 2.4.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index f52d785e05cd..714b6f18b19d 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.0 +Version: 2.4.1 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 63ab510eb683..ee0de73f21d5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b10e11849a74..b89e0fed2a66 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 74c6d232b36a..3105a8182afc 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index fbdc979d120d..f85248b53080 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 53d1b2b58f1f..3350990c0461 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 98145481adc7..9a2cb90a2f30 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index f0c8ebe25e63..9b927a9ba525 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index c635cd3b9394..718875f88e74 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ed3efffa180b..583a2fff7638 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7f9a91e2fba0..72473771c2e9 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.0 -SPARK_VERSION_SHORT: 2.4.0 +SPARK_VERSION: 2.4.1-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index c7b8354b0968..737ebfd7900c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index a91c13362caa..7c6b78ac2b31 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index db239f23e1de..6343a15c43a8 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index cf19fed1bef5..2f4886832b75 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index da5dba82f3cf..9404e2a52d90 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index e591ce802256..73dbc8d99449 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index e8b13cdc2bef..912d82b764e5 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index c3d3b888656b..8454f5c108dc 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 468ace0ff8d7..05f7cf870af8 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index db8fda66c3cd..0e935ac112af 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index aa1e1267f57b..7977389718cb 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7d07e18f04a2..7193df69900d 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f545f212b2f0..4b318b00ff5b 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6b388ffcf18b..6fbb1b7db6aa 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index fc369c73edc0..9a42d9a69967 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index f4cb94147d9e..702f60ab9f00 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 22f60efae9af..62322da714bd 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index b17e67d78b98..c8e45b27c78f 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bf0d406da9ed..6201e0dd3c9a 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index 4cc619e697e4..28804be0301c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 53e3424394ba..87282141fa64 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.0" +__version__ = "2.4.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 4532f0b8686e..d09548348eb5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 47fe3f2765b2..a9d9c8de9959 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index f59895e9a1bd..793167103179 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 0eacbbfac2b9..e6457289b859 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 2afdc15be446..e70711ce7257 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ef1d012d2f80..65cf33f32972 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f4be074642ae..c327b85ee866 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index e5c1065451f7..d9d742444e7e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b522bfeac130..c428be67aa3d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 8e71adabb3ec..37e96b807653 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c00e1b851ba0..dd00c2f6cd2c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml From b9b594ade9106ad96adb413c7a27ec7b4f8a849a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 22 Oct 2018 16:34:33 -0700 Subject: [PATCH 192/879] [SPARK-25795][R][EXAMPLE] Fix CSV SparkR SQL Example ## What changes were proposed in this pull request? This PR aims to fix the following SparkR example in Spark 2.3.0 ~ 2.4.0. ```r > df <- read.df("examples/src/main/resources/people.csv", "csv") > namesAndAges <- select(df, "name", "age") ... Caused by: org.apache.spark.sql.AnalysisException: cannot resolve '`name`' given input columns: [_c0];; 'Project ['name, 'age] +- AnalysisBarrier +- Relation[_c0#97] csv ``` - https://dist.apache.org/repos/dist/dev/spark/v2.4.0-rc3-docs/_site/sql-programming-guide.html#manually-specifying-options - http://spark.apache.org/docs/2.3.2/sql-programming-guide.html#manually-specifying-options - http://spark.apache.org/docs/2.3.1/sql-programming-guide.html#manually-specifying-options - http://spark.apache.org/docs/2.3.0/sql-programming-guide.html#manually-specifying-options ## How was this patch tested? Manual test in SparkR. (Please note that `RSparkSQLExample.R` fails at the last JDBC example) ```r > df <- read.df("examples/src/main/resources/people.csv", "csv", sep=";", inferSchema=T, header=T) > namesAndAges <- select(df, "name", "age") ``` Closes #22791 from dongjoon-hyun/SPARK-25795. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 3b4556745e90a13f4ae7ebae4ab682617de25c38) Signed-off-by: Dongjoon Hyun --- examples/src/main/r/RSparkSQLExample.R | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/r/RSparkSQLExample.R b/examples/src/main/r/RSparkSQLExample.R index a5ed723da47c..effba948e531 100644 --- a/examples/src/main/r/RSparkSQLExample.R +++ b/examples/src/main/r/RSparkSQLExample.R @@ -114,7 +114,7 @@ write.df(namesAndAges, "namesAndAges.parquet", "parquet") # $example on:manual_load_options_csv$ -df <- read.df("examples/src/main/resources/people.csv", "csv") +df <- read.df("examples/src/main/resources/people.csv", "csv", sep=";", inferSchema=T, header=T) namesAndAges <- select(df, "name", "age") # $example off:manual_load_options_csv$ From 4099565cdddd887640b60e9c57d9dc7989e0c3ed Mon Sep 17 00:00:00 2001 From: Kazuaki Ishizaki Date: Tue, 23 Oct 2018 12:19:31 +0800 Subject: [PATCH 193/879] [SPARK-24499][SQL][DOC][FOLLOW-UP] Fix spelling in doc ## What changes were proposed in this pull request? This PR replaces `turing` with `tuning` in files and a file name. Currently, in the left side menu, `Turing` is shown. [This page](https://dist.apache.org/repos/dist/dev/spark/v2.4.0-rc4-docs/_site/sql-performance-turing.html) is one of examples. ![image](https://user-images.githubusercontent.com/1315079/47332714-20a96180-d6bb-11e8-9a5a-0a8dad292626.png) ## How was this patch tested? `grep -rin turing docs` && `find docs -name "*turing*"` Closes #22800 from kiszk/SPARK-24499-follow. Authored-by: Kazuaki Ishizaki Signed-off-by: Wenchen Fan (cherry picked from commit c391dc65efb21357bdd80b28fba3851773759bc6) Signed-off-by: Wenchen Fan --- docs/_data/menu-sql.yaml | 10 +++++----- docs/sql-migration-guide-upgrade.md | 2 +- ...performance-turing.md => sql-performance-tuning.md} | 0 3 files changed, 6 insertions(+), 6 deletions(-) rename docs/{sql-performance-turing.md => sql-performance-tuning.md} (100%) diff --git a/docs/_data/menu-sql.yaml b/docs/_data/menu-sql.yaml index 671876323269..cd065ea01dda 100644 --- a/docs/_data/menu-sql.yaml +++ b/docs/_data/menu-sql.yaml @@ -36,15 +36,15 @@ url: sql-data-sources-avro.html - text: Troubleshooting url: sql-data-sources-troubleshooting.html -- text: Performance Turing - url: sql-performance-turing.html +- text: Performance Tuning + url: sql-performance-tuning.html subitems: - text: Caching Data In Memory - url: sql-performance-turing.html#caching-data-in-memory + url: sql-performance-tuning.html#caching-data-in-memory - text: Other Configuration Options - url: sql-performance-turing.html#other-configuration-options + url: sql-performance-tuning.html#other-configuration-options - text: Broadcast Hint for SQL Queries - url: sql-performance-turing.html#broadcast-hint-for-sql-queries + url: sql-performance-tuning.html#broadcast-hint-for-sql-queries - text: Distributed SQL Engine url: sql-distributed-sql-engine.html subitems: diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 062e07bd23c5..af561f286c70 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -270,7 +270,7 @@ displayTitle: Spark SQL Upgrading Guide - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. - - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](sql-performance-turing.html#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). + - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](sql-performance-tuning.html#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. diff --git a/docs/sql-performance-turing.md b/docs/sql-performance-tuning.md similarity index 100% rename from docs/sql-performance-turing.md rename to docs/sql-performance-tuning.md From d5e694805033781ffd161d63ddc4e73bf62d389a Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Tue, 23 Oct 2018 14:20:29 +0800 Subject: [PATCH 194/879] [SPARK-25805][SQL][TEST] Fix test for SPARK-25159 The original test would sometimes fail if the listener bus did not keep up, so just wait till the listener bus is empty. Tested by adding a sleep in the listener, which made the test consistently fail without the fix, but pass consistently after the fix. Closes #22799 from squito/SPARK-25805. Authored-by: Imran Rashid Signed-off-by: Wenchen Fan (cherry picked from commit 78c8bd2e68a77ee3c12c233289a8804e339bd71d) Signed-off-by: Wenchen Fan --- .../src/test/scala/org/apache/spark/sql/DataFrameSuite.scala | 1 + 1 file changed, 1 insertion(+) 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 4a7bd2f5b065..e84cd8ccea6c 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 @@ -2549,6 +2549,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val df = spark.read.json(path.getCanonicalPath) assert(df.columns === Array("i", "p")) + spark.sparkContext.listenerBus.waitUntilEmpty(10000) assert(numJobs == 1) } } From a20660b6fcee2d436ef182337255cea5e2eb7216 Mon Sep 17 00:00:00 2001 From: WeichenXu Date: Thu, 25 Oct 2018 23:03:16 +0800 Subject: [PATCH 195/879] [SPARK-25347][ML][DOC] Spark datasource for image/libsvm user guide ## What changes were proposed in this pull request? Spark datasource for image/libsvm user guide ## How was this patch tested? Scala: 1 Java: 2 Python: 3 R: 4 Closes #22675 from WeichenXu123/add_image_source_doc. Authored-by: WeichenXu Signed-off-by: Wenchen Fan (cherry picked from commit 6540c2f8f31bbde4df57e48698f46bb1815740ff) Signed-off-by: Wenchen Fan --- docs/_data/menu-ml.yaml | 2 + docs/ml-datasource.md | 108 ++++++++++++++++++ .../ml/source/image/ImageDataSource.scala | 17 +-- 3 files changed, 120 insertions(+), 7 deletions(-) create mode 100644 docs/ml-datasource.md diff --git a/docs/_data/menu-ml.yaml b/docs/_data/menu-ml.yaml index b5a6641e2e7e..8e366f7f029a 100644 --- a/docs/_data/menu-ml.yaml +++ b/docs/_data/menu-ml.yaml @@ -1,5 +1,7 @@ - text: Basic statistics url: ml-statistics.html +- text: Data sources + url: ml-datasource - text: Pipelines url: ml-pipeline.html - text: Extracting, transforming and selecting features diff --git a/docs/ml-datasource.md b/docs/ml-datasource.md new file mode 100644 index 000000000000..15083326240a --- /dev/null +++ b/docs/ml-datasource.md @@ -0,0 +1,108 @@ +--- +layout: global +title: Data sources +displayTitle: Data sources +--- + +In this section, we introduce how to use data source in ML to load data. +Beside some general data sources such as Parquet, CSV, JSON and JDBC, we also provide some specific data sources for ML. + +**Table of Contents** + +* This will become a table of contents (this text will be scraped). +{:toc} + +## Image data source + +This image data source is used to load image files from a directory, it can load compressed image (jpeg, png, etc.) into raw image representation via `ImageIO` in Java library. +The loaded DataFrame has one `StructType` column: "image", containing image data stored as image schema. +The schema of the `image` column is: + - origin: `StringType` (represents the file path of the image) + - height: `IntegerType` (height of the image) + - width: `IntegerType` (width of the image) + - nChannels: `IntegerType` (number of image channels) + - mode: `IntegerType` (OpenCV-compatible type) + - data: `BinaryType` (Image bytes in OpenCV-compatible order: row-wise BGR in most cases) + + +
    +
    +[`ImageDataSource`](api/scala/index.html#org.apache.spark.ml.source.image.ImageDataSource) +implements a Spark SQL data source API for loading image data as a DataFrame. + +{% highlight scala %} +scala> val df = spark.read.format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens") +df: org.apache.spark.sql.DataFrame = [image: struct] + +scala> df.select("image.origin", "image.width", "image.height").show(truncate=false) ++-----------------------------------------------------------------------+-----+------+ +|origin |width|height| ++-----------------------------------------------------------------------+-----+------+ +|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 | +|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 | +|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 | +|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 | ++-----------------------------------------------------------------------+-----+------+ +{% endhighlight %} +
    + +
    +[`ImageDataSource`](api/java/org/apache/spark/ml/source/image/ImageDataSource.html) +implements Spark SQL data source API for loading image data as DataFrame. + +{% highlight java %} +Dataset imagesDF = spark.read().format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens"); +imageDF.select("image.origin", "image.width", "image.height").show(false); +/* +Will output: ++-----------------------------------------------------------------------+-----+------+ +|origin |width|height| ++-----------------------------------------------------------------------+-----+------+ +|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 | +|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 | +|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 | +|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 | ++-----------------------------------------------------------------------+-----+------+ +*/ +{% endhighlight %} +
    + +
    +In PySpark we provide Spark SQL data source API for loading image data as DataFrame. + +{% highlight python %} +>>> df = spark.read.format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens") +>>> df.select("image.origin", "image.width", "image.height").show(truncate=False) ++-----------------------------------------------------------------------+-----+------+ +|origin |width|height| ++-----------------------------------------------------------------------+-----+------+ +|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 | +|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 | +|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 | +|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 | ++-----------------------------------------------------------------------+-----+------+ +{% endhighlight %} +
    + +
    +In SparkR we provide Spark SQL data source API for loading image data as DataFrame. + +{% highlight r %} +> df = read.df("data/mllib/images/origin/kittens", "image") +> head(select(df, df$image.origin, df$image.width, df$image.height)) + +1 file:///spark/data/mllib/images/origin/kittens/54893.jpg +2 file:///spark/data/mllib/images/origin/kittens/DP802813.jpg +3 file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg +4 file:///spark/data/mllib/images/origin/kittens/DP153539.jpg + width height +1 300 311 +2 199 313 +3 300 200 +4 300 296 + +{% endhighlight %} +
    + + +
    diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageDataSource.scala b/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageDataSource.scala index a111c95248cf..d4d74082dc8c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageDataSource.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageDataSource.scala @@ -19,14 +19,17 @@ package org.apache.spark.ml.source.image /** * `image` package implements Spark SQL data source API for loading image data as `DataFrame`. - * The loaded `DataFrame` has one `StructType` column: `image`. + * It can load compressed image (jpeg, png, etc.) into raw image representation via `ImageIO` + * in Java library. + * The loaded `DataFrame` has one `StructType` column: `image`, containing image data stored + * as image schema. * The schema of the `image` column is: - * - origin: String (represents the file path of the image) - * - height: Int (height of the image) - * - width: Int (width of the image) - * - nChannels: Int (number of the image channels) - * - mode: Int (OpenCV-compatible type) - * - data: BinaryType (Image bytes in OpenCV-compatible order: row-wise BGR in most cases) + * - origin: `StringType` (represents the file path of the image) + * - height: `IntegerType` (height of the image) + * - width: `IntegerType` (width of the image) + * - nChannels: `IntegerType` (number of image channels) + * - mode: `IntegerType` (OpenCV-compatible type) + * - data: `BinaryType` (Image bytes in OpenCV-compatible order: row-wise BGR in most cases) * * To use image data source, you need to set "image" as the format in `DataFrameReader` and * optionally specify the data source options, for example: From 45ed76d6f66d58c99c20d9e757aa2177d9707968 Mon Sep 17 00:00:00 2001 From: Steve Date: Thu, 25 Oct 2018 13:00:59 -0700 Subject: [PATCH 196/879] [SPARK-25803][K8S] Fix docker-image-tool.sh -n option ## What changes were proposed in this pull request? docker-image-tool.sh uses getopts in which a colon signifies that an option takes an argument. Since -n does not take an argument it should not have a colon. ## How was this patch tested? Following the reproduction in [JIRA](https://issues.apache.org/jira/browse/SPARK-25803):- 0. Created a custom Dockerfile to use for the spark-r container image. In each of the steps below the path to this Dockerfile is passed with the '-R' option. (spark-r is used here simply as an example, the bug applies to all options) 1. Built container images without '-n'. The [result](https://gist.github.com/sel/59f0911bb1a6a485c2487cf7ca770f9d) is that the '-R' option is honoured and the hello-world image is built for spark-r, as expected. 2. Built container images with '-n' to reproduce the issue The [result](https://gist.github.com/sel/e5cabb9f3bdad5d087349e7fbed75141) is that the '-R' option is ignored and the default container image for spark-r is built 3. Applied the patch and re-built container images with '-n' and did not reproduce the issue The [result](https://gist.github.com/sel/6af14b95012ba8ff267a4fce6e3bd3bf) is that the '-R' option is honoured and the hello-world image is built for spark-r, as expected. Closes #22798 from sel/fix-docker-image-tool-nocache. Authored-by: Steve Signed-off-by: Marcelo Vanzin (cherry picked from commit 9b98d9166ee2c130ba38a09e8c0aa12e29676b76) Signed-off-by: Marcelo Vanzin --- bin/docker-image-tool.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index 228494de6d5a..5e8eafff50f2 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -145,7 +145,7 @@ PYDOCKERFILE= RDOCKERFILE= NOCACHEARG= BUILD_PARAMS= -while getopts f:p:R:mr:t:n:b: option +while getopts f:p:R:mr:t:nb: option do case "${option}" in From 1b075f26f2aa3db93e168c8b8bb5d67e96ffb490 Mon Sep 17 00:00:00 2001 From: Devaraj K Date: Thu, 25 Oct 2018 13:16:08 -0700 Subject: [PATCH 197/879] [SPARK-24787][CORE] Revert hsync in EventLoggingListener and make FsHistoryProvider to read lastBlockBeingWritten data for logs ## What changes were proposed in this pull request? `hsync` has been added as part of SPARK-19531 to get the latest data in the history sever ui, but that is causing the performance overhead and also leading to drop many history log events. `hsync` uses the force `FileChannel.force` to sync the data to the disk and happens for the data pipeline, it is costly operation and making the application to face overhead and drop the events. I think getting the latest data in history server can be done in different way (no impact to application while writing events), there is an api `DFSInputStream.getFileLength()` which gives the file length including the `lastBlockBeingWrittenLength`(different from `FileStatus.getLen()`), this api can be used when the file status length and previously cached length are equal to verify whether any new data has been written or not, if there is any update in data length then the history server can update the in progress history log. And also I made this change as configurable with the default value false, and can be enabled for history server if users want to see the updated data in ui. ## How was this patch tested? Added new test and verified manually, with the added conf `spark.history.fs.inProgressAbsoluteLengthCheck.enabled=true`, history server is reading the logs including the last block data which is being written and updating the Web UI with the latest data. Closes #22752 from devaraj-kavali/SPARK-24787. Authored-by: Devaraj K Signed-off-by: Marcelo Vanzin (cherry picked from commit 46d2d2c74d9aaf30e158aeda58a189f6c8e48b9c) Signed-off-by: Marcelo Vanzin --- .../deploy/history/FsHistoryProvider.scala | 22 ++++++++++- .../scheduler/EventLoggingListener.scala | 8 +--- .../history/FsHistoryProviderSuite.scala | 37 ++++++++++++++++++- 3 files changed, 56 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index c23a659e76df..c4517d3dfd93 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -34,7 +34,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore import com.google.common.io.ByteStreams import com.google.common.util.concurrent.MoreExecutors import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} -import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem} import org.apache.hadoop.hdfs.protocol.HdfsConstants import org.apache.hadoop.security.AccessControlException import org.fusesource.leveldbjni.internal.NativeDB @@ -449,7 +449,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) listing.write(info.copy(lastProcessed = newLastScanTime, fileSize = entry.getLen())) } - if (info.fileSize < entry.getLen()) { + if (shouldReloadLog(info, entry)) { if (info.appId.isDefined && fastInProgressParsing) { // When fast in-progress parsing is on, we don't need to re-parse when the // size changes, but we do need to invalidate any existing UIs. @@ -541,6 +541,24 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + private[history] def shouldReloadLog(info: LogInfo, entry: FileStatus): Boolean = { + var result = info.fileSize < entry.getLen + if (!result && info.logPath.endsWith(EventLoggingListener.IN_PROGRESS)) { + try { + result = Utils.tryWithResource(fs.open(entry.getPath)) { in => + in.getWrappedStream match { + case dfsIn: DFSInputStream => info.fileSize < dfsIn.getFileLength + case _ => false + } + } + } catch { + case e: Exception => + logDebug(s"Failed to check the length for the file : ${info.logPath}", e) + } + } + result + } + private def cleanAppData(appId: String, attemptId: Option[String], logPath: String): Unit = { try { val app = load(appId) diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 69bc51c1ecf9..1311cf8607d1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -20,7 +20,6 @@ package org.apache.spark.scheduler import java.io._ import java.net.URI import java.nio.charset.StandardCharsets -import java.util.EnumSet import java.util.Locale import scala.collection.mutable @@ -29,8 +28,6 @@ import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.hadoop.fs.permission.FsPermission -import org.apache.hadoop.hdfs.DFSOutputStream -import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ @@ -144,10 +141,7 @@ private[spark] class EventLoggingListener( // scalastyle:on println if (flushLogger) { writer.foreach(_.flush()) - hadoopDataStream.foreach(ds => ds.getWrappedStream match { - case wrapped: DFSOutputStream => wrapped.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)) - case _ => ds.hflush() - }) + hadoopDataStream.foreach(_.hflush()) } if (testing) { loggedEvents += eventJson diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index b4eba755eccb..f06b9935ad9e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -27,8 +27,8 @@ import scala.concurrent.duration._ import scala.language.postfixOps import com.google.common.io.{ByteStreams, Files} -import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.hadoop.fs.{FileStatus, FileSystem, FSDataInputStream, Path} +import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem} import org.apache.hadoop.security.AccessControlException import org.json4s.jackson.JsonMethods._ import org.mockito.ArgumentMatcher @@ -856,6 +856,39 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc assert(!mockedProvider.isBlacklisted(accessDeniedPath)) } + test("check in-progress event logs absolute length") { + val path = new Path("testapp.inprogress") + val provider = new FsHistoryProvider(createTestConf()) + val mockedProvider = spy(provider) + val mockedFs = mock(classOf[FileSystem]) + val in = mock(classOf[FSDataInputStream]) + val dfsIn = mock(classOf[DFSInputStream]) + when(mockedProvider.fs).thenReturn(mockedFs) + when(mockedFs.open(path)).thenReturn(in) + when(in.getWrappedStream).thenReturn(dfsIn) + when(dfsIn.getFileLength).thenReturn(200) + // FileStatus.getLen is more than logInfo fileSize + var fileStatus = new FileStatus(200, false, 0, 0, 0, path) + var logInfo = new LogInfo(path.toString, 0, Some("appId"), Some("attemptId"), 100) + assert(mockedProvider.shouldReloadLog(logInfo, fileStatus)) + + fileStatus = new FileStatus() + fileStatus.setPath(path) + // DFSInputStream.getFileLength is more than logInfo fileSize + logInfo = new LogInfo(path.toString, 0, Some("appId"), Some("attemptId"), 100) + assert(mockedProvider.shouldReloadLog(logInfo, fileStatus)) + // DFSInputStream.getFileLength is equal to logInfo fileSize + logInfo = new LogInfo(path.toString, 0, Some("appId"), Some("attemptId"), 200) + assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + // in.getWrappedStream returns other than DFSInputStream + val bin = mock(classOf[BufferedInputStream]) + when(in.getWrappedStream).thenReturn(bin) + assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + // fs.open throws exception + when(mockedFs.open(path)).thenThrow(new IOException("Throwing intentionally")) + assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + } + /** * Asks the provider to check for logs and calls a function to perform checks on the updated * app list. Example: From db121a2a1fde96fe77eedff18706df5c8e2e731d Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 25 Oct 2018 14:15:03 -0700 Subject: [PATCH 198/879] [SPARK-25656][SQL][DOC][EXAMPLE][BRANCH-2.4] Add a doc and examples about extra data source options ## What changes were proposed in this pull request? Our current doc does not explain how we are passing the data source specific options to the underlying data source. According to [the review comment](https://github.com/apache/spark/pull/22622#discussion_r222911529), this PR aims to add more detailed information and examples. This is a backport of #22801. `orc.column.encoding.direct` is removed since it's not supported in ORC 1.5.2. ## How was this patch tested? Manual. Closes #22839 from dongjoon-hyun/SPARK-25656-2.4. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- docs/sql-data-sources-load-save-functions.md | 43 ++++++++++++++++++ .../sql/JavaSQLDataSourceExample.java | 6 +++ examples/src/main/python/sql/datasource.py | 8 ++++ examples/src/main/r/RSparkSQLExample.R | 6 ++- examples/src/main/resources/users.orc | Bin 0 -> 547 bytes .../examples/sql/SQLDataSourceExample.scala | 6 +++ 6 files changed, 68 insertions(+), 1 deletion(-) create mode 100644 examples/src/main/resources/users.orc diff --git a/docs/sql-data-sources-load-save-functions.md b/docs/sql-data-sources-load-save-functions.md index e1dd0a3f54e4..a3191b229037 100644 --- a/docs/sql-data-sources-load-save-functions.md +++ b/docs/sql-data-sources-load-save-functions.md @@ -82,6 +82,49 @@ To load a CSV file you can use:
    +The extra options are also used during write operation. +For example, you can control bloom filters and dictionary encodings for ORC data sources. +The following ORC example will create bloom filter on `favorite_color` and use dictionary encoding for `name` and `favorite_color`. +For Parquet, there exists `parquet.enable.dictionary`, too. +To find more detailed information about the extra ORC/Parquet options, +visit the official Apache ORC/Parquet websites. + +
    + +
    +{% include_example manual_save_options_orc scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example manual_save_options_orc java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example manual_save_options_orc python/sql/datasource.py %} +
    + +
    +{% include_example manual_save_options_orc r/RSparkSQLExample.R %} +
    + +
    + +{% highlight sql %} +CREATE TABLE users_with_options ( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING ORC +OPTIONS ( + orc.bloom.filter.columns 'favorite_color', + orc.dictionary.key.threshold '1.0' +) +{% endhighlight %} + +
    + +
    + ### Run SQL on files directly Instead of using read API to load a file into DataFrame and query it, you can also query that diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java index ef3c90477569..97e9ca311267 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java @@ -123,6 +123,12 @@ private static void runBasicDataSourceExample(SparkSession spark) { .option("header", "true") .load("examples/src/main/resources/people.csv"); // $example off:manual_load_options_csv$ + // $example on:manual_save_options_orc$ + usersDF.write().format("orc") + .option("orc.bloom.filter.columns", "favorite_color") + .option("orc.dictionary.key.threshold", "1.0") + .save("users_with_options.orc"); + // $example off:manual_save_options_orc$ // $example on:direct_sql$ Dataset sqlDF = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"); diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index d8c879dfe02e..888e92a62007 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -57,6 +57,14 @@ def basic_datasource_example(spark): format="csv", sep=":", inferSchema="true", header="true") # $example off:manual_load_options_csv$ + # $example on:manual_save_options_orc$ + df = spark.read.orc("examples/src/main/resources/users.orc") + (df.write.format("orc") + .option("orc.bloom.filter.columns", "favorite_color") + .option("orc.dictionary.key.threshold", "1.0") + .save("users_with_options.orc")) + # $example off:manual_save_options_orc$ + # $example on:write_sorting_and_bucketing$ df.write.bucketBy(42, "name").sortBy("age").saveAsTable("people_bucketed") # $example off:write_sorting_and_bucketing$ diff --git a/examples/src/main/r/RSparkSQLExample.R b/examples/src/main/r/RSparkSQLExample.R index effba948e531..f3d2ab8cbc84 100644 --- a/examples/src/main/r/RSparkSQLExample.R +++ b/examples/src/main/r/RSparkSQLExample.R @@ -114,10 +114,14 @@ write.df(namesAndAges, "namesAndAges.parquet", "parquet") # $example on:manual_load_options_csv$ -df <- read.df("examples/src/main/resources/people.csv", "csv", sep=";", inferSchema=T, header=T) +df <- read.df("examples/src/main/resources/people.csv", "csv", sep = ";", inferSchema = TRUE, header = TRUE) namesAndAges <- select(df, "name", "age") # $example off:manual_load_options_csv$ +# $example on:manual_save_options_orc$ +df <- read.df("examples/src/main/resources/users.orc", "orc") +write.orc(df, "users_with_options.orc", orc.bloom.filter.columns = "favorite_color", orc.dictionary.key.threshold = 1.0) +# $example off:manual_save_options_orc$ # $example on:direct_sql$ df <- sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") diff --git a/examples/src/main/resources/users.orc b/examples/src/main/resources/users.orc new file mode 100644 index 0000000000000000000000000000000000000000..12478a5d03c26cb30b35af232a5764e076eaab1f GIT binary patch literal 547 zcmZ`#Jxc>Y5S`t<-He+fZZ-y&D1Mwxz(Or-4vQp$h^RSIU8P1nQP2b~QLz($LH>cQ z{tF8ce~yK{&c!BZT$uM}cG)*?rrFvo0%&DDwa5Zri!?d488{WO8PdpWktqx{m#HrO)INGvp)yr>5J3sxN1B9l z09)*Y`hL|YB_YBFybP~vd4M;e{Nxp&KdZC<;PMiYxg|pG772wj63;#7=$#qnd}2(&;MDi2oBw}Np|@jC6Rq*6F*-*nT9esXxyz3iqHb5=Cf&h^!ClJ)|Q zIq7{gBx=h%s>CV}haSWKJceUEhKjAnu?l}#tPS?J0iPHx>i*sY9Qt^a{vGU literal 0 HcmV?d00001 diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala index 7d83aacb1154..49177334ebae 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala @@ -56,6 +56,12 @@ object SQLDataSourceExample { .option("header", "true") .load("examples/src/main/resources/people.csv") // $example off:manual_load_options_csv$ + // $example on:manual_save_options_orc$ + usersDF.write.format("orc") + .option("orc.bloom.filter.columns", "favorite_color") + .option("orc.dictionary.key.threshold", "1.0") + .save("users_with_options.orc") + // $example off:manual_save_options_orc$ // $example on:direct_sql$ val sqlDF = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") From a9f200e11da3d26158f9f75e48756d47d61bfacb Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 26 Oct 2018 07:38:55 +0800 Subject: [PATCH 199/879] [SPARK-25832][SQL][BRANCH-2.4] Revert newly added map related functions ## What changes were proposed in this pull request? - Revert [SPARK-23935][SQL] Adding map_entries function: https://github.com/apache/spark/pull/21236 - Revert [SPARK-23937][SQL] Add map_filter SQL function: https://github.com/apache/spark/pull/21986 - Revert [SPARK-23940][SQL] Add transform_values SQL function: https://github.com/apache/spark/pull/22045 - Revert [SPARK-23939][SQL] Add transform_keys function: https://github.com/apache/spark/pull/22013 - Revert [SPARK-23938][SQL] Add map_zip_with function: https://github.com/apache/spark/pull/22017 - Revert the changes of map_entries in [SPARK-24331][SPARKR][SQL] Adding arrays_overlap, array_repeat, map_entries to SparkR: https://github.com/apache/spark/pull/21434/ ## How was this patch tested? The existing tests. Closes #22827 from gatorsmile/revertMap2.4. Authored-by: gatorsmile Signed-off-by: Wenchen Fan --- R/pkg/NAMESPACE | 1 - R/pkg/R/functions.R | 15 +- R/pkg/R/generics.R | 4 - R/pkg/tests/fulltests/test_sparkSQL.R | 7 +- python/pyspark/sql/functions.py | 20 - .../catalyst/analysis/FunctionRegistry.scala | 5 - .../sql/catalyst/analysis/TypeCoercion.scala | 25 -- .../expressions/collectionOperations.scala | 168 ------- .../expressions/higherOrderFunctions.scala | 330 -------------- .../CollectionExpressionsSuite.scala | 24 - .../HigherOrderFunctionsSuite.scala | 315 ------------- .../org/apache/spark/sql/functions.scala | 7 - .../inputs/higher-order-functions.sql | 23 - .../inputs/typeCoercion/native/mapZipWith.sql | 78 ---- .../results/higher-order-functions.sql.out | 66 +-- .../typeCoercion/native/mapZipWith.sql.out | 179 -------- .../spark/sql/DataFrameFunctionsSuite.scala | 425 ------------------ 17 files changed, 3 insertions(+), 1689 deletions(-) delete mode 100644 sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapZipWith.sql delete mode 100644 sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 96ff389faf4a..d77c62a0ae5d 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -313,7 +313,6 @@ exportMethods("%<=>%", "lower", "lpad", "ltrim", - "map_entries", "map_from_arrays", "map_keys", "map_values", diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 63bd427acb1a..1e702446539e 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -219,7 +219,7 @@ NULL #' head(select(tmp, sort_array(tmp$v1))) #' head(select(tmp, sort_array(tmp$v1, asc = FALSE))) #' tmp3 <- mutate(df, v3 = create_map(df$model, df$cyl)) -#' head(select(tmp3, map_entries(tmp3$v3), map_keys(tmp3$v3), map_values(tmp3$v3))) +#' head(select(tmp3, map_keys(tmp3$v3), map_values(tmp3$v3))) #' head(select(tmp3, element_at(tmp3$v3, "Valiant"))) #' tmp4 <- mutate(df, v4 = create_array(df$mpg, df$cyl), v5 = create_array(df$cyl, df$hp)) #' head(select(tmp4, concat(tmp4$v4, tmp4$v5), arrays_overlap(tmp4$v4, tmp4$v5))) @@ -3252,19 +3252,6 @@ setMethod("flatten", column(jc) }) -#' @details -#' \code{map_entries}: Returns an unordered array of all entries in the given map. -#' -#' @rdname column_collection_functions -#' @aliases map_entries map_entries,Column-method -#' @note map_entries since 2.4.0 -setMethod("map_entries", - signature(x = "Column"), - function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", "map_entries", x@jc) - column(jc) - }) - #' @details #' \code{map_from_arrays}: Creates a new map column. The array in the first column is used for #' keys. The array in the second column is used for values. All elements in the array for key diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 27c1b312d645..e74f1c8e58b1 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -1076,10 +1076,6 @@ setGeneric("lpad", function(x, len, pad) { standardGeneric("lpad") }) #' @name NULL setGeneric("ltrim", function(x, trimString) { standardGeneric("ltrim") }) -#' @rdname column_collection_functions -#' @name NULL -setGeneric("map_entries", function(x) { standardGeneric("map_entries") }) - #' @rdname column_collection_functions #' @name NULL setGeneric("map_from_arrays", function(x, y) { standardGeneric("map_from_arrays") }) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index ce3922fa0350..0101d0cf2e76 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -1570,13 +1570,8 @@ test_that("column functions", { result <- collect(select(df, flatten(df[[1]])))[[1]] expect_equal(result, list(list(1L, 2L, 3L, 4L), list(5L, 6L, 7L, 8L))) - # Test map_entries(), map_keys(), map_values() and element_at() + # Test map_keys(), map_values() and element_at() df <- createDataFrame(list(list(map = as.environment(list(x = 1, y = 2))))) - result <- collect(select(df, map_entries(df$map)))[[1]] - expected_entries <- list(listToStruct(list(key = "x", value = 1)), - listToStruct(list(key = "y", value = 2))) - expect_equal(result, list(expected_entries)) - result <- collect(select(df, map_keys(df$map)))[[1]] expect_equal(result, list(list("x", "y"))) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 8c541798bcc0..785e55e2a1eb 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2540,26 +2540,6 @@ def map_values(col): return Column(sc._jvm.functions.map_values(_to_java_column(col))) -@since(2.4) -def map_entries(col): - """ - Collection function: Returns an unordered array of all entries in the given map. - - :param col: name of column or expression - - >>> from pyspark.sql.functions import map_entries - >>> df = spark.sql("SELECT map(1, 'a', 2, 'b') as data") - >>> df.select(map_entries("data").alias("entries")).show() - +----------------+ - | entries| - +----------------+ - |[[1, a], [2, b]]| - +----------------+ - """ - sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.map_entries(_to_java_column(col))) - - @since(2.4) def map_from_entries(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 695267ac130a..efd760c785b4 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 @@ -414,7 +414,6 @@ object FunctionRegistry { expression[MapFromArrays]("map_from_arrays"), expression[MapKeys]("map_keys"), expression[MapValues]("map_values"), - expression[MapEntries]("map_entries"), expression[MapFromEntries]("map_from_entries"), expression[MapConcat]("map_concat"), expression[Size]("size"), @@ -433,13 +432,9 @@ object FunctionRegistry { expression[ArrayRemove]("array_remove"), expression[ArrayDistinct]("array_distinct"), expression[ArrayTransform]("transform"), - expression[MapFilter]("map_filter"), expression[ArrayFilter]("filter"), expression[ArrayExists]("exists"), expression[ArrayAggregate]("aggregate"), - expression[TransformValues]("transform_values"), - expression[TransformKeys]("transform_keys"), - expression[MapZipWith]("map_zip_with"), expression[ZipWith]("zip_with"), CreateStruct.registryEntry, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala index 72ac80e0a0a1..1659156b8695 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/TypeCoercion.scala @@ -54,7 +54,6 @@ object TypeCoercion { BooleanEquality :: FunctionArgumentConversion :: ConcatCoercion(conf) :: - MapZipWithCoercion :: EltCoercion(conf) :: CaseWhenCoercion :: IfCoercion :: @@ -763,30 +762,6 @@ object TypeCoercion { } } - /** - * Coerces key types of two different [[MapType]] arguments of the [[MapZipWith]] expression - * to a common type. - */ - object MapZipWithCoercion extends TypeCoercionRule { - override protected def coerceTypes(plan: LogicalPlan): LogicalPlan = plan resolveExpressions { - // Lambda function isn't resolved when the rule is executed. - case m @ MapZipWith(left, right, function) if m.arguments.forall(a => a.resolved && - MapType.acceptsType(a.dataType)) && !m.leftKeyType.sameType(m.rightKeyType) => - findWiderTypeForTwo(m.leftKeyType, m.rightKeyType) match { - case Some(finalKeyType) if !Cast.forceNullable(m.leftKeyType, finalKeyType) && - !Cast.forceNullable(m.rightKeyType, finalKeyType) => - val newLeft = castIfNotSameType( - left, - MapType(finalKeyType, m.leftValueType, m.leftValueContainsNull)) - val newRight = castIfNotSameType( - right, - MapType(finalKeyType, m.rightValueType, m.rightValueContainsNull)) - MapZipWith(newLeft, newRight, function) - case _ => m - } - } - } - /** * Coerces the types of [[Elt]] children to expected ones. * diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala index b24d7486f345..0a39b43ece0c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/collectionOperations.scala @@ -340,174 +340,6 @@ case class MapValues(child: Expression) override def prettyName: String = "map_values" } -/** - * Returns an unordered array of all entries in the given map. - */ -@ExpressionDescription( - usage = "_FUNC_(map) - Returns an unordered array of all entries in the given map.", - examples = """ - Examples: - > SELECT _FUNC_(map(1, 'a', 2, 'b')); - [{"key":1,"value":"a"},{"key":2,"value":"b"}] - """, - since = "2.4.0") -case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInputTypes { - - override def inputTypes: Seq[AbstractDataType] = Seq(MapType) - - @transient private lazy val childDataType: MapType = child.dataType.asInstanceOf[MapType] - - override def dataType: DataType = { - ArrayType( - StructType( - StructField("key", childDataType.keyType, false) :: - StructField("value", childDataType.valueType, childDataType.valueContainsNull) :: - Nil), - false) - } - - override protected def nullSafeEval(input: Any): Any = { - val childMap = input.asInstanceOf[MapData] - val keys = childMap.keyArray() - val values = childMap.valueArray() - val length = childMap.numElements() - val resultData = new Array[AnyRef](length) - var i = 0 - while (i < length) { - val key = keys.get(i, childDataType.keyType) - val value = values.get(i, childDataType.valueType) - val row = new GenericInternalRow(Array[Any](key, value)) - resultData.update(i, row) - i += 1 - } - new GenericArrayData(resultData) - } - - override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - nullSafeCodeGen(ctx, ev, c => { - val arrayData = ctx.freshName("arrayData") - val numElements = ctx.freshName("numElements") - val keys = ctx.freshName("keys") - val values = ctx.freshName("values") - val isKeyPrimitive = CodeGenerator.isPrimitiveType(childDataType.keyType) - val isValuePrimitive = CodeGenerator.isPrimitiveType(childDataType.valueType) - - val wordSize = UnsafeRow.WORD_SIZE - val structSize = UnsafeRow.calculateBitSetWidthInBytes(2) + wordSize * 2 - val (isPrimitive, elementSize) = if (isKeyPrimitive && isValuePrimitive) { - (true, structSize + wordSize) - } else { - (false, -1) - } - - val allocation = - s""" - |ArrayData $arrayData = ArrayData.allocateArrayData( - | $elementSize, $numElements, " $prettyName failed."); - """.stripMargin - - val code = if (isPrimitive) { - val genCodeForPrimitive = genCodeForPrimitiveElements( - ctx, arrayData, keys, values, ev.value, numElements, structSize) - s""" - |if ($arrayData instanceof UnsafeArrayData) { - | $genCodeForPrimitive - |} else { - | ${genCodeForAnyElements(ctx, arrayData, keys, values, ev.value, numElements)} - |} - """.stripMargin - } else { - s"${genCodeForAnyElements(ctx, arrayData, keys, values, ev.value, numElements)}" - } - - s""" - |final int $numElements = $c.numElements(); - |final ArrayData $keys = $c.keyArray(); - |final ArrayData $values = $c.valueArray(); - |$allocation - |$code - """.stripMargin - }) - } - - private def getKey(varName: String, index: String) = - CodeGenerator.getValue(varName, childDataType.keyType, index) - - private def getValue(varName: String, index: String) = - CodeGenerator.getValue(varName, childDataType.valueType, index) - - private def genCodeForPrimitiveElements( - ctx: CodegenContext, - arrayData: String, - keys: String, - values: String, - resultArrayData: String, - numElements: String, - structSize: Int): String = { - val unsafeArrayData = ctx.freshName("unsafeArrayData") - val baseObject = ctx.freshName("baseObject") - val unsafeRow = ctx.freshName("unsafeRow") - val structsOffset = ctx.freshName("structsOffset") - val offset = ctx.freshName("offset") - val z = ctx.freshName("z") - val calculateHeader = "UnsafeArrayData.calculateHeaderPortionInBytes" - - val baseOffset = Platform.BYTE_ARRAY_OFFSET - val wordSize = UnsafeRow.WORD_SIZE - val structSizeAsLong = s"${structSize}L" - - val setKey = CodeGenerator.setColumn(unsafeRow, childDataType.keyType, 0, getKey(keys, z)) - - val valueAssignmentChecked = CodeGenerator.createArrayAssignment( - unsafeRow, childDataType.valueType, values, "1", z, childDataType.valueContainsNull) - - s""" - |UnsafeArrayData $unsafeArrayData = (UnsafeArrayData)$arrayData; - |Object $baseObject = $unsafeArrayData.getBaseObject(); - |final int $structsOffset = $calculateHeader($numElements) + $numElements * $wordSize; - |UnsafeRow $unsafeRow = new UnsafeRow(2); - |for (int $z = 0; $z < $numElements; $z++) { - | long $offset = $structsOffset + $z * $structSizeAsLong; - | $unsafeArrayData.setLong($z, ($offset << 32) + $structSizeAsLong); - | $unsafeRow.pointTo($baseObject, $baseOffset + $offset, $structSize); - | $setKey; - | $valueAssignmentChecked - |} - |$resultArrayData = $arrayData; - """.stripMargin - } - - private def genCodeForAnyElements( - ctx: CodegenContext, - arrayData: String, - keys: String, - values: String, - resultArrayData: String, - numElements: String): String = { - val z = ctx.freshName("z") - val isValuePrimitive = CodeGenerator.isPrimitiveType(childDataType.valueType) - val getValueWithCheck = if (childDataType.valueContainsNull && isValuePrimitive) { - s"$values.isNullAt($z) ? null : (Object)${getValue(values, z)}" - } else { - getValue(values, z) - } - - val rowClass = classOf[GenericInternalRow].getName - val genericArrayDataClass = classOf[GenericArrayData].getName - val genericArrayData = ctx.freshName("genericArrayData") - val rowObject = s"new $rowClass(new Object[]{${getKey(keys, z)}, $getValueWithCheck})" - s""" - |$genericArrayDataClass $genericArrayData = ($genericArrayDataClass)$arrayData; - |for (int $z = 0; $z < $numElements; $z++) { - | $genericArrayData.update($z, $rowObject); - |} - |$resultArrayData = $arrayData; - """.stripMargin - } - - override def prettyName: String = "map_entries" -} - /** * Returns the union of all the given maps. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index b07d9466ba0d..32f97537706b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -260,56 +260,6 @@ case class ArrayTransform( override def prettyName: String = "transform" } -/** - * Filters entries in a map using the provided function. - */ -@ExpressionDescription( -usage = "_FUNC_(expr, func) - Filters entries in a map using the function.", -examples = """ - Examples: - > SELECT _FUNC_(map(1, 0, 2, 2, 3, -1), (k, v) -> k > v); - {1:0,3:-1} - """, -since = "2.4.0") -case class MapFilter( - argument: Expression, - function: Expression) - extends MapBasedSimpleHigherOrderFunction with CodegenFallback { - - @transient lazy val (keyVar, valueVar) = { - val args = function.asInstanceOf[LambdaFunction].arguments - (args.head.asInstanceOf[NamedLambdaVariable], args.tail.head.asInstanceOf[NamedLambdaVariable]) - } - - @transient lazy val MapType(keyType, valueType, valueContainsNull) = argument.dataType - - override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): MapFilter = { - copy(function = f(function, (keyType, false) :: (valueType, valueContainsNull) :: Nil)) - } - - override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = { - val m = argumentValue.asInstanceOf[MapData] - val f = functionForEval - val retKeys = new mutable.ListBuffer[Any] - val retValues = new mutable.ListBuffer[Any] - m.foreach(keyType, valueType, (k, v) => { - keyVar.value.set(k) - valueVar.value.set(v) - if (f.eval(inputRow).asInstanceOf[Boolean]) { - retKeys += k - retValues += v - } - }) - ArrayBasedMapData(retKeys.toArray, retValues.toArray) - } - - override def dataType: DataType = argument.dataType - - override def functionType: AbstractDataType = BooleanType - - override def prettyName: String = "map_filter" -} - /** * Filters the input array using the given lambda function. */ @@ -491,286 +441,6 @@ case class ArrayAggregate( override def prettyName: String = "aggregate" } -/** - * Transform Keys for every entry of the map by applying the transform_keys function. - * Returns map with transformed key entries - */ -@ExpressionDescription( - usage = "_FUNC_(expr, func) - Transforms elements in a map using the function.", - examples = """ - Examples: - > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + 1); - {2:1,3:2,4:3} - > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); - {2:1,4:2,6:3} - """, - since = "2.4.0") -case class TransformKeys( - argument: Expression, - function: Expression) - extends MapBasedSimpleHigherOrderFunction with CodegenFallback { - - @transient lazy val MapType(keyType, valueType, valueContainsNull) = argument.dataType - - override def dataType: DataType = MapType(function.dataType, valueType, valueContainsNull) - - override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): TransformKeys = { - copy(function = f(function, (keyType, false) :: (valueType, valueContainsNull) :: Nil)) - } - - @transient lazy val LambdaFunction( - _, (keyVar: NamedLambdaVariable) :: (valueVar: NamedLambdaVariable) :: Nil, _) = function - - - override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = { - val map = argumentValue.asInstanceOf[MapData] - val resultKeys = new GenericArrayData(new Array[Any](map.numElements)) - var i = 0 - while (i < map.numElements) { - keyVar.value.set(map.keyArray().get(i, keyVar.dataType)) - valueVar.value.set(map.valueArray().get(i, valueVar.dataType)) - val result = functionForEval.eval(inputRow) - if (result == null) { - throw new RuntimeException("Cannot use null as map key!") - } - resultKeys.update(i, result) - i += 1 - } - new ArrayBasedMapData(resultKeys, map.valueArray()) - } - - override def prettyName: String = "transform_keys" -} - -/** - * Returns a map that applies the function to each value of the map. - */ -@ExpressionDescription( - usage = "_FUNC_(expr, func) - Transforms values in the map using the function.", - examples = """ - Examples: - > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> v + 1); - {1:2,2:3,3:4} - > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); - {1:2,2:4,3:6} - """, - since = "2.4.0") -case class TransformValues( - argument: Expression, - function: Expression) - extends MapBasedSimpleHigherOrderFunction with CodegenFallback { - - @transient lazy val MapType(keyType, valueType, valueContainsNull) = argument.dataType - - override def dataType: DataType = MapType(keyType, function.dataType, function.nullable) - - override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction) - : TransformValues = { - copy(function = f(function, (keyType, false) :: (valueType, valueContainsNull) :: Nil)) - } - - @transient lazy val LambdaFunction( - _, (keyVar: NamedLambdaVariable) :: (valueVar: NamedLambdaVariable) :: Nil, _) = function - - override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = { - val map = argumentValue.asInstanceOf[MapData] - val resultValues = new GenericArrayData(new Array[Any](map.numElements)) - var i = 0 - while (i < map.numElements) { - keyVar.value.set(map.keyArray().get(i, keyVar.dataType)) - valueVar.value.set(map.valueArray().get(i, valueVar.dataType)) - resultValues.update(i, functionForEval.eval(inputRow)) - i += 1 - } - new ArrayBasedMapData(map.keyArray(), resultValues) - } - - override def prettyName: String = "transform_values" -} - -/** - * Merges two given maps into a single map by applying function to the pair of values with - * the same key. - */ -@ExpressionDescription( - usage = - """ - _FUNC_(map1, map2, function) - Merges two given maps into a single map by applying - function to the pair of values with the same key. For keys only presented in one map, - NULL will be passed as the value for the missing key. If an input map contains duplicated - keys, only the first entry of the duplicated key is passed into the lambda function. - """, - examples = """ - Examples: - > SELECT _FUNC_(map(1, 'a', 2, 'b'), map(1, 'x', 2, 'y'), (k, v1, v2) -> concat(v1, v2)); - {1:"ax",2:"by"} - """, - since = "2.4.0") -case class MapZipWith(left: Expression, right: Expression, function: Expression) - extends HigherOrderFunction with CodegenFallback { - - def functionForEval: Expression = functionsForEval.head - - @transient lazy val MapType(leftKeyType, leftValueType, leftValueContainsNull) = left.dataType - - @transient lazy val MapType(rightKeyType, rightValueType, rightValueContainsNull) = right.dataType - - @transient lazy val keyType = - TypeCoercion.findCommonTypeDifferentOnlyInNullFlags(leftKeyType, rightKeyType).get - - @transient lazy val ordering = TypeUtils.getInterpretedOrdering(keyType) - - override def arguments: Seq[Expression] = left :: right :: Nil - - override def argumentTypes: Seq[AbstractDataType] = MapType :: MapType :: Nil - - override def functions: Seq[Expression] = function :: Nil - - override def functionTypes: Seq[AbstractDataType] = AnyDataType :: Nil - - override def dataType: DataType = MapType(keyType, function.dataType, function.nullable) - - override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): MapZipWith = { - val arguments = Seq((keyType, false), (leftValueType, true), (rightValueType, true)) - copy(function = f(function, arguments)) - } - - override def checkArgumentDataTypes(): TypeCheckResult = { - super.checkArgumentDataTypes() match { - case TypeCheckResult.TypeCheckSuccess => - if (leftKeyType.sameType(rightKeyType)) { - TypeUtils.checkForOrderingExpr(leftKeyType, s"function $prettyName") - } else { - TypeCheckResult.TypeCheckFailure(s"The input to function $prettyName should have " + - s"been two ${MapType.simpleString}s with compatible key types, but the key types are " + - s"[${leftKeyType.catalogString}, ${rightKeyType.catalogString}].") - } - case failure => failure - } - } - - override def checkInputDataTypes(): TypeCheckResult = checkArgumentDataTypes() - - override def eval(input: InternalRow): Any = { - val value1 = left.eval(input) - if (value1 == null) { - null - } else { - val value2 = right.eval(input) - if (value2 == null) { - null - } else { - nullSafeEval(input, value1, value2) - } - } - } - - @transient lazy val LambdaFunction(_, Seq( - keyVar: NamedLambdaVariable, - value1Var: NamedLambdaVariable, - value2Var: NamedLambdaVariable), - _) = function - - /** - * The function accepts two key arrays and returns a collection of keys with indexes - * to value arrays. Indexes are represented as an array of two items. This is a small - * optimization leveraging mutability of arrays. - */ - @transient private lazy val getKeysWithValueIndexes: - (ArrayData, ArrayData) => mutable.Iterable[(Any, Array[Option[Int]])] = { - if (TypeUtils.typeWithProperEquals(keyType)) { - getKeysWithIndexesFast - } else { - getKeysWithIndexesBruteForce - } - } - - private def assertSizeOfArrayBuffer(size: Int): Unit = { - if (size > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { - throw new RuntimeException(s"Unsuccessful try to zip maps with $size " + - s"unique keys due to exceeding the array size limit " + - s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") - } - } - - private def getKeysWithIndexesFast(keys1: ArrayData, keys2: ArrayData) = { - val hashMap = new mutable.LinkedHashMap[Any, Array[Option[Int]]] - for((z, array) <- Array((0, keys1), (1, keys2))) { - var i = 0 - while (i < array.numElements()) { - val key = array.get(i, keyType) - hashMap.get(key) match { - case Some(indexes) => - if (indexes(z).isEmpty) { - indexes(z) = Some(i) - } - case None => - val indexes = Array[Option[Int]](None, None) - indexes(z) = Some(i) - hashMap.put(key, indexes) - } - i += 1 - } - } - hashMap - } - - private def getKeysWithIndexesBruteForce(keys1: ArrayData, keys2: ArrayData) = { - val arrayBuffer = new mutable.ArrayBuffer[(Any, Array[Option[Int]])] - for((z, array) <- Array((0, keys1), (1, keys2))) { - var i = 0 - while (i < array.numElements()) { - val key = array.get(i, keyType) - var found = false - var j = 0 - while (!found && j < arrayBuffer.size) { - val (bufferKey, indexes) = arrayBuffer(j) - if (ordering.equiv(bufferKey, key)) { - found = true - if(indexes(z).isEmpty) { - indexes(z) = Some(i) - } - } - j += 1 - } - if (!found) { - assertSizeOfArrayBuffer(arrayBuffer.size) - val indexes = Array[Option[Int]](None, None) - indexes(z) = Some(i) - arrayBuffer += Tuple2(key, indexes) - } - i += 1 - } - } - arrayBuffer - } - - private def nullSafeEval(inputRow: InternalRow, value1: Any, value2: Any): Any = { - val mapData1 = value1.asInstanceOf[MapData] - val mapData2 = value2.asInstanceOf[MapData] - val keysWithIndexes = getKeysWithValueIndexes(mapData1.keyArray(), mapData2.keyArray()) - val size = keysWithIndexes.size - val keys = new GenericArrayData(new Array[Any](size)) - val values = new GenericArrayData(new Array[Any](size)) - val valueData1 = mapData1.valueArray() - val valueData2 = mapData2.valueArray() - var i = 0 - for ((key, Array(index1, index2)) <- keysWithIndexes) { - val v1 = index1.map(valueData1.get(_, leftValueType)).getOrElse(null) - val v2 = index2.map(valueData2.get(_, rightValueType)).getOrElse(null) - keyVar.value.set(key) - value1Var.value.set(v1) - value2Var.value.set(v2) - keys.update(i, key) - values.update(i, functionForEval.eval(inputRow)) - i += 1 - } - new ArrayBasedMapData(keys, values) - } - - override def prettyName: String = "map_zip_with" -} - // scalastyle:off line.size.limit @ExpressionDescription( usage = "_FUNC_(left, right, func) - Merges the two given arrays, element-wise, into a single array using function. If one array is shorter, nulls are appended at the end to match the length of the longer array, before applying function.", diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala index c7db4ec9e16b..29014a217672 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CollectionExpressionsSuite.scala @@ -83,30 +83,6 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(MapValues(m2), null) } - test("MapEntries") { - def r(values: Any*): InternalRow = create_row(values: _*) - - // Primitive-type keys/values - val mi0 = Literal.create(Map(1 -> 1, 2 -> null, 3 -> 2), MapType(IntegerType, IntegerType)) - val mi1 = Literal.create(Map[Int, Int](), MapType(IntegerType, IntegerType)) - val mi2 = Literal.create(null, MapType(IntegerType, IntegerType)) - val mid0 = Literal.create(Map(1 -> 1.1, 2 -> 2.2), MapType(IntegerType, DoubleType)) - - checkEvaluation(MapEntries(mi0), Seq(r(1, 1), r(2, null), r(3, 2))) - checkEvaluation(MapEntries(mi1), Seq.empty) - checkEvaluation(MapEntries(mi2), null) - checkEvaluation(MapEntries(mid0), Seq(r(1, 1.1), r(2, 2.2))) - - // Non-primitive-type keys/values - val ms0 = Literal.create(Map("a" -> "c", "b" -> null), MapType(StringType, StringType)) - val ms1 = Literal.create(Map[Int, Int](), MapType(StringType, StringType)) - val ms2 = Literal.create(null, MapType(StringType, StringType)) - - checkEvaluation(MapEntries(ms0), Seq(r("a", "c"), r("b", null))) - checkEvaluation(MapEntries(ms1), Seq.empty) - checkEvaluation(MapEntries(ms2), null) - } - test("Map Concat") { val m0 = Literal.create(Map("a" -> "1", "b" -> "2"), MapType(StringType, StringType, valueContainsNull = false)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala index e13f4d98295b..a0155ca98b8e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala @@ -88,11 +88,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper ArrayFilter(expr, createLambda(et, cn, f)).bind(validateBinding) } - def transformKeys(expr: Expression, f: (Expression, Expression) => Expression): Expression = { - val MapType(kt, vt, vcn) = expr.dataType - TransformKeys(expr, createLambda(kt, false, vt, vcn, f)).bind(validateBinding) - } - def aggregate( expr: Expression, zero: Expression, @@ -115,11 +110,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper aggregate(expr, zero, merge, identity) } - def transformValues(expr: Expression, f: (Expression, Expression) => Expression): Expression = { - val MapType(kt, vt, vcn) = expr.dataType - TransformValues(expr, createLambda(kt, false, vt, vcn, f)).bind(validateBinding) - } - test("ArrayTransform") { val ai0 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType, containsNull = false)) val ai1 = Literal.create(Seq[Integer](1, null, 3), ArrayType(IntegerType, containsNull = true)) @@ -161,55 +151,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper Seq("[1, 3, 5]", null, "[4, 6]")) } - test("MapFilter") { - def mapFilter(expr: Expression, f: (Expression, Expression) => Expression): Expression = { - val MapType(kt, vt, vcn) = expr.dataType - MapFilter(expr, createLambda(kt, false, vt, vcn, f)).bind(validateBinding) - } - val mii0 = Literal.create(Map(1 -> 0, 2 -> 10, 3 -> -1), - MapType(IntegerType, IntegerType, valueContainsNull = false)) - val mii1 = Literal.create(Map(1 -> null, 2 -> 10, 3 -> null), - MapType(IntegerType, IntegerType, valueContainsNull = true)) - val miin = Literal.create(null, MapType(IntegerType, IntegerType, valueContainsNull = false)) - - val kGreaterThanV: (Expression, Expression) => Expression = (k, v) => k > v - - checkEvaluation(mapFilter(mii0, kGreaterThanV), Map(1 -> 0, 3 -> -1)) - checkEvaluation(mapFilter(mii1, kGreaterThanV), Map()) - checkEvaluation(mapFilter(miin, kGreaterThanV), null) - - val valueIsNull: (Expression, Expression) => Expression = (_, v) => v.isNull - - checkEvaluation(mapFilter(mii0, valueIsNull), Map()) - checkEvaluation(mapFilter(mii1, valueIsNull), Map(1 -> null, 3 -> null)) - checkEvaluation(mapFilter(miin, valueIsNull), null) - - val msi0 = Literal.create(Map("abcdf" -> 5, "abc" -> 10, "" -> 0), - MapType(StringType, IntegerType, valueContainsNull = false)) - val msi1 = Literal.create(Map("abcdf" -> 5, "abc" -> 10, "" -> null), - MapType(StringType, IntegerType, valueContainsNull = true)) - val msin = Literal.create(null, MapType(StringType, IntegerType, valueContainsNull = false)) - - val isLengthOfKey: (Expression, Expression) => Expression = (k, v) => Length(k) === v - - checkEvaluation(mapFilter(msi0, isLengthOfKey), Map("abcdf" -> 5, "" -> 0)) - checkEvaluation(mapFilter(msi1, isLengthOfKey), Map("abcdf" -> 5)) - checkEvaluation(mapFilter(msin, isLengthOfKey), null) - - val mia0 = Literal.create(Map(1 -> Seq(0, 1, 2), 2 -> Seq(10), -3 -> Seq(-1, 0, -2, 3)), - MapType(IntegerType, ArrayType(IntegerType), valueContainsNull = false)) - val mia1 = Literal.create(Map(1 -> Seq(0, 1, 2), 2 -> null, -3 -> Seq(-1, 0, -2, 3)), - MapType(IntegerType, ArrayType(IntegerType), valueContainsNull = true)) - val mian = Literal.create( - null, MapType(IntegerType, ArrayType(IntegerType), valueContainsNull = false)) - - val customFunc: (Expression, Expression) => Expression = (k, v) => Size(v) + k > 3 - - checkEvaluation(mapFilter(mia0, customFunc), Map(1 -> Seq(0, 1, 2))) - checkEvaluation(mapFilter(mia1, customFunc), Map(1 -> Seq(0, 1, 2))) - checkEvaluation(mapFilter(mian, customFunc), null) - } - test("ArrayFilter") { val ai0 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType, containsNull = false)) val ai1 = Literal.create(Seq[Integer](1, null, 3), ArrayType(IntegerType, containsNull = true)) @@ -308,262 +249,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper 15) } - test("TransformKeys") { - val ai0 = Literal.create( - Map(1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4), - MapType(IntegerType, IntegerType, valueContainsNull = false)) - val ai1 = Literal.create( - Map.empty[Int, Int], - MapType(IntegerType, IntegerType, valueContainsNull = true)) - val ai2 = Literal.create( - Map(1 -> 1, 2 -> null, 3 -> 3), - MapType(IntegerType, IntegerType, valueContainsNull = true)) - val ai3 = Literal.create(null, MapType(IntegerType, IntegerType, valueContainsNull = false)) - - val plusOne: (Expression, Expression) => Expression = (k, v) => k + 1 - val plusValue: (Expression, Expression) => Expression = (k, v) => k + v - val modKey: (Expression, Expression) => Expression = (k, v) => k % 3 - - checkEvaluation(transformKeys(ai0, plusOne), Map(2 -> 1, 3 -> 2, 4 -> 3, 5 -> 4)) - checkEvaluation(transformKeys(ai0, plusValue), Map(2 -> 1, 4 -> 2, 6 -> 3, 8 -> 4)) - checkEvaluation( - transformKeys(transformKeys(ai0, plusOne), plusValue), Map(3 -> 1, 5 -> 2, 7 -> 3, 9 -> 4)) - checkEvaluation(transformKeys(ai0, modKey), - ArrayBasedMapData(Array(1, 2, 0, 1), Array(1, 2, 3, 4))) - checkEvaluation(transformKeys(ai1, plusOne), Map.empty[Int, Int]) - checkEvaluation(transformKeys(ai1, plusOne), Map.empty[Int, Int]) - checkEvaluation( - transformKeys(transformKeys(ai1, plusOne), plusValue), Map.empty[Int, Int]) - checkEvaluation(transformKeys(ai2, plusOne), Map(2 -> 1, 3 -> null, 4 -> 3)) - checkEvaluation( - transformKeys(transformKeys(ai2, plusOne), plusOne), Map(3 -> 1, 4 -> null, 5 -> 3)) - checkEvaluation(transformKeys(ai3, plusOne), null) - - val as0 = Literal.create( - Map("a" -> "xy", "bb" -> "yz", "ccc" -> "zx"), - MapType(StringType, StringType, valueContainsNull = false)) - val as1 = Literal.create( - Map("a" -> "xy", "bb" -> "yz", "ccc" -> null), - MapType(StringType, StringType, valueContainsNull = true)) - val as2 = Literal.create(null, - MapType(StringType, StringType, valueContainsNull = false)) - val as3 = Literal.create(Map.empty[StringType, StringType], - MapType(StringType, StringType, valueContainsNull = true)) - - val concatValue: (Expression, Expression) => Expression = (k, v) => Concat(Seq(k, v)) - val convertKeyToKeyLength: (Expression, Expression) => Expression = - (k, v) => Length(k) + 1 - - checkEvaluation( - transformKeys(as0, concatValue), Map("axy" -> "xy", "bbyz" -> "yz", "ccczx" -> "zx")) - checkEvaluation( - transformKeys(transformKeys(as0, concatValue), concatValue), - Map("axyxy" -> "xy", "bbyzyz" -> "yz", "ccczxzx" -> "zx")) - checkEvaluation(transformKeys(as3, concatValue), Map.empty[String, String]) - checkEvaluation( - transformKeys(transformKeys(as3, concatValue), convertKeyToKeyLength), - Map.empty[Int, String]) - checkEvaluation(transformKeys(as0, convertKeyToKeyLength), - Map(2 -> "xy", 3 -> "yz", 4 -> "zx")) - checkEvaluation(transformKeys(as1, convertKeyToKeyLength), - Map(2 -> "xy", 3 -> "yz", 4 -> null)) - checkEvaluation(transformKeys(as2, convertKeyToKeyLength), null) - checkEvaluation(transformKeys(as3, convertKeyToKeyLength), Map.empty[Int, String]) - - val ax0 = Literal.create( - Map(1 -> "x", 2 -> "y", 3 -> "z"), - MapType(IntegerType, StringType, valueContainsNull = false)) - - checkEvaluation(transformKeys(ax0, plusOne), Map(2 -> "x", 3 -> "y", 4 -> "z")) - } - - test("TransformValues") { - val ai0 = Literal.create( - Map(1 -> 1, 2 -> 2, 3 -> 3), - MapType(IntegerType, IntegerType, valueContainsNull = false)) - val ai1 = Literal.create( - Map(1 -> 1, 2 -> null, 3 -> 3), - MapType(IntegerType, IntegerType, valueContainsNull = true)) - val ai2 = Literal.create( - Map.empty[Int, Int], - MapType(IntegerType, IntegerType, valueContainsNull = true)) - val ai3 = Literal.create(null, MapType(IntegerType, IntegerType, valueContainsNull = false)) - - val plusOne: (Expression, Expression) => Expression = (k, v) => v + 1 - val valueUpdate: (Expression, Expression) => Expression = (k, v) => k * k - - checkEvaluation(transformValues(ai0, plusOne), Map(1 -> 2, 2 -> 3, 3 -> 4)) - checkEvaluation(transformValues(ai0, valueUpdate), Map(1 -> 1, 2 -> 4, 3 -> 9)) - checkEvaluation( - transformValues(transformValues(ai0, plusOne), valueUpdate), Map(1 -> 1, 2 -> 4, 3 -> 9)) - checkEvaluation(transformValues(ai1, plusOne), Map(1 -> 2, 2 -> null, 3 -> 4)) - checkEvaluation(transformValues(ai1, valueUpdate), Map(1 -> 1, 2 -> 4, 3 -> 9)) - checkEvaluation( - transformValues(transformValues(ai1, plusOne), valueUpdate), Map(1 -> 1, 2 -> 4, 3 -> 9)) - checkEvaluation(transformValues(ai2, plusOne), Map.empty[Int, Int]) - checkEvaluation(transformValues(ai3, plusOne), null) - - val as0 = Literal.create( - Map("a" -> "xy", "bb" -> "yz", "ccc" -> "zx"), - MapType(StringType, StringType, valueContainsNull = false)) - val as1 = Literal.create( - Map("a" -> "xy", "bb" -> null, "ccc" -> "zx"), - MapType(StringType, StringType, valueContainsNull = true)) - val as2 = Literal.create(Map.empty[StringType, StringType], - MapType(StringType, StringType, valueContainsNull = true)) - val as3 = Literal.create(null, MapType(StringType, StringType, valueContainsNull = true)) - - val concatValue: (Expression, Expression) => Expression = (k, v) => Concat(Seq(k, v)) - val valueTypeUpdate: (Expression, Expression) => Expression = - (k, v) => Length(v) + 1 - - checkEvaluation( - transformValues(as0, concatValue), Map("a" -> "axy", "bb" -> "bbyz", "ccc" -> "ccczx")) - checkEvaluation(transformValues(as0, valueTypeUpdate), - Map("a" -> 3, "bb" -> 3, "ccc" -> 3)) - checkEvaluation( - transformValues(transformValues(as0, concatValue), concatValue), - Map("a" -> "aaxy", "bb" -> "bbbbyz", "ccc" -> "cccccczx")) - checkEvaluation(transformValues(as1, concatValue), - Map("a" -> "axy", "bb" -> null, "ccc" -> "ccczx")) - checkEvaluation(transformValues(as1, valueTypeUpdate), - Map("a" -> 3, "bb" -> null, "ccc" -> 3)) - checkEvaluation( - transformValues(transformValues(as1, concatValue), concatValue), - Map("a" -> "aaxy", "bb" -> null, "ccc" -> "cccccczx")) - checkEvaluation(transformValues(as2, concatValue), Map.empty[String, String]) - checkEvaluation(transformValues(as2, valueTypeUpdate), Map.empty[String, Int]) - checkEvaluation( - transformValues(transformValues(as2, concatValue), valueTypeUpdate), - Map.empty[String, Int]) - checkEvaluation(transformValues(as3, concatValue), null) - - val ax0 = Literal.create( - Map(1 -> "x", 2 -> "y", 3 -> "z"), - MapType(IntegerType, StringType, valueContainsNull = false)) - - checkEvaluation(transformValues(ax0, valueUpdate), Map(1 -> 1, 2 -> 4, 3 -> 9)) - } - - test("MapZipWith") { - def map_zip_with( - left: Expression, - right: Expression, - f: (Expression, Expression, Expression) => Expression): Expression = { - val MapType(kt, vt1, _) = left.dataType - val MapType(_, vt2, _) = right.dataType - MapZipWith(left, right, createLambda(kt, false, vt1, true, vt2, true, f)) - .bind(validateBinding) - } - - val mii0 = Literal.create(Map(1 -> 10, 2 -> 20, 3 -> 30), - MapType(IntegerType, IntegerType, valueContainsNull = false)) - val mii1 = Literal.create(Map(1 -> -1, 2 -> -2, 4 -> -4), - MapType(IntegerType, IntegerType, valueContainsNull = false)) - val mii2 = Literal.create(Map(1 -> null, 2 -> -2, 3 -> null), - MapType(IntegerType, IntegerType, valueContainsNull = true)) - val mii3 = Literal.create(Map(), MapType(IntegerType, IntegerType, valueContainsNull = false)) - val mii4 = MapFromArrays( - Literal.create(Seq(2, 2), ArrayType(IntegerType, false)), - Literal.create(Seq(20, 200), ArrayType(IntegerType, false))) - val miin = Literal.create(null, MapType(IntegerType, IntegerType, valueContainsNull = false)) - - val multiplyKeyWithValues: (Expression, Expression, Expression) => Expression = { - (k, v1, v2) => k * v1 * v2 - } - - checkEvaluation( - map_zip_with(mii0, mii1, multiplyKeyWithValues), - Map(1 -> -10, 2 -> -80, 3 -> null, 4 -> null)) - checkEvaluation( - map_zip_with(mii0, mii2, multiplyKeyWithValues), - Map(1 -> null, 2 -> -80, 3 -> null)) - checkEvaluation( - map_zip_with(mii0, mii3, multiplyKeyWithValues), - Map(1 -> null, 2 -> null, 3 -> null)) - checkEvaluation( - map_zip_with(mii0, mii4, multiplyKeyWithValues), - Map(1 -> null, 2 -> 800, 3 -> null)) - checkEvaluation( - map_zip_with(mii4, mii0, multiplyKeyWithValues), - Map(2 -> 800, 1 -> null, 3 -> null)) - checkEvaluation( - map_zip_with(mii0, miin, multiplyKeyWithValues), - null) - assert(map_zip_with(mii0, mii1, multiplyKeyWithValues).dataType === - MapType(IntegerType, IntegerType, valueContainsNull = true)) - - val mss0 = Literal.create(Map("a" -> "x", "b" -> "y", "d" -> "z"), - MapType(StringType, StringType, valueContainsNull = false)) - val mss1 = Literal.create(Map("d" -> "b", "b" -> "d"), - MapType(StringType, StringType, valueContainsNull = false)) - val mss2 = Literal.create(Map("c" -> null, "b" -> "t", "a" -> null), - MapType(StringType, StringType, valueContainsNull = true)) - val mss3 = Literal.create(Map(), MapType(StringType, StringType, valueContainsNull = false)) - val mss4 = MapFromArrays( - Literal.create(Seq("a", "a"), ArrayType(StringType, false)), - Literal.create(Seq("a", "n"), ArrayType(StringType, false))) - val mssn = Literal.create(null, MapType(StringType, StringType, valueContainsNull = false)) - - val concat: (Expression, Expression, Expression) => Expression = { - (k, v1, v2) => Concat(Seq(k, v1, v2)) - } - - checkEvaluation( - map_zip_with(mss0, mss1, concat), - Map("a" -> null, "b" -> "byd", "d" -> "dzb")) - checkEvaluation( - map_zip_with(mss1, mss2, concat), - Map("d" -> null, "b" -> "bdt", "c" -> null, "a" -> null)) - checkEvaluation( - map_zip_with(mss0, mss3, concat), - Map("a" -> null, "b" -> null, "d" -> null)) - checkEvaluation( - map_zip_with(mss0, mss4, concat), - Map("a" -> "axa", "b" -> null, "d" -> null)) - checkEvaluation( - map_zip_with(mss4, mss0, concat), - Map("a" -> "aax", "b" -> null, "d" -> null)) - checkEvaluation( - map_zip_with(mss0, mssn, concat), - null) - assert(map_zip_with(mss0, mss1, concat).dataType === - MapType(StringType, StringType, valueContainsNull = true)) - - def b(data: Byte*): Array[Byte] = Array[Byte](data: _*) - - val mbb0 = Literal.create(Map(b(1, 2) -> b(4), b(2, 1) -> b(5), b(1, 3) -> b(8)), - MapType(BinaryType, BinaryType, valueContainsNull = false)) - val mbb1 = Literal.create(Map(b(2, 1) -> b(7), b(1, 2) -> b(3), b(1, 1) -> b(6)), - MapType(BinaryType, BinaryType, valueContainsNull = false)) - val mbb2 = Literal.create(Map(b(1, 3) -> null, b(1, 2) -> b(2), b(2, 1) -> null), - MapType(BinaryType, BinaryType, valueContainsNull = true)) - val mbb3 = Literal.create(Map(), MapType(BinaryType, BinaryType, valueContainsNull = false)) - val mbb4 = MapFromArrays( - Literal.create(Seq(b(2, 1), b(2, 1)), ArrayType(BinaryType, false)), - Literal.create(Seq(b(1), b(9)), ArrayType(BinaryType, false))) - val mbbn = Literal.create(null, MapType(BinaryType, BinaryType, valueContainsNull = false)) - - checkEvaluation( - map_zip_with(mbb0, mbb1, concat), - Map(b(1, 2) -> b(1, 2, 4, 3), b(2, 1) -> b(2, 1, 5, 7), b(1, 3) -> null, b(1, 1) -> null)) - checkEvaluation( - map_zip_with(mbb1, mbb2, concat), - Map(b(2, 1) -> null, b(1, 2) -> b(1, 2, 3, 2), b(1, 1) -> null, b(1, 3) -> null)) - checkEvaluation( - map_zip_with(mbb0, mbb3, concat), - Map(b(1, 2) -> null, b(2, 1) -> null, b(1, 3) -> null)) - checkEvaluation( - map_zip_with(mbb0, mbb4, concat), - Map(b(1, 2) -> null, b(2, 1) -> b(2, 1, 5, 1), b(1, 3) -> null)) - checkEvaluation( - map_zip_with(mbb4, mbb0, concat), - Map(b(2, 1) -> b(2, 1, 1, 5), b(1, 2) -> null, b(1, 3) -> null)) - checkEvaluation( - map_zip_with(mbb0, mbbn, concat), - null) - } - test("ZipWith") { def zip_with( left: Expression, 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 6a43ce160efe..5dedc9db0a61 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 @@ -3780,13 +3780,6 @@ object functions { */ def map_values(e: Column): Column = withExpr { MapValues(e.expr) } - /** - * Returns an unordered array of all entries in the given map. - * @group collection_funcs - * @since 2.4.0 - */ - def map_entries(e: Column): Column = withExpr { MapEntries(e.expr) } - /** * Returns a map created from the given array of entries. * @group collection_funcs diff --git a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql index 02ad5e353868..504306470fae 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/higher-order-functions.sql @@ -60,26 +60,3 @@ select zip_with(array('a', 'b', 'c'), array('d', 'e', 'f'), (x, y) -> concat(x, -- Zip with array coalesce select zip_with(array('a'), array('d', null, 'f'), (x, y) -> coalesce(x, y)) as v; - -create or replace temporary view nested as values - (1, map(1, 1, 2, 2, 3, 3)), - (2, map(4, 4, 5, 5, 6, 6)) - as t(x, ys); - --- Identity Transform Keys in a map -select transform_keys(ys, (k, v) -> k) as v from nested; - --- Transform Keys in a map by adding constant -select transform_keys(ys, (k, v) -> k + 1) as v from nested; - --- Transform Keys in a map using values -select transform_keys(ys, (k, v) -> k + v) as v from nested; - --- Identity Transform values in a map -select transform_values(ys, (k, v) -> v) as v from nested; - --- Transform values in a map by adding constant -select transform_values(ys, (k, v) -> v + 1) as v from nested; - --- Transform values in a map using values -select transform_values(ys, (k, v) -> k + v) as v from nested; diff --git a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapZipWith.sql b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapZipWith.sql deleted file mode 100644 index 1727ee725db2..000000000000 --- a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapZipWith.sql +++ /dev/null @@ -1,78 +0,0 @@ -CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( - map(true, false), - map(2Y, 1Y), - map(2S, 1S), - map(2, 1), - map(2L, 1L), - map(922337203685477897945456575809789456, 922337203685477897945456575809789456), - map(9.22337203685477897945456575809789456, 9.22337203685477897945456575809789456), - map(2.0D, 1.0D), - map(float(2.0), float(1.0)), - map(date '2016-03-14', date '2016-03-13'), - map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), - map('true', 'false', '2', '1'), - map('2016-03-14', '2016-03-13'), - map('2016-11-15 20:54:00.000', '2016-11-12 20:54:00.000'), - map('922337203685477897945456575809789456', 'text'), - map(array(1L, 2L), array(1L, 2L)), map(array(1, 2), array(1, 2)), - map(struct(1S, 2L), struct(1S, 2L)), map(struct(1, 2), struct(1, 2)) -) AS various_maps( - boolean_map, - tinyint_map, - smallint_map, - int_map, - bigint_map, - decimal_map1, decimal_map2, - double_map, - float_map, - date_map, - timestamp_map, - string_map1, string_map2, string_map3, string_map4, - array_map1, array_map2, - struct_map1, struct_map2 -); - -SELECT map_zip_with(tinyint_map, smallint_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(smallint_map, int_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(int_map, bigint_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(double_map, float_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(decimal_map1, decimal_map2, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(decimal_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(decimal_map1, double_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(decimal_map2, int_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(decimal_map2, double_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(array_map1, array_map2, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; - -SELECT map_zip_with(struct_map1, struct_map2, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps; diff --git a/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out index 32d20d1b7341..cdc509aeb16c 100644 --- a/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/higher-order-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 27 +-- Number of queries: 20 -- !query 0 @@ -189,67 +189,3 @@ select zip_with(array('a'), array('d', null, 'f'), (x, y) -> coalesce(x, y)) as struct> -- !query 19 output ["a",null,"f"] - - --- !query 20 -create or replace temporary view nested as values - (1, map(1, 1, 2, 2, 3, 3)), - (2, map(4, 4, 5, 5, 6, 6)) - as t(x, ys) --- !query 20 schema -struct<> --- !query 20 output - - --- !query 21 -select transform_keys(ys, (k, v) -> k) as v from nested --- !query 21 schema -struct> --- !query 21 output -{1:1,2:2,3:3} -{4:4,5:5,6:6} - - --- !query 22 -select transform_keys(ys, (k, v) -> k + 1) as v from nested --- !query 22 schema -struct> --- !query 22 output -{2:1,3:2,4:3} -{5:4,6:5,7:6} - - --- !query 23 -select transform_keys(ys, (k, v) -> k + v) as v from nested --- !query 23 schema -struct> --- !query 23 output -{10:5,12:6,8:4} -{2:1,4:2,6:3} - - --- !query 24 -select transform_values(ys, (k, v) -> v) as v from nested --- !query 24 schema -struct> --- !query 24 output -{1:1,2:2,3:3} -{4:4,5:5,6:6} - - --- !query 25 -select transform_values(ys, (k, v) -> v + 1) as v from nested --- !query 25 schema -struct> --- !query 25 output -{1:2,2:3,3:4} -{4:5,5:6,6:7} - - --- !query 26 -select transform_values(ys, (k, v) -> k + v) as v from nested --- !query 26 schema -struct> --- !query 26 output -{1:2,2:4,3:6} -{4:8,5:10,6:12} diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out deleted file mode 100644 index 35740094ba53..000000000000 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out +++ /dev/null @@ -1,179 +0,0 @@ --- Automatically generated by SQLQueryTestSuite --- Number of queries: 16 - - --- !query 0 -CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( - map(true, false), - map(2Y, 1Y), - map(2S, 1S), - map(2, 1), - map(2L, 1L), - map(922337203685477897945456575809789456, 922337203685477897945456575809789456), - map(9.22337203685477897945456575809789456, 9.22337203685477897945456575809789456), - map(2.0D, 1.0D), - map(float(2.0), float(1.0)), - map(date '2016-03-14', date '2016-03-13'), - map(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), - map('true', 'false', '2', '1'), - map('2016-03-14', '2016-03-13'), - map('2016-11-15 20:54:00.000', '2016-11-12 20:54:00.000'), - map('922337203685477897945456575809789456', 'text'), - map(array(1L, 2L), array(1L, 2L)), map(array(1, 2), array(1, 2)), - map(struct(1S, 2L), struct(1S, 2L)), map(struct(1, 2), struct(1, 2)) -) AS various_maps( - boolean_map, - tinyint_map, - smallint_map, - int_map, - bigint_map, - decimal_map1, decimal_map2, - double_map, - float_map, - date_map, - timestamp_map, - string_map1, string_map2, string_map3, string_map4, - array_map1, array_map2, - struct_map1, struct_map2 -) --- !query 0 schema -struct<> --- !query 0 output - - - --- !query 1 -SELECT map_zip_with(tinyint_map, smallint_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 1 schema -struct>> --- !query 1 output -{2:{"k":2,"v1":1,"v2":1}} - - --- !query 2 -SELECT map_zip_with(smallint_map, int_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 2 schema -struct>> --- !query 2 output -{2:{"k":2,"v1":1,"v2":1}} - - --- !query 3 -SELECT map_zip_with(int_map, bigint_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 3 schema -struct>> --- !query 3 output -{2:{"k":2,"v1":1,"v2":1}} - - --- !query 4 -SELECT map_zip_with(double_map, float_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 4 schema -struct>> --- !query 4 output -{2.0:{"k":2.0,"v1":1.0,"v2":1.0}} - - --- !query 5 -SELECT map_zip_with(decimal_map1, decimal_map2, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 5 schema -struct<> --- !query 5 output -org.apache.spark.sql.AnalysisException -cannot resolve 'map_zip_with(various_maps.`decimal_map1`, various_maps.`decimal_map2`, lambdafunction(named_struct(NamePlaceholder(), `k`, NamePlaceholder(), `v1`, NamePlaceholder(), `v2`), `k`, `v1`, `v2`))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,0), decimal(36,35)].; line 1 pos 7 - - --- !query 6 -SELECT map_zip_with(decimal_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 6 schema -struct>> --- !query 6 output -{2:{"k":2,"v1":null,"v2":1},922337203685477897945456575809789456:{"k":922337203685477897945456575809789456,"v1":922337203685477897945456575809789456,"v2":null}} - - --- !query 7 -SELECT map_zip_with(decimal_map1, double_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 7 schema -struct>> --- !query 7 output -{2.0:{"k":2.0,"v1":null,"v2":1.0},9.223372036854779E35:{"k":9.223372036854779E35,"v1":922337203685477897945456575809789456,"v2":null}} - - --- !query 8 -SELECT map_zip_with(decimal_map2, int_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 8 schema -struct<> --- !query 8 output -org.apache.spark.sql.AnalysisException -cannot resolve 'map_zip_with(various_maps.`decimal_map2`, various_maps.`int_map`, lambdafunction(named_struct(NamePlaceholder(), `k`, NamePlaceholder(), `v1`, NamePlaceholder(), `v2`), `k`, `v1`, `v2`))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,35), int].; line 1 pos 7 - - --- !query 9 -SELECT map_zip_with(decimal_map2, double_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 9 schema -struct>> --- !query 9 output -{2.0:{"k":2.0,"v1":null,"v2":1.0},9.223372036854778:{"k":9.223372036854778,"v1":9.22337203685477897945456575809789456,"v2":null}} - - --- !query 10 -SELECT map_zip_with(string_map1, int_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 10 schema -struct>> --- !query 10 output -{"2":{"k":"2","v1":"1","v2":1},"true":{"k":"true","v1":"false","v2":null}} - - --- !query 11 -SELECT map_zip_with(string_map2, date_map, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 11 schema -struct>> --- !query 11 output -{"2016-03-14":{"k":"2016-03-14","v1":"2016-03-13","v2":2016-03-13}} - - --- !query 12 -SELECT map_zip_with(timestamp_map, string_map3, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 12 schema -struct>> --- !query 12 output -{"2016-11-15 20:54:00":{"k":"2016-11-15 20:54:00","v1":2016-11-12 20:54:00.0,"v2":null},"2016-11-15 20:54:00.000":{"k":"2016-11-15 20:54:00.000","v1":null,"v2":"2016-11-12 20:54:00.000"}} - - --- !query 13 -SELECT map_zip_with(decimal_map1, string_map4, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 13 schema -struct>> --- !query 13 output -{"922337203685477897945456575809789456":{"k":"922337203685477897945456575809789456","v1":922337203685477897945456575809789456,"v2":"text"}} - - --- !query 14 -SELECT map_zip_with(array_map1, array_map2, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 14 schema -struct,struct,v1:array,v2:array>>> --- !query 14 output -{[1,2]:{"k":[1,2],"v1":[1,2],"v2":[1,2]}} - - --- !query 15 -SELECT map_zip_with(struct_map1, struct_map2, (k, v1, v2) -> struct(k, v1, v2)) m -FROM various_maps --- !query 15 schema -struct,struct,v1:struct,v2:struct>>> --- !query 15 output -{{"col1":1,"col2":2}:{"k":{"col1":1,"col2":2},"v1":{"col1":1,"col2":2},"v2":{"col1":1,"col2":2}}} 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 60ebc5e6cc09..d4f9b90cb06d 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 @@ -509,58 +509,6 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { ) } - test("map_entries") { - // Primitive-type elements - val idf = Seq( - Map[Int, Int](1 -> 100, 2 -> 200, 3 -> 300), - Map[Int, Int](), - null - ).toDF("m") - val iExpected = Seq( - Row(Seq(Row(1, 100), Row(2, 200), Row(3, 300))), - Row(Seq.empty), - Row(null) - ) - - def testPrimitiveType(): Unit = { - checkAnswer(idf.select(map_entries('m)), iExpected) - checkAnswer(idf.selectExpr("map_entries(m)"), iExpected) - checkAnswer(idf.selectExpr("map_entries(map(1, null, 2, null))"), - Seq.fill(iExpected.length)(Row(Seq(Row(1, null), Row(2, null))))) - } - - // Test with local relation, the Project will be evaluated without codegen - testPrimitiveType() - // Test with cached relation, the Project will be evaluated with codegen - idf.cache() - testPrimitiveType() - - // Non-primitive-type elements - val sdf = Seq( - Map[String, String]("a" -> "f", "b" -> "o", "c" -> "o"), - Map[String, String]("a" -> null, "b" -> null), - Map[String, String](), - null - ).toDF("m") - val sExpected = Seq( - Row(Seq(Row("a", "f"), Row("b", "o"), Row("c", "o"))), - Row(Seq(Row("a", null), Row("b", null))), - Row(Seq.empty), - Row(null) - ) - - def testNonPrimitiveType(): Unit = { - checkAnswer(sdf.select(map_entries('m)), sExpected) - checkAnswer(sdf.selectExpr("map_entries(m)"), sExpected) - } - - // Test with local relation, the Project will be evaluated without codegen - testNonPrimitiveType() - // Test with cached relation, the Project will be evaluated with codegen - sdf.cache() - testNonPrimitiveType() - } - test("map_concat function") { val df1 = Seq( (Map[Int, Int](1 -> 100, 2 -> 200), Map[Int, Int](3 -> 300, 4 -> 400)), @@ -2070,57 +2018,6 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { assert(ex3.getMessage.contains("cannot resolve '`a`'")) } - test("map_filter") { - val dfInts = Seq( - Map(1 -> 10, 2 -> 20, 3 -> 30), - Map(1 -> -1, 2 -> -2, 3 -> -3), - Map(1 -> 10, 2 -> 5, 3 -> -3)).toDF("m") - - checkAnswer(dfInts.selectExpr( - "map_filter(m, (k, v) -> k * 10 = v)", "map_filter(m, (k, v) -> k = -v)"), - Seq( - Row(Map(1 -> 10, 2 -> 20, 3 -> 30), Map()), - Row(Map(), Map(1 -> -1, 2 -> -2, 3 -> -3)), - Row(Map(1 -> 10), Map(3 -> -3)))) - - val dfComplex = Seq( - Map(1 -> Seq(Some(1)), 2 -> Seq(Some(1), Some(2)), 3 -> Seq(Some(1), Some(2), Some(3))), - Map(1 -> null, 2 -> Seq(Some(-2), Some(-2)), 3 -> Seq[Option[Int]](None))).toDF("m") - - checkAnswer(dfComplex.selectExpr( - "map_filter(m, (k, v) -> k = v[0])", "map_filter(m, (k, v) -> k = size(v))"), - Seq( - Row(Map(1 -> Seq(1)), Map(1 -> Seq(1), 2 -> Seq(1, 2), 3 -> Seq(1, 2, 3))), - Row(Map(), Map(2 -> Seq(-2, -2))))) - - // Invalid use cases - val df = Seq( - (Map(1 -> "a"), 1), - (Map.empty[Int, String], 2), - (null, 3) - ).toDF("s", "i") - - val ex1 = intercept[AnalysisException] { - df.selectExpr("map_filter(s, (x, y, z) -> x + y + z)") - } - assert(ex1.getMessage.contains("The number of lambda function arguments '3' does not match")) - - val ex2 = intercept[AnalysisException] { - df.selectExpr("map_filter(s, x -> x)") - } - assert(ex2.getMessage.contains("The number of lambda function arguments '1' does not match")) - - val ex3 = intercept[AnalysisException] { - df.selectExpr("map_filter(i, (k, v) -> k > v)") - } - assert(ex3.getMessage.contains("data type mismatch: argument 1 requires map type")) - - val ex4 = intercept[AnalysisException] { - df.selectExpr("map_filter(a, (k, v) -> k > v)") - } - assert(ex4.getMessage.contains("cannot resolve '`a`'")) - } - test("filter function - array for primitive type not containing null") { val df = Seq( Seq(1, 9, 8, 7), @@ -2449,328 +2346,6 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { assert(ex5.getMessage.contains("cannot resolve '`a`'")) } - test("map_zip_with function - map of primitive types") { - val df = Seq( - (Map(8 -> 6L, 3 -> 5L, 6 -> 2L), Map[Integer, Integer]((6, 4), (8, 2), (3, 2))), - (Map(10 -> 6L, 8 -> 3L), Map[Integer, Integer]((8, 4), (4, null))), - (Map.empty[Int, Long], Map[Integer, Integer]((5, 1))), - (Map(5 -> 1L), null) - ).toDF("m1", "m2") - - checkAnswer(df.selectExpr("map_zip_with(m1, m2, (k, v1, v2) -> k == v1 + v2)"), - Seq( - Row(Map(8 -> true, 3 -> false, 6 -> true)), - Row(Map(10 -> null, 8 -> false, 4 -> null)), - Row(Map(5 -> null)), - Row(null))) - } - - test("map_zip_with function - map of non-primitive types") { - val df = Seq( - (Map("z" -> "a", "y" -> "b", "x" -> "c"), Map("x" -> "a", "z" -> "c")), - (Map("b" -> "a", "c" -> "d"), Map("c" -> "a", "b" -> null, "d" -> "k")), - (Map("a" -> "d"), Map.empty[String, String]), - (Map("a" -> "d"), null) - ).toDF("m1", "m2") - - checkAnswer(df.selectExpr("map_zip_with(m1, m2, (k, v1, v2) -> (v1, v2))"), - Seq( - Row(Map("z" -> Row("a", "c"), "y" -> Row("b", null), "x" -> Row("c", "a"))), - Row(Map("b" -> Row("a", null), "c" -> Row("d", "a"), "d" -> Row(null, "k"))), - Row(Map("a" -> Row("d", null))), - Row(null))) - } - - test("map_zip_with function - invalid") { - val df = Seq( - (Map(1 -> 2), Map(1 -> "a"), Map("a" -> "b"), Map(Map(1 -> 2) -> 2), 1) - ).toDF("mii", "mis", "mss", "mmi", "i") - - val ex1 = intercept[AnalysisException] { - df.selectExpr("map_zip_with(mii, mis, (x, y) -> x + y)") - } - assert(ex1.getMessage.contains("The number of lambda function arguments '2' does not match")) - - val ex2 = intercept[AnalysisException] { - df.selectExpr("map_zip_with(mis, mmi, (x, y, z) -> concat(x, y, z))") - } - assert(ex2.getMessage.contains("The input to function map_zip_with should have " + - "been two maps with compatible key types")) - - val ex3 = intercept[AnalysisException] { - df.selectExpr("map_zip_with(i, mis, (x, y, z) -> concat(x, y, z))") - } - assert(ex3.getMessage.contains("type mismatch: argument 1 requires map type")) - - val ex4 = intercept[AnalysisException] { - df.selectExpr("map_zip_with(mis, i, (x, y, z) -> concat(x, y, z))") - } - assert(ex4.getMessage.contains("type mismatch: argument 2 requires map type")) - - val ex5 = intercept[AnalysisException] { - df.selectExpr("map_zip_with(mmi, mmi, (x, y, z) -> x)") - } - assert(ex5.getMessage.contains("function map_zip_with does not support ordering on type map")) - } - - test("transform keys function - primitive data types") { - val dfExample1 = Seq( - Map[Int, Int](1 -> 1, 9 -> 9, 8 -> 8, 7 -> 7) - ).toDF("i") - - val dfExample2 = Seq( - Map[Int, Double](1 -> 1.0, 2 -> 1.40, 3 -> 1.70) - ).toDF("j") - - val dfExample3 = Seq( - Map[Int, Boolean](25 -> true, 26 -> false) - ).toDF("x") - - val dfExample4 = Seq( - Map[Array[Int], Boolean](Array(1, 2) -> false) - ).toDF("y") - - - def testMapOfPrimitiveTypesCombination(): Unit = { - checkAnswer(dfExample1.selectExpr("transform_keys(i, (k, v) -> k + v)"), - Seq(Row(Map(2 -> 1, 18 -> 9, 16 -> 8, 14 -> 7)))) - - checkAnswer(dfExample2.selectExpr("transform_keys(j, " + - "(k, v) -> map_from_arrays(ARRAY(1, 2, 3), ARRAY('one', 'two', 'three'))[k])"), - Seq(Row(Map("one" -> 1.0, "two" -> 1.4, "three" -> 1.7)))) - - checkAnswer(dfExample2.selectExpr("transform_keys(j, (k, v) -> CAST(v * 2 AS BIGINT) + k)"), - Seq(Row(Map(3 -> 1.0, 4 -> 1.4, 6 -> 1.7)))) - - checkAnswer(dfExample2.selectExpr("transform_keys(j, (k, v) -> k + v)"), - Seq(Row(Map(2.0 -> 1.0, 3.4 -> 1.4, 4.7 -> 1.7)))) - - checkAnswer(dfExample3.selectExpr("transform_keys(x, (k, v) -> k % 2 = 0 OR v)"), - Seq(Row(Map(true -> true, true -> false)))) - - checkAnswer(dfExample3.selectExpr("transform_keys(x, (k, v) -> if(v, 2 * k, 3 * k))"), - Seq(Row(Map(50 -> true, 78 -> false)))) - - checkAnswer(dfExample3.selectExpr("transform_keys(x, (k, v) -> if(v, 2 * k, 3 * k))"), - Seq(Row(Map(50 -> true, 78 -> false)))) - - checkAnswer(dfExample4.selectExpr("transform_keys(y, (k, v) -> array_contains(k, 3) AND v)"), - Seq(Row(Map(false -> false)))) - } - - // Test with local relation, the Project will be evaluated without codegen - testMapOfPrimitiveTypesCombination() - dfExample1.cache() - dfExample2.cache() - dfExample3.cache() - dfExample4.cache() - // Test with cached relation, the Project will be evaluated with codegen - testMapOfPrimitiveTypesCombination() - } - - test("transform keys function - Invalid lambda functions and exceptions") { - val dfExample1 = Seq( - Map[String, String]("a" -> null) - ).toDF("i") - - val dfExample2 = Seq( - Seq(1, 2, 3, 4) - ).toDF("j") - - val ex1 = intercept[AnalysisException] { - dfExample1.selectExpr("transform_keys(i, k -> k)") - } - assert(ex1.getMessage.contains("The number of lambda function arguments '1' does not match")) - - val ex2 = intercept[AnalysisException] { - dfExample1.selectExpr("transform_keys(i, (k, v, x) -> k + 1)") - } - assert(ex2.getMessage.contains( - "The number of lambda function arguments '3' does not match")) - - val ex3 = intercept[Exception] { - dfExample1.selectExpr("transform_keys(i, (k, v) -> v)").show() - } - assert(ex3.getMessage.contains("Cannot use null as map key!")) - - val ex4 = intercept[AnalysisException] { - dfExample2.selectExpr("transform_keys(j, (k, v) -> k + 1)") - } - assert(ex4.getMessage.contains( - "data type mismatch: argument 1 requires map type")) - } - - test("transform values function - test primitive data types") { - val dfExample1 = Seq( - Map[Int, Int](1 -> 1, 9 -> 9, 8 -> 8, 7 -> 7) - ).toDF("i") - - val dfExample2 = Seq( - Map[Boolean, String](false -> "abc", true -> "def") - ).toDF("x") - - val dfExample3 = Seq( - Map[String, Int]("a" -> 1, "b" -> 2, "c" -> 3) - ).toDF("y") - - val dfExample4 = Seq( - Map[Int, Double](1 -> 1.0, 2 -> 1.40, 3 -> 1.70) - ).toDF("z") - - val dfExample5 = Seq( - Map[Int, Array[Int]](1 -> Array(1, 2)) - ).toDF("c") - - def testMapOfPrimitiveTypesCombination(): Unit = { - checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> k + v)"), - Seq(Row(Map(1 -> 2, 9 -> 18, 8 -> 16, 7 -> 14)))) - - checkAnswer(dfExample2.selectExpr( - "transform_values(x, (k, v) -> if(k, v, CAST(k AS String)))"), - Seq(Row(Map(false -> "false", true -> "def")))) - - checkAnswer(dfExample2.selectExpr("transform_values(x, (k, v) -> NOT k AND v = 'abc')"), - Seq(Row(Map(false -> true, true -> false)))) - - checkAnswer(dfExample3.selectExpr("transform_values(y, (k, v) -> v * v)"), - Seq(Row(Map("a" -> 1, "b" -> 4, "c" -> 9)))) - - checkAnswer(dfExample3.selectExpr( - "transform_values(y, (k, v) -> k || ':' || CAST(v as String))"), - Seq(Row(Map("a" -> "a:1", "b" -> "b:2", "c" -> "c:3")))) - - checkAnswer( - dfExample3.selectExpr("transform_values(y, (k, v) -> concat(k, cast(v as String)))"), - Seq(Row(Map("a" -> "a1", "b" -> "b2", "c" -> "c3")))) - - checkAnswer( - dfExample4.selectExpr( - "transform_values(" + - "z,(k, v) -> map_from_arrays(ARRAY(1, 2, 3), " + - "ARRAY('one', 'two', 'three'))[k] || '_' || CAST(v AS String))"), - Seq(Row(Map(1 -> "one_1.0", 2 -> "two_1.4", 3 ->"three_1.7")))) - - checkAnswer( - dfExample4.selectExpr("transform_values(z, (k, v) -> k-v)"), - Seq(Row(Map(1 -> 0.0, 2 -> 0.6000000000000001, 3 -> 1.3)))) - - checkAnswer( - dfExample5.selectExpr("transform_values(c, (k, v) -> k + cardinality(v))"), - Seq(Row(Map(1 -> 3)))) - } - - // Test with local relation, the Project will be evaluated without codegen - testMapOfPrimitiveTypesCombination() - dfExample1.cache() - dfExample2.cache() - dfExample3.cache() - dfExample4.cache() - dfExample5.cache() - // Test with cached relation, the Project will be evaluated with codegen - testMapOfPrimitiveTypesCombination() - } - - test("transform values function - test empty") { - val dfExample1 = Seq( - Map.empty[Integer, Integer] - ).toDF("i") - - val dfExample2 = Seq( - Map.empty[BigInt, String] - ).toDF("j") - - def testEmpty(): Unit = { - checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> NULL)"), - Seq(Row(Map.empty[Integer, Integer]))) - - checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> k)"), - Seq(Row(Map.empty[Integer, Integer]))) - - checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> v)"), - Seq(Row(Map.empty[Integer, Integer]))) - - checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> 0)"), - Seq(Row(Map.empty[Integer, Integer]))) - - checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> 'value')"), - Seq(Row(Map.empty[Integer, String]))) - - checkAnswer(dfExample1.selectExpr("transform_values(i, (k, v) -> true)"), - Seq(Row(Map.empty[Integer, Boolean]))) - - checkAnswer(dfExample2.selectExpr("transform_values(j, (k, v) -> k + cast(v as BIGINT))"), - Seq(Row(Map.empty[BigInt, BigInt]))) - } - - testEmpty() - dfExample1.cache() - dfExample2.cache() - testEmpty() - } - - test("transform values function - test null values") { - val dfExample1 = Seq( - Map[Int, Integer](1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4) - ).toDF("a") - - val dfExample2 = Seq( - Map[Int, String](1 -> "a", 2 -> "b", 3 -> null) - ).toDF("b") - - def testNullValue(): Unit = { - checkAnswer(dfExample1.selectExpr("transform_values(a, (k, v) -> null)"), - Seq(Row(Map[Int, Integer](1 -> null, 2 -> null, 3 -> null, 4 -> null)))) - - checkAnswer(dfExample2.selectExpr( - "transform_values(b, (k, v) -> IF(v IS NULL, k + 1, k + 2))"), - Seq(Row(Map(1 -> 3, 2 -> 4, 3 -> 4)))) - } - - testNullValue() - dfExample1.cache() - dfExample2.cache() - testNullValue() - } - - test("transform values function - test invalid functions") { - val dfExample1 = Seq( - Map[Int, Int](1 -> 1, 9 -> 9, 8 -> 8, 7 -> 7) - ).toDF("i") - - val dfExample2 = Seq( - Map[String, String]("a" -> "b") - ).toDF("j") - - val dfExample3 = Seq( - Seq(1, 2, 3, 4) - ).toDF("x") - - def testInvalidLambdaFunctions(): Unit = { - - val ex1 = intercept[AnalysisException] { - dfExample1.selectExpr("transform_values(i, k -> k)") - } - assert(ex1.getMessage.contains("The number of lambda function arguments '1' does not match")) - - val ex2 = intercept[AnalysisException] { - dfExample2.selectExpr("transform_values(j, (k, v, x) -> k + 1)") - } - assert(ex2.getMessage.contains("The number of lambda function arguments '3' does not match")) - - val ex3 = intercept[AnalysisException] { - dfExample3.selectExpr("transform_values(x, (k, v) -> k + 1)") - } - assert(ex3.getMessage.contains( - "data type mismatch: argument 1 requires map type")) - } - - testInvalidLambdaFunctions() - dfExample1.cache() - dfExample2.cache() - dfExample3.cache() - testInvalidLambdaFunctions() - } - test("arrays zip_with function - for primitive types") { val df1 = Seq[(Seq[Integer], Seq[Integer])]( (Seq(9001, 9002, 9003), Seq(4, 5, 6)), From 39e108f168128abc5e0f369645b699a87ce6c91b Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Fri, 26 Oct 2018 11:07:55 +0800 Subject: [PATCH 200/879] [SPARK-25793][ML] call SaveLoadV2_0.load for classNameV2_0 ## What changes were proposed in this pull request? The following code in BisectingKMeansModel.load calls the wrong version of load. ``` case (SaveLoadV2_0.thisClassName, SaveLoadV2_0.thisFormatVersion) => val model = SaveLoadV1_0.load(sc, path) ``` Closes #22790 from huaxingao/spark-25793. Authored-by: Huaxin Gao Signed-off-by: Wenchen Fan (cherry picked from commit dc9b320807881403ca9f1e2e6d01de4b52db3975) Signed-off-by: Wenchen Fan --- .../spark/mllib/clustering/BisectingKMeansModel.scala | 6 +++--- .../spark/mllib/clustering/BisectingKMeansSuite.scala | 3 ++- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala index 9d115afcea75..4c5794fbffc8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala @@ -109,10 +109,10 @@ class BisectingKMeansModel private[clustering] ( @Since("2.0.0") override def save(sc: SparkContext, path: String): Unit = { - BisectingKMeansModel.SaveLoadV1_0.save(sc, this, path) + BisectingKMeansModel.SaveLoadV2_0.save(sc, this, path) } - override protected def formatVersion: String = "1.0" + override protected def formatVersion: String = "2.0" } @Since("2.0.0") @@ -126,7 +126,7 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { val model = SaveLoadV1_0.load(sc, path) model case (SaveLoadV2_0.thisClassName, SaveLoadV2_0.thisFormatVersion) => - val model = SaveLoadV1_0.load(sc, path) + val model = SaveLoadV2_0.load(sc, path) model case _ => throw new Exception( s"BisectingKMeansModel.load did not recognize model with (className, format version):" + diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala index 35f7932ae822..4a4d8b5c89de 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala @@ -187,11 +187,12 @@ class BisectingKMeansSuite extends SparkFunSuite with MLlibTestSparkContext { val points = (1 until 8).map(i => Vectors.dense(i)) val data = sc.parallelize(points, 2) - val model = new BisectingKMeans().run(data) + val model = new BisectingKMeans().setDistanceMeasure(DistanceMeasure.COSINE).run(data) try { model.save(sc, path) val sameModel = BisectingKMeansModel.load(sc, path) assert(model.k === sameModel.k) + assert(model.distanceMeasure === sameModel.distanceMeasure) model.clusterCenters.zip(sameModel.clusterCenters).foreach(c => c._1 === c._2) } finally { Utils.deleteRecursively(tempDir) From b739fb0d783adad68e7197caaa931a83eb1725bd Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 25 Oct 2018 20:26:13 -0700 Subject: [PATCH 201/879] [SPARK-25840][BUILD] `make-distribution.sh` should not fail due to missing LICENSE-binary ## What changes were proposed in this pull request? We vote for the artifacts. All releases are in the form of the source materials needed to make changes to the software being released. (http://www.apache.org/legal/release-policy.html#artifacts) From Spark 2.4.0, the source artifact and binary artifact starts to contain own proper LICENSE files (LICENSE, LICENSE-binary). It's great to have them. However, unfortunately, `dev/make-distribution.sh` inside source artifacts start to fail because it expects `LICENSE-binary` and source artifact have only the LICENSE file. https://dist.apache.org/repos/dist/dev/spark/v2.4.0-rc4-bin/spark-2.4.0.tgz `dev/make-distribution.sh` is used during the voting phase because we are voting on that source artifact instead of GitHub repository. Individual contributors usually don't have the downstream repository and starts to try build the voting source artifacts to help the verification for the source artifact during voting phase. (Personally, I did before.) This PR aims to recover that script to work in any way. This doesn't aim for source artifacts to reproduce the compiled artifacts. ## How was this patch tested? Manual. ``` $ rm LICENSE-binary $ dev/make-distribution.sh ``` Closes #22840 from dongjoon-hyun/SPARK-25840. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 79f3babcc6e189d7405464b9ac1eb1c017e51f5d) Signed-off-by: Dongjoon Hyun --- dev/make-distribution.sh | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/dev/make-distribution.sh b/dev/make-distribution.sh index 668682fbb913..84f4ae9a64ff 100755 --- a/dev/make-distribution.sh +++ b/dev/make-distribution.sh @@ -212,9 +212,13 @@ mkdir -p "$DISTDIR/examples/src/main" cp -r "$SPARK_HOME/examples/src/main" "$DISTDIR/examples/src/" # Copy license and ASF files -cp "$SPARK_HOME/LICENSE-binary" "$DISTDIR/LICENSE" -cp -r "$SPARK_HOME/licenses-binary" "$DISTDIR/licenses" -cp "$SPARK_HOME/NOTICE-binary" "$DISTDIR/NOTICE" +if [ -e "$SPARK_HOME/LICENSE-binary" ]; then + cp "$SPARK_HOME/LICENSE-binary" "$DISTDIR/LICENSE" + cp -r "$SPARK_HOME/licenses-binary" "$DISTDIR/licenses" + cp "$SPARK_HOME/NOTICE-binary" "$DISTDIR/NOTICE" +else + echo "Skipping copying LICENSE files" +fi if [ -e "$SPARK_HOME/CHANGES.txt" ]; then cp "$SPARK_HOME/CHANGES.txt" "$DISTDIR" From adfd1057dae3b48c05d7443e3aee23157965e6d1 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 25 Oct 2018 20:37:07 -0700 Subject: [PATCH 202/879] [MINOR][TEST][BRANCH-2.4] Regenerate golden file `datetime.sql.out` ## What changes were proposed in this pull request? `datetime.sql.out` is a generated golden file, but it's a little bit broken during manual [reverting](https://github.com/dongjoon-hyun/spark/commit/5d744499667fcd08825bca0ac6d5d90d6e110ebc#diff-79dd276be45ede6f34e24ad7005b0a7cR87). This doens't cause test failure because the difference is inside `comments` and blank lines. We had better fix this minor issue before RC5. ## How was this patch tested? Pass the Jenkins. Closes #22837 from dongjoon-hyun/fix_datetime_sql_out. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- .../src/test/resources/sql-tests/results/datetime.sql.out | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out index 4e1cfa6e48c1..63aa00426ea3 100644 --- a/sql/core/src/test/resources/sql-tests/results/datetime.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/datetime.sql.out @@ -82,9 +82,10 @@ struct 1 2 2 3 + -- !query 9 select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), weekday('1582-10-15 13:10:15') --- !query 3 schema +-- !query 9 schema struct --- !query 3 output +-- !query 9 output 5 3 5 NULL 4 From eff1c5016cc78ad3d26e5e83cd0d72c56c35cf0a Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 26 Oct 2018 13:53:51 +0900 Subject: [PATCH 203/879] [SPARK-25822][PYSPARK] Fix a race condition when releasing a Python worker ## What changes were proposed in this pull request? There is a race condition when releasing a Python worker. If `ReaderIterator.handleEndOfDataSection` is not running in the task thread, when a task is early terminated (such as `take(N)`), the task completion listener may close the worker but "handleEndOfDataSection" can still put the worker into the worker pool to reuse. https://github.com/zsxwing/spark/commit/0e07b483d2e7c68f3b5c3c118d0bf58c501041b7 is a patch to reproduce this issue. I also found a user reported this in the mail list: http://mail-archives.apache.org/mod_mbox/spark-user/201610.mbox/%3CCAAUq=H+YLUEpd23nwvq13Ms5hOStkhX3ao4f4zQV6sgO5zM-xAmail.gmail.com%3E This PR fixes the issue by using `compareAndSet` to make sure we will never return a closed worker to the work pool. ## How was this patch tested? Jenkins. Closes #22816 from zsxwing/fix-socket-closed. Authored-by: Shixiong Zhu Signed-off-by: Takuya UESHIN (cherry picked from commit 86d469aeaa492c0642db09b27bb0879ead5d7166) Signed-off-by: Takuya UESHIN --- .../spark/api/python/PythonRunner.scala | 21 ++++++++++--------- .../execution/python/ArrowPythonRunner.scala | 4 ++-- .../execution/python/PythonUDFRunner.scala | 4 ++-- 3 files changed, 15 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 6e53a044e9a8..f73e95eac8f7 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -106,15 +106,17 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", memoryMb.get.toString) } val worker: Socket = env.createPythonWorker(pythonExec, envVars.asScala.toMap) - // Whether is the worker released into idle pool - val released = new AtomicBoolean(false) + // Whether is the worker released into idle pool or closed. When any codes try to release or + // close a worker, they should use `releasedOrClosed.compareAndSet` to flip the state to make + // sure there is only one winner that is going to release or close the worker. + val releasedOrClosed = new AtomicBoolean(false) // Start a thread to feed the process input from our parent's iterator val writerThread = newWriterThread(env, worker, inputIterator, partitionIndex, context) context.addTaskCompletionListener[Unit] { _ => writerThread.shutdownOnTaskCompletion() - if (!reuseWorker || !released.get) { + if (!reuseWorker || releasedOrClosed.compareAndSet(false, true)) { try { worker.close() } catch { @@ -131,7 +133,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) val stdoutIterator = newReaderIterator( - stream, writerThread, startTime, env, worker, released, context) + stream, writerThread, startTime, env, worker, releasedOrClosed, context) new InterruptibleIterator(context, stdoutIterator) } @@ -148,7 +150,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( startTime: Long, env: SparkEnv, worker: Socket, - released: AtomicBoolean, + releasedOrClosed: AtomicBoolean, context: TaskContext): Iterator[OUT] /** @@ -392,7 +394,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( startTime: Long, env: SparkEnv, worker: Socket, - released: AtomicBoolean, + releasedOrClosed: AtomicBoolean, context: TaskContext) extends Iterator[OUT] { @@ -463,9 +465,8 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( } // Check whether the worker is ready to be re-used. if (stream.readInt() == SpecialLengths.END_OF_STREAM) { - if (reuseWorker) { + if (reuseWorker && releasedOrClosed.compareAndSet(false, true)) { env.releasePythonWorker(pythonExec, envVars.asScala.toMap, worker) - released.set(true) } } eos = true @@ -565,9 +566,9 @@ private[spark] class PythonRunner(funcs: Seq[ChainedPythonFunctions]) startTime: Long, env: SparkEnv, worker: Socket, - released: AtomicBoolean, + releasedOrClosed: AtomicBoolean, context: TaskContext): Iterator[Array[Byte]] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, released, context) { + new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { protected override def read(): Array[Byte] = { if (writerThread.exception.isDefined) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index 18992d7a9f97..04623b1ab3c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -117,9 +117,9 @@ class ArrowPythonRunner( startTime: Long, env: SparkEnv, worker: Socket, - released: AtomicBoolean, + releasedOrClosed: AtomicBoolean, context: TaskContext): Iterator[ColumnarBatch] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, released, context) { + new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { private val allocator = ArrowUtils.rootAllocator.newChildAllocator( s"stdin reader for $pythonExec", 0, Long.MaxValue) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala index cc61faa7e705..752d271c4cc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala @@ -59,9 +59,9 @@ class PythonUDFRunner( startTime: Long, env: SparkEnv, worker: Socket, - released: AtomicBoolean, + releasedOrClosed: AtomicBoolean, context: TaskContext): Iterator[Array[Byte]] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, released, context) { + new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { protected override def read(): Array[Byte] = { if (writerThread.exception.isDefined) { From f37bceadf2135348c006c3d37ab7d6101cfe2267 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 26 Oct 2018 13:17:24 +0800 Subject: [PATCH 204/879] [SPARK-25842][SQL] Deprecate rangeBetween APIs introduced in SPARK-21608 ## What changes were proposed in this pull request? See the detailed information at https://issues.apache.org/jira/browse/SPARK-25841 on why these APIs should be deprecated and redesigned. This patch also reverts https://github.com/apache/spark/commit/8acb51f08b448628b65e90af3b268994f9550e45 which applies to 2.4. ## How was this patch tested? Only deprecation and doc changes. Closes #22841 from rxin/SPARK-25842. Authored-by: Reynold Xin Signed-off-by: Wenchen Fan (cherry picked from commit 89d748b33c8636a1b1411c505921b0a585e1e6cb) Signed-off-by: Wenchen Fan --- python/pyspark/sql/functions.py | 30 -------- python/pyspark/sql/window.py | 70 +++++-------------- .../apache/spark/sql/expressions/Window.scala | 46 +----------- .../spark/sql/expressions/WindowSpec.scala | 45 +----------- .../org/apache/spark/sql/functions.scala | 12 ++-- 5 files changed, 28 insertions(+), 175 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 785e55e2a1eb..9485c286eba3 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -855,36 +855,6 @@ def ntile(n): return Column(sc._jvm.functions.ntile(int(n))) -@since(2.4) -def unboundedPreceding(): - """ - Window function: returns the special frame boundary that represents the first row - in the window partition. - """ - sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.unboundedPreceding()) - - -@since(2.4) -def unboundedFollowing(): - """ - Window function: returns the special frame boundary that represents the last row - in the window partition. - """ - sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.unboundedFollowing()) - - -@since(2.4) -def currentRow(): - """ - Window function: returns the special frame boundary that represents the current row - in the window partition. - """ - sc = SparkContext._active_spark_context - return Column(sc._jvm.functions.currentRow()) - - # ---------------------- Date/Timestamp functions ------------------------------ @since(1.5) diff --git a/python/pyspark/sql/window.py b/python/pyspark/sql/window.py index d19ced954f04..e76563dfaa9c 100644 --- a/python/pyspark/sql/window.py +++ b/python/pyspark/sql/window.py @@ -16,11 +16,9 @@ # import sys -if sys.version >= '3': - long = int from pyspark import since, SparkContext -from pyspark.sql.column import Column, _to_seq, _to_java_column +from pyspark.sql.column import _to_seq, _to_java_column __all__ = ["Window", "WindowSpec"] @@ -126,45 +124,20 @@ def rangeBetween(start, end): and "5" means the five off after the current row. We recommend users use ``Window.unboundedPreceding``, ``Window.unboundedFollowing``, - ``Window.currentRow``, ``pyspark.sql.functions.unboundedPreceding``, - ``pyspark.sql.functions.unboundedFollowing`` and ``pyspark.sql.functions.currentRow`` - to specify special boundary values, rather than using integral values directly. + and ``Window.currentRow`` to specify special boundary values, rather than using integral + values directly. :param start: boundary start, inclusive. - The frame is unbounded if this is ``Window.unboundedPreceding``, - a column returned by ``pyspark.sql.functions.unboundedPreceding``, or + The frame is unbounded if this is ``Window.unboundedPreceding``, or any value less than or equal to max(-sys.maxsize, -9223372036854775808). :param end: boundary end, inclusive. - The frame is unbounded if this is ``Window.unboundedFollowing``, - a column returned by ``pyspark.sql.functions.unboundedFollowing``, or + The frame is unbounded if this is ``Window.unboundedFollowing``, or any value greater than or equal to min(sys.maxsize, 9223372036854775807). - - >>> from pyspark.sql import functions as F, SparkSession, Window - >>> spark = SparkSession.builder.getOrCreate() - >>> df = spark.createDataFrame( - ... [(1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")], ["id", "category"]) - >>> window = Window.orderBy("id").partitionBy("category").rangeBetween( - ... F.currentRow(), F.lit(1)) - >>> df.withColumn("sum", F.sum("id").over(window)).show() - +---+--------+---+ - | id|category|sum| - +---+--------+---+ - | 1| b| 3| - | 2| b| 5| - | 3| b| 3| - | 1| a| 4| - | 1| a| 4| - | 2| a| 2| - +---+--------+---+ """ - if isinstance(start, (int, long)) and isinstance(end, (int, long)): - if start <= Window._PRECEDING_THRESHOLD: - start = Window.unboundedPreceding - if end >= Window._FOLLOWING_THRESHOLD: - end = Window.unboundedFollowing - elif isinstance(start, Column) and isinstance(end, Column): - start = start._jc - end = end._jc + if start <= Window._PRECEDING_THRESHOLD: + start = Window.unboundedPreceding + if end >= Window._FOLLOWING_THRESHOLD: + end = Window.unboundedFollowing sc = SparkContext._active_spark_context jspec = sc._jvm.org.apache.spark.sql.expressions.Window.rangeBetween(start, end) return WindowSpec(jspec) @@ -239,34 +212,27 @@ def rangeBetween(self, start, end): and "5" means the five off after the current row. We recommend users use ``Window.unboundedPreceding``, ``Window.unboundedFollowing``, - ``Window.currentRow``, ``pyspark.sql.functions.unboundedPreceding``, - ``pyspark.sql.functions.unboundedFollowing`` and ``pyspark.sql.functions.currentRow`` - to specify special boundary values, rather than using integral values directly. + and ``Window.currentRow`` to specify special boundary values, rather than using integral + values directly. :param start: boundary start, inclusive. - The frame is unbounded if this is ``Window.unboundedPreceding``, - a column returned by ``pyspark.sql.functions.unboundedPreceding``, or + The frame is unbounded if this is ``Window.unboundedPreceding``, or any value less than or equal to max(-sys.maxsize, -9223372036854775808). :param end: boundary end, inclusive. - The frame is unbounded if this is ``Window.unboundedFollowing``, - a column returned by ``pyspark.sql.functions.unboundedFollowing``, or + The frame is unbounded if this is ``Window.unboundedFollowing``, or any value greater than or equal to min(sys.maxsize, 9223372036854775807). """ - if isinstance(start, (int, long)) and isinstance(end, (int, long)): - if start <= Window._PRECEDING_THRESHOLD: - start = Window.unboundedPreceding - if end >= Window._FOLLOWING_THRESHOLD: - end = Window.unboundedFollowing - elif isinstance(start, Column) and isinstance(end, Column): - start = start._jc - end = end._jc + if start <= Window._PRECEDING_THRESHOLD: + start = Window.unboundedPreceding + if end >= Window._FOLLOWING_THRESHOLD: + end = Window.unboundedFollowing return WindowSpec(self._jspec.rangeBetween(start, end)) def _test(): import doctest SparkContext('local[4]', 'PythonTest') - (failure_count, test_count) = doctest.testmod(optionflags=doctest.NORMALIZE_WHITESPACE) + (failure_count, test_count) = doctest.testmod() if failure_count: sys.exit(-1) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index cd819bab1b14..14dec8f0810f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -215,52 +215,10 @@ object Window { } /** - * Creates a [[WindowSpec]] with the frame boundaries defined, - * from `start` (inclusive) to `end` (inclusive). - * - * Both `start` and `end` are relative to the current row. For example, "lit(0)" means - * "current row", while "lit(-1)" means one off before the current row, and "lit(5)" means the - * five off after the current row. - * - * Users should use `unboundedPreceding()`, `unboundedFollowing()`, and `currentRow()` from - * [[org.apache.spark.sql.functions]] to specify special boundary values, literals are not - * transformed to [[org.apache.spark.sql.catalyst.expressions.SpecialFrameBoundary]]s. - * - * A range-based boundary is based on the actual value of the ORDER BY - * expression(s). An offset is used to alter the value of the ORDER BY expression, for - * instance if the current order by expression has a value of 10 and the lower bound offset - * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a - * number of constraints on the ORDER BY expressions: there can be only one expression and this - * expression must have a numerical/date/timestamp data type. An exception can be made when the - * offset is unbounded, because no value modification is needed, in this case multiple and - * non-numerical/date/timestamp data type ORDER BY expression are allowed. - * - * {{{ - * import org.apache.spark.sql.expressions.Window - * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) - * .toDF("id", "category") - * val byCategoryOrderedById = - * Window.partitionBy('category).orderBy('id).rangeBetween(currentRow(), lit(1)) - * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() - * - * +---+--------+---+ - * | id|category|sum| - * +---+--------+---+ - * | 1| b| 3| - * | 2| b| 5| - * | 3| b| 3| - * | 1| a| 4| - * | 1| a| 4| - * | 2| a| 2| - * +---+--------+---+ - * }}} - * - * @param start boundary start, inclusive. The frame is unbounded if the expression is - * [[org.apache.spark.sql.catalyst.expressions.UnboundedPreceding]]. - * @param end boundary end, inclusive. The frame is unbounded if the expression is - * [[org.apache.spark.sql.catalyst.expressions.UnboundedFollowing]]. + * This function has been deprecated in Spark 2.4. See SPARK-25842 for more information. * @since 2.3.0 */ + @deprecated("Use the version with Long parameter types", "2.4.0") def rangeBetween(start: Column, end: Column): WindowSpec = { spec.rangeBetween(start, end) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala index 4c41aa3c5fb6..0cc43a58237d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala @@ -210,51 +210,10 @@ class WindowSpec private[sql]( } /** - * Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). - * - * Both `start` and `end` are relative to the current row. For example, "lit(0)" means - * "current row", while "lit(-1)" means one off before the current row, and "lit(5)" means the - * five off after the current row. - * - * Users should use `unboundedPreceding()`, `unboundedFollowing()`, and `currentRow()` from - * [[org.apache.spark.sql.functions]] to specify special boundary values, literals are not - * transformed to [[org.apache.spark.sql.catalyst.expressions.SpecialFrameBoundary]]s. - * - * A range-based boundary is based on the actual value of the ORDER BY - * expression(s). An offset is used to alter the value of the ORDER BY expression, for - * instance if the current order by expression has a value of 10 and the lower bound offset - * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a - * number of constraints on the ORDER BY expressions: there can be only one expression and this - * expression must have a numerical/date/timestamp data type. An exception can be made when the - * offset is unbounded, because no value modification is needed, in this case multiple and - * non-numerical/date/timestamp data type ORDER BY expression are allowed. - * - * {{{ - * import org.apache.spark.sql.expressions.Window - * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) - * .toDF("id", "category") - * val byCategoryOrderedById = - * Window.partitionBy('category).orderBy('id).rangeBetween(currentRow(), lit(1)) - * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() - * - * +---+--------+---+ - * | id|category|sum| - * +---+--------+---+ - * | 1| b| 3| - * | 2| b| 5| - * | 3| b| 3| - * | 1| a| 4| - * | 1| a| 4| - * | 2| a| 2| - * +---+--------+---+ - * }}} - * - * @param start boundary start, inclusive. The frame is unbounded if the expression is - * [[org.apache.spark.sql.catalyst.expressions.UnboundedPreceding]]. - * @param end boundary end, inclusive. The frame is unbounded if the expression is - * [[org.apache.spark.sql.catalyst.expressions.UnboundedFollowing]]. + * This function has been deprecated in Spark 2.4. See SPARK-25842 for more information. * @since 2.3.0 */ + @deprecated("Use the version with Long parameter types", "2.4.0") def rangeBetween(start: Column, end: Column): WindowSpec = { new WindowSpec( partitionSpec, 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 5dedc9db0a61..9c4ad48f267b 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 @@ -830,30 +830,30 @@ object functions { // Window functions ////////////////////////////////////////////////////////////////////////////////////////////// /** - * Window function: returns the special frame boundary that represents the first row in the - * window partition. + * This function has been deprecated in Spark 2.4. See SPARK-25842 for more information. * * @group window_funcs * @since 2.3.0 */ + @deprecated("Use Window.unboundedPreceding", "2.4.0") def unboundedPreceding(): Column = Column(UnboundedPreceding) /** - * Window function: returns the special frame boundary that represents the last row in the - * window partition. + * This function has been deprecated in Spark 2.4. See SPARK-25842 for more information. * * @group window_funcs * @since 2.3.0 */ + @deprecated("Use Window.unboundedFollowing", "2.4.0") def unboundedFollowing(): Column = Column(UnboundedFollowing) /** - * Window function: returns the special frame boundary that represents the current row in the - * window partition. + * This function has been deprecated in Spark 2.4. See SPARK-25842 for more information. * * @group window_funcs * @since 2.3.0 */ + @deprecated("Use Window.currentRow", "2.4.0") def currentRow(): Column = Column(CurrentRow) /** From b47b8271df76a8579e61b27ea25b6de8bbe3649b Mon Sep 17 00:00:00 2001 From: seancxmao Date: Fri, 26 Oct 2018 18:53:55 +0800 Subject: [PATCH 205/879] [SPARK-25797][SQL][DOCS] Add migration doc for solving issues caused by view canonicalization approach change ## What changes were proposed in this pull request? Since Spark 2.2, view definitions are stored in a different way from prior versions. This may cause Spark unable to read views created by prior versions. See [SPARK-25797](https://issues.apache.org/jira/browse/SPARK-25797) for more details. Basically, we have 2 options. 1) Make Spark 2.2+ able to get older view definitions back. Since the expanded text is buggy and unusable, we have to use original text (this is possible with [SPARK-25459](https://issues.apache.org/jira/browse/SPARK-25459)). However, because older Spark versions don't save the context for the database, we cannot always get correct view definitions without view default database. 2) Recreate the views by `ALTER VIEW AS` or `CREATE OR REPLACE VIEW AS`. This PR aims to add migration doc to help users troubleshoot this issue by above option 2. ## How was this patch tested? N/A. Docs are generated and checked locally ``` cd docs SKIP_API=1 jekyll serve --watch ``` Closes #22846 from seancxmao/SPARK-25797. Authored-by: seancxmao Signed-off-by: Wenchen Fan (cherry picked from commit 6fd5ff3951ed9ac7c0b20f2666d8bc39929bfb5c) Signed-off-by: Wenchen Fan --- docs/sql-migration-guide-upgrade.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index af561f286c70..9a7f5b6434dd 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -296,6 +296,8 @@ displayTitle: Spark SQL Upgrading Guide - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). + - Since Spark 2.2, view definitions are stored in a different way from prior versions. This may cause Spark unable to read views created by prior versions. In such cases, you need to recreate the views using `ALTER VIEW AS` or `CREATE OR REPLACE VIEW AS` with newer Spark versions. + ## Upgrading From Spark SQL 2.0 to 2.1 - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. From 26e1d3ef8223e8caca32d42060212dd12dad6d64 Mon Sep 17 00:00:00 2001 From: Stavros Kontopoulos Date: Fri, 26 Oct 2018 08:49:27 -0500 Subject: [PATCH 206/879] [SPARK-25835][K8S] Create kubernetes-tests profile and use the detected SCALA_VERSION - Fixes the scala version propagation issue. - Disables the tests under the k8s profile, now we will run them manually. Adds a test specific profile otherwise tests will not run if we just remove the module from the kubernetes profile (quickest solution I can think of). Manually by running the tests with different versions of scala. Closes #22838 from skonto/propagate-scala2.12. Authored-by: Stavros Kontopoulos Signed-off-by: Sean Owen (cherry picked from commit 7d44bc26408b2189804fd305797afcefb7b2b0e0) Signed-off-by: Sean Owen --- pom.xml | 7 +++++++ .../integration-tests/dev/dev-run-integration-tests.sh | 3 ++- 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 28804be0301c..349de835bfc9 100644 --- a/pom.xml +++ b/pom.xml @@ -2716,6 +2716,13 @@ kubernetes resource-managers/kubernetes/core + + + + + + kubernetes-integration-tests + resource-managers/kubernetes/integration-tests diff --git a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh index b28b8b82ca01..cb5cf693d52d 100755 --- a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh +++ b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh @@ -28,6 +28,7 @@ NAMESPACE= SERVICE_ACCOUNT= INCLUDE_TAGS="k8s" EXCLUDE_TAGS= +SCALA_VERSION="$($TEST_ROOT_DIR/build/mvn org.apache.maven.plugins:maven-help-plugin:2.1.1:evaluate -Dexpression=scala.binary.version | grep -v '\[' )" # Parse arguments while (( "$#" )); do @@ -103,4 +104,4 @@ then properties=( ${properties[@]} -Dtest.exclude.tags=$EXCLUDE_TAGS ) fi -$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pkubernetes -Phadoop-2.7 ${properties[@]} +$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pscala-$SCALA_VERSION -Pkubernetes -Pkubernetes-integration-tests -Phadoop-2.7 ${properties[@]} From 40ed093b7a8122afdcc8f2fc83bff45ca67a60e1 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Fri, 26 Oct 2018 22:14:43 +0800 Subject: [PATCH 207/879] [SPARK-24709][SQL][FOLLOW-UP] Make schema_of_json's input json as literal only The main purpose of `schema_of_json` is the usage of combination with `from_json` (to make up the leak of schema inference) which takes its schema only as literal; however, currently `schema_of_json` allows JSON input as non-literal expressions (e.g, column). This was mistakenly allowed - we don't have to take other usages rather then the main purpose into account for now. This PR makes a followup to only allow literals for `schema_of_json`'s JSON input. We can allow non literal expressions later when it's needed or there are some usecase for it. Unit tests were added. Closes #22775 from HyukjinKwon/SPARK-25447-followup. Lead-authored-by: hyukjinkwon Co-authored-by: Hyukjin Kwon Signed-off-by: Wenchen Fan (cherry picked from commit 33e337c1180a12edf1ae97f0221e389f23192461) Signed-off-by: Wenchen Fan --- python/pyspark/sql/functions.py | 22 ++++++------ .../expressions/jsonExpressions.scala | 21 ++++++++--- .../org/apache/spark/sql/functions.scala | 16 +++++++-- .../sql-tests/inputs/json-functions.sql | 5 +++ .../sql-tests/results/json-functions.sql.out | 36 ++++++++++++++++++- .../apache/spark/sql/JsonFunctionsSuite.scala | 2 +- 6 files changed, 83 insertions(+), 19 deletions(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 9485c286eba3..a59d5c9b8662 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -2316,23 +2316,25 @@ def to_json(col, options={}): @ignore_unicode_prefix @since(2.4) -def schema_of_json(col): +def schema_of_json(json): """ - Parses a column containing a JSON string and infers its schema in DDL format. + Parses a JSON string and infers its schema in DDL format. - :param col: string column in json format + :param json: a JSON string or a string literal containing a JSON string. - >>> from pyspark.sql.types import * - >>> data = [(1, '{"a": 1}')] - >>> df = spark.createDataFrame(data, ("key", "value")) - >>> df.select(schema_of_json(df.value).alias("json")).collect() - [Row(json=u'struct')] - >>> df.select(schema_of_json(lit('{"a": 0}')).alias("json")).collect() + >>> df = spark.range(1) + >>> df.select(schema_of_json('{"a": 0}').alias("json")).collect() [Row(json=u'struct')] """ + if isinstance(json, basestring): + col = _create_column_from_literal(json) + elif isinstance(json, Column): + col = _to_java_column(json) + else: + raise TypeError("schema argument should be a column or string") sc = SparkContext._active_spark_context - jc = sc._jvm.functions.schema_of_json(_to_java_column(col)) + jc = sc._jvm.functions.schema_of_json(col) return Column(jc) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala index bd9090a07471..6650e45eddcc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -744,14 +744,27 @@ case class StructsToJson( """, since = "2.4.0") case class SchemaOfJson(child: Expression) - extends UnaryExpression with String2StringExpression with CodegenFallback { + extends UnaryExpression with CodegenFallback { + + override def dataType: DataType = StringType + + override def nullable: Boolean = false private val jsonOptions = new JSONOptions(Map.empty, "UTC") private val jsonFactory = new JsonFactory() jsonOptions.setJacksonOptions(jsonFactory) - override def convert(v: UTF8String): UTF8String = { - val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, v)) { parser => + @transient + private lazy val json = child.eval().asInstanceOf[UTF8String] + + override def checkInputDataTypes(): TypeCheckResult = child match { + case Literal(s, StringType) if s != null => super.checkInputDataTypes() + case _ => TypeCheckResult.TypeCheckFailure( + s"The input json should be a string literal and not null; however, got ${child.sql}.") + } + + override def eval(v: InternalRow): Any = { + val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser => parser.nextToken() inferField(parser, jsonOptions) } @@ -765,7 +778,7 @@ object JsonExprUtils { def evalSchemaExpr(exp: Expression): DataType = exp match { case Literal(s, StringType) => DataType.fromDDL(s.toString) case e @ SchemaOfJson(_: Literal) => - val ddlSchema = e.eval().asInstanceOf[UTF8String] + val ddlSchema = e.eval(EmptyRow).asInstanceOf[UTF8String] DataType.fromDDL(ddlSchema.toString) case e => throw new AnalysisException( "Schema should be specified in DDL format as a string literal" + 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 9c4ad48f267b..ac34ba684af8 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 @@ -3602,14 +3602,24 @@ object functions { } /** - * Parses a column containing a JSON string and infers its schema. + * Parses a JSON string and infers its schema in DDL format. * - * @param e a string column containing JSON data. + * @param json a JSON string. + * + * @group collection_funcs + * @since 2.4.0 + */ + def schema_of_json(json: String): Column = schema_of_json(lit(json)) + + /** + * Parses a JSON string and infers its schema in DDL format. + * + * @param json a string literal containing a JSON string. * * @group collection_funcs * @since 2.4.0 */ - def schema_of_json(e: Column): Column = withExpr(new SchemaOfJson(e.expr)) + def schema_of_json(json: Column): Column = withExpr(new SchemaOfJson(json.expr)) /** * (Scala-specific) Converts a column containing a `StructType`, `ArrayType` or diff --git a/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql index 0f22c0eeed58..e391e93f9c68 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/json-functions.sql @@ -56,3 +56,8 @@ select from_json('[{"a": 1}, 2]', 'array>'); select to_json(array('1', '2', '3')); select to_json(array(array(1, 2, 3), array(4))); +select schema_of_json(null); +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a'); +SELECT schema_of_json(jsonField) FROM jsonTable; +-- Clean up +DROP VIEW IF EXISTS jsonTable; diff --git a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out index e550b43e08c2..94c1c9777cc5 100644 --- a/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/json-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 40 +-- Number of queries: 44 -- !query 0 @@ -370,3 +370,37 @@ select to_json(array(array(1, 2, 3), array(4))) struct -- !query 39 output [[1,2,3],[4]] + + +-- !query 40 +select schema_of_json(null) +-- !query 40 schema +struct<> +-- !query 40 output +org.apache.spark.sql.AnalysisException +cannot resolve 'schemaofjson(NULL)' due to data type mismatch: The input json should be a string literal and not null; however, got NULL.; line 1 pos 7 + + +-- !query 41 +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a') +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +SELECT schema_of_json(jsonField) FROM jsonTable +-- !query 42 schema +struct<> +-- !query 42 output +org.apache.spark.sql.AnalysisException +cannot resolve 'schemaofjson(jsontable.`jsonField`)' due to data type mismatch: The input json should be a string literal and not null; however, got jsontable.`jsonField`.; line 1 pos 7 + + +-- !query 43 +DROP VIEW IF EXISTS jsonTable +-- !query 43 schema +struct<> +-- !query 43 output + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index fe4bf15fa392..53ae1e0249e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala @@ -391,7 +391,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { test("SPARK-24709: infers schemas of json strings and pass them to from_json") { val in = Seq("""{"a": [1, 2, 3]}""").toDS() - val out = in.select(from_json('value, schema_of_json(lit("""{"a": [1]}"""))) as "parsed") + val out = in.select(from_json('value, schema_of_json("""{"a": [1]}""")) as "parsed") val expected = StructType(StructField( "parsed", StructType(StructField( From 075447b3965489ffba4e6afb2b120880bc307505 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 26 Oct 2018 16:26:31 +0000 Subject: [PATCH 208/879] Preparing Spark release v2.4.0-rc5 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 714b6f18b19d..f52d785e05cd 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.0 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index ee0de73f21d5..63ab510eb683 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b89e0fed2a66..b10e11849a74 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 3105a8182afc..74c6d232b36a 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f85248b53080..fbdc979d120d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 3350990c0461..53d1b2b58f1f 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 9a2cb90a2f30..98145481adc7 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 9b927a9ba525..f0c8ebe25e63 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 718875f88e74..c635cd3b9394 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 583a2fff7638..ed3efffa180b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 72473771c2e9..7f9a91e2fba0 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1-SNAPSHOT -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.0 +SPARK_VERSION_SHORT: 2.4.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 737ebfd7900c..c7b8354b0968 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 7c6b78ac2b31..a91c13362caa 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 6343a15c43a8..db239f23e1de 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 2f4886832b75..cf19fed1bef5 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9404e2a52d90..da5dba82f3cf 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 73dbc8d99449..e591ce802256 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 912d82b764e5..e8b13cdc2bef 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 8454f5c108dc..c3d3b888656b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 05f7cf870af8..468ace0ff8d7 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 0e935ac112af..db8fda66c3cd 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 7977389718cb..aa1e1267f57b 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7193df69900d..7d07e18f04a2 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 4b318b00ff5b..f545f212b2f0 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6fbb1b7db6aa..6b388ffcf18b 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 9a42d9a69967..fc369c73edc0 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 702f60ab9f00..f4cb94147d9e 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 62322da714bd..22f60efae9af 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index c8e45b27c78f..b17e67d78b98 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 6201e0dd3c9a..bf0d406da9ed 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 349de835bfc9..f0e5ed9c563c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 87282141fa64..53e3424394ba 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1.dev0" +__version__ = "2.4.0" diff --git a/repl/pom.xml b/repl/pom.xml index d09548348eb5..4532f0b8686e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index a9d9c8de9959..47fe3f2765b2 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 793167103179..f59895e9a1bd 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index e6457289b859..0eacbbfac2b9 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index e70711ce7257..2afdc15be446 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 65cf33f32972..ef1d012d2f80 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c327b85ee866..f4be074642ae 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d9d742444e7e..e5c1065451f7 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c428be67aa3d..b522bfeac130 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 37e96b807653..8e71adabb3ec 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index dd00c2f6cd2c..c00e1b851ba0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml From d868dc2b819da75cdb16fee6d5779f9d1e575f87 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 26 Oct 2018 16:26:36 +0000 Subject: [PATCH 209/879] Preparing development version 2.4.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index f52d785e05cd..714b6f18b19d 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.0 +Version: 2.4.1 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 63ab510eb683..ee0de73f21d5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b10e11849a74..b89e0fed2a66 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 74c6d232b36a..3105a8182afc 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index fbdc979d120d..f85248b53080 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 53d1b2b58f1f..3350990c0461 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 98145481adc7..9a2cb90a2f30 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index f0c8ebe25e63..9b927a9ba525 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index c635cd3b9394..718875f88e74 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ed3efffa180b..583a2fff7638 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7f9a91e2fba0..72473771c2e9 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.0 -SPARK_VERSION_SHORT: 2.4.0 +SPARK_VERSION: 2.4.1-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index c7b8354b0968..737ebfd7900c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index a91c13362caa..7c6b78ac2b31 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index db239f23e1de..6343a15c43a8 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index cf19fed1bef5..2f4886832b75 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index da5dba82f3cf..9404e2a52d90 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index e591ce802256..73dbc8d99449 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index e8b13cdc2bef..912d82b764e5 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index c3d3b888656b..8454f5c108dc 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 468ace0ff8d7..05f7cf870af8 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index db8fda66c3cd..0e935ac112af 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index aa1e1267f57b..7977389718cb 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7d07e18f04a2..7193df69900d 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f545f212b2f0..4b318b00ff5b 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6b388ffcf18b..6fbb1b7db6aa 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index fc369c73edc0..9a42d9a69967 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index f4cb94147d9e..702f60ab9f00 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 22f60efae9af..62322da714bd 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index b17e67d78b98..c8e45b27c78f 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bf0d406da9ed..6201e0dd3c9a 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index f0e5ed9c563c..349de835bfc9 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 53e3424394ba..87282141fa64 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.0" +__version__ = "2.4.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 4532f0b8686e..d09548348eb5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 47fe3f2765b2..a9d9c8de9959 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index f59895e9a1bd..793167103179 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 0eacbbfac2b9..e6457289b859 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 2afdc15be446..e70711ce7257 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ef1d012d2f80..65cf33f32972 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f4be074642ae..c327b85ee866 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index e5c1065451f7..d9d742444e7e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b522bfeac130..c428be67aa3d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 8e71adabb3ec..37e96b807653 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c00e1b851ba0..dd00c2f6cd2c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml From 1757a603fa123e3a81a7bfc06f9b58ee328f11b0 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 27 Oct 2018 00:43:16 +0800 Subject: [PATCH 210/879] HOT-FIX pyspark import --- python/pyspark/sql/functions.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index a59d5c9b8662..9583a9859bfd 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -27,7 +27,7 @@ from pyspark import since, SparkContext from pyspark.rdd import ignore_unicode_prefix, PythonEvalType -from pyspark.sql.column import Column, _to_java_column, _to_seq +from pyspark.sql.column import Column, _to_java_column, _to_seq, _create_column_from_literal from pyspark.sql.dataframe import DataFrame from pyspark.sql.types import StringType, DataType # Keep UserDefinedFunction import for backwards compatible import; moved in SPARK-22409 From 4a7ead480ac8ddb07e34e9ff5360b0c07973c95e Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 26 Oct 2018 16:47:00 +0000 Subject: [PATCH 211/879] Preparing Spark release v2.4.0-rc5 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 714b6f18b19d..f52d785e05cd 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.0 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index ee0de73f21d5..63ab510eb683 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b89e0fed2a66..b10e11849a74 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 3105a8182afc..74c6d232b36a 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f85248b53080..fbdc979d120d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 3350990c0461..53d1b2b58f1f 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 9a2cb90a2f30..98145481adc7 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 9b927a9ba525..f0c8ebe25e63 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 718875f88e74..c635cd3b9394 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 583a2fff7638..ed3efffa180b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 72473771c2e9..7f9a91e2fba0 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1-SNAPSHOT -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.0 +SPARK_VERSION_SHORT: 2.4.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 737ebfd7900c..c7b8354b0968 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 7c6b78ac2b31..a91c13362caa 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 6343a15c43a8..db239f23e1de 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 2f4886832b75..cf19fed1bef5 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9404e2a52d90..da5dba82f3cf 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 73dbc8d99449..e591ce802256 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 912d82b764e5..e8b13cdc2bef 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 8454f5c108dc..c3d3b888656b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 05f7cf870af8..468ace0ff8d7 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 0e935ac112af..db8fda66c3cd 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 7977389718cb..aa1e1267f57b 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7193df69900d..7d07e18f04a2 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 4b318b00ff5b..f545f212b2f0 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6fbb1b7db6aa..6b388ffcf18b 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 9a42d9a69967..fc369c73edc0 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 702f60ab9f00..f4cb94147d9e 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 62322da714bd..22f60efae9af 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index c8e45b27c78f..b17e67d78b98 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 6201e0dd3c9a..bf0d406da9ed 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 349de835bfc9..f0e5ed9c563c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 87282141fa64..53e3424394ba 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1.dev0" +__version__ = "2.4.0" diff --git a/repl/pom.xml b/repl/pom.xml index d09548348eb5..4532f0b8686e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index a9d9c8de9959..47fe3f2765b2 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 793167103179..f59895e9a1bd 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index e6457289b859..0eacbbfac2b9 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index e70711ce7257..2afdc15be446 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 65cf33f32972..ef1d012d2f80 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c327b85ee866..f4be074642ae 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d9d742444e7e..e5c1065451f7 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c428be67aa3d..b522bfeac130 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 37e96b807653..8e71adabb3ec 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index dd00c2f6cd2c..c00e1b851ba0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml From cb2827d286fe2cd8026d95ae1ae55c16c6331699 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Fri, 26 Oct 2018 16:47:05 +0000 Subject: [PATCH 212/879] Preparing development version 2.4.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index f52d785e05cd..714b6f18b19d 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.0 +Version: 2.4.1 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 63ab510eb683..ee0de73f21d5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b10e11849a74..b89e0fed2a66 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 74c6d232b36a..3105a8182afc 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index fbdc979d120d..f85248b53080 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 53d1b2b58f1f..3350990c0461 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 98145481adc7..9a2cb90a2f30 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index f0c8ebe25e63..9b927a9ba525 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index c635cd3b9394..718875f88e74 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ed3efffa180b..583a2fff7638 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7f9a91e2fba0..72473771c2e9 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.0 -SPARK_VERSION_SHORT: 2.4.0 +SPARK_VERSION: 2.4.1-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index c7b8354b0968..737ebfd7900c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index a91c13362caa..7c6b78ac2b31 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index db239f23e1de..6343a15c43a8 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index cf19fed1bef5..2f4886832b75 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index da5dba82f3cf..9404e2a52d90 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index e591ce802256..73dbc8d99449 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index e8b13cdc2bef..912d82b764e5 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index c3d3b888656b..8454f5c108dc 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 468ace0ff8d7..05f7cf870af8 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index db8fda66c3cd..0e935ac112af 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index aa1e1267f57b..7977389718cb 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7d07e18f04a2..7193df69900d 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f545f212b2f0..4b318b00ff5b 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6b388ffcf18b..6fbb1b7db6aa 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index fc369c73edc0..9a42d9a69967 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index f4cb94147d9e..702f60ab9f00 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 22f60efae9af..62322da714bd 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index b17e67d78b98..c8e45b27c78f 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bf0d406da9ed..6201e0dd3c9a 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index f0e5ed9c563c..349de835bfc9 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 53e3424394ba..87282141fa64 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.0" +__version__ = "2.4.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 4532f0b8686e..d09548348eb5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 47fe3f2765b2..a9d9c8de9959 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index f59895e9a1bd..793167103179 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 0eacbbfac2b9..e6457289b859 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 2afdc15be446..e70711ce7257 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ef1d012d2f80..65cf33f32972 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f4be074642ae..c327b85ee866 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index e5c1065451f7..d9d742444e7e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b522bfeac130..c428be67aa3d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 8e71adabb3ec..37e96b807653 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c00e1b851ba0..dd00c2f6cd2c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml From 313a1f0a7aa325ea4038530fc12fad695c7d9809 Mon Sep 17 00:00:00 2001 From: shane knapp Date: Fri, 26 Oct 2018 16:37:36 -0500 Subject: [PATCH 213/879] [SPARK-25854][BUILD] fix `build/mvn` not to fail during Zinc server shutdown ## What changes were proposed in this pull request? the final line in the mvn helper script in build/ attempts to shut down the zinc server. due to the zinc server being set up w/a 30min timeout, by the time the mvn test instantiation finishes, the server times out. this means that when the mvn script tries to shut down zinc, it returns w/an exit code of 1. this will then automatically fail the entire build (even if the build passes). ## How was this patch tested? i set up a test build: https://amplab.cs.berkeley.edu/jenkins/job/sknapp-testing-spark-branch-2.4-test-maven-hadoop-2.7/ Closes #22854 from shaneknapp/fix-mvn-helper-script. Authored-by: shane knapp Signed-off-by: Sean Owen (cherry picked from commit 6aa506394958bfb30cd2a9085a5e8e8be927de51) Signed-off-by: Sean Owen --- build/mvn | 10 +++++++--- 1 file changed, 7 insertions(+), 3 deletions(-) diff --git a/build/mvn b/build/mvn index b60ea644b262..3816993b4e5c 100755 --- a/build/mvn +++ b/build/mvn @@ -153,7 +153,7 @@ if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`"${ZINC_BIN}" -status -port ${ZINC_PORT}` export ZINC_OPTS=${ZINC_OPTS:-"$_COMPILE_JVM_OPTS"} "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} "${ZINC_BIN}" -start -port ${ZINC_PORT} \ - -server 127.0.0.1 -idle-timeout 30m \ + -server 127.0.0.1 -idle-timeout 3h \ -scala-compiler "${SCALA_COMPILER}" \ -scala-library "${SCALA_LIBRARY}" &>/dev/null fi @@ -163,8 +163,12 @@ export MAVEN_OPTS=${MAVEN_OPTS:-"$_COMPILE_JVM_OPTS"} echo "Using \`mvn\` from path: $MVN_BIN" 1>&2 -# Last, call the `mvn` command as usual +# call the `mvn` command as usual +# SPARK-25854 "${MVN_BIN}" -DzincPort=${ZINC_PORT} "$@" +MVN_RETCODE=$? -# Try to shut down zinc explicitly +# Try to shut down zinc explicitly if the server is still running. "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} + +exit $MVN_RETCODE From f575616db69bcca9edf2a3dfe234003ab4c71b30 Mon Sep 17 00:00:00 2001 From: Huaxin Gao Date: Sat, 27 Oct 2018 15:14:29 -0700 Subject: [PATCH 214/879] [SPARK-25859][ML] add scala/java/python example and doc for PrefixSpan ## What changes were proposed in this pull request? add scala/java/python example and doc for PrefixSpan in branch 2.4 ## How was this patch tested? Manually tested Author: Huaxin Gao Closes #22863 from huaxingao/mydocbranch. --- docs/ml-frequent-pattern-mining.md | 46 +++++++++++++ .../examples/ml/JavaPrefixSpanExample.java | 68 +++++++++++++++++++ .../src/main/python/ml/prefixspan_example.py | 48 +++++++++++++ .../spark/examples/ml/PrefixSpanExample.scala | 62 +++++++++++++++++ 4 files changed, 224 insertions(+) create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java create mode 100644 examples/src/main/python/ml/prefixspan_example.py create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala diff --git a/docs/ml-frequent-pattern-mining.md b/docs/ml-frequent-pattern-mining.md index 81634de8aade..c0928ab77d5a 100644 --- a/docs/ml-frequent-pattern-mining.md +++ b/docs/ml-frequent-pattern-mining.md @@ -85,3 +85,49 @@ Refer to the [R API docs](api/R/spark.fpGrowth.html) for more details. + +## PrefixSpan + +PrefixSpan is a sequential pattern mining algorithm described in +[Pei et al., Mining Sequential Patterns by Pattern-Growth: The +PrefixSpan Approach](http://dx.doi.org/10.1109%2FTKDE.2004.77). We refer +the reader to the referenced paper for formalizing the sequential +pattern mining problem. + +`spark.ml`'s PrefixSpan implementation takes the following parameters: + +* `minSupport`: the minimum support required to be considered a frequent + sequential pattern. +* `maxPatternLength`: the maximum length of a frequent sequential + pattern. Any frequent pattern exceeding this length will not be + included in the results. +* `maxLocalProjDBSize`: the maximum number of items allowed in a + prefix-projected database before local iterative processing of the + projected database begins. This parameter should be tuned with respect + to the size of your executors. +* `sequenceCol`: the name of the sequence column in dataset (default "sequence"), rows with + nulls in this column are ignored. + +**Examples** + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.fpm.PrefixSpan) for more details. + +{% include_example scala/org/apache/spark/examples/ml/PrefixSpanExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/fpm/PrefixSpan.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.fpm.PrefixSpan) for more details. + +{% include_example python/ml/prefixspan_example.py %} +
    + +
    diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java new file mode 100644 index 000000000000..98ffd4faf957 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java @@ -0,0 +1,68 @@ +/* + * 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.examples.ml; + +// $example on$ + +import org.apache.spark.ml.fpm.PrefixSpan; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.*; + +import java.util.Arrays; +import java.util.List; +// $example off$ + +/** + * An example demonstrating PrefixSpan. + * Run with + *
    + * bin/run-example ml.JavaPrefixSpanExample
    + * 
    + */ +public class JavaPrefixSpanExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaPrefixSpanExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3))), + RowFactory.create(Arrays.asList(Arrays.asList(1), Arrays.asList(3, 2), Arrays.asList(1,2))), + RowFactory.create(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(5))), + RowFactory.create(Arrays.asList(Arrays.asList(6))) + ); + StructType schema = new StructType(new StructField[]{ new StructField( + "sequence", new ArrayType(new ArrayType(DataTypes.IntegerType, true), true), + false, Metadata.empty()) + }); + Dataset sequenceDF = spark.createDataFrame(data, schema); + + PrefixSpan prefixSpan = new PrefixSpan().setMinSupport(0.5).setMaxPatternLength(5); + + // Finding frequent sequential patterns + prefixSpan.findFrequentSequentialPatterns(sequenceDF).show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/python/ml/prefixspan_example.py b/examples/src/main/python/ml/prefixspan_example.py new file mode 100644 index 000000000000..88d1d4197341 --- /dev/null +++ b/examples/src/main/python/ml/prefixspan_example.py @@ -0,0 +1,48 @@ +# +# 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. +# + +""" +An example demonstrating PrefixSpan. +Run with: + bin/spark-submit examples/src/main/python/ml/prefixspan_example.py +""" +# $example on$ +from pyspark.ml.fpm import PrefixSpan +# $example off$ +from pyspark.sql import Row, SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("PrefixSpanExample")\ + .getOrCreate() + sc = spark.sparkContext + + # $example on$ + df = sc.parallelize([Row(sequence=[[1, 2], [3]]), + Row(sequence=[[1], [3, 2], [1, 2]]), + Row(sequence=[[1, 2], [5]]), + Row(sequence=[[6]])]).toDF() + + prefixSpan = PrefixSpan(minSupport=0.5, maxPatternLength=5, + maxLocalProjDBSize=32000000) + + # Find frequent sequential patterns. + prefixSpan.findFrequentSequentialPatterns(df).show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala new file mode 100644 index 000000000000..0a2d31097a02 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala @@ -0,0 +1,62 @@ +/* + * 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.examples.ml + +// scalastyle:off println + +// $example on$ +import org.apache.spark.ml.fpm.PrefixSpan +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating PrefixSpan. + * Run with + * {{{ + * bin/run-example ml.PrefixSpanExample + * }}} + */ +object PrefixSpanExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + import spark.implicits._ + + // $example on$ + val smallTestData = Seq( + Seq(Seq(1, 2), Seq(3)), + Seq(Seq(1), Seq(3, 2), Seq(1, 2)), + Seq(Seq(1, 2), Seq(5)), + Seq(Seq(6))) + + val df = smallTestData.toDF("sequence") + val result = new PrefixSpan() + .setMinSupport(0.5) + .setMaxPatternLength(5) + .setMaxLocalProjDBSize(32000000) + .findFrequentSequentialPatterns(df) + .show() + // $example off$ + + spark.stop() + } +} +// scalastyle:on println From 0f74bac647c9f8fce112eada7913504b2c6d08fa Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 28 Oct 2018 10:50:46 +0800 Subject: [PATCH 215/879] [SPARK-24709][SQL][2.4] use str instead of basestring in isinstance ## What changes were proposed in this pull request? after backport https://github.com/apache/spark/pull/22775 to 2.4, the 2.4 sbt Jenkins QA job is broken, see https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test/job/spark-branch-2.4-test-sbt-hadoop-2.7/147/console This PR adds `if sys.version >= '3': basestring = str` which onlly exists in master. ## How was this patch tested? existing test Closes #22858 from cloud-fan/python. Authored-by: Wenchen Fan Signed-off-by: hyukjinkwon --- python/pyspark/sql/functions.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 9583a9859bfd..e1d6ea34daac 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -25,6 +25,9 @@ if sys.version < "3": from itertools import imap as map +if sys.version >= '3': + basestring = str + from pyspark import since, SparkContext from pyspark.rdd import ignore_unicode_prefix, PythonEvalType from pyspark.sql.column import Column, _to_java_column, _to_seq, _create_column_from_literal From 00771dced9c73cddfc6325b3ffb00b32864a02a3 Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Sun, 28 Oct 2018 17:51:35 -0700 Subject: [PATCH 216/879] [SPARK-25816][SQL] Fix attribute resolution in nested extractors ## What changes were proposed in this pull request? Extractors are made of 2 expressions, one of them defines the the value to be extract from (called `child`) and the other defines the way of extraction (called `extraction`). In this term extractors have 2 children so they shouldn't be `UnaryExpression`s. `ResolveReferences` was changed in this commit: https://github.com/apache/spark/commit/36b826f5d17ae7be89135cb2c43ff797f9e7fe48 which resulted a regression with nested extractors. An extractor need to define its children as the set of both `child` and `extraction`; and should try to resolve both in `ResolveReferences`. This PR changes `UnresolvedExtractValue` to a `BinaryExpression`. ## How was this patch tested? added UT Closes #22817 from peter-toth/SPARK-25816. Authored-by: Peter Toth Signed-off-by: gatorsmile (cherry picked from commit ca2fca143277deaff58a69b7f1e0360cfc70561f) Signed-off-by: gatorsmile --- .../apache/spark/sql/catalyst/analysis/unresolved.scala | 5 ++++- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 7 +++++++ 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index c1ec736c32ed..857cf382b8f2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -407,7 +407,10 @@ case class ResolvedStar(expressions: Seq[NamedExpression]) extends Star with Une * can be key of Map, index of Array, field name of Struct. */ case class UnresolvedExtractValue(child: Expression, extraction: Expression) - extends UnaryExpression with Unevaluable { + extends BinaryExpression with Unevaluable { + + override def left: Expression = child + override def right: Expression = extraction override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def foldable: Boolean = throw new UnresolvedException(this, "foldable") 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 e84cd8ccea6c..2ca0e5f12f2a 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 @@ -2590,4 +2590,11 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { Row ("abc", 1)) } } + + test("SPARK-25816 ResolveReferences works with nested extractors") { + val df = Seq((1, Map(1 -> "a")), (2, Map(2 -> "b"))).toDF("key", "map") + val swappedDf = df.select($"key".as("map"), $"map".as("key")) + + checkAnswer(swappedDf.filter($"key"($"map") > "a"), Row(2, Map(2 -> "b"))) + } } From b6ba0dd4773d4f5de02cbb49b70182ce94899671 Mon Sep 17 00:00:00 2001 From: Bruce Robbins Date: Mon, 29 Oct 2018 13:44:58 +0800 Subject: [PATCH 217/879] [DOC] Fix doc for spark.sql.parquet.recordLevelFilter.enabled ## What changes were proposed in this pull request? Updated the doc string value for spark.sql.parquet.recordLevelFilter.enabled to indicate that spark.sql.parquet.enableVectorizedReader must be disabled. The code in ParquetFileFormat uses spark.sql.parquet.recordLevelFilter.enabled only after falling back to parquet-mr (see else for this if statement): https://github.com/apache/spark/blob/d5573c578a1eea9ee04886d9df37c7178e67bb30/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L412 https://github.com/apache/spark/blob/d5573c578a1eea9ee04886d9df37c7178e67bb30/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala#L427-L430 Tests also bear this out. ## How was this patch tested? This is just a doc string fix: I built Spark and ran a single test. Closes #22865 from bersprockets/confdocfix. Authored-by: Bruce Robbins Signed-off-by: Wenchen Fan (cherry picked from commit 4e990d9dd2407dc257712c4b12b507f0990ca4e9) Signed-off-by: Wenchen Fan --- .../src/main/scala/org/apache/spark/sql/internal/SQLConf.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 05264d33e440..08def90cdcd0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -442,7 +442,8 @@ object SQLConf { val PARQUET_RECORD_FILTER_ENABLED = buildConf("spark.sql.parquet.recordLevelFilter.enabled") .doc("If true, enables Parquet's native record-level filtering using the pushed down " + "filters. This configuration only has an effect when 'spark.sql.parquet.filterPushdown' " + - "is enabled.") + "is enabled and the vectorized reader is not used. You can ensure the vectorized reader " + + "is not used by setting 'spark.sql.parquet.enableVectorizedReader' to false.") .booleanConf .createWithDefault(false) From 7f4fce426025d54f41d8e87928582563a8ad689e Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sun, 28 Oct 2018 23:01:35 -0700 Subject: [PATCH 218/879] [SPARK-25179][PYTHON][DOCS] Document BinaryType support in Arrow conversion ## What changes were proposed in this pull request? This PR targets to document binary type in "Apache Arrow in Spark". ## How was this patch tested? Manually built the documentation and checked. Closes #22871 from HyukjinKwon/SPARK-25179. Authored-by: hyukjinkwon Signed-off-by: gatorsmile (cherry picked from commit fbaf150507a289ec0ac02fdbf4009c42cd9bc164) Signed-off-by: gatorsmile --- docs/sql-pyspark-pandas-with-arrow.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/sql-pyspark-pandas-with-arrow.md b/docs/sql-pyspark-pandas-with-arrow.md index e8e9f55bd12b..d04b955f9bf8 100644 --- a/docs/sql-pyspark-pandas-with-arrow.md +++ b/docs/sql-pyspark-pandas-with-arrow.md @@ -127,8 +127,9 @@ For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/p ### Supported SQL Types -Currently, all Spark SQL data types are supported by Arrow-based conversion except `BinaryType`, `MapType`, -`ArrayType` of `TimestampType`, and nested `StructType`. +Currently, all Spark SQL data types are supported by Arrow-based conversion except `MapType`, +`ArrayType` of `TimestampType`, and nested `StructType`. `BinaryType` is supported only when +installed PyArrow is equal to or higher then 0.10.0. ### Setting Arrow Batch Size From 0a4c03f7d084f1d2aa48673b99f3b9496893ce8d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 29 Oct 2018 06:15:29 +0000 Subject: [PATCH 219/879] Preparing Spark release v2.4.0-rc5 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 714b6f18b19d..f52d785e05cd 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.0 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index ee0de73f21d5..63ab510eb683 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b89e0fed2a66..b10e11849a74 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 3105a8182afc..74c6d232b36a 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f85248b53080..fbdc979d120d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 3350990c0461..53d1b2b58f1f 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 9a2cb90a2f30..98145481adc7 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 9b927a9ba525..f0c8ebe25e63 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 718875f88e74..c635cd3b9394 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 583a2fff7638..ed3efffa180b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 72473771c2e9..7f9a91e2fba0 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1-SNAPSHOT -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.0 +SPARK_VERSION_SHORT: 2.4.0 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 737ebfd7900c..c7b8354b0968 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 7c6b78ac2b31..a91c13362caa 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 6343a15c43a8..db239f23e1de 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 2f4886832b75..cf19fed1bef5 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9404e2a52d90..da5dba82f3cf 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 73dbc8d99449..e591ce802256 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 912d82b764e5..e8b13cdc2bef 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 8454f5c108dc..c3d3b888656b 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 05f7cf870af8..468ace0ff8d7 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 0e935ac112af..db8fda66c3cd 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 7977389718cb..aa1e1267f57b 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7193df69900d..7d07e18f04a2 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 4b318b00ff5b..f545f212b2f0 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6fbb1b7db6aa..6b388ffcf18b 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 9a42d9a69967..fc369c73edc0 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 702f60ab9f00..f4cb94147d9e 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 62322da714bd..22f60efae9af 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index c8e45b27c78f..b17e67d78b98 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 6201e0dd3c9a..bf0d406da9ed 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/pom.xml b/pom.xml index 349de835bfc9..f0e5ed9c563c 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 87282141fa64..53e3424394ba 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1.dev0" +__version__ = "2.4.0" diff --git a/repl/pom.xml b/repl/pom.xml index d09548348eb5..4532f0b8686e 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index a9d9c8de9959..47fe3f2765b2 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 793167103179..f59895e9a1bd 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index e6457289b859..0eacbbfac2b9 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index e70711ce7257..2afdc15be446 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 65cf33f32972..ef1d012d2f80 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c327b85ee866..f4be074642ae 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d9d742444e7e..e5c1065451f7 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c428be67aa3d..b522bfeac130 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 37e96b807653..8e71adabb3ec 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index dd00c2f6cd2c..c00e1b851ba0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.0 ../pom.xml From 22bec3c6dab1147eee0342993aa8f64202603a8d Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Mon, 29 Oct 2018 06:15:33 +0000 Subject: [PATCH 220/879] Preparing development version 2.4.1-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index f52d785e05cd..714b6f18b19d 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.0 +Version: 2.4.1 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 63ab510eb683..ee0de73f21d5 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b10e11849a74..b89e0fed2a66 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 74c6d232b36a..3105a8182afc 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index fbdc979d120d..f85248b53080 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 53d1b2b58f1f..3350990c0461 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 98145481adc7..9a2cb90a2f30 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index f0c8ebe25e63..9b927a9ba525 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index c635cd3b9394..718875f88e74 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index ed3efffa180b..583a2fff7638 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7f9a91e2fba0..72473771c2e9 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.0 -SPARK_VERSION_SHORT: 2.4.0 +SPARK_VERSION: 2.4.1-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index c7b8354b0968..737ebfd7900c 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index a91c13362caa..7c6b78ac2b31 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index db239f23e1de..6343a15c43a8 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index cf19fed1bef5..2f4886832b75 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index da5dba82f3cf..9404e2a52d90 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index e591ce802256..73dbc8d99449 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index e8b13cdc2bef..912d82b764e5 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index c3d3b888656b..8454f5c108dc 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 468ace0ff8d7..05f7cf870af8 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index db8fda66c3cd..0e935ac112af 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index aa1e1267f57b..7977389718cb 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7d07e18f04a2..7193df69900d 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index f545f212b2f0..4b318b00ff5b 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6b388ffcf18b..6fbb1b7db6aa 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index fc369c73edc0..9a42d9a69967 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index f4cb94147d9e..702f60ab9f00 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 22f60efae9af..62322da714bd 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index b17e67d78b98..c8e45b27c78f 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index bf0d406da9ed..6201e0dd3c9a 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index f0e5ed9c563c..349de835bfc9 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 53e3424394ba..87282141fa64 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.0" +__version__ = "2.4.1.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index 4532f0b8686e..d09548348eb5 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 47fe3f2765b2..a9d9c8de9959 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index f59895e9a1bd..793167103179 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 0eacbbfac2b9..e6457289b859 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 2afdc15be446..e70711ce7257 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index ef1d012d2f80..65cf33f32972 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index f4be074642ae..c327b85ee866 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index e5c1065451f7..d9d742444e7e 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index b522bfeac130..c428be67aa3d 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 8e71adabb3ec..37e96b807653 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index c00e1b851ba0..dd00c2f6cd2c 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.0 + 2.4.1-SNAPSHOT ../pom.xml From 5cc2987dbba609d99df0b367abe25238c9498cba Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Mon, 29 Oct 2018 16:47:50 +0100 Subject: [PATCH 221/879] [SPARK-25767][SQL] Fix lazily evaluated stream of expressions in code generation ## What changes were proposed in this pull request? Code generation is incorrect if `outputVars` parameter of `consume` method in `CodegenSupport` contains a lazily evaluated stream of expressions. This PR fixes the issue by forcing the evaluation of `inputVars` before generating the code for UnsafeRow. ## How was this patch tested? Tested with the sample program provided in https://issues.apache.org/jira/browse/SPARK-25767 Closes #22789 from peter-toth/SPARK-25767. Authored-by: Peter Toth Signed-off-by: Herman van Hovell (cherry picked from commit 7fe5cff0581ca9d8221533215098f40f69362018) Signed-off-by: Herman van Hovell --- .../spark/sql/execution/WholeStageCodegenExec.scala | 5 ++++- .../spark/sql/execution/WholeStageCodegenSuite.scala | 11 +++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 1fc4de9e5601..ded8dd30dc49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -146,7 +146,10 @@ trait CodegenSupport extends SparkPlan { if (outputVars != null) { assert(outputVars.length == output.length) // outputVars will be used to generate the code for UnsafeRow, so we should copy them - outputVars.map(_.copy()) + outputVars.map(_.copy()) match { + case stream: Stream[ExprCode] => stream.force + case other => other + } } else { assert(row != null, "outputVars and row cannot both be null.") ctx.currentVars = null diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index b714dcd5269f..09ad0fdd6636 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -319,4 +319,15 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { assert(df.limit(1).collect() === Array(Row("bat", 8.0))) } } + + test("SPARK-25767: Lazy evaluated stream of expressions handled correctly") { + val a = Seq(1).toDF("key") + val b = Seq((1, "a")).toDF("key", "value") + val c = Seq(1).toDF("key") + + val ab = a.join(b, Stream("key"), "left") + val abc = ab.join(c, Seq("key"), "left") + + checkAnswer(abc, Row(1, "a")) + } } From 3d2fce5a3275a8f5d50a6894198297cddc022843 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 31 Oct 2018 15:14:10 -0700 Subject: [PATCH 222/879] [SPARK-25899][TESTS] Fix flaky CoarseGrainedSchedulerBackendSuite ## What changes were proposed in this pull request? I saw CoarseGrainedSchedulerBackendSuite failed in my PR and finally reproduced the following error on a very busy machine: ``` sbt.ForkMain$ForkError: org.scalatest.exceptions.TestFailedDueToTimeoutException: The code passed to eventually never returned normally. Attempted 400 times over 10.009828643999999 seconds. Last failure message: ArrayBuffer("2", "0", "3") had length 3 instead of expected length 4. ``` The logs in this test shows executor 1 was not up when the test failed. ``` 18/10/30 11:34:03.563 dispatcher-event-loop-12 INFO CoarseGrainedSchedulerBackend$DriverEndpoint: Registered executor NettyRpcEndpointRef(spark-client://Executor) (172.17.0.2:43656) with ID 2 18/10/30 11:34:03.593 dispatcher-event-loop-3 INFO CoarseGrainedSchedulerBackend$DriverEndpoint: Registered executor NettyRpcEndpointRef(spark-client://Executor) (172.17.0.2:43658) with ID 3 18/10/30 11:34:03.629 dispatcher-event-loop-6 INFO CoarseGrainedSchedulerBackend$DriverEndpoint: Registered executor NettyRpcEndpointRef(spark-client://Executor) (172.17.0.2:43654) with ID 0 18/10/30 11:34:03.885 pool-1-thread-1-ScalaTest-running-CoarseGrainedSchedulerBackendSuite INFO CoarseGrainedSchedulerBackendSuite: ===== FINISHED o.a.s.scheduler.CoarseGrainedSchedulerBackendSuite: 'compute max number of concurrent tasks can be launched' ===== ``` And the following logs in executor 1 shows it was still doing the initialization when the timeout happened (at 18/10/30 11:34:03.885). ``` 18/10/30 11:34:03.463 netty-rpc-connection-0 INFO TransportClientFactory: Successfully created connection to 54b6b6217301/172.17.0.2:33741 after 37 ms (0 ms spent in bootstraps) 18/10/30 11:34:03.959 main INFO DiskBlockManager: Created local directory at /home/jenkins/workspace/core/target/tmp/spark-383518bc-53bd-4d9c-885b-d881f03875bf/executor-61c406e4-178f-40a6-ac2c-7314ee6fb142/blockmgr-03fb84a1-eedc-4055-8743-682eb3ac5c67 18/10/30 11:34:03.993 main INFO MemoryStore: MemoryStore started with capacity 546.3 MB ``` Hence, I think our current 10 seconds is not enough on a slow Jenkins machine. This PR just increases the timeout from 10 seconds to 60 seconds to make the test more stable. ## How was this patch tested? Jenkins Closes #22910 from zsxwing/fix-flaky-test. Authored-by: Shixiong Zhu Signed-off-by: gatorsmile (cherry picked from commit 6be3cce751fd0abf00d668c771f56093f2fa6817) Signed-off-by: gatorsmile --- .../scheduler/CoarseGrainedSchedulerBackendSuite.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala index 80c9c6f0422a..c5a39669366c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -30,6 +30,8 @@ import org.apache.spark.util.{RpcUtils, SerializableBuffer} class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with Eventually { + private val executorUpTimeout = 60.seconds + test("serialized task larger than max RPC message size") { val conf = new SparkConf conf.set("spark.rpc.message.maxSize", "1") @@ -51,7 +53,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo .setMaster("local-cluster[4, 3, 1024]") .setAppName("test") sc = new SparkContext(conf) - eventually(timeout(10.seconds)) { + eventually(timeout(executorUpTimeout)) { // Ensure all executors have been launched. assert(sc.getExecutorIds().length == 4) } @@ -64,7 +66,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo .setMaster("local-cluster[4, 3, 1024]") .setAppName("test") sc = new SparkContext(conf) - eventually(timeout(10.seconds)) { + eventually(timeout(executorUpTimeout)) { // Ensure all executors have been launched. assert(sc.getExecutorIds().length == 4) } @@ -96,7 +98,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo try { sc.addSparkListener(listener) - eventually(timeout(10.seconds)) { + eventually(timeout(executorUpTimeout)) { // Ensure all executors have been launched. assert(sc.getExecutorIds().length == 4) } From 73894462cfb80b7c3e61c743b5a2f3be5d2282dd Mon Sep 17 00:00:00 2001 From: Patrick Brown Date: Thu, 1 Nov 2018 09:34:29 -0700 Subject: [PATCH 223/879] [SPARK-25837][CORE] Fix potential slowdown in AppStatusListener when cleaning up stages MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? * Update `AppStatusListener` `cleanupStages` method to remove tasks for those stages in a single pass instead of 1 for each stage. * This fixes an issue where the cleanupStages method would get backed up, causing a backup in the executor in ElementTrackingStore, resulting in stages and jobs not getting cleaned up properly. Tasks seem most susceptible to this as there are a lot of them, however a similar issue could arise in other locations the `KVStore` `view` method is used. A broader fix might involve updates to `KVStoreView` and `InMemoryView` as it appears this interface and implementation can lead to multiple and inefficient traversals of the stored data. ## How was this patch tested? Using existing tests in AppStatusListenerSuite This is my original work and I license the work to the project under the project’s open source license. Closes #22883 from patrickbrownsync/cleanup-stages-fix. Authored-by: Patrick Brown Signed-off-by: Marcelo Vanzin (cherry picked from commit e9d3ca0b7993995f24f5c555a570bc2521119e12) Signed-off-by: Marcelo Vanzin --- .../spark/status/AppStatusListener.scala | 19 +++++++++---------- 1 file changed, 9 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 513c929e703f..fdbef6f69c5d 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -1002,16 +1002,6 @@ private[spark] class AppStatusListener( kvstore.delete(e.getClass(), e.id) } - val tasks = kvstore.view(classOf[TaskDataWrapper]) - .index("stage") - .first(key) - .last(key) - .asScala - - tasks.foreach { t => - kvstore.delete(t.getClass(), t.taskId) - } - // Check whether there are remaining attempts for the same stage. If there aren't, then // also delete the RDD graph data. val remainingAttempts = kvstore.view(classOf[StageDataWrapper]) @@ -1034,6 +1024,15 @@ private[spark] class AppStatusListener( cleanupCachedQuantiles(key) } + + // Delete tasks for all stages in one pass, as deleting them for each stage individually is slow + val tasks = kvstore.view(classOf[TaskDataWrapper]).asScala + val keys = stages.map { s => (s.info.stageId, s.info.attemptId) }.toSet + tasks.foreach { t => + if (keys.contains((t.stageId, t.stageAttemptId))) { + kvstore.delete(t.getClass(), t.taskId) + } + } } private def cleanupTasks(stage: LiveStage): Unit = { From 8c508da2add315ca1615219ae444bff99b1f9eca Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 1 Nov 2018 23:18:20 -0700 Subject: [PATCH 224/879] [SPARK-25918][SQL] LOAD DATA LOCAL INPATH should handle a relative path ## What changes were proposed in this pull request? Unfortunately, it seems that we missed this in 2.4.0. In Spark 2.4, if the default file system is not the local file system, `LOAD DATA LOCAL INPATH` only works in case of absolute paths. This PR aims to fix it to support relative paths. This is a regression in 2.4.0. ```scala $ ls kv1.txt kv1.txt scala> spark.sql("LOAD DATA LOCAL INPATH 'kv1.txt' INTO TABLE t") org.apache.spark.sql.AnalysisException: LOAD DATA input path does not exist: kv1.txt; ``` ## How was this patch tested? Pass the Jenkins Closes #22927 from dongjoon-hyun/SPARK-LOAD. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit e91b607719886b57d1550a70c0f9df4342d72989) Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/execution/command/tables.scala | 5 +++-- .../spark/sql/hive/execution/HiveCommandSuite.scala | 9 +++++++++ 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala index 64831e5089a6..5687993a3226 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala @@ -376,7 +376,8 @@ object LoadDataCommand { * @return qualified path object */ private[sql] def makeQualified(defaultUri: URI, workingDir: Path, path: Path): Path = { - val pathUri = if (path.isAbsolute()) path.toUri() else new Path(workingDir, path).toUri() + val newPath = new Path(workingDir, path) + val pathUri = if (path.isAbsolute()) path.toUri() else newPath.toUri() if (pathUri.getScheme == null || pathUri.getAuthority == null && defaultUri.getAuthority != null) { val scheme = if (pathUri.getScheme == null) defaultUri.getScheme else pathUri.getScheme @@ -393,7 +394,7 @@ object LoadDataCommand { throw new IllegalArgumentException(e) } } else { - path + newPath } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala index 6937e97a47dc..9147a98c9445 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCommandSuite.scala @@ -20,11 +20,13 @@ package org.apache.spark.sql.hive.execution import java.io.File import com.google.common.io.Files +import org.apache.hadoop.fs.{FileContext, FsConstants, Path} import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.execution.command.LoadDataCommand import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType @@ -439,4 +441,11 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } + test("SPARK-25918: LOAD DATA LOCAL INPATH should handle a relative path") { + val localFS = FileContext.getLocalFSFileContext() + val workingDir = localFS.getWorkingDirectory + val r = LoadDataCommand.makeQualified( + FsConstants.LOCAL_FS_URI, workingDir, new Path("kv1.txt")) + assert(r === new Path(s"$workingDir/kv1.txt")) + } } From ea11d114264560638129eac1db3aa1dc12a206a2 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 2 Nov 2018 10:56:30 -0500 Subject: [PATCH 225/879] [SPARK-25023] Clarify Spark security documentation ## What changes were proposed in this pull request? Clarify documentation about security. ## How was this patch tested? None, just documentation Closes #22852 from tgravescs/SPARK-25023. Authored-by: Thomas Graves Signed-off-by: Thomas Graves (cherry picked from commit c00186f90cfcc33492d760f874ead34f0e3da6ed) Signed-off-by: Thomas Graves --- docs/index.md | 5 +++++ docs/quick-start.md | 5 +++++ docs/running-on-kubernetes.md | 5 +++++ docs/running-on-mesos.md | 5 +++++ docs/running-on-yarn.md | 5 +++++ docs/security.md | 17 +++++++++++++++-- docs/spark-standalone.md | 5 +++++ 7 files changed, 45 insertions(+), 2 deletions(-) diff --git a/docs/index.md b/docs/index.md index 40f628b794c0..0300528135a6 100644 --- a/docs/index.md +++ b/docs/index.md @@ -10,6 +10,11 @@ It provides high-level APIs in Java, Scala, Python and R, and an optimized engine that supports general execution graphs. It also supports a rich set of higher-level tools including [Spark SQL](sql-programming-guide.html) for SQL and structured data processing, [MLlib](ml-guide.html) for machine learning, [GraphX](graphx-programming-guide.html) for graph processing, and [Spark Streaming](streaming-programming-guide.html). +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) before downloading and running Spark. + # Downloading Get Spark from the [downloads page](https://spark.apache.org/downloads.html) of the project website. This documentation is for Spark version {{site.SPARK_VERSION}}. Spark uses Hadoop's client libraries for HDFS and YARN. Downloads are pre-packaged for a handful of popular Hadoop versions. diff --git a/docs/quick-start.md b/docs/quick-start.md index ef7af6c3f6ce..28186c11887f 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -17,6 +17,11 @@ you can download a package for any version of Hadoop. Note that, before Spark 2.0, the main programming interface of Spark was the Resilient Distributed Dataset (RDD). After Spark 2.0, RDDs are replaced by Dataset, which is strongly-typed like an RDD, but with richer optimizations under the hood. The RDD interface is still supported, and you can get a more detailed reference at the [RDD programming guide](rdd-programming-guide.html). However, we highly recommend you to switch to use Dataset, which has better performance than RDD. See the [SQL programming guide](sql-programming-guide.html) to get more information about Dataset. +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) before running Spark. + # Interactive Analysis with the Spark Shell ## Basics diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index f19aa412389f..754b1ff4e3a3 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -12,6 +12,11 @@ Kubernetes scheduler that has been added to Spark. In future versions, there may be behavioral changes around configuration, container images and entrypoints.** +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. + # Prerequisites * A runnable distribution of Spark 2.3 or above. diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index b473e654563d..2502cd4ca86f 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -13,6 +13,11 @@ The advantages of deploying Spark with Mesos include: [frameworks](https://mesos.apache.org/documentation/latest/frameworks/) - scalable partitioning between multiple instances of Spark +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. + # How it Works In a standalone cluster deployment, the cluster manager in the below diagram is a Spark master diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index e3d67c34d53e..f265075e351e 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -9,6 +9,11 @@ Support for running on [YARN (Hadoop NextGen)](http://hadoop.apache.org/docs/stable/hadoop-yarn/hadoop-yarn-site/YARN.html) was added to Spark in version 0.6.0, and improved in subsequent releases. +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. + # Launching Spark on YARN Ensure that `HADOOP_CONF_DIR` or `YARN_CONF_DIR` points to the directory which contains the (client side) configuration files for the Hadoop cluster. diff --git a/docs/security.md b/docs/security.md index 7fb3e17de94c..2948fbcb7d9d 100644 --- a/docs/security.md +++ b/docs/security.md @@ -6,7 +6,20 @@ title: Security * This will become a table of contents (this text will be scraped). {:toc} -# Spark RPC +# Spark Security: Things You Need To Know + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Spark supports multiple deployments types and each one supports different levels of security. Not +all deployment types will be secure in all environments and none are secure by default. Be +sure to evaluate your environment, what Spark supports, and take the appropriate measure to secure +your Spark deployment. + +There are many different types of security concerns. Spark does not necessarily protect against +all things. Listed below are some of the things Spark supports. Also check the deployment +documentation for the type of deployment you are using for deployment specific settings. Anything +not documented, Spark does not support. + +# Spark RPC (Communication protocol between Spark processes) ## Authentication @@ -123,7 +136,7 @@ The following table describes the different options available for configuring th Spark supports encrypting temporary data written to local disks. This covers shuffle files, shuffle spills and data blocks stored on disk (for both caching and broadcast variables). It does not cover encrypting output data generated by applications with APIs such as `saveAsHadoopFile` or -`saveAsTable`. +`saveAsTable`. It also may not cover temporary files created explicitly by the user. The following settings cover enabling encryption for data written to disk: diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 7975b0c8b11c..49ef2e1ce2a1 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -8,6 +8,11 @@ title: Spark Standalone Mode In addition to running on the Mesos or YARN cluster managers, Spark also provides a simple standalone deploy mode. You can launch a standalone cluster either manually, by starting a master and workers by hand, or use our provided [launch scripts](#cluster-launch-scripts). It is also possible to run these daemons on a single machine for testing. +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. + # Installing Spark Standalone to a Cluster To install Spark Standalone mode, you simply place a compiled version of Spark on each node on the cluster. You can obtain pre-built versions of Spark with each release or [build it yourself](building-spark.html). From 11e07812cacd52cb2d50591ab1d66c33e57834d5 Mon Sep 17 00:00:00 2001 From: James Lamb Date: Fri, 2 Nov 2018 11:05:10 -0500 Subject: [PATCH 226/879] [SPARK-25909] fix documentation on cluster managers ## What changes were proposed in this pull request? Propose changing the documentation to state that there are 4, not 3, cluster managers available. ## How was this patch tested? This is a docs-only patch and doesn't need any new testing beyond the normal CI process for Spark. Closes #22922 from jameslamb/bugfix/cluster_docs. Authored-by: James Lamb Signed-off-by: Sean Owen (cherry picked from commit c71db43e11fb90d6675421604ad29f596f2b8bfe) Signed-off-by: Sean Owen --- docs/cluster-overview.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 7277e2fb2731..1f0822f7a317 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -45,7 +45,7 @@ There are several useful things to note about this architecture: # Cluster Manager Types -The system currently supports three cluster managers: +The system currently supports several cluster managers: * [Standalone](spark-standalone.html) -- a simple cluster manager included with Spark that makes it easy to set up a cluster. From 881a60403ffcc178e9177726470fe07ece0cb6f8 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Fri, 2 Nov 2018 13:24:55 -0700 Subject: [PATCH 227/879] [SPARK-25827][CORE] Avoid converting incoming encrypted blocks to byte buffers ## What changes were proposed in this pull request? Avoid converting encrypted bocks to regular ByteBuffers, to ensure they can be sent over the network for replication & remote reads even when > 2GB. Also updates some TODOs with links to a SPARK-25905 for improving the handling here. ## How was this patch tested? Tested on a cluster with encrypted data > 2GB (after SPARK-25904 was applied as well). Closes #22917 from squito/real_SPARK-25827. Authored-by: Imran Rashid Signed-off-by: Marcelo Vanzin (cherry picked from commit 7ea594e7876258296f340daddefcaf71a64ab824) Signed-off-by: Marcelo Vanzin --- .../org/apache/spark/network/BlockTransferService.scala | 4 +++- .../main/scala/org/apache/spark/storage/BlockManager.scala | 2 +- .../src/main/scala/org/apache/spark/storage/DiskStore.scala | 5 +++-- .../scala/org/apache/spark/util/io/ChunkedByteBuffer.scala | 6 ++++-- 4 files changed, 11 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index eef8c31e05ab..a58c8fa2e763 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -27,7 +27,7 @@ import scala.reflect.ClassTag import org.apache.spark.internal.Logging import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ShuffleClient} -import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.storage.{BlockId, EncryptedManagedBuffer, StorageLevel} import org.apache.spark.util.ThreadUtils private[spark] @@ -104,6 +104,8 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo data match { case f: FileSegmentManagedBuffer => result.success(f) + case e: EncryptedManagedBuffer => + result.success(e) case _ => val ret = ByteBuffer.allocate(data.size.toInt) ret.put(data.nioByteBuffer()) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index c01a45315191..e35dd7252124 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -721,7 +721,7 @@ private[spark] class BlockManager( * Get block from remote block managers as serialized bytes. */ def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = { - // TODO if we change this method to return the ManagedBuffer, then getRemoteValues + // TODO SPARK-25905 if we change this method to return the ManagedBuffer, then getRemoteValues // could just use the inputStream on the temp file, rather than reading the file into memory. // Until then, replication can cause the process to use too much memory and get killed // even though we've read the data to disk. diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index d88bd710d1ea..841e16afc754 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -201,7 +201,7 @@ private class DiskBlockData( private def open() = new FileInputStream(file).getChannel } -private class EncryptedBlockData( +private[spark] class EncryptedBlockData( file: File, blockSize: Long, conf: SparkConf, @@ -263,7 +263,8 @@ private class EncryptedBlockData( } } -private class EncryptedManagedBuffer(val blockData: EncryptedBlockData) extends ManagedBuffer { +private[spark] class EncryptedManagedBuffer( + val blockData: EncryptedBlockData) extends ManagedBuffer { // This is the size of the decrypted data override def size(): Long = blockData.size diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala index 9547cb49bbee..da2be84723a0 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala @@ -29,7 +29,7 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.config import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.util.{ByteArrayWritableChannel, LimitedInputStream} -import org.apache.spark.storage.StorageUtils +import org.apache.spark.storage.{EncryptedManagedBuffer, StorageUtils} import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils @@ -173,11 +173,13 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { private[spark] object ChunkedByteBuffer { - // TODO eliminate this method if we switch BlockManager to getting InputStreams + // TODO SPARK-25905 eliminate this method if we switch BlockManager to getting InputStreams def fromManagedBuffer(data: ManagedBuffer): ChunkedByteBuffer = { data match { case f: FileSegmentManagedBuffer => fromFile(f.getFile, f.getOffset, f.getLength) + case e: EncryptedManagedBuffer => + e.blockData.toChunkedByteBuffer(ByteBuffer.allocate _) case other => new ChunkedByteBuffer(other.nioByteBuffer()) } From 5bc4e7d1ac52382d796b9c1dbe37b695880135e8 Mon Sep 17 00:00:00 2001 From: Alex Hagerman Date: Sat, 3 Nov 2018 12:56:59 -0500 Subject: [PATCH 228/879] [SPARK-25933][DOCUMENTATION] Fix pstats.Stats() reference in configuration.md ## What changes were proposed in this pull request? Change ptats.Stats() to pstats.Stats() for `spark.python.profile.dump` in configuration.md. ## How was this patch tested? Doc test Closes #22933 from AlexHagerman/doc_fix. Authored-by: Alex Hagerman Signed-off-by: Sean Owen (cherry picked from commit 1a7abf3f453f7d6012d7e842cf05f29f3afbb3bc) Signed-off-by: Sean Owen --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 613e214783d5..dfe781513052 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -445,7 +445,7 @@ Apart from these, the following properties are also available, and may be useful The directory which is used to dump the profile result before driver exiting. The results will be dumped as separated file for each RDD. They can be loaded - by ptats.Stats(). If this is specified, the profile result will not be displayed + by pstats.Stats(). If this is specified, the profile result will not be displayed automatically. From af2ec972d7022fac369e7b6125f04134983486cf Mon Sep 17 00:00:00 2001 From: Stavros Kontopoulos Date: Mon, 5 Nov 2018 08:40:25 -0600 Subject: [PATCH 229/879] [SPARK-25930][K8S] Fix scala string detection in k8s tests ## What changes were proposed in this pull request? - Issue is described in detail in [SPARK-25930](https://issues.apache.org/jira/browse/SPARK-25930). Since we rely on the std output, pick always the last line which contains the wanted value. Although minor, current implementation breaks tests. ## How was this patch tested? manually. rm -rf ~/.m2 and then run the tests. Closes #22931 from skonto/fix_scala_detection. Authored-by: Stavros Kontopoulos Signed-off-by: Sean Owen (cherry picked from commit 1fb3759f2b60a2e7c5e2a82afe1a580d848e0f8c) Signed-off-by: Sean Owen --- .../integration-tests/dev/dev-run-integration-tests.sh | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh index cb5cf693d52d..1b9de488a837 100755 --- a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh +++ b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh @@ -28,7 +28,12 @@ NAMESPACE= SERVICE_ACCOUNT= INCLUDE_TAGS="k8s" EXCLUDE_TAGS= -SCALA_VERSION="$($TEST_ROOT_DIR/build/mvn org.apache.maven.plugins:maven-help-plugin:2.1.1:evaluate -Dexpression=scala.binary.version | grep -v '\[' )" +MVN="$TEST_ROOT_DIR/build/mvn" + +SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version 2>/dev/null\ + | grep -v "INFO"\ + | grep -v "WARNING"\ + | tail -n 1) # Parse arguments while (( "$#" )); do From 8526f2ee5362df63febe47ab4c64aa6d1b71f990 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 5 Nov 2018 17:34:23 -0600 Subject: [PATCH 230/879] [MINOR] Fix typos and misspellings ## What changes were proposed in this pull request? Fix typos and misspellings, per https://github.com/apache/spark-website/pull/158#issuecomment-435790366 ## How was this patch tested? Existing tests. Closes #22950 from srowen/Typos. Authored-by: Sean Owen Signed-off-by: Sean Owen (cherry picked from commit c0d1bf0322be12230c30cb200f19a02e4d5e0d49) Signed-off-by: Sean Owen --- .../java/org/apache/spark/ExecutorPlugin.java | 6 +++--- .../org/apache/spark/ExecutorPluginSuite.java | 4 ++-- docs/sql-migration-guide-upgrade.md | 2 +- .../ml/r/AFTSurvivalRegressionWrapper.scala | 6 +++--- .../org/apache/spark/ml/stat/Summarizer.scala | 4 ++-- .../stat/MultivariateOnlineSummarizer.scala | 2 +- python/pyspark/ml/stat.py | 2 +- .../spark/sql/hive/CachedTableSuite.scala | 17 ++++++++--------- 8 files changed, 21 insertions(+), 22 deletions(-) diff --git a/core/src/main/java/org/apache/spark/ExecutorPlugin.java b/core/src/main/java/org/apache/spark/ExecutorPlugin.java index ec0b57f1a281..f86520c81df3 100644 --- a/core/src/main/java/org/apache/spark/ExecutorPlugin.java +++ b/core/src/main/java/org/apache/spark/ExecutorPlugin.java @@ -20,18 +20,18 @@ import org.apache.spark.annotation.DeveloperApi; /** - * A plugin which can be automaticaly instantiated within each Spark executor. Users can specify + * A plugin which can be automatically instantiated within each Spark executor. Users can specify * plugins which should be created with the "spark.executor.plugins" configuration. An instance * of each plugin will be created for every executor, including those created by dynamic allocation, * before the executor starts running any tasks. * * The specific api exposed to the end users still considered to be very unstable. We will - * hopefully be able to keep compatability by providing default implementations for any methods + * hopefully be able to keep compatibility by providing default implementations for any methods * added, but make no guarantees this will always be possible across all Spark releases. * * Spark does nothing to verify the plugin is doing legitimate things, or to manage the resources * it uses. A plugin acquires the same privileges as the user running the task. A bad plugin - * could also intefere with task execution and make the executor fail in unexpected ways. + * could also interfere with task execution and make the executor fail in unexpected ways. */ @DeveloperApi public interface ExecutorPlugin { diff --git a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java index 686eb28010c6..80cd70282a51 100644 --- a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java +++ b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java @@ -63,10 +63,10 @@ private SparkConf initializeSparkConf(String pluginNames) { @Test public void testPluginClassDoesNotExist() { - SparkConf conf = initializeSparkConf("nonexistant.plugin"); + SparkConf conf = initializeSparkConf("nonexistent.plugin"); try { sc = new JavaSparkContext(conf); - fail("No exception thrown for nonexistant plugin"); + fail("No exception thrown for nonexistent plugin"); } catch (Exception e) { // We cannot catch ClassNotFoundException directly because Java doesn't think it'll be thrown assertTrue(e.toString().startsWith("java.lang.ClassNotFoundException")); diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 9a7f5b6434dd..7b4804059f44 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -107,7 +107,7 @@ displayTitle: Spark SQL Upgrading Guide - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. - - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. + - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was written as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala index 48485e02edda..1b5f77a9ae89 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala @@ -62,7 +62,7 @@ private[r] object AFTSurvivalRegressionWrapper extends MLReadable[AFTSurvivalReg private val FORMULA_REGEXP = """Surv\(([^,]+), ([^,]+)\) ~ (.+)""".r private def formulaRewrite(formula: String): (String, String) = { - var rewritedFormula: String = null + var rewrittenFormula: String = null var censorCol: String = null try { val FORMULA_REGEXP(label, censor, features) = formula @@ -71,14 +71,14 @@ private[r] object AFTSurvivalRegressionWrapper extends MLReadable[AFTSurvivalReg throw new UnsupportedOperationException( "Terms of survreg formula can not support dot operator.") } - rewritedFormula = label.trim + "~" + features.trim + rewrittenFormula = label.trim + "~" + features.trim censorCol = censor.trim } catch { case e: MatchError => throw new SparkException(s"Could not parse formula: $formula") } - (rewritedFormula, censorCol) + (rewrittenFormula, censorCol) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala index d40827edb6d6..ed7d7e085264 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala @@ -96,7 +96,7 @@ object Summarizer extends Logging { * - numNonzeros: a vector with the number of non-zeros for each coefficients * - max: the maximum for each coefficient. * - min: the minimum for each coefficient. - * - normL2: the Euclidian norm for each coefficient. + * - normL2: the Euclidean norm for each coefficient. * - normL1: the L1 norm of each coefficient (sum of the absolute values). * @param metrics metrics that can be provided. * @return a builder. @@ -536,7 +536,7 @@ private[ml] object SummaryBuilderImpl extends Logging { } /** - * L2 (Euclidian) norm of each dimension. + * L2 (Euclidean) norm of each dimension. */ def normL2: Vector = { require(requestedMetrics.contains(NormL2)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 8121880cfb23..0554b6d8ff5b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -273,7 +273,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S } /** - * L2 (Euclidian) norm of each dimension. + * L2 (Euclidean) norm of each dimension. * */ @Since("1.2.0") diff --git a/python/pyspark/ml/stat.py b/python/pyspark/ml/stat.py index 370154fc6d62..3f421024acdc 100644 --- a/python/pyspark/ml/stat.py +++ b/python/pyspark/ml/stat.py @@ -336,7 +336,7 @@ def metrics(*metrics): - numNonzeros: a vector with the number of non-zeros for each coefficients - max: the maximum for each coefficient. - min: the minimum for each coefficient. - - normL2: the Euclidian norm for each coefficient. + - normL2: the Euclidean norm for each coefficient. - normL1: the L1 norm of each coefficient (sum of the absolute values). :param metrics: diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 569f00c053e5..b492f39df62f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.hive import java.io.File import org.apache.spark.sql.{AnalysisException, Dataset, QueryTest, SaveMode} -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation} @@ -97,24 +96,24 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } - test("DROP nonexistant table") { - sql("DROP TABLE IF EXISTS nonexistantTable") + test("DROP nonexistent table") { + sql("DROP TABLE IF EXISTS nonexistentTable") } - test("uncache of nonexistant tables") { - val expectedErrorMsg = "Table or view not found: nonexistantTable" + test("uncache of nonexistent tables") { + val expectedErrorMsg = "Table or view not found: nonexistentTable" // make sure table doesn't exist - var e = intercept[AnalysisException](spark.table("nonexistantTable")).getMessage + var e = intercept[AnalysisException](spark.table("nonexistentTable")).getMessage assert(e.contains(expectedErrorMsg)) e = intercept[AnalysisException] { - spark.catalog.uncacheTable("nonexistantTable") + spark.catalog.uncacheTable("nonexistentTable") }.getMessage assert(e.contains(expectedErrorMsg)) e = intercept[AnalysisException] { - sql("UNCACHE TABLE nonexistantTable") + sql("UNCACHE TABLE nonexistentTable") }.getMessage assert(e.contains(expectedErrorMsg)) - sql("UNCACHE TABLE IF EXISTS nonexistantTable") + sql("UNCACHE TABLE IF EXISTS nonexistentTable") } test("no error on uncache of non-cached table") { From f98c0ad02ea087ae79fef277801d0b71a5019b48 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Tue, 6 Nov 2018 10:39:58 +0800 Subject: [PATCH 231/879] [SPARK-25906][SHELL] Documents '-I' option (from Scala REPL) in spark-shell ## What changes were proposed in this pull request? This PR targets to document `-I` option from Spark 2.4.x (previously `-i` option until Spark 2.3.x). After we upgraded Scala to 2.11.12, `-i` option (`:load`) was replaced to `-I`(SI-7898). Existing `-i` became `:paste` which does not respect Spark's implicit import (for instance `toDF`, symbol as column, etc.). Therefore, `-i` option does not correctly from Spark 2.4.x and it's not documented. I checked other Scala REPL options but looks not applicable or working from quick tests. This PR only targets to document `-I` for now. ## How was this patch tested? Manually tested. **Mac:** ```bash $ ./bin/spark-shell --help Usage: ./bin/spark-shell [options] Scala REPL options: -I preload , enforcing line-by-line interpretation Options: --master MASTER_URL spark://host:port, mesos://host:port, yarn, k8s://https://host:port, or local (Default: local[*]). --deploy-mode DEPLOY_MODE Whether to launch the driver program locally ("client") or on one of the worker machines inside the cluster ("cluster") (Default: client). ... ``` **Windows:** ```cmd C:\...\spark>.\bin\spark-shell --help Usage: .\bin\spark-shell.cmd [options] Scala REPL options: -I preload , enforcing line-by-line interpretation Options: --master MASTER_URL spark://host:port, mesos://host:port, yarn, k8s://https://host:port, or local (Default: local[*]). --deploy-mode DEPLOY_MODE Whether to launch the driver program locally ("client") or on one of the worker machines inside the cluster ("cluster") (Default: client). ... ``` Closes #22919 from HyukjinKwon/SPARK-25906. Authored-by: hyukjinkwon Signed-off-by: hyukjinkwon (cherry picked from commit cc38abc27a671f345e3b4c170977a1976a02a0d0) Signed-off-by: hyukjinkwon --- bin/spark-shell | 5 ++++- bin/spark-shell2.cmd | 8 +++++++- 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/bin/spark-shell b/bin/spark-shell index 421f36cac3d4..e92013797498 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -32,7 +32,10 @@ if [ -z "${SPARK_HOME}" ]; then source "$(dirname "$0")"/find-spark-home fi -export _SPARK_CMD_USAGE="Usage: ./bin/spark-shell [options]" +export _SPARK_CMD_USAGE="Usage: ./bin/spark-shell [options] + +Scala REPL options: + -I preload , enforcing line-by-line interpretation" # SPARK-4161: scala does not assume use of the java classpath, # so we need to add the "-Dscala.usejavacp=true" flag manually. We diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd index aaf71906c652..549bf43bb607 100644 --- a/bin/spark-shell2.cmd +++ b/bin/spark-shell2.cmd @@ -20,7 +20,13 @@ rem rem Figure out where the Spark framework is installed call "%~dp0find-spark-home.cmd" -set _SPARK_CMD_USAGE=Usage: .\bin\spark-shell.cmd [options] +set LF=^ + + +rem two empty lines are required +set _SPARK_CMD_USAGE=Usage: .\bin\spark-shell.cmd [options]^%LF%%LF%^%LF%%LF%^ +Scala REPL options:^%LF%%LF%^ + -I ^ preload ^, enforcing line-by-line interpretation rem SPARK-4161: scala does not assume use of the java classpath, rem so we need to add the "-Dscala.usejavacp=true" flag manually. We From 52e9711d01694158ecb3691f2ec25c0ebe4b0207 Mon Sep 17 00:00:00 2001 From: Dhruve Ashar Date: Tue, 6 Nov 2018 08:25:32 -0600 Subject: [PATCH 232/879] [SPARK-22148][SPARK-15815][SCHEDULER] Acquire new executors to avoid hang because of blacklisting ## What changes were proposed in this pull request? Every time a task is unschedulable because of the condition where no. of task failures < no. of executors available, we currently abort the taskSet - failing the job. This change tries to acquire new executors so that we can complete the job successfully. We try to acquire a new executor only when we can kill an existing idle executor. We fallback to the older implementation where we abort the job if we cannot find an idle executor. ## How was this patch tested? I performed some manual tests to check and validate the behavior. ```scala val rdd = sc.parallelize(Seq(1 to 10), 3) import org.apache.spark.TaskContext val mapped = rdd.mapPartitionsWithIndex ( (index, iterator) => { if (index == 2) { Thread.sleep(30 * 1000); val attemptNum = TaskContext.get.attemptNumber; if (attemptNum < 3) throw new Exception("Fail for blacklisting")}; iterator.toList.map (x => x + " -> " + index).iterator } ) mapped.collect ``` Closes #22288 from dhruve/bug/SPARK-22148. Lead-authored-by: Dhruve Ashar Co-authored-by: Dhruve Ashar Co-authored-by: Tom Graves Signed-off-by: Thomas Graves (cherry picked from commit fdd3bace1da01e5958fe0345c38e889e740ce25e) Signed-off-by: Thomas Graves --- .../spark/internal/config/package.scala | 8 + .../spark/scheduler/BlacklistTracker.scala | 30 ++- .../spark/scheduler/TaskSchedulerImpl.scala | 71 ++++++- .../spark/scheduler/TaskSetManager.scala | 41 ++-- .../scheduler/BlacklistIntegrationSuite.scala | 7 +- .../scheduler/TaskSchedulerImplSuite.scala | 189 +++++++++++++++++- docs/configuration.md | 8 + 7 files changed, 318 insertions(+), 36 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index e7238192b8f1..5836d27be175 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -592,6 +592,14 @@ package object config { .checkValue(v => v > 0, "The value should be a positive time value.") .createWithDefaultString("365d") + private[spark] val UNSCHEDULABLE_TASKSET_TIMEOUT = + ConfigBuilder("spark.scheduler.blacklist.unschedulableTaskSetTimeout") + .doc("The timeout in seconds to wait to acquire a new executor and schedule a task " + + "before aborting a TaskSet which is unschedulable because of being completely blacklisted.") + .timeConf(TimeUnit.SECONDS) + .checkValue(v => v >= 0, "The value should be a non negative time value.") + .createWithDefault(120) + private[spark] val BARRIER_MAX_CONCURRENT_TASKS_CHECK_INTERVAL = ConfigBuilder("spark.scheduler.barrier.maxConcurrentTasksCheck.interval") .doc("Time in seconds to wait between a max concurrent tasks check failure and the next " + diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 980fbbe516b9..ef6d02d85c27 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -146,21 +146,31 @@ private[scheduler] class BlacklistTracker ( nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry) } + private def killExecutor(exec: String, msg: String): Unit = { + allocationClient match { + case Some(a) => + logInfo(msg) + a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false, + force = true) + case None => + logInfo(s"Not attempting to kill blacklisted executor id $exec " + + s"since allocation client is not defined.") + } + } + private def killBlacklistedExecutor(exec: String): Unit = { if (conf.get(config.BLACKLIST_KILL_ENABLED)) { - allocationClient match { - case Some(a) => - logInfo(s"Killing blacklisted executor id $exec " + - s"since ${config.BLACKLIST_KILL_ENABLED.key} is set.") - a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false, - force = true) - case None => - logWarning(s"Not attempting to kill blacklisted executor id $exec " + - s"since allocation client is not defined.") - } + killExecutor(exec, + s"Killing blacklisted executor id $exec since ${config.BLACKLIST_KILL_ENABLED.key} is set.") } } + private[scheduler] def killBlacklistedIdleExecutor(exec: String): Unit = { + killExecutor(exec, + s"Killing blacklisted idle executor id $exec because of task unschedulability and trying " + + "to acquire a new executor.") + } + private def killExecutorsOnBlacklistedNode(node: String): Unit = { if (conf.get(config.BLACKLIST_KILL_ENABLED)) { allocationClient match { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 8b7117066863..24d77f88db98 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -34,7 +34,7 @@ import org.apache.spark.rpc.RpcEndpoint import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} +import org.apache.spark.util.{AccumulatorV2, SystemClock, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. @@ -116,6 +116,11 @@ private[spark] class TaskSchedulerImpl( protected val executorIdToHost = new HashMap[String, String] + private val abortTimer = new Timer(true) + private val clock = new SystemClock + // Exposed for testing + val unschedulableTaskSetToExpiryTime = new HashMap[TaskSetManager, Long] + // Listener object to pass upcalls into var dagScheduler: DAGScheduler = null @@ -414,9 +419,53 @@ private[spark] class TaskSchedulerImpl( launchedAnyTask |= launchedTaskAtCurrentMaxLocality } while (launchedTaskAtCurrentMaxLocality) } + if (!launchedAnyTask) { - taskSet.abortIfCompletelyBlacklisted(hostToExecutors) + taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors).foreach { taskIndex => + // If the taskSet is unschedulable we try to find an existing idle blacklisted + // executor. If we cannot find one, we abort immediately. Else we kill the idle + // executor and kick off an abortTimer which if it doesn't schedule a task within the + // the timeout will abort the taskSet if we were unable to schedule any task from the + // taskSet. + // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per + // task basis. + // Note 2: The taskSet can still be aborted when there are more than one idle + // blacklisted executors and dynamic allocation is on. This can happen when a killed + // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on + // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort + // timer to expire and abort the taskSet. + executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match { + case Some ((executorId, _)) => + if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) { + blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId)) + + val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000 + unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout + logInfo(s"Waiting for $timeout ms for completely " + + s"blacklisted task to be schedulable again before aborting $taskSet.") + abortTimer.schedule( + createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout) + } + case None => // Abort Immediately + logInfo("Cannot schedule any task because of complete blacklisting. No idle" + + s" executors can be found to kill. Aborting $taskSet." ) + taskSet.abortSinceCompletelyBlacklisted(taskIndex) + } + } + } else { + // We want to defer killing any taskSets as long as we have a non blacklisted executor + // which can be used to schedule a task from any active taskSets. This ensures that the + // job can make progress. + // Note: It is theoretically possible that a taskSet never gets scheduled on a + // non-blacklisted executor and the abort timer doesn't kick in because of a constant + // submission of new TaskSets. See the PR for more details. + if (unschedulableTaskSetToExpiryTime.nonEmpty) { + logInfo("Clearing the expiry times for all unschedulable taskSets as a task was " + + "recently scheduled.") + unschedulableTaskSetToExpiryTime.clear() + } } + if (launchedAnyTask && taskSet.isBarrier) { // Check whether the barrier tasks are partially launched. // TODO SPARK-24818 handle the assert failure case (that can happen when some locality @@ -452,6 +501,23 @@ private[spark] class TaskSchedulerImpl( return tasks } + private def createUnschedulableTaskSetAbortTimer( + taskSet: TaskSetManager, + taskIndex: Int): TimerTask = { + new TimerTask() { + override def run() { + if (unschedulableTaskSetToExpiryTime.contains(taskSet) && + unschedulableTaskSetToExpiryTime(taskSet) <= clock.getTimeMillis()) { + logInfo("Cannot schedule any task because of complete blacklisting. " + + s"Wait time for scheduling expired. Aborting $taskSet.") + taskSet.abortSinceCompletelyBlacklisted(taskIndex) + } else { + this.cancel() + } + } + } + } + /** * Shuffle offers around to avoid always placing tasks on the same workers. Exposed to allow * overriding in tests, so it can be deterministic. @@ -587,6 +653,7 @@ private[spark] class TaskSchedulerImpl( barrierCoordinator.stop() } starvationTimer.cancel() + abortTimer.cancel() } override def defaultParallelism(): Int = backend.defaultParallelism() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index d5e85a11cb27..6bf60dd8e9df 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -623,8 +623,8 @@ private[spark] class TaskSetManager( * * It is possible that this taskset has become impossible to schedule *anywhere* due to the * blacklist. The most common scenario would be if there are fewer executors than - * spark.task.maxFailures. We need to detect this so we can fail the task set, otherwise the job - * will hang. + * spark.task.maxFailures. We need to detect this so we can avoid the job from being hung. + * We try to acquire new executor/s by killing an existing idle blacklisted executor. * * There's a tradeoff here: we could make sure all tasks in the task set are schedulable, but that * would add extra time to each iteration of the scheduling loop. Here, we take the approach of @@ -635,9 +635,9 @@ private[spark] class TaskSetManager( * failures (this is because the method picks one unscheduled task, and then iterates through each * executor until it finds one that the task isn't blacklisted on). */ - private[scheduler] def abortIfCompletelyBlacklisted( - hostToExecutors: HashMap[String, HashSet[String]]): Unit = { - taskSetBlacklistHelperOpt.foreach { taskSetBlacklist => + private[scheduler] def getCompletelyBlacklistedTaskIfAny( + hostToExecutors: HashMap[String, HashSet[String]]): Option[Int] = { + taskSetBlacklistHelperOpt.flatMap { taskSetBlacklist => val appBlacklist = blacklistTracker.get // Only look for unschedulable tasks when at least one executor has registered. Otherwise, // task sets will be (unnecessarily) aborted in cases when no executors have registered yet. @@ -658,11 +658,11 @@ private[spark] class TaskSetManager( } } - pendingTask.foreach { indexInTaskSet => + pendingTask.find { indexInTaskSet => // try to find some executor this task can run on. Its possible that some *other* // task isn't schedulable anywhere, but we will discover that in some later call, // when that unschedulable task is the last task remaining. - val blacklistedEverywhere = hostToExecutors.forall { case (host, execsOnHost) => + hostToExecutors.forall { case (host, execsOnHost) => // Check if the task can run on the node val nodeBlacklisted = appBlacklist.isNodeBlacklisted(host) || @@ -679,22 +679,27 @@ private[spark] class TaskSetManager( } } } - if (blacklistedEverywhere) { - val partition = tasks(indexInTaskSet).partitionId - abort(s""" - |Aborting $taskSet because task $indexInTaskSet (partition $partition) - |cannot run anywhere due to node and executor blacklist. - |Most recent failure: - |${taskSetBlacklist.getLatestFailureReason} - | - |Blacklisting behavior can be configured via spark.blacklist.*. - |""".stripMargin) - } } + } else { + None } } } + private[scheduler] def abortSinceCompletelyBlacklisted(indexInTaskSet: Int): Unit = { + taskSetBlacklistHelperOpt.foreach { taskSetBlacklist => + val partition = tasks(indexInTaskSet).partitionId + abort(s""" + |Aborting $taskSet because task $indexInTaskSet (partition $partition) + |cannot run anywhere due to node and executor blacklist. + |Most recent failure: + |${taskSetBlacklist.getLatestFailureReason} + | + |Blacklisting behavior can be configured via spark.blacklist.*. + |""".stripMargin) + } + } + /** * Marks the task as getting result and notifies the DAG Scheduler */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index fe22d70850c7..29bb8232f44f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -96,15 +96,16 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM assertDataStructuresEmpty(noFailure = true) } - // Make sure that if we've failed on all executors, but haven't hit task.maxFailures yet, the job - // doesn't hang + // Make sure that if we've failed on all executors, but haven't hit task.maxFailures yet, we try + // to acquire a new executor and if we aren't able to get one, the job doesn't hang and we abort testScheduler( "SPARK-15865 Progress with fewer executors than maxTaskFailures", extraConfs = Seq( config.BLACKLIST_ENABLED.key -> "true", "spark.testing.nHosts" -> "2", "spark.testing.nExecutorsPerHost" -> "1", - "spark.testing.nCoresPerExecutor" -> "1" + "spark.testing.nCoresPerExecutor" -> "1", + "spark.scheduler.blacklist.unschedulableTaskSetTimeout" -> "0s" ) ) { def runBackend(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 9e1d13e369ad..29172b4664e3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -20,10 +20,12 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer import scala.collection.mutable.HashMap +import scala.concurrent.duration._ import org.mockito.Matchers.{anyInt, anyObject, anyString, eq => meq} import org.mockito.Mockito.{atLeast, atMost, never, spy, times, verify, when} import org.scalatest.BeforeAndAfterEach +import org.scalatest.concurrent.Eventually import org.scalatest.mockito.MockitoSugar import org.apache.spark._ @@ -40,7 +42,7 @@ class FakeSchedulerBackend extends SchedulerBackend { } class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfterEach - with Logging with MockitoSugar { + with Logging with MockitoSugar with Eventually { var failedTaskSetException: Option[Throwable] = None var failedTaskSetReason: String = null @@ -82,10 +84,12 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B setupHelper() } - def setupSchedulerWithMockTaskSetBlacklist(): TaskSchedulerImpl = { + def setupSchedulerWithMockTaskSetBlacklist(confs: (String, String)*): TaskSchedulerImpl = { blacklist = mock[BlacklistTracker] val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite") conf.set(config.BLACKLIST_ENABLED, true) + confs.foreach { case (k, v) => conf.set(k, v) } + sc = new SparkContext(conf) taskScheduler = new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4)) { @@ -466,7 +470,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } } - test("abort stage when all executors are blacklisted") { + test("abort stage when all executors are blacklisted and we cannot acquire new executor") { taskScheduler = setupSchedulerWithMockTaskSetBlacklist() val taskSet = FakeTask.createTaskSet(numTasks = 10, stageAttemptId = 0) taskScheduler.submitTasks(taskSet) @@ -503,6 +507,185 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B verify(tsm).abort(anyString(), anyObject()) } + test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " + + "executor can be acquired") { + // set the abort timer to fail immediately + taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0") + + // We have only 1 task remaining with 1 executor + val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet) + val tsm = stageToMockTaskSetManager(0) + + // submit an offer with one executor + val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten + + // Fail the running task + val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get + taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0)) + // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite + // Reason being - handleFailedTask is run by an executor service and there is a momentary delay + // before it is launched and this fails the assertion check. + tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason) + when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", failedTask.index)).thenReturn(true) + + // make an offer on the blacklisted executor. We won't schedule anything, and set the abort + // timer to kick in immediately + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten.size === 0) + // Wait for the abort timer to kick in. Even though we configure the timeout to be 0, there is a + // slight delay as the abort timer is launched in a separate thread. + eventually(timeout(500.milliseconds)) { + assert(tsm.isZombie) + } + } + + test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") { + taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10") + + // We have only 1 task remaining with 1 executor + val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet) + val tsm = stageToMockTaskSetManager(0) + + // submit an offer with one executor + val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten + + // Fail the running task + val failedTask = firstTaskAttempts.head + taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0)) + // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite + // Reason being - handleFailedTask is run by an executor service and there is a momentary delay + // before it is launched and this fails the assertion check. + tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason) + when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", failedTask.index)).thenReturn(true) + + // make an offer on the blacklisted executor. We won't schedule anything, and set the abort + // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted + // executor and try to acquire a new one. + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten.size === 0) + assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm)) + assert(!tsm.isZombie) + + // Offer a new executor which should be accepted + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor1", "host0", 1) + )).flatten.size === 1) + assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty) + assert(!tsm.isZombie) + } + + // This is to test a scenario where we have two taskSets completely blacklisted and on acquiring + // a new executor we don't want the abort timer for the second taskSet to expire and abort the job + test("SPARK-22148 abort timer should clear unschedulableTaskSetToExpiryTime for all TaskSets") { + taskScheduler = setupSchedulerWithMockTaskSetBlacklist() + + // We have 2 taskSets with 1 task remaining in each with 1 executor completely blacklisted + val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet1) + val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet2) + val tsm = stageToMockTaskSetManager(0) + + // submit an offer with one executor + val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten + + assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty) + + // Fail the running task + val failedTask = firstTaskAttempts.head + taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0)) + tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason) + when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", failedTask.index)).thenReturn(true) + + // make an offer. We will schedule the task from the second taskSet. Since a task was scheduled + // we do not kick off the abort timer for taskSet1 + val secondTaskAttempts = taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten + + assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty) + + val tsm2 = stageToMockTaskSetManager(1) + val failedTask2 = secondTaskAttempts.head + taskScheduler.statusUpdate(failedTask2.taskId, TaskState.FAILED, ByteBuffer.allocate(0)) + tsm2.handleFailedTask(failedTask2.taskId, TaskState.FAILED, UnknownReason) + when(tsm2.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", failedTask2.index)).thenReturn(true) + + // make an offer on the blacklisted executor. We won't schedule anything, and set the abort + // timer for taskSet1 and taskSet2 + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten.size === 0) + assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm)) + assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm2)) + assert(taskScheduler.unschedulableTaskSetToExpiryTime.size == 2) + + // Offer a new executor which should be accepted + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor1", "host1", 1) + )).flatten.size === 1) + + // Check if all the taskSets are cleared + assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty) + + assert(!tsm.isZombie) + } + + // this test is to check that we don't abort a taskSet which is not being scheduled on other + // executors as it is waiting on locality timeout and not being aborted because it is still not + // completely blacklisted. + test("SPARK-22148 Ensure we don't abort the taskSet if we haven't been completely blacklisted") { + taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0", + // This is to avoid any potential flakiness in the test because of large pauses in jenkins + config.LOCALITY_WAIT.key -> "30s" + ) + + val preferredLocation = Seq(ExecutorCacheTaskLocation("host0", "executor0")) + val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0, + preferredLocation) + taskScheduler.submitTasks(taskSet1) + + val tsm = stageToMockTaskSetManager(0) + + // submit an offer with one executor + var taskAttempts = taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten + + // Fail the running task + val failedTask = taskAttempts.head + taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0)) + tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason) + when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", failedTask.index)).thenReturn(true) + + // make an offer but we won't schedule anything yet as scheduler locality is still PROCESS_LOCAL + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor1", "host0", 1) + )).flatten.isEmpty) + + assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty) + + assert(!tsm.isZombie) + } + /** * Helper for performance tests. Takes the explicitly blacklisted nodes and executors; verifies * that the blacklists are used efficiently to ensure scheduling is not O(numPendingTasks). diff --git a/docs/configuration.md b/docs/configuration.md index dfe781513052..d392a6ad5cc1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -1599,6 +1599,14 @@ Apart from these, the following properties are also available, and may be useful driver using more memory. + + spark.scheduler.blacklist.unschedulableTaskSetTimeout + 120s + + The timeout in seconds to wait to acquire a new executor and schedule a task before aborting a + TaskSet which is unschedulable because of being completely blacklisted. + + spark.blacklist.enabled From d6c713fa563987ebc0275dc5e30e5579c2e767ed Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Tue, 6 Nov 2018 23:18:55 +0800 Subject: [PATCH 233/879] [SPARK-25866][ML] Update KMeans formatVersion ## What changes were proposed in this pull request? When we added the `distanceMeasure`, we didn't update the `formatVersion` for `KMeans`. Despite this is not a big issue, as that information is used nowhere, we are returning a wrong information. ## How was this patch tested? NA Closes #22873 from mgaido91/SPARK-25866. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan (cherry picked from commit 6b425874d311146d8fbf7685c1b5d8e97d73b101) Signed-off-by: Wenchen Fan --- .../scala/org/apache/spark/mllib/clustering/KMeansModel.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index d5c8188144ce..b0709547ab1b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -113,7 +113,7 @@ class KMeansModel (@Since("1.0.0") val clusterCenters: Array[Vector], KMeansModel.SaveLoadV2_0.save(sc, this, path) } - override protected def formatVersion: String = "1.0" + override protected def formatVersion: String = "2.0" } @Since("1.4.0") From 4c91b224a42863bab599d0aba4cf1b75da60a4ee Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 6 Nov 2018 14:52:02 -0800 Subject: [PATCH 234/879] [MINOR] update known_translations ## What changes were proposed in this pull request? update known_translations after running `translate-contributors.py` during 2.4.0 release ## How was this patch tested? N/A Closes #22949 from cloud-fan/contributors. Authored-by: Wenchen Fan Signed-off-by: gatorsmile (cherry picked from commit a241a150d52b24ce952efab0830af4c0c9343c1b) Signed-off-by: gatorsmile --- dev/create-release/known_translations | 58 +++++++++++++++++++++++++++ 1 file changed, 58 insertions(+) diff --git a/dev/create-release/known_translations b/dev/create-release/known_translations index 87bf2f220481..65c00cce8c9c 100644 --- a/dev/create-release/known_translations +++ b/dev/create-release/known_translations @@ -203,3 +203,61 @@ shenh062326 - Shen Hong aokolnychyi - Anton Okolnychyi linbojin - Linbo Jin lw-lin - Liwei Lin +10110346 - Xian Liu +Achuth17 - Achuth Narayan Rajagopal +Adamyuanyuan - Adam Wang +DylanGuedes - Dylan Guedes +JiahuiJiang - Jiahui Jiang +KevinZwx - Kevin Zhang +LantaoJin - Lantao Jin +Lemonjing - Rann Tao +LucaCanali - Luca Canali +XD-DENG - Xiaodong Deng +aai95 - Aleksei Izmalkin +akonopko - Alexander Konopko +ankuriitg - Ankur Gupta +arucard21 - Riaas Mokiem +attilapiros - Attila Zsolt Piros +bravo-zhang - Bravo Zhang +caneGuy - Kang Zhou +chaoslawful - Xiaozhe Wang +cluo512 - Chuan Luo +codeatri - Neha Patil +crafty-coder - Carlos Pena +debugger87 - Chaozhong Yang +e-dorigatti - Emilio Dorigatti +eric-maynard - Eric Maynard +felixalbani - Felix Albani +fjh100456 - Jinhua Fu +guoxiaolongzte - Xiaolong Guo +heary-cao - Xuewen Cao +huangweizhe123 - Weizhe Huang +ivoson - Tengfei Huang +jinxing64 - Jin Xing +liu-zhaokun - Zhaokun Liu +liutang123 - Lijia Liu +maropu - Takeshi Yamamuro +maryannxue - Maryann Xue +mcteo - Thomas Dunne +mn-mikke - Marek Novotny +myroslavlisniak - Myroslav Lisniak +npoggi - Nicolas Poggi +pgandhi999 - Parth Gandhi +rimolive - Ricardo Martinelli De Oliveira +sadhen - Darcy Shen +sandeep-katta - Sandeep Katta +seancxmao - Chenxiao Mao +sel - Steve Larkin +shimamoto - Takako Shimamoto +shivusondur - Shivakumar Sondur +skonto - Stavros Kontopoulos +trystanleftwich - Trystan Leftwich +ueshin - Takuya Ueshin +uzmijnlm - Weizhe Huang +xuanyuanking - Yuanjian Li +xubo245 - Bo Xu +xueyumusic - Xue Yu +yanlin-Lynn - Yanlin Wang +yucai - Yucai Yu +zhengruifeng - Ruifeng Zheng +zuotingbing - Tingbing Zuo From 947462f5a36e2751f5a9160c676efbd4e5b08eb4 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Thu, 8 Nov 2018 16:32:25 +0800 Subject: [PATCH 235/879] Revert "[SPARK-23831][SQL] Add org.apache.derby to IsolatedClientLoader" This reverts commit a75571b46f813005a6d4b076ec39081ffab11844. --- .../apache/spark/sql/hive/client/IsolatedClientLoader.scala | 1 - .../apache/spark/sql/hive/HiveExternalCatalogSuite.scala | 6 ------ 2 files changed, 7 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 6a90c44a2633..2f34f69b5cf4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -182,7 +182,6 @@ private[hive] class IsolatedClientLoader( name.startsWith("org.slf4j") || name.startsWith("org.apache.log4j") || // log4j1.x name.startsWith("org.apache.logging.log4j") || // log4j2 - name.startsWith("org.apache.derby.") || name.startsWith("org.apache.spark.") || (sharesHadoopClasses && isHadoopClass) || name.startsWith("scala.") || diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 1de258f06094..0a522b6a11c8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -113,10 +113,4 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { catalog.createDatabase(newDb("dbWithNullDesc").copy(description = null), ignoreIfExists = false) assert(catalog.getDatabase("dbWithNullDesc").description == "") } - - test("SPARK-23831: Add org.apache.derby to IsolatedClientLoader") { - val client1 = HiveUtils.newClientForMetadata(new SparkConf, new Configuration) - val client2 = HiveUtils.newClientForMetadata(new SparkConf, new Configuration) - assert(!client1.equals(client2)) - } } From 47a668c2f03d77078259531ddaccf80b001a8b5c Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Wed, 7 Nov 2018 13:18:52 +0100 Subject: [PATCH 236/879] [SPARK-25904][CORE] Allocate arrays smaller than Int.MaxValue JVMs can't allocate arrays of length exactly Int.MaxValue, so ensure we never try to allocate an array that big. This commit changes some defaults & configs to gracefully fallover to something that doesn't require one large array in some cases; in other cases it simply improves an error message for cases which will still fail. Closes #22818 from squito/SPARK-25827. Authored-by: Imran Rashid Signed-off-by: Imran Rashid (cherry picked from commit 8fbc1830f962c446b915d0d8ff2b13c5c75d22fc) --- .../apache/spark/internal/config/package.scala | 17 ++++++++++------- .../org/apache/spark/storage/DiskStore.scala | 6 ++++-- .../spark/storage/memory/MemoryStore.scala | 7 ++++--- .../spark/util/io/ChunkedByteBuffer.scala | 2 +- .../apache/spark/mllib/linalg/Matrices.scala | 13 +++++++------ .../org/apache/spark/sql/internal/SQLConf.scala | 6 +++--- .../scala/org/apache/spark/sql/Dataset.scala | 7 +++---- 7 files changed, 32 insertions(+), 26 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 5836d27be175..e2162dbb4358 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -387,8 +387,9 @@ package object config { .internal() .doc("The chunk size in bytes during writing out the bytes of ChunkedByteBuffer.") .bytesConf(ByteUnit.BYTE) - .checkValue(_ <= Int.MaxValue, "The chunk size during writing out the bytes of" + - " ChunkedByteBuffer should not larger than Int.MaxValue.") + .checkValue(_ <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, + "The chunk size during writing out the bytes of" + + " ChunkedByteBuffer should not larger than Int.MaxValue - 15.") .createWithDefault(64 * 1024 * 1024) private[spark] val CHECKPOINT_COMPRESS = @@ -459,8 +460,9 @@ package object config { "otherwise specified. These buffers reduce the number of disk seeks and system calls " + "made in creating intermediate shuffle files.") .bytesConf(ByteUnit.KiB) - .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, - s"The file buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, + s"The file buffer size must be greater than 0 and less than" + + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") private[spark] val SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE = @@ -468,8 +470,9 @@ package object config { .doc("The file system for this buffer size after each partition " + "is written in unsafe shuffle writer. In KiB unless otherwise specified.") .bytesConf(ByteUnit.KiB) - .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, - s"The buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, + s"The buffer size must be greater than 0 and less than" + + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") private[spark] val SHUFFLE_DISK_WRITE_BUFFER_SIZE = @@ -580,7 +583,7 @@ package object config { .internal() .doc("For testing only, controls the size of chunks when memory mapping a file") .bytesConf(ByteUnit.BYTE) - .createWithDefault(Int.MaxValue) + .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) private[spark] val BARRIER_SYNC_TIMEOUT = ConfigBuilder("spark.barrier.sync.timeout") diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index 841e16afc754..29963a95cb07 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -33,6 +33,7 @@ import org.apache.spark.internal.{config, Logging} import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.util.{AbstractFileRegion, JavaUtils} import org.apache.spark.security.CryptoStreamUtils +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils import org.apache.spark.util.io.ChunkedByteBuffer @@ -217,7 +218,7 @@ private[spark] class EncryptedBlockData( var remaining = blockSize val chunks = new ListBuffer[ByteBuffer]() while (remaining > 0) { - val chunkSize = math.min(remaining, Int.MaxValue) + val chunkSize = math.min(remaining, ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) val chunk = allocator(chunkSize.toInt) remaining -= chunkSize JavaUtils.readFully(source, chunk) @@ -235,7 +236,8 @@ private[spark] class EncryptedBlockData( // This is used by the block transfer service to replicate blocks. The upload code reads // all bytes into memory to send the block to the remote executor, so it's ok to do this // as long as the block fits in a Java array. - assert(blockSize <= Int.MaxValue, "Block is too large to be wrapped in a byte buffer.") + assert(blockSize <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, + "Block is too large to be wrapped in a byte buffer.") val dst = ByteBuffer.allocate(blockSize.toInt) val in = open() try { diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 06fd56e54d9c..8513359934be 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -34,6 +34,7 @@ import org.apache.spark.memory.{MemoryManager, MemoryMode} import org.apache.spark.serializer.{SerializationStream, SerializerManager} import org.apache.spark.storage._ import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.{SizeEstimator, Utils} import org.apache.spark.util.collection.SizeTrackingVector import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} @@ -333,11 +334,11 @@ private[spark] class MemoryStore( // Initial per-task memory to request for unrolling blocks (bytes). val initialMemoryThreshold = unrollMemoryThreshold - val chunkSize = if (initialMemoryThreshold > Int.MaxValue) { + val chunkSize = if (initialMemoryThreshold > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { logWarning(s"Initial memory threshold of ${Utils.bytesToString(initialMemoryThreshold)} " + s"is too large to be set as chunk size. Chunk size has been capped to " + - s"${Utils.bytesToString(Int.MaxValue)}") - Int.MaxValue + s"${Utils.bytesToString(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH)}") + ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH } else { initialMemoryThreshold.toInt } diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala index da2be84723a0..870830fff4c3 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala @@ -97,7 +97,7 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { * @throws UnsupportedOperationException if this buffer's size exceeds the maximum array size. */ def toArray: Array[Byte] = { - if (size >= Integer.MAX_VALUE) { + if (size >= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { throw new UnsupportedOperationException( s"cannot call toArray because buffer size ($size bytes) exceeds maximum array size") } 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 bf9b4cfe15b2..e474cfa002fa 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 @@ -30,6 +30,7 @@ import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.array.ByteArrayMethods /** * Trait for a local matrix. @@ -456,7 +457,7 @@ object DenseMatrix { */ @Since("1.3.0") def zeros(numRows: Int, numCols: Int): DenseMatrix = { - require(numRows.toLong * numCols <= Int.MaxValue, + require(numRows.toLong * numCols <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols)) } @@ -469,7 +470,7 @@ object DenseMatrix { */ @Since("1.3.0") def ones(numRows: Int, numCols: Int): DenseMatrix = { - require(numRows.toLong * numCols <= Int.MaxValue, + require(numRows.toLong * numCols <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0)) } @@ -499,7 +500,7 @@ object DenseMatrix { */ @Since("1.3.0") def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { - require(numRows.toLong * numCols <= Int.MaxValue, + require(numRows.toLong * numCols <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble())) } @@ -513,7 +514,7 @@ object DenseMatrix { */ @Since("1.3.0") def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { - require(numRows.toLong * numCols <= Int.MaxValue, + require(numRows.toLong * numCols <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian())) } @@ -846,8 +847,8 @@ object SparseMatrix { s"density must be a double in the range 0.0 <= d <= 1.0. Currently, density: $density") val size = numRows.toLong * numCols val expected = size * density - assert(expected < Int.MaxValue, - "The expected number of nonzeros cannot be greater than Int.MaxValue.") + assert(expected < ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, + "The expected number of nonzeros cannot be greater than Int.MaxValue - 15.") val nnz = math.ceil(expected).toInt if (density == 0.0) { new SparseMatrix(numRows, numCols, new Array[Int](numCols + 1), Array.empty, Array.empty) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 08def90cdcd0..99e601ad9f4c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -27,7 +27,6 @@ import scala.collection.immutable import scala.util.matching.Regex import org.apache.hadoop.fs.Path -import org.tukaani.xz.LZMA2Options import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.internal.Logging @@ -36,6 +35,7 @@ import org.apache.spark.network.util.ByteUnit import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -1208,7 +1208,7 @@ object SQLConf { .doc("Threshold for number of rows guaranteed to be held in memory by the sort merge " + "join operator") .intConf - .createWithDefault(Int.MaxValue) + .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) val SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD = buildConf("spark.sql.sortMergeJoinExec.buffer.spill.threshold") @@ -1442,7 +1442,7 @@ object SQLConf { "'SELECT x FROM t ORDER BY y LIMIT m', if m is under this threshold, do a top-K sort" + " in memory, otherwise do a global sort which spills to disk if necessary.") .intConf - .createWithDefault(Int.MaxValue) + .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala index fa14aa14ee96..13e6a83d7a56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Dataset.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql import java.io.CharArrayWriter -import java.sql.{Date, Timestamp} import scala.collection.JavaConverters._ import scala.language.implicitConversions @@ -46,7 +45,6 @@ import org.apache.spark.sql.catalyst.parser.{ParseException, ParserUtils} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection} -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.arrow.{ArrowBatchStreamWriter, ArrowConverters} import org.apache.spark.sql.execution.command._ @@ -57,6 +55,7 @@ import org.apache.spark.sql.streaming.DataStreamWriter import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils import org.apache.spark.storage.StorageLevel +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.Utils @@ -287,7 +286,7 @@ class Dataset[T] private[sql]( _numRows: Int, truncate: Int = 20, vertical: Boolean = false): String = { - val numRows = _numRows.max(0).min(Int.MaxValue - 1) + val numRows = _numRows.max(0).min(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH - 1) // Get rows represented by Seq[Seq[String]], we may get one more line if it has more data. val tmpRows = getRows(numRows, truncate) @@ -3264,7 +3263,7 @@ class Dataset[T] private[sql]( _numRows: Int, truncate: Int): Array[Any] = { EvaluatePython.registerPicklers() - val numRows = _numRows.max(0).min(Int.MaxValue - 1) + val numRows = _numRows.max(0).min(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH - 1) val rows = getRows(numRows, truncate).map(_.toArray).toArray val toJava: (Any) => Any = EvaluatePython.toJava(_, ArrayType(ArrayType(StringType))) val iter: Iterator[Array[Byte]] = new SerDeUtil.AutoBatchedPickler( From d4b1da839e16c2a1752d6020f3470b4fbe94843b Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 8 Nov 2018 23:58:37 +0800 Subject: [PATCH 237/879] hot fix: add missing import --- .../main/scala/org/apache/spark/internal/config/package.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index e2162dbb4358..bde0995cd008 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -21,6 +21,7 @@ import java.util.concurrent.TimeUnit import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.ByteUnit +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils package object config { From 8b18dc00725c19cd7d3cdc6752f4a4137d5af350 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 8 Nov 2018 15:49:36 -0800 Subject: [PATCH 238/879] [MINOR] update HiveExternalCatalogVersionsSuite to test 2.4.0 ## What changes were proposed in this pull request? Since Spark 2.4.0 is released, we should test it in HiveExternalCatalogVersionsSuite ## How was this patch tested? N/A Closes #22984 from cloud-fan/minor. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun (cherry picked from commit 973f7c01df0788b6f5d21224d96c33f14c5b8c64) Signed-off-by: Dongjoon Hyun --- .../spark/sql/hive/HiveExternalCatalogVersionsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index 46b66c14d0fb..cce2f203895f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -203,7 +203,7 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { object PROCESS_TABLES extends QueryTest with SQLTestUtils { // Tests the latest version of every release line. - val testingVersions = Seq("2.1.3", "2.2.2", "2.3.2") + val testingVersions = Seq("2.2.2", "2.3.2", "2.4.0") protected var spark: SparkSession = _ From bb58a9727bebe323748231b3431111baa3510269 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Fri, 9 Nov 2018 08:22:26 -0800 Subject: [PATCH 239/879] [SPARK-25988][SQL] Keep names unchanged when deduplicating the column names in Analyzer ## What changes were proposed in this pull request? When the queries do not use the column names with the same case, users might hit various errors. Below is a typical test failure they can hit. ``` Expected only partition pruning predicates: ArrayBuffer(isnotnull(tdate#237), (cast(tdate#237 as string) >= 2017-08-15)); org.apache.spark.sql.AnalysisException: Expected only partition pruning predicates: ArrayBuffer(isnotnull(tdate#237), (cast(tdate#237 as string) >= 2017-08-15)); at org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils$.prunePartitionsByFilter(ExternalCatalogUtils.scala:146) at org.apache.spark.sql.catalyst.catalog.InMemoryCatalog.listPartitionsByFilter(InMemoryCatalog.scala:560) at org.apache.spark.sql.catalyst.catalog.SessionCatalog.listPartitionsByFilter(SessionCatalog.scala:925) ``` ## How was this patch tested? Added two test cases. Closes #22990 from gatorsmile/fix1283. Authored-by: gatorsmile Signed-off-by: gatorsmile (cherry picked from commit 657fd00b5204859c2e6d7c19a71a3ec5ecf7c869) Signed-off-by: gatorsmile --- .../sql/catalyst/analysis/Analyzer.scala | 3 +- .../sql/catalyst/analysis/unresolved.scala | 1 + .../expressions/namedExpressions.scala | 5 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 53 +++++++++++++++++++ 4 files changed, 60 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 4a83067bd896..cb4b3b876206 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -824,7 +824,8 @@ class Analyzer( } private def dedupAttr(attr: Attribute, attrMap: AttributeMap[Attribute]): Attribute = { - attrMap.get(attr).getOrElse(attr).withQualifier(attr.qualifier) + val exprId = attrMap.getOrElse(attr, attr).exprId + attr.withExprId(exprId) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 857cf382b8f2..36cad3cf7478 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -112,6 +112,7 @@ case class UnresolvedAttribute(nameParts: Seq[String]) extends Attribute with Un override def withQualifier(newQualifier: Seq[String]): UnresolvedAttribute = this override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute.quoted(newName) override def withMetadata(newMetadata: Metadata): Attribute = this + override def withExprId(newExprId: ExprId): UnresolvedAttribute = this override def toString: String = s"'$name" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala index 584a2946bd56..049ea7769139 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -115,6 +115,7 @@ abstract class Attribute extends LeafExpression with NamedExpression with NullIn def withQualifier(newQualifier: Seq[String]): Attribute def withName(newName: String): Attribute def withMetadata(newMetadata: Metadata): Attribute + def withExprId(newExprId: ExprId): Attribute override def toAttribute: Attribute = this def newInstance(): Attribute @@ -299,7 +300,7 @@ case class AttributeReference( } } - def withExprId(newExprId: ExprId): AttributeReference = { + override def withExprId(newExprId: ExprId): AttributeReference = { if (exprId == newExprId) { this } else { @@ -362,6 +363,8 @@ case class PrettyAttribute( throw new UnsupportedOperationException override def qualifier: Seq[String] = throw new UnsupportedOperationException override def exprId: ExprId = throw new UnsupportedOperationException + override def withExprId(newExprId: ExprId): Attribute = + throw new UnsupportedOperationException override def nullable: Boolean = true } 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 631ab1b7ece7..dbb0790a4682 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 @@ -2856,6 +2856,59 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkAnswer(sql("select 26393499451 / (1e6 * 1000)"), Row(BigDecimal("26.3934994510000"))) } } + + test("SPARK-25988: self join with aliases on partitioned tables #1") { + withTempView("tmpView1", "tmpView2") { + withTable("tab1", "tab2") { + sql( + """ + |CREATE TABLE `tab1` (`col1` INT, `TDATE` DATE) + |USING CSV + |PARTITIONED BY (TDATE) + """.stripMargin) + spark.table("tab1").where("TDATE >= '2017-08-15'").createOrReplaceTempView("tmpView1") + sql("CREATE TABLE `tab2` (`TDATE` DATE) USING parquet") + sql( + """ + |CREATE OR REPLACE TEMPORARY VIEW tmpView2 AS + |SELECT N.tdate, col1 AS aliasCol1 + |FROM tmpView1 N + |JOIN tab2 Z + |ON N.tdate = Z.tdate + """.stripMargin) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") { + sql("SELECT * FROM tmpView2 x JOIN tmpView2 y ON x.tdate = y.tdate").collect() + } + } + } + } + + test("SPARK-25988: self join with aliases on partitioned tables #2") { + withTempView("tmp") { + withTable("tab1", "tab2") { + sql( + """ + |CREATE TABLE `tab1` (`EX` STRING, `TDATE` DATE) + |USING parquet + |PARTITIONED BY (tdate) + """.stripMargin) + sql("CREATE TABLE `tab2` (`TDATE` DATE) USING parquet") + sql( + """ + |CREATE OR REPLACE TEMPORARY VIEW TMP as + |SELECT N.tdate, EX AS new_ex + |FROM tab1 N + |JOIN tab2 Z + |ON N.tdate = Z.tdate + """.stripMargin) + sql( + """ + |SELECT * FROM TMP x JOIN TMP y + |ON x.tdate = y.tdate + """.stripMargin).queryExecution.executedPlan + } + } + } } case class Foo(bar: Option[String]) From 1375f3477149b9e38bf5cfaa1b62c0a23e71bbd9 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 9 Nov 2018 09:44:04 -0800 Subject: [PATCH 240/879] [SPARK-25979][SQL] Window function: allow parentheses around window reference ## What changes were proposed in this pull request? Very minor parser bug, but possibly problematic for code-generated queries: Consider the following two queries: ``` SELECT avg(k) OVER (w) FROM kv WINDOW w AS (PARTITION BY v ORDER BY w) ORDER BY 1 ``` and ``` SELECT avg(k) OVER w FROM kv WINDOW w AS (PARTITION BY v ORDER BY w) ORDER BY 1 ``` The former, with parens around the OVER condition, fails to parse while the latter, without parens, succeeds: ``` Error in SQL statement: ParseException: mismatched input '(' expecting {, ',', 'FROM', 'WHERE', 'GROUP', 'ORDER', 'HAVING', 'LIMIT', 'LATERAL', 'WINDOW', 'UNION', 'EXCEPT', 'MINUS', 'INTERSECT', 'SORT', 'CLUSTER', 'DISTRIBUTE'}(line 1, pos 19) == SQL == SELECT avg(k) OVER (w) FROM kv WINDOW w AS (PARTITION BY v ORDER BY w) ORDER BY 1 -------------------^^^ ``` This was found when running the cockroach DB tests. I tried PostgreSQL, The SQL with parentheses is also workable. ## How was this patch tested? Unit test Closes #22987 from gengliangwang/windowParentheses. Authored-by: Gengliang Wang Signed-off-by: gatorsmile (cherry picked from commit 1db799795cf3c15798fbfb6043ec5775e16ba5ea) Signed-off-by: gatorsmile --- .../spark/sql/catalyst/parser/SqlBase.g4 | 1 + .../resources/sql-tests/inputs/window.sql | 6 ++++++ .../sql-tests/results/window.sql.out | 19 ++++++++++++++++++- 3 files changed, 25 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 index 94283f59011a..0792a7b7eff5 100644 --- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 +++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 @@ -690,6 +690,7 @@ namedWindow windowSpec : name=identifier #windowRef + | '('name=identifier')' #windowRef | '(' ( CLUSTER BY partition+=expression (',' partition+=expression)* | ((PARTITION | DISTRIBUTE) BY partition+=expression (',' partition+=expression)*)? diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql index cda4db4b449f..faab4c61c864 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -109,3 +109,9 @@ last_value(false, false) OVER w AS last_value_contain_null FROM testData WINDOW w AS () ORDER BY cate, val; + +-- parentheses around window reference +SELECT cate, sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val); diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 4afbcd62853d..8190e21129b5 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 22 +-- Number of queries: 23 -- !query 0 @@ -363,3 +363,20 @@ NULL a false true false false true false 1 b false true false false true false 2 b false true false false true false 3 b false true false false true false + + +-- !query 22 +SELECT cate, sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val) +-- !query 22 schema +struct +-- !query 22 output +NULL 3 +a 2 +a 2 +a 4 +b 1 +b 3 +b 6 From 3bc4c3330f8da2979ce034c205bc3d0bed5f39f8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 13 Nov 2018 10:28:25 +0800 Subject: [PATCH 241/879] [SPARK-26029][BUILD][2.4] Bump previousSparkVersion in MimaBuild.scala to be 2.3.0 ## What changes were proposed in this pull request? Although it's a little late, we should still update mima for branch 2.4, to avoid future breaking changes. Note that, when merging, we should forward port it to master branch, so that the excluding rules are still in `v24excludes`. TODO: update the release process document to mention about mima update. ## How was this patch tested? N/A Closes #23015 from cloud-fan/mima-2.4. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan --- project/MimaBuild.scala | 2 +- project/MimaExcludes.scala | 45 +++++++++++++++++++++++++++++++++++++- 2 files changed, 45 insertions(+), 2 deletions(-) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index adde213e361f..fbf9b8e326d5 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -88,7 +88,7 @@ object MimaBuild { def mimaSettings(sparkHome: File, projectRef: ProjectRef) = { val organization = "org.apache.spark" - val previousSparkVersion = "2.2.0" + val previousSparkVersion = "2.3.0" val project = projectRef.project val fullId = "spark-" + project + "_2.11" mimaDefaultSettings ++ diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index b7e9cbc077c1..4246355f088e 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -105,7 +105,50 @@ object MimaExcludes { ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasValidationIndicatorCol.validationIndicatorCol"), // [SPARK-23042] Use OneHotEncoderModel to encode labels in MultilayerPerceptronClassifier - ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.classification.LabelConverter") + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.ml.classification.LabelConverter"), + + // [SPARK-21842][MESOS] Support Kerberos ticket renewal and creation in Mesos + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkHadoopUtil.getDateOfNextUpdate"), + + // [SPARK-23366] Improve hot reading path in ReadAheadInputStream + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.io.ReadAheadInputStream.this"), + + // [SPARK-22941][CORE] Do not exit JVM when submit fails with in-process launcher. + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.addJarToClasspath"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.mergeFileLists"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.deploy.SparkSubmit.prepareSubmitEnvironment$default$2"), + + // Data Source V2 API changes + // TODO: they are unstable APIs and should not be tracked by mima. + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.ReadSupportWithSchema"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanColumnarBatch.createDataReaderFactories"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanColumnarBatch.createBatchDataReaderFactories"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanColumnarBatch.planBatchInputPartitions"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.SupportsScanUnsafeRow"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.DataSourceReader.createDataReaderFactories"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.DataSourceReader.planInputPartitions"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.SupportsPushDownCatalystFilters"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.DataReader"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsReportStatistics.getStatistics"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.reader.SupportsReportStatistics.estimateStatistics"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.DataReaderFactory"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.reader.streaming.ContinuousDataReader"), + ProblemFilters.exclude[MissingClassProblem]("org.apache.spark.sql.sources.v2.writer.SupportsWriteInternalRow"), + ProblemFilters.exclude[DirectMissingMethodProblem]("org.apache.spark.sql.sources.v2.writer.DataWriterFactory.createDataWriter"), + ProblemFilters.exclude[ReversedMissingMethodProblem]("org.apache.spark.sql.sources.v2.writer.DataWriterFactory.createDataWriter"), + + // Changes to HasRawPredictionCol. + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasRawPredictionCol.rawPredictionCol"), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasRawPredictionCol.org$apache$spark$ml$param$shared$HasRawPredictionCol$_setter_$rawPredictionCol_="), + ProblemFilters.exclude[InheritedNewAbstractMethodProblem]("org.apache.spark.ml.param.shared.HasRawPredictionCol.getRawPredictionCol"), + + // [SPARK-15526][ML][FOLLOWUP] Make JPMML provided scope to avoid including unshaded JARs + (problem: Problem) => problem match { + case MissingClassProblem(cls) => + !cls.fullName.startsWith("org.spark_project.jpmml") && + !cls.fullName.startsWith("org.spark_project.dmg.pmml") + case _ => true + } ) // Exclude rules for 2.3.x From 65e5b26590e66ac4220b5f60e11b7966746c8b08 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Mon, 12 Nov 2018 19:03:30 -0800 Subject: [PATCH 242/879] [SPARK-26010][R] fix vignette eval with Java 11 ## What changes were proposed in this pull request? changes in vignette only to disable eval ## How was this patch tested? Jenkins Author: Felix Cheung Closes #23007 from felixcheung/rjavavervig. (cherry picked from commit 88c82627267a9731b2438f0cc28dd656eb3dc834) Signed-off-by: Felix Cheung --- R/pkg/vignettes/sparkr-vignettes.Rmd | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd index 090363c5f8a3..b13f338d2163 100644 --- a/R/pkg/vignettes/sparkr-vignettes.Rmd +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -57,6 +57,20 @@ First, let's load and attach the package. library(SparkR) ``` +```{r, include=FALSE} +# disable eval if java version not supported +override_eval <- tryCatch(!is.numeric(SparkR:::checkJavaVersion()), + error = function(e) { TRUE }, + warning = function(e) { TRUE }) + +if (override_eval) { + opts_hooks$set(eval = function(options) { + options$eval = FALSE + options + }) +} +``` + `SparkSession` is the entry point into SparkR which connects your R program to a Spark cluster. You can create a `SparkSession` using `sparkR.session` and pass in options such as the application name, any Spark packages depended on, etc. We use default settings in which it runs in local mode. It auto downloads Spark package in the background if no previous installation is found. For more details about setup, see [Spark Session](#SetupSparkSession). From e2e1f0ad87117c5434a99925a73bc7910fa305fc Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 13 Nov 2018 17:05:39 +0800 Subject: [PATCH 243/879] [SPARK-25921][PYSPARK] Fix barrier task run without BarrierTaskContext while python worker reuse ## What changes were proposed in this pull request? Running a barrier job after a normal spark job causes the barrier job to run without a BarrierTaskContext. This is because while python worker reuse, BarrierTaskContext._getOrCreate() will still return a TaskContext after firstly submit a normal spark job, we'll get a `AttributeError: 'TaskContext' object has no attribute 'barrier'`. Fix this by adding check logic in BarrierTaskContext._getOrCreate() and make sure it will return BarrierTaskContext in this scenario. ## How was this patch tested? Add new UT in pyspark-core. Closes #22962 from xuanyuanking/SPARK-25921. Authored-by: Yuanjian Li Signed-off-by: Wenchen Fan (cherry picked from commit c00e72f3d7530eb2ae43d4d45e8efde783daf6ff) Signed-off-by: Wenchen Fan --- python/pyspark/taskcontext.py | 4 ++-- python/pyspark/tests.py | 15 +++++++++++++++ 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/python/pyspark/taskcontext.py b/python/pyspark/taskcontext.py index b61643eb0a16..98b505c9046b 100644 --- a/python/pyspark/taskcontext.py +++ b/python/pyspark/taskcontext.py @@ -147,8 +147,8 @@ def __init__(self): @classmethod def _getOrCreate(cls): """Internal function to get or create global BarrierTaskContext.""" - if cls._taskContext is None: - cls._taskContext = BarrierTaskContext() + if not isinstance(cls._taskContext, BarrierTaskContext): + cls._taskContext = object.__new__(cls) return cls._taskContext @classmethod diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 050c2dd01836..131c51e108ca 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -614,6 +614,21 @@ def context_barrier(x): times = rdd.barrier().mapPartitions(f).map(context_barrier).collect() self.assertTrue(max(times) - min(times) < 1) + def test_barrier_with_python_worker_reuse(self): + """ + Verify that BarrierTaskContext.barrier() with reused python worker. + """ + self.sc._conf.set("spark.python.work.reuse", "true") + rdd = self.sc.parallelize(range(4), 4) + # start a normal job first to start all worker + result = rdd.map(lambda x: x ** 2).collect() + self.assertEqual([0, 1, 4, 9], result) + # make sure `spark.python.work.reuse=true` + self.assertEqual(self.sc._conf.get("spark.python.work.reuse"), "true") + + # worker will be reused in this barrier job + self.test_barrier() + def test_barrier_infos(self): """ Verify that BarrierTaskContext.getTaskInfos() returns a list of all task infos in the From ca426bfa56045f01de0ea14480a375753073e025 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 14 Nov 2018 10:19:20 -0800 Subject: [PATCH 244/879] [SPARK-26042][SS][TESTS] Fix a potential hang in KafkaContinuousSourceTopicDeletionSuite ## What changes were proposed in this pull request? As initializing lazy vals shares the same lock, a thread is trying to initialize `executedPlan` when `isRDD` is running, this thread will hang forever. This PR just materializes `executedPlan` so that accessing it when `toRdd` is running doesn't need to wait for a lock ## How was this patch tested? Jenkins Closes #23023 from zsxwing/SPARK-26042. Authored-by: Shixiong Zhu Signed-off-by: Shixiong Zhu (cherry picked from commit 4035c98a0c03cf61d1fb9a9916df513ab1081a9b) Signed-off-by: Shixiong Zhu --- .../streaming/continuous/ContinuousExecution.scala | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index f104422b868f..2e24fa6bb8ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -259,7 +259,12 @@ class ContinuousExecution( reportTimeTaken("runContinuous") { SQLExecution.withNewExecutionId( - sparkSessionForQuery, lastExecution)(lastExecution.toRdd) + sparkSessionForQuery, lastExecution) { + // Materialize `executedPlan` so that accessing it when `toRdd` is running doesn't need to + // wait for a lock + lastExecution.executedPlan + lastExecution.toRdd + } } } catch { case t: Throwable From ba638a783442f6a5b7b8e0a363edfb398eb2b6c7 Mon Sep 17 00:00:00 2001 From: gss2002 Date: Wed, 14 Nov 2018 13:02:13 -0800 Subject: [PATCH 245/879] =?UTF-8?q?[SPARK-25778]=20WriteAheadLogBackedBloc?= =?UTF-8?q?kRDD=20in=20YARN=20Cluster=20Mode=20Fails=20=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …due lack of access to tmpDir from $PWD to HDFS WriteAheadLogBackedBlockRDD usage of java.io.tmpdir will fail if $PWD resolves to a folder in HDFS and the Spark YARN Cluster job does not have the correct access to this folder in regards to the dummy folder. So this patch provides an option to set spark.streaming.receiver.blockStore.tmpdir to override java.io.tmpdir which sets $PWD from YARN Cluster mode. ## What changes were proposed in this pull request? This change provides an option to override the java.io.tmpdir option so that when $PWD is resolved in YARN Cluster mode Spark does not attempt to use this folder and instead use the folder provided with the following option: spark.streaming.receiver.blockStore.tmpdir ## How was this patch tested? Patch was manually tested on a Spark Streaming Job with Write Ahead logs in Cluster mode. Closes #22867 from gss2002/SPARK-25778. Authored-by: gss2002 Signed-off-by: Marcelo Vanzin (cherry picked from commit 2b671e729250b980aa9e4ea2d483f44fa0e129cb) Signed-off-by: Marcelo Vanzin --- .../spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index 844760ab61d2..f677c492d561 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -136,7 +136,7 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( // this dummy directory should not already exist otherwise the WAL will try to recover // past events from the directory and throw errors. val nonExistentDirectory = new File( - System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString).getAbsolutePath + System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString).toURI.toString writeAheadLog = WriteAheadLogUtils.createLogForReceiver( SparkEnv.get.conf, nonExistentDirectory, hadoopConf) dataRead = writeAheadLog.read(partition.walRecordHandle) From aaa21d812593d7c85d4c794e38d77cef8ecaeb6b Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Thu, 15 Nov 2018 20:09:53 +0800 Subject: [PATCH 246/879] [SPARK-26057][SQL] Transform also analyzed plans when dedup references ## What changes were proposed in this pull request? In SPARK-24865 `AnalysisBarrier` was removed and in order to improve resolution speed, the `analyzed` flag was (re-)introduced in order to process only plans which are not yet analyzed. This should not be the case when performing attribute deduplication as in that case we need to transform also the plans which were already analyzed, otherwise we can miss to rewrite some attributes leading to invalid plans. ## How was this patch tested? added UT Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23035 from mgaido91/SPARK-26057. Authored-by: Marco Gaido Signed-off-by: Wenchen Fan (cherry picked from commit b46f75a5af372422de0f8e07ff920fa6ccd33c7e) Signed-off-by: Wenchen Fan --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../org/apache/spark/sql/DataFrameSuite.scala | 25 +++++++++++++++++++ 2 files changed, 26 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index cb4b3b876206..b9ab3363197d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -871,7 +871,7 @@ class Analyzer( private def dedupOuterReferencesInSubquery( plan: LogicalPlan, attrMap: AttributeMap[Attribute]): LogicalPlan = { - plan resolveOperatorsDown { case currentFragment => + plan transformDown { case currentFragment => currentFragment transformExpressions { case OuterReference(a: Attribute) => OuterReference(dedupAttr(a, attrMap)) 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 2ca0e5f12f2a..5075209d7454 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 @@ -2597,4 +2597,29 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer(swappedDf.filter($"key"($"map") > "a"), Row(2, Map(2 -> "b"))) } + + test("SPARK-26057: attribute deduplication on already analyzed plans") { + withTempView("a", "b", "v") { + val df1 = Seq(("1-1", 6)).toDF("id", "n") + df1.createOrReplaceTempView("a") + val df3 = Seq("1-1").toDF("id") + df3.createOrReplaceTempView("b") + spark.sql( + """ + |SELECT a.id, n as m + |FROM a + |WHERE EXISTS( + | SELECT 1 + | FROM b + | WHERE b.id = a.id) + """.stripMargin).createOrReplaceTempView("v") + val res = spark.sql( + """ + |SELECT a.id, n, m + | FROM a + | LEFT OUTER JOIN v ON v.id = a.id + """.stripMargin) + checkAnswer(res, Row("1-1", 6, 6)) + } + } } From 96834fb7762dcd9d20330e6640b8bbd34ea31be5 Mon Sep 17 00:00:00 2001 From: Shanyu Zhao Date: Thu, 15 Nov 2018 10:30:16 -0600 Subject: [PATCH 247/879] [SPARK-26011][SPARK-SUBMIT] Yarn mode pyspark app without python main resource does not honor "spark.jars.packages" SparkSubmit determines pyspark app by the suffix of primary resource but Livy uses "spark-internal" as the primary resource when calling spark-submit, therefore args.isPython is set to false in SparkSubmit.scala. In Yarn mode, SparkSubmit module is responsible for resolving maven coordinates and adding them to "spark.submit.pyFiles" so that python's system path can be set correctly. The fix is to resolve maven coordinates not only when args.isPython is true, but also when primary resource is spark-internal. Tested the patch with Livy submitting pyspark app, spark-submit, pyspark with or without packages config. Signed-off-by: Shanyu Zhao Closes #23009 from shanyu/shanyu-26011. Authored-by: Shanyu Zhao Signed-off-by: Sean Owen (cherry picked from commit 9a5fda60e532dc7203d21d5fbe385cd561906ccb) Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 1d32d964dc9d..8ba36c68334b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -318,7 +318,7 @@ private[spark] class SparkSubmit extends Logging { if (!StringUtils.isBlank(resolvedMavenCoordinates)) { args.jars = mergeFileLists(args.jars, resolvedMavenCoordinates) - if (args.isPython) { + if (args.isPython || isInternal(args.primaryResource)) { args.pyFiles = mergeFileLists(args.pyFiles, resolvedMavenCoordinates) } } From 6148a77a5da9ca33fb115269f1cba29cddfc652e Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 16 Nov 2018 08:35:00 +0800 Subject: [PATCH 248/879] [SPARK-25883][BACKPORT][SQL][MINOR] Override method `prettyName` in `from_avro`/`to_avro` Back port https://github.com/apache/spark/pull/22890 to branch-2.4. It is a bug fix for this issue: https://issues.apache.org/jira/browse/SPARK-26063 ## What changes were proposed in this pull request? Previously in from_avro/to_avro, we override the method `simpleString` and `sql` for the string output. However, the override only affects the alias naming: ``` Project [from_avro('col, ... , (mode,PERMISSIVE)) AS from_avro(col, struct, Map(mode -> PERMISSIVE))#11] ``` It only makes the alias name quite long: `from_avro(col, struct, Map(mode -> PERMISSIVE))`). We should follow `from_csv`/`from_json` here, to override the method prettyName only, and we will get a clean alias name ``` ... AS from_avro(col)#11 ``` ## How was this patch tested? Manual check Closes #23047 from gengliangwang/backport_avro_pretty_name. Authored-by: Gengliang Wang Signed-off-by: hyukjinkwon --- .../org/apache/spark/sql/avro/AvroDataToCatalyst.scala | 8 +------- .../org/apache/spark/sql/avro/CatalystDataToAvro.scala | 8 +------- 2 files changed, 2 insertions(+), 14 deletions(-) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala index 915769fa708b..8641b9ffad8b 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -51,13 +51,7 @@ case class AvroDataToCatalyst(child: Expression, jsonFormatSchema: String) deserializer.deserialize(result) } - override def simpleString: String = { - s"from_avro(${child.sql}, ${dataType.simpleString})" - } - - override def sql: String = { - s"from_avro(${child.sql}, ${dataType.catalogString})" - } + override def prettyName: String = "from_avro" override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val expr = ctx.addReferenceObj("this", this) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala index 141ff3782adf..6ed330d92f5e 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala @@ -52,13 +52,7 @@ case class CatalystDataToAvro(child: Expression) extends UnaryExpression { out.toByteArray } - override def simpleString: String = { - s"to_avro(${child.sql}, ${child.dataType.simpleString})" - } - - override def sql: String = { - s"to_avro(${child.sql}, ${child.dataType.catalogString})" - } + override def prettyName: String = "to_avro" override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val expr = ctx.addReferenceObj("this", this) From 2d67be932181e1b6e076393d5b1f9d01726988aa Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 16 Nov 2018 12:46:57 +0800 Subject: [PATCH 249/879] [SPARK-23207][SQL][FOLLOW-UP] Use `SQLConf.get.enableRadixSort` instead of `SparkEnv.get.conf.get(SQLConf.RADIX_SORT_ENABLED)`. ## What changes were proposed in this pull request? This is a follow-up of #20393. We should read the conf `"spark.sql.sort.enableRadixSort"` from `SQLConf` instead of `SparkConf`, i.e., use `SQLConf.get.enableRadixSort` instead of `SparkEnv.get.conf.get(SQLConf.RADIX_SORT_ENABLED)`, otherwise the config is never read. ## How was this patch tested? Existing tests. Closes #23046 from ueshin/issues/SPARK-23207/conf. Authored-by: Takuya UESHIN Signed-off-by: Wenchen Fan (cherry picked from commit dad2d826ae9138f06751e5d092531a9e06028c21) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/exchange/ShuffleExchangeExec.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index aba94885f941..64a2be86e924 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -269,7 +269,7 @@ object ShuffleExchangeExec { } // The comparator for comparing row hashcode, which should always be Integer. val prefixComparator = PrefixComparators.LONG - val canUseRadixSort = SparkEnv.get.conf.get(SQLConf.RADIX_SORT_ENABLED) + val canUseRadixSort = SQLConf.get.enableRadixSort // The prefix computer generates row hashcode as the prefix, so we may decrease the // probability that the prefixes are equal when input rows choose column values from a // limited range. From ed9fa790c1b69448ecc75bf7f75900996e319f03 Mon Sep 17 00:00:00 2001 From: Rob Vesse Date: Fri, 16 Nov 2018 08:53:29 -0600 Subject: [PATCH 250/879] [SPARK-25023] More detailed security guidance for K8S ## What changes were proposed in this pull request? Highlights specific security issues to be aware of with Spark on K8S and recommends K8S mechanisms that should be used to secure clusters. ## How was this patch tested? N/A - Documentation only CC felixcheung tgravescs skonto Closes #23013 from rvesse/SPARK-25023. Authored-by: Rob Vesse Signed-off-by: Sean Owen (cherry picked from commit 2aef79a65a145b76a88f1d4d9367091fd238b949) Signed-off-by: Sean Owen --- docs/running-on-kubernetes.md | 16 +++++++++++++++- 1 file changed, 15 insertions(+), 1 deletion(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 754b1ff4e3a3..41d21228b470 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -15,7 +15,19 @@ container images and entrypoints.** # Security Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. -Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. +Please see [Spark Security](security.html) and the specific advice below before running Spark. + +## User Identity + +Images built from the project provided Dockerfiles do not contain any [`USER`](https://docs.docker.com/engine/reference/builder/#user) directives. This means that the resulting images will be running the Spark processes as `root` inside the container. On unsecured clusters this may provide an attack vector for privilege escalation and container breakout. Therefore security conscious deployments should consider providing custom images with `USER` directives specifying an unprivileged UID and GID. + +Alternatively the [Pod Template](#pod-template) feature can be used to add a [Security Context](https://kubernetes.io/docs/tasks/configure-pod-container/security-context/#volumes-and-file-systems) with a `runAsUser` to the pods that Spark submits. Please bear in mind that this requires cooperation from your users and as such may not be a suitable solution for shared environments. Cluster administrators should use [Pod Security Policies](https://kubernetes.io/docs/concepts/policy/pod-security-policy/#users-and-groups) if they wish to limit the users that pods may run as. + +## Volume Mounts + +As described later in this document under [Using Kubernetes Volumes](#using-kubernetes-volumes) Spark on K8S provides configuration options that allow for mounting certain volume types into the driver and executor pods. In particular it allows for [`hostPath`](https://kubernetes.io/docs/concepts/storage/volumes/#hostpath) volumes which as described in the Kubernetes documentation have known security vulnerabilities. + +Cluster administrators should use [Pod Security Policies](https://kubernetes.io/docs/concepts/policy/pod-security-policy/) to limit the ability to mount `hostPath` volumes appropriately for their environments. # Prerequisites @@ -197,6 +209,8 @@ Starting with Spark 2.4.0, users can mount the following types of Kubernetes [vo * [emptyDir](https://kubernetes.io/docs/concepts/storage/volumes/#emptydir): an initially empty volume created when a pod is assigned to a node. * [persistentVolumeClaim](https://kubernetes.io/docs/concepts/storage/volumes/#persistentvolumeclaim): used to mount a `PersistentVolume` into a pod. +**NB:** Please see the [Security](#security) section of this document for security issues related to volume mounts. + To mount a volume of any of the types above into the driver pod, use the following configuration property: ``` From be42bfe5cb4d03f855f322613d0de247ef4474fe Mon Sep 17 00:00:00 2001 From: Matt Molek Date: Fri, 16 Nov 2018 10:00:21 -0600 Subject: [PATCH 251/879] [SPARK-25934][MESOS] Don't propagate SPARK_CONF_DIR from spark submit ## What changes were proposed in this pull request? Don't propagate SPARK_CONF_DIR to the driver in mesos cluster mode. ## How was this patch tested? I built the 2.3.2 tag with this patch added and deployed a test job to a mesos cluster to confirm that the incorrect SPARK_CONF_DIR was no longer passed from the submit command. Closes #22937 from mpmolek/fix-conf-dir. Authored-by: Matt Molek Signed-off-by: Sean Owen (cherry picked from commit 696b75a81013ad61d25e0552df2b019c7531f983) Signed-off-by: Sean Owen --- .../spark/deploy/rest/RestSubmissionClient.scala | 8 +++++--- .../deploy/rest/StandaloneRestSubmitSuite.scala | 12 ++++++++++++ 2 files changed, 17 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 31a8e3e60c06..afa413fe165d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -408,6 +408,10 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { } private[spark] object RestSubmissionClient { + + // SPARK_HOME and SPARK_CONF_DIR are filtered out because they are usually wrong + // on the remote machine (SPARK-12345) (SPARK-25934) + private val BLACKLISTED_SPARK_ENV_VARS = Set("SPARK_ENV_LOADED", "SPARK_HOME", "SPARK_CONF_DIR") private val REPORT_DRIVER_STATUS_INTERVAL = 1000 private val REPORT_DRIVER_STATUS_MAX_TRIES = 10 val PROTOCOL_VERSION = "v1" @@ -417,9 +421,7 @@ private[spark] object RestSubmissionClient { */ private[rest] def filterSystemEnvironment(env: Map[String, String]): Map[String, String] = { env.filterKeys { k => - // SPARK_HOME is filtered out because it is usually wrong on the remote machine (SPARK-12345) - (k.startsWith("SPARK_") && k != "SPARK_ENV_LOADED" && k != "SPARK_HOME") || - k.startsWith("MESOS_") + (k.startsWith("SPARK_") && !BLACKLISTED_SPARK_ENV_VARS.contains(k)) || k.startsWith("MESOS_") } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 54c168a8218f..75fb716813ae 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -376,6 +376,18 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach { assert(filteredVariables == Map("SPARK_VAR" -> "1")) } + test("client does not send 'SPARK_HOME' env var by default") { + val environmentVariables = Map("SPARK_VAR" -> "1", "SPARK_HOME" -> "1") + val filteredVariables = RestSubmissionClient.filterSystemEnvironment(environmentVariables) + assert(filteredVariables == Map("SPARK_VAR" -> "1")) + } + + test("client does not send 'SPARK_CONF_DIR' env var by default") { + val environmentVariables = Map("SPARK_VAR" -> "1", "SPARK_CONF_DIR" -> "1") + val filteredVariables = RestSubmissionClient.filterSystemEnvironment(environmentVariables) + assert(filteredVariables == Map("SPARK_VAR" -> "1")) + } + test("client includes mesos env vars") { val environmentVariables = Map("SPARK_VAR" -> "1", "MESOS_VAR" -> "1", "OTHER_VAR" -> "1") val filteredVariables = RestSubmissionClient.filterSystemEnvironment(environmentVariables) From 77c0629cbac6046cc3f2ea1025c43d86af344d62 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 16 Nov 2018 09:51:41 -0800 Subject: [PATCH 252/879] [SPARK-26069][TESTS] Fix flaky test: RpcIntegrationSuite.sendRpcWithStreamFailures ## What changes were proposed in this pull request? The test failure is because `assertErrorAndClosed` misses one possible error message: `java.nio.channels.ClosedChannelException`. This happens when the second `uploadStream` is called after the channel has been closed. This can be reproduced by adding `Thread.sleep(1000)` below this line: https://github.com/apache/spark/blob/03306a6df39c9fd6cb581401c13c4dfc6bbd632e/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java#L217 This PR fixes the above issue and also improves the test failure messages of `assertErrorAndClosed`. ## How was this patch tested? Jenkins Closes #23041 from zsxwing/SPARK-26069. Authored-by: Shixiong Zhu Signed-off-by: Shixiong Zhu (cherry picked from commit 99cbc51b3250c07a3e8cc95c9b74e9d1725bac77) Signed-off-by: Shixiong Zhu --- .../spark/network/RpcIntegrationSuite.java | 18 ++++++++++++------ 1 file changed, 12 insertions(+), 6 deletions(-) diff --git a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java index 1f4d75c7e2ec..45f4a1808562 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java @@ -371,7 +371,10 @@ private void assertErrorsContain(Set errors, Set contains) { private void assertErrorAndClosed(RpcResult result, String expectedError) { assertTrue("unexpected success: " + result.successMessages, result.successMessages.isEmpty()); - // we expect 1 additional error, which contains *either* "closed" or "Connection reset" + // we expect 1 additional error, which should contain one of the follow messages: + // - "closed" + // - "Connection reset" + // - "java.nio.channels.ClosedChannelException" Set errors = result.errorMessages; assertEquals("Expected 2 errors, got " + errors.size() + "errors: " + errors, 2, errors.size()); @@ -379,15 +382,18 @@ private void assertErrorAndClosed(RpcResult result, String expectedError) { Set containsAndClosed = Sets.newHashSet(expectedError); containsAndClosed.add("closed"); containsAndClosed.add("Connection reset"); + containsAndClosed.add("java.nio.channels.ClosedChannelException"); Pair, Set> r = checkErrorsContain(errors, containsAndClosed); - Set errorsNotFound = r.getRight(); - assertEquals(1, errorsNotFound.size()); - String err = errorsNotFound.iterator().next(); - assertTrue(err.equals("closed") || err.equals("Connection reset")); + assertTrue("Got a non-empty set " + r.getLeft(), r.getLeft().isEmpty()); - assertTrue(r.getLeft().isEmpty()); + Set errorsNotFound = r.getRight(); + assertEquals( + "The size of " + errorsNotFound.toString() + " was not 2", 2, errorsNotFound.size()); + for (String err: errorsNotFound) { + assertTrue("Found a wrong error " + err, containsAndClosed.contains(err)); + } } private Pair, Set> checkErrorsContain( From c23b801d3c87b12e729b98910833b441db05bd45 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 16 Nov 2018 15:43:27 -0800 Subject: [PATCH 253/879] [SPARK-26092][SS] Use CheckpointFileManager to write the streaming metadata file ## What changes were proposed in this pull request? Use CheckpointFileManager to write the streaming `metadata` file so that the `metadata` file will never be a partial file. ## How was this patch tested? Jenkins Closes #23060 from zsxwing/SPARK-26092. Authored-by: Shixiong Zhu Signed-off-by: Shixiong Zhu (cherry picked from commit 058c4602b000b24deb764a810ef8b43c41fe63ae) Signed-off-by: Shixiong Zhu --- .../streaming/CheckpointFileManager.scala | 2 +- .../execution/streaming/StreamExecution.scala | 1 + .../execution/streaming/StreamMetadata.scala | 23 +++++++++++++------ 3 files changed, 18 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala index 606ba250ad9d..b3e4240c315b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala @@ -56,7 +56,7 @@ trait CheckpointFileManager { * @param overwriteIfPossible If true, then the implementations must do a best-effort attempt to * overwrite the file if it already exists. It should not throw * any exception if the file exists. However, if false, then the - * implementation must not overwrite if the file alraedy exists and + * implementation must not overwrite if the file already exists and * must throw `FileAlreadyExistsException` in that case. */ def createAtomic(path: Path, overwriteIfPossible: Boolean): CancellableFSDataOutputStream diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index f6c60c1c9212..de338440c873 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -87,6 +87,7 @@ abstract class StreamExecution( val resolvedCheckpointRoot = { val checkpointPath = new Path(checkpointRoot) val fs = checkpointPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) + fs.mkdirs(checkpointPath) checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala index 0bc54eac4ee8..516afbea5d9d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala @@ -19,16 +19,18 @@ package org.apache.spark.sql.execution.streaming import java.io.{InputStreamReader, OutputStreamWriter} import java.nio.charset.StandardCharsets +import java.util.ConcurrentModificationException import scala.util.control.NonFatal import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, FSDataInputStream, FSDataOutputStream, Path} +import org.apache.hadoop.fs.{FileAlreadyExistsException, FSDataInputStream, Path} import org.json4s.NoTypeHints import org.json4s.jackson.Serialization import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream import org.apache.spark.sql.streaming.StreamingQuery /** @@ -70,19 +72,26 @@ object StreamMetadata extends Logging { metadata: StreamMetadata, metadataFile: Path, hadoopConf: Configuration): Unit = { - var output: FSDataOutputStream = null + var output: CancellableFSDataOutputStream = null try { - val fs = metadataFile.getFileSystem(hadoopConf) - output = fs.create(metadataFile) + val fileManager = CheckpointFileManager.create(metadataFile.getParent, hadoopConf) + output = fileManager.createAtomic(metadataFile, overwriteIfPossible = false) val writer = new OutputStreamWriter(output) Serialization.write(metadata, writer) writer.close() } catch { - case NonFatal(e) => + case e: FileAlreadyExistsException => + if (output != null) { + output.cancel() + } + throw new ConcurrentModificationException( + s"Multiple streaming queries are concurrently using $metadataFile", e) + case e: Throwable => + if (output != null) { + output.cancel() + } logError(s"Error writing stream metadata $metadata to $metadataFile", e) throw e - } finally { - IOUtils.closeQuietly(output) } } } From 33f55d445c1d148d8cc8c210cae67d8f9f717dc1 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Sat, 17 Nov 2018 15:07:20 +0800 Subject: [PATCH 254/879] [SPARK-26079][SQL] Ensure listener event delivery in StreamingQueryListenersConfSuite. Events are dispatched on a separate thread, so need to wait for them to be actually delivered before checking that the listener got them. Closes #23050 from vanzin/SPARK-26079. Authored-by: Marcelo Vanzin Signed-off-by: hyukjinkwon --- .../spark/sql/streaming/StreamingQueryListenersConfSuite.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala index 1aaf8a9aa2d5..ddbc175e7ea4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala @@ -30,7 +30,6 @@ class StreamingQueryListenersConfSuite extends StreamTest with BeforeAndAfter { import testImplicits._ - override protected def sparkConf: SparkConf = super.sparkConf.set("spark.sql.streaming.streamingQueryListeners", "org.apache.spark.sql.streaming.TestListener") @@ -41,6 +40,8 @@ class StreamingQueryListenersConfSuite extends StreamTest with BeforeAndAfter { StopStream ) + spark.sparkContext.listenerBus.waitUntilEmpty(5000) + assert(TestListener.queryStartedEvent != null) assert(TestListener.queryTerminatedEvent != null) } From 096e0d8f05e6076a325795688dfaa8174a7b6b25 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sat, 17 Nov 2018 18:18:41 +0800 Subject: [PATCH 255/879] [MINOR][SQL] Fix typo in CTAS plan database string ## What changes were proposed in this pull request? Since [Spark 1.6.0](https://github.com/apache/spark/commit/56d7da14ab8f89bf4f303b27f51fd22d23967ffb#diff-6f38a103058a6e233b7ad80718452387R96), there was a redundant '}' character in CTAS string plan's database argument string; `default}`. This PR aims to fix it. **BEFORE** ```scala scala> sc.version res1: String = 1.6.0 scala> sql("create table t as select 1").explain == Physical Plan == ExecutedCommand CreateTableAsSelect [Database:default}, TableName: t, InsertIntoHiveTable] +- Project [1 AS _c0#3] +- OneRowRelation$ ``` **AFTER** ```scala scala> sql("create table t as select 1").explain == Physical Plan == Execute CreateHiveTableAsSelectCommand CreateHiveTableAsSelectCommand [Database:default, TableName: t, InsertIntoHiveTable] +- *(1) Project [1 AS 1#4] +- Scan OneRowRelation[] ``` ## How was this patch tested? Manual. Closes #23064 from dongjoon-hyun/SPARK-FIX. Authored-by: Dongjoon Hyun Signed-off-by: hyukjinkwon (cherry picked from commit b538c442cb3982cc4c3aac812a7d4764209dfbb7) Signed-off-by: hyukjinkwon --- .../sql/hive/execution/CreateHiveTableAsSelectCommand.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index aa573b54a2b6..630bea5161f1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -96,7 +96,7 @@ case class CreateHiveTableAsSelectCommand( } override def argString: String = { - s"[Database:${tableDesc.database}}, " + + s"[Database:${tableDesc.database}, " + s"TableName: ${tableDesc.identifier.table}, " + s"InsertIntoHiveTable]" } From c28a27a2546ebbe0c001662126625638fcbb1100 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Tue, 20 Nov 2018 08:56:22 -0600 Subject: [PATCH 256/879] [SPARK-26118][WEB UI] Introducing spark.ui.requestHeaderSize for setting HTTP requestHeaderSize MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Introducing spark.ui.requestHeaderSize for configuring Jetty's HTTP requestHeaderSize. This way long authorization field does not lead to HTTP 413. ## How was this patch tested? Manually with curl (which version must be at least 7.55). With the original default value (8k limit): ```bash # Starting history server with default requestHeaderSize $ ./sbin/start-history-server.sh starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out # Creating huge header $ echo -n "X-Custom-Header: " > cookie $ printf 'A%.0s' {1..9500} >> cookie # HTTP GET with huge header fails with 431 $ curl -H cookie http://458apiros-MBP.lan:18080/

    Bad Message 431

    reason: Request Header Fields Too Large
    # The log contains the error $ tail -1 /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out 18/11/19 21:24:28 WARN HttpParser: Header is too large 8193>8192 ``` After: ```bash # Creating the history properties file with the increased requestHeaderSize $ echo spark.ui.requestHeaderSize=10000 > history.properties # Starting Spark History Server with the settings $ ./sbin/start-history-server.sh --properties-file history.properties starting org.apache.spark.deploy.history.HistoryServer, logging to /Users/attilapiros/github/spark/logs/spark-attilapiros-org.apache.spark.deploy.history.HistoryServer-1-apiros-MBP.lan.out # HTTP GET with huge header gives back HTML5 (I have added here only just a part of the response) $ curl -H cookie http://458apiros-MBP.lan:18080/ ... History Server ... ``` Closes #23090 from attilapiros/JettyHeaderSize. Authored-by: “attilapiros” Signed-off-by: Imran Rashid (cherry picked from commit ab61ddb34d58ab5701191c8fd3a24a62f6ebf37b) Signed-off-by: Imran Rashid --- .../scala/org/apache/spark/internal/config/package.scala | 6 ++++++ core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 6 ++++-- docs/configuration.md | 8 ++++++++ 3 files changed, 18 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index bde0995cd008..3b3c45fc18fb 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -528,6 +528,12 @@ package object config { .stringConf .createOptional + private[spark] val UI_REQUEST_HEADER_SIZE = + ConfigBuilder("spark.ui.requestHeaderSize") + .doc("Value for HTTP request header size in bytes.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("8k") + private[spark] val EXTRA_LISTENERS = ConfigBuilder("spark.extraListeners") .doc("Class names of listeners to add to SparkContext during initialization.") .stringConf diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 52a955111231..316af9b79d28 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -356,13 +356,15 @@ private[spark] object JettyUtils extends Logging { (connector, connector.getLocalPort()) } + val httpConfig = new HttpConfiguration() + httpConfig.setRequestHeaderSize(conf.get(UI_REQUEST_HEADER_SIZE).toInt) // If SSL is configured, create the secure connector first. val securePort = sslOptions.createJettySslContextFactory().map { factory => val securePort = sslOptions.port.getOrElse(if (port > 0) Utils.userPort(port, 400) else 0) val secureServerName = if (serverName.nonEmpty) s"$serverName (HTTPS)" else serverName val connectionFactories = AbstractConnectionFactory.getFactories(factory, - new HttpConnectionFactory()) + new HttpConnectionFactory(httpConfig)) def sslConnect(currentPort: Int): (ServerConnector, Int) = { newConnector(connectionFactories, currentPort) @@ -377,7 +379,7 @@ private[spark] object JettyUtils extends Logging { // Bind the HTTP port. def httpConnect(currentPort: Int): (ServerConnector, Int) = { - newConnector(Array(new HttpConnectionFactory()), currentPort) + newConnector(Array(new HttpConnectionFactory(httpConfig)), currentPort) } val (httpConnector, httpPort) = Utils.startServiceOnPort[ServerConnector](port, httpConnect, diff --git a/docs/configuration.md b/docs/configuration.md index d392a6ad5cc1..944e5e4c2e97 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -929,6 +929,14 @@ Apart from these, the following properties are also available, and may be useful
    spark.com.test.filter1.param.name2=bar + + spark.ui.requestHeaderSize + 8k + + The maximum allowed size for a HTTP request header, in bytes unless otherwise specified. + This setting applies for the Spark History Server too. + + ### Compression and Serialization From 3bb9fff687a1701b75552bae6a4f8bee3fa6460b Mon Sep 17 00:00:00 2001 From: Simeon Simeonov Date: Tue, 20 Nov 2018 21:29:56 +0100 Subject: [PATCH 257/879] [SPARK-26084][SQL] Fixes unresolved AggregateExpression.references exception ## What changes were proposed in this pull request? This PR fixes an exception in `AggregateExpression.references` called on unresolved expressions. It implements the solution proposed in [SPARK-26084](https://issues.apache.org/jira/browse/SPARK-26084), a minor refactoring that removes the unnecessary dependence on `AttributeSet.toSeq`, which requires expression IDs and, therefore, can only execute successfully for resolved expressions. The refactored implementation is both simpler and faster, eliminating the conversion of a `Set` to a `Seq` and back to `Set`. ## How was this patch tested? Added a new test based on the failing case in [SPARK-26084](https://issues.apache.org/jira/browse/SPARK-26084). hvanhovell Closes #23075 from ssimeonov/ss_SPARK-26084. Authored-by: Simeon Simeonov Signed-off-by: Herman van Hovell (cherry picked from commit db136d360e54e13f1d7071a0428964a202cf7e31) Signed-off-by: Herman van Hovell --- .../expressions/aggregate/interfaces.scala | 8 ++--- .../aggregate/AggregateExpressionSuite.scala | 34 +++++++++++++++++++ 2 files changed, 37 insertions(+), 5 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/AggregateExpressionSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index e1d16a2cd38b..56c2ee6b53fe 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -128,12 +128,10 @@ case class AggregateExpression( override def nullable: Boolean = aggregateFunction.nullable override def references: AttributeSet = { - val childReferences = mode match { - case Partial | Complete => aggregateFunction.references.toSeq - case PartialMerge | Final => aggregateFunction.aggBufferAttributes + mode match { + case Partial | Complete => aggregateFunction.references + case PartialMerge | Final => AttributeSet(aggregateFunction.aggBufferAttributes) } - - AttributeSet(childReferences) } override def toString: String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/AggregateExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/AggregateExpressionSuite.scala new file mode 100644 index 000000000000..8e9c9972071a --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/AggregateExpressionSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Add, AttributeSet} + +class AggregateExpressionSuite extends SparkFunSuite { + + test("test references from unresolved aggregate functions") { + val x = UnresolvedAttribute("x") + val y = UnresolvedAttribute("y") + val actual = AggregateExpression(Sum(Add(x, y)), mode = Complete, isDistinct = false).references + val expected = AttributeSet(x :: y :: Nil) + assert(expected == actual, s"Expected: $expected. Actual: $actual") + } + +} From d8e05d23a046eee559b0c71bcfba5b9809c3d9eb Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Wed, 21 Nov 2018 09:31:12 +0800 Subject: [PATCH 258/879] [SPARK-26120][TESTS][SS][SPARKR] Fix a streaming query leak in Structured Streaming R tests ## What changes were proposed in this pull request? Stop the streaming query in `Specify a schema by using a DDL-formatted string when reading` to avoid outputting annoying logs. ## How was this patch tested? Jenkins Closes #23089 from zsxwing/SPARK-26120. Authored-by: Shixiong Zhu Signed-off-by: hyukjinkwon (cherry picked from commit 4b7f7ef5007c2c8a5090f22c6e08927e9f9a407b) Signed-off-by: hyukjinkwon --- R/pkg/tests/fulltests/test_streaming.R | 1 + 1 file changed, 1 insertion(+) diff --git a/R/pkg/tests/fulltests/test_streaming.R b/R/pkg/tests/fulltests/test_streaming.R index bfb1a046490e..6f0d2aefee88 100644 --- a/R/pkg/tests/fulltests/test_streaming.R +++ b/R/pkg/tests/fulltests/test_streaming.R @@ -127,6 +127,7 @@ test_that("Specify a schema by using a DDL-formatted string when reading", { expect_false(awaitTermination(q, 5 * 1000)) callJMethod(q@ssq, "processAllAvailable") expect_equal(head(sql("SELECT count(*) FROM people3"))[[1]], 3) + stopQuery(q) expect_error(read.stream(path = parquetPath, schema = "name stri"), "DataType stri is not supported.") From 8705a9dd787eef01169aaa33c15a74438c1eb0f4 Mon Sep 17 00:00:00 2001 From: Shahid Date: Wed, 21 Nov 2018 09:31:35 -0600 Subject: [PATCH 259/879] [SPARK-26109][WEBUI] Duration in the task summary metrics table and the task table are different ## What changes were proposed in this pull request? Task summary table displays the summary of the task table in the stage page. However, the 'Duration' metrics of 'task summary' table and 'task table' are not matching. The reason is because, in the 'task summary' we display 'executorRunTime' as the duration, and in the 'task table' the actual duration of the task. Except duration metrics, all other metrics are properly displaying in the task summary. In Spark2.2, used to show 'executorRunTime' as duration in the 'taskTable'. That is why, in summary metrics also the 'exeuctorRunTime' shows as the duration. So, we need to show 'executorRunTime' as the duration in the tasks table to follow the same behaviour as the previous versions of spark. ## How was this patch tested? Before patch: ![screenshot from 2018-11-19 04-32-06](https://user-images.githubusercontent.com/23054875/48679263-1e4fff80-ebb4-11e8-9ed5-16d892039e01.png) After patch: ![screenshot from 2018-11-19 04-37-39](https://user-images.githubusercontent.com/23054875/48679343-e39a9700-ebb4-11e8-8df9-9dc3a28d4bce.png) Closes #23081 from shahidki31/duratinSummary. Authored-by: Shahid Signed-off-by: Sean Owen (cherry picked from commit 540afc2b18ef61cceb50b9a5b327e6fcdbe1e7e4) Signed-off-by: Sean Owen --- .../src/main/scala/org/apache/spark/ui/jobs/StagePage.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 7428bbe6c559..ca3a13e8847c 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -848,7 +848,7 @@ private[ui] class TaskPagedTable( {UIUtils.formatDate(task.launchTime)} - {formatDuration(task.duration)} + {formatDuration(task.taskMetrics.map(_.executorRunTime))} {UIUtils.formatDuration(AppStatusUtils.schedulerDelay(task))} @@ -1001,7 +1001,9 @@ private[ui] object ApiHelper { HEADER_EXECUTOR -> TaskIndexNames.EXECUTOR, HEADER_HOST -> TaskIndexNames.HOST, HEADER_LAUNCH_TIME -> TaskIndexNames.LAUNCH_TIME, - HEADER_DURATION -> TaskIndexNames.DURATION, + // SPARK-26109: Duration of task as executorRunTime to make it consistent with the + // aggregated tasks summary metrics table and the previous versions of Spark. + HEADER_DURATION -> TaskIndexNames.EXEC_RUN_TIME, HEADER_SCHEDULER_DELAY -> TaskIndexNames.SCHEDULER_DELAY, HEADER_DESER_TIME -> TaskIndexNames.DESER_TIME, HEADER_GC_TIME -> TaskIndexNames.GC_TIME, From d63ab5a4f5aeecfa227edc84aa38e866446f5238 Mon Sep 17 00:00:00 2001 From: Alon Doron Date: Fri, 23 Nov 2018 08:55:00 +0800 Subject: [PATCH 260/879] [SPARK-26021][SQL] replace minus zero with zero in Platform.putDouble/Float GROUP BY treats -0.0 and 0.0 as different values which is unlike hive's behavior. In addition current behavior with codegen is unpredictable (see example in JIRA ticket). ## What changes were proposed in this pull request? In Platform.putDouble/Float() checking if the value is -0.0, and if so replacing with 0.0. This is used by UnsafeRow so it won't have -0.0 values. ## How was this patch tested? Added tests Closes #23043 from adoron/adoron-spark-26021-replace-minus-zero-with-zero. Authored-by: Alon Doron Signed-off-by: Wenchen Fan (cherry picked from commit 0ec7b99ea2b638453ed38bb092905bee4f907fe5) Signed-off-by: Wenchen Fan --- .../java/org/apache/spark/unsafe/Platform.java | 10 ++++++++++ .../org/apache/spark/unsafe/PlatformUtilSuite.java | 14 ++++++++++++++ .../spark/sql/catalyst/expressions/UnsafeRow.java | 6 ------ .../catalyst/expressions/codegen/UnsafeWriter.java | 6 ------ .../apache/spark/sql/DataFrameAggregateSuite.scala | 14 ++++++++++++++ .../scala/org/apache/spark/sql/QueryTest.scala | 5 ++++- 6 files changed, 42 insertions(+), 13 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index aca6fca00c48..bc94f2171228 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -120,6 +120,11 @@ public static float getFloat(Object object, long offset) { } public static void putFloat(Object object, long offset, float value) { + if (Float.isNaN(value)) { + value = Float.NaN; + } else if (value == -0.0f) { + value = 0.0f; + } _UNSAFE.putFloat(object, offset, value); } @@ -128,6 +133,11 @@ public static double getDouble(Object object, long offset) { } public static void putDouble(Object object, long offset, double value) { + if (Double.isNaN(value)) { + value = Double.NaN; + } else if (value == -0.0d) { + value = 0.0d; + } _UNSAFE.putDouble(object, offset, value); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java index 3ad9ac7b4de9..ab34324eb54c 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java @@ -157,4 +157,18 @@ public void heapMemoryReuse() { Assert.assertEquals(onheap4.size(), 1024 * 1024 + 7); Assert.assertEquals(obj3, onheap4.getBaseObject()); } + + @Test + // SPARK-26021 + public void writeMinusZeroIsReplacedWithZero() { + byte[] doubleBytes = new byte[Double.BYTES]; + byte[] floatBytes = new byte[Float.BYTES]; + Platform.putDouble(doubleBytes, Platform.BYTE_ARRAY_OFFSET, -0.0d); + Platform.putFloat(floatBytes, Platform.BYTE_ARRAY_OFFSET, -0.0f); + double doubleFromPlatform = Platform.getDouble(doubleBytes, Platform.BYTE_ARRAY_OFFSET); + float floatFromPlatform = Platform.getFloat(floatBytes, Platform.BYTE_ARRAY_OFFSET); + + Assert.assertEquals(Double.doubleToLongBits(0.0d), Double.doubleToLongBits(doubleFromPlatform)); + Assert.assertEquals(Float.floatToIntBits(0.0f), Float.floatToIntBits(floatFromPlatform)); + } } 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 a76e6ef8c91c..9bf9452855f5 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 @@ -224,9 +224,6 @@ public void setLong(int ordinal, long value) { public void setDouble(int ordinal, double value) { assertIndexIsValid(ordinal); setNotNullAt(ordinal); - if (Double.isNaN(value)) { - value = Double.NaN; - } Platform.putDouble(baseObject, getFieldOffset(ordinal), value); } @@ -255,9 +252,6 @@ public void setByte(int ordinal, byte value) { public void setFloat(int ordinal, float value) { assertIndexIsValid(ordinal); setNotNullAt(ordinal); - if (Float.isNaN(value)) { - value = Float.NaN; - } Platform.putFloat(baseObject, getFieldOffset(ordinal), value); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java index 278165500200..95263a0da95a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java @@ -199,16 +199,10 @@ protected final void writeLong(long offset, long value) { } protected final void writeFloat(long offset, float value) { - if (Float.isNaN(value)) { - value = Float.NaN; - } Platform.putFloat(getBuffer(), offset, value); } protected final void writeDouble(long offset, double value) { - if (Double.isNaN(value)) { - value = Double.NaN; - } Platform.putDouble(getBuffer(), offset, value); } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index d0106c44b7db..41dc72de49be 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -727,4 +727,18 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { "grouping expressions: [current_date(None)], value: [key: int, value: string], " + "type: GroupBy]")) } + + test("SPARK-26021: Double and Float 0.0/-0.0 should be equal when grouping") { + val colName = "i" + val doubles = Seq(0.0d, -0.0d, 0.0d).toDF(colName).groupBy(colName).count().collect() + val floats = Seq(0.0f, -0.0f, 0.0f).toDF(colName).groupBy(colName).count().collect() + + assert(doubles.length == 1) + assert(floats.length == 1) + // using compare since 0.0 == -0.0 is true + assert(java.lang.Double.compare(doubles(0).getDouble(0), 0.0d) == 0) + assert(java.lang.Float.compare(floats(0).getFloat(0), 0.0f) == 0) + assert(doubles(0).getLong(1) == 3) + assert(floats(0).getLong(1) == 3) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index baca9c1cfb9a..8ba67239fb90 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -289,7 +289,7 @@ object QueryTest { def prepareRow(row: Row): Row = { Row.fromSeq(row.toSeq.map { case null => null - case d: java.math.BigDecimal => BigDecimal(d) + case bd: java.math.BigDecimal => BigDecimal(bd) // Equality of WrappedArray differs for AnyVal and AnyRef in Scala 2.12.2+ case seq: Seq[_] => seq.map { case b: java.lang.Byte => b.byteValue @@ -303,6 +303,9 @@ object QueryTest { // Convert array to Seq for easy equality check. case b: Array[_] => b.toSeq case r: Row => prepareRow(r) + // spark treats -0.0 as 0.0 + case d: Double if d == -0.0d => 0.0d + case f: Float if f == -0.0f => 0.0f case o => o }) } From 709a8cc09df51db87afa40bf4176b6653069fbf6 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 22 Nov 2018 23:21:46 -0800 Subject: [PATCH 261/879] [SPARK-24553][UI][FOLLOWUP][2.4 BACKPORT] Fix unnecessary UI redirect ## What changes were proposed in this pull request? This is a backport PR of #23116 . This PR is a follow-up PR of #21600 to fix the unnecessary UI redirect. ## How was this patch tested? Local verification Closes #23121 from jerryshao/SPARK-24553-branch-2.4. Authored-by: jerryshao Signed-off-by: Dongjoon Hyun --- .../main/scala/org/apache/spark/ui/jobs/StageTable.scala | 2 +- .../scala/org/apache/spark/ui/storage/StoragePage.scala | 2 +- .../org/apache/spark/ui/storage/StoragePageSuite.scala | 6 +++--- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index d01acdae59c9..5b86b93acb4e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -383,7 +383,7 @@ private[ui] class StagePagedTable( {if (cachedRddInfos.nonEmpty) { Text("RDD: ") ++ cachedRddInfos.map { i => - {i.name} + {i.name} } }}
    {s.details}
    diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 3eb546e336e9..2488197814ff 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -78,7 +78,7 @@ private[ui] class StoragePage(parent: SparkUITab, store: AppStatusStore) extends {rdd.id} - {rdd.name} diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala index cdc7f541b955..06f01a60868f 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StoragePageSuite.scala @@ -81,19 +81,19 @@ class StoragePageSuite extends SparkFunSuite { Seq("1", "rdd1", "Memory Deserialized 1x Replicated", "10", "100%", "100.0 B", "0.0 B")) // Check the url assert(((xmlNodes \\ "tr")(0) \\ "td" \ "a")(0).attribute("href").map(_.text) === - Some("http://localhost:4040/storage/rdd?id=1")) + Some("http://localhost:4040/storage/rdd/?id=1")) assert(((xmlNodes \\ "tr")(1) \\ "td").map(_.text.trim) === Seq("2", "rdd2", "Disk Serialized 1x Replicated", "5", "50%", "0.0 B", "200.0 B")) // Check the url assert(((xmlNodes \\ "tr")(1) \\ "td" \ "a")(0).attribute("href").map(_.text) === - Some("http://localhost:4040/storage/rdd?id=2")) + Some("http://localhost:4040/storage/rdd/?id=2")) assert(((xmlNodes \\ "tr")(2) \\ "td").map(_.text.trim) === Seq("3", "rdd3", "Disk Memory Serialized 1x Replicated", "10", "100%", "400.0 B", "500.0 B")) // Check the url assert(((xmlNodes \\ "tr")(2) \\ "td" \ "a")(0).attribute("href").map(_.text) === - Some("http://localhost:4040/storage/rdd?id=3")) + Some("http://localhost:4040/storage/rdd/?id=3")) } test("empty rddTable") { From 14d501baea544640c8c12be03092b4dc265a24f4 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 23 Nov 2018 06:18:44 -0600 Subject: [PATCH 262/879] [SPARK-26069][TESTS][FOLLOWUP] Add another possible error message ## What changes were proposed in this pull request? `org.apache.spark.network.RpcIntegrationSuite.sendRpcWithStreamFailures` is still flaky and here is error message: ``` sbt.ForkMain$ForkError: java.lang.AssertionError: Got a non-empty set [Failed to send RPC RPC 8249697863992194475 to /172.17.0.2:41177: java.io.IOException: Broken pipe] at org.junit.Assert.fail(Assert.java:88) at org.junit.Assert.assertTrue(Assert.java:41) at org.apache.spark.network.RpcIntegrationSuite.assertErrorAndClosed(RpcIntegrationSuite.java:389) at org.apache.spark.network.RpcIntegrationSuite.sendRpcWithStreamFailures(RpcIntegrationSuite.java:347) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.junit.runners.model.FrameworkMethod$1.runReflectiveCall(FrameworkMethod.java:50) at org.junit.internal.runners.model.ReflectiveCallable.run(ReflectiveCallable.java:12) at org.junit.runners.model.FrameworkMethod.invokeExplosively(FrameworkMethod.java:47) at org.junit.internal.runners.statements.InvokeMethod.evaluate(InvokeMethod.java:17) at org.junit.runners.ParentRunner.runLeaf(ParentRunner.java:325) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:78) at org.junit.runners.BlockJUnit4ClassRunner.runChild(BlockJUnit4ClassRunner.java:57) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268) at org.junit.internal.runners.statements.RunBefores.evaluate(RunBefores.java:26) at org.junit.internal.runners.statements.RunAfters.evaluate(RunAfters.java:27) at org.junit.runners.ParentRunner.run(ParentRunner.java:363) at org.junit.runners.Suite.runChild(Suite.java:128) at org.junit.runners.Suite.runChild(Suite.java:27) at org.junit.runners.ParentRunner$3.run(ParentRunner.java:290) at org.junit.runners.ParentRunner$1.schedule(ParentRunner.java:71) at org.junit.runners.ParentRunner.runChildren(ParentRunner.java:288) at org.junit.runners.ParentRunner.access$000(ParentRunner.java:58) at org.junit.runners.ParentRunner$2.evaluate(ParentRunner.java:268) at org.junit.runners.ParentRunner.run(ParentRunner.java:363) at org.junit.runner.JUnitCore.run(JUnitCore.java:137) at org.junit.runner.JUnitCore.run(JUnitCore.java:115) at com.novocode.junit.JUnitRunner$1.execute(JUnitRunner.java:132) at sbt.ForkMain$Run$2.call(ForkMain.java:296) at sbt.ForkMain$Run$2.call(ForkMain.java:286) at java.util.concurrent.FutureTask.run(FutureTask.java:266) at java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149) at java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624) at java.lang.Thread.run(Thread.java:748) ``` This happened when the second RPC message was being sent but the connection was closed at the same time. ## How was this patch tested? Jenkins Closes #23109 from zsxwing/SPARK-26069-2. Authored-by: Shixiong Zhu Signed-off-by: Sean Owen (cherry picked from commit 92fc0a8f9619a8e7f8382d6a5c288aeceb03a472) Signed-off-by: Sean Owen --- .../spark/network/RpcIntegrationSuite.java | 20 +++++++++++-------- 1 file changed, 12 insertions(+), 8 deletions(-) diff --git a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java index 45f4a1808562..1c0aa4da27ff 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java @@ -371,18 +371,20 @@ private void assertErrorsContain(Set errors, Set contains) { private void assertErrorAndClosed(RpcResult result, String expectedError) { assertTrue("unexpected success: " + result.successMessages, result.successMessages.isEmpty()); - // we expect 1 additional error, which should contain one of the follow messages: - // - "closed" - // - "Connection reset" - // - "java.nio.channels.ClosedChannelException" Set errors = result.errorMessages; assertEquals("Expected 2 errors, got " + errors.size() + "errors: " + errors, 2, errors.size()); + // We expect 1 additional error due to closed connection and here are possible keywords in the + // error message. + Set possibleClosedErrors = Sets.newHashSet( + "closed", + "Connection reset", + "java.nio.channels.ClosedChannelException", + "java.io.IOException: Broken pipe" + ); Set containsAndClosed = Sets.newHashSet(expectedError); - containsAndClosed.add("closed"); - containsAndClosed.add("Connection reset"); - containsAndClosed.add("java.nio.channels.ClosedChannelException"); + containsAndClosed.addAll(possibleClosedErrors); Pair, Set> r = checkErrorsContain(errors, containsAndClosed); @@ -390,7 +392,9 @@ private void assertErrorAndClosed(RpcResult result, String expectedError) { Set errorsNotFound = r.getRight(); assertEquals( - "The size of " + errorsNotFound.toString() + " was not 2", 2, errorsNotFound.size()); + "The size of " + errorsNotFound + " was not " + (possibleClosedErrors.size() - 1), + possibleClosedErrors.size() - 1, + errorsNotFound.size()); for (String err: errorsNotFound) { assertTrue("Found a wrong error " + err, containsAndClosed.contains(err)); } From a2a8873ff08ae1ee46e27b339f7844bf51270712 Mon Sep 17 00:00:00 2001 From: liuxian Date: Sat, 24 Nov 2018 09:10:15 -0600 Subject: [PATCH 263/879] [SPARK-25786][CORE] If the ByteBuffer.hasArray is false , it will throw UnsupportedOperationException for Kryo `deserialize` for kryo, the type of input parameter is ByteBuffer, if it is not backed by an accessible byte array. it will throw `UnsupportedOperationException` Exception Info: ``` java.lang.UnsupportedOperationException was thrown. java.lang.UnsupportedOperationException at java.nio.ByteBuffer.array(ByteBuffer.java:994) at org.apache.spark.serializer.KryoSerializerInstance.deserialize(KryoSerializer.scala:362) ``` Added a unit test Closes #22779 from 10110346/InputStreamKryo. Authored-by: liuxian Signed-off-by: Sean Owen (cherry picked from commit 7f5f7a967d36d78f73d8fa1e178dfdb324d73bf1) Signed-off-by: Sean Owen --- .../apache/spark/serializer/KryoSerializer.scala | 16 +++++++++++++--- .../spark/serializer/KryoSerializerSuite.scala | 12 ++++++++++++ 2 files changed, 25 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 72427dd6ce4d..424a3ed9bb5b 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -41,7 +41,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ -import org.apache.spark.util.{BoundedPriorityQueue, SerializableConfiguration, SerializableJobConf, Utils} +import org.apache.spark.util.{BoundedPriorityQueue, ByteBufferInputStream, SerializableConfiguration, SerializableJobConf, Utils} import org.apache.spark.util.collection.CompactBuffer /** @@ -358,7 +358,12 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boole override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { val kryo = borrowKryo() try { - input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) + if (bytes.hasArray) { + input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) + } else { + input.setBuffer(new Array[Byte](4096)) + input.setInputStream(new ByteBufferInputStream(bytes)) + } kryo.readClassAndObject(input).asInstanceOf[T] } finally { releaseKryo(kryo) @@ -370,7 +375,12 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boole val oldClassLoader = kryo.getClassLoader try { kryo.setClassLoader(loader) - input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) + if (bytes.hasArray) { + input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) + } else { + input.setBuffer(new Array[Byte](4096)) + input.setInputStream(new ByteBufferInputStream(bytes)) + } kryo.readClassAndObject(input).asInstanceOf[T] } finally { kryo.setClassLoader(oldClassLoader) diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index ac25bcef5434..fcb1315c13fb 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.serializer import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, FileOutputStream} +import java.nio.ByteBuffer import scala.collection.JavaConverters._ import scala.collection.mutable @@ -497,6 +498,17 @@ class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSpar deserializationStream.close() assert(serInstance.deserialize[Any](helloHello) === ((hello, hello))) } + + test("SPARK-25786: ByteBuffer.array -- UnsupportedOperationException") { + val serInstance = new KryoSerializer(conf).newInstance().asInstanceOf[KryoSerializerInstance] + val obj = "UnsupportedOperationException" + val serObj = serInstance.serialize(obj) + val byteBuffer = ByteBuffer.allocateDirect(serObj.array().length) + byteBuffer.put(serObj.array()) + byteBuffer.flip() + assert(serInstance.deserialize[Any](serObj) === (obj)) + assert(serInstance.deserialize[Any](byteBuffer) === (obj)) + } } class ClassLoaderTestingObject From c379611ee99aa023c7180398060da2b7f8b06033 Mon Sep 17 00:00:00 2001 From: Lee moon soo Date: Sat, 24 Nov 2018 16:09:13 -0800 Subject: [PATCH 264/879] [MINOR][K8S] Invalid property "spark.driver.pod.name" is referenced in docs. ## What changes were proposed in this pull request? "Running on Kubernetes" references `spark.driver.pod.name` few places, and it should be `spark.kubernetes.driver.pod.name`. ## How was this patch tested? See changes Closes #23133 from Leemoonsoo/fix-driver-pod-name-prop. Authored-by: Lee moon soo Signed-off-by: Dongjoon Hyun (cherry picked from commit eea4a0330b913cd45e369f09ec3d1dbb1b81f1b5) Signed-off-by: Dongjoon Hyun --- docs/running-on-kubernetes.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index 41d21228b470..02770439f538 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -153,7 +153,7 @@ hostname via `spark.driver.host` and your spark driver's port to `spark.driver.p ### Client Mode Executor Pod Garbage Collection -If you run your Spark driver in a pod, it is highly recommended to set `spark.driver.pod.name` to the name of that pod. +If you run your Spark driver in a pod, it is highly recommended to set `spark.kubernetes.driver.pod.name` to the name of that pod. When this property is set, the Spark scheduler will deploy the executor pods with an [OwnerReference](https://kubernetes.io/docs/concepts/workloads/controllers/garbage-collection/), which in turn will ensure that once the driver pod is deleted from the cluster, all of the application's executor pods will also be deleted. @@ -162,7 +162,7 @@ an OwnerReference pointing to that pod will be added to each executor pod's Owne setting the OwnerReference to a pod that is not actually that driver pod, or else the executors may be terminated prematurely when the wrong pod is deleted. -If your application is not running inside a pod, or if `spark.driver.pod.name` is not set when your application is +If your application is not running inside a pod, or if `spark.kubernetes.driver.pod.name` is not set when your application is actually running in a pod, keep in mind that the executor pods may not be properly deleted from the cluster when the application exits. The Spark scheduler attempts to delete these pods, but if the network request to the API server fails for any reason, these pods will remain in the cluster. The executor processes should exit when they cannot reach the From 9b2b0cf84f8f296ff026b6e76d78fe8a54305b13 Mon Sep 17 00:00:00 2001 From: Shahid Date: Mon, 26 Nov 2018 13:13:06 -0800 Subject: [PATCH 265/879] [SPARK-25451][SPARK-26100][CORE] Aggregated metrics table doesn't show the right number of the total tasks Total tasks in the aggregated table and the tasks table are not matching some times in the WEBUI. We need to force update the executor summary of the particular executorId, when ever last task of that executor has reached. Currently it force update based on last task on the stage end. So, for some particular executorId task might miss at the stage end. Tests to reproduce: ``` bin/spark-shell --master yarn --conf spark.executor.instances=3 sc.parallelize(1 to 10000, 10).map{ x => throw new RuntimeException("Bad executor")}.collect() ``` Before patch: ![screenshot from 2018-11-15 02-24-05](https://user-images.githubusercontent.com/23054875/48511776-b0d36480-e87d-11e8-89a8-ab97216e2c21.png) After patch: ![screenshot from 2018-11-15 02-32-38](https://user-images.githubusercontent.com/23054875/48512141-c39a6900-e87e-11e8-8535-903e1d11d13e.png) Closes #23038 from shahidki31/SPARK-25451. Authored-by: Shahid Signed-off-by: Marcelo Vanzin (cherry picked from commit fbf62b7100be992cbc4eb67e154682db6c91e60e) Signed-off-by: Marcelo Vanzin --- .../spark/status/AppStatusListener.scala | 19 +++++++- .../org/apache/spark/status/LiveEntity.scala | 2 + .../spark/status/AppStatusListenerSuite.scala | 45 +++++++++++++++++++ 3 files changed, 64 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index fdbef6f69c5d..e6f0d08a0bce 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -431,6 +431,7 @@ private[spark] class AppStatusListener( val locality = event.taskInfo.taskLocality.toString() val count = stage.localitySummary.getOrElse(locality, 0L) + 1L stage.localitySummary = stage.localitySummary ++ Map(locality -> count) + stage.activeTasksPerExecutor(event.taskInfo.executorId) += 1 maybeUpdate(stage, now) stage.jobs.foreach { job => @@ -516,6 +517,7 @@ private[spark] class AppStatusListener( if (killedDelta > 0) { stage.killedSummary = killedTasksSummary(event.reason, stage.killedSummary) } + stage.activeTasksPerExecutor(event.taskInfo.executorId) -= 1 // [SPARK-24415] Wait for all tasks to finish before removing stage from live list val removeStage = stage.activeTasks == 0 && @@ -540,7 +542,11 @@ private[spark] class AppStatusListener( if (killedDelta > 0) { job.killedSummary = killedTasksSummary(event.reason, job.killedSummary) } - conditionalLiveUpdate(job, now, removeStage) + if (removeStage) { + update(job, now) + } else { + maybeUpdate(job, now) + } } val esummary = stage.executorSummary(event.taskInfo.executorId) @@ -551,7 +557,16 @@ private[spark] class AppStatusListener( if (metricsDelta != null) { esummary.metrics = LiveEntityHelpers.addMetrics(esummary.metrics, metricsDelta) } - conditionalLiveUpdate(esummary, now, removeStage) + + val isLastTask = stage.activeTasksPerExecutor(event.taskInfo.executorId) == 0 + + // If the last task of the executor finished, then update the esummary + // for both live and history events. + if (isLastTask) { + update(esummary, now) + } else { + maybeUpdate(esummary, now) + } if (!stage.cleaning && stage.savedTasks.get() > maxTasksPerStage) { stage.cleaning = true diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 093b477cdcdd..ad3d3bffafe2 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -373,6 +373,8 @@ private class LiveStage extends LiveEntity { val executorSummaries = new HashMap[String, LiveExecutorStageSummary]() + val activeTasksPerExecutor = new HashMap[String, Int]().withDefaultValue(0) + var blackListedExecutors = new HashSet[String]() // Used for cleanup of tasks after they reach the configured limit. Not written to the store. diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index a6ccd17283fc..b6ddbe01fda9 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1274,6 +1274,51 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(allJobs.head.numFailedStages == 1) } + test("SPARK-25451: total tasks in the executor summary should match total stage tasks") { + val testConf = conf.clone.set(LIVE_ENTITY_UPDATE_PERIOD, Long.MaxValue) + + val listener = new AppStatusListener(store, testConf, true) + + val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details") + listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage), null)) + listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) + + val tasks = createTasks(4, Array("1", "2")) + tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber, task)) + } + + time += 1 + tasks(0).markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptId, "taskType", + Success, tasks(0), null)) + time += 1 + tasks(1).markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptId, "taskType", + Success, tasks(1), null)) + + stage.failureReason = Some("Failed") + listener.onStageCompleted(SparkListenerStageCompleted(stage)) + time += 1 + listener.onJobEnd(SparkListenerJobEnd(1, time, JobFailed(new RuntimeException("Bad Executor")))) + + time += 1 + tasks(2).markFinished(TaskState.FAILED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptId, "taskType", + ExecutorLostFailure("1", true, Some("Lost executor")), tasks(2), null)) + time += 1 + tasks(3).markFinished(TaskState.FAILED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptId, "taskType", + ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), null)) + + val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info) + esummary.foreach { execSummary => + assert(execSummary.failedTasks === 1) + assert(execSummary.succeededTasks === 1) + assert(execSummary.killedTasks === 0) + } + } + test("driver logs") { val listener = new AppStatusListener(store, conf, true) From 400d61b5d8c18b5c5a389e6cd5c583a5ccdbeac8 Mon Sep 17 00:00:00 2001 From: Sergey Zhemzhitsky Date: Wed, 28 Nov 2018 20:22:24 +0800 Subject: [PATCH 266/879] [SPARK-26114][CORE] ExternalSorter's readingIterator field leak ## What changes were proposed in this pull request? This pull request fixes [SPARK-26114](https://issues.apache.org/jira/browse/SPARK-26114) issue that occurs when trying to reduce the number of partitions by means of coalesce without shuffling after shuffle-based transformations. The leak occurs because of not cleaning up `ExternalSorter`'s `readingIterator` field as it's done for its `map` and `buffer` fields. Additionally there are changes to the `CompletionIterator` to prevent capturing its `sub`-iterator and holding it even after the completion iterator completes. It is necessary because in some cases, e.g. in case of standard scala's `flatMap` iterator (which is used is `CoalescedRDD`'s `compute` method) the next value of the main iterator is assigned to `flatMap`'s `cur` field only after it is available. For DAGs where ShuffledRDD is a parent of CoalescedRDD it means that the data should be fetched from the map-side of the shuffle, but the process of fetching this data consumes quite a lot of memory in addition to the memory already consumed by the iterator held by `flatMap`'s `cur` field (until it is reassigned). For the following data ```scala import org.apache.hadoop.io._ import org.apache.hadoop.io.compress._ import org.apache.commons.lang._ import org.apache.spark._ // generate 100M records of sample data sc.makeRDD(1 to 1000, 1000) .flatMap(item => (1 to 100000) .map(i => new Text(RandomStringUtils.randomAlphanumeric(3).toLowerCase) -> new Text(RandomStringUtils.randomAlphanumeric(1024)))) .saveAsSequenceFile("/tmp/random-strings", Some(classOf[GzipCodec])) ``` and the following job ```scala import org.apache.hadoop.io._ import org.apache.spark._ import org.apache.spark.storage._ val rdd = sc.sequenceFile("/tmp/random-strings", classOf[Text], classOf[Text]) rdd .map(item => item._1.toString -> item._2.toString) .repartitionAndSortWithinPartitions(new HashPartitioner(1000)) .coalesce(10,false) .count ``` ... executed like the following ```bash spark-shell \ --num-executors=5 \ --executor-cores=2 \ --master=yarn \ --deploy-mode=client \ --conf spark.executor.memoryOverhead=512 \ --conf spark.executor.memory=1g \ --conf spark.dynamicAllocation.enabled=false \ --conf spark.executor.extraJavaOptions='-XX:+HeapDumpOnOutOfMemoryError -XX:HeapDumpPath=/tmp -Dio.netty.noUnsafe=true' ``` ... executors are always failing with OutOfMemoryErrors. The main issue is multiple leaks of ExternalSorter references. For example, in case of 2 tasks per executor it is expected to be 2 simultaneous instances of ExternalSorter per executor but heap dump generated on OutOfMemoryError shows that there are more ones. ![run1-noparams-dominator-tree-externalsorter](https://user-images.githubusercontent.com/1523889/48703665-782ce580-ec05-11e8-95a9-d6c94e8285ab.png) P.S. This PR does not cover cases with CoGroupedRDDs which use ExternalAppendOnlyMap internally, which itself can lead to OutOfMemoryErrors in many places. ## How was this patch tested? - Existing unit tests - New unit tests - Job executions on the live environment Here is the screenshot before applying this patch ![run3-noparams-failure-ui-5x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700395-f769eb80-ebfc-11e8-831b-e94c757d416c.png) Here is the screenshot after applying this patch ![run3-noparams-success-ui-5x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700610-7a8b4180-ebfd-11e8-9761-baaf38a58e66.png) And in case of reducing the number of executors even more the job is still stable ![run3-noparams-success-ui-2x2-repartition-and-sort](https://user-images.githubusercontent.com/1523889/48700619-82e37c80-ebfd-11e8-98ed-a38e1f1f1fd9.png) Closes #23083 from szhem/SPARK-26114-externalsorter-leak. Authored-by: Sergey Zhemzhitsky Signed-off-by: Wenchen Fan (cherry picked from commit 438f8fd675d8f819373b6643dea3a77d954b6822) Signed-off-by: Wenchen Fan --- .../spark/util/CompletionIterator.scala | 7 ++++-- .../util/collection/ExternalSorter.scala | 3 ++- .../spark/util/CompletionIteratorSuite.scala | 22 +++++++++++++++++++ 3 files changed, 29 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala index 21acaa95c564..f4d6c7a28d2e 100644 --- a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala @@ -25,11 +25,14 @@ private[spark] abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterator[A] { private[this] var completed = false - def next(): A = sub.next() + private[this] var iter = sub + def next(): A = iter.next() def hasNext: Boolean = { - val r = sub.hasNext + val r = iter.hasNext if (!r && !completed) { completed = true + // reassign to release resources of highly resource consuming iterators early + iter = Iterator.empty.asInstanceOf[I] completion() } r diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index b159200d7922..547a862467c8 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -727,9 +727,10 @@ private[spark] class ExternalSorter[K, V, C]( spills.clear() forceSpillFiles.foreach(s => s.file.delete()) forceSpillFiles.clear() - if (map != null || buffer != null) { + if (map != null || buffer != null || readingIterator != null) { map = null // So that the memory can be garbage-collected buffer = null // So that the memory can be garbage-collected + readingIterator = null // So that the memory can be garbage-collected releaseMemory() } } diff --git a/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala index 688fcd9f9aab..29421f7aa9e3 100644 --- a/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/CompletionIteratorSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.util +import java.lang.ref.PhantomReference +import java.lang.ref.ReferenceQueue + import org.apache.spark.SparkFunSuite class CompletionIteratorSuite extends SparkFunSuite { @@ -44,4 +47,23 @@ class CompletionIteratorSuite extends SparkFunSuite { assert(!completionIter.hasNext) assert(numTimesCompleted === 1) } + test("reference to sub iterator should not be available after completion") { + var sub = Iterator(1, 2, 3) + + val refQueue = new ReferenceQueue[Iterator[Int]] + val ref = new PhantomReference[Iterator[Int]](sub, refQueue) + + val iter = CompletionIterator[Int, Iterator[Int]](sub, {}) + sub = null + iter.toArray + + for (_ <- 1 to 100 if !ref.isEnqueued) { + System.gc() + if (!ref.isEnqueued) { + Thread.sleep(10) + } + } + assert(ref.isEnqueued) + assert(refQueue.poll() === ref) + } } From ac26a1dd550b17b9ef8e4b1732e44528365a0395 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 28 Nov 2018 20:38:42 +0800 Subject: [PATCH 267/879] [SPARK-26147][SQL] only pull out unevaluable python udf from join condition https://github.com/apache/spark/pull/22326 made a mistake that, not all python UDFs are unevaluable in join condition. Only python UDFs that refer to attributes from both join side are unevaluable. This PR fixes this mistake. a new test Closes #23153 from cloud-fan/join. Authored-by: Wenchen Fan Signed-off-by: Wenchen Fan (cherry picked from commit affe80958d366f399466a9dba8e03da7f3b7b9bf) Signed-off-by: Wenchen Fan --- python/pyspark/sql/tests.py | 12 + .../spark/sql/catalyst/optimizer/joins.scala | 22 +- ...PullOutPythonUDFInJoinConditionSuite.scala | 217 ++++++++++++++++++ 3 files changed, 240 insertions(+), 11 deletions(-) create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index b05de54773eb..4fc1b4f597a7 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -564,6 +564,18 @@ def test_udf_in_join_condition(self): with self.sql_conf({"spark.sql.crossJoin.enabled": True}): self.assertEqual(df.collect(), [Row(a=1, b=1)]) + def test_udf_in_left_outer_join_condition(self): + # regression test for SPARK-26147 + from pyspark.sql.functions import udf, col + left = self.spark.createDataFrame([Row(a=1)]) + right = self.spark.createDataFrame([Row(b=1)]) + f = udf(lambda a: str(a), StringType()) + # The join condition can't be pushed down, as it refers to attributes from both sides. + # The Python UDF only refer to attributes from one side, so it's evaluable. + df = left.join(right, f("a") == col("b").cast("string"), how="left_outer") + with self.sql_conf({"spark.sql.crossJoin.enabled": True}): + self.assertEqual(df.collect(), [Row(a=1, b=1)]) + def test_udf_in_left_semi_join_condition(self): # regression test for SPARK-25314 from pyspark.sql.functions import udf diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 7149edee0173..6ebb194d71c2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -155,19 +155,20 @@ object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { } /** - * PythonUDF in join condition can not be evaluated, this rule will detect the PythonUDF - * and pull them out from join condition. For python udf accessing attributes from only one side, - * they are pushed down by operation push down rules. If not (e.g. user disables filter push - * down rules), we need to pull them out in this rule too. + * PythonUDF in join condition can't be evaluated if it refers to attributes from both join sides. + * See `ExtractPythonUDFs` for details. This rule will detect un-evaluable PythonUDF and pull them + * out from join condition. */ object PullOutPythonUDFInJoinCondition extends Rule[LogicalPlan] with PredicateHelper { - def hasPythonUDF(expression: Expression): Boolean = { - expression.collectFirst { case udf: PythonUDF => udf }.isDefined + + private def hasUnevaluablePythonUDF(expr: Expression, j: Join): Boolean = { + expr.find { e => + PythonUDF.isScalarPythonUDF(e) && !canEvaluate(e, j.left) && !canEvaluate(e, j.right) + }.isDefined } override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case j @ Join(_, _, joinType, condition) - if condition.isDefined && hasPythonUDF(condition.get) => + case j @ Join(_, _, joinType, Some(cond)) if hasUnevaluablePythonUDF(cond, j) => if (!joinType.isInstanceOf[InnerLike] && joinType != LeftSemi) { // The current strategy only support InnerLike and LeftSemi join because for other type, // it breaks SQL semantic if we run the join condition as a filter after join. If we pass @@ -179,10 +180,9 @@ object PullOutPythonUDFInJoinCondition extends Rule[LogicalPlan] with PredicateH } // If condition expression contains python udf, it will be moved out from // the new join conditions. - val (udf, rest) = - splitConjunctivePredicates(condition.get).partition(hasPythonUDF) + val (udf, rest) = splitConjunctivePredicates(cond).partition(hasUnevaluablePythonUDF(_, j)) val newCondition = if (rest.isEmpty) { - logWarning(s"The join condition:$condition of the join plan contains PythonUDF only," + + logWarning(s"The join condition:$cond of the join plan contains PythonUDF only," + s" it will be moved out and the join plan will be turned to cross join.") None } else { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala new file mode 100644 index 000000000000..3f1c91df7f2e --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala @@ -0,0 +1,217 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.api.python.PythonEvalType +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.PythonUDF +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.internal.SQLConf._ +import org.apache.spark.sql.types.{BooleanType, IntegerType} + +class PullOutPythonUDFInJoinConditionSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Extract PythonUDF From JoinCondition", Once, + PullOutPythonUDFInJoinCondition) :: + Batch("Check Cartesian Products", Once, + CheckCartesianProducts) :: Nil + } + + val attrA = 'a.int + val attrB = 'b.int + val attrC = 'c.int + val attrD = 'd.int + + val testRelationLeft = LocalRelation(attrA, attrB) + val testRelationRight = LocalRelation(attrC, attrD) + + // This join condition refers to attributes from 2 tables, but the PythonUDF inside it only + // refer to attributes from one side. + val evaluableJoinCond = { + val pythonUDF = PythonUDF("evaluable", null, + IntegerType, + Seq(attrA), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + pythonUDF === attrC + } + + // This join condition is a PythonUDF which refers to attributes from 2 tables. + val unevaluableJoinCond = PythonUDF("unevaluable", null, + BooleanType, + Seq(attrA, attrC), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + + val unsupportedJoinTypes = Seq(LeftOuter, RightOuter, FullOuter, LeftAnti) + + private def comparePlanWithCrossJoinEnable(query: LogicalPlan, expected: LogicalPlan): Unit = { + // AnalysisException thrown by CheckCartesianProducts while spark.sql.crossJoin.enabled=false + val exception = intercept[AnalysisException] { + Optimize.execute(query.analyze) + } + assert(exception.message.startsWith("Detected implicit cartesian product")) + + // pull out the python udf while set spark.sql.crossJoin.enabled=true + withSQLConf(CROSS_JOINS_ENABLED.key -> "true") { + val optimized = Optimize.execute(query.analyze) + comparePlans(optimized, expected) + } + } + + test("inner join condition with python udf") { + val query1 = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(unevaluableJoinCond)) + val expected1 = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = None).where(unevaluableJoinCond).analyze + comparePlanWithCrossJoinEnable(query1, expected1) + + // evaluable PythonUDF will not be touched + val query2 = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(evaluableJoinCond)) + comparePlans(Optimize.execute(query2), query2) + } + + test("left semi join condition with python udf") { + val query1 = testRelationLeft.join( + testRelationRight, + joinType = LeftSemi, + condition = Some(unevaluableJoinCond)) + val expected1 = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = None).where(unevaluableJoinCond).select('a, 'b).analyze + comparePlanWithCrossJoinEnable(query1, expected1) + + // evaluable PythonUDF will not be touched + val query2 = testRelationLeft.join( + testRelationRight, + joinType = LeftSemi, + condition = Some(evaluableJoinCond)) + comparePlans(Optimize.execute(query2), query2) + } + + test("unevaluable python udf and common condition") { + val query = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(unevaluableJoinCond && 'a.attr === 'c.attr)) + val expected = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some('a.attr === 'c.attr)).where(unevaluableJoinCond).analyze + val optimized = Optimize.execute(query.analyze) + comparePlans(optimized, expected) + } + + test("unevaluable python udf or common condition") { + val query = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(unevaluableJoinCond || 'a.attr === 'c.attr)) + val expected = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = None).where(unevaluableJoinCond || 'a.attr === 'c.attr).analyze + comparePlanWithCrossJoinEnable(query, expected) + } + + test("pull out whole complex condition with multiple unevaluable python udf") { + val pythonUDF1 = PythonUDF("pythonUDF1", null, + BooleanType, + Seq(attrA, attrC), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + val condition = (unevaluableJoinCond || 'a.attr === 'c.attr) && pythonUDF1 + + val query = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(condition)) + val expected = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = None).where(condition).analyze + comparePlanWithCrossJoinEnable(query, expected) + } + + test("partial pull out complex condition with multiple unevaluable python udf") { + val pythonUDF1 = PythonUDF("pythonUDF1", null, + BooleanType, + Seq(attrA, attrC), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + val condition = (unevaluableJoinCond || pythonUDF1) && 'a.attr === 'c.attr + + val query = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(condition)) + val expected = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some('a.attr === 'c.attr)).where(unevaluableJoinCond || pythonUDF1).analyze + val optimized = Optimize.execute(query.analyze) + comparePlans(optimized, expected) + } + + test("pull out unevaluable python udf when it's mixed with evaluable one") { + val query = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(evaluableJoinCond && unevaluableJoinCond)) + val expected = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(evaluableJoinCond)).where(unevaluableJoinCond).analyze + val optimized = Optimize.execute(query.analyze) + comparePlans(optimized, expected) + } + + test("throw an exception for not support join type") { + for (joinType <- unsupportedJoinTypes) { + val e = intercept[AnalysisException] { + val query = testRelationLeft.join( + testRelationRight, + joinType, + condition = Some(unevaluableJoinCond)) + Optimize.execute(query.analyze) + } + assert(e.message.contentEquals( + s"Using PythonUDF in join condition of join type $joinType is not supported.")) + + val query2 = testRelationLeft.join( + testRelationRight, + joinType, + condition = Some(evaluableJoinCond)) + comparePlans(Optimize.execute(query2), query2) + } + } +} From 99a9107c94c423ef662ba47075615f2cf819281a Mon Sep 17 00:00:00 2001 From: Mark Pavey Date: Wed, 28 Nov 2018 07:19:47 -0800 Subject: [PATCH 268/879] =?UTF-8?q?[SPARK-26137][CORE]=20Use=20Java=20syst?= =?UTF-8?q?em=20property=20"file.separator"=20inste=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit … of hard coded "/" in DependencyUtils ## What changes were proposed in this pull request? Use Java system property "file.separator" instead of hard coded "/" in DependencyUtils. ## How was this patch tested? Manual test: Submit Spark application via REST API that reads data from Elasticsearch using spark-elasticsearch library. Without fix application fails with error: 18/11/22 10:36:20 ERROR Version: Multiple ES-Hadoop versions detected in the classpath; please use only one jar:file:/C:/<...>/spark-2.4.0-bin-hadoop2.6/work/driver-20181122103610-0001/myApp-assembly-1.0.jar jar:file:/C:/<...>/myApp-assembly-1.0.jar 18/11/22 10:36:20 ERROR Main: Application [MyApp] failed: java.lang.Error: Multiple ES-Hadoop versions detected in the classpath; please use only one jar:file:/C:/<...>/spark-2.4.0-bin-hadoop2.6/work/driver-20181122103610-0001/myApp-assembly-1.0.jar jar:file:/C:/<...>/myApp-assembly-1.0.jar at org.elasticsearch.hadoop.util.Version.(Version.java:73) at org.elasticsearch.hadoop.rest.RestService.findPartitions(RestService.java:214) at org.elasticsearch.spark.rdd.AbstractEsRDD.esPartitions$lzycompute(AbstractEsRDD.scala:73) at org.elasticsearch.spark.rdd.AbstractEsRDD.esPartitions(AbstractEsRDD.scala:72) at org.elasticsearch.spark.rdd.AbstractEsRDD.getPartitions(AbstractEsRDD.scala:44) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:253) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:251) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.rdd.RDD.partitions(RDD.scala:251) at org.apache.spark.rdd.MapPartitionsRDD.getPartitions(MapPartitionsRDD.scala:49) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:253) at org.apache.spark.rdd.RDD$$anonfun$partitions$2.apply(RDD.scala:251) at scala.Option.getOrElse(Option.scala:121) at org.apache.spark.rdd.RDD.partitions(RDD.scala:251) at org.apache.spark.SparkContext.runJob(SparkContext.scala:2126) at org.apache.spark.rdd.RDD$$anonfun$collect$1.apply(RDD.scala:945) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:151) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:112) at org.apache.spark.rdd.RDD.withScope(RDD.scala:363) at org.apache.spark.rdd.RDD.collect(RDD.scala:944) ... at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:498) at org.apache.spark.deploy.worker.DriverWrapper$.main(DriverWrapper.scala:65) at org.apache.spark.deploy.worker.DriverWrapper.main(DriverWrapper.scala) With fix application runs successfully. Closes #23102 from markpavey/JIRA_SPARK-26137_DependencyUtilsFileSeparatorFix. Authored-by: Mark Pavey Signed-off-by: Sean Owen (cherry picked from commit ce61bac1d84f8577b180400e44bd9bf22292e0b6) Signed-off-by: Sean Owen --- .../apache/spark/deploy/DependencyUtils.scala | 3 ++- .../spark/deploy/SparkSubmitSuite.scala | 19 +++++++++++++++++++ 2 files changed, 21 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala index 178bdcfccb60..5a17a6b6e169 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala @@ -61,11 +61,12 @@ private[deploy] object DependencyUtils extends Logging { hadoopConf: Configuration, secMgr: SecurityManager): String = { val targetDir = Utils.createTempDir() + val userJarName = userJar.split(File.separatorChar).last Option(jars) .map { resolveGlobPaths(_, hadoopConf) .split(",") - .filterNot(_.contains(userJar.split("/").last)) + .filterNot(_.contains(userJarName)) .mkString(",") } .filterNot(_ == "") diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 652c36ffa6e7..c093789244bf 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -962,6 +962,25 @@ class SparkSubmitSuite } } + test("remove copies of application jar from classpath") { + val fs = File.separator + val sparkConf = new SparkConf(false) + val hadoopConf = new Configuration() + val secMgr = new SecurityManager(sparkConf) + + val appJarName = "myApp.jar" + val jar1Name = "myJar1.jar" + val jar2Name = "myJar2.jar" + val userJar = s"file:/path${fs}to${fs}app${fs}jar$fs$appJarName" + val jars = s"file:/$jar1Name,file:/$appJarName,file:/$jar2Name" + + val resolvedJars = DependencyUtils + .resolveAndDownloadJars(jars, userJar, sparkConf, hadoopConf, secMgr) + + assert(!resolvedJars.contains(appJarName)) + assert(resolvedJars.contains(jar1Name) && resolvedJars.contains(jar2Name)) + } + test("Avoid re-upload remote resources in yarn client mode") { val hadoopConf = new Configuration() updateConfWithFakeS3Fs(hadoopConf) From 7200915fa9cd66fc2259369ce0fe115c2edff1f3 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 29 Nov 2018 22:37:02 +0800 Subject: [PATCH 269/879] [SPARK-26211][SQL] Fix InSet for binary, and struct and array with null. ## What changes were proposed in this pull request? Currently `InSet` doesn't work properly for binary type, or struct and array type with null value in the set. Because, as for binary type, the `HashSet` doesn't work properly for `Array[Byte]`, and as for struct and array type with null value in the set, the `ordering` will throw a `NPE`. ## How was this patch tested? Added a few tests. Closes #23176 from ueshin/issues/SPARK-26211/inset. Authored-by: Takuya UESHIN Signed-off-by: Wenchen Fan (cherry picked from commit b9b68a6dc7d0f735163e980392ea957f2d589923) Signed-off-by: Wenchen Fan --- .../sql/catalyst/expressions/predicates.scala | 33 ++++++------ .../catalyst/expressions/PredicateSuite.scala | 50 ++++++++++++++++++- 2 files changed, 63 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 7f21a628c0bc..eedfbc279d1c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -369,31 +369,26 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with } @transient lazy val set: Set[Any] = child.dataType match { - case _: AtomicType => hset + case t: AtomicType if !t.isInstanceOf[BinaryType] => hset case _: NullType => hset case _ => // for structs use interpreted ordering to be able to compare UnsafeRows with non-UnsafeRows - TreeSet.empty(TypeUtils.getInterpretedOrdering(child.dataType)) ++ hset + TreeSet.empty(TypeUtils.getInterpretedOrdering(child.dataType)) ++ (hset - null) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val setTerm = ctx.addReferenceObj("set", set) - val childGen = child.genCode(ctx) - val setIsNull = if (hasNull) { - s"${ev.isNull} = !${ev.value};" - } else { - "" - } - ev.copy(code = - code""" - |${childGen.code} - |${CodeGenerator.JAVA_BOOLEAN} ${ev.isNull} = ${childGen.isNull}; - |${CodeGenerator.JAVA_BOOLEAN} ${ev.value} = false; - |if (!${ev.isNull}) { - | ${ev.value} = $setTerm.contains(${childGen.value}); - | $setIsNull - |} - """.stripMargin) + nullSafeCodeGen(ctx, ev, c => { + val setTerm = ctx.addReferenceObj("set", set) + val setIsNull = if (hasNull) { + s"${ev.isNull} = !${ev.value};" + } else { + "" + } + s""" + |${ev.value} = $setTerm.contains($c); + |$setIsNull + """.stripMargin + }) } override def sql: String = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala index ac76b17ef476..3b60d1d88b3c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -268,7 +268,7 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(InSet(nl, nS), null) val primitiveTypes = Seq(IntegerType, FloatType, DoubleType, StringType, ByteType, ShortType, - LongType, BinaryType, BooleanType, DecimalType.USER_DEFAULT, TimestampType) + LongType, BooleanType, DecimalType.USER_DEFAULT, TimestampType) primitiveTypes.foreach { t => val dataGen = RandomDataGenerator.forType(t, nullable = true).get val inputData = Seq.fill(10) { @@ -293,6 +293,54 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { } } + test("INSET: binary") { + val hS = HashSet[Any]() + Array(1.toByte, 2.toByte) + Array(3.toByte) + val nS = HashSet[Any]() + Array(1.toByte, 2.toByte) + Array(3.toByte) + null + val onetwo = Literal(Array(1.toByte, 2.toByte)) + val three = Literal(Array(3.toByte)) + val threefour = Literal(Array(3.toByte, 4.toByte)) + val nl = Literal(null, onetwo.dataType) + checkEvaluation(InSet(onetwo, hS), true) + checkEvaluation(InSet(three, hS), true) + checkEvaluation(InSet(three, nS), true) + checkEvaluation(InSet(threefour, hS), false) + checkEvaluation(InSet(threefour, nS), null) + checkEvaluation(InSet(nl, hS), null) + checkEvaluation(InSet(nl, nS), null) + } + + test("INSET: struct") { + val hS = HashSet[Any]() + Literal.create((1, "a")).value + Literal.create((2, "b")).value + val nS = HashSet[Any]() + Literal.create((1, "a")).value + Literal.create((2, "b")).value + null + val oneA = Literal.create((1, "a")) + val twoB = Literal.create((2, "b")) + val twoC = Literal.create((2, "c")) + val nl = Literal(null, oneA.dataType) + checkEvaluation(InSet(oneA, hS), true) + checkEvaluation(InSet(twoB, hS), true) + checkEvaluation(InSet(twoB, nS), true) + checkEvaluation(InSet(twoC, hS), false) + checkEvaluation(InSet(twoC, nS), null) + checkEvaluation(InSet(nl, hS), null) + checkEvaluation(InSet(nl, nS), null) + } + + test("INSET: array") { + val hS = HashSet[Any]() + Literal.create(Seq(1, 2)).value + Literal.create(Seq(3)).value + val nS = HashSet[Any]() + Literal.create(Seq(1, 2)).value + Literal.create(Seq(3)).value + null + val onetwo = Literal.create(Seq(1, 2)) + val three = Literal.create(Seq(3)) + val threefour = Literal.create(Seq(3, 4)) + val nl = Literal(null, onetwo.dataType) + checkEvaluation(InSet(onetwo, hS), true) + checkEvaluation(InSet(three, hS), true) + checkEvaluation(InSet(three, nS), true) + checkEvaluation(InSet(threefour, hS), false) + checkEvaluation(InSet(threefour, nS), null) + checkEvaluation(InSet(nl, hS), null) + checkEvaluation(InSet(nl, nS), null) + } + private case class MyStruct(a: Long, b: String) private case class MyStruct2(a: MyStruct, b: Array[Int]) private val udt = new ExamplePointUDT From 94206c722fc119c6a3c35a8c7a94a48faed5cd44 Mon Sep 17 00:00:00 2001 From: Shahid Date: Thu, 29 Nov 2018 09:48:18 -0800 Subject: [PATCH 270/879] [SPARK-26186][SPARK-26184][CORE] Last updated time is not getting updated for the Inprogress application ## What changes were proposed in this pull request? When the 'spark.history.fs.inProgressOptimization.enabled' is true, inProgress application's last updated time is not getting updated in the History UI. Also, during the cleaning time, InProgress application is getting removed from the listing, even if the last updated time is within the cleaning threshold time. In this PR, if the fastInprogressOptimization enabled, we update the `lastUpdateTime` of the application as last scan time. This will update the `lastUpdateTime` in the historyUI and also while cleaning, it won't remove if the updateTime is within the cleaning interval ## How was this patch tested? Added UT, attached screen shot. Before patch: ![screenshot from 2018-11-27 23-22-38](https://user-images.githubusercontent.com/23054875/49101600-9b5a3380-f29c-11e8-8efc-3fb594e4279a.png) ![screenshot from 2018-11-27 23-20-11](https://user-images.githubusercontent.com/23054875/49101601-9c8b6080-f29c-11e8-928e-643a8c8f4477.png) After Patch: ![screenshot from 2018-11-27 23-37-10](https://user-images.githubusercontent.com/23054875/49101911-669aac00-f29d-11e8-8181-663e4a08ab0e.png) ![screenshot from 2018-11-27 23-39-04](https://user-images.githubusercontent.com/23054875/49102010-a5306680-f29d-11e8-947a-e8a2a09a785a.png) Closes #23158 from shahidki31/HistoryLastUpdateTime. Authored-by: Shahid Signed-off-by: Marcelo Vanzin (cherry picked from commit 24e78b7f163acf6129d934633ae6d3e6d568656a) Signed-off-by: Marcelo Vanzin --- .../deploy/history/FsHistoryProvider.scala | 22 +++++++++++ .../history/FsHistoryProviderSuite.scala | 39 +++++++++++++++++++ 2 files changed, 61 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index c4517d3dfd93..38ed5e0bc169 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -453,6 +453,28 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) if (info.appId.isDefined && fastInProgressParsing) { // When fast in-progress parsing is on, we don't need to re-parse when the // size changes, but we do need to invalidate any existing UIs. + // Also, we need to update the `lastUpdated time` to display the updated time in + // the HistoryUI and to avoid cleaning the inprogress app while running. + val appInfo = listing.read(classOf[ApplicationInfoWrapper], info.appId.get) + + val attemptList = appInfo.attempts.map { attempt => + if (attempt.info.attemptId == info.attemptId) { + new AttemptInfoWrapper( + attempt.info.copy(lastUpdated = new Date(newLastScanTime)), + attempt.logPath, + attempt.fileSize, + attempt.adminAcls, + attempt.viewAcls, + attempt.adminAclsGroups, + attempt.viewAclsGroups) + } else { + attempt + } + } + + val updatedAppInfo = new ApplicationInfoWrapper(appInfo.info, attemptList) + listing.write(updatedAppInfo) + invalidateUI(info.appId.get, info.attemptId) false } else { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index f06b9935ad9e..98ffd722b6f9 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -330,6 +330,45 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc assert(!log2.exists()) } + test("should not clean inprogress application with lastUpdated time less than maxTime") { + val firstFileModifiedTime = TimeUnit.DAYS.toMillis(1) + val secondFileModifiedTime = TimeUnit.DAYS.toMillis(6) + val maxAge = TimeUnit.DAYS.toMillis(7) + val clock = new ManualClock(0) + val provider = new FsHistoryProvider( + createTestConf().set(MAX_LOG_AGE_S, maxAge / 1000), clock) + val log = newLogFile("inProgressApp1", None, inProgress = true) + writeFile(log, true, None, + SparkListenerApplicationStart( + "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")) + ) + clock.setTime(firstFileModifiedTime) + log.setLastModified(clock.getTimeMillis()) + provider.checkForLogs() + writeFile(log, true, None, + SparkListenerApplicationStart( + "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")), + SparkListenerJobStart(0, 1L, Nil, null) + ) + + clock.setTime(secondFileModifiedTime) + log.setLastModified(clock.getTimeMillis()) + provider.checkForLogs() + clock.setTime(TimeUnit.DAYS.toMillis(10)) + writeFile(log, true, None, + SparkListenerApplicationStart( + "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")), + SparkListenerJobStart(0, 1L, Nil, null), + SparkListenerJobEnd(0, 1L, JobSucceeded) + ) + log.setLastModified(clock.getTimeMillis()) + provider.checkForLogs() + // This should not trigger any cleanup + updateAndCheck(provider) { list => + list.size should be(1) + } + } + test("log cleaner for inProgress files") { val firstFileModifiedTime = TimeUnit.SECONDS.toMillis(10) val secondFileModifiedTime = TimeUnit.SECONDS.toMillis(20) From 4661ac76a5d9d01dd505e564d8cf837348fa8653 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Fri, 30 Nov 2018 12:00:55 +0800 Subject: [PATCH 271/879] [SPARK-26188][SQL] FileIndex: don't infer data types of partition columns if user specifies schema ## What changes were proposed in this pull request? This PR is to fix a regression introduced in: https://github.com/apache/spark/pull/21004/files#r236998030 If user specifies schema, Spark don't need to infer data type for of partition columns, otherwise the data type might not match with the one user provided. E.g. for partition directory `p=4d`, after data type inference the column value will be `4.0`. See https://issues.apache.org/jira/browse/SPARK-26188 for more details. Note that user specified schema **might not cover all the data columns**: ``` val schema = new StructType() .add("id", StringType) .add("ex", ArrayType(StringType)) val df = spark.read .schema(schema) .format("parquet") .load(src.toString) assert(df.schema.toList === List( StructField("ex", ArrayType(StringType)), StructField("part", IntegerType), // inferred partitionColumn dataType StructField("id", StringType))) // used user provided partitionColumn dataType ``` For the missing columns in user specified schema, Spark still need to infer their data types if `partitionColumnTypeInferenceEnabled` is enabled. To implement the partially inference, refactor `PartitioningUtils.parsePartitions` and pass the user specified schema as parameter to cast partition values. ## How was this patch tested? Add unit test. Closes #23165 from gengliangwang/fixFileIndex. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan (cherry picked from commit 9cfc3ee6253bed21924424ccaadea0287a6f15f4) Signed-off-by: Wenchen Fan --- .../PartitioningAwareFileIndex.scala | 47 ++----------------- .../datasources/PartitioningUtils.scala | 39 ++++++++++++--- .../datasources/FileIndexSuite.scala | 16 +++++++ .../ParquetPartitionDiscoverySuite.scala | 22 +++++++-- 4 files changed, 72 insertions(+), 52 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index cc8af7b92c45..7b0e4dbcc25f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -126,33 +126,15 @@ abstract class PartitioningAwareFileIndex( val caseInsensitiveOptions = CaseInsensitiveMap(parameters) val timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION) .getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone) - val inferredPartitionSpec = PartitioningUtils.parsePartitions( + + val caseSensitive = sparkSession.sqlContext.conf.caseSensitiveAnalysis + PartitioningUtils.parsePartitions( leafDirs, typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled, basePaths = basePaths, + userSpecifiedSchema = userSpecifiedSchema, + caseSensitive = caseSensitive, timeZoneId = timeZoneId) - userSpecifiedSchema match { - case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => - val userPartitionSchema = - combineInferredAndUserSpecifiedPartitionSchema(inferredPartitionSpec) - - // we need to cast into the data type that user specified. - def castPartitionValuesToUserSchema(row: InternalRow) = { - InternalRow((0 until row.numFields).map { i => - val dt = inferredPartitionSpec.partitionColumns.fields(i).dataType - Cast( - Literal.create(row.get(i, dt), dt), - userPartitionSchema.fields(i).dataType, - Option(timeZoneId)).eval() - }: _*) - } - - PartitionSpec(userPartitionSchema, inferredPartitionSpec.partitions.map { part => - part.copy(values = castPartitionValuesToUserSchema(part.values)) - }) - case _ => - inferredPartitionSpec - } } private def prunePartitions( @@ -233,25 +215,6 @@ abstract class PartitioningAwareFileIndex( val name = path.getName !((name.startsWith("_") && !name.contains("=")) || name.startsWith(".")) } - - /** - * In the read path, only managed tables by Hive provide the partition columns properly when - * initializing this class. All other file based data sources will try to infer the partitioning, - * and then cast the inferred types to user specified dataTypes if the partition columns exist - * inside `userSpecifiedSchema`, otherwise we can hit data corruption bugs like SPARK-18510, or - * inconsistent data types as reported in SPARK-21463. - * @param spec A partition inference result - * @return The PartitionSchema resolved from inference and cast according to `userSpecifiedSchema` - */ - private def combineInferredAndUserSpecifiedPartitionSchema(spec: PartitionSpec): StructType = { - val equality = sparkSession.sessionState.conf.resolver - val resolved = spec.partitionColumns.map { partitionField => - // SPARK-18510: try to get schema from userSpecifiedSchema, otherwise fallback to inferred - userSpecifiedSchema.flatMap(_.find(f => equality(f.name, partitionField.name))).getOrElse( - partitionField) - } - StructType(resolved) - } } object PartitioningAwareFileIndex { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala index 3183fd30e5e0..9d2c9ba0c1a5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningUtils.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCoercion} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils @@ -94,18 +94,34 @@ object PartitioningUtils { paths: Seq[Path], typeInference: Boolean, basePaths: Set[Path], + userSpecifiedSchema: Option[StructType], + caseSensitive: Boolean, timeZoneId: String): PartitionSpec = { - parsePartitions(paths, typeInference, basePaths, DateTimeUtils.getTimeZone(timeZoneId)) + parsePartitions(paths, typeInference, basePaths, userSpecifiedSchema, + caseSensitive, DateTimeUtils.getTimeZone(timeZoneId)) } private[datasources] def parsePartitions( paths: Seq[Path], typeInference: Boolean, basePaths: Set[Path], + userSpecifiedSchema: Option[StructType], + caseSensitive: Boolean, timeZone: TimeZone): PartitionSpec = { + val userSpecifiedDataTypes = if (userSpecifiedSchema.isDefined) { + val nameToDataType = userSpecifiedSchema.get.fields.map(f => f.name -> f.dataType).toMap + if (!caseSensitive) { + CaseInsensitiveMap(nameToDataType) + } else { + nameToDataType + } + } else { + Map.empty[String, DataType] + } + // First, we need to parse every partition's path and see if we can find partition values. val (partitionValues, optDiscoveredBasePaths) = paths.map { path => - parsePartition(path, typeInference, basePaths, timeZone) + parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes, timeZone) }.unzip // We create pairs of (path -> path's partition value) here @@ -147,7 +163,7 @@ object PartitioningUtils { columnNames.zip(literals).map { case (name, Literal(_, dataType)) => // We always assume partition columns are nullable since we've no idea whether null values // will be appended in the future. - StructField(name, dataType, nullable = true) + StructField(name, userSpecifiedDataTypes.getOrElse(name, dataType), nullable = true) } } @@ -185,6 +201,7 @@ object PartitioningUtils { path: Path, typeInference: Boolean, basePaths: Set[Path], + userSpecifiedDataTypes: Map[String, DataType], timeZone: TimeZone): (Option[PartitionValues], Option[Path]) = { val columns = ArrayBuffer.empty[(String, Literal)] // Old Hadoop versions don't have `Path.isRoot` @@ -206,7 +223,7 @@ object PartitioningUtils { // Let's say currentPath is a path of "/table/a=1/", currentPath.getName will give us a=1. // Once we get the string, we try to parse it and find the partition column and value. val maybeColumn = - parsePartitionColumn(currentPath.getName, typeInference, timeZone) + parsePartitionColumn(currentPath.getName, typeInference, userSpecifiedDataTypes, timeZone) maybeColumn.foreach(columns += _) // Now, we determine if we should stop. @@ -239,6 +256,7 @@ object PartitioningUtils { private def parsePartitionColumn( columnSpec: String, typeInference: Boolean, + userSpecifiedDataTypes: Map[String, DataType], timeZone: TimeZone): Option[(String, Literal)] = { val equalSignIndex = columnSpec.indexOf('=') if (equalSignIndex == -1) { @@ -250,7 +268,16 @@ object PartitioningUtils { val rawColumnValue = columnSpec.drop(equalSignIndex + 1) assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") - val literal = inferPartitionColumnValue(rawColumnValue, typeInference, timeZone) + val literal = if (userSpecifiedDataTypes.contains(columnName)) { + // SPARK-26188: if user provides corresponding column schema, get the column value without + // inference, and then cast it as user specified data type. + val columnValue = inferPartitionColumnValue(rawColumnValue, false, timeZone) + val castedValue = + Cast(columnValue, userSpecifiedDataTypes(columnName), Option(timeZone.getID)).eval() + Literal.create(castedValue, userSpecifiedDataTypes(columnName)) + } else { + inferPartitionColumnValue(rawColumnValue, typeInference, timeZone) + } Some(columnName -> literal) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala index 18bb4bfe661c..e2ffe6376377 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileIndexSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.util.{KnownSizeEstimation, SizeEstimator} class FileIndexSuite extends SharedSQLContext { @@ -49,6 +50,21 @@ class FileIndexSuite extends SharedSQLContext { } } + test("SPARK-26188: don't infer data types of partition columns if user specifies schema") { + withTempDir { dir => + val partitionDirectory = new File(dir, s"a=4d") + partitionDirectory.mkdir() + val file = new File(partitionDirectory, "text.txt") + stringToFile(file, "text") + val path = new Path(dir.getCanonicalPath) + val schema = StructType(Seq(StructField("a", StringType, false))) + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, Some(schema)) + val partitionValues = fileIndex.partitionSpec().partitions.map(_.values) + assert(partitionValues.length == 1 && partitionValues(0).numFields == 1 && + partitionValues(0).getString(0) == "4d") + } + } + test("InMemoryFileIndex: input paths are converted to qualified paths") { withTempDir { dir => val file = new File(dir, "text.txt") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala index 9966ed94a839..f808ca458aaa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetPartitionDiscoverySuite.scala @@ -101,7 +101,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha "hdfs://host:9000/path/a=10.5/b=hello") var exception = intercept[AssertionError] { - parsePartitions(paths.map(new Path(_)), true, Set.empty[Path], timeZoneId) + parsePartitions(paths.map(new Path(_)), true, Set.empty[Path], None, true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -115,6 +115,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/")), + None, + true, timeZoneId) // Valid @@ -128,6 +130,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/something=true/table")), + None, + true, timeZoneId) // Valid @@ -141,6 +145,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/table=true")), + None, + true, timeZoneId) // Invalid @@ -154,6 +160,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/")), + None, + true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -174,6 +182,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/tmp/tables/")), + None, + true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -181,13 +191,13 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partition") { def check(path: String, expected: Option[PartitionValues]): Unit = { - val actual = parsePartition(new Path(path), true, Set.empty[Path], timeZone)._1 + val actual = parsePartition(new Path(path), true, Set.empty[Path], Map.empty, timeZone)._1 assert(expected === actual) } def checkThrows[T <: Throwable: Manifest](path: String, expected: String): Unit = { val message = intercept[T] { - parsePartition(new Path(path), true, Set.empty[Path], timeZone) + parsePartition(new Path(path), true, Set.empty[Path], Map.empty, timeZone) }.getMessage assert(message.contains(expected)) @@ -231,6 +241,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha path = new Path("file://path/a=10"), typeInference = true, basePaths = Set(new Path("file://path/a=10")), + Map.empty, timeZone = timeZone)._1 assert(partitionSpec1.isEmpty) @@ -240,6 +251,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha path = new Path("file://path/a=10"), typeInference = true, basePaths = Set(new Path("file://path")), + Map.empty, timeZone = timeZone)._1 assert(partitionSpec2 == @@ -258,6 +270,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, rootPaths, + None, + true, timeZoneId) assert(actualSpec.partitionColumns === spec.partitionColumns) assert(actualSpec.partitions.length === spec.partitions.length) @@ -370,7 +384,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partitions with type inference disabled") { def check(paths: Seq[String], spec: PartitionSpec): Unit = { val actualSpec = - parsePartitions(paths.map(new Path(_)), false, Set.empty[Path], timeZoneId) + parsePartitions(paths.map(new Path(_)), false, Set.empty[Path], None, true, timeZoneId) assert(actualSpec === spec) } From b68decf190e402e3d29fa05726b16bd57fe1b078 Mon Sep 17 00:00:00 2001 From: schintap Date: Fri, 30 Nov 2018 12:48:56 -0600 Subject: [PATCH 272/879] [SPARK-26201] Fix python broadcast with encryption MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Python with rpc and disk encryption enabled along with a python broadcast variable and just read the value back on the driver side the job failed with: Traceback (most recent call last): File "broadcast.py", line 37, in words_new.value File "/pyspark.zip/pyspark/broadcast.py", line 137, in value File "pyspark.zip/pyspark/broadcast.py", line 122, in load_from_path File "pyspark.zip/pyspark/broadcast.py", line 128, in load EOFError: Ran out of input To reproduce use configs: --conf spark.network.crypto.enabled=true --conf spark.io.encryption.enabled=true Code: words_new = sc.broadcast(["scala", "java", "hadoop", "spark", "akka"]) words_new.value print(words_new.value) ## How was this patch tested? words_new = sc.broadcast([“scala”, “java”, “hadoop”, “spark”, “akka”]) textFile = sc.textFile(“README.md”) wordCounts = textFile.flatMap(lambda line: line.split()).map(lambda word: (word + words_new.value[1], 1)).reduceByKey(lambda a, b: a+b) count = wordCounts.count() print(count) words_new.value print(words_new.value) Closes #23166 from redsanket/SPARK-26201. Authored-by: schintap Signed-off-by: Thomas Graves (cherry picked from commit 9b23be2e95fec756066ca0ed3188c3db2602b757) Signed-off-by: Thomas Graves --- .../apache/spark/api/python/PythonRDD.scala | 29 ++++++++++++++++--- python/pyspark/broadcast.py | 21 ++++++++++---- python/pyspark/test_broadcast.py | 15 ++++++++++ 3 files changed, 56 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 8b5a7a9aefea..5ed5070558af 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -660,6 +660,7 @@ private[spark] class PythonBroadcast(@transient var path: String) extends Serial with Logging { private var encryptionServer: PythonServer[Unit] = null + private var decryptionServer: PythonServer[Unit] = null /** * Read data from disks, then copy it to `out` @@ -708,16 +709,36 @@ private[spark] class PythonBroadcast(@transient var path: String) extends Serial override def handleConnection(sock: Socket): Unit = { val env = SparkEnv.get val in = sock.getInputStream() - val dir = new File(Utils.getLocalDir(env.conf)) - val file = File.createTempFile("broadcast", "", dir) - path = file.getAbsolutePath - val out = env.serializerManager.wrapForEncryption(new FileOutputStream(path)) + val abspath = new File(path).getAbsolutePath + val out = env.serializerManager.wrapForEncryption(new FileOutputStream(abspath)) DechunkedInputStream.dechunkAndCopyToOutput(in, out) } } Array(encryptionServer.port, encryptionServer.secret) } + def setupDecryptionServer(): Array[Any] = { + decryptionServer = new PythonServer[Unit]("broadcast-decrypt-server-for-driver") { + override def handleConnection(sock: Socket): Unit = { + val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream())) + Utils.tryWithSafeFinally { + val in = SparkEnv.get.serializerManager.wrapForEncryption(new FileInputStream(path)) + Utils.tryWithSafeFinally { + Utils.copyStream(in, out, false) + } { + in.close() + } + out.flush() + } { + JavaUtils.closeQuietly(out) + } + } + } + Array(decryptionServer.port, decryptionServer.secret) + } + + def waitTillBroadcastDataSent(): Unit = decryptionServer.getResult() + def waitTillDataReceived(): Unit = encryptionServer.getResult() } // scalastyle:on no.finalize diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 1c7f2a7418df..29358b5740e5 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -77,11 +77,12 @@ def __init__(self, sc=None, value=None, pickle_registry=None, path=None, # we're on the driver. We want the pickled data to end up in a file (maybe encrypted) f = NamedTemporaryFile(delete=False, dir=sc._temp_dir) self._path = f.name - python_broadcast = sc._jvm.PythonRDD.setupBroadcast(self._path) + self._sc = sc + self._python_broadcast = sc._jvm.PythonRDD.setupBroadcast(self._path) if sc._encryption_enabled: # with encryption, we ask the jvm to do the encryption for us, we send it data # over a socket - port, auth_secret = python_broadcast.setupEncryptionServer() + port, auth_secret = self._python_broadcast.setupEncryptionServer() (encryption_sock_file, _) = local_connect_and_auth(port, auth_secret) broadcast_out = ChunkedStream(encryption_sock_file, 8192) else: @@ -89,12 +90,14 @@ def __init__(self, sc=None, value=None, pickle_registry=None, path=None, broadcast_out = f self.dump(value, broadcast_out) if sc._encryption_enabled: - python_broadcast.waitTillDataReceived() - self._jbroadcast = sc._jsc.broadcast(python_broadcast) + self._python_broadcast.waitTillDataReceived() + self._jbroadcast = sc._jsc.broadcast(self._python_broadcast) self._pickle_registry = pickle_registry else: # we're on an executor self._jbroadcast = None + self._sc = None + self._python_broadcast = None if sock_file is not None: # the jvm is doing decryption for us. Read the value # immediately from the sock_file @@ -134,7 +137,15 @@ def value(self): """ Return the broadcasted value """ if not hasattr(self, "_value") and self._path is not None: - self._value = self.load_from_path(self._path) + # we only need to decrypt it here when encryption is enabled and + # if its on the driver, since executor decryption is handled already + if self._sc is not None and self._sc._encryption_enabled: + port, auth_secret = self._python_broadcast.setupDecryptionServer() + (decrypted_sock_file, _) = local_connect_and_auth(port, auth_secret) + self._python_broadcast.waitTillBroadcastDataSent() + return self.load(decrypted_sock_file) + else: + self._value = self.load_from_path(self._path) return self._value def unpersist(self, blocking=False): diff --git a/python/pyspark/test_broadcast.py b/python/pyspark/test_broadcast.py index a00329c18ad8..4b6dbf7b5020 100644 --- a/python/pyspark/test_broadcast.py +++ b/python/pyspark/test_broadcast.py @@ -75,6 +75,21 @@ def test_broadcast_with_encryption(self): def test_broadcast_no_encryption(self): self._test_multiple_broadcasts() + def _test_broadcast_on_driver(self, *extra_confs): + conf = SparkConf() + for key, value in extra_confs: + conf.set(key, value) + conf.setMaster("local-cluster[2,1,1024]") + self.sc = SparkContext(conf=conf) + bs = self.sc.broadcast(value=5) + self.assertEqual(5, bs.value) + + def test_broadcast_value_driver_no_encryption(self): + self._test_broadcast_on_driver() + + def test_broadcast_value_driver_encryption(self): + self._test_broadcast_on_driver(("spark.io.encryption.enabled", "true")) + class BroadcastFrameProtocolTest(unittest.TestCase): From 3ec03ecf29ec71590f3d7179f202021be9a4009a Mon Sep 17 00:00:00 2001 From: liuxian Date: Sat, 1 Dec 2018 07:11:31 -0600 Subject: [PATCH 273/879] [MINOR][DOC] Correct some document description errors ## What changes were proposed in this pull request? Correct some document description errors. ## How was this patch tested? N/A Closes #23162 from 10110346/docerror. Authored-by: liuxian Signed-off-by: Sean Owen (cherry picked from commit 60e4239a1e3506d342099981b6e3b3b8431a203e) Signed-off-by: Sean Owen --- .../org/apache/spark/internal/config/package.scala | 10 +++++----- .../scala/org/apache/spark/sql/internal/SQLConf.scala | 6 +++--- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index 3b3c45fc18fb..5c17b9b3a320 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -240,7 +240,7 @@ package object config { private[spark] val LISTENER_BUS_EVENT_QUEUE_CAPACITY = ConfigBuilder("spark.scheduler.listenerbus.eventqueue.capacity") .intConf - .checkValue(_ > 0, "The capacity of listener bus event queue must not be negative") + .checkValue(_ > 0, "The capacity of listener bus event queue must be positive") .createWithDefault(10000) private[spark] val LISTENER_BUS_METRICS_MAX_LISTENER_CLASSES_TIMED = @@ -389,8 +389,8 @@ package object config { .doc("The chunk size in bytes during writing out the bytes of ChunkedByteBuffer.") .bytesConf(ByteUnit.BYTE) .checkValue(_ <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, - "The chunk size during writing out the bytes of" + - " ChunkedByteBuffer should not larger than Int.MaxValue - 15.") + "The chunk size during writing out the bytes of ChunkedByteBuffer should" + + s" be less than or equal to ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") .createWithDefault(64 * 1024 * 1024) private[spark] val CHECKPOINT_COMPRESS = @@ -462,7 +462,7 @@ package object config { "made in creating intermediate shuffle files.") .bytesConf(ByteUnit.KiB) .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, - s"The file buffer size must be greater than 0 and less than" + + s"The file buffer size must be positive and less than or equal to" + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") @@ -472,7 +472,7 @@ package object config { "is written in unsafe shuffle writer. In KiB unless otherwise specified.") .bytesConf(ByteUnit.KiB) .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, - s"The buffer size must be greater than 0 and less than" + + s"The buffer size must be positive and less than or equal to" + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 99e601ad9f4c..3dad1e34af23 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -306,7 +306,7 @@ object SQLConf { "factor as the estimated data size, in case the data is compressed in the file and lead to" + " a heavily underestimated result.") .doubleConf - .checkValue(_ > 0, "the value of fileDataSizeFactor must be larger than 0") + .checkValue(_ > 0, "the value of fileDataSizeFactor must be greater than 0") .createWithDefault(1.0) val PARQUET_SCHEMA_MERGING_ENABLED = buildConf("spark.sql.parquet.mergeSchema") @@ -648,7 +648,7 @@ object SQLConf { val BUCKETING_MAX_BUCKETS = buildConf("spark.sql.sources.bucketing.maxBuckets") .doc("The maximum number of buckets allowed. Defaults to 100000") .intConf - .checkValue(_ > 0, "the value of spark.sql.sources.bucketing.maxBuckets must be larger than 0") + .checkValue(_ > 0, "the value of spark.sql.sources.bucketing.maxBuckets must be greater than 0") .createWithDefault(100000) val CROSS_JOINS_ENABLED = buildConf("spark.sql.crossJoin.enabled") @@ -1116,7 +1116,7 @@ object SQLConf { .internal() .doc("The number of bins when generating histograms.") .intConf - .checkValue(num => num > 1, "The number of bins must be larger than 1.") + .checkValue(num => num > 1, "The number of bins must be greater than 1.") .createWithDefault(254) val PERCENTILE_ACCURACY = From 58a4c0ce7530577561be3c5106628dcad06eee18 Mon Sep 17 00:00:00 2001 From: hyukjinkwon Date: Sun, 2 Dec 2018 17:41:08 +0800 Subject: [PATCH 274/879] [SPARK-26080][PYTHON] Skips Python resource limit on Windows in Python worker ## What changes were proposed in this pull request? `resource` package is a Unix specific package. See https://docs.python.org/2/library/resource.html and https://docs.python.org/3/library/resource.html. Note that we document Windows support: > Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). This should be backported into branch-2.4 to restore Windows support in Spark 2.4.1. ## How was this patch tested? Manually mocking the changed logics. Closes #23055 from HyukjinKwon/SPARK-26080. Lead-authored-by: hyukjinkwon Co-authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit 9cda9a892d03f60a76cd5d9b4546e72c50962c85) Signed-off-by: Hyukjin Kwon --- docs/configuration.md | 2 ++ python/pyspark/worker.py | 19 ++++++++++++------- 2 files changed, 14 insertions(+), 7 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 944e5e4c2e97..042e57dd2016 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -190,6 +190,8 @@ of the most common options to set are: and it is up to the application to avoid exceeding the overhead memory space shared with other non-JVM processes. When PySpark is run in YARN or Kubernetes, this memory is added to executor resource requests. + + NOTE: Python memory usage may not be limited on platforms that do not support resource limiting, such as Windows. diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 8c59f1f999f1..953b468e9651 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -22,7 +22,12 @@ import os import sys import time -import resource +# 'resource' is a Unix specific module. +has_resource_module = True +try: + import resource +except ImportError: + has_resource_module = False import socket import traceback @@ -268,9 +273,9 @@ def main(infile, outfile): # set up memory limits memory_limit_mb = int(os.environ.get('PYSPARK_EXECUTOR_MEMORY_MB', "-1")) - total_memory = resource.RLIMIT_AS - try: - if memory_limit_mb > 0: + if memory_limit_mb > 0 and has_resource_module: + total_memory = resource.RLIMIT_AS + try: (soft_limit, hard_limit) = resource.getrlimit(total_memory) msg = "Current mem limits: {0} of max {1}\n".format(soft_limit, hard_limit) print(msg, file=sys.stderr) @@ -283,9 +288,9 @@ def main(infile, outfile): print(msg, file=sys.stderr) resource.setrlimit(total_memory, (new_limit, new_limit)) - except (resource.error, OSError, ValueError) as e: - # not all systems support resource limits, so warn instead of failing - print("WARN: Failed to set memory limit: {0}\n".format(e), file=sys.stderr) + except (resource.error, OSError, ValueError) as e: + # not all systems support resource limits, so warn instead of failing + print("WARN: Failed to set memory limit: {0}\n".format(e), file=sys.stderr) # initialize global state taskContext = None From 91b86b7f3fdd87c2c95603a53df5cc9f373c681c Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Sun, 2 Dec 2018 08:52:01 -0600 Subject: [PATCH 275/879] [SPARK-26198][SQL] Fix Metadata serialize null values throw NPE ## What changes were proposed in this pull request? How to reproduce this issue: ```scala scala> val meta = new org.apache.spark.sql.types.MetadataBuilder().putNull("key").build().json java.lang.NullPointerException at org.apache.spark.sql.types.Metadata$.org$apache$spark$sql$types$Metadata$$toJsonValue(Metadata.scala:196) at org.apache.spark.sql.types.Metadata$$anonfun$1.apply(Metadata.scala:180) ``` This pr fix `NullPointerException` when `Metadata` serialize `null` values. ## How was this patch tested? unit tests Closes #23164 from wangyum/SPARK-26198. Authored-by: Yuming Wang Signed-off-by: Sean Owen (cherry picked from commit 676bbb2446af1f281b8f76a5428b7ba75b7588b3) Signed-off-by: Sean Owen --- .../src/main/scala/org/apache/spark/sql/types/Metadata.scala | 2 ++ .../scala/org/apache/spark/sql/types/MetadataSuite.scala | 5 +++++ 2 files changed, 7 insertions(+) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index 7c15dc0de4b6..e79ab7a6bdbb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -190,6 +190,8 @@ object Metadata { JBool(x) case x: String => JString(x) + case null => + JNull case x: Metadata => toJsonValue(x.map) case other => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala index 210e65708170..b4aeac562d2b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala @@ -26,6 +26,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.## !== 0) assert(meta.getString("key") === "value") assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getString("no_such_key")) intercept[ClassCastException](meta.getBoolean("key")) } @@ -36,6 +37,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.## !== 0) assert(meta.getLong("key") === 12) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getLong("no_such_key")) intercept[ClassCastException](meta.getBoolean("key")) } @@ -46,6 +48,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.## !== 0) assert(meta.getDouble("key") === 12) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getDouble("no_such_key")) intercept[ClassCastException](meta.getBoolean("key")) } @@ -56,6 +59,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.## !== 0) assert(meta.getBoolean("key") === true) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getBoolean("no_such_key")) intercept[ClassCastException](meta.getString("key")) } @@ -69,6 +73,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.getLong("key") === 0) assert(meta.getBoolean("key") === false) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getLong("no_such_key")) } } From f716a4788dbb6055713d75875e3fdd0ceed93452 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 3 Dec 2018 23:54:26 +0800 Subject: [PATCH 276/879] [SPARK-26181][SQL] the `hasMinMaxStats` method of `ColumnStatsMap` is not correct ## What changes were proposed in this pull request? For now the `hasMinMaxStats` will return the same as `hasCountStats`, which is obviously not as expected. ## How was this patch tested? Existing tests. Closes #23152 from adrian-wang/minmaxstats. Authored-by: Daoyuan Wang Signed-off-by: Wenchen Fan (cherry picked from commit 8534d753ecb21ea64ffbaefb5eaca38ba0464c6d) Signed-off-by: Wenchen Fan --- .../statsEstimation/FilterEstimation.scala | 14 +++++++--- .../FilterEstimationSuite.scala | 27 +++++++++++++++++++ .../sql/hive/execution/SQLQuerySuite.scala | 14 ++++++++++ 3 files changed, 52 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala index 5a3eeefaedb1..2c5beef43f52 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala @@ -670,6 +670,14 @@ case class FilterEstimation(plan: Filter) extends Logging { logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft) return None case _ => + if (!colStatsMap.hasMinMaxStats(attrLeft)) { + logDebug("[CBO] No min/max statistics for " + attrLeft) + return None + } + if (!colStatsMap.hasMinMaxStats(attrRight)) { + logDebug("[CBO] No min/max statistics for " + attrRight) + return None + } } val colStatLeft = colStatsMap(attrLeft) @@ -879,13 +887,13 @@ case class ColumnStatsMap(originalMap: AttributeMap[ColumnStat]) { } def hasCountStats(a: Attribute): Boolean = - get(a).map(_.hasCountStats).getOrElse(false) + get(a).exists(_.hasCountStats) def hasDistinctCount(a: Attribute): Boolean = - get(a).map(_.distinctCount.isDefined).getOrElse(false) + get(a).exists(_.distinctCount.isDefined) def hasMinMaxStats(a: Attribute): Boolean = - get(a).map(_.hasCountStats).getOrElse(false) + get(a).exists(_.hasMinMaxStats) /** * Gets column stat for the given attribute. Prefer the column stat in updatedMap than that in diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala index 47bfa6256958..b0a47e783512 100755 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.plans.LeftOuter import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.{ColumnStatsMap, FilterEstimation} import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ @@ -821,6 +822,32 @@ class FilterEstimationSuite extends StatsEstimationTestBase { expectedRowCount = 3) } + test("ColumnStatsMap tests") { + val attrNoDistinct = AttributeReference("att_without_distinct", IntegerType)() + val attrNoCount = AttributeReference("att_without_count", BooleanType)() + val attrNoMinMax = AttributeReference("att_without_min_max", DateType)() + val colStatNoDistinct = ColumnStat(distinctCount = None, min = Some(1), max = Some(10), + nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)) + val colStatNoCount = ColumnStat(distinctCount = Some(2), min = Some(false), max = Some(true), + nullCount = None, avgLen = Some(1), maxLen = Some(1)) + val colStatNoMinMax = ColumnStat(distinctCount = Some(1), min = None, max = None, + nullCount = Some(1), avgLen = None, maxLen = None) + val columnStatsMap = ColumnStatsMap(AttributeMap(Seq( + attrNoDistinct -> colStatNoDistinct, + attrNoCount -> colStatNoCount, + attrNoMinMax -> colStatNoMinMax + ))) + assert(!columnStatsMap.hasDistinctCount(attrNoDistinct)) + assert(columnStatsMap.hasDistinctCount(attrNoCount)) + assert(columnStatsMap.hasDistinctCount(attrNoMinMax)) + assert(!columnStatsMap.hasCountStats(attrNoDistinct)) + assert(!columnStatsMap.hasCountStats(attrNoCount)) + assert(columnStatsMap.hasCountStats(attrNoMinMax)) + assert(columnStatsMap.hasMinMaxStats(attrNoDistinct)) + assert(columnStatsMap.hasMinMaxStats(attrNoCount)) + assert(!columnStatsMap.hasMinMaxStats(attrNoMinMax)) + } + private def childStatsTestPlan(outList: Seq[Attribute], tableRowCount: BigInt): StatsTestPlan = { StatsTestPlan( outputList = outList, 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 dfcde8cc0d39..fab2a27cdef1 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 @@ -2276,4 +2276,18 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + + test("SPARK-26181 hasMinMaxStats method of ColumnStatsMap is not correct") { + withSQLConf(SQLConf.CBO_ENABLED.key -> "true") { + withTable("all_null") { + sql("create table all_null (attr1 int, attr2 int)") + sql("insert into all_null values (null, null)") + sql("analyze table all_null compute statistics for columns attr1, attr2") + // check if the stats can be calculated without Cast exception. + sql("select * from all_null where attr1 < 1").queryExecution.stringWithStats + sql("select * from all_null where attr1 < attr2").queryExecution.stringWithStats + } + } + } + } From 349e25bd42764accbbc619a70dd59ce524cfdbf2 Mon Sep 17 00:00:00 2001 From: Stavros Kontopoulos Date: Mon, 3 Dec 2018 14:57:18 -0800 Subject: [PATCH 277/879] [SPARK-26256][K8S] Fix labels for pod deletion Adds proper labels when deleting executor pods. Manually with tests. Closes #23209 from skonto/fix-deletion-labels. Authored-by: Stavros Kontopoulos Signed-off-by: Marcelo Vanzin (cherry picked from commit a24e1a126c55fc06f5867c0e5e5b0ee71201e018) Signed-off-by: Marcelo Vanzin --- .../scheduler/cluster/k8s/ExecutorPodsAllocator.scala | 2 ++ .../cluster/k8s/ExecutorPodsAllocatorSuite.scala | 10 +++++++++- 2 files changed, 11 insertions(+), 1 deletion(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 77bb9c3fcc9f..35d68e3ab821 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -89,6 +89,8 @@ private[spark] class ExecutorPodsAllocator( Utils.tryLogNonFatalError { kubernetesClient .pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .withLabel(SPARK_EXECUTOR_ID_LABEL, execId.toString) .delete() } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 0e617b002101..ee964cb3cb28 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -138,7 +138,15 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.notifySubscribers() snapshotsStore.replaceSnapshot(Seq.empty[Pod]) waitForExecutorPodsClock.setTime(podCreationTimeout + 1) - when(podOperations.withLabel(SPARK_EXECUTOR_ID_LABEL, "1")).thenReturn(labeledPods) + when(podOperations + .withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)) + .thenReturn(podOperations) + when(podOperations + withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) + .thenReturn(podOperations) + when(podOperations + .withLabel(SPARK_EXECUTOR_ID_LABEL, "1")) + .thenReturn(labeledPods) snapshotsStore.notifySubscribers() verify(labeledPods).delete() verify(podOperations).create(podWithAttachedContainerForId(2)) From 90fcd12af936792a99738789ba1eeb9a1e7e3ce1 Mon Sep 17 00:00:00 2001 From: Shahid Date: Mon, 3 Dec 2018 15:11:43 -0800 Subject: [PATCH 278/879] [SPARK-26219][CORE][BRANCH-2.4] Executor summary should get updated for failure jobs in the history server UI Back port the commit https://github.com/apache/spark/pull/23181 into Spark2.4 branch Added UT Closes #23191 from shahidki31/branch-2.4. Authored-by: Shahid Signed-off-by: Marcelo Vanzin --- .../spark/status/AppStatusListener.scala | 19 ++-- .../spark/status/AppStatusListenerSuite.scala | 94 ++++++++++++------- 2 files changed, 66 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index e6f0d08a0bce..5b564efa9684 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -599,9 +599,14 @@ private[spark] class AppStatusListener( } } - // Force an update on live applications when the number of active tasks reaches 0. This is - // checked in some tests (e.g. SQLTestUtilsBase) so it needs to be reliably up to date. - conditionalLiveUpdate(exec, now, exec.activeTasks == 0) + // Force an update on both live and history applications when the number of active tasks + // reaches 0. This is checked in some tests (e.g. SQLTestUtilsBase) so it needs to be + // reliably up to date. + if (exec.activeTasks == 0) { + update(exec, now) + } else { + maybeUpdate(exec, now) + } } } @@ -954,14 +959,6 @@ private[spark] class AppStatusListener( } } - private def conditionalLiveUpdate(entity: LiveEntity, now: Long, condition: Boolean): Unit = { - if (condition) { - liveUpdate(entity, now) - } else { - maybeUpdate(entity, now) - } - } - private def cleanupExecutors(count: Long): Unit = { // Because the limit is on the number of *dead* executors, we need to calculate whether // there are actually enough dead executors to be deleted. diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index b6ddbe01fda9..f34be48a4d00 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -1274,48 +1274,70 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(allJobs.head.numFailedStages == 1) } - test("SPARK-25451: total tasks in the executor summary should match total stage tasks") { - val testConf = conf.clone.set(LIVE_ENTITY_UPDATE_PERIOD, Long.MaxValue) + Seq(true, false).foreach { live => + test(s"Total tasks in the executor summary should match total stage tasks (live = $live)") { - val listener = new AppStatusListener(store, testConf, true) + val testConf = if (live) { + conf.clone().set(LIVE_ENTITY_UPDATE_PERIOD, Long.MaxValue) + } else { + conf.clone().set(LIVE_ENTITY_UPDATE_PERIOD, -1L) + } - val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details") - listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage), null)) - listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) + val listener = new AppStatusListener(store, testConf, live) - val tasks = createTasks(4, Array("1", "2")) - tasks.foreach { task => - listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber, task)) - } + Seq("1", "2").foreach { execId => + listener.onExecutorAdded(SparkListenerExecutorAdded(0L, execId, + new ExecutorInfo("host1", 1, Map.empty))) + } + val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details") + listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage), null)) + listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) - time += 1 - tasks(0).markFinished(TaskState.FINISHED, time) - listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptId, "taskType", - Success, tasks(0), null)) - time += 1 - tasks(1).markFinished(TaskState.FINISHED, time) - listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptId, "taskType", - Success, tasks(1), null)) + val tasks = createTasks(4, Array("1", "2")) + tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber, task)) + } - stage.failureReason = Some("Failed") - listener.onStageCompleted(SparkListenerStageCompleted(stage)) - time += 1 - listener.onJobEnd(SparkListenerJobEnd(1, time, JobFailed(new RuntimeException("Bad Executor")))) + time += 1 + tasks(0).markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + Success, tasks(0), null)) + time += 1 + tasks(1).markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + Success, tasks(1), null)) - time += 1 - tasks(2).markFinished(TaskState.FAILED, time) - listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptId, "taskType", - ExecutorLostFailure("1", true, Some("Lost executor")), tasks(2), null)) - time += 1 - tasks(3).markFinished(TaskState.FAILED, time) - listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptId, "taskType", - ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), null)) - - val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info) - esummary.foreach { execSummary => - assert(execSummary.failedTasks === 1) - assert(execSummary.succeededTasks === 1) - assert(execSummary.killedTasks === 0) + stage.failureReason = Some("Failed") + listener.onStageCompleted(SparkListenerStageCompleted(stage)) + time += 1 + listener.onJobEnd(SparkListenerJobEnd(1, time, JobFailed( + new RuntimeException("Bad Executor")))) + + time += 1 + tasks(2).markFinished(TaskState.FAILED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + ExecutorLostFailure("1", true, Some("Lost executor")), tasks(2), null)) + time += 1 + tasks(3).markFinished(TaskState.FAILED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), null)) + + val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info) + esummary.foreach { execSummary => + assert(execSummary.failedTasks === 1) + assert(execSummary.succeededTasks === 1) + assert(execSummary.killedTasks === 0) + } + + val allExecutorSummary = store.view(classOf[ExecutorSummaryWrapper]).asScala.map(_.info) + assert(allExecutorSummary.size === 2) + allExecutorSummary.foreach { allExecSummary => + assert(allExecSummary.failedTasks === 1) + assert(allExecSummary.activeTasks === 0) + assert(allExecSummary.completedTasks === 1) + } + store.delete(classOf[ExecutorSummaryWrapper], "1") + store.delete(classOf[ExecutorSummaryWrapper], "2") } } From a091216a6d34ec998de05dca441ae5a368c13c22 Mon Sep 17 00:00:00 2001 From: Yuming Wang Date: Tue, 4 Dec 2018 07:57:58 -0600 Subject: [PATCH 279/879] [SPARK-24423][FOLLOW-UP][SQL] Fix error example ## What changes were proposed in this pull request? ![image](https://user-images.githubusercontent.com/5399861/49172173-42ad9800-f37b-11e8-8135-7adc323357ae.png) It will throw: ``` requirement failed: When reading JDBC data sources, users need to specify all or none for the following options: 'partitionColumn', 'lowerBound', 'upperBound', and 'numPartitions' ``` and ``` User-defined partition column subq.c1 not found in the JDBC relation ... ``` This PR fix this error example. ## How was this patch tested? manual tests Closes #23170 from wangyum/SPARK-24499. Authored-by: Yuming Wang Signed-off-by: Sean Owen (cherry picked from commit 06a3b6aafa510ede2f1376b29a46f99447286c67) Signed-off-by: Sean Owen --- docs/sql-data-sources-jdbc.md | 6 +++--- .../sql/execution/datasources/jdbc/JDBCOptions.scala | 10 +++++++--- .../scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 10 +++++++--- 3 files changed, 17 insertions(+), 9 deletions(-) diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md index 057e8217241a..0f2bc4941744 100644 --- a/docs/sql-data-sources-jdbc.md +++ b/docs/sql-data-sources-jdbc.md @@ -64,9 +64,9 @@ the following case-insensitive options: Example:
    spark.read.format("jdbc")
    -    .option("dbtable", "(select c1, c2 from t1) as subq")
    -    .option("partitionColumn", "subq.c1"
    -    .load() + .option("url", jdbcUrl)
    + .option("query", "select c1, c2 from t1")
    + .load()
    diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 7dfbb9d8b5c0..b4469cb538fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -137,9 +137,13 @@ class JDBCOptions( |the partition columns using the supplied subquery alias to resolve any ambiguity. |Example : |spark.read.format("jdbc") - | .option("dbtable", "(select c1, c2 from t1) as subq") - | .option("partitionColumn", "subq.c1" - | .load() + | .option("url", jdbcUrl) + | .option("dbtable", "(select c1, c2 from t1) as subq") + | .option("partitionColumn", "c1") + | .option("lowerBound", "1") + | .option("upperBound", "100") + | .option("numPartitions", "3") + | .load() """.stripMargin ) 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 7fa0e7fc162c..71e83767964a 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 @@ -1348,9 +1348,13 @@ class JDBCSuite extends QueryTest |the partition columns using the supplied subquery alias to resolve any ambiguity. |Example : |spark.read.format("jdbc") - | .option("dbtable", "(select c1, c2 from t1) as subq") - | .option("partitionColumn", "subq.c1" - | .load() + | .option("url", jdbcUrl) + | .option("dbtable", "(select c1, c2 from t1) as subq") + | .option("partitionColumn", "c1") + | .option("lowerBound", "1") + | .option("upperBound", "100") + | .option("numPartitions", "3") + | .load() """.stripMargin val e5 = intercept[RuntimeException] { sql( From 51739d1ae06801138f06dcc441fc10c9b821789b Mon Sep 17 00:00:00 2001 From: Shahid Date: Tue, 4 Dec 2018 11:00:58 -0800 Subject: [PATCH 280/879] [SPARK-26119][CORE][WEBUI] Task summary table should contain only successful tasks' metrics ## What changes were proposed in this pull request? Task summary table in the stage page currently displays the summary of all the tasks. However, we should display the task summary of only successful tasks, to follow the behavior of previous versions of spark. ## How was this patch tested? Added UT. attached screenshot Before patch: ![screenshot from 2018-11-20 00-36-18](https://user-images.githubusercontent.com/23054875/48729339-62e3a580-ec5d-11e8-81f0-0d191a234ffe.png) ![screenshot from 2018-11-20 01-18-37](https://user-images.githubusercontent.com/23054875/48731112-41d18380-ec62-11e8-8c31-1ffbfa04e746.png) Closes #23088 from shahidki31/summaryMetrics. Authored-by: Shahid Signed-off-by: Marcelo Vanzin (cherry picked from commit 35f9163adf5c067229afbe57ed60d5dd5f2422c8) Signed-off-by: Marcelo Vanzin --- .../apache/spark/status/AppStatusStore.scala | 73 +++++++++++++------ .../spark/status/AppStatusStoreSuite.scala | 33 ++++++++- 2 files changed, 81 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index e237281c552b..84716f8471bd 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -146,11 +146,20 @@ private[spark] class AppStatusStore( // cheaper for disk stores (avoids deserialization). val count = { Utils.tryWithResource( - store.view(classOf[TaskDataWrapper]) - .parent(stageKey) - .index(TaskIndexNames.EXEC_RUN_TIME) - .first(0L) - .closeableIterator() + if (store.isInstanceOf[InMemoryStore]) { + store.view(classOf[TaskDataWrapper]) + .parent(stageKey) + .index(TaskIndexNames.STATUS) + .first("SUCCESS") + .last("SUCCESS") + .closeableIterator() + } else { + store.view(classOf[TaskDataWrapper]) + .parent(stageKey) + .index(TaskIndexNames.EXEC_RUN_TIME) + .first(0L) + .closeableIterator() + } ) { it => var _count = 0L while (it.hasNext()) { @@ -219,30 +228,50 @@ private[spark] class AppStatusStore( // stabilize once the stage finishes. It's also slow, especially with disk stores. val indices = quantiles.map { q => math.min((q * count).toLong, count - 1) } + // TODO: Summary metrics needs to display all the successful tasks' metrics (SPARK-26119). + // For InMemory case, it is efficient to find using the following code. But for diskStore case + // we need an efficient solution to avoid deserialization time overhead. For that, we need to + // rework on the way indexing works, so that we can index by specific metrics for successful + // and failed tasks differently (would be tricky). Also would require changing the disk store + // version (to invalidate old stores). def scanTasks(index: String)(fn: TaskDataWrapper => Long): IndexedSeq[Double] = { - Utils.tryWithResource( - store.view(classOf[TaskDataWrapper]) + if (store.isInstanceOf[InMemoryStore]) { + val quantileTasks = store.view(classOf[TaskDataWrapper]) .parent(stageKey) .index(index) .first(0L) - .closeableIterator() - ) { it => - var last = Double.NaN - var currentIdx = -1L - indices.map { idx => - if (idx == currentIdx) { - last - } else { - val diff = idx - currentIdx - currentIdx = idx - if (it.skip(diff - 1)) { - last = fn(it.next()).toDouble + .asScala + .filter { _.status == "SUCCESS"} // Filter "SUCCESS" tasks + .toIndexedSeq + + indices.map { index => + fn(quantileTasks(index.toInt)).toDouble + }.toIndexedSeq + } else { + Utils.tryWithResource( + store.view(classOf[TaskDataWrapper]) + .parent(stageKey) + .index(index) + .first(0L) + .closeableIterator() + ) { it => + var last = Double.NaN + var currentIdx = -1L + indices.map { idx => + if (idx == currentIdx) { last } else { - Double.NaN + val diff = idx - currentIdx + currentIdx = idx + if (it.skip(diff - 1)) { + last = fn(it.next()).toDouble + last + } else { + Double.NaN + } } - } - }.toIndexedSeq + }.toIndexedSeq + } } } diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala index 92f90f3d96dd..75a658161d3f 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala @@ -77,6 +77,34 @@ class AppStatusStoreSuite extends SparkFunSuite { assert(store.count(classOf[CachedQuantile]) === 2) } + test("only successfull task have taskSummary") { + val store = new InMemoryStore() + (0 until 5).foreach { i => store.write(newTaskData(i, status = "FAILED")) } + val appStore = new AppStatusStore(store).taskSummary(stageId, attemptId, uiQuantiles) + assert(appStore.size === 0) + } + + test("summary should contain task metrics of only successfull tasks") { + val store = new InMemoryStore() + + for (i <- 0 to 5) { + if (i % 2 == 1) { + store.write(newTaskData(i, status = "FAILED")) + } else { + store.write(newTaskData(i)) + } + } + + val summary = new AppStatusStore(store).taskSummary(stageId, attemptId, uiQuantiles).get + + val values = Array(0.0, 2.0, 4.0) + + val dist = new Distribution(values, 0, values.length).getQuantiles(uiQuantiles.sorted) + dist.zip(summary.executorRunTime).foreach { case (expected, actual) => + assert(expected === actual) + } + } + private def compareQuantiles(count: Int, quantiles: Array[Double]): Unit = { val store = new InMemoryStore() val values = (0 until count).map { i => @@ -93,12 +121,11 @@ class AppStatusStoreSuite extends SparkFunSuite { } } - private def newTaskData(i: Int): TaskDataWrapper = { + private def newTaskData(i: Int, status: String = "SUCCESS"): TaskDataWrapper = { new TaskDataWrapper( - i, i, i, i, i, i, i.toString, i.toString, i.toString, i.toString, false, Nil, None, + i, i, i, i, i, i, i.toString, i.toString, status, i.toString, false, Nil, None, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, stageId, attemptId) } - } From d9b707e7c39a55a22dd55f8a4f537d861a3ce57c Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 5 Dec 2018 19:30:25 +0800 Subject: [PATCH 281/879] [SPARK-26133][ML][FOLLOWUP] Fix doc for OneHotEncoder ## What changes were proposed in this pull request? This fixes doc of renamed OneHotEncoder in PySpark. ## How was this patch tested? N/A Closes #23230 from viirya/remove_one_hot_encoder_followup. Authored-by: Liang-Chi Hsieh Signed-off-by: Hyukjin Kwon --- python/pyspark/ml/feature.py | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index eccb7acae5b9..93ece80af1e6 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1731,22 +1731,22 @@ class OneHotEncoderEstimator(JavaEstimator, HasInputCols, HasOutputCols, HasHand at most a single one-value per row that indicates the input category index. For example with 5 categories, an input value of 2.0 would map to an output vector of `[0.0, 0.0, 1.0, 0.0]`. - The last category is not included by default (configurable via `dropLast`), + The last category is not included by default (configurable via :py:attr:`dropLast`), because it makes the vector entries sum up to one, and hence linearly dependent. So an input value of 4.0 maps to `[0.0, 0.0, 0.0, 0.0]`. - Note: This is different from scikit-learn's OneHotEncoder, which keeps all categories. - The output vectors are sparse. + .. note:: This is different from scikit-learn's OneHotEncoder, which keeps all categories. + The output vectors are sparse. - When `handleInvalid` is configured to 'keep', an extra "category" indicating invalid values is - added as last category. So when `dropLast` is true, invalid values are encoded as all-zeros - vector. + When :py:attr:`handleInvalid` is configured to 'keep', an extra "category" indicating invalid + values is added as last category. So when :py:attr:`dropLast` is true, invalid values are + encoded as all-zeros vector. - Note: When encoding multi-column by using `inputCols` and `outputCols` params, input/output - cols come in pairs, specified by the order in the arrays, and each pair is treated - independently. + .. note:: When encoding multi-column by using :py:attr:`inputCols` and + :py:attr:`outputCols` params, input/output cols come in pairs, specified by the order in + the arrays, and each pair is treated independently. - See `StringIndexer` for converting categorical values into category indices + .. seealso:: :py:class:`StringIndexer` for converting categorical values into category indices >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(0.0,), (1.0,), (2.0,)], ["input"]) @@ -1754,7 +1754,7 @@ class OneHotEncoderEstimator(JavaEstimator, HasInputCols, HasOutputCols, HasHand >>> model = ohe.fit(df) >>> model.transform(df).head().output SparseVector(2, {0: 1.0}) - >>> ohePath = temp_path + "/oheEstimator" + >>> ohePath = temp_path + "/ohe" >>> ohe.save(ohePath) >>> loadedOHE = OneHotEncoderEstimator.load(ohePath) >>> loadedOHE.getInputCols() == ohe.getInputCols() From c9fd14c823eaa1ef8c43e6c7ccbf9fbdaad4c786 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 5 Dec 2018 19:36:51 +0800 Subject: [PATCH 282/879] Revert "[SPARK-26133][ML][FOLLOWUP] Fix doc for OneHotEncoder" This reverts commit d9b707e7c39a55a22dd55f8a4f537d861a3ce57c. --- python/pyspark/ml/feature.py | 22 +++++++++++----------- 1 file changed, 11 insertions(+), 11 deletions(-) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 93ece80af1e6..eccb7acae5b9 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -1731,22 +1731,22 @@ class OneHotEncoderEstimator(JavaEstimator, HasInputCols, HasOutputCols, HasHand at most a single one-value per row that indicates the input category index. For example with 5 categories, an input value of 2.0 would map to an output vector of `[0.0, 0.0, 1.0, 0.0]`. - The last category is not included by default (configurable via :py:attr:`dropLast`), + The last category is not included by default (configurable via `dropLast`), because it makes the vector entries sum up to one, and hence linearly dependent. So an input value of 4.0 maps to `[0.0, 0.0, 0.0, 0.0]`. - .. note:: This is different from scikit-learn's OneHotEncoder, which keeps all categories. - The output vectors are sparse. + Note: This is different from scikit-learn's OneHotEncoder, which keeps all categories. + The output vectors are sparse. - When :py:attr:`handleInvalid` is configured to 'keep', an extra "category" indicating invalid - values is added as last category. So when :py:attr:`dropLast` is true, invalid values are - encoded as all-zeros vector. + When `handleInvalid` is configured to 'keep', an extra "category" indicating invalid values is + added as last category. So when `dropLast` is true, invalid values are encoded as all-zeros + vector. - .. note:: When encoding multi-column by using :py:attr:`inputCols` and - :py:attr:`outputCols` params, input/output cols come in pairs, specified by the order in - the arrays, and each pair is treated independently. + Note: When encoding multi-column by using `inputCols` and `outputCols` params, input/output + cols come in pairs, specified by the order in the arrays, and each pair is treated + independently. - .. seealso:: :py:class:`StringIndexer` for converting categorical values into category indices + See `StringIndexer` for converting categorical values into category indices >>> from pyspark.ml.linalg import Vectors >>> df = spark.createDataFrame([(0.0,), (1.0,), (2.0,)], ["input"]) @@ -1754,7 +1754,7 @@ class OneHotEncoderEstimator(JavaEstimator, HasInputCols, HasOutputCols, HasHand >>> model = ohe.fit(df) >>> model.transform(df).head().output SparseVector(2, {0: 1.0}) - >>> ohePath = temp_path + "/ohe" + >>> ohePath = temp_path + "/oheEstimator" >>> ohe.save(ohePath) >>> loadedOHE = OneHotEncoderEstimator.load(ohePath) >>> loadedOHE.getInputCols() == ohe.getInputCols() From 910bfc87b2462cecc502782cf427ccdd0a1bc966 Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Wed, 5 Dec 2018 09:09:47 -0800 Subject: [PATCH 283/879] [SPARK-26233][SQL][BACKPORT-2.4] CheckOverflow when encoding a decimal value ## What changes were proposed in this pull request? When we encode a Decimal from external source we don't check for overflow. That method is useful not only in order to enforce that we can represent the correct value in the specified range, but it also changes the underlying data to the right precision/scale. Since in our code generation we assume that a decimal has exactly the same precision and scale of its data type, missing to enforce it can lead to corrupted output/results when there are subsequent transformations. ## How was this patch tested? added UT Closes #23232 from mgaido91/SPARK-26233_2.4. Authored-by: Marco Gaido Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/catalyst/encoders/RowEncoder.scala | 4 ++-- .../test/scala/org/apache/spark/sql/DatasetSuite.scala | 9 +++++++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 3340789398f9..13f72bd3eb87 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -108,12 +108,12 @@ object RowEncoder { returnNullable = false) case d: DecimalType => - StaticInvoke( + CheckOverflow(StaticInvoke( Decimal.getClass, d, "fromDecimal", inputObject :: Nil, - returnNullable = false) + returnNullable = false), d) case StringType => StaticInvoke( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 4e593ff046a5..f6f51b5cac8e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1547,6 +1547,15 @@ class DatasetSuite extends QueryTest with SharedSQLContext { df.where($"city".contains(new java.lang.Character('A'))), Seq(Row("Amsterdam"))) } + + test("SPARK-26233: serializer should enforce decimal precision and scale") { + val s = StructType(Seq(StructField("a", StringType), StructField("b", DecimalType(38, 8)))) + val encoder = RowEncoder(s) + implicit val uEnc = encoder + val df = spark.range(2).map(l => Row(l.toString, BigDecimal.valueOf(l + 0.1111))) + checkAnswer(df.groupBy(col("a")).agg(first(col("b"))), + Seq(Row("0", BigDecimal.valueOf(0.1111)), Row("1", BigDecimal.valueOf(1.1111)))) + } } case class TestDataUnion(x: Int, y: Int, z: Int) From 14b75ebea54842854d839a4fbec0780ceaac007e Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 8 Dec 2018 11:10:11 -0800 Subject: [PATCH 284/879] [SPARK-26266][BUILD] Update to Scala 2.12.8 (branch-2.4) ## What changes were proposed in this pull request? Back-port of https://github.com/apache/spark/pull/23218 ; updates Scala 2.12 build to 2.12.8 ## How was this patch tested? Existing tests. Closes #23264 from srowen/SPARK-26266.2. Authored-by: Sean Owen Signed-off-by: Dongjoon Hyun --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 349de835bfc9..3a0168a6faf1 100644 --- a/pom.xml +++ b/pom.xml @@ -2767,7 +2767,7 @@ scala-2.12 - 2.12.7 + 2.12.8 2.12 From a073b1c691a7afd7e130c6332fe472b157e4e8ec Mon Sep 17 00:00:00 2001 From: seancxmao Date: Sat, 8 Dec 2018 17:53:12 -0800 Subject: [PATCH 285/879] [SPARK-25132][SQL][FOLLOWUP][DOC] Add migration doc for case-insensitive field resolution when reading from Parquet ## What changes were proposed in this pull request? #22148 introduces a behavior change. According to discussion at #22184, this PR updates migration guide when upgrade from Spark 2.3 to 2.4. ## How was this patch tested? N/A Closes #23238 from seancxmao/SPARK-25132-doc-2.4. Authored-by: seancxmao Signed-off-by: Dongjoon Hyun (cherry picked from commit 55276d3a26474e7479941db3e9c065d86344885f) Signed-off-by: Dongjoon Hyun --- docs/sql-migration-guide-upgrade.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md index 7b4804059f44..05c33f6e5a8f 100644 --- a/docs/sql-migration-guide-upgrade.md +++ b/docs/sql-migration-guide-upgrade.md @@ -113,6 +113,8 @@ displayTitle: Spark SQL Upgrading Guide - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. + - In version 2.3 and earlier, when reading from a Parquet data source table, Spark always returns null for any column whose column names in Hive metastore schema and Parquet schema are in different letter cases, no matter whether `spark.sql.caseSensitive` is set to `true` or `false`. Since 2.4, when `spark.sql.caseSensitive` is set to `false`, Spark does case insensitive column name resolution between Hive metastore schema and Parquet schema, so even column names are in different letter cases, Spark returns corresponding column values. An exception is thrown if there is ambiguity, i.e. more than one Parquet column is matched. This change also applies to Parquet Hive tables when `spark.sql.hive.convertMetastoreParquet` is set to `true`. + ## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. From 33460c58a9274e22bd662858c71292275ae4aa24 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sun, 9 Dec 2018 10:50:41 -0800 Subject: [PATCH 286/879] [SPARK-26021][2.4][SQL][FOLLOWUP] only deal with NaN and -0.0 in UnsafeWriter backport https://github.com/apache/spark/pull/23239 to 2.4 --------- ## What changes were proposed in this pull request? A followup of https://github.com/apache/spark/pull/23043 There are 4 places we need to deal with NaN and -0.0: 1. comparison expressions. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same. 2. Join keys. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same. 3. grouping keys. `-0.0` and `0.0` should be assigned to the same group. Different NaNs should be assigned to the same group. 4. window partition keys. `-0.0` and `0.0` should be treated as same. Different NaNs should be treated as same. The case 1 is OK. Our comparison already handles NaN and -0.0, and for struct/array/map, we will recursively compare the fields/elements. Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary directly, and different NaNs have different binary representation, and the same thing happens for -0.0 and 0.0. To fix it, a simple solution is: normalize float/double when building unsafe data (`UnsafeRow`, `UnsafeArrayData`, `UnsafeMapData`). Then we don't need to worry about it anymore. Following this direction, this PR moves the handling of NaN and -0.0 from `Platform` to `UnsafeWriter`, so that places like `UnsafeRow.setFloat` will not handle them, which reduces the perf overhead. It's also easier to add comments explaining why we do it in `UnsafeWriter`. ## How was this patch tested? existing tests Closes #23265 from cloud-fan/minor. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/unsafe/Platform.java | 10 ------ .../spark/unsafe/PlatformUtilSuite.java | 14 -------- .../expressions/codegen/UnsafeWriter.java | 35 +++++++++++++++++++ .../codegen/UnsafeRowWriterSuite.scala | 20 +++++++++++ .../apache/spark/sql/DataFrameJoinSuite.scala | 12 +++++++ .../sql/DataFrameWindowFunctionsSuite.scala | 14 ++++++++ 6 files changed, 81 insertions(+), 24 deletions(-) diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index bc94f2171228..aca6fca00c48 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -120,11 +120,6 @@ public static float getFloat(Object object, long offset) { } public static void putFloat(Object object, long offset, float value) { - if (Float.isNaN(value)) { - value = Float.NaN; - } else if (value == -0.0f) { - value = 0.0f; - } _UNSAFE.putFloat(object, offset, value); } @@ -133,11 +128,6 @@ public static double getDouble(Object object, long offset) { } public static void putDouble(Object object, long offset, double value) { - if (Double.isNaN(value)) { - value = Double.NaN; - } else if (value == -0.0d) { - value = 0.0d; - } _UNSAFE.putDouble(object, offset, value); } diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java index ab34324eb54c..3ad9ac7b4de9 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/PlatformUtilSuite.java @@ -157,18 +157,4 @@ public void heapMemoryReuse() { Assert.assertEquals(onheap4.size(), 1024 * 1024 + 7); Assert.assertEquals(obj3, onheap4.getBaseObject()); } - - @Test - // SPARK-26021 - public void writeMinusZeroIsReplacedWithZero() { - byte[] doubleBytes = new byte[Double.BYTES]; - byte[] floatBytes = new byte[Float.BYTES]; - Platform.putDouble(doubleBytes, Platform.BYTE_ARRAY_OFFSET, -0.0d); - Platform.putFloat(floatBytes, Platform.BYTE_ARRAY_OFFSET, -0.0f); - double doubleFromPlatform = Platform.getDouble(doubleBytes, Platform.BYTE_ARRAY_OFFSET); - float floatFromPlatform = Platform.getFloat(floatBytes, Platform.BYTE_ARRAY_OFFSET); - - Assert.assertEquals(Double.doubleToLongBits(0.0d), Double.doubleToLongBits(doubleFromPlatform)); - Assert.assertEquals(Float.floatToIntBits(0.0f), Float.floatToIntBits(floatFromPlatform)); - } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java index 95263a0da95a..7553ab8cf700 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java @@ -198,11 +198,46 @@ protected final void writeLong(long offset, long value) { Platform.putLong(getBuffer(), offset, value); } + // We need to take care of NaN and -0.0 in several places: + // 1. When compare values, different NaNs should be treated as same, `-0.0` and `0.0` should be + // treated as same. + // 2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 0.0 should belong + // to the same group. + // 3. As join keys, different NaNs should be treated as same, `-0.0` and `0.0` should be + // treated as same. + // 4. As window partition keys, different NaNs should be treated as same, `-0.0` and `0.0` + // should be treated as same. + // + // Case 1 is fine, as we handle NaN and -0.0 well during comparison. For complex types, we + // recursively compare the fields/elements, so it's also fine. + // + // Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary directly, and different + // NaNs have different binary representation, and the same thing happens for -0.0 and 0.0. + // + // Here we normalize NaN and -0.0, so that `UnsafeProjection` will normalize them when writing + // float/double columns and nested fields to `UnsafeRow`. + // + // Note that, we must do this for all the `UnsafeProjection`s, not only the ones that extract + // join/grouping/window partition keys. `UnsafeProjection` copies unsafe data directly for complex + // types, so nested float/double may not be normalized. We need to make sure that all the unsafe + // data(`UnsafeRow`, `UnsafeArrayData`, `UnsafeMapData`) will have flat/double normalized during + // creation. protected final void writeFloat(long offset, float value) { + if (Float.isNaN(value)) { + value = Float.NaN; + } else if (value == -0.0f) { + value = 0.0f; + } Platform.putFloat(getBuffer(), offset, value); } + // See comments for `writeFloat`. protected final void writeDouble(long offset, double value) { + if (Double.isNaN(value)) { + value = Double.NaN; + } else if (value == -0.0d) { + value = 0.0d; + } Platform.putDouble(getBuffer(), offset, value); } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala index fb651b76fc16..22e1fa6dfed4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala @@ -50,4 +50,24 @@ class UnsafeRowWriterSuite extends SparkFunSuite { assert(res1 == res2) } + test("SPARK-26021: normalize float/double NaN and -0.0") { + val unsafeRowWriter1 = new UnsafeRowWriter(4) + unsafeRowWriter1.resetRowWriter() + unsafeRowWriter1.write(0, Float.NaN) + unsafeRowWriter1.write(1, Double.NaN) + unsafeRowWriter1.write(2, 0.0f) + unsafeRowWriter1.write(3, 0.0) + val res1 = unsafeRowWriter1.getRow + + val unsafeRowWriter2 = new UnsafeRowWriter(4) + unsafeRowWriter2.resetRowWriter() + unsafeRowWriter2.write(0, 0.0f/0.0f) + unsafeRowWriter2.write(1, 0.0/0.0) + unsafeRowWriter2.write(2, -0.0f) + unsafeRowWriter2.write(3, -0.0) + val res2 = unsafeRowWriter2.getRow + + // The two rows should be the equal + assert(res1 == res2) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index e6b30f9956da..c9f41ab1c017 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -295,4 +295,16 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { df.join(df, df("id") <=> df("id")).queryExecution.optimizedPlan } } + + test("NaN and -0.0 in join keys") { + val df1 = Seq(Float.NaN -> Double.NaN, 0.0f -> 0.0, -0.0f -> -0.0).toDF("f", "d") + val df2 = Seq(Float.NaN -> Double.NaN, 0.0f -> 0.0, -0.0f -> -0.0).toDF("f", "d") + val joined = df1.join(df2, Seq("f", "d")) + checkAnswer(joined, Seq( + Row(Float.NaN, Double.NaN), + Row(0.0f, 0.0), + Row(0.0f, 0.0), + Row(0.0f, 0.0), + Row(0.0f, 0.0))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 97a843978f0b..bbeb1d10ba7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -658,4 +658,18 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { |GROUP BY a |HAVING SUM(b) = 5 AND RANK() OVER(ORDER BY a) = 1""".stripMargin)) } + + test("NaN and -0.0 in window partition keys") { + val df = Seq( + (Float.NaN, Double.NaN, 1), + (0.0f/0.0f, 0.0/0.0, 1), + (0.0f, 0.0, 1), + (-0.0f, -0.0, 1)).toDF("f", "d", "i") + val result = df.select($"f", count("i").over(Window.partitionBy("f", "d"))) + checkAnswer(result, Seq( + Row(Float.NaN, 2), + Row(Float.NaN, 2), + Row(0.0f, 2), + Row(0.0f, 2))) + } } From e80577c5911c01720f124ce552ce4c210b50b5e1 Mon Sep 17 00:00:00 2001 From: gatorsmile Date: Mon, 10 Dec 2018 14:57:20 +0800 Subject: [PATCH 287/879] [SPARK-26307][SQL] Fix CTAS when INSERT a partitioned table using Hive serde ## What changes were proposed in this pull request? This is a Spark 2.3 regression introduced in https://github.com/apache/spark/pull/20521. We should add the partition info for InsertIntoHiveTable in CreateHiveTableAsSelectCommand. Otherwise, we will hit the following error by running the newly added test case: ``` [info] - CTAS: INSERT a partitioned table using Hive serde *** FAILED *** (829 milliseconds) [info] org.apache.spark.SparkException: Requested partitioning does not match the tab1 table: [info] Requested partitions: [info] Table partitions: part [info] at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.processInsert(InsertIntoHiveTable.scala:179) [info] at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.run(InsertIntoHiveTable.scala:107) ``` ## How was this patch tested? Added a test case. Closes #23255 from gatorsmile/fixCTAS. Authored-by: gatorsmile Signed-off-by: Wenchen Fan (cherry picked from commit 3bc83de3cce86a06c275c86b547a99afd781761f) Signed-off-by: Wenchen Fan --- .../execution/CreateHiveTableAsSelectCommand.scala | 4 +++- .../scala/org/apache/spark/sql/hive/InsertSuite.scala | 11 +++++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index 630bea5161f1..fd1e931ee0c7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -57,9 +57,11 @@ case class CreateHiveTableAsSelectCommand( return Seq.empty } + // For CTAS, there is no static partition values to insert. + val partition = tableDesc.partitionColumnNames.map(_ -> None).toMap InsertIntoHiveTable( tableDesc, - Map.empty, + partition, query, overwrite = false, ifPartitionNotExists = false, diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 5879748d05b2..510de3a7eab5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -752,6 +752,17 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } + test("SPARK-26307: CTAS - INSERT a partitioned table using Hive serde") { + withTable("tab1") { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + val df = Seq(("a", 100)).toDF("part", "id") + df.write.format("hive").partitionBy("part").mode("overwrite").saveAsTable("tab1") + df.write.format("hive").partitionBy("part").mode("append").saveAsTable("tab1") + } + } + } + + Seq("LOCAL", "").foreach { local => Seq(true, false).foreach { caseSensitivity => Seq("orc", "parquet").foreach { format => From 2f038d0cefb39cb2b92c22ff8dc130417f342910 Mon Sep 17 00:00:00 2001 From: Yuanjian Li Date: Tue, 11 Dec 2018 10:03:47 -0800 Subject: [PATCH 288/879] [SPARK-26327][SQL][BACKPORT-2.4] Bug fix for `FileSourceScanExec` metrics update ## What changes were proposed in this pull request? Backport #23277 to branch 2.4 without the metrics renaming. ## How was this patch tested? New test case in `SQLMetricsSuite`. Closes #23287 from xuanyuanking/SPARK-26327-2.4. Authored-by: Yuanjian Li Signed-off-by: Dongjoon Hyun --- .../sql/execution/DataSourceScanExec.scala | 26 +++++++++++++------ .../execution/metric/SQLMetricsSuite.scala | 15 +++++++++++ 2 files changed, 33 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 36ed016773b6..5433c30afd6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala @@ -185,19 +185,14 @@ case class FileSourceScanExec( partitionSchema = relation.partitionSchema, relation.sparkSession.sessionState.conf) + private var metadataTime = 0L + @transient private lazy val selectedPartitions: Seq[PartitionDirectory] = { val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) val startTime = System.nanoTime() val ret = relation.location.listFiles(partitionFilters, dataFilters) val timeTakenMs = ((System.nanoTime() - startTime) + optimizerMetadataTimeNs) / 1000 / 1000 - - metrics("numFiles").add(ret.map(_.files.size.toLong).sum) - metrics("metadataTime").add(timeTakenMs) - - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, - metrics("numFiles") :: metrics("metadataTime") :: Nil) - + metadataTime = timeTakenMs ret } @@ -308,6 +303,8 @@ case class FileSourceScanExec( } private lazy val inputRDD: RDD[InternalRow] = { + // Update metrics for taking effect in both code generation node and normal node. + updateDriverMetrics() val readFile: (PartitionedFile) => Iterator[InternalRow] = relation.fileFormat.buildReaderWithPartitionValues( sparkSession = relation.sparkSession, @@ -524,6 +521,19 @@ case class FileSourceScanExec( } } + /** + * Send the updated metrics to driver, while this function calling, selectedPartitions has + * been initialized. See SPARK-26327 for more detail. + */ + private def updateDriverMetrics() = { + metrics("numFiles").add(selectedPartitions.map(_.files.size.toLong).sum) + metrics("metadataTime").add(metadataTime) + + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, + metrics("numFiles") :: metrics("metadataTime") :: Nil) + } + override def doCanonicalize(): FileSourceScanExec = { FileSourceScanExec( relation, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index 085a44548848..c550bf20b92b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -570,4 +570,19 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared } } } + + test("SPARK-26327: FileSourceScanExec metrics") { + withTable("testDataForScan") { + spark.range(10).selectExpr("id", "id % 3 as p") + .write.partitionBy("p").saveAsTable("testDataForScan") + // The execution plan only has 1 FileScan node. + val df = spark.sql( + "SELECT * FROM testDataForScan WHERE p = 1") + testSparkPlanMetrics(df, 1, Map( + 0L -> (("Scan parquet default.testdataforscan", Map( + "number of output rows" -> 3L, + "number of files" -> 2L)))) + ) + } + } } From e35d287dd9fd5b7bd7e06025f535772b482b443c Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 11 Dec 2018 12:22:58 -0800 Subject: [PATCH 289/879] [SPARK-26265][CORE][BRANCH-2.4] Fix deadlock in BytesToBytesMap.MapIterator when locking both BytesToBytesMap.MapIterator and TaskMemoryManager ## What changes were proposed in this pull request? In `BytesToBytesMap.MapIterator.advanceToNextPage`, We will first lock this `MapIterator` and then `TaskMemoryManager` when going to free a memory page by calling `freePage`. At the same time, it is possibly that another memory consumer first locks `TaskMemoryManager` and then this `MapIterator` when it acquires memory and causes spilling on this `MapIterator`. So it ends with the `MapIterator` object holds lock to the `MapIterator` object and waits for lock on `TaskMemoryManager`, and the other consumer holds lock to `TaskMemoryManager` and waits for lock on the `MapIterator` object. To avoid deadlock here, this patch proposes to keep reference to the page to free and free it after releasing the lock of `MapIterator`. This backports the fix to branch-2.4. ## How was this patch tested? Added test and manually test by running the test 100 times to make sure there is no deadlock. Closes #23289 from viirya/SPARK-26265-2.4. Authored-by: Liang-Chi Hsieh Signed-off-by: Dongjoon Hyun --- .../spark/unsafe/map/BytesToBytesMap.java | 12 ++++- .../spark/memory/TestMemoryConsumer.java | 4 +- .../map/AbstractBytesToBytesMapSuite.java | 47 +++++++++++++++++++ 3 files changed, 60 insertions(+), 3 deletions(-) diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 9b6cbab38cbc..64650336c937 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -267,11 +267,18 @@ private MapIterator(int numRecords, Location loc, boolean destructive) { } private void advanceToNextPage() { + // SPARK-26265: We will first lock this `MapIterator` and then `TaskMemoryManager` when going + // to free a memory page by calling `freePage`. At the same time, it is possibly that another + // memory consumer first locks `TaskMemoryManager` and then this `MapIterator` when it + // acquires memory and causes spilling on this `MapIterator`. To avoid deadlock here, we keep + // reference to the page to free and free it after releasing the lock of `MapIterator`. + MemoryBlock pageToFree = null; + synchronized (this) { int nextIdx = dataPages.indexOf(currentPage) + 1; if (destructive && currentPage != null) { dataPages.remove(currentPage); - freePage(currentPage); + pageToFree = currentPage; nextIdx --; } if (dataPages.size() > nextIdx) { @@ -295,6 +302,9 @@ private void advanceToNextPage() { } } } + if (pageToFree != null) { + freePage(pageToFree); + } } @Override diff --git a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java index 0bbaea6b834b..6aa577d1bf79 100644 --- a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java +++ b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java @@ -38,12 +38,12 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { return used; } - void use(long size) { + public void use(long size) { long got = taskMemoryManager.acquireExecutionMemory(size, this); used += got; } - void free(long size) { + public void free(long size) { used -= size; taskMemoryManager.releaseExecutionMemory(size, this); } diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java index 53a233f698c7..278d28f7bf47 100644 --- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java +++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -33,6 +33,8 @@ import org.apache.spark.SparkConf; import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.memory.TestMemoryManager; import org.apache.spark.network.util.JavaUtils; @@ -667,4 +669,49 @@ public void testPeakMemoryUsed() { } } + @Test + public void avoidDeadlock() throws InterruptedException { + memoryManager.limit(PAGE_SIZE_BYTES); + MemoryMode mode = useOffHeapMemoryAllocator() ? MemoryMode.OFF_HEAP: MemoryMode.ON_HEAP; + TestMemoryConsumer c1 = new TestMemoryConsumer(taskMemoryManager, mode); + BytesToBytesMap map = + new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.5, 1024, false); + + Thread thread = new Thread(() -> { + int i = 0; + long used = 0; + while (i < 10) { + c1.use(10000000); + used += 10000000; + i++; + } + c1.free(used); + }); + + try { + int i; + for (i = 0; i < 1024; i++) { + final long[] arr = new long[]{i}; + final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8); + loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); + } + + // Starts to require memory at another memory consumer. + thread.start(); + + BytesToBytesMap.MapIterator iter = map.destructiveIterator(); + for (i = 0; i < 1024; i++) { + iter.next(); + } + assertFalse(iter.hasNext()); + } finally { + map.free(); + thread.join(); + for (File spillFile : spillFilesCreated) { + assertFalse("Spill file " + spillFile.getPath() + " was not cleaned up", + spillFile.exists()); + } + } + } + } From 2282c933880e515febcf57217859e07ad8b40780 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 11 Dec 2018 14:44:58 -0800 Subject: [PATCH 290/879] This is a dummy commit to trigger AFS gitbox sync From a2c5bea602004c08362804d2a09f720cf92ee5db Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 13 Dec 2018 13:14:59 +0800 Subject: [PATCH 291/879] [SPARK-26355][PYSPARK] Add a workaround for PyArrow 0.11. In PyArrow 0.11, there is a API breaking change. - [ARROW-1949](https://issues.apache.org/jira/browse/ARROW-1949) - [Python/C++] Add option to Array.from_pandas and pyarrow.array to perform unsafe casts. This causes test failures in `ScalarPandasUDFTests.test_vectorized_udf_null_(byte|short|int|long)`: ``` File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/worker.py", line 377, in main process() File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/worker.py", line 372, in process serializer.dump_stream(func(split_index, iterator), outfile) File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 317, in dump_stream batch = _create_batch(series, self._timezone) File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 286, in _create_batch arrs = [create_array(s, t) for s, t in series] File "/Users/ueshin/workspace/apache-spark/spark/python/pyspark/serializers.py", line 284, in create_array return pa.Array.from_pandas(s, mask=mask, type=t) File "pyarrow/array.pxi", line 474, in pyarrow.lib.Array.from_pandas return array(obj, mask=mask, type=type, safe=safe, from_pandas=True, File "pyarrow/array.pxi", line 169, in pyarrow.lib.array return _ndarray_to_array(values, mask, type, from_pandas, safe, File "pyarrow/array.pxi", line 69, in pyarrow.lib._ndarray_to_array check_status(NdarrayToArrow(pool, values, mask, from_pandas, File "pyarrow/error.pxi", line 81, in pyarrow.lib.check_status raise ArrowInvalid(message) ArrowInvalid: Floating point value truncated ``` We should add a workaround to support PyArrow 0.11. In my local environment. Closes #23305 from ueshin/issues/SPARK-26355/pyarrow_0.11. Authored-by: Takuya UESHIN Signed-off-by: Hyukjin Kwon (cherry picked from commit 8edae94fa7ec1a1cc2c69e0924da0da85d4aac83) Signed-off-by: Hyukjin Kwon --- python/pyspark/serializers.py | 5 ++++- python/pyspark/sql/tests.py | 11 +++++++++-- 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index ff9a612b77f6..5398bf4c410d 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -248,7 +248,10 @@ def create_array(s, t): # TODO: see ARROW-2432. Remove when the minimum PyArrow version becomes 0.10.0. return pa.Array.from_pandas(s.apply( lambda v: decimal.Decimal('NaN') if v is None else v), mask=mask, type=t) - return pa.Array.from_pandas(s, mask=mask, type=t) + elif LooseVersion(pa.__version__) < LooseVersion("0.11.0"): + # TODO: see ARROW-1949. Remove when the minimum PyArrow version becomes 0.11.0. + return pa.Array.from_pandas(s, mask=mask, type=t) + return pa.Array.from_pandas(s, mask=mask, type=t, safe=False) arrs = [create_array(s, t) for s, t in series] return pa.RecordBatch.from_arrays(arrs, ["_%d" % i for i in xrange(len(arrs))]) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 4fc1b4f597a7..2098b43b6bb1 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -5945,8 +5945,15 @@ def invalid_positional_types(pdf): with QuietTest(self.sc): with self.assertRaisesRegexp(Exception, "KeyError: 'id'"): grouped_df.apply(column_name_typo).collect() - with self.assertRaisesRegexp(Exception, "No cast implemented"): - grouped_df.apply(invalid_positional_types).collect() + from distutils.version import LooseVersion + import pyarrow as pa + if LooseVersion(pa.__version__) < LooseVersion("0.11.0"): + # TODO: see ARROW-1949. Remove when the minimum PyArrow version becomes 0.11.0. + with self.assertRaisesRegexp(Exception, "No cast implemented"): + grouped_df.apply(invalid_positional_types).collect() + else: + with self.assertRaisesRegexp(Exception, "an integer is required"): + grouped_df.apply(invalid_positional_types).collect() def test_positional_assignment_conf(self): import pandas as pd From aec68a8ff18360cd2d1f2b103e6fe64d78e3d770 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sat, 15 Dec 2018 00:23:28 +0800 Subject: [PATCH 292/879] [SPARK-26370][SQL] Fix resolution of higher-order function for the same identifier. When using a higher-order function with the same variable name as the existing columns in `Filter` or something which uses `Analyzer.resolveExpressionBottomUp` during the resolution, e.g.,: ```scala val df = Seq( (Seq(1, 9, 8, 7), 1, 2), (Seq(5, 9, 7), 2, 2), (Seq.empty, 3, 2), (null, 4, 2) ).toDF("i", "x", "d") checkAnswer(df.filter("exists(i, x -> x % d == 0)"), Seq(Row(Seq(1, 9, 8, 7), 1, 2))) checkAnswer(df.select("x").filter("exists(i, x -> x % d == 0)"), Seq(Row(1))) ``` the following exception happens: ``` java.lang.ClassCastException: org.apache.spark.sql.catalyst.expressions.BoundReference cannot be cast to org.apache.spark.sql.catalyst.expressions.NamedExpression at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:237) at scala.collection.mutable.ResizableArray.foreach(ResizableArray.scala:62) at scala.collection.mutable.ResizableArray.foreach$(ResizableArray.scala:55) at scala.collection.mutable.ArrayBuffer.foreach(ArrayBuffer.scala:49) at scala.collection.TraversableLike.map(TraversableLike.scala:237) at scala.collection.TraversableLike.map$(TraversableLike.scala:230) at scala.collection.AbstractTraversable.map(Traversable.scala:108) at org.apache.spark.sql.catalyst.expressions.HigherOrderFunction.$anonfun$functionsForEval$1(higherOrderFunctions.scala:147) at scala.collection.TraversableLike.$anonfun$map$1(TraversableLike.scala:237) at scala.collection.immutable.List.foreach(List.scala:392) at scala.collection.TraversableLike.map(TraversableLike.scala:237) at scala.collection.TraversableLike.map$(TraversableLike.scala:230) at scala.collection.immutable.List.map(List.scala:298) at org.apache.spark.sql.catalyst.expressions.HigherOrderFunction.functionsForEval(higherOrderFunctions.scala:145) at org.apache.spark.sql.catalyst.expressions.HigherOrderFunction.functionsForEval$(higherOrderFunctions.scala:145) at org.apache.spark.sql.catalyst.expressions.ArrayExists.functionsForEval$lzycompute(higherOrderFunctions.scala:369) at org.apache.spark.sql.catalyst.expressions.ArrayExists.functionsForEval(higherOrderFunctions.scala:369) at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.functionForEval(higherOrderFunctions.scala:176) at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.functionForEval$(higherOrderFunctions.scala:176) at org.apache.spark.sql.catalyst.expressions.ArrayExists.functionForEval(higherOrderFunctions.scala:369) at org.apache.spark.sql.catalyst.expressions.ArrayExists.nullSafeEval(higherOrderFunctions.scala:387) at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.eval(higherOrderFunctions.scala:190) at org.apache.spark.sql.catalyst.expressions.SimpleHigherOrderFunction.eval$(higherOrderFunctions.scala:185) at org.apache.spark.sql.catalyst.expressions.ArrayExists.eval(higherOrderFunctions.scala:369) at org.apache.spark.sql.catalyst.expressions.GeneratedClass$SpecificPredicate.eval(Unknown Source) at org.apache.spark.sql.execution.FilterExec.$anonfun$doExecute$3(basicPhysicalOperators.scala:216) at org.apache.spark.sql.execution.FilterExec.$anonfun$doExecute$3$adapted(basicPhysicalOperators.scala:215) ... ``` because the `UnresolvedAttribute`s in `LambdaFunction` are unexpectedly resolved by the rule. This pr modified to use a placeholder `UnresolvedNamedLambdaVariable` to prevent unexpected resolution. Added a test and modified some tests. Closes #23320 from ueshin/issues/SPARK-26370/hof_resolution. Authored-by: Takuya UESHIN Signed-off-by: Wenchen Fan (cherry picked from commit 3dda58af2b7f42beab736d856bf17b4d35c8866c) Signed-off-by: Wenchen Fan --- .../analysis/higherOrderFunctions.scala | 5 ++-- .../expressions/higherOrderFunctions.scala | 26 +++++++++++++++++-- .../sql/catalyst/parser/AstBuilder.scala | 7 +++-- .../ResolveLambdaVariablesSuite.scala | 10 ++++--- .../parser/ExpressionParserSuite.scala | 6 +++-- .../spark/sql/DataFrameFunctionsSuite.scala | 20 ++++++++++++++ 6 files changed, 62 insertions(+), 12 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala index dd08190e1e8a..c8c75808df88 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala @@ -148,13 +148,14 @@ case class ResolveLambdaVariables(conf: SQLConf) extends Rule[LogicalPlan] { val lambdaMap = l.arguments.map(v => canonicalizer(v.name) -> v).toMap l.mapChildren(resolve(_, parentLambdaMap ++ lambdaMap)) - case u @ UnresolvedAttribute(name +: nestedFields) => + case u @ UnresolvedNamedLambdaVariable(name +: nestedFields) => parentLambdaMap.get(canonicalizer(name)) match { case Some(lambda) => nestedFields.foldLeft(lambda: Expression) { (expr, fieldName) => ExtractValue(expr, Literal(fieldName), conf.resolver) } - case None => u + case None => + UnresolvedAttribute(u.nameParts) } case _ => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index 32f97537706b..17cd2a73d523 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -22,12 +22,34 @@ import java.util.concurrent.atomic.AtomicReference import scala.collection.mutable import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.array.ByteArrayMethods +/** + * A placeholder of lambda variables to prevent unexpected resolution of [[LambdaFunction]]. + */ +case class UnresolvedNamedLambdaVariable(nameParts: Seq[String]) + extends LeafExpression with NamedExpression with Unevaluable { + + override def name: String = + nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") + + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override def qualifier: Seq[String] = throw new UnresolvedException(this, "qualifier") + override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") + override def newInstance(): NamedExpression = throw new UnresolvedException(this, "newInstance") + override lazy val resolved = false + + override def toString: String = s"lambda '$name" + + override def sql: String = name +} + /** * A named lambda variable. */ @@ -79,7 +101,7 @@ case class LambdaFunction( object LambdaFunction { val identity: LambdaFunction = { - val id = UnresolvedAttribute.quoted("id") + val id = UnresolvedNamedLambdaVariable(Seq("id")) LambdaFunction(id, Seq(id)) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala index c6d21058e6f1..80a4d18adfda 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala @@ -1336,9 +1336,12 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitLambda(ctx: LambdaContext): Expression = withOrigin(ctx) { val arguments = ctx.IDENTIFIER().asScala.map { name => - UnresolvedAttribute.quoted(name.getText) + UnresolvedNamedLambdaVariable(UnresolvedAttribute.quoted(name.getText).nameParts) } - LambdaFunction(expression(ctx.expression), arguments) + val function = expression(ctx.expression).transformUp { + case a: UnresolvedAttribute => UnresolvedNamedLambdaVariable(a.nameParts) + } + LambdaFunction(function, arguments) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala index c4171c75ecd0..a5847ba7c522 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala @@ -49,19 +49,21 @@ class ResolveLambdaVariablesSuite extends PlanTest { comparePlans(Analyzer.execute(plan(e1)), plan(e2)) } + private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) + test("resolution - no op") { checkExpression(key, key) } test("resolution - simple") { - val in = ArrayTransform(values1, LambdaFunction('x.attr + 1, 'x.attr :: Nil)) + val in = ArrayTransform(values1, LambdaFunction(lv('x) + 1, lv('x) :: Nil)) val out = ArrayTransform(values1, LambdaFunction(lvInt + 1, lvInt :: Nil)) checkExpression(in, out) } test("resolution - nested") { val in = ArrayTransform(values2, LambdaFunction( - ArrayTransform('x.attr, LambdaFunction('x.attr + 1, 'x.attr :: Nil)), 'x.attr :: Nil)) + ArrayTransform(lv('x), LambdaFunction(lv('x) + 1, lv('x) :: Nil)), lv('x) :: Nil)) val out = ArrayTransform(values2, LambdaFunction( ArrayTransform(lvArray, LambdaFunction(lvInt + 1, lvInt :: Nil)), lvArray :: Nil)) checkExpression(in, out) @@ -75,14 +77,14 @@ class ResolveLambdaVariablesSuite extends PlanTest { test("fail - name collisions") { val p = plan(ArrayTransform(values1, - LambdaFunction('x.attr + 'X.attr, 'x.attr :: 'X.attr :: Nil))) + LambdaFunction(lv('x) + lv('X), lv('x) :: lv('X) :: Nil))) val msg = intercept[AnalysisException](Analyzer.execute(p)).getMessage assert(msg.contains("arguments should not have names that are semantically the same")) } test("fail - lambda arguments") { val p = plan(ArrayTransform(values1, - LambdaFunction('x.attr + 'y.attr + 'z.attr, 'x.attr :: 'y.attr :: 'z.attr :: Nil))) + LambdaFunction(lv('x) + lv('y) + lv('z), lv('x) :: lv('y) :: lv('z) :: Nil))) val msg = intercept[AnalysisException](Analyzer.execute(p)).getMessage assert(msg.contains("does not match the number of arguments expected")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala index 781fc1e957ae..1eec9e77ac7c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ExpressionParserSuite.scala @@ -246,9 +246,11 @@ class ExpressionParserSuite extends PlanTest { intercept("foo(a x)", "extraneous input 'x'") } + private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) + test("lambda functions") { - assertEqual("x -> x + 1", LambdaFunction('x + 1, Seq('x.attr))) - assertEqual("(x, y) -> x + y", LambdaFunction('x + 'y, Seq('x.attr, 'y.attr))) + assertEqual("x -> x + 1", LambdaFunction(lv('x) + 1, Seq(lv('x)))) + assertEqual("(x, y) -> x + y", LambdaFunction(lv('x) + lv('y), Seq(lv('x), lv('y)))) } test("window function expressions") { 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 d4f9b90cb06d..99abfda0015e 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 @@ -2486,6 +2486,26 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { } assert(ex.getMessage.contains("Cannot use null as map key")) } + + test("SPARK-26370: Fix resolution of higher-order function for the same identifier") { + val df = Seq( + (Seq(1, 9, 8, 7), 1, 2), + (Seq(5, 9, 7), 2, 2), + (Seq.empty, 3, 2), + (null, 4, 2) + ).toDF("i", "x", "d") + + checkAnswer(df.selectExpr("x", "exists(i, x -> x % d == 0)"), + Seq( + Row(1, true), + Row(2, false), + Row(3, false), + Row(4, null))) + checkAnswer(df.filter("exists(i, x -> x % d == 0)"), + Seq(Row(Seq(1, 9, 8, 7), 1, 2))) + checkAnswer(df.select("x").filter("exists(i, x -> x % d == 0)"), + Seq(Row(1))) + } } object DataFrameFunctionsSuite { From 6019d9a4240fd470b19107dc7af8038e029efd09 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 15 Dec 2018 13:52:07 +0800 Subject: [PATCH 293/879] [SPARK-26265][CORE][FOLLOWUP] Put freePage into a finally block ## What changes were proposed in this pull request? Based on the [comment](https://github.com/apache/spark/pull/23272#discussion_r240735509), it seems to be better to put `freePage` into a `finally` block. This patch as a follow-up to do so. ## How was this patch tested? Existing tests. Closes #23294 from viirya/SPARK-26265-followup. Authored-by: Liang-Chi Hsieh Signed-off-by: Hyukjin Kwon (cherry picked from commit 1b604c1fd0b9ef17b394818fbd6c546bc01cdd8c) Signed-off-by: Hyukjin Kwon --- .../spark/unsafe/map/BytesToBytesMap.java | 57 ++++++++++--------- 1 file changed, 30 insertions(+), 27 deletions(-) diff --git a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java index 64650336c937..983c65d6267b 100644 --- a/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java +++ b/core/src/main/java/org/apache/spark/unsafe/map/BytesToBytesMap.java @@ -274,36 +274,39 @@ private void advanceToNextPage() { // reference to the page to free and free it after releasing the lock of `MapIterator`. MemoryBlock pageToFree = null; - synchronized (this) { - int nextIdx = dataPages.indexOf(currentPage) + 1; - if (destructive && currentPage != null) { - dataPages.remove(currentPage); - pageToFree = currentPage; - nextIdx --; - } - if (dataPages.size() > nextIdx) { - currentPage = dataPages.get(nextIdx); - pageBaseObject = currentPage.getBaseObject(); - offsetInPage = currentPage.getBaseOffset(); - recordsInPage = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); - offsetInPage += UnsafeAlignedOffset.getUaoSize(); - } else { - currentPage = null; - if (reader != null) { - handleFailedDelete(); + try { + synchronized (this) { + int nextIdx = dataPages.indexOf(currentPage) + 1; + if (destructive && currentPage != null) { + dataPages.remove(currentPage); + pageToFree = currentPage; + nextIdx--; } - try { - Closeables.close(reader, /* swallowIOException = */ false); - reader = spillWriters.getFirst().getReader(serializerManager); - recordsInPage = -1; - } catch (IOException e) { - // Scala iterator does not handle exception - Platform.throwException(e); + if (dataPages.size() > nextIdx) { + currentPage = dataPages.get(nextIdx); + pageBaseObject = currentPage.getBaseObject(); + offsetInPage = currentPage.getBaseOffset(); + recordsInPage = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); + offsetInPage += UnsafeAlignedOffset.getUaoSize(); + } else { + currentPage = null; + if (reader != null) { + handleFailedDelete(); + } + try { + Closeables.close(reader, /* swallowIOException = */ false); + reader = spillWriters.getFirst().getReader(serializerManager); + recordsInPage = -1; + } catch (IOException e) { + // Scala iterator does not handle exception + Platform.throwException(e); + } } } - } - if (pageToFree != null) { - freePage(pageToFree); + } finally { + if (pageToFree != null) { + freePage(pageToFree); + } } } From 869bfc906abc89ec6f6370c97e5b107212204af4 Mon Sep 17 00:00:00 2001 From: Jing Chen He Date: Sat, 15 Dec 2018 08:41:16 -0600 Subject: [PATCH 294/879] [SPARK-26315][PYSPARK] auto cast threshold from Integer to Float in approxSimilarityJoin of BucketedRandomProjectionLSHModel ## What changes were proposed in this pull request? If the input parameter 'threshold' to the function approxSimilarityJoin is not a float, we would get an exception. The fix is to convert the 'threshold' into a float before calling the java implementation method. ## How was this patch tested? Added a new test case. Without this fix, the test will throw an exception as reported in the JIRA. With the fix, the test passes. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23313 from jerryjch/SPARK-26315. Authored-by: Jing Chen He Signed-off-by: Sean Owen (cherry picked from commit 860f4497f2a59b21d455ec8bfad9ae15d2fd4d2e) Signed-off-by: Sean Owen --- python/pyspark/ml/feature.py | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index eccb7acae5b9..bc4f4c9ab5cd 100755 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -193,6 +193,7 @@ def approxSimilarityJoin(self, datasetA, datasetB, threshold, distCol="distCol") "datasetA" and "datasetB", and a column "distCol" is added to show the distance between each pair. """ + threshold = TypeConverters.toFloat(threshold) return self._call_java("approxSimilarityJoin", datasetA, datasetB, threshold, distCol) @@ -240,6 +241,16 @@ class BucketedRandomProjectionLSH(JavaEstimator, LSHParams, HasInputCol, HasOutp | 3| 6| 2.23606797749979| +---+---+-----------------+ ... + >>> model.approxSimilarityJoin(df, df2, 3, distCol="EuclideanDistance").select( + ... col("datasetA.id").alias("idA"), + ... col("datasetB.id").alias("idB"), + ... col("EuclideanDistance")).show() + +---+---+-----------------+ + |idA|idB|EuclideanDistance| + +---+---+-----------------+ + | 3| 6| 2.23606797749979| + +---+---+-----------------+ + ... >>> brpPath = temp_path + "/brp" >>> brp.save(brpPath) >>> brp2 = BucketedRandomProjectionLSH.load(brpPath) From d650075b39e8dc6012a71f8d21577c5c5f19ba90 Mon Sep 17 00:00:00 2001 From: jiake Date: Sun, 16 Dec 2018 17:20:58 -0800 Subject: [PATCH 295/879] [SPARK-26316][SPARK-21052][BRANCH-2.4] Revert hash join metrics in that causes performance degradation ## What changes were proposed in this pull request? revert spark 21052 in spark 2.4 because of the discussion in [PR23269](https://github.com/apache/spark/pull/23269) ## How was this patch tested? N/A Closes #23318 from JkSelf/branch-2.4-revert21052. Authored-by: jiake Signed-off-by: Dongjoon Hyun --- .../joins/BroadcastHashJoinExec.scala | 27 +----- .../spark/sql/execution/joins/HashJoin.scala | 7 +- .../sql/execution/joins/HashedRelation.scala | 31 ------ .../joins/ShuffledHashJoinExec.scala | 6 +- .../execution/metric/SQLMetricsSuite.scala | 94 +------------------ 5 files changed, 6 insertions(+), 159 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index a6f3ea47c849..b25a34cc10d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -48,8 +48,7 @@ case class BroadcastHashJoinExec( extends BinaryExecNode with HashJoin with CodegenSupport { override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "avgHashProbe" -> SQLMetrics.createAverageMetric(sparkContext, "avg hash probe")) + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) @@ -63,13 +62,12 @@ case class BroadcastHashJoinExec( protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - val avgHashProbe = longMetric("avgHashProbe") val broadcastRelation = buildPlan.executeBroadcast[HashedRelation]() streamedPlan.execute().mapPartitions { streamedIter => val hashed = broadcastRelation.value.asReadOnlyCopy() TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize) - join(streamedIter, hashed, numOutputRows, avgHashProbe) + join(streamedIter, hashed, numOutputRows) } } @@ -111,23 +109,6 @@ case class BroadcastHashJoinExec( } } - /** - * Returns the codes used to add a task completion listener to update avg hash probe - * at the end of the task. - */ - private def genTaskListener(avgHashProbe: String, relationTerm: String): String = { - val listenerClass = classOf[TaskCompletionListener].getName - val taskContextClass = classOf[TaskContext].getName - s""" - | $taskContextClass$$.MODULE$$.get().addTaskCompletionListener(new $listenerClass() { - | @Override - | public void onTaskCompletion($taskContextClass context) { - | $avgHashProbe.set($relationTerm.getAverageProbesPerLookup()); - | } - | }); - """.stripMargin - } - /** * Returns a tuple of Broadcast of HashedRelation and the variable name for it. */ @@ -137,15 +118,11 @@ case class BroadcastHashJoinExec( val broadcast = ctx.addReferenceObj("broadcast", broadcastRelation) val clsName = broadcastRelation.value.getClass.getName - // At the end of the task, we update the avg hash probe. - val avgHashProbe = metricTerm(ctx, "avgHashProbe") - // Inline mutable state since not many join operations in a task val relationTerm = ctx.addMutableState(clsName, "relation", v => s""" | $v = (($clsName) $broadcast.value()).asReadOnlyCopy(); | incPeakExecutionMemory($v.estimatedSize()); - | ${genTaskListener(avgHashProbe, v)} """.stripMargin, forceInline = true) (broadcastRelation, relationTerm) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index dab873bf9b9a..b197bf6c8998 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -194,8 +194,7 @@ trait HashJoin { protected def join( streamedIter: Iterator[InternalRow], hashed: HashedRelation, - numOutputRows: SQLMetric, - avgHashProbe: SQLMetric): Iterator[InternalRow] = { + numOutputRows: SQLMetric): Iterator[InternalRow] = { val joinedIter = joinType match { case _: InnerLike => @@ -213,10 +212,6 @@ trait HashJoin { s"BroadcastHashJoin should not take $x as the JoinType") } - // At the end of the task, we update the avg hash probe. - TaskContext.get().addTaskCompletionListener[Unit](_ => - avgHashProbe.set(hashed.getAverageProbesPerLookup)) - val resultProj = createResultProjection joinedIter.map { r => numOutputRows += 1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 86eb47a70f1a..956430407db5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -81,10 +81,6 @@ private[execution] sealed trait HashedRelation extends KnownSizeEstimation { */ def close(): Unit - /** - * Returns the average number of probes per key lookup. - */ - def getAverageProbesPerLookup: Double } private[execution] object HashedRelation { @@ -281,7 +277,6 @@ private[joins] class UnsafeHashedRelation( read(() => in.readInt(), () => in.readLong(), in.readBytes) } - override def getAverageProbesPerLookup: Double = binaryMap.getAverageProbesPerLookup } private[joins] object UnsafeHashedRelation { @@ -395,10 +390,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap // The number of unique keys. private var numKeys = 0L - // Tracking average number of probes per key lookup. - private var numKeyLookups = 0L - private var numProbes = 0L - // needed by serializer def this() = { this( @@ -483,8 +474,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap */ def getValue(key: Long, resultRow: UnsafeRow): UnsafeRow = { if (isDense) { - numKeyLookups += 1 - numProbes += 1 if (key >= minKey && key <= maxKey) { val value = array((key - minKey).toInt) if (value > 0) { @@ -493,14 +482,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap } } else { var pos = firstSlot(key) - numKeyLookups += 1 - numProbes += 1 while (array(pos + 1) != 0) { if (array(pos) == key) { return getRow(array(pos + 1), resultRow) } pos = nextSlot(pos) - numProbes += 1 } } null @@ -528,8 +514,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap */ def get(key: Long, resultRow: UnsafeRow): Iterator[UnsafeRow] = { if (isDense) { - numKeyLookups += 1 - numProbes += 1 if (key >= minKey && key <= maxKey) { val value = array((key - minKey).toInt) if (value > 0) { @@ -538,14 +522,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap } } else { var pos = firstSlot(key) - numKeyLookups += 1 - numProbes += 1 while (array(pos + 1) != 0) { if (array(pos) == key) { return valueIter(array(pos + 1), resultRow) } pos = nextSlot(pos) - numProbes += 1 } } null @@ -585,11 +566,8 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap private def updateIndex(key: Long, address: Long): Unit = { var pos = firstSlot(key) assert(numKeys < array.length / 2) - numKeyLookups += 1 - numProbes += 1 while (array(pos) != key && array(pos + 1) != 0) { pos = nextSlot(pos) - numProbes += 1 } if (array(pos + 1) == 0) { // this is the first value for this key, put the address in array. @@ -721,8 +699,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap writeLong(maxKey) writeLong(numKeys) writeLong(numValues) - writeLong(numKeyLookups) - writeLong(numProbes) writeLong(array.length) writeLongArray(writeBuffer, array, array.length) @@ -764,8 +740,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap maxKey = readLong() numKeys = readLong() numValues = readLong() - numKeyLookups = readLong() - numProbes = readLong() val length = readLong().toInt mask = length - 2 @@ -784,10 +758,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap read(() => in.readBoolean(), () => in.readLong(), in.readBytes) } - /** - * Returns the average number of probes per key lookup. - */ - def getAverageProbesPerLookup: Double = numProbes.toDouble / numKeyLookups } private[joins] class LongHashedRelation( @@ -840,7 +810,6 @@ private[joins] class LongHashedRelation( map = in.readObject().asInstanceOf[LongToUnsafeRowMap] } - override def getAverageProbesPerLookup: Double = map.getAverageProbesPerLookup } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 2b59ed6e4d16..524804d61e59 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -42,8 +42,7 @@ case class ShuffledHashJoinExec( override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"), - "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map"), - "avgHashProbe" -> SQLMetrics.createAverageMetric(sparkContext, "avg hash probe")) + "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map")) override def requiredChildDistribution: Seq[Distribution] = HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil @@ -63,10 +62,9 @@ case class ShuffledHashJoinExec( protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - val avgHashProbe = longMetric("avgHashProbe") streamedPlan.execute().zipPartitions(buildPlan.execute()) { (streamIter, buildIter) => val hashed = buildHashedRelation(buildIter) - join(streamIter, hashed, numOutputRows, avgHashProbe) + join(streamIter, hashed, numOutputRows) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala index c550bf20b92b..9fd28681d595 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/metric/SQLMetricsSuite.scala @@ -231,50 +231,6 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared ) } - test("BroadcastHashJoin metrics: track avg probe") { - // The executed plan looks like: - // Project [a#210, b#211, b#221] - // +- BroadcastHashJoin [a#210], [a#220], Inner, BuildRight - // :- Project [_1#207 AS a#210, _2#208 AS b#211] - // : +- Filter isnotnull(_1#207) - // : +- LocalTableScan [_1#207, _2#208] - // +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, binary, true])) - // +- Project [_1#217 AS a#220, _2#218 AS b#221] - // +- Filter isnotnull(_1#217) - // +- LocalTableScan [_1#217, _2#218] - // - // Assume the execution plan with node id is - // WholeStageCodegen disabled: - // Project(nodeId = 0) - // BroadcastHashJoin(nodeId = 1) - // ...(ignored) - // - // WholeStageCodegen enabled: - // WholeStageCodegen(nodeId = 0) - // Project(nodeId = 1) - // BroadcastHashJoin(nodeId = 2) - // Project(nodeId = 3) - // Filter(nodeId = 4) - // ...(ignored) - Seq(true, false).foreach { enableWholeStage => - val df1 = generateRandomBytesDF() - val df2 = generateRandomBytesDF() - val df = df1.join(broadcast(df2), "a") - val nodeIds = if (enableWholeStage) { - Set(2L) - } else { - Set(1L) - } - val metrics = getSparkPlanMetrics(df, 2, nodeIds, enableWholeStage).get - nodeIds.foreach { nodeId => - val probes = metrics(nodeId)._2("avg hash probe (min, med, max)") - probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => - assert(probe.toDouble > 1.0) - } - } - } - } - test("ShuffledHashJoin metrics") { withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "40", "spark.sql.shuffle.partitions" -> "2", @@ -287,59 +243,11 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val metrics = getSparkPlanMetrics(df, 1, Set(1L)) testSparkPlanMetrics(df, 1, Map( 1L -> (("ShuffledHashJoin", Map( - "number of output rows" -> 2L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")))) + "number of output rows" -> 2L)))) ) } } - test("ShuffledHashJoin metrics: track avg probe") { - // The executed plan looks like: - // Project [a#308, b#309, b#319] - // +- ShuffledHashJoin [a#308], [a#318], Inner, BuildRight - // :- Exchange hashpartitioning(a#308, 2) - // : +- Project [_1#305 AS a#308, _2#306 AS b#309] - // : +- Filter isnotnull(_1#305) - // : +- LocalTableScan [_1#305, _2#306] - // +- Exchange hashpartitioning(a#318, 2) - // +- Project [_1#315 AS a#318, _2#316 AS b#319] - // +- Filter isnotnull(_1#315) - // +- LocalTableScan [_1#315, _2#316] - // - // Assume the execution plan with node id is - // WholeStageCodegen disabled: - // Project(nodeId = 0) - // ShuffledHashJoin(nodeId = 1) - // ...(ignored) - // - // WholeStageCodegen enabled: - // WholeStageCodegen(nodeId = 0) - // Project(nodeId = 1) - // ShuffledHashJoin(nodeId = 2) - // ...(ignored) - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "5000000", - "spark.sql.shuffle.partitions" -> "2", - "spark.sql.join.preferSortMergeJoin" -> "false") { - Seq(true, false).foreach { enableWholeStage => - val df1 = generateRandomBytesDF(65535 * 5) - val df2 = generateRandomBytesDF(65535) - val df = df1.join(df2, "a") - val nodeIds = if (enableWholeStage) { - Set(2L) - } else { - Set(1L) - } - val metrics = getSparkPlanMetrics(df, 1, nodeIds, enableWholeStage).get - nodeIds.foreach { nodeId => - val probes = metrics(nodeId)._2("avg hash probe (min, med, max)") - probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => - assert(probe.toDouble > 1.0) - } - } - } - } - } - test("BroadcastHashJoin(outer) metrics") { val df1 = Seq((1, "a"), (1, "b"), (4, "c")).toDF("key", "value") val df2 = Seq((1, "a"), (1, "b"), (2, "c"), (3, "d")).toDF("key2", "value") From e743e848484bf7d97e1b4f33ea83f8520ae7da04 Mon Sep 17 00:00:00 2001 From: Kris Mok Date: Mon, 17 Dec 2018 13:41:20 +0800 Subject: [PATCH 296/879] [SPARK-26352][SQL] join reorder should not change the order of output attributes ## What changes were proposed in this pull request? The optimizer rule `org.apache.spark.sql.catalyst.optimizer.ReorderJoin` performs join reordering on inner joins. This was introduced from SPARK-12032 (https://github.com/apache/spark/pull/10073) in 2015-12. After it had reordered the joins, though, it didn't check whether or not the output attribute order is still the same as before. Thus, it's possible to have a mismatch between the reordered output attributes order vs the schema that a DataFrame thinks it has. The same problem exists in the CBO version of join reordering (`CostBasedJoinReorder`) too. This can be demonstrated with the example: ```scala spark.sql("create table table_a (x int, y int) using parquet") spark.sql("create table table_b (i int, j int) using parquet") spark.sql("create table table_c (a int, b int) using parquet") val df = spark.sql(""" with df1 as (select * from table_a cross join table_b) select * from df1 join table_c on a = x and b = i """) ``` here's what the DataFrame thinks: ``` scala> df.printSchema root |-- x: integer (nullable = true) |-- y: integer (nullable = true) |-- i: integer (nullable = true) |-- j: integer (nullable = true) |-- a: integer (nullable = true) |-- b: integer (nullable = true) ``` here's what the optimized plan thinks, after join reordering: ``` scala> df.queryExecution.optimizedPlan.output.foreach(a => println(s"|-- ${a.name}: ${a.dataType.typeName}")) |-- x: integer |-- y: integer |-- a: integer |-- b: integer |-- i: integer |-- j: integer ``` If we exclude the `ReorderJoin` rule (using Spark 2.4's optimizer rule exclusion feature), it's back to normal: ``` scala> spark.conf.set("spark.sql.optimizer.excludedRules", "org.apache.spark.sql.catalyst.optimizer.ReorderJoin") scala> val df = spark.sql("with df1 as (select * from table_a cross join table_b) select * from df1 join table_c on a = x and b = i") df: org.apache.spark.sql.DataFrame = [x: int, y: int ... 4 more fields] scala> df.queryExecution.optimizedPlan.output.foreach(a => println(s"|-- ${a.name}: ${a.dataType.typeName}")) |-- x: integer |-- y: integer |-- i: integer |-- j: integer |-- a: integer |-- b: integer ``` Note that this output attribute ordering problem leads to data corruption, and can manifest itself in various symptoms: * Silently corrupting data, if the reordered columns happen to either have matching types or have sufficiently-compatible types (e.g. all fixed length primitive types are considered as "sufficiently compatible" in an `UnsafeRow`), then only the resulting data is going to be wrong but it might not trigger any alarms immediately. Or * Weird Java-level exceptions like `java.lang.NegativeArraySizeException`, or even SIGSEGVs. ## How was this patch tested? Added new unit test in `JoinReorderSuite` and new end-to-end test in `JoinSuite`. Also made `JoinReorderSuite` and `StarJoinReorderSuite` assert more strongly on maintaining output attribute order. Closes #23303 from rednaxelafx/fix-join-reorder. Authored-by: Kris Mok Signed-off-by: Wenchen Fan (cherry picked from commit 56448c662398f4c5319a337e6601450270a6a27c) Signed-off-by: Wenchen Fan --- .../optimizer/CostBasedJoinReorder.scala | 10 +++++ .../spark/sql/catalyst/optimizer/joins.scala | 12 +++++- .../optimizer/JoinOptimizationSuite.scala | 3 ++ .../catalyst/optimizer/JoinReorderSuite.scala | 38 +++++++++++++++++-- .../StarJoinCostBasedReorderSuite.scala | 21 +++++++++- .../optimizer/StarJoinReorderSuite.scala | 28 ++++++++++++-- .../org/apache/spark/sql/JoinSuite.scala | 14 +++++++ 7 files changed, 116 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index 064ca68b7a62..01634a9d852c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala @@ -48,6 +48,7 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { if projectList.forall(_.isInstanceOf[Attribute]) => reorder(p, p.output) } + // After reordering is finished, convert OrderedJoin back to Join result transformDown { case OrderedJoin(left, right, jt, cond) => Join(left, right, jt, cond) @@ -175,11 +176,20 @@ object JoinReorderDP extends PredicateHelper with Logging { assert(topOutputSet == p.outputSet) // Keep the same order of final output attributes. p.copy(projectList = output) + case finalPlan if !sameOutput(finalPlan, output) => + Project(output, finalPlan) case finalPlan => finalPlan } } + private def sameOutput(plan: LogicalPlan, expectedOutput: Seq[Attribute]): Boolean = { + val thisOutput = plan.output + thisOutput.length == expectedOutput.length && thisOutput.zip(expectedOutput).forall { + case (a1, a2) => a1.semanticEquals(a2) + } + } + /** Find all possible plans at the next level, based on existing levels. */ private def searchLevel( existingLevels: Seq[JoinPlanMap], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 6ebb194d71c2..0b6471289a47 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -86,9 +86,9 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { } def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case ExtractFiltersAndInnerJoins(input, conditions) + case p @ ExtractFiltersAndInnerJoins(input, conditions) if input.size > 2 && conditions.nonEmpty => - if (SQLConf.get.starSchemaDetection && !SQLConf.get.cboEnabled) { + val reordered = if (SQLConf.get.starSchemaDetection && !SQLConf.get.cboEnabled) { val starJoinPlan = StarSchemaDetection.reorderStarJoins(input, conditions) if (starJoinPlan.nonEmpty) { val rest = input.filterNot(starJoinPlan.contains(_)) @@ -99,6 +99,14 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { } else { createOrderedJoin(input, conditions) } + + if (p.sameOutput(reordered)) { + reordered + } else { + // Reordering the joins have changed the order of the columns. + // Inject a projection to make sure we restore to the expected ordering. + Project(p.output, reordered) + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala index ccd9d8dd4d21..e9438b2eee55 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala @@ -102,16 +102,19 @@ class JoinOptimizationSuite extends PlanTest { x.join(y).join(z).where(("x.b".attr === "z.b".attr) && ("y.d".attr === "z.a".attr)), x.join(z, condition = Some("x.b".attr === "z.b".attr)) .join(y, condition = Some("y.d".attr === "z.a".attr)) + .select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*) ), ( x.join(y, Cross).join(z, Cross) .where(("x.b".attr === "z.b".attr) && ("y.d".attr === "z.a".attr)), x.join(z, Cross, Some("x.b".attr === "z.b".attr)) .join(y, Cross, Some("y.d".attr === "z.a".attr)) + .select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*) ), ( x.join(y, Inner).join(z, Cross).where("x.b".attr === "z.a".attr), x.join(z, Cross, Some("x.b".attr === "z.a".attr)).join(y, Inner) + .select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*) ) ) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala index 565b0a10154a..c94a8b9e318f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} -import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} -import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.{Cross, Inner, PlanTest} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan} import org.apache.spark.sql.internal.SQLConf.{CBO_ENABLED, JOIN_REORDER_ENABLED} @@ -124,7 +124,8 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { // the original order (t1 J t2) J t3. val bestPlan = t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) - .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(outputsOf(t1, t2, t3): _*) assertEqualPlans(originalPlan, bestPlan) } @@ -139,7 +140,9 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { val bestPlan = t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(outputsOf(t1, t2, t3): _*) // this is redundant but we'll take it for now .join(t4) + .select(outputsOf(t1, t2, t4, t3): _*) assertEqualPlans(originalPlan, bestPlan) } @@ -202,6 +205,7 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { t1.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) .join(t4.join(t3, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))), Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) + .select(outputsOf(t1, t4, t2, t3): _*) assertEqualPlans(originalPlan, bestPlan) } @@ -219,6 +223,23 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { } } + test("SPARK-26352: join reordering should not change the order of attributes") { + // This test case does not rely on CBO. + // It's similar to the test case above, but catches a reordering bug that the one above doesn't + val tab1 = LocalRelation('x.int, 'y.int) + val tab2 = LocalRelation('i.int, 'j.int) + val tab3 = LocalRelation('a.int, 'b.int) + val original = + tab1.join(tab2, Cross) + .join(tab3, Inner, Some('a === 'x && 'b === 'i)) + val expected = + tab1.join(tab3, Inner, Some('a === 'x)) + .join(tab2, Cross, Some('b === 'i)) + .select(outputsOf(tab1, tab2, tab3): _*) + + assertEqualPlans(original, expected) + } + test("reorder recursively") { // Original order: // Join @@ -266,8 +287,17 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { private def assertEqualPlans( originalPlan: LogicalPlan, groundTruthBestPlan: LogicalPlan): Unit = { - val optimized = Optimize.execute(originalPlan.analyze) + val analyzed = originalPlan.analyze + val optimized = Optimize.execute(analyzed) val expected = groundTruthBestPlan.analyze + + assert(analyzed.sameOutput(expected)) // if this fails, the expected plan itself is incorrect + assert(analyzed.sameOutput(optimized)) + compareJoinOrder(optimized, expected) } + + private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = { + plans.map(_.output).reduce(_ ++ _) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala index d4d23ad69b2c..baae934e1e4f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala @@ -218,6 +218,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) .join(t2, Inner, Some(nameToAttr("f1_c2") === nameToAttr("t2_c1"))) .join(t1, Inner, Some(nameToAttr("f1_c1") === nameToAttr("t1_c1"))) + .select(outputsOf(f1, t1, t2, d1, d2): _*) assertEqualPlans(query, expected) } @@ -256,6 +257,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas .join(t3.join(t2, Inner, Some(nameToAttr("t2_c2") === nameToAttr("t3_c1"))), Inner, Some(nameToAttr("d1_c2") === nameToAttr("t2_c1"))) .join(t1, Inner, Some(nameToAttr("t1_c1") === nameToAttr("f1_c1"))) + .select(outputsOf(d1, t1, t2, f1, d2, t3): _*) assertEqualPlans(query, expected) } @@ -297,6 +299,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) .join(t1.join(t2, Inner, Some(nameToAttr("t1_c1") === nameToAttr("t2_c1"))), Inner, Some(nameToAttr("t1_c2") === nameToAttr("t4_c2"))) + .select(outputsOf(d1, t1, t2, t3, t4, f1, d2): _*) assertEqualPlans(query, expected) } @@ -347,6 +350,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas Some(nameToAttr("d3_c2") === nameToAttr("t1_c1"))) .join(t5.join(t6, Inner, Some(nameToAttr("t5_c2") === nameToAttr("t6_c2"))), Inner, Some(nameToAttr("d2_c2") === nameToAttr("t5_c1"))) + .select(outputsOf(d1, t3, t4, f1, d2, t5, t6, d3, t1, t2): _*) assertEqualPlans(query, expected) } @@ -375,6 +379,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas f1.join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) + .select(outputsOf(d1, d2, f1, d3): _*) assertEqualPlans(query, expected) } @@ -400,13 +405,27 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas f1.join(t3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("t3_c1"))) .join(t2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("t2_c1"))) .join(t1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("t1_c1"))) + .select(outputsOf(t1, f1, t2, t3): _*) assertEqualPlans(query, expected) } private def assertEqualPlans( plan1: LogicalPlan, plan2: LogicalPlan): Unit = { - val optimized = Optimize.execute(plan1.analyze) + val analyzed = plan1.analyze + val optimized = Optimize.execute(analyzed) val expected = plan2.analyze + + assert(equivalentOutput(analyzed, expected)) // if this fails, the expected itself is incorrect + assert(equivalentOutput(analyzed, optimized)) + compareJoinOrder(optimized, expected) } + + private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = { + plans.map(_.output).reduce(_ ++ _) + } + + private def equivalentOutput(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = { + normalizeExprIds(plan1).output == normalizeExprIds(plan2).output + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala index 4e0883e91e84..9dc653b9d6c4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala @@ -182,6 +182,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk1"))) .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d2, f1, d3, s3): _*) assertEqualPlans(query, expected) } @@ -220,6 +221,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") < nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, f1, d2, s3, d3): _*) assertEqualPlans(query, expected) } @@ -255,7 +257,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("s3_c2"))) - + .select(outputsOf(d1, f1, d2, s3, d3): _*) assertEqualPlans(query, expected) } @@ -292,6 +294,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_c2"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") < nameToAttr("s3_pk1"))) + .select(outputsOf(d1, f1, d2, s3, d3): _*) assertEqualPlans(query, expected) } @@ -395,6 +398,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("f11_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, f11, f1, d2, s3): _*) assertEqualPlans(query, equivQuery) } @@ -430,6 +434,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_c4"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -465,6 +470,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -499,6 +505,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -532,6 +539,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") < nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") < nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") < nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -565,13 +573,27 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } - private def assertEqualPlans( plan1: LogicalPlan, plan2: LogicalPlan): Unit = { - val optimized = Optimize.execute(plan1.analyze) + private def assertEqualPlans(plan1: LogicalPlan, plan2: LogicalPlan): Unit = { + val analyzed = plan1.analyze + val optimized = Optimize.execute(analyzed) val expected = plan2.analyze + + assert(equivalentOutput(analyzed, expected)) // if this fails, the expected itself is incorrect + assert(equivalentOutput(analyzed, optimized)) + compareJoinOrder(optimized, expected) } + + private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = { + plans.map(_.output).reduce(_ ++ _) + } + + private def equivalentOutput(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = { + normalizeExprIds(plan1).output == normalizeExprIds(plan2).output + } } 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 44767dfc9249..52fa22c67a40 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 @@ -893,4 +893,18 @@ class JoinSuite extends QueryTest with SharedSQLContext { checkAnswer(res, Row(0, 0, 0)) } } + + test("SPARK-26352: join reordering should not change the order of columns") { + withTable("tab1", "tab2", "tab3") { + spark.sql("select 1 as x, 100 as y").write.saveAsTable("tab1") + spark.sql("select 42 as i, 200 as j").write.saveAsTable("tab2") + spark.sql("select 1 as a, 42 as b").write.saveAsTable("tab3") + + val df = spark.sql(""" + with tmp as (select * from tab1 cross join tab2) + select * from tmp join tab3 on a = x and b = i + """) + checkAnswer(df, Row(1, 100, 42, 200, 1, 42)) + } + } } From 0a69787fd05b41cd5272ba95072310dba5be3978 Mon Sep 17 00:00:00 2001 From: Kris Mok Date: Mon, 17 Dec 2018 22:48:59 +0800 Subject: [PATCH 297/879] [SPARK-26352][SQL][FOLLOWUP-2.4] Fix missing sameOutput in branch-2.4 ## What changes were proposed in this pull request? After https://github.com/apache/spark/pull/23303 was merged to branch-2.3/2.4, the builds on those branches were broken due to missing a `LogicalPlan.sameOutput` function which came from https://github.com/apache/spark/pull/22713 only available on master. This PR is to follow-up with the broken 2.3/2.4 branches and make a copy of the new `LogicalPlan.sameOutput` into `ReorderJoin` to make it locally available. ## How was this patch tested? Fix the build of 2.3/2.4. Closes #23330 from rednaxelafx/clean-build-2.4. Authored-by: Kris Mok Signed-off-by: Wenchen Fan --- .../spark/sql/catalyst/optimizer/joins.scala | 17 ++++++++++++++++- .../catalyst/optimizer/JoinReorderSuite.scala | 19 +++++++++++++++++-- 2 files changed, 33 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 0b6471289a47..2feb4720f9f9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -100,7 +100,7 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { createOrderedJoin(input, conditions) } - if (p.sameOutput(reordered)) { + if (sameOutput(p, reordered)) { reordered } else { // Reordering the joins have changed the order of the columns. @@ -108,6 +108,21 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { Project(p.output, reordered) } } + + /** + * Returns true iff output of both plans are semantically the same, ie.: + * - they contain the same number of `Attribute`s; + * - references are the same; + * - the order is equal too. + * NOTE: this is copied over from SPARK-25691 from master. + */ + def sameOutput(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = { + val output1 = plan1.output + val output2 = plan2.output + output1.length == output2.length && output1.zip(output2).forall { + case (a1, a2) => a1.semanticEquals(a2) + } + } } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala index c94a8b9e318f..38a70f0691dd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala @@ -291,8 +291,8 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { val optimized = Optimize.execute(analyzed) val expected = groundTruthBestPlan.analyze - assert(analyzed.sameOutput(expected)) // if this fails, the expected plan itself is incorrect - assert(analyzed.sameOutput(optimized)) + assert(sameOutput(analyzed, expected)) // if this fails, the expected plan itself is incorrect + assert(sameOutput(analyzed, optimized)) compareJoinOrder(optimized, expected) } @@ -300,4 +300,19 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = { plans.map(_.output).reduce(_ ++ _) } + + /** + * Returns true iff output of both plans are semantically the same, ie.: + * - they contain the same number of `Attribute`s; + * - references are the same; + * - the order is equal too. + * NOTE: this is copied over from SPARK-25691 from master. + */ + def sameOutput(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = { + val output1 = plan1.output + val output2 = plan2.output + output1.length == output2.length && output1.zip(output2).forall { + case (a1, a2) => a1.semanticEquals(a2) + } + } } From 16986b29e22553797e0e78df445eac94a44285c7 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Tue, 18 Dec 2018 10:09:56 -0800 Subject: [PATCH 298/879] [SPARK-26382][CORE] prefix comparator should handle -0.0 ## What changes were proposed in this pull request? This is kind of a followup of https://github.com/apache/spark/pull/23239 The `UnsafeProject` will normalize special float/double values(NaN and -0.0), so the sorter doesn't have to handle it. However, for consistency and future-proof, this PR proposes to normalize `-0.0` in the prefix comparator, so that it's same with the normal ordering. Note that prefix comparator handles NaN as well. This is not a bug fix, but a safe guard. ## How was this patch tested? existing tests Closes #23334 from cloud-fan/sort. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun (cherry picked from commit befca983d2da4f7828aa7a7cd7345d17c4f291dd) Signed-off-by: Dongjoon Hyun --- .../unsafe/sort/PrefixComparators.java | 2 ++ .../unsafe/sort/PrefixComparatorsSuite.scala | 17 +++++++++++++++-- 2 files changed, 17 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java index 0910db22af00..bef1bdadb27a 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java @@ -69,6 +69,8 @@ public static final class DoublePrefixComparator { * details see http://stereopsis.com/radix.html. */ public static long computePrefix(double value) { + // normalize -0.0 to 0.0, as they should be equal + value = value == -0.0 ? 0.0 : value; // Java's doubleToLongBits already canonicalizes all NaN values to the smallest possible // positive NaN, so there's nothing special we need to do for NaNs. long bits = Double.doubleToLongBits(value); diff --git a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala index 73546ef1b7a6..38cb37c52459 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/unsafe/sort/PrefixComparatorsSuite.scala @@ -125,6 +125,7 @@ class PrefixComparatorsSuite extends SparkFunSuite with PropertyChecks { val nan2Prefix = PrefixComparators.DoublePrefixComparator.computePrefix(nan2) assert(nan1Prefix === nan2Prefix) val doubleMaxPrefix = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MaxValue) + // NaN is greater than the max double value. assert(PrefixComparators.DOUBLE.compare(nan1Prefix, doubleMaxPrefix) === 1) } @@ -134,22 +135,34 @@ class PrefixComparatorsSuite extends SparkFunSuite with PropertyChecks { assert(java.lang.Double.doubleToRawLongBits(negativeNan) < 0) val prefix = PrefixComparators.DoublePrefixComparator.computePrefix(negativeNan) val doubleMaxPrefix = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MaxValue) + // -NaN is greater than the max double value. assert(PrefixComparators.DOUBLE.compare(prefix, doubleMaxPrefix) === 1) } test("double prefix comparator handles other special values properly") { - val nullValue = 0L + // See `SortPrefix.nullValue` for how we deal with nulls for float/double type + val smallestNullPrefix = 0L + val largestNullPrefix = -1L val nan = PrefixComparators.DoublePrefixComparator.computePrefix(Double.NaN) val posInf = PrefixComparators.DoublePrefixComparator.computePrefix(Double.PositiveInfinity) val negInf = PrefixComparators.DoublePrefixComparator.computePrefix(Double.NegativeInfinity) val minValue = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MinValue) val maxValue = PrefixComparators.DoublePrefixComparator.computePrefix(Double.MaxValue) val zero = PrefixComparators.DoublePrefixComparator.computePrefix(0.0) + val minusZero = PrefixComparators.DoublePrefixComparator.computePrefix(-0.0) + + // null is greater than everything including NaN, when we need to treat it as the largest value. + assert(PrefixComparators.DOUBLE.compare(largestNullPrefix, nan) === 1) + // NaN is greater than the positive infinity. assert(PrefixComparators.DOUBLE.compare(nan, posInf) === 1) assert(PrefixComparators.DOUBLE.compare(posInf, maxValue) === 1) assert(PrefixComparators.DOUBLE.compare(maxValue, zero) === 1) assert(PrefixComparators.DOUBLE.compare(zero, minValue) === 1) assert(PrefixComparators.DOUBLE.compare(minValue, negInf) === 1) - assert(PrefixComparators.DOUBLE.compare(negInf, nullValue) === 1) + // null is smaller than everything including negative infinity, when we need to treat it as + // the smallest value. + assert(PrefixComparators.DOUBLE.compare(negInf, smallestNullPrefix) === 1) + // 0.0 should be equal to -0.0. + assert(PrefixComparators.DOUBLE.compare(zero, minusZero) === 0) } } From f0976386641143ab58ad2eb38224274d65336bd6 Mon Sep 17 00:00:00 2001 From: Jackey Lee Date: Tue, 18 Dec 2018 12:15:36 -0600 Subject: [PATCH 299/879] [SPARK-26394][CORE] Fix annotation error for Utils.timeStringAsMs ## What changes were proposed in this pull request? Change microseconds to milliseconds in annotation of Utils.timeStringAsMs. Closes #23346 from stczwd/stczwd. Authored-by: Jackey Lee Signed-off-by: Sean Owen (cherry picked from commit 428eb2ad0ad8a141427120b13de3287962258c2d) Signed-off-by: Sean Owen --- core/src/main/scala/org/apache/spark/util/Utils.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index c8b148be8453..8f86b472b937 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1085,7 +1085,7 @@ private[spark] object Utils extends Logging { } /** - * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If * no suffix is provided, the passed number is assumed to be in ms. */ def timeStringAsMs(str: String): Long = { From 63b7a074ea2fd223b03a71588a237333ae279d1f Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Tue, 18 Dec 2018 23:21:52 -0800 Subject: [PATCH 300/879] [SPARK-26366][SQL] ReplaceExceptWithFilter should consider NULL as False ## What changes were proposed in this pull request? In `ReplaceExceptWithFilter` we do not consider properly the case in which the condition returns NULL. Indeed, in that case, since negating NULL still returns NULL, so it is not true the assumption that negating the condition returns all the rows which didn't satisfy it, rows returning NULL may not be returned. This happens when constraints inferred by `InferFiltersFromConstraints` are not enough, as it happens with `OR` conditions. The rule had also problems with non-deterministic conditions: in such a scenario, this rule would change the probability of the output. The PR fixes these problem by: - returning False for the condition when it is Null (in this way we do return all the rows which didn't satisfy it); - avoiding any transformation when the condition is non-deterministic. ## How was this patch tested? added UTs Closes #23315 from mgaido91/SPARK-26366. Authored-by: Marco Gaido Signed-off-by: gatorsmile (cherry picked from commit 834b8609793525a5a486013732d8c98e1c6e6504) Signed-off-by: gatorsmile --- .../optimizer/ReplaceExceptWithFilter.scala | 32 ++++++++------ .../optimizer/ReplaceOperatorSuite.scala | 44 ++++++++++++++----- .../org/apache/spark/sql/DatasetSuite.scala | 11 +++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 38 ++++++++++++++++ 4 files changed, 101 insertions(+), 24 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala index efd3944eba7f..4996d24dfd29 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.catalyst.rules.Rule * Note: * Before flipping the filter condition of the right node, we should: * 1. Combine all it's [[Filter]]. - * 2. Apply InferFiltersFromConstraints rule (to take into account of NULL values in the condition). + * 2. Update the attribute references to the left node; + * 3. Add a Coalesce(condition, False) (to take into account of NULL values in the condition). */ object ReplaceExceptWithFilter extends Rule[LogicalPlan] { @@ -47,23 +48,28 @@ object ReplaceExceptWithFilter extends Rule[LogicalPlan] { plan.transform { case e @ Except(left, right, false) if isEligible(left, right) => - val newCondition = transformCondition(left, skipProject(right)) - newCondition.map { c => - Distinct(Filter(Not(c), left)) - }.getOrElse { + val filterCondition = combineFilters(skipProject(right)).asInstanceOf[Filter].condition + if (filterCondition.deterministic) { + transformCondition(left, filterCondition).map { c => + Distinct(Filter(Not(c), left)) + }.getOrElse { + e + } + } else { e } } } - private def transformCondition(left: LogicalPlan, right: LogicalPlan): Option[Expression] = { - val filterCondition = - InferFiltersFromConstraints(combineFilters(right)).asInstanceOf[Filter].condition - - val attributeNameMap: Map[String, Attribute] = left.output.map(x => (x.name, x)).toMap - - if (filterCondition.references.forall(r => attributeNameMap.contains(r.name))) { - Some(filterCondition.transform { case a: AttributeReference => attributeNameMap(a.name) }) + private def transformCondition(plan: LogicalPlan, condition: Expression): Option[Expression] = { + val attributeNameMap: Map[String, Attribute] = plan.output.map(x => (x.name, x)).toMap + if (condition.references.forall(r => attributeNameMap.contains(r.name))) { + val rewrittenCondition = condition.transform { + case a: AttributeReference => attributeNameMap(a.name) + } + // We need to consider as False when the condition is NULL, otherwise we do not return those + // rows containing NULL which are instead filtered in the Except right plan + Some(Coalesce(Seq(rewrittenCondition, Literal.FalseLiteral))) } else { None } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala index 3b1b2d588ef6..c8e15c7da763 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala @@ -20,11 +20,12 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Alias, Literal, Not} +import org.apache.spark.sql.catalyst.expressions.{Alias, Coalesce, If, Literal, Not} import org.apache.spark.sql.catalyst.expressions.aggregate.First import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi, PlanTest} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.types.BooleanType class ReplaceOperatorSuite extends PlanTest { @@ -65,8 +66,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), Filter(attributeB === 2, Filter(attributeA === 1, table1)))).analyze comparePlans(optimized, correctAnswer) @@ -84,8 +84,8 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), table1)).analyze + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), + table1)).analyze comparePlans(optimized, correctAnswer) } @@ -104,8 +104,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), Project(Seq(attributeA, attributeB), table1))).analyze comparePlans(optimized, correctAnswer) @@ -125,8 +124,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), Filter(attributeB === 2, Filter(attributeA === 1, table1)))).analyze comparePlans(optimized, correctAnswer) @@ -146,8 +144,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA === 1 && attributeB === 2)), + Filter(Not(Coalesce(Seq(attributeA === 1 && attributeB === 2, Literal.FalseLiteral))), Project(Seq(attributeA, attributeB), Filter(attributeB < 1, Filter(attributeA >= 2, table1))))).analyze @@ -229,4 +226,29 @@ class ReplaceOperatorSuite extends PlanTest { comparePlans(optimized, query) } + + test("SPARK-26366: ReplaceExceptWithFilter should handle properly NULL") { + val basePlan = LocalRelation(Seq('a.int, 'b.int)) + val otherPlan = basePlan.where('a.in(1, 2) || 'b.in()) + val except = Except(basePlan, otherPlan, false) + val result = OptimizeIn(Optimize.execute(except.analyze)) + val correctAnswer = Aggregate(basePlan.output, basePlan.output, + Filter(!Coalesce(Seq( + 'a.in(1, 2) || If('b.isNotNull, Literal.FalseLiteral, Literal(null, BooleanType)), + Literal.FalseLiteral)), + basePlan)).analyze + comparePlans(result, correctAnswer) + } + + test("SPARK-26366: ReplaceExceptWithFilter should not transform non-detrministic") { + val basePlan = LocalRelation(Seq('a.int, 'b.int)) + val otherPlan = basePlan.where('a > rand(1L)) + val except = Except(basePlan, otherPlan, false) + val result = Optimize.execute(except.analyze) + val condition = basePlan.output.zip(otherPlan.output).map { case (a1, a2) => + a1 <=> a2 }.reduce( _ && _) + val correctAnswer = Aggregate(basePlan.output, otherPlan.output, + Join(basePlan, otherPlan, LeftAnti, Option(condition))).analyze + comparePlans(result, correctAnswer) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index f6f51b5cac8e..50406bcdddf6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1556,6 +1556,17 @@ class DatasetSuite extends QueryTest with SharedSQLContext { checkAnswer(df.groupBy(col("a")).agg(first(col("b"))), Seq(Row("0", BigDecimal.valueOf(0.1111)), Row("1", BigDecimal.valueOf(1.1111)))) } + + test("SPARK-26366: return nulls which are not filtered in except") { + val inputDF = sqlContext.createDataFrame( + sparkContext.parallelize(Seq(Row("0", "a"), Row("1", null))), + StructType(Seq( + StructField("a", StringType, nullable = true), + StructField("b", StringType, nullable = true)))) + + val exceptDF = inputDF.filter(col("a").isin("0") or col("b") > "c") + checkAnswer(inputDF.except(exceptDF), Seq(Row("1", null))) + } } case class TestDataUnion(x: Int, y: Int, z: Int) 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 dbb0790a4682..beb175373754 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 @@ -2909,6 +2909,44 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } } + + test("SPARK-26366: verify ReplaceExceptWithFilter") { + Seq(true, false).foreach { enabled => + withSQLConf(SQLConf.REPLACE_EXCEPT_WITH_FILTER.key -> enabled.toString) { + val df = spark.createDataFrame( + sparkContext.parallelize(Seq(Row(0, 3, 5), + Row(0, 3, null), + Row(null, 3, 5), + Row(0, null, 5), + Row(0, null, null), + Row(null, null, 5), + Row(null, 3, null), + Row(null, null, null))), + StructType(Seq(StructField("c1", IntegerType), + StructField("c2", IntegerType), + StructField("c3", IntegerType)))) + val where = "c2 >= 3 OR c1 >= 0" + val whereNullSafe = + """ + |(c2 IS NOT NULL AND c2 >= 3) + |OR (c1 IS NOT NULL AND c1 >= 0) + """.stripMargin + + val df_a = df.filter(where) + val df_b = df.filter(whereNullSafe) + checkAnswer(df.except(df_a), df.except(df_b)) + + val whereWithIn = "c2 >= 3 OR c1 in (2)" + val whereWithInNullSafe = + """ + |(c2 IS NOT NULL AND c2 >= 3) + """.stripMargin + val dfIn_a = df.filter(whereWithIn) + val dfIn_b = df.filter(whereWithInNullSafe) + checkAnswer(df.except(dfIn_a), df.except(dfIn_b)) + } + } + } } case class Foo(bar: Option[String]) From 74c1cd15ce49d16e4e6e3c605359bec5f39e9712 Mon Sep 17 00:00:00 2001 From: zhoukang Date: Thu, 20 Dec 2018 08:26:25 -0600 Subject: [PATCH 301/879] [SPARK-24687][CORE] Avoid job hanging when generate task binary causes fatal error ## What changes were proposed in this pull request? When NoClassDefFoundError thrown,it will cause job hang. `Exception in thread "dag-scheduler-event-loop" java.lang.NoClassDefFoundError: Lcom/xxx/data/recommend/aggregator/queue/QueueName; at java.lang.Class.getDeclaredFields0(Native Method) at java.lang.Class.privateGetDeclaredFields(Class.java:2436) at java.lang.Class.getDeclaredField(Class.java:1946) at java.io.ObjectStreamClass.getDeclaredSUID(ObjectStreamClass.java:1659) at java.io.ObjectStreamClass.access$700(ObjectStreamClass.java:72) at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:480) at java.io.ObjectStreamClass$2.run(ObjectStreamClass.java:468) at java.security.AccessController.doPrivileged(Native Method) at java.io.ObjectStreamClass.(ObjectStreamClass.java:468) at java.io.ObjectStreamClass.lookup(ObjectStreamClass.java:365) at java.io.ObjectOutputStream.writeClass(ObjectOutputStream.java:1212) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1119) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeArray(ObjectOutputStream.java:1377) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1173) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.defaultWriteFields(ObjectOutputStream.java:1547) at java.io.ObjectOutputStream.writeSerialData(ObjectOutputStream.java:1508) at java.io.ObjectOutputStream.writeOrdinaryObject(ObjectOutputStream.java:1431) at java.io.ObjectOutputStream.writeObject0(ObjectOutputStream.java:1177) at java.io.ObjectOutputStream.writeArray(ObjectOutputStream.java:1377)` It is caused by NoClassDefFoundError will not catch up during task seriazation. `var taskBinary: Broadcast[Array[Byte]] = null try { // For ShuffleMapTask, serialize and broadcast (rdd, shuffleDep). // For ResultTask, serialize and broadcast (rdd, func). val taskBinaryBytes: Array[Byte] = stage match { case stage: ShuffleMapStage => JavaUtils.bufferToArray( closureSerializer.serialize((stage.rdd, stage.shuffleDep): AnyRef)) case stage: ResultStage => JavaUtils.bufferToArray(closureSerializer.serialize((stage.rdd, stage.func): AnyRef)) } taskBinary = sc.broadcast(taskBinaryBytes) } catch { // In the case of a failure during serialization, abort the stage. case e: NotSerializableException => abortStage(stage, "Task not serializable: " + e.toString, Some(e)) runningStages -= stage // Abort execution return case NonFatal(e) => abortStage(stage, s"Task serialization failed: $e\n${Utils.exceptionString(e)}", Some(e)) runningStages -= stage return }` image below shows that stage 33 blocked and never be scheduled. 2018-06-28 4 28 42 2018-06-28 4 28 49 ## How was this patch tested? UT Closes #21664 from caneGuy/zhoukang/fix-noclassdeferror. Authored-by: zhoukang Signed-off-by: Sean Owen (cherry picked from commit 7c8f4756c34a0b00931c2987c827a18d989e6c08) Signed-off-by: Sean Owen --- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 036fa38ce240..d314b7322da4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1167,9 +1167,11 @@ private[spark] class DAGScheduler( // Abort execution return - case NonFatal(e) => + case e: Throwable => abortStage(stage, s"Task serialization failed: $e\n${Utils.exceptionString(e)}", Some(e)) runningStages -= stage + + // Abort execution return } From daeb0811058c76e2d6cecb6de5ebe287c3be3a94 Mon Sep 17 00:00:00 2001 From: Ngone51 Date: Thu, 20 Dec 2018 10:25:52 -0800 Subject: [PATCH 302/879] [SPARK-26392][YARN] Cancel pending allocate requests by taking locality preference into account ## What changes were proposed in this pull request? Right now, we cancel pending allocate requests by its sending order. I thing we can take locality preference into account when do this to perfom least impact on task locality preference. ## How was this patch tested? N.A. Closes #23344 from Ngone51/dev-cancel-pending-allocate-requests-by-taking-locality-preference-into-account. Authored-by: Ngone51 Signed-off-by: Marcelo Vanzin (cherry picked from commit 3d6b44d9ea92dc1eabb8f211176861e51240bf93) Signed-off-by: Marcelo Vanzin --- .../spark/deploy/yarn/YarnAllocator.scala | 29 ++++++++----------- 1 file changed, 12 insertions(+), 17 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 8a7551de7c08..f4dc80ad4a62 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -287,20 +287,20 @@ private[yarn] class YarnAllocator( s"pending: $numPendingAllocate, running: ${runningExecutors.size}, " + s"executorsStarting: ${numExecutorsStarting.get}") + // Split the pending container request into three groups: locality matched list, locality + // unmatched list and non-locality list. Take the locality matched container request into + // consideration of container placement, treat as allocated containers. + // For locality unmatched and locality free container requests, cancel these container + // requests, since required locality preference has been changed, recalculating using + // container placement strategy. + val (localRequests, staleRequests, anyHostRequests) = splitPendingAllocationsByLocality( + hostToLocalTaskCounts, pendingAllocate) + if (missing > 0) { logInfo(s"Will request $missing executor container(s), each with " + s"${resource.getVirtualCores} core(s) and " + s"${resource.getMemory} MB memory (including $memoryOverhead MB of overhead)") - // Split the pending container request into three groups: locality matched list, locality - // unmatched list and non-locality list. Take the locality matched container request into - // consideration of container placement, treat as allocated containers. - // For locality unmatched and locality free container requests, cancel these container - // requests, since required locality preference has been changed, recalculating using - // container placement strategy. - val (localRequests, staleRequests, anyHostRequests) = splitPendingAllocationsByLocality( - hostToLocalTaskCounts, pendingAllocate) - // cancel "stale" requests for locations that are no longer needed staleRequests.foreach { stale => amClient.removeContainerRequest(stale) @@ -360,14 +360,9 @@ private[yarn] class YarnAllocator( val numToCancel = math.min(numPendingAllocate, -missing) logInfo(s"Canceling requests for $numToCancel executor container(s) to have a new desired " + s"total $targetNumExecutors executors.") - - val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, ANY_HOST, resource) - if (!matchingRequests.isEmpty) { - matchingRequests.iterator().next().asScala - .take(numToCancel).foreach(amClient.removeContainerRequest) - } else { - logWarning("Expected to find pending requests, but found none.") - } + // cancel pending allocate requests by taking locality preference into account + val cancelRequests = (staleRequests ++ anyHostRequests ++ localRequests).take(numToCancel) + cancelRequests.foreach(amClient.removeContainerRequest) } } From c29f7e20def505d3fdc2ee14a6d696a677828a4d Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 20 Dec 2018 10:05:56 -0800 Subject: [PATCH 303/879] [SPARK-26409][SQL][TESTS] SQLConf should be serializable in test sessions ## What changes were proposed in this pull request? `SQLConf` is supposed to be serializable. However, currently it is not serializable in `WithTestConf`. `WithTestConf` uses the method `overrideConfs` in closure, while the classes which implements it (`TestHiveSessionStateBuilder` and `TestSQLSessionStateBuilder`) are not serializable. This PR is to use a local variable to fix it. ## How was this patch tested? Add unit test. Closes #23352 from gengliangwang/serializableSQLConf. Authored-by: Gengliang Wang Signed-off-by: gatorsmile (cherry picked from commit 6692bacf3e74e7a17d8e676e8a06ab198f85d328) Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/internal/BaseSessionStateBuilder.scala | 3 ++- .../test/scala/org/apache/spark/sql/SerializationSuite.scala | 5 +++++ 2 files changed, 7 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala index 3a0db7e16c23..9c1a15c46acd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/BaseSessionStateBuilder.scala @@ -308,13 +308,14 @@ private[sql] trait WithTestConf { self: BaseSessionStateBuilder => def overrideConfs: Map[String, String] override protected lazy val conf: SQLConf = { + val overrideConfigurations = overrideConfs val conf = parentState.map(_.conf.clone()).getOrElse { new SQLConf { clear() override def clear(): Unit = { super.clear() // Make sure we start with the default test configs even after clear - overrideConfs.foreach { case (key, value) => setConfString(key, value) } + overrideConfigurations.foreach { case (key, value) => setConfString(key, value) } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala index cd6b2647e0be..1a1c956aed3d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala @@ -27,4 +27,9 @@ class SerializationSuite extends SparkFunSuite with SharedSQLContext { val spark = SparkSession.builder.getOrCreate() new JavaSerializer(new SparkConf()).newInstance().serialize(spark.sqlContext) } + + test("[SPARK-26409] SQLConf should be serializable") { + val spark = SparkSession.builder.getOrCreate() + new JavaSerializer(new SparkConf()).newInstance().serialize(spark.sessionState.conf) + } } From 90a14d58b4e87a603e35a9ab679f6049b10e9c7b Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 21 Dec 2018 16:09:30 +0800 Subject: [PATCH 304/879] [SPARK-26422][R] Support to disable Hive support in SparkR even for Hadoop versions unsupported by Hive fork ## What changes were proposed in this pull request? Currently, even if I explicitly disable Hive support in SparkR session as below: ```r sparkSession <- sparkR.session("local[4]", "SparkR", Sys.getenv("SPARK_HOME"), enableHiveSupport = FALSE) ``` produces when the Hadoop version is not supported by our Hive fork: ``` java.lang.reflect.InvocationTargetException ... Caused by: java.lang.IllegalArgumentException: Unrecognized Hadoop major version number: 3.1.1.3.1.0.0-78 at org.apache.hadoop.hive.shims.ShimLoader.getMajorVersion(ShimLoader.java:174) at org.apache.hadoop.hive.shims.ShimLoader.loadShims(ShimLoader.java:139) at org.apache.hadoop.hive.shims.ShimLoader.getHadoopShims(ShimLoader.java:100) at org.apache.hadoop.hive.conf.HiveConf$ConfVars.(HiveConf.java:368) ... 43 more Error in handleErrors(returnStatus, conn) : java.lang.ExceptionInInitializerError at org.apache.hadoop.hive.conf.HiveConf.(HiveConf.java:105) at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:348) at org.apache.spark.util.Utils$.classForName(Utils.scala:193) at org.apache.spark.sql.SparkSession$.hiveClassesArePresent(SparkSession.scala:1116) at org.apache.spark.sql.api.r.SQLUtils$.getOrCreateSparkSession(SQLUtils.scala:52) at org.apache.spark.sql.api.r.SQLUtils.getOrCreateSparkSession(SQLUtils.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) ``` The root cause is that: ``` SparkSession.hiveClassesArePresent ``` check if the class is loadable or not to check if that's in classpath but `org.apache.hadoop.hive.conf.HiveConf` has a check for Hadoop version as static logic which is executed right away. This throws an `IllegalArgumentException` and that's not caught: https://github.com/apache/spark/blob/36edbac1c8337a4719f90e4abd58d38738b2e1fb/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala#L1113-L1121 So, currently, if users have a Hive built-in Spark with unsupported Hadoop version by our fork (namely 3+), there's no way to use SparkR even though it could work. This PR just propose to change the order of bool comparison so that we can don't execute `SparkSession.hiveClassesArePresent` when: 1. `enableHiveSupport` is explicitly disabled 2. `spark.sql.catalogImplementation` is `in-memory` so that we **only** check `SparkSession.hiveClassesArePresent` when Hive support is explicitly enabled by short circuiting. ## How was this patch tested? It's difficult to write a test since we don't run tests against Hadoop 3 yet. See https://github.com/apache/spark/pull/21588. Manually tested. Closes #23356 from HyukjinKwon/SPARK-26422. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit 305e9b5ad22b428501fd42d3730d73d2e09ad4c5) Signed-off-by: Hyukjin Kwon --- .../scala/org/apache/spark/sql/api/r/SQLUtils.scala | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index af20764f9a96..4c717957c552 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -49,9 +49,17 @@ private[sql] object SQLUtils extends Logging { sparkConfigMap: JMap[Object, Object], enableHiveSupport: Boolean): SparkSession = { val spark = - if (SparkSession.hiveClassesArePresent && enableHiveSupport && + if (enableHiveSupport && jsc.sc.conf.get(CATALOG_IMPLEMENTATION.key, "hive").toLowerCase(Locale.ROOT) == - "hive") { + "hive" && + // Note that the order of conditions here are on purpose. + // `SparkSession.hiveClassesArePresent` checks if Hive's `HiveConf` is loadable or not; + // however, `HiveConf` itself has some static logic to check if Hadoop version is + // supported or not, which throws an `IllegalArgumentException` if unsupported. + // If this is checked first, there's no way to disable Hive support in the case above. + // So, we intentionally check if Hive classes are loadable or not only when + // Hive support is explicitly enabled by short-circuiting. See also SPARK-26422. + SparkSession.hiveClassesArePresent) { SparkSession.builder().sparkContext(withHiveExternalCatalog(jsc.sc)).getOrCreate() } else { if (enableHiveSupport) { From fdeb6db4053a217be3082da9da105c36b9acaa7c Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Sat, 22 Dec 2018 10:35:14 -0800 Subject: [PATCH 305/879] [SPARK-26402][SQL] Accessing nested fields with different cases in case insensitive mode ## What changes were proposed in this pull request? GetStructField with different optional names should be semantically equal. We will use this as building block to compare the nested fields used in the plans to be optimized by catalyst optimizer. This PR also fixes a bug below that accessing nested fields with different cases in case insensitive mode will result `AnalysisException`. ``` sql("create table t (s struct) using json") sql("select s.I from t group by s.i") ``` which is currently failing ``` org.apache.spark.sql.AnalysisException: expression 'default.t.`s`' is neither present in the group by, nor is it an aggregate function ``` as cloud-fan pointed out. ## How was this patch tested? New tests are added. Closes #23353 from dbtsai/nestedEqual. Lead-authored-by: DB Tsai Co-authored-by: DB Tsai Signed-off-by: Dongjoon Hyun (cherry picked from commit a5a24d92bdf6e6a8e33bdc8833bedba033576b4c) Signed-off-by: Dongjoon Hyun --- .../catalyst/expressions/Canonicalize.scala | 4 ++- .../expressions/CanonicalizeSuite.scala | 29 ++++++++++++++++++ .../BinaryComparisonSimplificationSuite.scala | 30 +++++++++++++++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 19 ++++++++++++ 4 files changed, 81 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala index fe6db8b344d3..4d218b936b3a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala @@ -26,6 +26,7 @@ package org.apache.spark.sql.catalyst.expressions * * The following rules are applied: * - Names and nullability hints for [[org.apache.spark.sql.types.DataType]]s are stripped. + * - Names for [[GetStructField]] are stripped. * - Commutative and associative operations ([[Add]] and [[Multiply]]) have their children ordered * by `hashCode`. * - [[EqualTo]] and [[EqualNullSafe]] are reordered by `hashCode`. @@ -37,10 +38,11 @@ object Canonicalize { expressionReorder(ignoreNamesTypes(e)) } - /** Remove names and nullability from types. */ + /** Remove names and nullability from types, and names from `GetStructField`. */ private[expressions] def ignoreNamesTypes(e: Expression): Expression = e match { case a: AttributeReference => AttributeReference("none", a.dataType.asNullable)(exprId = a.exprId) + case GetStructField(child, ordinal, Some(_)) => GetStructField(child, ordinal, None) case _ => e } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala index 28e6940f3cca..9802a6e5891b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.plans.logical.Range +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} class CanonicalizeSuite extends SparkFunSuite { @@ -50,4 +51,32 @@ class CanonicalizeSuite extends SparkFunSuite { assert(range.where(arrays1).sameResult(range.where(arrays2))) assert(!range.where(arrays1).sameResult(range.where(arrays3))) } + + test("SPARK-26402: accessing nested fields with different cases in case insensitive mode") { + val expId = NamedExpression.newExprId + val qualifier = Seq.empty[String] + val structType = StructType( + StructField("a", StructType(StructField("b", IntegerType, false) :: Nil), false) :: Nil) + + // GetStructField with different names are semantically equal + val fieldA1 = GetStructField( + AttributeReference("data1", structType, false)(expId, qualifier), + 0, Some("a1")) + val fieldA2 = GetStructField( + AttributeReference("data2", structType, false)(expId, qualifier), + 0, Some("a2")) + assert(fieldA1.semanticEquals(fieldA2)) + + val fieldB1 = GetStructField( + GetStructField( + AttributeReference("data1", structType, false)(expId, qualifier), + 0, Some("a1")), + 0, Some("b1")) + val fieldB2 = GetStructField( + GetStructField( + AttributeReference("data2", structType, false)(expId, qualifier), + 0, Some("a2")), + 0, Some("b2")) + assert(fieldB1.semanticEquals(fieldB2)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala index a313681eeb8f..5794691a365a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLite import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper { @@ -92,4 +93,33 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper val correctAnswer = nonNullableRelation.analyze comparePlans(actual, correctAnswer) } + + test("SPARK-26402: accessing nested fields with different cases in case insensitive mode") { + val expId = NamedExpression.newExprId + val qualifier = Seq.empty[String] + val structType = StructType( + StructField("a", StructType(StructField("b", IntegerType, false) :: Nil), false) :: Nil) + + val fieldA1 = GetStructField( + GetStructField( + AttributeReference("data1", structType, false)(expId, qualifier), + 0, Some("a1")), + 0, Some("b1")) + val fieldA2 = GetStructField( + GetStructField( + AttributeReference("data2", structType, false)(expId, qualifier), + 0, Some("a2")), + 0, Some("b2")) + + // GetStructField with different names are semantically equal; thus, `EqualTo(fieldA1, fieldA2)` + // will be optimized to `TrueLiteral` by `SimplifyBinaryComparison`. + val originalQuery = nonNullableRelation + .where(EqualTo(fieldA1, fieldA2)) + .analyze + + val optimized = Optimize.execute(originalQuery) + val correctAnswer = nonNullableRelation.analyze + + comparePlans(optimized, correctAnswer) + } } 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 beb175373754..806f0b2239fe 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 @@ -2947,6 +2947,25 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } } + + test("SPARK-26402: accessing nested fields with different cases in case insensitive mode") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val msg = intercept[AnalysisException] { + withTable("t") { + sql("create table t (s struct) using json") + checkAnswer(sql("select s.I from t group by s.i"), Nil) + } + }.message + assert(msg.contains("No such struct field I in i")) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + withTable("t") { + sql("create table t (s struct) using json") + checkAnswer(sql("select s.I from t group by s.i"), Nil) + } + } + } } case class Foo(bar: Option[String]) From c2bff77bd87a3d490458e2e79a35d770fede3b10 Mon Sep 17 00:00:00 2001 From: wangyanlin01 Date: Tue, 25 Dec 2018 15:53:42 +0800 Subject: [PATCH 306/879] [SPARK-26426][SQL] fix ExpresionInfo assert error in windows operation system. ## What changes were proposed in this pull request? fix ExpresionInfo assert error in windows operation system, when running unit tests. ## How was this patch tested? unit tests Closes #23363 from yanlin-Lynn/unit-test-windows. Authored-by: wangyanlin01 Signed-off-by: Hyukjin Kwon --- .../apache/spark/sql/catalyst/expressions/ExpressionInfo.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java index ab13ac9cc548..d5a1b77c0ec8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java @@ -79,7 +79,7 @@ public ExpressionInfo( assert name != null; assert arguments != null; assert examples != null; - assert examples.isEmpty() || examples.startsWith("\n Examples:"); + assert examples.isEmpty() || examples.startsWith(System.lineSeparator() + " Examples:"); assert note != null; assert since != null; From fa1abe24f1ac9c44425c9997d25c787cfb5ecaad Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Thu, 27 Dec 2018 11:23:05 -0800 Subject: [PATCH 307/879] Revert [SPARK-26021][SQL] replace minus zero with zero in Platform.putDouble/Float This PR reverts https://github.com/apache/spark/pull/23043 and its followup https://github.com/apache/spark/pull/23265, from branch 2.4, because it has behavior changes. existing tests Closes #23389 from cloud-fan/revert. Authored-by: Wenchen Fan Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/expressions/UnsafeRow.java | 6 ++++ .../expressions/codegen/UnsafeWriter.java | 29 ------------------- .../codegen/UnsafeRowWriterSuite.scala | 20 ------------- .../spark/sql/DataFrameAggregateSuite.scala | 14 --------- .../apache/spark/sql/DataFrameJoinSuite.scala | 12 -------- .../sql/DataFrameWindowFunctionsSuite.scala | 14 --------- .../org/apache/spark/sql/QueryTest.scala | 5 +--- 7 files changed, 7 insertions(+), 93 deletions(-) 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 9bf9452855f5..a76e6ef8c91c 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 @@ -224,6 +224,9 @@ public void setLong(int ordinal, long value) { public void setDouble(int ordinal, double value) { assertIndexIsValid(ordinal); setNotNullAt(ordinal); + if (Double.isNaN(value)) { + value = Double.NaN; + } Platform.putDouble(baseObject, getFieldOffset(ordinal), value); } @@ -252,6 +255,9 @@ public void setByte(int ordinal, byte value) { public void setFloat(int ordinal, float value) { assertIndexIsValid(ordinal); setNotNullAt(ordinal); + if (Float.isNaN(value)) { + value = Float.NaN; + } Platform.putFloat(baseObject, getFieldOffset(ordinal), value); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java index 7553ab8cf700..278165500200 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java @@ -198,45 +198,16 @@ protected final void writeLong(long offset, long value) { Platform.putLong(getBuffer(), offset, value); } - // We need to take care of NaN and -0.0 in several places: - // 1. When compare values, different NaNs should be treated as same, `-0.0` and `0.0` should be - // treated as same. - // 2. In GROUP BY, different NaNs should belong to the same group, -0.0 and 0.0 should belong - // to the same group. - // 3. As join keys, different NaNs should be treated as same, `-0.0` and `0.0` should be - // treated as same. - // 4. As window partition keys, different NaNs should be treated as same, `-0.0` and `0.0` - // should be treated as same. - // - // Case 1 is fine, as we handle NaN and -0.0 well during comparison. For complex types, we - // recursively compare the fields/elements, so it's also fine. - // - // Case 2, 3 and 4 are problematic, as they compare `UnsafeRow` binary directly, and different - // NaNs have different binary representation, and the same thing happens for -0.0 and 0.0. - // - // Here we normalize NaN and -0.0, so that `UnsafeProjection` will normalize them when writing - // float/double columns and nested fields to `UnsafeRow`. - // - // Note that, we must do this for all the `UnsafeProjection`s, not only the ones that extract - // join/grouping/window partition keys. `UnsafeProjection` copies unsafe data directly for complex - // types, so nested float/double may not be normalized. We need to make sure that all the unsafe - // data(`UnsafeRow`, `UnsafeArrayData`, `UnsafeMapData`) will have flat/double normalized during - // creation. protected final void writeFloat(long offset, float value) { if (Float.isNaN(value)) { value = Float.NaN; - } else if (value == -0.0f) { - value = 0.0f; } Platform.putFloat(getBuffer(), offset, value); } - // See comments for `writeFloat`. protected final void writeDouble(long offset, double value) { if (Double.isNaN(value)) { value = Double.NaN; - } else if (value == -0.0d) { - value = 0.0d; } Platform.putDouble(getBuffer(), offset, value); } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala index 22e1fa6dfed4..fb651b76fc16 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala @@ -50,24 +50,4 @@ class UnsafeRowWriterSuite extends SparkFunSuite { assert(res1 == res2) } - test("SPARK-26021: normalize float/double NaN and -0.0") { - val unsafeRowWriter1 = new UnsafeRowWriter(4) - unsafeRowWriter1.resetRowWriter() - unsafeRowWriter1.write(0, Float.NaN) - unsafeRowWriter1.write(1, Double.NaN) - unsafeRowWriter1.write(2, 0.0f) - unsafeRowWriter1.write(3, 0.0) - val res1 = unsafeRowWriter1.getRow - - val unsafeRowWriter2 = new UnsafeRowWriter(4) - unsafeRowWriter2.resetRowWriter() - unsafeRowWriter2.write(0, 0.0f/0.0f) - unsafeRowWriter2.write(1, 0.0/0.0) - unsafeRowWriter2.write(2, -0.0f) - unsafeRowWriter2.write(3, -0.0) - val res2 = unsafeRowWriter2.getRow - - // The two rows should be the equal - assert(res1 == res2) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index 41dc72de49be..d0106c44b7db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -727,18 +727,4 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { "grouping expressions: [current_date(None)], value: [key: int, value: string], " + "type: GroupBy]")) } - - test("SPARK-26021: Double and Float 0.0/-0.0 should be equal when grouping") { - val colName = "i" - val doubles = Seq(0.0d, -0.0d, 0.0d).toDF(colName).groupBy(colName).count().collect() - val floats = Seq(0.0f, -0.0f, 0.0f).toDF(colName).groupBy(colName).count().collect() - - assert(doubles.length == 1) - assert(floats.length == 1) - // using compare since 0.0 == -0.0 is true - assert(java.lang.Double.compare(doubles(0).getDouble(0), 0.0d) == 0) - assert(java.lang.Float.compare(floats(0).getFloat(0), 0.0f) == 0) - assert(doubles(0).getLong(1) == 3) - assert(floats(0).getLong(1) == 3) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala index c9f41ab1c017..e6b30f9956da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameJoinSuite.scala @@ -295,16 +295,4 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { df.join(df, df("id") <=> df("id")).queryExecution.optimizedPlan } } - - test("NaN and -0.0 in join keys") { - val df1 = Seq(Float.NaN -> Double.NaN, 0.0f -> 0.0, -0.0f -> -0.0).toDF("f", "d") - val df2 = Seq(Float.NaN -> Double.NaN, 0.0f -> 0.0, -0.0f -> -0.0).toDF("f", "d") - val joined = df1.join(df2, Seq("f", "d")) - checkAnswer(joined, Seq( - Row(Float.NaN, Double.NaN), - Row(0.0f, 0.0), - Row(0.0f, 0.0), - Row(0.0f, 0.0), - Row(0.0f, 0.0))) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index bbeb1d10ba7e..97a843978f0b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -658,18 +658,4 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { |GROUP BY a |HAVING SUM(b) = 5 AND RANK() OVER(ORDER BY a) = 1""".stripMargin)) } - - test("NaN and -0.0 in window partition keys") { - val df = Seq( - (Float.NaN, Double.NaN, 1), - (0.0f/0.0f, 0.0/0.0, 1), - (0.0f, 0.0, 1), - (-0.0f, -0.0, 1)).toDF("f", "d", "i") - val result = df.select($"f", count("i").over(Window.partitionBy("f", "d"))) - checkAnswer(result, Seq( - Row(Float.NaN, 2), - Row(Float.NaN, 2), - Row(0.0f, 2), - Row(0.0f, 2))) - } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 8ba67239fb90..baca9c1cfb9a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -289,7 +289,7 @@ object QueryTest { def prepareRow(row: Row): Row = { Row.fromSeq(row.toSeq.map { case null => null - case bd: java.math.BigDecimal => BigDecimal(bd) + case d: java.math.BigDecimal => BigDecimal(d) // Equality of WrappedArray differs for AnyVal and AnyRef in Scala 2.12.2+ case seq: Seq[_] => seq.map { case b: java.lang.Byte => b.byteValue @@ -303,9 +303,6 @@ object QueryTest { // Convert array to Seq for easy equality check. case b: Array[_] => b.toSeq case r: Row => prepareRow(r) - // spark treats -0.0 as 0.0 - case d: Double if d == -0.0d => 0.0d - case f: Float if f == -0.0f => 0.0f case o => o }) } From 4fb3f6dd0adcc50f30836f58391c5e1bc9a80ccf Mon Sep 17 00:00:00 2001 From: seancxmao Date: Fri, 28 Dec 2018 07:40:59 -0600 Subject: [PATCH 308/879] [SPARK-26444][WEBUI] Stage color doesn't change with it's status ## What changes were proposed in this pull request? On job page, in event timeline section, stage color doesn't change according to its status. Below are some screenshots. ACTIVE: active COMPLETE: complete FAILED: failed This PR lets stage color change with it's status. The main idea is to make css style class name match the corresponding stage status. ## How was this patch tested? Manually tested locally. ``` // active/complete stage sc.parallelize(1 to 3, 3).map { n => Thread.sleep(10* 1000); n }.count // failed stage sc.parallelize(1 to 3, 3).map { n => Thread.sleep(10* 1000); throw new Exception() }.count ``` Note we need to clear browser cache to let new `timeline-view.css` take effect. Below are screenshots after this PR. ACTIVE: active-after COMPLETE: complete-after FAILED: failed-after Closes #23385 from seancxmao/timeline-stage-color. Authored-by: seancxmao Signed-off-by: Sean Owen (cherry picked from commit 5bef4fedfe1916320223b1245bacb58f151cee66) Signed-off-by: Sean Owen --- .../org/apache/spark/ui/static/timeline-view.css | 8 ++++---- .../src/main/scala/org/apache/spark/ui/jobs/JobPage.scala | 2 +- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css index 3bf3e8bfa1f3..10bceae2fbdd 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css +++ b/core/src/main/resources/org/apache/spark/ui/static/timeline-view.css @@ -98,12 +98,12 @@ rect.getting-result-time-proportion { cursor: pointer; } -.vis-timeline .vis-item.stage.succeeded { +.vis-timeline .vis-item.stage.complete { background-color: #A0DFFF; border-color: #3EC0FF; } -.vis-timeline .vis-item.stage.succeeded.vis-selected { +.vis-timeline .vis-item.stage.complete.vis-selected { background-color: #A0DFFF; border-color: #3EC0FF; z-index: auto; @@ -130,12 +130,12 @@ rect.getting-result-time-proportion { stroke: #FF4D6D; } -.vis-timeline .vis-item.stage.running { +.vis-timeline .vis-item.stage.active { background-color: #A2FCC0; border-color: #36F572; } -.vis-timeline .vis-item.stage.running.vis-selected { +.vis-timeline .vis-item.stage.active.vis-selected { background-color: #A2FCC0; border-color: #36F572; z-index: auto; diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala index 55444a2c0c9a..33391bbb44e0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -62,7 +62,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP val stageId = stage.stageId val attemptId = stage.attemptId val name = stage.name - val status = stage.status.toString + val status = stage.status.toString.toLowerCase(Locale.ROOT) val submissionTime = stage.submissionTime.get.getTime() val completionTime = stage.completionTime.map(_.getTime()) .getOrElse(System.currentTimeMillis()) From 180212427a073e1fd01fd945bd59cb7c5f17d5eb Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Sat, 29 Dec 2018 12:11:45 -0800 Subject: [PATCH 309/879] [SPARK-26496][SS][TEST] Avoid to use Random.nextString in StreamingInnerJoinSuite MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Similar with https://github.com/apache/spark/pull/21446. Looks random string is not quite safe as a directory name. ```scala scala> val prefix = Random.nextString(10); val dir = new File("/tmp", "del_" + prefix + "-" + UUID.randomUUID.toString); dir.mkdirs() prefix: String = 窽텘⒘駖ⵚ駢⡞Ρ닋੎ dir: java.io.File = /tmp/del_窽텘⒘駖ⵚ駢⡞Ρ닋੎-a3f99855-c429-47a0-a108-47bca6905745 res40: Boolean = false // nope, didn't like this one ``` ## How was this patch tested? Unit test was added, and manually. Closes #23405 from HyukjinKwon/SPARK-26496. Authored-by: Hyukjin Kwon Signed-off-by: Dongjoon Hyun (cherry picked from commit e63243df8aca9f44255879e931e0c372beef9fc2) Signed-off-by: Dongjoon Hyun --- .../org/apache/spark/sql/streaming/StreamingJoinSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index c5cc8df4356a..42fe9f34ee3e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -350,7 +350,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with withTempDir { tempDir => val queryId = UUID.randomUUID val opId = 0 - val path = Utils.createDirectory(tempDir.getAbsolutePath, Random.nextString(10)).toString + val path = Utils.createDirectory(tempDir.getAbsolutePath, Random.nextFloat.toString).toString val stateInfo = StatefulOperatorStateInfo(path, queryId, opId, 0L, 5) implicit val sqlContext = spark.sqlContext From 1e99f4ec5d030b80971603f090afa4e51079c5e7 Mon Sep 17 00:00:00 2001 From: Imran Rashid Date: Thu, 3 Jan 2019 11:10:55 +0800 Subject: [PATCH 310/879] [SPARK-26019][PYSPARK] Allow insecure py4j gateways Spark always creates secure py4j connections between java and python, but it also allows users to pass in their own connection. This restores the ability for users to pass in an _insecure_ connection, though it forces them to set the env variable 'PYSPARK_ALLOW_INSECURE_GATEWAY=1', and still issues a warning. Added test cases verifying the failure without the extra configuration, and verifying things still work with an insecure configuration (in particular, accumulators, as those were broken with an insecure py4j gateway before). For the tests, I added ways to create insecure gateways, but I tried to put in protections to make sure that wouldn't get used incorrectly. Closes #23337 from squito/SPARK-26019. Authored-by: Imran Rashid Signed-off-by: Hyukjin Kwon --- .../api/python/PythonGatewayServer.scala | 11 +++++-- .../apache/spark/api/python/PythonRDD.scala | 6 ++-- python/pyspark/accumulators.py | 7 ++-- python/pyspark/context.py | 14 ++++++++ python/pyspark/java_gateway.py | 23 ++++++++++--- python/pyspark/tests.py | 32 +++++++++++++++++++ 6 files changed, 81 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala b/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala index 9ddc4a491018..17c65f6170d6 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonGatewayServer.scala @@ -43,12 +43,17 @@ private[spark] object PythonGatewayServer extends Logging { // with the same secret, in case the app needs callbacks from the JVM to the underlying // python processes. val localhost = InetAddress.getLoopbackAddress() - val gatewayServer: GatewayServer = new GatewayServer.GatewayServerBuilder() - .authToken(secret) + val builder = new GatewayServer.GatewayServerBuilder() .javaPort(0) .javaAddress(localhost) .callbackClient(GatewayServer.DEFAULT_PYTHON_PORT, localhost, secret) - .build() + if (sys.env.getOrElse("_PYSPARK_CREATE_INSECURE_GATEWAY", "0") != "1") { + builder.authToken(secret) + } else { + assert(sys.env.getOrElse("SPARK_TESTING", "0") == "1", + "Creating insecure Java gateways only allowed for testing") + } + val gatewayServer: GatewayServer = builder.build() gatewayServer.start() val boundPort: Int = gatewayServer.getListeningPort diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 5ed5070558af..81494b167af5 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -616,8 +616,10 @@ private[spark] class PythonAccumulatorV2( if (socket == null || socket.isClosed) { socket = new Socket(serverHost, serverPort) logInfo(s"Connected to AccumulatorServer at host: $serverHost port: $serverPort") - // send the secret just for the initial authentication when opening a new connection - socket.getOutputStream.write(secretToken.getBytes(StandardCharsets.UTF_8)) + if (secretToken != null) { + // send the secret just for the initial authentication when opening a new connection + socket.getOutputStream.write(secretToken.getBytes(StandardCharsets.UTF_8)) + } } socket } diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index 00ec094e7e3b..855d8fb4a859 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -262,9 +262,10 @@ def authenticate_and_accum_updates(): raise Exception( "The value of the provided token to the AccumulatorServer is not correct.") - # first we keep polling till we've received the authentication token - poll(authenticate_and_accum_updates) - # now we've authenticated, don't need to check for the token anymore + if auth_token is not None: + # first we keep polling till we've received the authentication token + poll(authenticate_and_accum_updates) + # now we've authenticated if needed, don't need to check for the token anymore poll(accum_updates) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 0924d3d95f04..6d99e9823f00 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -112,6 +112,20 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, ValueError:... """ self._callsite = first_spark_call() or CallSite(None, None, None) + if gateway is not None and gateway.gateway_parameters.auth_token is None: + allow_insecure_env = os.environ.get("PYSPARK_ALLOW_INSECURE_GATEWAY", "0") + if allow_insecure_env == "1" or allow_insecure_env.lower() == "true": + warnings.warn( + "You are passing in an insecure Py4j gateway. This " + "presents a security risk, and will be completely forbidden in Spark 3.0") + else: + raise ValueError( + "You are trying to pass an insecure Py4j gateway to Spark. This" + " presents a security risk. If you are sure you understand and accept this" + " risk, you can set the environment variable" + " 'PYSPARK_ALLOW_INSECURE_GATEWAY=1', but" + " note this option will be removed in Spark 3.0") + SparkContext._ensure_initialized(self, gateway=gateway, conf=conf) try: self._do_init(master, appName, sparkHome, pyFiles, environment, batchSize, serializer, diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index c8c5f801f89b..feb6b7bd6aa3 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -41,8 +41,20 @@ def launch_gateway(conf=None): """ launch jvm gateway :param conf: spark configuration passed to spark-submit - :return: + :return: a JVM gateway """ + return _launch_gateway(conf) + + +def _launch_gateway(conf=None, insecure=False): + """ + launch jvm gateway + :param conf: spark configuration passed to spark-submit + :param insecure: True to create an insecure gateway; only for testing + :return: a JVM gateway + """ + if insecure and os.environ.get("SPARK_TESTING", "0") != "1": + raise ValueError("creating insecure gateways is only for testing") if "PYSPARK_GATEWAY_PORT" in os.environ: gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"]) gateway_secret = os.environ["PYSPARK_GATEWAY_SECRET"] @@ -74,6 +86,8 @@ def launch_gateway(conf=None): env = dict(os.environ) env["_PYSPARK_DRIVER_CONN_INFO_PATH"] = conn_info_file + if insecure: + env["_PYSPARK_CREATE_INSECURE_GATEWAY"] = "1" # Launch the Java gateway. # We open a pipe to stdin so that the Java gateway can die when the pipe is broken @@ -116,9 +130,10 @@ def killChild(): atexit.register(killChild) # Connect to the gateway - gateway = JavaGateway( - gateway_parameters=GatewayParameters(port=gateway_port, auth_token=gateway_secret, - auto_convert=True)) + gateway_params = GatewayParameters(port=gateway_port, auto_convert=True) + if not insecure: + gateway_params.auth_token = gateway_secret + gateway = JavaGateway(gateway_parameters=gateway_params) # Import the classes used by PySpark java_import(gateway.jvm, "org.apache.spark.SparkConf") diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 131c51e108ca..a2d825ba3625 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -61,6 +61,7 @@ from pyspark import keyword_only from pyspark.conf import SparkConf from pyspark.context import SparkContext +from pyspark.java_gateway import _launch_gateway from pyspark.rdd import RDD from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ @@ -2381,6 +2382,37 @@ def test_startTime(self): with SparkContext() as sc: self.assertGreater(sc.startTime, 0) + def test_forbid_insecure_gateway(self): + # By default, we fail immediately if you try to create a SparkContext + # with an insecure gateway + gateway = _launch_gateway(insecure=True) + log4j = gateway.jvm.org.apache.log4j + old_level = log4j.LogManager.getRootLogger().getLevel() + try: + log4j.LogManager.getRootLogger().setLevel(log4j.Level.FATAL) + with self.assertRaises(Exception) as context: + SparkContext(gateway=gateway) + self.assertIn("insecure Py4j gateway", str(context.exception)) + self.assertIn("PYSPARK_ALLOW_INSECURE_GATEWAY", str(context.exception)) + self.assertIn("removed in Spark 3.0", str(context.exception)) + finally: + log4j.LogManager.getRootLogger().setLevel(old_level) + + def test_allow_insecure_gateway_with_conf(self): + with SparkContext._lock: + SparkContext._gateway = None + SparkContext._jvm = None + gateway = _launch_gateway(insecure=True) + try: + os.environ["PYSPARK_ALLOW_INSECURE_GATEWAY"] = "1" + with SparkContext(gateway=gateway) as sc: + a = sc.accumulator(1) + rdd = sc.parallelize([1, 2, 3]) + rdd.foreach(lambda x: a.add(x)) + self.assertEqual(7, a.value) + finally: + os.environ.pop("PYSPARK_ALLOW_INSECURE_GATEWAY", None) + class ConfTests(unittest.TestCase): def test_memory_conf(self): From bd6e5701cebf52542bc7e2321565f6207f3929e2 Mon Sep 17 00:00:00 2001 From: Liupengcheng Date: Thu, 3 Jan 2019 10:26:14 -0600 Subject: [PATCH 311/879] [SPARK-26501][CORE][TEST] Fix unexpected overriden of exitFn in SparkSubmitSuite ## What changes were proposed in this pull request? The overriden of SparkSubmit's exitFn at some previous tests in SparkSubmitSuite may cause the following tests pass even they failed when they were run separately. This PR is to fix this problem. ## How was this patch tested? unittest Closes #23404 from liupc/Fix-SparkSubmitSuite-exitFn. Authored-by: Liupengcheng Signed-off-by: Sean Owen (cherry picked from commit 88b074f3f06ddd236d63e8bf31edebe1d3e94fe4) Signed-off-by: Sean Owen --- .../spark/deploy/SparkSubmitSuite.scala | 40 ++++++++++--------- 1 file changed, 22 insertions(+), 18 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index c093789244bf..887a88f567fe 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -72,27 +72,31 @@ trait TestPrematureExit { mainObject.printStream = printStream @volatile var exitedCleanly = false + val original = mainObject.exitFn mainObject.exitFn = (_) => exitedCleanly = true - - @volatile var exception: Exception = null - val thread = new Thread { - override def run() = try { - mainObject.main(input) - } catch { - // Capture the exception to check whether the exception contains searchString or not - case e: Exception => exception = e + try { + @volatile var exception: Exception = null + val thread = new Thread { + override def run() = try { + mainObject.main(input) + } catch { + // Capture the exception to check whether the exception contains searchString or not + case e: Exception => exception = e + } } - } - thread.start() - thread.join() - if (exitedCleanly) { - val joined = printStream.lineBuffer.mkString("\n") - assert(joined.contains(searchString)) - } else { - assert(exception != null) - if (!exception.getMessage.contains(searchString)) { - throw exception + thread.start() + thread.join() + if (exitedCleanly) { + val joined = printStream.lineBuffer.mkString("\n") + assert(joined.contains(searchString)) + } else { + assert(exception != null) + if (!exception.getMessage.contains(searchString)) { + throw exception + } } + } finally { + mainObject.exitFn = original } } } From c0f4082ef0c05aee9f31fb74509d023552de8469 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 3 Jan 2019 20:01:19 -0800 Subject: [PATCH 312/879] [MINOR][NETWORK][TEST] Fix TransportFrameDecoderSuite to use ByteBuf instead of ByteBuffer ## What changes were proposed in this pull request? `fireChannelRead` expects `io.netty.buffer.ByteBuf`.I checked that this is the only place which misuse `java.nio.ByteBuffer` in `network` module. ## How was this patch tested? Pass the Jenkins with the existing tests. Closes #23442 from dongjoon-hyun/SPARK-NETWORK-COMMON. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 27e42c1de502da80fa3e22bb69de47fb00158174) Signed-off-by: Dongjoon Hyun --- .../apache/spark/network/util/TransportFrameDecoderSuite.java | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java index b53e41303751..7d40387c5f1a 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.network.util; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -69,7 +68,7 @@ public void testInterception() throws Exception { decoder.channelRead(ctx, len); decoder.channelRead(ctx, dataBuf); verify(interceptor, times(interceptedReads)).handle(any(ByteBuf.class)); - verify(ctx).fireChannelRead(any(ByteBuffer.class)); + verify(ctx).fireChannelRead(any(ByteBuf.class)); assertEquals(0, len.refCnt()); assertEquals(0, dataBuf.refCnt()); } finally { From 977d86e2d435f2422b31e87e1b4e4016e7b850fa Mon Sep 17 00:00:00 2001 From: Marco Gaido Date: Fri, 4 Jan 2019 14:53:20 -0800 Subject: [PATCH 313/879] [SPARK-26078][SQL][BACKPORT-2.4] Dedup self-join attributes on IN subqueries ## What changes were proposed in this pull request? When there is a self-join as result of a IN subquery, the join condition may be invalid, resulting in trivially true predicates and return wrong results. The PR deduplicates the subquery output in order to avoid the issue. ## How was this patch tested? added UT Closes #23449 from mgaido91/SPARK-26078_2.4. Authored-by: Marco Gaido Signed-off-by: Dongjoon Hyun --- .../sql/catalyst/optimizer/subquery.scala | 99 ++++++++++++------- .../org/apache/spark/sql/SubquerySuite.scala | 36 +++++++ 2 files changed, 97 insertions(+), 38 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala index e9b7a8b76e68..5e70d59b04a9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/subquery.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -43,31 +43,53 @@ import org.apache.spark.sql.types._ * condition. */ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { - private def dedupJoin(joinPlan: LogicalPlan): LogicalPlan = joinPlan match { + + private def buildJoin( + outerPlan: LogicalPlan, + subplan: LogicalPlan, + joinType: JoinType, + condition: Option[Expression]): Join = { + // Deduplicate conflicting attributes if any. + val dedupSubplan = dedupSubqueryOnSelfJoin(outerPlan, subplan, None, condition) + Join(outerPlan, dedupSubplan, joinType, condition) + } + + private def dedupSubqueryOnSelfJoin( + outerPlan: LogicalPlan, + subplan: LogicalPlan, + valuesOpt: Option[Seq[Expression]], + condition: Option[Expression] = None): LogicalPlan = { // SPARK-21835: It is possibly that the two sides of the join have conflicting attributes, // the produced join then becomes unresolved and break structural integrity. We should - // de-duplicate conflicting attributes. We don't use transformation here because we only - // care about the most top join converted from correlated predicate subquery. - case j @ Join(left, right, joinType @ (LeftSemi | LeftAnti | ExistenceJoin(_)), joinCond) => - val duplicates = right.outputSet.intersect(left.outputSet) - if (duplicates.nonEmpty) { - val aliasMap = AttributeMap(duplicates.map { dup => - dup -> Alias(dup, dup.toString)() - }.toSeq) - val aliasedExpressions = right.output.map { ref => - aliasMap.getOrElse(ref, ref) - } - val newRight = Project(aliasedExpressions, right) - val newJoinCond = joinCond.map { condExpr => - condExpr transform { - case a: Attribute => aliasMap.getOrElse(a, a).toAttribute + // de-duplicate conflicting attributes. + // SPARK-26078: it may also happen that the subquery has conflicting attributes with the outer + // values. In this case, the resulting join would contain trivially true conditions (eg. + // id#3 = id#3) which cannot be de-duplicated after. In this method, if there are conflicting + // attributes in the join condition, the subquery's conflicting attributes are changed using + // a projection which aliases them and resolves the problem. + val outerReferences = valuesOpt.map(values => + AttributeSet(values.flatMap(_.references))).getOrElse(AttributeSet.empty) + val outerRefs = outerPlan.outputSet ++ outerReferences + val duplicates = outerRefs.intersect(subplan.outputSet) + if (duplicates.nonEmpty) { + condition.foreach { e => + val conflictingAttrs = e.references.intersect(duplicates) + if (conflictingAttrs.nonEmpty) { + throw new AnalysisException("Found conflicting attributes " + + s"${conflictingAttrs.mkString(",")} in the condition joining outer plan:\n " + + s"$outerPlan\nand subplan:\n $subplan") } - } - Join(left, newRight, joinType, newJoinCond) - } else { - j } - case _ => joinPlan + val rewrites = AttributeMap(duplicates.map { dup => + dup -> Alias(dup, dup.toString)() + }.toSeq) + val aliasedExpressions = subplan.output.map { ref => + rewrites.getOrElse(ref, ref) + } + Project(aliasedExpressions, subplan) + } else { + subplan + } } def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -85,17 +107,16 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { withSubquery.foldLeft(newFilter) { case (p, Exists(sub, conditions, _)) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftSemi, joinCond)) + buildJoin(outerPlan, sub, LeftSemi, joinCond) case (p, Not(Exists(sub, conditions, _))) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftAnti, joinCond)) + buildJoin(outerPlan, sub, LeftAnti, joinCond) case (p, InSubquery(values, ListQuery(sub, conditions, _, _))) => - val inConditions = values.zip(sub.output).map(EqualTo.tupled) - val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions ++ conditions, p) // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftSemi, joinCond)) + val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values)) + val inConditions = values.zip(newSub.output).map(EqualTo.tupled) + val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions ++ conditions, p) + Join(outerPlan, newSub, LeftSemi, joinCond) case (p, Not(InSubquery(values, ListQuery(sub, conditions, _, _)))) => // This is a NULL-aware (left) anti join (NAAJ) e.g. col NOT IN expr // Construct the condition. A NULL in one of the conditions is regarded as a positive @@ -103,7 +124,10 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Note that will almost certainly be planned as a Broadcast Nested Loop join. // Use EXISTS if performance matters to you. - val inConditions = values.zip(sub.output).map(EqualTo.tupled) + + // Deduplicate conflicting attributes if any. + val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values)) + val inConditions = values.zip(newSub.output).map(EqualTo.tupled) val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions, p) // Expand the NOT IN expression with the NULL-aware semantic // to its full form. That is from: @@ -118,8 +142,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // will have the final conditions in the LEFT ANTI as // (A.A1 = B.B1 OR ISNULL(A.A1 = B.B1)) AND (B.B2 = A.A2) AND B.B3 > 1 val finalJoinCond = (nullAwareJoinConds ++ conditions).reduceLeft(And) - // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftAnti, Option(finalJoinCond))) + Join(outerPlan, newSub, LeftAnti, Option(finalJoinCond)) case (p, predicate) => val (newCond, inputPlan) = rewriteExistentialExpr(Seq(predicate), p) Project(p.output, Filter(newCond.get, inputPlan)) @@ -140,16 +163,16 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { e transformUp { case Exists(sub, conditions, _) => val exists = AttributeReference("exists", BooleanType, nullable = false)() - // Deduplicate conflicting attributes if any. - newPlan = dedupJoin( - Join(newPlan, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And))) + newPlan = + buildJoin(newPlan, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) exists case InSubquery(values, ListQuery(sub, conditions, _, _)) => val exists = AttributeReference("exists", BooleanType, nullable = false)() - val inConditions = values.zip(sub.output).map(EqualTo.tupled) - val newConditions = (inConditions ++ conditions).reduceLeftOption(And) // Deduplicate conflicting attributes if any. - newPlan = dedupJoin(Join(newPlan, sub, ExistenceJoin(exists), newConditions)) + val newSub = dedupSubqueryOnSelfJoin(newPlan, sub, Some(values)) + val inConditions = values.zip(newSub.output).map(EqualTo.tupled) + val newConditions = (inConditions ++ conditions).reduceLeftOption(And) + newPlan = Join(newPlan, newSub, ExistenceJoin(exists), newConditions) exists } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala index cbffed994bb4..35197798f6f6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SubquerySuite.scala @@ -1268,4 +1268,40 @@ class SubquerySuite extends QueryTest with SharedSQLContext { assert(getNumSortsInQuery(query5) == 1) } } + + test("SPARK-26078: deduplicate fake self joins for IN subqueries") { + withTempView("a", "b") { + Seq("a" -> 2, "b" -> 1).toDF("id", "num").createTempView("a") + Seq("a" -> 2, "b" -> 1).toDF("id", "num").createTempView("b") + + val df1 = spark.sql( + """ + |SELECT id,num,source FROM ( + | SELECT id, num, 'a' as source FROM a + | UNION ALL + | SELECT id, num, 'b' as source FROM b + |) AS c WHERE c.id IN (SELECT id FROM b WHERE num = 2) + """.stripMargin) + checkAnswer(df1, Seq(Row("a", 2, "a"), Row("a", 2, "b"))) + val df2 = spark.sql( + """ + |SELECT id,num,source FROM ( + | SELECT id, num, 'a' as source FROM a + | UNION ALL + | SELECT id, num, 'b' as source FROM b + |) AS c WHERE c.id NOT IN (SELECT id FROM b WHERE num = 2) + """.stripMargin) + checkAnswer(df2, Seq(Row("b", 1, "a"), Row("b", 1, "b"))) + val df3 = spark.sql( + """ + |SELECT id,num,source FROM ( + | SELECT id, num, 'a' as source FROM a + | UNION ALL + | SELECT id, num, 'b' as source FROM b + |) AS c WHERE c.id IN (SELECT id FROM b WHERE num = 2) OR + |c.id IN (SELECT id FROM b WHERE num = 3) + """.stripMargin) + checkAnswer(df3, Seq(Row("a", 2, "a"), Row("a", 2, "b"))) + } + } } From 46a88d227b2c0ba69e81c1b1937cd1108665e137 Mon Sep 17 00:00:00 2001 From: shane knapp Date: Fri, 4 Jan 2019 18:27:26 -0800 Subject: [PATCH 314/879] [SPARK-26537][BUILD] change git-wip-us to gitbox ## What changes were proposed in this pull request? due to apache recently moving from git-wip-us.apache.org to gitbox.apache.org, we need to update the packaging scripts to point to the new repo location. this will also need to be backported to 2.4, 2.3, 2.1, 2.0 and 1.6. ## How was this patch tested? the build system will test this. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23454 from shaneknapp/update-apache-repo. Authored-by: shane knapp Signed-off-by: Dongjoon Hyun (cherry picked from commit bccb8602d7bc78894689e9b2e5fe685763d32d23) Signed-off-by: Dongjoon Hyun --- dev/create-release/release-tag.sh | 2 +- dev/create-release/release-util.sh | 4 ++-- pom.xml | 2 +- 3 files changed, 4 insertions(+), 4 deletions(-) mode change 100644 => 100755 dev/create-release/release-util.sh diff --git a/dev/create-release/release-tag.sh b/dev/create-release/release-tag.sh index 628bc0504c9c..010082d960a2 100755 --- a/dev/create-release/release-tag.sh +++ b/dev/create-release/release-tag.sh @@ -61,7 +61,7 @@ done init_java init_maven_sbt -ASF_SPARK_REPO="git-wip-us.apache.org/repos/asf/spark.git" +ASF_SPARK_REPO="gitbox.apache.org/repos/asf/spark.git" rm -rf spark git clone "https://$ASF_USERNAME:$ASF_PASSWORD@$ASF_SPARK_REPO" -b $GIT_BRANCH diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh old mode 100644 new mode 100755 index 7426b0d6ca08..c925de9be52d --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -19,8 +19,8 @@ DRY_RUN=${DRY_RUN:-0} GPG="gpg --no-tty --batch" -ASF_REPO="https://git-wip-us.apache.org/repos/asf/spark.git" -ASF_REPO_WEBUI="https://git-wip-us.apache.org/repos/asf?p=spark.git" +ASF_REPO="https://gitbox.apache.org/repos/asf/spark.git" +ASF_REPO_WEBUI="https://gitbox.apache.org/repos/asf?p=spark.git" function error { echo "$*" diff --git a/pom.xml b/pom.xml index 3a0168a6faf1..c8a1b03d493b 100644 --- a/pom.xml +++ b/pom.xml @@ -39,7 +39,7 @@ scm:git:git@github.com:apache/spark.git - scm:git:https://git-wip-us.apache.org/repos/asf/spark.git + scm:git:https://gitbox.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git HEAD From 0751e02e24f3c66d12be4a3123ce3ae1c1c1872d Mon Sep 17 00:00:00 2001 From: Kris Mok Date: Sat, 5 Jan 2019 14:37:04 -0800 Subject: [PATCH 315/879] [SPARK-26545] Fix typo in EqualNullSafe's truth table comment ## What changes were proposed in this pull request? The truth table comment in EqualNullSafe incorrectly marked FALSE results as UNKNOWN. ## How was this patch tested? N/A Closes #23461 from rednaxelafx/fix-typo. Authored-by: Kris Mok Signed-off-by: gatorsmile (cherry picked from commit 4ab5b5b9185f60f671d90d94732d0d784afa5f84) Signed-off-by: gatorsmile --- .../apache/spark/sql/catalyst/expressions/predicates.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index eedfbc279d1c..365fcad7d362 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -655,9 +655,9 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp // +---------+---------+---------+---------+ // | <=> | TRUE | FALSE | UNKNOWN | // +---------+---------+---------+---------+ - // | TRUE | TRUE | FALSE | UNKNOWN | - // | FALSE | FALSE | TRUE | UNKNOWN | - // | UNKNOWN | UNKNOWN | UNKNOWN | TRUE | + // | TRUE | TRUE | FALSE | FALSE | + // | FALSE | FALSE | TRUE | FALSE | + // | UNKNOWN | FALSE | FALSE | TRUE | // +---------+---------+---------+---------+ override def eval(input: InternalRow): Any = { val input1 = left.eval(input) From 0356ac7881764dfe0d978c238b02e3f30447ebcc Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 6 Jan 2019 19:59:31 -0800 Subject: [PATCH 316/879] [SPARK-26554][BUILD] Update `release-util.sh` to avoid GitBox fake 200 headers ## What changes were proposed in this pull request? Unlike the previous Apache Git repository, new GitBox repository returns a fake HTTP 200 header instead of `404 Not Found` header. This makes release scripts out of order. This PR aims to fix it to handle the html body message instead of the fake HTTP headers. This is a release blocker. ```bash $ curl -s --head --fail "https://gitbox.apache.org/repos/asf?p=spark.git;a=commit;h=v3.0.0" HTTP/1.1 200 OK Date: Sun, 06 Jan 2019 22:42:39 GMT Server: Apache/2.4.18 (Ubuntu) Vary: Accept-Encoding Access-Control-Allow-Origin: * Access-Control-Allow-Methods: POST, GET, OPTIONS Access-Control-Allow-Headers: X-PINGOTHER Access-Control-Max-Age: 1728000 Content-Type: text/html; charset=utf-8 ``` **BEFORE** ```bash $ ./do-release-docker.sh -d /tmp/test -n Branch [branch-2.4]: Current branch version is 2.4.1-SNAPSHOT. Release [2.4.1]: RC # [1]: v2.4.1-rc1 already exists. Continue anyway [y/n]? ``` **AFTER** ```bash $ ./do-release-docker.sh -d /tmp/test -n Branch [branch-2.4]: Current branch version is 2.4.1-SNAPSHOT. Release [2.4.1]: RC # [1]: This is a dry run. Please confirm the ref that will be built for testing. Ref [v2.4.1-rc1]: ``` ## How was this patch tested? Manual. Closes #23476 from dongjoon-hyun/SPARK-26554. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit fe039faddf13c6a30f7aea69324aa4d4bb84c632) Signed-off-by: Dongjoon Hyun --- dev/create-release/release-util.sh | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh index c925de9be52d..9a340528b506 100755 --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -73,7 +73,9 @@ function fcreate_secure { } function check_for_tag { - curl -s --head --fail "$ASF_REPO_WEBUI;a=commit;h=$1" >/dev/null + # Check HTML body messages instead of header status codes. Apache GitBox returns + # a header with `200 OK` status code for both existing and non-existing tag URLs + ! curl -s --fail "$ASF_REPO_WEBUI;a=commit;h=$1" | grep '404 Not Found' > /dev/null } function get_release_info { From fe4c61ca1e298f91ba64578dee548cb3cd5da215 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 6 Jan 2019 22:45:18 -0800 Subject: [PATCH 317/879] [MINOR][BUILD] Fix script name in `release-tag.sh` usage message ## What changes were proposed in this pull request? This PR fixes the old script name in `release-tag.sh`. $ ./release-tag.sh --help | head -n1 usage: tag-release.sh ## How was this patch tested? Manual. $ ./release-tag.sh --help | head -n1 usage: release-tag.sh Closes #23477 from dongjoon-hyun/SPARK-RELEASE-TAG. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 468d25ec7419b4c55955ead877232aae5654260e) Signed-off-by: Dongjoon Hyun --- dev/create-release/release-tag.sh | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/dev/create-release/release-tag.sh b/dev/create-release/release-tag.sh index 010082d960a2..8024440759eb 100755 --- a/dev/create-release/release-tag.sh +++ b/dev/create-release/release-tag.sh @@ -21,8 +21,9 @@ SELF=$(cd $(dirname $0) && pwd) . "$SELF/release-util.sh" function exit_with_usage { + local NAME=$(basename $0) cat << EOF -usage: tag-release.sh +usage: $NAME Tags a Spark release on a particular branch. Inputs are specified with the following environment variables: From cb1aad69b781bf9612b9b14f5338b338344365f4 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 7 Jan 2019 18:36:52 +0800 Subject: [PATCH 318/879] [SPARK-26559][ML][PYSPARK] ML image can't work with numpy versions prior to 1.9 ## What changes were proposed in this pull request? Due to [API change](https://github.com/numpy/numpy/pull/4257/files#diff-c39521d89f7e61d6c0c445d93b62f7dc) at 1.9, PySpark image doesn't work with numpy version prior to 1.9. When running image test with numpy version prior to 1.9, we can see error: ``` test_read_images (pyspark.ml.tests.test_image.ImageReaderTest) ... ERROR test_read_images_multiple_times (pyspark.ml.tests.test_image.ImageReaderTest2) ... ok ====================================================================== ERROR: test_read_images (pyspark.ml.tests.test_image.ImageReaderTest) ---------------------------------------------------------------------- Traceback (most recent call last): File "/Users/viirya/docker_tmp/repos/spark-1/python/pyspark/ml/tests/test_image.py", line 36, in test_read_images self.assertEqual(ImageSchema.toImage(array, origin=first_row[0]), first_row) File "/Users/viirya/docker_tmp/repos/spark-1/python/pyspark/ml/image.py", line 193, in toImage data = bytearray(array.astype(dtype=np.uint8).ravel().tobytes()) AttributeError: 'numpy.ndarray' object has no attribute 'tobytes' ---------------------------------------------------------------------- Ran 2 tests in 29.040s FAILED (errors=1) ``` ## How was this patch tested? Manually test with numpy version prior and after 1.9. Closes #23484 from viirya/fix-pyspark-image. Authored-by: Liang-Chi Hsieh Signed-off-by: Hyukjin Kwon (cherry picked from commit a927c764c1eee066efc1c2c713dfee411de79245) Signed-off-by: Hyukjin Kwon --- python/pyspark/ml/image.py | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) diff --git a/python/pyspark/ml/image.py b/python/pyspark/ml/image.py index edb90a357854..a1aacea88e42 100644 --- a/python/pyspark/ml/image.py +++ b/python/pyspark/ml/image.py @@ -28,6 +28,7 @@ import warnings import numpy as np +from distutils.version import LooseVersion from pyspark import SparkContext from pyspark.sql.types import Row, _create_row, _parse_datatype_json_string @@ -190,7 +191,11 @@ def toImage(self, array, origin=""): # Running `bytearray(numpy.array([1]))` fails in specific Python versions # with a specific Numpy version, for example in Python 3.6.0 and NumPy 1.13.3. # Here, it avoids it by converting it to bytes. - data = bytearray(array.astype(dtype=np.uint8).ravel().tobytes()) + if LooseVersion(np.__version__) >= LooseVersion('1.9'): + data = bytearray(array.astype(dtype=np.uint8).ravel().tobytes()) + else: + # Numpy prior to 1.9 don't have `tobytes` method. + data = bytearray(array.astype(dtype=np.uint8).ravel()) # Creating new Row with _create_row(), because Row(name = value, ... ) # orders fields by name, which conflicts with expected schema order From b4202e79833f3adc00afe00f43e8d9165c9c8e48 Mon Sep 17 00:00:00 2001 From: wuyi Date: Mon, 7 Jan 2019 16:22:28 -0600 Subject: [PATCH 319/879] [SPARK-26269][YARN][BRANCH-2.4] Yarnallocator should have same blacklist behaviour with yarn to maxmize use of cluster resource ## What changes were proposed in this pull request? As I mentioned in jira [SPARK-26269](https://issues.apache.org/jira/browse/SPARK-26269), in order to maxmize the use of cluster resource, this pr try to make `YarnAllocator` have the same blacklist behaviour with YARN. ## How was this patch tested? Added. Closes #23368 from Ngone51/dev-YarnAllocator-should-have-same-blacklist-behaviour-with-YARN-branch-2.4. Lead-authored-by: wuyi Co-authored-by: Ngone51 Signed-off-by: Thomas Graves --- .../spark/deploy/yarn/YarnAllocator.scala | 31 +++++-- .../yarn/YarnAllocatorBlacklistTracker.scala | 4 +- .../YarnAllocatorBlacklistTrackerSuite.scala | 2 +- .../deploy/yarn/YarnAllocatorSuite.scala | 83 ++++++++++++++++++- 4 files changed, 107 insertions(+), 13 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index f4dc80ad4a62..3357084146e8 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -578,13 +578,23 @@ private[yarn] class YarnAllocator( (true, memLimitExceededLogMessage( completedContainer.getDiagnostics, PMEM_EXCEEDED_PATTERN)) - case _ => - // all the failures which not covered above, like: - // disk failure, kill by app master or resource manager, ... - allocatorBlacklistTracker.handleResourceAllocationFailure(hostOpt) - (true, "Container marked as failed: " + containerId + onHostStr + - ". Exit status: " + completedContainer.getExitStatus + - ". Diagnostics: " + completedContainer.getDiagnostics) + case other_exit_status => + // SPARK-26269: follow YARN's blacklisting behaviour(see https://github + // .com/apache/hadoop/blob/228156cfd1b474988bc4fedfbf7edddc87db41e3/had + // oop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/ap + // ache/hadoop/yarn/util/Apps.java#L273 for details) + if (NOT_APP_AND_SYSTEM_FAULT_EXIT_STATUS.contains(other_exit_status)) { + (false, s"Container marked as failed: $containerId$onHostStr" + + s". Exit status: ${completedContainer.getExitStatus}" + + s". Diagnostics: ${completedContainer.getDiagnostics}.") + } else { + // completed container from a bad node + allocatorBlacklistTracker.handleResourceAllocationFailure(hostOpt) + (true, s"Container from a bad node: $containerId$onHostStr" + + s". Exit status: ${completedContainer.getExitStatus}" + + s". Diagnostics: ${completedContainer.getDiagnostics}.") + } + } if (exitCausedByApp) { @@ -722,4 +732,11 @@ private object YarnAllocator { "Consider boosting spark.yarn.executor.memoryOverhead or " + "disabling yarn.nodemanager.vmem-check-enabled because of YARN-4714." } + val NOT_APP_AND_SYSTEM_FAULT_EXIT_STATUS = Set( + ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, + ContainerExitStatus.KILLED_BY_APPMASTER, + ContainerExitStatus.KILLED_AFTER_APP_COMPLETION, + ContainerExitStatus.ABORTED, + ContainerExitStatus.DISKS_FAILED + ) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala index ceac7cda5f8b..268976b62950 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala @@ -120,7 +120,9 @@ private[spark] class YarnAllocatorBlacklistTracker( if (removals.nonEmpty) { logInfo(s"removing nodes from YARN application master's blacklist: $removals") } - amClient.updateBlacklist(additions.asJava, removals.asJava) + if (additions.nonEmpty || removals.nonEmpty) { + amClient.updateBlacklist(additions.asJava, removals.asJava) + } currentBlacklistedYarnNodes = nodesToBlacklist } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala index aeac68e6ed33..201910731e93 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala @@ -87,7 +87,7 @@ class YarnAllocatorBlacklistTrackerSuite extends SparkFunSuite with Matchers // expired blacklisted nodes (simulating a resource request) yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set("host1", "host2")) // no change is communicated to YARN regarding the blacklisting - verify(amClientMock).updateBlacklist(Collections.emptyList(), Collections.emptyList()) + verify(amClientMock, times(0)).updateBlacklist(Collections.emptyList(), Collections.emptyList()) } test("combining scheduler and allocation blacklist") { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 3f783baed110..2fb892ecbc33 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy.yarn +import java.util.Collections + import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration @@ -24,6 +26,7 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.mockito.ArgumentCaptor import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfterEach, Matchers} @@ -86,7 +89,8 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter def createAllocator( maxExecutors: Int = 5, - rmClient: AMRMClient[ContainerRequest] = rmClient): YarnAllocator = { + rmClient: AMRMClient[ContainerRequest] = rmClient, + additionalConfigs: Map[String, String] = Map()): YarnAllocator = { val args = Array( "--jar", "somejar.jar", "--class", "SomeClass") @@ -95,6 +99,11 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter .set("spark.executor.instances", maxExecutors.toString) .set("spark.executor.cores", "5") .set("spark.executor.memory", "2048") + + for ((name, value) <- additionalConfigs) { + sparkConfClone.set(name, value) + } + new YarnAllocator( "not used", mock(classOf[RpcEndpointRef]), @@ -108,14 +117,29 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter clock) } - def createContainer(host: String): Container = { - // When YARN 2.6+ is required, avoid deprecation by using version with long second arg - val containerId = ContainerId.newInstance(appAttemptId, containerNum) + def createContainer( + host: String, + containerNumber: Int = containerNum, + resource: Resource = containerResource): Container = { + val containerId: ContainerId = ContainerId.newContainerId(appAttemptId, containerNum) containerNum += 1 val nodeId = NodeId.newInstance(host, 1000) Container.newInstance(containerId, nodeId, "", containerResource, RM_REQUEST_PRIORITY, null) } + def createContainers(hosts: Seq[String], containerIds: Seq[Int]): Seq[Container] = { + hosts.zip(containerIds).map{case (host, id) => createContainer(host, id)} + } + + def createContainerStatus( + containerId: ContainerId, + exitStatus: Int, + containerState: ContainerState = ContainerState.COMPLETE, + diagnostics: String = "diagnostics"): ContainerStatus = { + ContainerStatus.newInstance(containerId, containerState, diagnostics, exitStatus) + } + + test("single container allocated") { // request a single container and receive it val handler = createAllocator(1) @@ -400,4 +424,55 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter clock.advance(50 * 1000L) handler.getNumExecutorsFailed should be (0) } + + test("SPARK-26269: YarnAllocator should have same blacklist behaviour with YARN") { + val rmClientSpy = spy(rmClient) + val maxExecutors = 11 + + val handler = createAllocator( + maxExecutors, + rmClientSpy, + Map( + "spark.yarn.blacklist.executor.launch.blacklisting.enabled" -> "true", + "spark.blacklist.application.maxFailedExecutorsPerNode" -> "0")) + handler.updateResourceRequests() + + val hosts = (0 until maxExecutors).map(i => s"host$i") + val ids = 0 to maxExecutors + val containers = createContainers(hosts, ids) + + val nonBlacklistedStatuses = Seq( + ContainerExitStatus.SUCCESS, + ContainerExitStatus.PREEMPTED, + ContainerExitStatus.KILLED_EXCEEDED_VMEM, + ContainerExitStatus.KILLED_EXCEEDED_PMEM, + ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, + ContainerExitStatus.KILLED_BY_APPMASTER, + ContainerExitStatus.KILLED_AFTER_APP_COMPLETION, + ContainerExitStatus.ABORTED, + ContainerExitStatus.DISKS_FAILED) + + val nonBlacklistedContainerStatuses = nonBlacklistedStatuses.zipWithIndex.map { + case (exitStatus, idx) => createContainerStatus(containers(idx).getId, exitStatus) + } + + val BLACKLISTED_EXIT_CODE = 1 + val blacklistedStatuses = Seq(ContainerExitStatus.INVALID, BLACKLISTED_EXIT_CODE) + + val blacklistedContainerStatuses = blacklistedStatuses.zip(9 until maxExecutors).map { + case (exitStatus, idx) => createContainerStatus(containers(idx).getId, exitStatus) + } + + handler.handleAllocatedContainers(containers.slice(0, 9)) + handler.processCompletedContainers(nonBlacklistedContainerStatuses) + verify(rmClientSpy, never()) + .updateBlacklist(hosts.slice(0, 9).asJava, Collections.emptyList()) + + handler.handleAllocatedContainers(containers.slice(9, 11)) + handler.processCompletedContainers(blacklistedContainerStatuses) + verify(rmClientSpy) + .updateBlacklist(hosts.slice(9, 10).asJava, Collections.emptyList()) + verify(rmClientSpy) + .updateBlacklist(hosts.slice(10, 11).asJava, Collections.emptyList()) + } } From faa4c2823b69c1643d7678ee1cb0b7295c611334 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Mon, 7 Jan 2019 16:53:07 -0800 Subject: [PATCH 320/879] [SPARK-26267][SS] Retry when detecting incorrect offsets from Kafka (2.4) ## What changes were proposed in this pull request? Backport #23324 to branch-2.4. ## How was this patch tested? Jenkins Closes #23365 from zsxwing/SPARK-26267-2.4. Authored-by: Shixiong Zhu Signed-off-by: Shixiong Zhu --- .../sql/kafka010/KafkaContinuousReader.scala | 4 +- .../sql/kafka010/KafkaMicroBatchReader.scala | 20 ++++- .../kafka010/KafkaOffsetRangeCalculator.scala | 2 + .../sql/kafka010/KafkaOffsetReader.scala | 80 +++++++++++++++++-- .../spark/sql/kafka010/KafkaSource.scala | 5 +- .../kafka010/KafkaMicroBatchSourceSuite.scala | 48 +++++++++++ 6 files changed, 146 insertions(+), 13 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala index 8ce56a249622..561d50135932 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReader.scala @@ -73,7 +73,7 @@ class KafkaContinuousReader( offset = start.orElse { val offsets = initialOffsets match { case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) - case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss) } logInfo(s"Initial offsets: $offsets") @@ -128,7 +128,7 @@ class KafkaContinuousReader( } override def needsReconfiguration(): Boolean = { - knownPartitions != null && offsetReader.fetchLatestOffsets().keySet != knownPartitions + knownPartitions != null && offsetReader.fetchLatestOffsets(None).keySet != knownPartitions } override def toString(): String = s"KafkaSource[$offsetReader]" diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala index 8cc989fce197..b6c803545578 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala @@ -93,7 +93,8 @@ private[kafka010] class KafkaMicroBatchReader( endPartitionOffsets = Option(end.orElse(null)) .map(_.asInstanceOf[KafkaSourceOffset].partitionToOffsets) .getOrElse { - val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets() + val latestPartitionOffsets = + kafkaOffsetReader.fetchLatestOffsets(Some(startPartitionOffsets)) maxOffsetsPerTrigger.map { maxOffsets => rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets) }.getOrElse { @@ -132,10 +133,21 @@ private[kafka010] class KafkaMicroBatchReader( }.toSeq logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) + val fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets + val untilOffsets = endPartitionOffsets + untilOffsets.foreach { case (tp, untilOffset) => + fromOffsets.get(tp).foreach { fromOffset => + if (untilOffset < fromOffset) { + reportDataLoss(s"Partition $tp's offset was changed from " + + s"$fromOffset to $untilOffset, some data may have been missed") + } + } + } + // Calculate offset ranges val offsetRanges = rangeCalculator.getRanges( - fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets, - untilOffsets = endPartitionOffsets, + fromOffsets = fromOffsets, + untilOffsets = untilOffsets, executorLocations = getSortedExecutorList()) // Reuse Kafka consumers only when all the offset ranges have distinct TopicPartitions, @@ -192,7 +204,7 @@ private[kafka010] class KafkaMicroBatchReader( case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaOffsetReader.fetchEarliestOffsets()) case LatestOffsetRangeLimit => - KafkaSourceOffset(kafkaOffsetReader.fetchLatestOffsets()) + KafkaSourceOffset(kafkaOffsetReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => kafkaOffsetReader.fetchSpecificOffsets(p, reportDataLoss) } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala index fb209c724afb..600879492405 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala @@ -37,6 +37,8 @@ private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Option[Int * the read tasks of the skewed partitions to multiple Spark tasks. * The number of Spark tasks will be *approximately* `numPartitions`. It can be less or more * depending on rounding errors or Kafka partitions that didn't receive any new data. + * + * Empty ranges (`KafkaOffsetRange.size <= 0`) will be dropped. */ def getRanges( fromOffsets: PartitionOffsetMap, diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index 82066697cb95..fc443d22bf5a 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -21,6 +21,7 @@ import java.{util => ju} import java.util.concurrent.{Executors, ThreadFactory} import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration import scala.util.control.NonFatal @@ -137,6 +138,12 @@ private[kafka010] class KafkaOffsetReader( // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() + + // Call `position` to wait until the potential offset request triggered by `poll(0)` is + // done. This is a workaround for KAFKA-7703, which an async `seekToBeginning` triggered by + // `poll(0)` may reset offsets that should have been set by another request. + partitions.asScala.map(p => p -> consumer.position(p)).foreach(_ => {}) + consumer.pause(partitions) assert(partitions.asScala == partitionOffsets.keySet, "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + @@ -192,19 +199,82 @@ private[kafka010] class KafkaOffsetReader( /** * Fetch the latest offsets for the topic partitions that are indicated * in the [[ConsumerStrategy]]. + * + * Kafka may return earliest offsets when we are requesting latest offsets if `poll` is called + * right before `seekToEnd` (KAFKA-7703). As a workaround, we will call `position` right after + * `poll` to wait until the potential offset request triggered by `poll(0)` is done. + * + * In addition, to avoid other unknown issues, we also use the given `knownOffsets` to audit the + * latest offsets returned by Kafka. If we find some incorrect offsets (a latest offset is less + * than an offset in `knownOffsets`), we will retry at most `maxOffsetFetchAttempts` times. When + * a topic is recreated, the latest offsets may be less than offsets in `knownOffsets`. We cannot + * distinguish this with KAFKA-7703, so we just return whatever we get from Kafka after retrying. */ - def fetchLatestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { + def fetchLatestOffsets( + knownOffsets: Option[PartitionOffsetMap]): PartitionOffsetMap = runUninterruptibly { withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() + + // Call `position` to wait until the potential offset request triggered by `poll(0)` is + // done. This is a workaround for KAFKA-7703, which an async `seekToBeginning` triggered by + // `poll(0)` may reset offsets that should have been set by another request. + partitions.asScala.map(p => p -> consumer.position(p)).foreach(_ => {}) + consumer.pause(partitions) logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") - consumer.seekToEnd(partitions) - val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got latest offsets for partition : $partitionOffsets") - partitionOffsets + if (knownOffsets.isEmpty) { + consumer.seekToEnd(partitions) + partitions.asScala.map(p => p -> consumer.position(p)).toMap + } else { + var partitionOffsets: PartitionOffsetMap = Map.empty + + /** + * Compare `knownOffsets` and `partitionOffsets`. Returns all partitions that have incorrect + * latest offset (offset in `knownOffsets` is great than the one in `partitionOffsets`). + */ + def findIncorrectOffsets(): Seq[(TopicPartition, Long, Long)] = { + var incorrectOffsets = ArrayBuffer[(TopicPartition, Long, Long)]() + partitionOffsets.foreach { case (tp, offset) => + knownOffsets.foreach(_.get(tp).foreach { knownOffset => + if (knownOffset > offset) { + val incorrectOffset = (tp, knownOffset, offset) + incorrectOffsets += incorrectOffset + } + }) + } + incorrectOffsets + } + + // Retry to fetch latest offsets when detecting incorrect offsets. We don't use + // `withRetriesWithoutInterrupt` to retry because: + // + // - `withRetriesWithoutInterrupt` will reset the consumer for each attempt but a fresh + // consumer has a much bigger chance to hit KAFKA-7703. + // - Avoid calling `consumer.poll(0)` which may cause KAFKA-7703. + var incorrectOffsets: Seq[(TopicPartition, Long, Long)] = Nil + var attempt = 0 + do { + consumer.seekToEnd(partitions) + partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + attempt += 1 + + incorrectOffsets = findIncorrectOffsets() + if (incorrectOffsets.nonEmpty) { + logWarning("Found incorrect offsets in some partitions " + + s"(partition, previous offset, fetched offset): $incorrectOffsets") + if (attempt < maxOffsetFetchAttempts) { + logWarning("Retrying to fetch latest offsets because of incorrect offsets") + Thread.sleep(offsetFetchAttemptIntervalMs) + } + } + } while (incorrectOffsets.nonEmpty && attempt < maxOffsetFetchAttempts) + + logDebug(s"Got latest offsets for partition : $partitionOffsets") + partitionOffsets + } } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 66ec7e0cd084..d65b3cea632c 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -130,7 +130,7 @@ private[kafka010] class KafkaSource( metadataLog.get(0).getOrElse { val offsets = startingOffsets match { case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) - case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => kafkaReader.fetchSpecificOffsets(p, reportDataLoss) } metadataLog.add(0, offsets) @@ -148,7 +148,8 @@ private[kafka010] class KafkaSource( // Make sure initialPartitionOffsets is initialized initialPartitionOffsets - val latest = kafkaReader.fetchLatestOffsets() + val latest = kafkaReader.fetchLatestOffsets( + currentPartitionOffsets.orElse(Some(initialPartitionOffsets))) val offsets = maxOffsetsPerTrigger match { case None => latest diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index d89e45e1e77f..5f058332de52 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -327,6 +327,54 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { ) } + test("subscribe topic by pattern with topic recreation between batches") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-good" + val topic2 = topicPrefix + "-bad" + testUtils.createTopic(topic, partitions = 1) + testUtils.sendMessages(topic, Array("1", "3")) + testUtils.createTopic(topic2, partitions = 1) + testUtils.sendMessages(topic2, Array("2", "4")) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("kafka.default.api.timeout.ms", "3000") + .option("startingOffsets", "earliest") + .option("subscribePattern", s"$topicPrefix-.*") + + val ds = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + .map(kv => kv._2.toInt) + + testStream(ds)( + StartStream(), + AssertOnQuery { q => + q.processAllAvailable() + true + }, + CheckAnswer(1, 2, 3, 4), + // Restart the stream in this test to make the test stable. When recreating a topic when a + // consumer is alive, it may not be able to see the recreated topic even if a fresh consumer + // has seen it. + StopStream, + // Recreate `topic2` and wait until it's available + WithOffsetSync(new TopicPartition(topic2, 0), expectedOffset = 1) { () => + testUtils.deleteTopic(topic2) + testUtils.createTopic(topic2) + testUtils.sendMessages(topic2, Array("6")) + }, + StartStream(), + ExpectFailure[IllegalStateException](e => { + // The offset of `topic2` should be changed from 2 to 1 + assert(e.getMessage.contains("was changed from 2 to 1")) + }) + ) + } + test("ensure that initial offset are written with an extra byte in the beginning (SPARK-19517)") { withTempDir { metadataPath => val topic = "kafka-initial-offset-current" From 3ece0aa479bd32732742d1d8e607de25520a9f5a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Mon, 7 Jan 2019 17:54:05 -0800 Subject: [PATCH 321/879] [SPARK-26554][BUILD][FOLLOWUP] Use GitHub instead of GitBox to check HEADER ## What changes were proposed in this pull request? This PR uses GitHub repository instead of GitBox because GitHub repo returns HTTP header status correctly. ## How was this patch tested? Manual. ``` $ ./do-release-docker.sh -d /tmp/test -n Branch [branch-2.4]: Current branch version is 2.4.1-SNAPSHOT. Release [2.4.1]: RC # [1]: This is a dry run. Please confirm the ref that will be built for testing. Ref [v2.4.1-rc1]: ``` Closes #23482 from dongjoon-hyun/SPARK-26554-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 6f35ede31cc72a81e3852b1ac7454589d1897bfc) Signed-off-by: Dongjoon Hyun --- dev/create-release/release-util.sh | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/dev/create-release/release-util.sh b/dev/create-release/release-util.sh index 9a340528b506..5486c18e95bc 100755 --- a/dev/create-release/release-util.sh +++ b/dev/create-release/release-util.sh @@ -21,6 +21,7 @@ DRY_RUN=${DRY_RUN:-0} GPG="gpg --no-tty --batch" ASF_REPO="https://gitbox.apache.org/repos/asf/spark.git" ASF_REPO_WEBUI="https://gitbox.apache.org/repos/asf?p=spark.git" +ASF_GITHUB_REPO="https://github.com/apache/spark" function error { echo "$*" @@ -73,9 +74,7 @@ function fcreate_secure { } function check_for_tag { - # Check HTML body messages instead of header status codes. Apache GitBox returns - # a header with `200 OK` status code for both existing and non-existing tag URLs - ! curl -s --fail "$ASF_REPO_WEBUI;a=commit;h=$1" | grep '404 Not Found' > /dev/null + curl -s --head --fail "$ASF_GITHUB_REPO/releases/tag/$1" > /dev/null } function get_release_info { From 6277a9f8f9e8f024110056c8d12eb7d205d6d1f4 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Wed, 9 Jan 2019 10:18:33 +0800 Subject: [PATCH 322/879] [SPARK-26571][SQL] Update Hive Serde mapping with canonical name of Parquet and Orc FileFormat ## What changes were proposed in this pull request? Currently Spark table maintains Hive catalog storage format, so that Hive client can read it. In `HiveSerDe.scala`, Spark uses a mapping from its data source to HiveSerde. The mapping is old, we need to update with latest canonical name of Parquet and Orc FileFormat. Otherwise the following queries will result in wrong Serde value in Hive table(default value `org.apache.hadoop.mapred.SequenceFileInputFormat`), and Hive client will fail to read the output table: ``` df.write.format("org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat").saveAsTable(..) ``` ``` df.write.format("org.apache.spark.sql.execution.datasources.orc.OrcFileFormat").saveAsTable(..) ``` This minor PR is to fix the mapping. ## How was this patch tested? Unit test. Closes #23491 from gengliangwang/fixHiveSerdeMap. Authored-by: Gengliang Wang Signed-off-by: Wenchen Fan (cherry picked from commit 311f32f37fbeaebe9dfa0b8dc2a111ee99b583b7) Signed-off-by: Dongjoon Hyun --- .../apache/spark/sql/internal/HiveSerDe.scala | 2 ++ .../sql/hive/HiveMetastoreCatalogSuite.scala | 18 ++++++++++++ .../sql/hive/orc/HiveOrcSourceSuite.scala | 29 ------------------- 3 files changed, 20 insertions(+), 29 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala index eca612f06f9b..bd25a6437033 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala @@ -74,8 +74,10 @@ object HiveSerDe { def sourceToSerDe(source: String): Option[HiveSerDe] = { val key = source.toLowerCase(Locale.ROOT) match { case s if s.startsWith("org.apache.spark.sql.parquet") => "parquet" + case s if s.startsWith("org.apache.spark.sql.execution.datasources.parquet") => "parquet" case s if s.startsWith("org.apache.spark.sql.orc") => "orc" case s if s.startsWith("org.apache.spark.sql.hive.orc") => "orc" + case s if s.startsWith("org.apache.spark.sql.execution.datasources.orc") => "orc" case s if s.equals("orcfile") => "orc" case s if s.equals("parquetfile") => "parquet" case s if s.equals("avrofile") => "avro" diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala index 688b619cd1bb..5c9261c206ea 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveMetastoreCatalogSuite.scala @@ -159,10 +159,28 @@ class DataSourceWithHiveMetastoreCatalogSuite "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe" )), + "org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat" -> (( + "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat", + "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat", + "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe" + )), + "orc" -> (( "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat", "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat", "org.apache.hadoop.hive.ql.io.orc.OrcSerde" + )), + + "org.apache.spark.sql.hive.orc" -> (( + "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat", + "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat", + "org.apache.hadoop.hive.ql.io.orc.OrcSerde" + )), + + "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat" -> (( + "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat", + "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat", + "org.apache.hadoop.hive.ql.io.orc.OrcSerde" )) ).foreach { case (provider, (inputFormat, outputFormat, serde)) => test(s"Persist non-partitioned $provider relation into metastore as managed table") { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala index c1ae2f6861cb..c0bf18115f94 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala @@ -21,11 +21,9 @@ import java.io.File import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT} -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.orc.OrcSuite import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -67,33 +65,6 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { """.stripMargin) } - test("SPARK-22972: hive orc source") { - val tableName = "normal_orc_as_source_hive" - withTable(tableName) { - sql( - s""" - |CREATE TABLE $tableName - |USING org.apache.spark.sql.hive.orc - |OPTIONS ( - | PATH '${new File(orcTableAsDir.getAbsolutePath).toURI}' - |) - """.stripMargin) - - val tableMetadata = spark.sessionState.catalog.getTableMetadata( - TableIdentifier(tableName)) - assert(tableMetadata.storage.inputFormat == - Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")) - assert(tableMetadata.storage.outputFormat == - Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) - assert(tableMetadata.storage.serde == - Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) - assert(HiveSerDe.sourceToSerDe("org.apache.spark.sql.hive.orc") - .equals(HiveSerDe.sourceToSerDe("orc"))) - assert(HiveSerDe.sourceToSerDe("org.apache.spark.sql.orc") - .equals(HiveSerDe.sourceToSerDe("orc"))) - } - } - test("SPARK-19459/SPARK-18220: read char/varchar column written by Hive") { val location = Utils.createTempDir() val uri = location.toURI From da0b69f4a68303f74350f0efd3ba90745c32768c Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Thu, 10 Jan 2019 00:40:21 -0800 Subject: [PATCH 323/879] [SPARK-22128][CORE][BUILD] Add `paranamer` dependency to `core` module ## What changes were proposed in this pull request? With Scala-2.12 profile, Spark application fails while Spark is okay. For example, our documented `SimpleApp` Java example succeeds to compile but it fails at runtime because it doesn't use `paranamer 2.8` and hits [SPARK-22128](https://issues.apache.org/jira/browse/SPARK-22128). This PR aims to declare it explicitly for the Spark applications. Note that this doesn't introduce new dependency to Spark itself. https://dist.apache.org/repos/dist/dev/spark/3.0.0-SNAPSHOT-2019_01_09_13_59-e853afb-docs/_site/quick-start.html The following is the dependency tree from the Spark application. **BEFORE** ``` $ mvn dependency:tree -Dincludes=com.thoughtworks.paranamer [INFO] --- maven-dependency-plugin:2.8:tree (default-cli) simple --- [INFO] my.test:simple:jar:1.0-SNAPSHOT [INFO] \- org.apache.spark:spark-sql_2.12:jar:3.0.0-SNAPSHOT:compile [INFO] \- org.apache.spark:spark-core_2.12:jar:3.0.0-SNAPSHOT:compile [INFO] \- org.apache.avro:avro:jar:1.8.2:compile [INFO] \- com.thoughtworks.paranamer:paranamer:jar:2.7:compile ``` **AFTER** ``` [INFO] --- maven-dependency-plugin:2.8:tree (default-cli) simple --- [INFO] my.test:simple:jar:1.0-SNAPSHOT [INFO] \- org.apache.spark:spark-sql_2.12:jar:3.0.0-SNAPSHOT:compile [INFO] \- org.apache.spark:spark-core_2.12:jar:3.0.0-SNAPSHOT:compile [INFO] \- com.thoughtworks.paranamer:paranamer:jar:2.8:compile ``` ## How was this patch tested? Pass the Jenkins. And manually test with the sample app is running. Closes #23502 from dongjoon-hyun/SPARK-26583. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit c7daa95d7f095500b416ba405660f98cd2a39727) Signed-off-by: Dongjoon Hyun --- core/pom.xml | 4 ++++ pom.xml | 1 + 2 files changed, 5 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 583a2fff7638..d8b078e419e6 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -33,6 +33,10 @@ Spark Project Core http://spark.apache.org/ + + com.thoughtworks.paranamer + paranamer + org.apache.avro avro diff --git a/pom.xml b/pom.xml index c8a1b03d493b..f802786d1cf3 100644 --- a/pom.xml +++ b/pom.xml @@ -1978,6 +1978,7 @@ com.thoughtworks.paranamer paranamer ${paranamer.version} + runtime org.apache.arrow From b9eb0e85de3317a7f4c89a90082f7793b645c6ea Mon Sep 17 00:00:00 2001 From: John Zhuge Date: Fri, 11 Jan 2019 09:21:13 -0800 Subject: [PATCH 324/879] [SPARK-26576][SQL] Broadcast hint not applied to partitioned table ## What changes were proposed in this pull request? Make sure broadcast hint is applied to partitioned tables. Since the issue exists in branch 2.0 to 2.4, but not in master, I created this PR for branch-2.4. ## How was this patch tested? - A new unit test in PruneFileSourcePartitionsSuite - Unit test suites touched by SPARK-14581: JoinOptimizationSuite, FilterPushdownSuite, ColumnPruningSuite, and PruneFiltersSuite cloud-fan davies rxin Closes #23507 from jzhuge/SPARK-26576. Authored-by: John Zhuge Signed-off-by: gatorsmile --- .../sql/catalyst/planning/patterns.scala | 3 --- .../PruneFileSourcePartitionsSuite.scala | 19 ++++++++++++++++++- 2 files changed, 18 insertions(+), 4 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 84be677e438a..d91b89057b97 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -66,9 +66,6 @@ object PhysicalOperation extends PredicateHelper { val substitutedCondition = substitute(aliases)(condition) (fields, filters ++ splitConjunctivePredicates(substitutedCondition), other, aliases) - case h: ResolvedHint => - collectProjectsAndFilters(h.child) - case other => (None, Nil, other, Map.empty) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index 94384185d190..8a9adf7ca6ae 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -17,15 +17,20 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.Matchers._ + import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project, ResolvedHint} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec +import org.apache.spark.sql.functions.broadcast import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType @@ -91,4 +96,16 @@ class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with Te assert(size2 < tableStats.get.sizeInBytes) } } + + test("SPARK-26576 Broadcast hint not applied to partitioned table") { + withTable("tbl") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + spark.range(10).selectExpr("id", "id % 3 as p").write.partitionBy("p").saveAsTable("tbl") + val df = spark.table("tbl") + val qe = df.join(broadcast(df), "p").queryExecution + qe.optimizedPlan.collect { case _: ResolvedHint => } should have size 1 + qe.sparkPlan.collect { case j: BroadcastHashJoinExec => j } should have size 1 + } + } + } } From 0e5b316dec8e3439a4020b3e5def2f5f2c0be5f2 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Fri, 11 Jan 2019 19:23:32 +0000 Subject: [PATCH 325/879] [SPARK-26551][SQL] Fix schema pruning error when selecting one complex field and having is not null predicate on another one ## What changes were proposed in this pull request? Schema pruning has errors when selecting one complex field and having is not null predicate on another one: ```scala val query = sql("select * from contacts") .where("name.middle is not null") .select( "id", "name.first", "name.middle", "name.last" ) .where("last = 'Jones'") .select(count("id")) ``` ``` java.lang.IllegalArgumentException: middle does not exist. Available: last [info] at org.apache.spark.sql.types.StructType.$anonfun$fieldIndex$1(StructType.scala:303) [info] at scala.collection.immutable.Map$Map1.getOrElse(Map.scala:119) [info] at org.apache.spark.sql.types.StructType.fieldIndex(StructType.scala:302) [info] at org.apache.spark.sql.execution.ProjectionOverSchema.$anonfun$getProjection$6(ProjectionOverSchema.scala:58) [info] at scala.Option.map(Option.scala:163) [info] at org.apache.spark.sql.execution.ProjectionOverSchema.getProjection(ProjectionOverSchema.scala:56) [info] at org.apache.spark.sql.execution.ProjectionOverSchema.unapply(ProjectionOverSchema.scala:32) [info] at org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaPruning$$anonfun$$nestedInanonfun$buildNewProjection$1$1.applyOrElse(Parque tSchemaPruning.scala:153) ``` ## How was this patch tested? Added tests. Closes #23474 from viirya/SPARK-26551. Authored-by: Liang-Chi Hsieh Signed-off-by: DB Tsai (cherry picked from commit 50ebf3a43b84c8538ec60437189221c2c527990b) Signed-off-by: DB Tsai --- .../parquet/ParquetSchemaPruning.scala | 34 +++++++++++++----- .../parquet/ParquetSchemaPruningSuite.scala | 36 +++++++++++++++++++ 2 files changed, 62 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala index 91080b15727d..840fcae8c691 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala @@ -116,10 +116,28 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { // For example, for a query `SELECT name.first FROM contacts WHERE name IS NOT NULL`, // we don't need to read nested fields of `name` struct other than `first` field. val (rootFields, optRootFields) = (projectionRootFields ++ filterRootFields) - .distinct.partition(_.contentAccessed) + .distinct.partition(!_.prunedIfAnyChildAccessed) optRootFields.filter { opt => - !rootFields.exists(_.field.name == opt.field.name) + !rootFields.exists { root => + root.field.name == opt.field.name && { + // Checking if current optional root field can be pruned. + // For each required root field, we merge it with the optional root field: + // 1. If this optional root field has nested fields and any nested field of it is used + // in the query, the merged field type must equal to the optional root field type. + // We can prune this optional root field. For example, for optional root field + // `struct>`, if its field + // `struct>` is used, we don't need to add this optional + // root field. + // 2. If this optional root field has no nested fields, the merged field type equals + // to the optional root field only if they are the same. If they are, we can prune + // this optional root field too. + val rootFieldType = StructType(Array(root.field)) + val optFieldType = StructType(Array(opt.field)) + val merged = optFieldType.merge(rootFieldType) + merged.sameType(optFieldType) + } + } } ++ rootFields } @@ -213,11 +231,11 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { // don't actually use any nested fields. These root field accesses might be excluded later // if there are any nested fields accesses in the query plan. case IsNotNull(SelectedField(field)) => - RootField(field, derivedFromAtt = false, contentAccessed = false) :: Nil + RootField(field, derivedFromAtt = false, prunedIfAnyChildAccessed = true) :: Nil case IsNull(SelectedField(field)) => - RootField(field, derivedFromAtt = false, contentAccessed = false) :: Nil + RootField(field, derivedFromAtt = false, prunedIfAnyChildAccessed = true) :: Nil case IsNotNull(_: Attribute) | IsNull(_: Attribute) => - expr.children.flatMap(getRootFields).map(_.copy(contentAccessed = false)) + expr.children.flatMap(getRootFields).map(_.copy(prunedIfAnyChildAccessed = true)) case _ => expr.children.flatMap(getRootFields) } @@ -271,9 +289,9 @@ private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { /** * This represents a "root" schema field (aka top-level, no-parent). `field` is the * `StructField` for field name and datatype. `derivedFromAtt` indicates whether it - * was derived from an attribute or had a proper child. `contentAccessed` means whether - * it was accessed with its content by the expressions refer it. + * was derived from an attribute or had a proper child. `prunedIfAnyChildAccessed` means + * whether this root field can be pruned if any of child field is used in the query. */ private case class RootField(field: StructField, derivedFromAtt: Boolean, - contentAccessed: Boolean = true) + prunedIfAnyChildAccessed: Boolean = false) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 434c4414edeb..966190e12c6b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.SchemaPruningTest import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.StructType @@ -217,6 +218,41 @@ class ParquetSchemaPruningSuite Row("Y.") :: Nil) } + testSchemaPruning("select one complex field and having is null predicate on another " + + "complex field") { + val query = sql("select * from contacts") + .where("name.middle is not null") + .select( + "id", + "name.first", + "name.middle", + "name.last" + ) + .where("last = 'Jones'") + .select(count("id")).toDF() + checkScan(query, + "struct>") + checkAnswer(query, Row(0) :: Nil) + } + + testSchemaPruning("select one deep nested complex field and having is null predicate on " + + "another deep nested complex field") { + val query = sql("select * from contacts") + .where("employer.company.address is not null") + .selectExpr( + "id", + "name.first", + "name.middle", + "name.last", + "employer.id as employer_id" + ) + .where("employer_id = 0") + .select(count("id")).toDF() + checkScan(query, + "struct>>") + checkAnswer(query, Row(1) :: Nil) + } + private def testSchemaPruning(testName: String)(testThunk: => Unit) { test(s"Spark vectorized reader - without partition data column - $testName") { withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { From bbf61eb4222d7b46e71dc91eeedf82d27226fc2c Mon Sep 17 00:00:00 2001 From: Mukul Murthy Date: Fri, 11 Jan 2019 11:46:14 -0800 Subject: [PATCH 326/879] [SPARK-26586][SS] Fix race condition that causes streams to run with unexpected confs ## What changes were proposed in this pull request? Fix race condition where streams can have unexpected conf values. New streaming queries should run with isolated SparkSessions so that they aren't affected by conf updates after they are started. In StreamExecution, the parent SparkSession is cloned and used to run each batch, but this cloning happens in a separate thread and may happen after DataStreamWriter.start() returns. If a stream is started and a conf key is set immediately after, the stream is likely to have the new value. ## How was this patch tested? New unit test that fails prior to the production change and passes with it. Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23513 from mukulmurthy/26586. Authored-by: Mukul Murthy Signed-off-by: Shixiong Zhu (cherry picked from commit ae382c94dd10ff494dde4de44e66182bf6dbe8f8) Signed-off-by: Shixiong Zhu --- .../execution/streaming/StreamExecution.scala | 5 ++-- .../test/DataStreamReaderWriterSuite.scala | 24 +++++++++++++++++++ 2 files changed, 27 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala index de338440c873..c1aa98a2f602 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamExecution.scala @@ -176,6 +176,9 @@ abstract class StreamExecution( lazy val streamMetrics = new MetricsReporter( this, s"spark.streaming.${Option(name).getOrElse(id)}") + /** Isolated spark session to run the batches with. */ + private val sparkSessionForStream = sparkSession.cloneSession() + /** * The thread that runs the micro-batches of this stream. Note that this thread must be * [[org.apache.spark.util.UninterruptibleThread]] to workaround KAFKA-1894: interrupting a @@ -265,8 +268,6 @@ abstract class StreamExecution( // force initialization of the logical plan so that the sources can be created logicalPlan - // Isolated spark session to run the batches with. - val sparkSessionForStream = sparkSession.cloneSession() // Adaptive execution can change num shuffle partitions, disallow sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") // Disable cost-based join optimization as we do not want stateful operations to be rearranged diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 8212fb912ec5..569114afe9e0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.streaming.test import java.io.File +import java.util.ConcurrentModificationException import java.util.Locale import java.util.concurrent.TimeUnit @@ -651,4 +652,27 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { LastOptions.clear() } + + test("SPARK-26586: Streams should have isolated confs") { + import testImplicits._ + val input = MemoryStream[Int] + input.addData(1 to 10) + spark.conf.set("testKey1", 0) + val queries = (1 to 10).map { i => + spark.conf.set("testKey1", i) + input.toDF().writeStream + .foreachBatch { (df: Dataset[Row], id: Long) => + val v = df.sparkSession.conf.get("testKey1").toInt + if (i != v) { + throw new ConcurrentModificationException(s"Stream $i has the wrong conf value $v") + } + } + .start() + } + try { + queries.foreach(_.processAllAvailable()) + } finally { + queries.foreach(_.stop()) + } + } } From bb9745904f4c439be21fa435f2bb5c66de164f61 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 11 Jan 2019 22:53:58 -0800 Subject: [PATCH 327/879] [SPARK-26607][SQL][TEST] Remove Spark 2.2.x testing from HiveExternalCatalogVersionsSuite ## What changes were proposed in this pull request? The vote of final release of `branch-2.2` passed and the branch goes EOL. This PR removes Spark 2.2.x from the testing coverage. ## How was this patch tested? Pass the Jenkins. Closes #23526 from dongjoon-hyun/SPARK-26607. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 3587a9a2275615b82492b89204b141636542ce52) Signed-off-by: Dongjoon Hyun --- .../spark/sql/hive/HiveExternalCatalogVersionsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index cce2f203895f..632a21a2a894 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -203,7 +203,7 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { object PROCESS_TABLES extends QueryTest with SQLTestUtils { // Tests the latest version of every release line. - val testingVersions = Seq("2.2.2", "2.3.2", "2.4.0") + val testingVersions = Seq("2.3.2", "2.4.0") protected var spark: SparkSession = _ From dde4d1d8c409a9ee5dbaae8c12b6f9de540b4198 Mon Sep 17 00:00:00 2001 From: Oleksii Shkarupin Date: Sat, 12 Jan 2019 11:06:39 -0800 Subject: [PATCH 328/879] [SPARK-26538][SQL] Set default precision and scale for elements of postgres numeric array ## What changes were proposed in this pull request? When determining CatalystType for postgres columns with type `numeric[]` set the type of array element to `DecimalType(38, 18)` instead of `DecimalType(0,0)`. ## How was this patch tested? Tested with modified `org.apache.spark.sql.jdbc.JDBCSuite`. Ran the `PostgresIntegrationSuite` manually. Closes #23456 from a-shkarupin/postgres_numeric_array. Lead-authored-by: Oleksii Shkarupin Co-authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 5b37092311bfc1255f1d4d81127ae4242ba1d1aa) Signed-off-by: Dongjoon Hyun --- .../spark/sql/jdbc/PostgresIntegrationSuite.scala | 12 ++++++++---- .../org/apache/spark/sql/jdbc/PostgresDialect.scala | 5 ++++- .../scala/org/apache/spark/sql/jdbc/JDBCSuite.scala | 3 +++ 3 files changed, 15 insertions(+), 5 deletions(-) diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala index be32cb89f488..e8d5b468df63 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/PostgresIntegrationSuite.scala @@ -46,14 +46,15 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { conn.prepareStatement("CREATE TABLE bar (c0 text, c1 integer, c2 double precision, c3 bigint, " + "c4 bit(1), c5 bit(10), c6 bytea, c7 boolean, c8 inet, c9 cidr, " + "c10 integer[], c11 text[], c12 real[], c13 numeric(2,2)[], c14 enum_type, " - + "c15 float4, c16 smallint)").executeUpdate() + + "c15 float4, c16 smallint, c17 numeric[])").executeUpdate() conn.prepareStatement("INSERT INTO bar VALUES ('hello', 42, 1.25, 123456789012345, B'0', " + "B'1000100101', E'\\\\xDEADBEEF', true, '172.16.0.42', '192.168.0.0/16', " - + """'{1, 2}', '{"a", null, "b"}', '{0.11, 0.22}', '{0.11, 0.22}', 'd1', 1.01, 1)""" + + """'{1, 2}', '{"a", null, "b"}', '{0.11, 0.22}', '{0.11, 0.22}', 'd1', 1.01, 1, """ + + "'{111.2222, 333.4444}')" ).executeUpdate() conn.prepareStatement("INSERT INTO bar VALUES (null, null, null, null, null, " + "null, null, null, null, null, " - + "null, null, null, null, null, null, null)" + + "null, null, null, null, null, null, null, null)" ).executeUpdate() conn.prepareStatement("CREATE TABLE ts_with_timezone " + @@ -85,7 +86,7 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(rows.length == 2) // Test the types, and values using the first row. val types = rows(0).toSeq.map(x => x.getClass) - assert(types.length == 17) + assert(types.length == 18) assert(classOf[String].isAssignableFrom(types(0))) assert(classOf[java.lang.Integer].isAssignableFrom(types(1))) assert(classOf[java.lang.Double].isAssignableFrom(types(2))) @@ -103,6 +104,7 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(classOf[String].isAssignableFrom(types(14))) assert(classOf[java.lang.Float].isAssignableFrom(types(15))) assert(classOf[java.lang.Short].isAssignableFrom(types(16))) + assert(classOf[Seq[BigDecimal]].isAssignableFrom(types(17))) assert(rows(0).getString(0).equals("hello")) assert(rows(0).getInt(1) == 42) assert(rows(0).getDouble(2) == 1.25) @@ -123,6 +125,8 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(rows(0).getString(14) == "d1") assert(rows(0).getFloat(15) == 1.01f) assert(rows(0).getShort(16) == 1) + assert(rows(0).getSeq(17) == + Seq("111.222200000000000000", "333.444400000000000000").map(BigDecimal(_).bigDecimal)) // Test reading null values using the second row. assert(0.until(16).forall(rows(1).isNullAt(_))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index f8d2bc8e0f13..5be45c973a5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -60,7 +60,10 @@ private object PostgresDialect extends JdbcDialect { case "bytea" => Some(BinaryType) case "timestamp" | "timestamptz" | "time" | "timetz" => Some(TimestampType) case "date" => Some(DateType) - case "numeric" | "decimal" => Some(DecimalType.bounded(precision, scale)) + case "numeric" | "decimal" if precision > 0 => Some(DecimalType.bounded(precision, scale)) + case "numeric" | "decimal" => + // SPARK-26538: handle numeric without explicit precision and scale. + Some(DecimalType. SYSTEM_DEFAULT) case _ => None } 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 71e83767964a..8e49c52c7318 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 @@ -825,8 +825,11 @@ class JDBCSuite extends QueryTest test("PostgresDialect type mapping") { val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") + val md = new MetadataBuilder().putLong("scale", 0) assert(Postgres.getCatalystType(java.sql.Types.OTHER, "json", 1, null) === Some(StringType)) assert(Postgres.getCatalystType(java.sql.Types.OTHER, "jsonb", 1, null) === Some(StringType)) + assert(Postgres.getCatalystType(java.sql.Types.ARRAY, "_numeric", 0, md) == + Some(ArrayType(DecimalType.SYSTEM_DEFAULT))) assert(Postgres.getJDBCType(FloatType).map(_.databaseTypeDefinition).get == "FLOAT4") assert(Postgres.getJDBCType(DoubleType).map(_.databaseTypeDefinition).get == "FLOAT8") val errMsg = intercept[IllegalArgumentException] { From 743dedb7a8bf24bf7cd3d8b68add20875623b1c4 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 14 Jan 2019 19:17:39 -0600 Subject: [PATCH 329/879] [MINOR][BUILD] Remove binary license/notice files in a source release for branch-2.4+ only ## What changes were proposed in this pull request? To skip some steps to remove binary license/notice files in a source release for branch2.3 (these files only exist in master/branch-2.4 now), this pr checked a Spark release version in `dev/create-release/release-build.sh`. ## How was this patch tested? Manually checked. Closes #23538 from maropu/FixReleaseScript. Authored-by: Takeshi Yamamuro Signed-off-by: Sean Owen (cherry picked from commit abc937b24756e5d7479bac7229b0b4c1dc82efeb) Signed-off-by: Sean Owen --- dev/create-release/release-build.sh | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/dev/create-release/release-build.sh b/dev/create-release/release-build.sh index 02c4193364cc..5e65d999f0b5 100755 --- a/dev/create-release/release-build.sh +++ b/dev/create-release/release-build.sh @@ -174,10 +174,14 @@ if [[ "$1" == "package" ]]; then # Source and binary tarballs echo "Packaging release source tarballs" cp -r spark spark-$SPARK_VERSION - # For source release, exclude copy of binary license/notice - rm spark-$SPARK_VERSION/LICENSE-binary - rm spark-$SPARK_VERSION/NOTICE-binary - rm -r spark-$SPARK_VERSION/licenses-binary + + # For source release in v2.4+, exclude copy of binary license/notice + if [[ $SPARK_VERSION > "2.4" ]]; then + rm spark-$SPARK_VERSION/LICENSE-binary + rm spark-$SPARK_VERSION/NOTICE-binary + rm -r spark-$SPARK_VERSION/licenses-binary + fi + tar cvzf spark-$SPARK_VERSION.tgz spark-$SPARK_VERSION echo $GPG_PASSPHRASE | $GPG --passphrase-fd 0 --armour --output spark-$SPARK_VERSION.tgz.asc \ --detach-sig spark-$SPARK_VERSION.tgz From 22ab94f97cec22086db287d64a05efc3a177f4c2 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E2=80=9Cattilapiros=E2=80=9D?= Date: Wed, 16 Jan 2019 09:00:21 -0600 Subject: [PATCH 330/879] [SPARK-26615][CORE] Fixing transport server/client resource leaks in the core unittests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit ## What changes were proposed in this pull request? Fixing resource leaks where TransportClient/TransportServer instances are not closed properly. In StandaloneSchedulerBackend the null check is added because during the SparkContextSchedulerCreationSuite #"local-cluster" test it turned out that client is not initialised as org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend#start isn't called. It throw an NPE and some resource remained in open. ## How was this patch tested? By executing the unittests and using some extra temporary logging for counting created and closed TransportClient/TransportServer instances. Closes #23540 from attilapiros/leaks. Authored-by: “attilapiros” Signed-off-by: Sean Owen (cherry picked from commit 819e5ea7c290f842c51ead8b4a6593678aeef6bf) Signed-off-by: Sean Owen --- .../cluster/StandaloneSchedulerBackend.scala | 5 +- .../SparkContextSchedulerCreationSuite.scala | 103 +++++++------ .../spark/deploy/client/AppClientSuite.scala | 75 +++++----- .../spark/deploy/master/MasterSuite.scala | 111 +++++++------- .../spark/storage/BlockManagerSuite.scala | 138 ++++++++---------- 5 files changed, 228 insertions(+), 204 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index f73a58ff5d48..6df821f5b3fd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -223,8 +223,9 @@ private[spark] class StandaloneSchedulerBackend( if (stopping.compareAndSet(false, true)) { try { super.stop() - client.stop() - + if (client != null) { + client.stop() + } val callback = shutdownCallback if (callback != null) { callback(this) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index f8938dfedee5..811b9757232e 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -23,110 +23,129 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend import org.apache.spark.scheduler.local.LocalSchedulerBackend +import org.apache.spark.util.Utils class SparkContextSchedulerCreationSuite extends SparkFunSuite with LocalSparkContext with PrivateMethodTester with Logging { - def createTaskScheduler(master: String): TaskSchedulerImpl = - createTaskScheduler(master, "client") + def noOp(taskSchedulerImpl: TaskSchedulerImpl): Unit = {} - def createTaskScheduler(master: String, deployMode: String): TaskSchedulerImpl = - createTaskScheduler(master, deployMode, new SparkConf()) + def createTaskScheduler(master: String)(body: TaskSchedulerImpl => Unit = noOp): Unit = + createTaskScheduler(master, "client")(body) + + def createTaskScheduler(master: String, deployMode: String)( + body: TaskSchedulerImpl => Unit): Unit = + createTaskScheduler(master, deployMode, new SparkConf())(body) def createTaskScheduler( master: String, deployMode: String, - conf: SparkConf): TaskSchedulerImpl = { + conf: SparkConf)(body: TaskSchedulerImpl => Unit): Unit = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. sc = new SparkContext("local", "test", conf) val createTaskSchedulerMethod = PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler) - val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master, deployMode) - sched.asInstanceOf[TaskSchedulerImpl] + val (_, sched) = + SparkContext invokePrivate createTaskSchedulerMethod(sc, master, deployMode) + try { + body(sched.asInstanceOf[TaskSchedulerImpl]) + } finally { + Utils.tryLogNonFatalError { + sched.stop() + } + } } test("bad-master") { val e = intercept[SparkException] { - createTaskScheduler("localhost:1234") + createTaskScheduler("localhost:1234")() } assert(e.getMessage.contains("Could not parse Master URL")) } test("local") { - val sched = createTaskScheduler("local") - sched.backend match { - case s: LocalSchedulerBackend => assert(s.totalCores === 1) - case _ => fail() + val sched = createTaskScheduler("local") { sched => + sched.backend match { + case s: LocalSchedulerBackend => assert(s.totalCores === 1) + case _ => fail() + } } } test("local-*") { - val sched = createTaskScheduler("local[*]") - sched.backend match { - case s: LocalSchedulerBackend => - assert(s.totalCores === Runtime.getRuntime.availableProcessors()) - case _ => fail() + val sched = createTaskScheduler("local[*]") { sched => + sched.backend match { + case s: LocalSchedulerBackend => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } } } test("local-n") { - val sched = createTaskScheduler("local[5]") - assert(sched.maxTaskFailures === 1) - sched.backend match { - case s: LocalSchedulerBackend => assert(s.totalCores === 5) - case _ => fail() + val sched = createTaskScheduler("local[5]") { sched => + assert(sched.maxTaskFailures === 1) + sched.backend match { + case s: LocalSchedulerBackend => assert(s.totalCores === 5) + case _ => fail() + } } } test("local-*-n-failures") { - val sched = createTaskScheduler("local[* ,2]") - assert(sched.maxTaskFailures === 2) - sched.backend match { - case s: LocalSchedulerBackend => - assert(s.totalCores === Runtime.getRuntime.availableProcessors()) - case _ => fail() + val sched = createTaskScheduler("local[* ,2]") { sched => + assert(sched.maxTaskFailures === 2) + sched.backend match { + case s: LocalSchedulerBackend => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } } } test("local-n-failures") { - val sched = createTaskScheduler("local[4, 2]") - assert(sched.maxTaskFailures === 2) - sched.backend match { - case s: LocalSchedulerBackend => assert(s.totalCores === 4) - case _ => fail() + val sched = createTaskScheduler("local[4, 2]") { sched => + assert(sched.maxTaskFailures === 2) + sched.backend match { + case s: LocalSchedulerBackend => assert(s.totalCores === 4) + case _ => fail() + } } } test("bad-local-n") { val e = intercept[SparkException] { - createTaskScheduler("local[2*]") + createTaskScheduler("local[2*]")() } assert(e.getMessage.contains("Could not parse Master URL")) } test("bad-local-n-failures") { val e = intercept[SparkException] { - createTaskScheduler("local[2*,4]") + createTaskScheduler("local[2*,4]")() } assert(e.getMessage.contains("Could not parse Master URL")) } test("local-default-parallelism") { val conf = new SparkConf().set("spark.default.parallelism", "16") - val sched = createTaskScheduler("local", "client", conf) - sched.backend match { - case s: LocalSchedulerBackend => assert(s.defaultParallelism() === 16) - case _ => fail() + val sched = createTaskScheduler("local", "client", conf) { sched => + sched.backend match { + case s: LocalSchedulerBackend => assert(s.defaultParallelism() === 16) + case _ => fail() + } } } test("local-cluster") { - createTaskScheduler("local-cluster[3, 14, 1024]").backend match { - case s: StandaloneSchedulerBackend => // OK - case _ => fail() + createTaskScheduler("local-cluster[3, 14, 1024]") { sched => + sched.backend match { + case s: StandaloneSchedulerBackend => // OK + case _ => fail() + } } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index a1707e6540b3..baeefea3158e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.client +import java.io.Closeable import java.util.concurrent.ConcurrentLinkedQueue import scala.concurrent.duration._ @@ -85,57 +86,59 @@ class AppClientSuite } test("interface methods of AppClient using local Master") { - val ci = new AppClientInst(masterRpcEnv.address.toSparkURL) + Utils.tryWithResource(new AppClientInst(masterRpcEnv.address.toSparkURL)) { ci => - ci.client.start() + ci.client.start() - // Client should connect with one Master which registers the application - eventually(timeout(10.seconds), interval(10.millis)) { - val apps = getApplications() - assert(ci.listener.connectedIdList.size === 1, "client listener should have one connection") - assert(apps.size === 1, "master should have 1 registered app") - } + // Client should connect with one Master which registers the application + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(ci.listener.connectedIdList.size === 1, "client listener should have one connection") + assert(apps.size === 1, "master should have 1 registered app") + } - // Send message to Master to request Executors, verify request by change in executor limit - val numExecutorsRequested = 1 - whenReady( + // Send message to Master to request Executors, verify request by change in executor limit + val numExecutorsRequested = 1 + whenReady( ci.client.requestTotalExecutors(numExecutorsRequested), timeout(10.seconds), interval(10.millis)) { acknowledged => - assert(acknowledged) - } + assert(acknowledged) + } - eventually(timeout(10.seconds), interval(10.millis)) { - val apps = getApplications() - assert(apps.head.getExecutorLimit === numExecutorsRequested, s"executor request failed") - } + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.head.getExecutorLimit === numExecutorsRequested, s"executor request failed") + } - // Send request to kill executor, verify request was made - val executorId: String = getApplications().head.executors.head._2.fullId - whenReady( + // Send request to kill executor, verify request was made + val executorId: String = getApplications().head.executors.head._2.fullId + whenReady( ci.client.killExecutors(Seq(executorId)), timeout(10.seconds), interval(10.millis)) { acknowledged => - assert(acknowledged) - } + assert(acknowledged) + } - // Issue stop command for Client to disconnect from Master - ci.client.stop() + // Issue stop command for Client to disconnect from Master + ci.client.stop() - // Verify Client is marked dead and unregistered from Master - eventually(timeout(10.seconds), interval(10.millis)) { - val apps = getApplications() - assert(ci.listener.deadReasonList.size === 1, "client should have been marked dead") - assert(apps.isEmpty, "master should have 0 registered apps") + // Verify Client is marked dead and unregistered from Master + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(ci.listener.deadReasonList.size === 1, "client should have been marked dead") + assert(apps.isEmpty, "master should have 0 registered apps") + } } } test("request from AppClient before initialized with master") { - val ci = new AppClientInst(masterRpcEnv.address.toSparkURL) + Utils.tryWithResource(new AppClientInst(masterRpcEnv.address.toSparkURL)) { ci => - // requests to master should fail immediately - whenReady(ci.client.requestTotalExecutors(3), timeout(1.seconds)) { success => - assert(success === false) + // requests to master should fail immediately + whenReady(ci.client.requestTotalExecutors(3), timeout(1.seconds)) { success => + assert(success === false) + } } } @@ -219,13 +222,17 @@ class AppClientSuite } /** Create AppClient and supporting objects */ - private class AppClientInst(masterUrl: String) { + private class AppClientInst(masterUrl: String) extends Closeable { val rpcEnv = RpcEnv.create("spark", Utils.localHostName(), 0, conf, securityManager) private val cmd = new Command(TestExecutor.getClass.getCanonicalName.stripSuffix("$"), List(), Map(), Seq(), Seq(), Seq()) private val desc = new ApplicationDescription("AppClientSuite", Some(1), 512, cmd, "ignored") val listener = new AppClientCollector val client = new StandaloneAppClient(rpcEnv, Array(masterUrl), desc, listener, new SparkConf) + + override def close(): Unit = { + rpcEnv.shutdown() + } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 84b3a29b58bf..07830fdd3b4c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -642,59 +642,70 @@ class MasterSuite extends SparkFunSuite val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") } - val worker1 = new MockWorker(master.self) - worker1.rpcEnv.setupEndpoint("worker", worker1) - val worker1Reg = RegisterWorker( - worker1.id, - "localhost", - 9998, - worker1.self, - 10, - 1024, - "http://localhost:8080", - RpcAddress("localhost2", 10000)) - master.self.send(worker1Reg) - val driver = DeployTestUtils.createDriverDesc().copy(supervise = true) - master.self.askSync[SubmitDriverResponse](RequestSubmitDriver(driver)) - - eventually(timeout(10.seconds)) { - assert(worker1.apps.nonEmpty) - } - - eventually(timeout(10.seconds)) { - val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) - assert(masterState.workers(0).state == WorkerState.DEAD) - } + var worker1: MockWorker = null + var worker2: MockWorker = null + try { + worker1 = new MockWorker(master.self) + worker1.rpcEnv.setupEndpoint("worker", worker1) + val worker1Reg = RegisterWorker( + worker1.id, + "localhost", + 9998, + worker1.self, + 10, + 1024, + "http://localhost:8080", + RpcAddress("localhost2", 10000)) + master.self.send(worker1Reg) + val driver = DeployTestUtils.createDriverDesc().copy(supervise = true) + master.self.askSync[SubmitDriverResponse](RequestSubmitDriver(driver)) + + eventually(timeout(10.seconds)) { + assert(worker1.apps.nonEmpty) + } - val worker2 = new MockWorker(master.self) - worker2.rpcEnv.setupEndpoint("worker", worker2) - master.self.send(RegisterWorker( - worker2.id, - "localhost", - 9999, - worker2.self, - 10, - 1024, - "http://localhost:8081", - RpcAddress("localhost", 10001))) - eventually(timeout(10.seconds)) { - assert(worker2.apps.nonEmpty) - } + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.workers(0).state == WorkerState.DEAD) + } - master.self.send(worker1Reg) - eventually(timeout(10.seconds)) { - val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + worker2 = new MockWorker(master.self) + worker2.rpcEnv.setupEndpoint("worker", worker2) + master.self.send(RegisterWorker( + worker2.id, + "localhost", + 9999, + worker2.self, + 10, + 1024, + "http://localhost:8081", + RpcAddress("localhost", 10001))) + eventually(timeout(10.seconds)) { + assert(worker2.apps.nonEmpty) + } - val worker = masterState.workers.filter(w => w.id == worker1.id) - assert(worker.length == 1) - // make sure the `DriverStateChanged` arrives at Master. - assert(worker(0).drivers.isEmpty) - assert(worker1.apps.isEmpty) - assert(worker1.drivers.isEmpty) - assert(worker2.apps.size == 1) - assert(worker2.drivers.size == 1) - assert(masterState.activeDrivers.length == 1) - assert(masterState.activeApps.length == 1) + master.self.send(worker1Reg) + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + + val worker = masterState.workers.filter(w => w.id == worker1.id) + assert(worker.length == 1) + // make sure the `DriverStateChanged` arrives at Master. + assert(worker(0).drivers.isEmpty) + assert(worker1.apps.isEmpty) + assert(worker1.drivers.isEmpty) + assert(worker2.apps.size == 1) + assert(worker2.drivers.size == 1) + assert(masterState.activeDrivers.length == 1) + assert(masterState.activeApps.length == 1) + } + } finally { + if (worker1 != null) { + worker1.rpcEnv.shutdown() + } + if (worker2 != null) { + worker2.rpcEnv.shutdown() + } } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 32d6e8b94e1a..abde4dfd226c 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -45,7 +45,6 @@ import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransport import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap} import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} -import org.apache.spark.network.util.TransportConf import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} @@ -65,9 +64,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE implicit val defaultSignaler: Signaler = ThreadSignaler var conf: SparkConf = null - var store: BlockManager = null - var store2: BlockManager = null - var store3: BlockManager = null + val allStores = ArrayBuffer[BlockManager]() var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null val securityMgr = new SecurityManager(new SparkConf(false)) @@ -105,6 +102,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val blockManager = new BlockManager(name, rpcEnv, master, serializerManager, bmConf, memManager, mapOutputTracker, shuffleManager, transfer, bmSecurityMgr, 0) memManager.setMemoryStore(blockManager.memoryStore) + allStores += blockManager blockManager.initialize("app-id") blockManager } @@ -141,18 +139,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE override def afterEach(): Unit = { try { conf = null - if (store != null) { - store.stop() - store = null - } - if (store2 != null) { - store2.stop() - store2 = null - } - if (store3 != null) { - store3.stop() - store3 = null - } + allStores.foreach(_.stop()) + allStores.clear() rpcEnv.shutdown() rpcEnv.awaitTermination() rpcEnv = null @@ -162,6 +150,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } } + private def stopBlockManager(blockManager: BlockManager): Unit = { + allStores -= blockManager + blockManager.stop() + } + test("StorageLevel object caching") { val level1 = StorageLevel(false, false, false, 3) // this should return the same object as level1 @@ -205,7 +198,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("master + 1 manager interaction") { - store = makeBlockManager(20000) + val store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -235,8 +228,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("master + 2 managers interaction") { - store = makeBlockManager(2000, "exec1") - store2 = makeBlockManager(2000, "exec2") + val store = makeBlockManager(2000, "exec1") + val store2 = makeBlockManager(2000, "exec2") val peers = master.getPeers(store.blockManagerId) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -251,7 +244,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("removing block") { - store = makeBlockManager(20000) + val store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -299,7 +292,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("removing rdd") { - store = makeBlockManager(20000) + val store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -332,7 +325,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("removing broadcast") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val driverStore = store val executorStore = makeBlockManager(2000, "executor") val a1 = new Array[Byte](400) @@ -398,11 +391,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } executorStore.stop() driverStore.stop() - store = null } test("reregistration on heart beat") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -419,7 +411,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("reregistration on block update") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -437,7 +429,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("reregistration doesn't dead lock") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -475,7 +467,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("correct BlockResult returned from get() calls") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](500), new Array[Byte](1000), new Array[Byte](1500)) val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray) @@ -546,27 +538,25 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("SPARK-9591: getRemoteBytes from another location when Exception throw") { conf.set("spark.shuffle.io.maxRetries", "0") - store = makeBlockManager(8000, "executor1") - store2 = makeBlockManager(8000, "executor2") - store3 = makeBlockManager(8000, "executor3") + val store = makeBlockManager(8000, "executor1") + val store2 = makeBlockManager(8000, "executor2") + val store3 = makeBlockManager(8000, "executor3") val list1 = List(new Array[Byte](4000)) store2.putIterator( "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) store3.putIterator( "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") - store2.stop() - store2 = null + stopBlockManager(store2) assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") - store3.stop() - store3 = null + stopBlockManager(store3) // Should return None instead of throwing an exception: assert(store.getRemoteBytes("list1").isEmpty) } test("SPARK-14252: getOrElseUpdate should still read from remote storage") { - store = makeBlockManager(8000, "executor1") - store2 = makeBlockManager(8000, "executor2") + val store = makeBlockManager(8000, "executor1") + val store2 = makeBlockManager(8000, "executor2") val list1 = List(new Array[Byte](4000)) store2.putIterator( "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) @@ -594,7 +584,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } private def testInMemoryLRUStorage(storageLevel: StorageLevel): Unit = { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -613,7 +603,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("in-memory LRU for partitions of same RDD") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -632,7 +622,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("in-memory LRU for partitions of multiple RDDs") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) @@ -655,7 +645,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("on-disk storage") { _conf => - store = makeBlockManager(1200, testConf = Some(_conf)) + val store = makeBlockManager(1200, testConf = Some(_conf)) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -695,7 +685,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE storageLevel: StorageLevel, getAsBytes: Boolean, testConf: SparkConf): Unit = { - store = makeBlockManager(12000, testConf = Some(testConf)) + val store = makeBlockManager(12000, testConf = Some(testConf)) val accessMethod = if (getAsBytes) store.getLocalBytesAndReleaseLock else store.getSingleAndReleaseLock val a1 = new Array[Byte](4000) @@ -724,7 +714,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("LRU with mixed storage levels") { _conf => - store = makeBlockManager(12000, testConf = Some(_conf)) + val store = makeBlockManager(12000, testConf = Some(_conf)) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -746,7 +736,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("in-memory LRU with streams") { _conf => - store = makeBlockManager(12000, testConf = Some(_conf)) + val store = makeBlockManager(12000, testConf = Some(_conf)) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) @@ -774,7 +764,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("LRU with mixed storage levels and streams") { _conf => - store = makeBlockManager(12000, testConf = Some(_conf)) + val store = makeBlockManager(12000, testConf = Some(_conf)) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) @@ -827,7 +817,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("overly large block") { - store = makeBlockManager(5000) + val store = makeBlockManager(5000) store.putSingle("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.getSingleAndReleaseLock("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](10000), StorageLevel.MEMORY_AND_DISK) @@ -838,13 +828,12 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("block compression") { try { conf.set("spark.shuffle.compress", "true") - store = makeBlockManager(20000, "exec1") + var store = makeBlockManager(20000, "exec1") store.putSingle( ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") - store.stop() - store = null + stopBlockManager(store) conf.set("spark.shuffle.compress", "false") store = makeBlockManager(20000, "exec2") @@ -852,8 +841,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 10000, "shuffle_0_0_0 was compressed") - store.stop() - store = null + stopBlockManager(store) conf.set("spark.broadcast.compress", "true") store = makeBlockManager(20000, "exec3") @@ -861,37 +849,32 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 1000, "broadcast_0 was not compressed") - store.stop() - store = null + stopBlockManager(store) conf.set("spark.broadcast.compress", "false") store = makeBlockManager(20000, "exec4") store.putSingle( BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 10000, "broadcast_0 was compressed") - store.stop() - store = null + stopBlockManager(store) conf.set("spark.rdd.compress", "true") store = makeBlockManager(20000, "exec5") store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) <= 1000, "rdd_0_0 was not compressed") - store.stop() - store = null + stopBlockManager(store) conf.set("spark.rdd.compress", "false") store = makeBlockManager(20000, "exec6") store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) >= 10000, "rdd_0_0 was compressed") - store.stop() - store = null + stopBlockManager(store) // Check that any other block types are also kept uncompressed store = makeBlockManager(20000, "exec7") store.putSingle("other_block", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 10000, "other_block was compressed") - store.stop() - store = null + stopBlockManager(store) } finally { System.clearProperty("spark.shuffle.compress") System.clearProperty("spark.broadcast.compress") @@ -905,7 +888,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) val memoryManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(new JavaSerializer(conf), conf) - store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, + val store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, serializerManager, conf, memoryManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) memoryManager.setMemoryStore(store.memoryStore) @@ -927,7 +910,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("turn off updated block statuses") { val conf = new SparkConf() conf.set(TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES, false) - store = makeBlockManager(12000, testConf = Some(conf)) + val store = makeBlockManager(12000, testConf = Some(conf)) store.registerTask(0) val list = List.fill(2)(new Array[Byte](2000)) @@ -955,7 +938,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("updated block statuses") { val conf = new SparkConf() conf.set(TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES, true) - store = makeBlockManager(12000, testConf = Some(conf)) + val store = makeBlockManager(12000, testConf = Some(conf)) store.registerTask(0) val list = List.fill(2)(new Array[Byte](2000)) val bigList = List.fill(8)(new Array[Byte](2000)) @@ -1053,7 +1036,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("query block statuses") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val list = List.fill(2)(new Array[Byte](2000)) // Tell master. By LRU, only list2 and list3 remains. @@ -1098,7 +1081,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("get matching blocks") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val list = List.fill(2)(new Array[Byte](100)) // insert some blocks @@ -1142,7 +1125,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) store.putSingle(rdd(0, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. @@ -1156,7 +1139,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("safely unroll blocks through putIterator (disk)") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val memoryStore = store.memoryStore val diskStore = store.diskStore val smallList = List.fill(40)(new Array[Byte](100)) @@ -1195,7 +1178,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("read-locked blocks cannot be evicted from memory") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val arr = new Array[Byte](4000) // First store a1 and a2, both in memory, and a3, on disk only store.putSingle("a1", arr, StorageLevel.MEMORY_ONLY_SER) @@ -1221,7 +1204,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE private def testReadWithLossOfOnDiskFiles( storageLevel: StorageLevel, readMethod: BlockManager => Option[_]): Unit = { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) assert(store.putSingle("blockId", new Array[Byte](4000), storageLevel)) assert(store.getStatus("blockId").isDefined) // Directly delete all files from the disk store, triggering failures when reading blocks: @@ -1261,7 +1244,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("SPARK-13328: refresh block locations (fetch should fail after hitting a threshold)") { val mockBlockTransferService = new MockBlockTransferService(conf.getInt("spark.block.failures.beforeLocationRefresh", 5)) - store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + val store = + makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) store.putSingle("item", 999L, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.getRemoteBytes("item").isEmpty) } @@ -1281,7 +1265,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn( blockManagerIds) - store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, + val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, transferService = Option(mockBlockTransferService)) val block = store.getRemoteBytes("item") .asInstanceOf[Option[ByteBuffer]] @@ -1302,8 +1286,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE throw new InterruptedException("Intentional interrupt") } } - store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) - store2 = makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService)) + val store = + makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + val store2 = + makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService)) intercept[InterruptedException] { store.putSingle("item", "value", StorageLevel.MEMORY_ONLY_2, tellMaster = true) } @@ -1313,8 +1299,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("SPARK-17484: master block locations are updated following an invalid remote block fetch") { - store = makeBlockManager(8000, "executor1") - store2 = makeBlockManager(8000, "executor2") + val store = makeBlockManager(8000, "executor1") + val store2 = makeBlockManager(8000, "executor2") store.putSingle("item", "value", StorageLevel.MEMORY_ONLY, tellMaster = true) assert(master.getLocations("item").nonEmpty) store.removeBlock("item", tellMaster = false) @@ -1411,7 +1397,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE Option(BlockLocationsAndStatus(blockLocations, blockStatus))) when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(blockLocations) - store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, + val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, transferService = Option(mockBlockTransferService)) val block = store.getRemoteBytes("item") .asInstanceOf[Option[ByteBuffer]] From e52acc2afcba8662b337b42e44a23ef118deea0f Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 16 Jan 2019 23:23:36 +0800 Subject: [PATCH 331/879] [MINOR][PYTHON] Fix SQLContext to SparkSession in Python API main page ## What changes were proposed in this pull request? This PR proposes to fix deprecated `SQLContext` to `SparkSession` in Python API main page. **Before:** ![screen shot 2019-01-16 at 5 30 19 pm](https://user-images.githubusercontent.com/6477701/51239583-bac82f80-19b4-11e9-9129-8dae2c23ec79.png) **After:** ![screen shot 2019-01-16 at 5 29 54 pm](https://user-images.githubusercontent.com/6477701/51239577-b734a880-19b4-11e9-8539-592cb772168d.png) ## How was this patch tested? Manually checked the doc after building it. I also checked by `grep -r "SQLContext"` and looks this is the only instance left. Closes #23565 from HyukjinKwon/minor-doc-change. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit e92088de4d6755f975eb8b44b4d75b81e5a0720e) Signed-off-by: Hyukjin Kwon --- python/docs/index.rst | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/docs/index.rst b/python/docs/index.rst index 421c8de86a3c..0e7b62361802 100644 --- a/python/docs/index.rst +++ b/python/docs/index.rst @@ -37,7 +37,7 @@ Core classes: A Discretized Stream (DStream), the basic abstraction in Spark Streaming. - :class:`pyspark.sql.SQLContext` + :class:`pyspark.sql.SparkSession` Main entry point for DataFrame and SQL functionality. From 3337477b759433f56d2a43be596196479f2b00de Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Wed, 16 Jan 2019 23:25:57 +0800 Subject: [PATCH 332/879] [SPARK-25992][PYTHON] Document SparkContext cannot be shared for multiprocessing This PR proposes to explicitly document that SparkContext cannot be shared for multiprocessing, and multi-processing execution is not guaranteed in PySpark. I have seen some cases that users attempt to use multiple processes via `multiprocessing` module time to time. For instance, see the example in the JIRA (https://issues.apache.org/jira/browse/SPARK-25992). Py4J itself does not support Python's multiprocessing out of the box (sharing the same JavaGateways for instance). In general, such pattern can cause errors with somewhat arbitrary symptoms difficult to diagnose. For instance, see the error message in JIRA: ``` Traceback (most recent call last): File "/Users/abdealijk/anaconda3/lib/python3.6/socketserver.py", line 317, in _handle_request_noblock self.process_request(request, client_address) File "/Users/abdealijk/anaconda3/lib/python3.6/socketserver.py", line 348, in process_request self.finish_request(request, client_address) File "/Users/abdealijk/anaconda3/lib/python3.6/socketserver.py", line 361, in finish_request self.RequestHandlerClass(request, client_address, self) File "/Users/abdealijk/anaconda3/lib/python3.6/socketserver.py", line 696, in __init__ self.handle() File "/usr/local/hadoop/spark2.3.1/python/pyspark/accumulators.py", line 238, in handle _accumulatorRegistry[aid] += update KeyError: 0 ``` The root cause of this was because global `_accumulatorRegistry` is not shared across processes. Using thread instead of process is quite easy in Python. See `threading` vs `multiprocessing` in Python - they can be usually direct replacement for each other. For instance, Python also support threadpool as well (`multiprocessing.pool.ThreadPool`) which can be direct replacement of process-based thread pool (`multiprocessing.Pool`). Manually tested, and manually built the doc. Closes #23564 from HyukjinKwon/SPARK-25992. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon (cherry picked from commit 670bc55f8d357a5cd894e290cc2834e952a7cfe0) Signed-off-by: Hyukjin Kwon --- python/pyspark/context.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 6d99e9823f00..aff36359c86a 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -63,6 +63,10 @@ class SparkContext(object): Main entry point for Spark functionality. A SparkContext represents the connection to a Spark cluster, and can be used to create L{RDD} and broadcast variables on that cluster. + + .. note:: :class:`SparkContext` instance is not supported to share across multiple + processes out of the box, and PySpark does not guarantee multi-processing execution. + Use threads instead for concurrent processing purpose. """ _gateway = None From 1843c16fda09a3e9373e8f7b3ff5f73455c50442 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 16 Jan 2019 09:42:14 -0800 Subject: [PATCH 333/879] [SPARK-26629][SS] Fixed error with multiple file stream in a query + restart on a batch that has no data for one file stream ## What changes were proposed in this pull request? When a streaming query has multiple file streams, and there is a batch where one of the file streams dont have data in that batch, then if the query has to restart from that, it will throw the following error. ``` java.lang.IllegalStateException: batch 1 doesn't exist at org.apache.spark.sql.execution.streaming.HDFSMetadataLog$.verifyBatchIds(HDFSMetadataLog.scala:300) at org.apache.spark.sql.execution.streaming.FileStreamSourceLog.get(FileStreamSourceLog.scala:120) at org.apache.spark.sql.execution.streaming.FileStreamSource.getBatch(FileStreamSource.scala:181) at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets$2.apply(MicroBatchExecution.scala:294) at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets$2.apply(MicroBatchExecution.scala:291) at scala.collection.Iterator$class.foreach(Iterator.scala:891) at scala.collection.AbstractIterator.foreach(Iterator.scala:1334) at scala.collection.IterableLike$class.foreach(IterableLike.scala:72) at org.apache.spark.sql.execution.streaming.StreamProgress.foreach(StreamProgress.scala:25) at org.apache.spark.sql.execution.streaming.MicroBatchExecution.org$apache$spark$sql$execution$streaming$MicroBatchExecution$$populateStartOffsets(MicroBatchExecution.scala:291) at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply$mcV$sp(MicroBatchExecution.scala:178) at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:175) at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1$$anonfun$apply$mcZ$sp$1.apply(MicroBatchExecution.scala:175) at org.apache.spark.sql.execution.streaming.ProgressReporter$class.reportTimeTaken(ProgressReporter.scala:251) at org.apache.spark.sql.execution.streaming.StreamExecution.reportTimeTaken(StreamExecution.scala:61) at org.apache.spark.sql.execution.streaming.MicroBatchExecution$$anonfun$runActivatedStream$1.apply$mcZ$sp(MicroBatchExecution.scala:175) at org.apache.spark.sql.execution.streaming.ProcessingTimeExecutor.execute(TriggerExecutor.scala:56) at org.apache.spark.sql.execution.streaming.MicroBatchExecution.runActivatedStream(MicroBatchExecution.scala:169) at org.apache.spark.sql.execution.streaming.StreamExecution.org$apache$spark$sql$execution$streaming$StreamExecution$$runStream(StreamExecution.scala:295) at org.apache.spark.sql.execution.streaming.StreamExecution$$anon$1.run(StreamExecution.scala:205) ``` Existing `HDFSMetadata.verifyBatchIds` threw error whenever the `batchIds` list was empty. In the context of `FileStreamSource.getBatch` (where verify is called) and `FileStreamSourceLog` (subclass of `HDFSMetadata`), this is usually okay because, in a streaming query with one file stream, the `batchIds` can never be empty: - A batch is planned only when the `FileStreamSourceLog` has seen new offset (that is, there are new data files). - So `FileStreamSource.getBatch` will be called on X to Y where X will always be > Y. This calls internally`HDFSMetadata.verifyBatchIds (X+1, Y)` with X+1-Y ids. For example.,`FileStreamSource.getBatch(4, 5)` will call `verify(batchIds = Seq(5), start = 5, end = 5)`. However, the invariant of X > Y is not true when there are two file stream sources, as a batch may be planned even when only one of the file streams has data. So one of the file stream may not have data, which can call `FileStreamSource.getBatch(X, X)` -> `verify(batchIds = Seq.empty, start = X+1, end = X)` -> failure. Note that `FileStreamSource.getBatch(X, X)` gets called **only when restarting a query in a batch where a file source did not have data**. This is because in normal planning of batches, `MicroBatchExecution` avoids calling `FileStreamSource.getBatch(X, X)` when offset X has not changed. However, when restarting a stream at such a batch, `MicroBatchExecution.populateStartOffsets()` calls `FileStreamSource.getBatch(X, X)` (DataSource V1 hack to initialize the source with last known offsets) thus hitting this issue. The minimum solution here is to skip verification when `FileStreamSource.getBatch(X, X)`. ## How was this patch tested? (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) Please review http://spark.apache.org/contributing.html before opening a pull request. Closes #23557 from tdas/SPARK-26629. Authored-by: Tathagata Das Signed-off-by: Shixiong Zhu (cherry picked from commit 06d5b173b687c23aa53e293ed6e12ec746393876) Signed-off-by: Shixiong Zhu --- .../streaming/FileStreamSourceLog.scala | 4 +- .../execution/streaming/HDFSMetadataLog.scala | 3 +- .../streaming/HDFSMetadataLogSuite.scala | 6 ++ .../sql/streaming/FileStreamSourceSuite.scala | 75 +++++++++++++++++-- 4 files changed, 80 insertions(+), 8 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala index 8628471fdb92..7b2ea9627a98 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala @@ -117,7 +117,9 @@ class FileStreamSourceLog( val batches = (existedBatches ++ retrievedBatches).map(i => i._1 -> i._2.get).toArray.sortBy(_._1) - HDFSMetadataLog.verifyBatchIds(batches.map(_._1), startId, endId) + if (startBatchId <= endBatchId) { + HDFSMetadataLog.verifyBatchIds(batches.map(_._1), startId, endId) + } batches } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index bd0a46115ceb..62d524ff19b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -262,7 +262,8 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path: object HDFSMetadataLog { /** - * Verify if batchIds are continuous and between `startId` and `endId`. + * Verify if batchIds are continuous and between `startId` and `endId` (both inclusive and + * startId assumed to be <= endId). * * @param batchIds the sorted ids to verify. * @param startId the start id. If it's set, batchIds should start with this id. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala index 9268306ce427..0e36e7f5da12 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala @@ -178,5 +178,11 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), None, Some(5L))) intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), Some(1L), Some(5L))) intercept[IllegalStateException](verifyBatchIds(Seq(1, 2, 4, 5), Some(1L), Some(5L))) + + // Related to SPARK-26629, this capatures the behavior for verifyBatchIds when startId > endId + intercept[IllegalStateException](verifyBatchIds(Seq(), Some(2L), Some(1L))) + intercept[AssertionError](verifyBatchIds(Seq(2), Some(2L), Some(1L))) + intercept[AssertionError](verifyBatchIds(Seq(1), Some(2L), Some(1L))) + intercept[AssertionError](verifyBatchIds(Seq(0), Some(2L), Some(1L))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index d4bd9c7987f2..fb0b3656173e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -48,21 +48,33 @@ abstract class FileStreamSourceTest * `FileStreamSource` actually being used in the execution. */ abstract class AddFileData extends AddData { + private val _qualifiedBasePath = PrivateMethod[Path]('qualifiedBasePath) + + private def isSamePath(fileSource: FileStreamSource, srcPath: File): Boolean = { + val path = (fileSource invokePrivate _qualifiedBasePath()).toString.stripPrefix("file:") + path == srcPath.getCanonicalPath + } + override def addData(query: Option[StreamExecution]): (Source, Offset) = { require( query.nonEmpty, "Cannot add data when there is no query for finding the active file stream source") val sources = getSourcesFromStreamingQuery(query.get) - if (sources.isEmpty) { + val source = if (sources.isEmpty) { throw new Exception( "Could not find file source in the StreamExecution logical plan to add data to") - } else if (sources.size > 1) { - throw new Exception( - "Could not select the file source in the StreamExecution logical plan as there" + - "are multiple file sources:\n\t" + sources.mkString("\n\t")) + } else if (sources.size == 1) { + sources.head + } else { + val matchedSources = sources.filter(isSamePath(_, src)) + if (matchedSources.size != 1) { + throw new Exception( + "Could not select the file source in StreamExecution as there are multiple" + + s" file sources and none / more than one matches $src:\n" + sources.mkString("\n")) + } + matchedSources.head } - val source = sources.head val newOffset = source.withBatchingLocked { addData(source) new FileStreamSourceOffset(source.currentLogOffset + 1) @@ -71,6 +83,9 @@ abstract class FileStreamSourceTest (source, newOffset) } + /** Source directory to add file data to */ + protected def src: File + protected def addData(source: FileStreamSource): Unit } @@ -1494,6 +1509,54 @@ class FileStreamSourceSuite extends FileStreamSourceTest { newSource.getBatch(None, FileStreamSourceOffset(1)) } } + + test("SPARK-26629: multiple file sources work with restarts when a source does not have data") { + withTempDirs { case (dir, tmp) => + val sourceDir1 = new File(dir, "source1") + val sourceDir2 = new File(dir, "source2") + sourceDir1.mkdirs() + sourceDir2.mkdirs() + + val source1 = createFileStream("text", s"${sourceDir1.getCanonicalPath}") + val source2 = createFileStream("text", s"${sourceDir2.getCanonicalPath}") + val unioned = source1.union(source2) + + def addMultiTextFileData( + source1Content: String, + source2Content: String): StreamAction = { + val actions = Seq( + AddTextFileData(source1Content, sourceDir1, tmp), + AddTextFileData(source2Content, sourceDir2, tmp) + ).filter(_.content != null) // don't write to a source dir if no content specified + StreamProgressLockedActions(actions, desc = actions.mkString("[ ", " | ", " ]")) + } + + testStream(unioned)( + StartStream(), + addMultiTextFileData(source1Content = "source1_0", source2Content = "source2_0"), + CheckNewAnswer("source1_0", "source2_0"), + StopStream, + + StartStream(), + addMultiTextFileData(source1Content = "source1_1", source2Content = null), + CheckNewAnswer("source1_1"), + StopStream, + + // Restart after a batch with one file source having no new data. + // This restart is needed to hit the issue in SPARK-26629. + + StartStream(), + addMultiTextFileData(source1Content = null, source2Content = "source2_2"), + CheckNewAnswer("source2_2"), + StopStream, + + StartStream(), + addMultiTextFileData(source1Content = "source1_3", source2Content = "source2_3"), + CheckNewAnswer("source1_3", "source2_3"), + StopStream + ) + } + } } class FileStreamSourceStressTestSuite extends FileStreamSourceTest { From d608325a9fbefbe3a05e2bc8f6a95b2fa9c4174b Mon Sep 17 00:00:00 2001 From: Kris Mok Date: Wed, 16 Jan 2019 15:21:11 -0800 Subject: [PATCH 334/879] [SPARK-26633][REPL] Add ExecutorClassLoader.getResourceAsStream ## What changes were proposed in this pull request? Add `ExecutorClassLoader.getResourceAsStream`, so that classes dynamically generated by the REPL can be accessed by user code as `InputStream`s for non-class-loading purposes, such as reading the class file for extracting method/constructor parameter names. Caveat: The convention in Java's `ClassLoader` is that `ClassLoader.getResourceAsStream()` should be considered as a convenience method of `ClassLoader.getResource()`, where the latter provides a `URL` for the resource, and the former invokes `openStream()` on it to serve the resource as an `InputStream`. The former should also catch `IOException` from `openStream()` and convert it to `null`. This PR breaks this convention by only overriding `ClassLoader.getResourceAsStream()` instead of also overriding `ClassLoader.getResource()`, so after this PR, it would be possible to get a non-null result from the former, but get a null result from the latter. This isn't ideal, but it's sufficient to cover the main use case and practically it shouldn't matter. To implement the convention properly, we'd need to register a URL protocol handler with Java to allow it to properly handle the `spark://` protocol, etc, which sounds like an overkill for the intent of this PR. Credit goes to zsxwing for the initial investigation and fix suggestion. ## How was this patch tested? Added new test case in `ExecutorClassLoaderSuite` and `ReplSuite`. Closes #23558 from rednaxelafx/executorclassloader-getresourceasstream. Authored-by: Kris Mok Signed-off-by: gatorsmile (cherry picked from commit dc3b35c5da42def803dd05e2db7506714018e27b) Signed-off-by: gatorsmile --- .../spark/repl/ExecutorClassLoader.scala | 31 ++++++++++++++++-- .../spark/repl/ExecutorClassLoaderSuite.scala | 11 +++++++ .../org/apache/spark/repl/ReplSuite.scala | 32 +++++++++++++++++++ 3 files changed, 72 insertions(+), 2 deletions(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index 88eb0ad1da3d..a4a11f09ad2b 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -33,8 +33,11 @@ import org.apache.spark.util.ParentClassLoader /** * A ClassLoader that reads classes from a Hadoop FileSystem or Spark RPC endpoint, used to load * classes defined by the interpreter when the REPL is used. Allows the user to specify if user - * class path should be first. This class loader delegates getting/finding resources to parent - * loader, which makes sense until REPL never provide resource dynamically. + * class path should be first. + * This class loader delegates getting/finding resources to parent loader, which makes sense because + * the REPL never produce resources dynamically. One exception is when getting a Class file as + * resource stream, in which case we will try to fetch the Class file in the same way as loading + * the class, so that dynamically generated Classes from the REPL can be picked up. * * Note: [[ClassLoader]] will preferentially load class from parent. Only when parent is null or * the load failed, that it will call the overridden `findClass` function. To avoid the potential @@ -71,6 +74,30 @@ class ExecutorClassLoader( parentLoader.getResources(name) } + override def getResourceAsStream(name: String): InputStream = { + if (userClassPathFirst) { + val res = getClassResourceAsStreamLocally(name) + if (res != null) res else parentLoader.getResourceAsStream(name) + } else { + val res = parentLoader.getResourceAsStream(name) + if (res != null) res else getClassResourceAsStreamLocally(name) + } + } + + private def getClassResourceAsStreamLocally(name: String): InputStream = { + // Class files can be dynamically generated from the REPL. Allow this class loader to + // load such files for purposes other than loading the class. + try { + if (name.endsWith(".class")) fetchFn(name) else null + } catch { + // The helper functions referenced by fetchFn throw CNFE to indicate failure to fetch + // the class. It matches what IOException was supposed to be used for, and + // ClassLoader.getResourceAsStream() catches IOException and returns null in that case. + // So we follow that model and handle CNFE here. + case _: ClassNotFoundException => null + } + } + override def findClass(name: String): Class[_] = { if (userClassPathFirst) { findClassLocally(name).getOrElse(parentLoader.loadClass(name)) diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index e5e2094368fb..1ff93a179b6e 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -208,6 +208,17 @@ class ExecutorClassLoaderSuite intercept[java.lang.ClassNotFoundException] { classLoader.loadClass("ReplFakeClassDoesNotExist").newInstance() } + + // classLoader.getResourceAsStream() should also be able to fetch the Class file + val fakeClassInputStream = classLoader.getResourceAsStream("ReplFakeClass2.class") + try { + val magic = new Array[Byte](4) + fakeClassInputStream.read(magic) + // first 4 bytes should match the magic number of Class file + assert(magic === Array[Byte](0xCA.toByte, 0xFE.toByte, 0xBA.toByte, 0xBE.toByte)) + } finally { + if (fakeClassInputStream != null) fakeClassInputStream.close() + } } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 4f3df729177f..a46cb6b3f401 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -260,4 +260,36 @@ class ReplSuite extends SparkFunSuite { assertContains("!!2!!", output2) } + test("SPARK-26633: ExecutorClassLoader.getResourceAsStream find REPL classes") { + val output = runInterpreterInPasteMode("local-cluster[1,1,1024]", + """ + |case class TestClass(value: Int) + | + |sc.parallelize(1 to 1).map { _ => + | val clz = classOf[TestClass] + | val name = clz.getName.replace('.', '/') + ".class"; + | val stream = clz.getClassLoader.getResourceAsStream(name) + | if (stream == null) { + | "failed: stream is null" + | } else { + | val magic = new Array[Byte](4) + | try { + | stream.read(magic) + | // the magic number of a Java Class file + | val expected = Array[Byte](0xCA.toByte, 0xFE.toByte, 0xBA.toByte, 0xBE.toByte) + | if (magic sameElements expected) { + | "successful" + | } else { + | "failed: unexpected contents from stream" + | } + | } finally { + | stream.close() + | } + | } + |}.collect() + """.stripMargin) + assertDoesNotContain("failed", output) + assertContains("successful", output) + } + } From 5a2128cdcfa63be36e751d328bc1bf5c60227752 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 17 Jan 2019 14:24:21 -0600 Subject: [PATCH 335/879] [SPARK-26638][PYSPARK][ML] Pyspark vector classes always return error for unary negation ## What changes were proposed in this pull request? Fix implementation of unary negation (`__neg__`) in Pyspark DenseVectors ## How was this patch tested? Existing tests, plus new doctest Closes #23570 from srowen/SPARK-26638. Authored-by: Sean Owen Signed-off-by: Sean Owen (cherry picked from commit 0b3abef1950f486001160ec578e4f628c199eeb4) Signed-off-by: Sean Owen --- python/pyspark/ml/linalg/__init__.py | 6 +++++- python/pyspark/mllib/linalg/__init__.py | 6 +++++- 2 files changed, 10 insertions(+), 2 deletions(-) diff --git a/python/pyspark/ml/linalg/__init__.py b/python/pyspark/ml/linalg/__init__.py index 2548fd0f50b3..9da983667be7 100644 --- a/python/pyspark/ml/linalg/__init__.py +++ b/python/pyspark/ml/linalg/__init__.py @@ -270,6 +270,8 @@ class DenseVector(Vector): DenseVector([3.0, 2.0]) >>> u % 2 DenseVector([1.0, 0.0]) + >>> -v + DenseVector([-1.0, -2.0]) """ def __init__(self, ar): if isinstance(ar, bytes): @@ -436,6 +438,9 @@ def __hash__(self): def __getattr__(self, item): return getattr(self.array, item) + def __neg__(self): + return DenseVector(-self.array) + def _delegate(op): def func(self, other): if isinstance(other, DenseVector): @@ -443,7 +448,6 @@ def func(self, other): return DenseVector(getattr(self.array, op)(other)) return func - __neg__ = _delegate("__neg__") __add__ = _delegate("__add__") __sub__ = _delegate("__sub__") __mul__ = _delegate("__mul__") diff --git a/python/pyspark/mllib/linalg/__init__.py b/python/pyspark/mllib/linalg/__init__.py index 4afd6666400b..94a3e2af4d2d 100644 --- a/python/pyspark/mllib/linalg/__init__.py +++ b/python/pyspark/mllib/linalg/__init__.py @@ -281,6 +281,8 @@ class DenseVector(Vector): DenseVector([3.0, 2.0]) >>> u % 2 DenseVector([1.0, 0.0]) + >>> -v + DenseVector([-1.0, -2.0]) """ def __init__(self, ar): if isinstance(ar, bytes): @@ -480,6 +482,9 @@ def __hash__(self): def __getattr__(self, item): return getattr(self.array, item) + def __neg__(self): + return DenseVector(-self.array) + def _delegate(op): def func(self, other): if isinstance(other, DenseVector): @@ -487,7 +492,6 @@ def func(self, other): return DenseVector(getattr(self.array, op)(other)) return func - __neg__ = _delegate("__neg__") __add__ = _delegate("__add__") __sub__ = _delegate("__sub__") __mul__ = _delegate("__mul__") From 123adbdbadedd0f77ac3cde0a1bb55c1b7c42b09 Mon Sep 17 00:00:00 2001 From: Shahid Date: Sun, 20 Jan 2019 18:11:14 -0600 Subject: [PATCH 336/879] [SPARK-26351][MLLIB] Update doc and minor correction in the mllib evaluation metrics ## What changes were proposed in this pull request? Currently, there are some minor inconsistencies in doc compared to the code. In this PR, I am correcting those inconsistencies. 1) Links related to the evaluation metrics in the docs are not working 2) Minor correction in the evaluation metrics formulas in docs. ## How was this patch tested? NA Closes #23589 from shahidki31/docCorrection. Authored-by: Shahid Signed-off-by: Sean Owen (cherry picked from commit 9a30e23211e165a44acc0dbe19693950f7a7cc73) Signed-off-by: Sean Owen --- docs/mllib-evaluation-metrics.md | 22 +++++++++---------- .../mllib/evaluation/RankingMetrics.scala | 2 ++ 2 files changed, 13 insertions(+), 11 deletions(-) diff --git a/docs/mllib-evaluation-metrics.md b/docs/mllib-evaluation-metrics.md index c65ecdcb67ee..896d95bd4884 100644 --- a/docs/mllib-evaluation-metrics.md +++ b/docs/mllib-evaluation-metrics.md @@ -413,13 +413,13 @@ A ranking system usually deals with a set of $M$ users $$U = \left\{u_0, u_1, ..., u_{M-1}\right\}$$ -Each user ($u_i$) having a set of $N$ ground truth relevant documents +Each user ($u_i$) having a set of $N_i$ ground truth relevant documents -$$D_i = \left\{d_0, d_1, ..., d_{N-1}\right\}$$ +$$D_i = \left\{d_0, d_1, ..., d_{N_i-1}\right\}$$ -And a list of $Q$ recommended documents, in order of decreasing relevance +And a list of $Q_i$ recommended documents, in order of decreasing relevance -$$R_i = \left[r_0, r_1, ..., r_{Q-1}\right]$$ +$$R_i = \left[r_0, r_1, ..., r_{Q_i-1}\right]$$ The goal of the ranking system is to produce the most relevant set of documents for each user. The relevance of the sets and the effectiveness of the algorithms can be measured using the metrics listed below. @@ -439,10 +439,10 @@ $$rel_D(r) = \begin{cases}1 & \text{if $r \in D$}, \\ 0 & \text{otherwise}.\end{ Precision at k - $p(k)=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{k} \sum_{j=0}^{\text{min}(\left|D\right|, k) - 1} rel_{D_i}(R_i(j))}$ + $p(k)=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{k} \sum_{j=0}^{\text{min}(Q_i, k) - 1} rel_{D_i}(R_i(j))}$ - Precision at k is a measure of + Precision at k is a measure of how many of the first k recommended documents are in the set of true relevant documents averaged across all users. In this metric, the order of the recommendations is not taken into account. @@ -450,10 +450,10 @@ $$rel_D(r) = \begin{cases}1 & \text{if $r \in D$}, \\ 0 & \text{otherwise}.\end{ Mean Average Precision - $MAP=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{\left|D_i\right|} \sum_{j=0}^{Q-1} \frac{rel_{D_i}(R_i(j))}{j + 1}}$ + $MAP=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{N_i} \sum_{j=0}^{Q_i-1} \frac{rel_{D_i}(R_i(j))}{j + 1}}$ - MAP is a measure of how + MAP is a measure of how many of the recommended documents are in the set of true relevant documents, where the order of the recommendations is taken into account (i.e. penalty for highly relevant documents is higher). @@ -462,10 +462,10 @@ $$rel_D(r) = \begin{cases}1 & \text{if $r \in D$}, \\ 0 & \text{otherwise}.\end{ Normalized Discounted Cumulative Gain $NDCG(k)=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{IDCG(D_i, k)}\sum_{j=0}^{n-1} - \frac{rel_{D_i}(R_i(j))}{\text{ln}(j+2)}} \\ + \frac{rel_{D_i}(R_i(j))}{\text{log}(j+2)}} \\ \text{Where} \\ - \hspace{5 mm} n = \text{min}\left(\text{max}\left(|R_i|,|D_i|\right),k\right) \\ - \hspace{5 mm} IDCG(D, k) = \sum_{j=0}^{\text{min}(\left|D\right|, k) - 1} \frac{1}{\text{ln}(j+2)}$ + \hspace{5 mm} n = \text{min}\left(\text{max}\left(Q_i, N_i\right),k\right) \\ + \hspace{5 mm} IDCG(D, k) = \sum_{j=0}^{\text{min}(\left|D\right|, k) - 1} \frac{1}{\text{log}(j+2)}$ NDCG at k is a diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala index b98aa0534152..4935d1141113 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala @@ -138,6 +138,8 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] var dcg = 0.0 var i = 0 while (i < n) { + // Base of the log doesn't matter for calculating NDCG, + // if the relevance value is binary. val gain = 1.0 / math.log(i + 2) if (i < pred.length && labSet.contains(pred(i))) { dcg += gain From 98141088888a4f51aeb281f14a8421ac1d735c85 Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Tue, 22 Jan 2019 09:00:52 -0800 Subject: [PATCH 337/879] [SPARK-26665][CORE] Fix a bug that BlockTransferService.fetchBlockSync may hang forever ## What changes were proposed in this pull request? `ByteBuffer.allocate` may throw `OutOfMemoryError` when the block is large but no enough memory is available. However, when this happens, right now BlockTransferService.fetchBlockSync will just hang forever as its `BlockFetchingListener. onBlockFetchSuccess` doesn't complete `Promise`. This PR catches `Throwable` and uses the error to complete `Promise`. ## How was this patch tested? Added a unit test. Since I cannot make `ByteBuffer.allocate` throw `OutOfMemoryError`, I passed a negative size to make `ByteBuffer.allocate` fail. Although the error type is different, it should trigger the same code path. Closes #23590 from zsxwing/SPARK-26665. Authored-by: Shixiong Zhu Signed-off-by: Shixiong Zhu (cherry picked from commit 66450bbc1bb4397f06ca9a6ecba4d16c82d711fd) Signed-off-by: Shixiong Zhu --- .../spark/network/BlockTransferService.scala | 12 +- .../network/BlockTransferServiceSuite.scala | 104 ++++++++++++++++++ 2 files changed, 112 insertions(+), 4 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index a58c8fa2e763..51ced697c6af 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -107,10 +107,14 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo case e: EncryptedManagedBuffer => result.success(e) case _ => - val ret = ByteBuffer.allocate(data.size.toInt) - ret.put(data.nioByteBuffer()) - ret.flip() - result.success(new NioManagedBuffer(ret)) + try { + val ret = ByteBuffer.allocate(data.size.toInt) + ret.put(data.nioByteBuffer()) + ret.flip() + result.success(new NioManagedBuffer(ret)) + } catch { + case e: Throwable => result.failure(e) + } } } }, tempFileManager) diff --git a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala new file mode 100644 index 000000000000..d7e4b9166fa0 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala @@ -0,0 +1,104 @@ +/* + * 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.network + +import java.io.InputStream +import java.nio.ByteBuffer + +import scala.concurrent.Future +import scala.concurrent.duration._ +import scala.reflect.ClassTag + +import org.scalatest.concurrent._ + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} +import org.apache.spark.storage.{BlockId, StorageLevel} + +class BlockTransferServiceSuite extends SparkFunSuite with TimeLimits { + + implicit val defaultSignaler: Signaler = ThreadSignaler + + test("fetchBlockSync should not hang when BlockFetchingListener.onBlockFetchSuccess fails") { + // Create a mocked `BlockTransferService` to call `BlockFetchingListener.onBlockFetchSuccess` + // with a bad `ManagedBuffer` which will trigger an exception in `onBlockFetchSuccess`. + val blockTransferService = new BlockTransferService { + override def init(blockDataManager: BlockDataManager): Unit = {} + + override def close(): Unit = {} + + override def port: Int = 0 + + override def hostName: String = "localhost-unused" + + override def fetchBlocks( + host: String, + port: Int, + execId: String, + blockIds: Array[String], + listener: BlockFetchingListener, + tempFileManager: DownloadFileManager): Unit = { + // Notify BlockFetchingListener with a bad ManagedBuffer asynchronously + new Thread() { + override def run(): Unit = { + // This is a bad buffer to trigger `IllegalArgumentException` in + // `BlockFetchingListener.onBlockFetchSuccess`. The real issue we hit is + // `ByteBuffer.allocate` throws `OutOfMemoryError`, but we cannot make it happen in + // a test. Instead, we use a negative size value to make `ByteBuffer.allocate` fail, + // and this should trigger the same code path as `OutOfMemoryError`. + val badBuffer = new ManagedBuffer { + override def size(): Long = -1 + + override def nioByteBuffer(): ByteBuffer = null + + override def createInputStream(): InputStream = null + + override def retain(): ManagedBuffer = this + + override def release(): ManagedBuffer = this + + override def convertToNetty(): AnyRef = null + } + listener.onBlockFetchSuccess("block-id-unused", badBuffer) + } + }.start() + } + + override def uploadBlock( + hostname: String, + port: Int, + execId: String, + blockId: BlockId, + blockData: ManagedBuffer, + level: StorageLevel, + classTag: ClassTag[_]): Future[Unit] = { + // This method is unused in this test + throw new UnsupportedOperationException("uploadBlock") + } + } + + val e = intercept[SparkException] { + failAfter(10.seconds) { + blockTransferService.fetchBlockSync( + "localhost-unused", 0, "exec-id-unused", "block-id-unused", null) + } + } + assert(e.getCause.isInstanceOf[IllegalArgumentException]) + } +} From 10d7713f1259d4ba49020bb880ded5218ece55fb Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 22 Jan 2019 16:46:00 -0800 Subject: [PATCH 338/879] [SPARK-26605][YARN] Update AM's credentials when creating tokens. This ensures new executors in client mode also get the new tokens, instead of being started with potentially expired tokens. Closes #23523 from vanzin/SPARK-26605. Authored-by: Marcelo Vanzin Signed-off-by: Marcelo Vanzin --- .../deploy/yarn/security/AMCredentialRenewer.scala | 14 +++++++++++--- 1 file changed, 11 insertions(+), 3 deletions(-) diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala index bc8d47dbd54c..51ef7d3a6036 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala @@ -86,6 +86,16 @@ private[yarn] class AMCredentialRenewer( val originalCreds = UserGroupInformation.getCurrentUser().getCredentials() val ugi = doLogin() + ugi.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + startInternal(ugi, originalCreds) + } + }) + + ugi + } + + private def startInternal(ugi: UserGroupInformation, originalCreds: Credentials): Unit = { val tgtRenewalTask = new Runnable() { override def run(): Unit = { ugi.checkTGTAndReloginFromKeytab() @@ -104,8 +114,6 @@ private[yarn] class AMCredentialRenewer( val existing = ugi.getCredentials() existing.mergeAll(originalCreds) ugi.addCredentials(existing) - - ugi } def stop(): Unit = { @@ -136,8 +144,8 @@ private[yarn] class AMCredentialRenewer( // This shouldn't really happen, since the driver should register way before tokens expire // (or the AM should time out the application). logWarning("Delegation tokens close to expiration but no driver has registered yet.") - SparkHadoopUtil.get.addDelegationTokens(tokens, sparkConf) } + SparkHadoopUtil.get.addDelegationTokens(tokens, sparkConf) } catch { case e: Exception => val delay = TimeUnit.SECONDS.toMillis(sparkConf.get(CREDENTIALS_RENEWAL_RETRY_WAIT)) From f36d0c56c256164f229b900778f593a0d8e4c7fc Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 22 Jan 2019 19:22:06 -0600 Subject: [PATCH 339/879] [SPARK-26228][MLLIB] OOM issue encountered when computing Gramian matrix Avoid memory problems in closure cleaning when handling large Gramians (>= 16K rows/cols) by using null as zeroValue Existing tests. Note that it's hard to test the case that triggers this issue as it would require a large amount of memory and run a while. I confirmed locally that a 16K x 16K Gramian failed with tons of driver memory before, and didn't fail upfront after this change. Closes #23600 from srowen/SPARK-26228. Authored-by: Sean Owen Signed-off-by: Sean Owen (cherry picked from commit 6dcad38ba3393188084f378b7ff6dfc12b685b13) Signed-off-by: Sean Owen --- .../mllib/linalg/distributed/RowMatrix.scala | 20 ++++++++++++++++--- 1 file changed, 17 insertions(+), 3 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 78a8810052ae..5109efb1e39e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -119,11 +119,25 @@ class RowMatrix @Since("1.0.0") ( val nt = if (n % 2 == 0) ((n / 2) * (n + 1)) else (n * ((n + 1) / 2)) // Compute the upper triangular part of the gram matrix. - val GU = rows.treeAggregate(new BDV[Double](nt))( - seqOp = (U, v) => { + val GU = rows.treeAggregate(null.asInstanceOf[BDV[Double]])( + seqOp = (maybeU, v) => { + val U = + if (maybeU == null) { + new BDV[Double](nt) + } else { + maybeU + } BLAS.spr(1.0, v, U.data) U - }, combOp = (U1, U2) => U1 += U2) + }, combOp = (U1, U2) => + if (U1 == null) { + U2 + } else if (U2 == null) { + U1 + } else { + U1 += U2 + } + ) RowMatrix.triuToFull(n, GU.data) } From 921c22b1fffc4844aa05c201ba15986be34a3782 Mon Sep 17 00:00:00 2001 From: Anton Okolnychyi Date: Thu, 24 Jan 2019 00:12:26 +0000 Subject: [PATCH 340/879] [SPARK-26706][SQL] Fix `illegalNumericPrecedence` for ByteType This PR contains a minor change in `Cast$mayTruncate` that fixes its logic for bytes. Right now, `mayTruncate(ByteType, LongType)` returns `false` while `mayTruncate(ShortType, LongType)` returns `true`. Consequently, `spark.range(1, 3).as[Byte]` and `spark.range(1, 3).as[Short]` behave differently. Potentially, this bug can silently corrupt someone's data. ```scala // executes silently even though Long is converted into Byte spark.range(Long.MaxValue - 10, Long.MaxValue).as[Byte] .map(b => b - 1) .show() +-----+ |value| +-----+ | -12| | -11| | -10| | -9| | -8| | -7| | -6| | -5| | -4| | -3| +-----+ // throws an AnalysisException: Cannot up cast `id` from bigint to smallint as it may truncate spark.range(Long.MaxValue - 10, Long.MaxValue).as[Short] .map(s => s - 1) .show() ``` This PR comes with a set of unit tests. Closes #23632 from aokolnychyi/cast-fix. Authored-by: Anton Okolnychyi Signed-off-by: DB Tsai --- .../spark/sql/catalyst/expressions/Cast.scala | 2 +- .../sql/catalyst/expressions/CastSuite.scala | 36 +++++++++++++++++++ .../org/apache/spark/sql/DatasetSuite.scala | 9 +++++ 3 files changed, 46 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index ee463bf5eb6a..ac02dac273b1 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 @@ -131,7 +131,7 @@ object Cast { private def illegalNumericPrecedence(from: DataType, to: DataType): Boolean = { val fromPrecedence = TypeCoercion.numericPrecedence.indexOf(from) val toPrecedence = TypeCoercion.numericPrecedence.indexOf(to) - toPrecedence > 0 && fromPrecedence > toPrecedence + toPrecedence >= 0 && fromPrecedence > toPrecedence } /** 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 d9f32c000a88..b1531ba70593 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 @@ -23,6 +23,7 @@ import java.util.{Calendar, Locale, TimeZone} import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCoercion.numericPrecedence import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -953,4 +954,39 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { val ret6 = cast(Literal.create((1, Map(1 -> "a", 2 -> "b", 3 -> "c"))), StringType) checkEvaluation(ret6, "[1, [1 -> a, 2 -> b, 3 -> c]]") } + + test("SPARK-26706: Fix Cast.mayTruncate for bytes") { + assert(!Cast.mayTruncate(ByteType, ByteType)) + assert(!Cast.mayTruncate(DecimalType.ByteDecimal, ByteType)) + assert(Cast.mayTruncate(ShortType, ByteType)) + assert(Cast.mayTruncate(IntegerType, ByteType)) + assert(Cast.mayTruncate(LongType, ByteType)) + assert(Cast.mayTruncate(FloatType, ByteType)) + assert(Cast.mayTruncate(DoubleType, ByteType)) + assert(Cast.mayTruncate(DecimalType.IntDecimal, ByteType)) + } + + test("canSafeCast and mayTruncate must be consistent for numeric types") { + import DataTypeTestUtils._ + + def isCastSafe(from: NumericType, to: NumericType): Boolean = (from, to) match { + case (_, dt: DecimalType) => dt.isWiderThan(from) + case (dt: DecimalType, _) => dt.isTighterThan(to) + case _ => numericPrecedence.indexOf(from) <= numericPrecedence.indexOf(to) + } + + numericTypes.foreach { from => + val (safeTargetTypes, unsafeTargetTypes) = numericTypes.partition(to => isCastSafe(from, to)) + + safeTargetTypes.foreach { to => + assert(Cast.canSafeCast(from, to), s"It should be possible to safely cast $from to $to") + assert(!Cast.mayTruncate(from, to), s"No truncation is expected when casting $from to $to") + } + + unsafeTargetTypes.foreach { to => + assert(!Cast.canSafeCast(from, to), s"It shouldn't be possible to safely cast $from to $to") + assert(Cast.mayTruncate(from, to), s"Truncation is expected when casting $from to $to") + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala index 50406bcdddf6..01d0877c8a88 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala @@ -1567,6 +1567,15 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val exceptDF = inputDF.filter(col("a").isin("0") or col("b") > "c") checkAnswer(inputDF.except(exceptDF), Seq(Row("1", null))) } + + test("SPARK-26706: Fix Cast.mayTruncate for bytes") { + val thrownException = intercept[AnalysisException] { + spark.range(Long.MaxValue - 10, Long.MaxValue).as[Byte] + .map(b => b - 1) + .collect() + } + assert(thrownException.message.contains("Cannot up cast `id` from bigint to tinyint")) + } } case class TestDataUnion(x: Int, y: Int, z: Int) From 63fa6f5abc0c529d017243a4eea505c1c4cbbbd4 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Thu, 24 Jan 2019 12:45:25 +0800 Subject: [PATCH 341/879] [SPARK-26682][SQL] Use taskAttemptID instead of attemptNumber for Hadoop. ## What changes were proposed in this pull request? Updates the attempt ID used by FileFormatWriter. Tasks in stage attempts use the same task attempt number and could conflict. Using Spark's task attempt ID guarantees that Hadoop TaskAttemptID instances are unique. ## How was this patch tested? Existing tests. Also validated that we no longer detect this failure case in our logs after deployment. Closes #23608 from rdblue/SPARK-26682-fix-hadoop-task-attempt-id. Authored-by: Ryan Blue Signed-off-by: Wenchen Fan (cherry picked from commit d5a97c1c2c86ae335e91008fa25b3359c4560915) Signed-off-by: Wenchen Fan --- .../spark/sql/execution/datasources/FileFormatWriter.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 774fe38f5c2e..2103a2d7e3d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -170,7 +170,7 @@ object FileFormatWriter extends Logging { description = description, sparkStageId = taskContext.stageId(), sparkPartitionId = taskContext.partitionId(), - sparkAttemptNumber = taskContext.attemptNumber(), + sparkAttemptNumber = taskContext.taskAttemptId().toInt & Integer.MAX_VALUE, committer, iterator = iter) }, From e8e9b110c7076139f551797ef7b74c7953b66f24 Mon Sep 17 00:00:00 2001 From: Bruce Robbins Date: Thu, 24 Jan 2019 11:18:08 +0100 Subject: [PATCH 342/879] [SPARK-26680][SQL] Eagerly create inputVars while conditions are appropriate ## What changes were proposed in this pull request? When a user passes a Stream to groupBy, ```CodegenSupport.consume``` ends up lazily generating ```inputVars``` from a Stream, since the field ```output``` will be a Stream. At the time ```output.zipWithIndex.map``` is called, conditions are correct. However, by the time the map operation actually executes, conditions are no longer appropriate. The closure used by the map operation ends up using a reference to the partially created ```inputVars```. As a result, a StackOverflowError occurs. This PR ensures that ```inputVars``` is eagerly created while conditions are appropriate. It seems this was also an issue with the code path for creating ```inputVars``` from ```outputVars``` (SPARK-25767). I simply extended the solution for that code path to encompass both code paths. ## How was this patch tested? SQL unit tests new test python tests Closes #23617 from bersprockets/SPARK-26680_opt1. Authored-by: Bruce Robbins Signed-off-by: Herman van Hovell (cherry picked from commit d4a30fa9af81a8bbb50d75f495ca3787f68f10e4) Signed-off-by: Herman van Hovell --- .../spark/sql/execution/WholeStageCodegenExec.scala | 12 +++++++----- .../spark/sql/execution/WholeStageCodegenSuite.scala | 9 +++++++++ 2 files changed, 16 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index ded8dd30dc49..7b01f619a461 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -142,14 +142,11 @@ trait CodegenSupport extends SparkPlan { * Note that `outputVars` and `row` can't both be null. */ final def consume(ctx: CodegenContext, outputVars: Seq[ExprCode], row: String = null): String = { - val inputVars = + val inputVarsCandidate = if (outputVars != null) { assert(outputVars.length == output.length) // outputVars will be used to generate the code for UnsafeRow, so we should copy them - outputVars.map(_.copy()) match { - case stream: Stream[ExprCode] => stream.force - case other => other - } + outputVars.map(_.copy()) } else { assert(row != null, "outputVars and row cannot both be null.") ctx.currentVars = null @@ -159,6 +156,11 @@ trait CodegenSupport extends SparkPlan { } } + val inputVars = inputVarsCandidate match { + case stream: Stream[ExprCode] => stream.force + case other => other + } + val rowVar = prepareRowVar(ctx, row, outputVars) // Set up the `currentVars` in the codegen context, as we generate the code of `inputVars` diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index 09ad0fdd6636..e03f08417162 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -330,4 +330,13 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { checkAnswer(abc, Row(1, "a")) } + + test("SPARK-26680: Stream in groupBy does not cause StackOverflowError") { + val groupByCols = Stream(col("key")) + val df = Seq((1, 2), (2, 3), (1, 3)).toDF("key", "value") + .groupBy(groupByCols: _*) + .max("value") + + checkAnswer(df, Seq(Row(1, 3), Row(2, 3))) + } } From 8d957d7724d36ce415029d454740352699bcc862 Mon Sep 17 00:00:00 2001 From: Gengliang Wang Date: Thu, 24 Jan 2019 18:24:49 -0800 Subject: [PATCH 343/879] [SPARK-26709][SQL] OptimizeMetadataOnlyQuery does not handle empty records correctly ## What changes were proposed in this pull request? When reading from empty tables, the optimization `OptimizeMetadataOnlyQuery` may return wrong results: ``` sql("CREATE TABLE t (col1 INT, p1 INT) USING PARQUET PARTITIONED BY (p1)") sql("INSERT INTO TABLE t PARTITION (p1 = 5) SELECT ID FROM range(1, 1)") sql("SELECT MAX(p1) FROM t") ``` The result is supposed to be `null`. However, with the optimization the result is `5`. The rule is originally ported from https://issues.apache.org/jira/browse/HIVE-1003 in #13494. In Hive, the rule is disabled by default in a later release(https://issues.apache.org/jira/browse/HIVE-15397), due to the same problem. It is hard to completely avoid the correctness issue. Because data sources like Parquet can be metadata-only. Spark can't tell whether it is empty or not without actually reading it. This PR disable the optimization by default. ## How was this patch tested? Unit test Closes #23635 from gengliangwang/optimizeMetadata. Lead-authored-by: Gengliang Wang Co-authored-by: Xiao Li Signed-off-by: gatorsmile (cherry picked from commit f5b9370da2745a744f8b2f077f1690e0e7035140) Signed-off-by: gatorsmile --- docs/sql-data-sources-parquet.md | 12 ------ .../apache/spark/sql/internal/SQLConf.scala | 6 ++- .../execution/OptimizeMetadataOnlyQuery.scala | 5 +++ .../org/apache/spark/sql/SQLQuerySuite.scala | 37 +++++++++++++++++++ .../sql/hive/execution/SQLQuerySuite.scala | 18 +++++++++ 5 files changed, 64 insertions(+), 14 deletions(-) diff --git a/docs/sql-data-sources-parquet.md b/docs/sql-data-sources-parquet.md index 4fed3eaf83e5..71fa57d9d809 100644 --- a/docs/sql-data-sources-parquet.md +++ b/docs/sql-data-sources-parquet.md @@ -295,18 +295,6 @@ Configuration of Parquet can be done using the `setConf` method on `SparkSession

    - - spark.sql.optimizer.metadataOnly - true - -

    - When true, enable the metadata-only query optimization that use the table's metadata to - produce the partition columns instead of table scans. It applies when all the columns scanned - are partition columns and the query has an aggregate operator that satisfies distinct - semantics. -

    - - spark.sql.parquet.writeLegacyFormat false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala index 3dad1e34af23..29bd3566fbfa 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala @@ -562,12 +562,14 @@ object SQLConf { .createWithDefault(HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString) val OPTIMIZER_METADATA_ONLY = buildConf("spark.sql.optimizer.metadataOnly") + .internal() .doc("When true, enable the metadata-only query optimization that use the table's metadata " + "to produce the partition columns instead of table scans. It applies when all the columns " + "scanned are partition columns and the query has an aggregate operator that satisfies " + - "distinct semantics.") + "distinct semantics. By default the optimization is disabled, since it may return " + + "incorrect results when the files are empty.") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val COLUMN_NAME_OF_CORRUPT_RECORD = buildConf("spark.sql.columnNameOfCorruptRecord") .doc("The name of internal column for storing raw/un-parsed JSON and CSV records that fail " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala index 3ca03ab2939a..45e5f415e8da 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala @@ -72,6 +72,11 @@ case class OptimizeMetadataOnlyQuery(catalog: SessionCatalog) extends Rule[Logic }) } if (isAllDistinctAgg) { + logWarning("Since configuration `spark.sql.optimizer.metadataOnly` is enabled, " + + "Spark will scan partition-level metadata without scanning data files. " + + "This could result in wrong results when the partition metadata exists but the " + + "inclusive data files are empty." + ) a.withNewChildren(Seq(replaceTableScanWithPartitionMetadata(child, rel, filters))) } else { a 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 806f0b2239fe..b8c4d73f1b2b 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 @@ -2966,6 +2966,43 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } } + + test("SPARK-26709: OptimizeMetadataOnlyQuery does not handle empty records correctly") { + Seq(true, false).foreach { enableOptimizeMetadataOnlyQuery => + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> enableOptimizeMetadataOnlyQuery.toString) { + withTable("t") { + sql("CREATE TABLE t (col1 INT, p1 INT) USING PARQUET PARTITIONED BY (p1)") + sql("INSERT INTO TABLE t PARTITION (p1 = 5) SELECT ID FROM range(1, 1)") + if (enableOptimizeMetadataOnlyQuery) { + // The result is wrong if we enable the configuration. + checkAnswer(sql("SELECT MAX(p1) FROM t"), Row(5)) + } else { + checkAnswer(sql("SELECT MAX(p1) FROM t"), Row(null)) + } + checkAnswer(sql("SELECT MAX(col1) FROM t"), Row(null)) + } + + withTempPath { path => + val tabLocation = path.getCanonicalPath + val partLocation1 = tabLocation + "/p=3" + val partLocation2 = tabLocation + "/p=1" + // SPARK-23271 empty RDD when saved should write a metadata only file + val df = spark.emptyDataFrame.select(lit(1).as("col")) + df.write.parquet(partLocation1) + val df2 = spark.range(10).toDF("col") + df2.write.parquet(partLocation2) + val readDF = spark.read.parquet(tabLocation) + if (enableOptimizeMetadataOnlyQuery) { + // The result is wrong if we enable the configuration. + checkAnswer(readDF.selectExpr("max(p)"), Row(3)) + } else { + checkAnswer(readDF.selectExpr("max(p)"), Row(1)) + } + checkAnswer(readDF.selectExpr("max(col)"), Row(9)) + } + } + } + } } case class Foo(bar: Option[String]) 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 fab2a27cdef1..2e5e5c069db0 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 @@ -2290,4 +2290,22 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("SPARK-26709: OptimizeMetadataOnlyQuery does not handle empty records correctly") { + Seq(true, false).foreach { enableOptimizeMetadataOnlyQuery => + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> enableOptimizeMetadataOnlyQuery.toString) { + withTable("t") { + sql("CREATE TABLE t (col1 INT, p1 INT) USING PARQUET PARTITIONED BY (p1)") + sql("INSERT INTO TABLE t PARTITION (p1 = 5) SELECT ID FROM range(1, 1)") + if (enableOptimizeMetadataOnlyQuery) { + // The result is wrong if we enable the configuration. + checkAnswer(sql("SELECT MAX(p1) FROM t"), Row(5)) + } else { + checkAnswer(sql("SELECT MAX(p1) FROM t"), Row(null)) + } + checkAnswer(sql("SELECT MAX(col1) FROM t"), Row(null)) + } + } + } + } + } From 08b63790a634d5e5c32584978906bc999fd40f61 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 25 Jan 2019 09:08:00 -0800 Subject: [PATCH 344/879] [SPARK-26427][BUILD][BACKPORT-2.4] Upgrade Apache ORC to 1.5.4 ## What changes were proposed in this pull request? This is a backport of #23364. To make Apache Spark 2.4.1 more robust, this PR aims to update Apache ORC dependency to the latest version 1.5.4 released at Dec. 20. ([Release Notes](https://issues.apache.org/jira/secure/ReleaseNote.jspa?projectId=12318320&version=12344187])) ``` [ORC-237] OrcFile.mergeFiles Specified block size is less than configured minimum value [ORC-409] Changes for extending MemoryManagerImpl [ORC-410] Fix a locale-dependent test in TestCsvReader [ORC-416] Avoid opening data reader when there is no stripe [ORC-417] Use dynamic Apache Maven mirror link [ORC-419] Ensure to call `close` at RecordReaderImpl constructor exception [ORC-432] openjdk 8 has a bug that prevents surefire from working [ORC-435] Ability to read stripes that are greater than 2GB [ORC-437] Make acid schema checks case insensitive [ORC-411] Update build to work with Java 10. [ORC-418] Fix broken docker build script ``` ## How was this patch tested? Build and pass Jenkins. Closes #23646 from dongjoon-hyun/SPARK-26427-2.4. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.6 | 6 +++--- dev/deps/spark-deps-hadoop-2.7 | 6 +++--- dev/deps/spark-deps-hadoop-3.1 | 6 +++--- pom.xml | 6 +++++- 4 files changed, 14 insertions(+), 10 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index 62ae04dbc255..acbfac97ba27 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -153,9 +153,9 @@ objenesis-2.5.1.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.5.2-nohive.jar -orc-mapreduce-1.5.2-nohive.jar -orc-shims-1.5.2.jar +orc-core-1.5.4-nohive.jar +orc-mapreduce-1.5.4-nohive.jar +orc-shims-1.5.4.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index dcb5d63aeff4..365a59b4ed03 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -154,9 +154,9 @@ objenesis-2.5.1.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.5.2-nohive.jar -orc-mapreduce-1.5.2-nohive.jar -orc-shims-1.5.2.jar +orc-core-1.5.4-nohive.jar +orc-mapreduce-1.5.4-nohive.jar +orc-shims-1.5.4.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index 641b4a15ad7c..524e370f854b 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -172,9 +172,9 @@ okhttp-2.7.5.jar okhttp-3.8.1.jar okio-1.13.0.jar opencsv-2.3.jar -orc-core-1.5.2-nohive.jar -orc-mapreduce-1.5.2-nohive.jar -orc-shims-1.5.2.jar +orc-core-1.5.4-nohive.jar +orc-mapreduce-1.5.4-nohive.jar +orc-shims-1.5.4.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar diff --git a/pom.xml b/pom.xml index f802786d1cf3..91be9d914b1d 100644 --- a/pom.xml +++ b/pom.xml @@ -131,7 +131,7 @@ 1.2.1 10.12.1.1 1.10.0 - 1.5.2 + 1.5.4 nohive 1.6.0 9.3.24.v20180605 @@ -1733,6 +1733,10 @@ ${orc.classifier} ${orc.deps.scope} + + javax.xml.bind + jaxb-api + org.apache.hadoop hadoop-common From 46a9018b3b29c36f33e4113984a7f43f91ac12fc Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Fri, 25 Jan 2019 14:58:03 -0800 Subject: [PATCH 345/879] [SPARK-26379][SS] Fix issue on adding current_timestamp/current_date to streaming query ## What changes were proposed in this pull request? This patch proposes to fix issue on adding `current_timestamp` / `current_date` with streaming query. The root reason is that Spark transforms `CurrentTimestamp`/`CurrentDate` to `CurrentBatchTimestamp` in MicroBatchExecution which makes transformed attributes not-yet-resolved. They will be resolved by IncrementalExecution. (In ContinuousExecution, Spark doesn't allow using `current_timestamp` and `current_date` so it has been OK.) It's OK for DataSource V1 sink because it simply leverages transformed logical plan and don't evaluate until they're resolved, but for DataSource V2 sink, Spark tries to extract the schema of transformed logical plan in prior to IncrementalExecution, and unresolved attributes will raise errors. This patch fixes the issue via having separate pre-resolved logical plan to pass the schema to StreamingWriteSupport safely. ## How was this patch tested? Added UT. Closes #23609 from HeartSaVioR/SPARK-26379. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../streaming/MicroBatchExecution.scala | 10 +++- .../spark/sql/streaming/StreamSuite.scala | 46 +++++++++++++++++++ 2 files changed, 55 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index 7a007b6eb34d..e58182cad395 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -496,12 +496,20 @@ class MicroBatchExecution( cd.dataType, cd.timeZoneId) } + // Pre-resolve new attributes to ensure all attributes are resolved before + // accessing schema of logical plan. Note that it only leverages the information + // of attributes, so we don't need to concern about the value of literals. + + val newAttrPlanPreResolvedForSchema = newAttributePlan transformAllExpressions { + case cbt: CurrentBatchTimestamp => cbt.toLiteral + } + val triggerLogicalPlan = sink match { case _: Sink => newAttributePlan case s: StreamWriteSupport => val writer = s.createStreamWriter( s"$runId", - newAttributePlan.schema, + newAttrPlanPreResolvedForSchema.schema, outputMode, new DataSourceOptions(extraOptions.asJava)) WriteToDataSourceV2(new MicroBatchWriter(currentBatchId, writer), newAttributePlan) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index f55ddb5419d2..766bee64dcbe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -34,6 +34,7 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.Range import org.apache.spark.sql.catalyst.streaming.InternalOutputModes +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution @@ -1082,6 +1083,51 @@ class StreamSuite extends StreamTest { assert(query.exception.isEmpty) } } + + test("SPARK-26379 Structured Streaming - Exception on adding current_timestamp / current_date" + + " to Dataset - use v2 sink") { + testCurrentTimestampOnStreamingQuery(useV2Sink = true) + } + + test("SPARK-26379 Structured Streaming - Exception on adding current_timestamp / current_date" + + " to Dataset - use v1 sink") { + testCurrentTimestampOnStreamingQuery(useV2Sink = false) + } + + private def testCurrentTimestampOnStreamingQuery(useV2Sink: Boolean): Unit = { + val input = MemoryStream[Int] + val df = input.toDS() + .withColumn("cur_timestamp", lit(current_timestamp())) + .withColumn("cur_date", lit(current_date())) + + def assertBatchOutputAndUpdateLastTimestamp( + rows: Seq[Row], + curTimestamp: Long, + curDate: Int, + expectedValue: Int): Long = { + assert(rows.size === 1) + val row = rows.head + assert(row.getInt(0) === expectedValue) + assert(row.getTimestamp(1).getTime >= curTimestamp) + val days = DateTimeUtils.millisToDays(row.getDate(2).getTime) + assert(days == curDate || days == curDate + 1) + row.getTimestamp(1).getTime + } + + var lastTimestamp = System.currentTimeMillis() + val currentDate = DateTimeUtils.millisToDays(lastTimestamp) + testStream(df, useV2Sink = useV2Sink) ( + AddData(input, 1), + CheckLastBatch { rows: Seq[Row] => + lastTimestamp = assertBatchOutputAndUpdateLastTimestamp(rows, lastTimestamp, currentDate, 1) + }, + Execute { _ => Thread.sleep(1000) }, + AddData(input, 2), + CheckLastBatch { rows: Seq[Row] => + lastTimestamp = assertBatchOutputAndUpdateLastTimestamp(rows, lastTimestamp, currentDate, 2) + } + ) + } } abstract class FakeSource extends StreamSourceProvider { From 448a06375fcdae04a17006bdb2d02a2b5fccaca7 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Sun, 27 Jan 2019 10:04:51 -0800 Subject: [PATCH 346/879] [SPARK-26379][SS][FOLLOWUP] Use dummy TimeZoneId to avoid UnresolvedException in CurrentBatchTimestamp ## What changes were proposed in this pull request? Spark replaces `CurrentTimestamp` with `CurrentBatchTimestamp`. However, `CurrentBatchTimestamp` is `TimeZoneAwareExpression` while `CurrentTimestamp` isn't. Without TimeZoneId, `CurrentBatchTimestamp` becomes unresolved and raises `UnresolvedException`. Since `CurrentDate` is `TimeZoneAwareExpression`, there is no problem with `CurrentDate`. This PR reverts the [previous patch](https://github.com/apache/spark/pull/23609) on `MicroBatchExecution` and fixes the root cause. ## How was this patch tested? Pass the Jenkins with the updated test cases. Closes #23660 from dongjoon-hyun/SPARK-26379. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun (cherry picked from commit 1ca6b8bc3df19503c00414e62161227725a99520) Signed-off-by: Dongjoon Hyun --- .../execution/streaming/MicroBatchExecution.scala | 15 +++++---------- .../apache/spark/sql/streaming/StreamSuite.scala | 10 +++------- 2 files changed, 8 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala index e58182cad395..3bcc26b21002 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MicroBatchExecution.scala @@ -489,27 +489,22 @@ class MicroBatchExecution( // Rewire the plan to use the new attributes that were returned by the source. val newAttributePlan = newBatchesPlan transformAllExpressions { case ct: CurrentTimestamp => + // CurrentTimestamp is not TimeZoneAwareExpression while CurrentBatchTimestamp is. + // Without TimeZoneId, CurrentBatchTimestamp is unresolved. Here, we use an explicit + // dummy string to prevent UnresolvedException and to prevent to be used in the future. CurrentBatchTimestamp(offsetSeqMetadata.batchTimestampMs, - ct.dataType) + ct.dataType, Some("Dummy TimeZoneId")) case cd: CurrentDate => CurrentBatchTimestamp(offsetSeqMetadata.batchTimestampMs, cd.dataType, cd.timeZoneId) } - // Pre-resolve new attributes to ensure all attributes are resolved before - // accessing schema of logical plan. Note that it only leverages the information - // of attributes, so we don't need to concern about the value of literals. - - val newAttrPlanPreResolvedForSchema = newAttributePlan transformAllExpressions { - case cbt: CurrentBatchTimestamp => cbt.toLiteral - } - val triggerLogicalPlan = sink match { case _: Sink => newAttributePlan case s: StreamWriteSupport => val writer = s.createStreamWriter( s"$runId", - newAttrPlanPreResolvedForSchema.schema, + newAttributePlan.schema, outputMode, new DataSourceOptions(extraOptions.asJava)) WriteToDataSourceV2(new MicroBatchWriter(currentBatchId, writer), newAttributePlan) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala index 766bee64dcbe..ae3689a0cac7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamSuite.scala @@ -1084,21 +1084,19 @@ class StreamSuite extends StreamTest { } } - test("SPARK-26379 Structured Streaming - Exception on adding current_timestamp / current_date" + + test("SPARK-26379 Structured Streaming - Exception on adding current_timestamp " + " to Dataset - use v2 sink") { testCurrentTimestampOnStreamingQuery(useV2Sink = true) } - test("SPARK-26379 Structured Streaming - Exception on adding current_timestamp / current_date" + + test("SPARK-26379 Structured Streaming - Exception on adding current_timestamp " + " to Dataset - use v1 sink") { testCurrentTimestampOnStreamingQuery(useV2Sink = false) } private def testCurrentTimestampOnStreamingQuery(useV2Sink: Boolean): Unit = { val input = MemoryStream[Int] - val df = input.toDS() - .withColumn("cur_timestamp", lit(current_timestamp())) - .withColumn("cur_date", lit(current_date())) + val df = input.toDS().withColumn("cur_timestamp", lit(current_timestamp())) def assertBatchOutputAndUpdateLastTimestamp( rows: Seq[Row], @@ -1109,8 +1107,6 @@ class StreamSuite extends StreamTest { val row = rows.head assert(row.getInt(0) === expectedValue) assert(row.getTimestamp(1).getTime >= curTimestamp) - val days = DateTimeUtils.millisToDays(row.getDate(2).getTime) - assert(days == curDate || days == curDate + 1) row.getTimestamp(1).getTime } From d5cc8909c72e958ce187df9c75847ad0125991ab Mon Sep 17 00:00:00 2001 From: maryannxue Date: Tue, 29 Jan 2019 21:33:46 +0900 Subject: [PATCH 347/879] [SPARK-26708][SQL][BRANCH-2.4] Incorrect result caused by inconsistency between a SQL cache's cached RDD and its physical plan ## What changes were proposed in this pull request? When performing non-cascading cache invalidation, `recache` is called on the other cache entries which are dependent on the cache being invalidated. It leads to the the physical plans of those cache entries being re-compiled. For those cache entries, if the cache RDD has already been persisted, chances are there will be inconsistency between the data and the new plan. It can cause a correctness issue if the new plan's `outputPartitioning` or `outputOrdering` is different from the that of the actual data, and meanwhile the cache is used by another query that asks for specific `outputPartitioning` or `outputOrdering` which happens to match the new plan but not the actual data. The fix is to keep the cache entry as it is if the data has been loaded, otherwise re-build the cache entry, with a new plan and an empty cache buffer. ## How was this patch tested? Added UT. Closes #23678 from maryannxue/spark-26708-2.4. Authored-by: maryannxue Signed-off-by: Takeshi Yamamuro --- .../spark/sql/execution/CacheManager.scala | 28 +++++++++--- .../execution/columnar/InMemoryRelation.scala | 10 +---- .../apache/spark/sql/DatasetCacheSuite.scala | 44 ++++++++++++++++++- 3 files changed, 67 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index c9929935fb8a..5b3059652922 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -166,16 +166,34 @@ class CacheManager extends Logging { val needToRecache = scala.collection.mutable.ArrayBuffer.empty[CachedData] while (it.hasNext) { val cd = it.next() - if (condition(cd.plan)) { - if (clearCache) { - cd.cachedRepresentation.cacheBuilder.clearCache() - } + // If `clearCache` is false (which means the recache request comes from a non-cascading + // cache invalidation) and the cache buffer has already been loaded, we do not need to + // re-compile a physical plan because the old plan will not be used any more by the + // CacheManager although it still lives in compiled `Dataset`s and it could still work. + // Otherwise, it means either `clearCache` is true, then we have to clear the cache buffer + // and re-compile the physical plan; or it is a non-cascading cache invalidation and cache + // buffer is still empty, then we could have a more efficient new plan by removing + // dependency on the previously removed cache entries. + // Note that the `CachedRDDBuilder`.`isCachedColumnBuffersLoaded` call is a non-locking + // status test and may not return the most accurate cache buffer state. So the worse case + // scenario can be: + // 1) The buffer has been loaded, but `isCachedColumnBuffersLoaded` returns false, then we + // will clear the buffer and build a new plan. It is inefficient but doesn't affect + // correctness. + // 2) The buffer has been cleared, but `isCachedColumnBuffersLoaded` returns true, then we + // will keep it as it is. It means the physical plan has been re-compiled already in the + // other thread. + val buildNewPlan = + clearCache || !cd.cachedRepresentation.cacheBuilder.isCachedColumnBuffersLoaded + if (condition(cd.plan) && buildNewPlan) { + cd.cachedRepresentation.cacheBuilder.clearCache() // Remove the cache entry before we create a new one, so that we can have a different // physical plan. it.remove() val plan = spark.sessionState.executePlan(cd.plan).executedPlan val newCache = InMemoryRelation( - cacheBuilder = cd.cachedRepresentation.cacheBuilder.withCachedPlan(plan), + cacheBuilder = cd.cachedRepresentation + .cacheBuilder.copy(cachedPlan = plan)(_cachedColumnBuffers = null), logicalPlan = cd.plan) needToRecache += cd.copy(cachedRepresentation = newCache) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index b752b771f43a..8eecd7a80e0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -74,14 +74,8 @@ case class CachedRDDBuilder( } } - def withCachedPlan(cachedPlan: SparkPlan): CachedRDDBuilder = { - new CachedRDDBuilder( - useCompression, - batchSize, - storageLevel, - cachedPlan = cachedPlan, - tableName - )(_cachedColumnBuffers) + def isCachedColumnBuffersLoaded: Boolean = { + _cachedColumnBuffers != null } private def buildBuffers(): RDD[CachedBatch] = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index 5c6a021d5b76..7c97f5c32b16 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -190,9 +190,9 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits df1.unpersist(blocking = true) - // df1 un-cached; df2's cache plan re-compiled + // df1 un-cached; df2's cache plan stays the same assert(df1.storageLevel == StorageLevel.NONE) - assertCacheDependency(df1.groupBy('a).agg(sum('b)), 0) + assertCacheDependency(df1.groupBy('a).agg(sum('b))) val df4 = df1.groupBy('a).agg(sum('b)).agg(sum("sum(b)")) assertCached(df4) @@ -206,4 +206,44 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits // first time use, load cache checkDataset(df5, Row(10)) } + + test("SPARK-26708 Cache data and cached plan should stay consistent") { + val df = spark.range(0, 5).toDF("a") + val df1 = df.withColumn("b", 'a + 1) + val df2 = df.filter('a > 1) + + df.cache() + // Add df1 to the CacheManager; the buffer is currently empty. + df1.cache() + // After calling collect(), df1's buffer has been loaded. + df1.collect() + // Add df2 to the CacheManager; the buffer is currently empty. + df2.cache() + + // Verify that df1 is a InMemoryRelation plan with dependency on another cached plan. + assertCacheDependency(df1) + val df1InnerPlan = df1.queryExecution.withCachedData + .asInstanceOf[InMemoryRelation].cacheBuilder.cachedPlan + // Verify that df2 is a InMemoryRelation plan with dependency on another cached plan. + assertCacheDependency(df2) + + df.unpersist(blocking = true) + + // Verify that df1's cache has stayed the same, since df1's cache already has data + // before df.unpersist(). + val df1Limit = df1.limit(2) + val df1LimitInnerPlan = df1Limit.queryExecution.withCachedData.collectFirst { + case i: InMemoryRelation => i.cacheBuilder.cachedPlan + } + assert(df1LimitInnerPlan.isDefined && df1LimitInnerPlan.get == df1InnerPlan) + + // Verify that df2's cache has been re-cached, with a new physical plan rid of dependency + // on df, since df2's cache had not been loaded before df.unpersist(). + val df2Limit = df2.limit(2) + val df2LimitInnerPlan = df2Limit.queryExecution.withCachedData.collectFirst { + case i: InMemoryRelation => i.cacheBuilder.cachedPlan + } + assert(df2LimitInnerPlan.isDefined && + df2LimitInnerPlan.get.find(_.isInstanceOf[InMemoryTableScanExec]).isEmpty) + } } From ae0592ddf7009934e9a5ee05a06a1cf80e354393 Mon Sep 17 00:00:00 2001 From: "ryne.yang" Date: Tue, 29 Jan 2019 12:40:28 -0800 Subject: [PATCH 348/879] [SPARK-26718][SS][BRANCH-2.4] Fixed integer overflow in SS kafka rateLimit calculation ## What changes were proposed in this pull request? Fix the integer overflow issue in rateLimit. ## How was this patch tested? Pass the Jenkins with newly added UT for the possible case where integer could be overflowed. Closes #23652 from linehrr/fix/integer_overflow_rateLimit. Authored-by: ryne.yang Signed-off-by: Dongjoon Hyun --- .../sql/kafka010/KafkaMicroBatchReader.scala | 10 +++++- .../spark/sql/kafka010/KafkaSource.scala | 10 +++++- .../kafka010/KafkaMicroBatchSourceSuite.scala | 35 +++++++++++++++++++ 3 files changed, 53 insertions(+), 2 deletions(-) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala index b6c803545578..1333bc261020 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReader.scala @@ -239,7 +239,15 @@ private[kafka010] class KafkaMicroBatchReader( val begin = from.get(tp).getOrElse(fromNew(tp)) val prorate = limit * (size / total) // Don't completely starve small topicpartitions - val off = begin + (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong + val prorateLong = (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong + // need to be careful of integer overflow + // therefore added canary checks where to see if off variable could be overflowed + // refer to [https://issues.apache.org/jira/browse/SPARK-26718] + val off = if (prorateLong > Long.MaxValue - begin) { + Long.MaxValue + } else { + begin + prorateLong + } // Paranoia, make sure not to return an offset that's past end Math.min(end, off) }.getOrElse(end) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index d65b3cea632c..464ad648d4bf 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -190,7 +190,15 @@ private[kafka010] class KafkaSource( val prorate = limit * (size / total) logDebug(s"rateLimit $tp prorated amount is $prorate") // Don't completely starve small topicpartitions - val off = begin + (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong + val prorateLong = (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong + // need to be careful of integer overflow + // therefore added canary checks where to see if off variable could be overflowed + // refer to [https://issues.apache.org/jira/browse/SPARK-26718] + val off = if (prorateLong > Long.MaxValue - begin) { + Long.MaxValue + } else { + begin + prorateLong + } logDebug(s"rateLimit $tp new offset is $off") // Paranoia, make sure not to return an offset that's past end Math.min(end, off) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 5f058332de52..34cf3358e086 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -199,6 +199,41 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { StopStream) } + test("SPARK-26718 Rate limit set to Long.Max should not overflow integer " + + "during end offset calculation") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 1) + // fill in 5 messages to trigger potential integer overflow + testUtils.sendMessages(topic, (0 until 5).map(_.toString).toArray, Some(0)) + + val partitionOffsets = Map( + new TopicPartition(topic, 0) -> 5L + ) + val startingOffsets = JsonUtils.partitionOffsets(partitionOffsets) + + val kafka = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + // use latest to force begin to be 5 + .option("startingOffsets", startingOffsets) + // use Long.Max to try to trigger overflow + .option("maxOffsetsPerTrigger", Long.MaxValue) + .option("subscribe", topic) + .option("kafka.metadata.max.age.ms", "1") + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt) + + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 30, 31, 32, 33, 34), + CheckAnswer(30, 31, 32, 33, 34), + StopStream + ) + } + test("maxOffsetsPerTrigger") { val topic = newTopic() testUtils.createTopic(topic, partitions = 3) From 710d81ea87b5ba0c3d49b3dfbc591129685c2a13 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 31 Jan 2019 00:10:23 +0900 Subject: [PATCH 349/879] [SPARK-26732][CORE][TEST] Wait for listener bus to process events in SparkContextInfoSuite. Otherwise the RDD data may be out of date by the time the test tries to check it. Tested with an artificial delay inserted in AppStatusListener. Closes #23654 from vanzin/SPARK-26732. Authored-by: Marcelo Vanzin Signed-off-by: Takeshi Yamamuro (cherry picked from commit 6a2f3dcc2bd601fd1fe7610854bc0f5bf90300f4) Signed-off-by: Takeshi Yamamuro --- core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index 8feb3dee050d..051a13c9413e 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -60,6 +60,7 @@ class SparkContextInfoSuite extends SparkFunSuite with LocalSparkContext { val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() assert(sc.getRDDStorageInfo.size === 0) rdd.collect() + sc.listenerBus.waitUntilEmpty(10000) assert(sc.getRDDStorageInfo.size === 1) assert(sc.getRDDStorageInfo.head.isCached) assert(sc.getRDDStorageInfo.head.memSize > 0) From d9403e47f5c04f3d3c3c3ea573d38c393c5a470b Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E9=9F=A9=E7=94=B0=E7=94=B000222924?= Date: Thu, 31 Jan 2019 09:17:33 -0800 Subject: [PATCH 350/879] [SPARK-26726] Synchronize the amount of memory used by the broadcast variable to the UI display MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …not synchronized to the UI display ## What changes were proposed in this pull request? The amount of memory used by the broadcast variable is not synchronized to the UI display. I added the case for BroadcastBlockId and updated the memory usage. ## How was this patch tested? We can test this patch with unit tests. Closes #23649 from httfighter/SPARK-26726. Lead-authored-by: 韩田田00222924 Co-authored-by: han.tiantian@zte.com.cn Signed-off-by: Marcelo Vanzin (cherry picked from commit f4a17e916b729f9dc46e859b50a416db1e37b92e) Signed-off-by: Marcelo Vanzin --- .../spark/status/AppStatusListener.scala | 44 +++++++++++++++---- .../spark/status/AppStatusListenerSuite.scala | 18 ++++++++ 2 files changed, 53 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 5b564efa9684..c4dd47d23f91 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -750,6 +750,7 @@ private[spark] class AppStatusListener( event.blockUpdatedInfo.blockId match { case block: RDDBlockId => updateRDDBlock(event, block) case stream: StreamBlockId => updateStreamBlock(event, stream) + case broadcast: BroadcastBlockId => updateBroadcastBlock(event, broadcast) case _ => } } @@ -808,15 +809,7 @@ private[spark] class AppStatusListener( // Update the executor stats first, since they are used to calculate the free memory // on tracked RDD distributions. maybeExec.foreach { exec => - if (exec.hasMemoryInfo) { - if (storageLevel.useOffHeap) { - exec.usedOffHeap = addDeltaToValue(exec.usedOffHeap, memoryDelta) - } else { - exec.usedOnHeap = addDeltaToValue(exec.usedOnHeap, memoryDelta) - } - } - exec.memoryUsed = addDeltaToValue(exec.memoryUsed, memoryDelta) - exec.diskUsed = addDeltaToValue(exec.diskUsed, diskDelta) + updateExecutorMemoryDiskInfo(exec, storageLevel, memoryDelta, diskDelta) } // Update the block entry in the RDD info, keeping track of the deltas above so that we @@ -918,6 +911,39 @@ private[spark] class AppStatusListener( } } + private def updateBroadcastBlock( + event: SparkListenerBlockUpdated, + broadcast: BroadcastBlockId): Unit = { + val executorId = event.blockUpdatedInfo.blockManagerId.executorId + liveExecutors.get(executorId).foreach { exec => + val now = System.nanoTime() + val storageLevel = event.blockUpdatedInfo.storageLevel + + // Whether values are being added to or removed from the existing accounting. + val diskDelta = event.blockUpdatedInfo.diskSize * (if (storageLevel.useDisk) 1 else -1) + val memoryDelta = event.blockUpdatedInfo.memSize * (if (storageLevel.useMemory) 1 else -1) + + updateExecutorMemoryDiskInfo(exec, storageLevel, memoryDelta, diskDelta) + maybeUpdate(exec, now) + } + } + + private def updateExecutorMemoryDiskInfo( + exec: LiveExecutor, + storageLevel: StorageLevel, + memoryDelta: Long, + diskDelta: Long): Unit = { + if (exec.hasMemoryInfo) { + if (storageLevel.useOffHeap) { + exec.usedOffHeap = addDeltaToValue(exec.usedOffHeap, memoryDelta) + } else { + exec.usedOnHeap = addDeltaToValue(exec.usedOnHeap, memoryDelta) + } + } + exec.memoryUsed = addDeltaToValue(exec.memoryUsed, memoryDelta) + exec.diskUsed = addDeltaToValue(exec.diskUsed, diskDelta) + } + private def getOrCreateStage(info: StageInfo): LiveStage = { val stage = liveStages.computeIfAbsent((info.stageId, info.attemptNumber), new Function[(Int, Int), LiveStage]() { diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index f34be48a4d00..6214089126ae 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -939,6 +939,24 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { intercept[NoSuchElementException] { check[StreamBlockData](stream1.name) { _ => () } } + + // Update a BroadcastBlock. + val broadcast1 = BroadcastBlockId(1L) + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo(bm1, broadcast1, level, 1L, 1L))) + + check[ExecutorSummaryWrapper](bm1.executorId) { exec => + assert(exec.info.memoryUsed === 1L) + assert(exec.info.diskUsed === 1L) + } + + // Drop a BroadcastBlock. + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo(bm1, broadcast1, StorageLevel.NONE, 1L, 1L))) + check[ExecutorSummaryWrapper](bm1.executorId) { exec => + assert(exec.info.memoryUsed === 0) + assert(exec.info.diskUsed === 0) + } } test("eviction of old data") { From 2b5e033eb937a8074e454e1995616f8a1bf370f8 Mon Sep 17 00:00:00 2001 From: Huon Wilson Date: Thu, 31 Jan 2019 17:27:11 -0600 Subject: [PATCH 351/879] [SPARK-26757][GRAPHX] Return 0 for `count` on empty Edge/Vertex RDDs ## What changes were proposed in this pull request? Previously a "java.lang.UnsupportedOperationException: empty collection" exception would be thrown due to using `reduce`, rather than `fold` or similar that can tolerate empty RDDs. This behaviour has existed for the Vertex RDDs since it was introduced in b30e0ae0351be1cbc0b1cf179293587b466ee026. It seems this behaviour was inherited by the Edge RDDs via copy-paste in ee29ef3800438501e0ff207feb00a28973fc0769. ## How was this patch tested? Two new unit tests. Closes #23681 from huonw/empty-graphx. Authored-by: Huon Wilson Signed-off-by: Sean Owen (cherry picked from commit da526985c7574dccdcc0cca7452e2e999a5b3012) Signed-off-by: Sean Owen --- .../org/apache/spark/graphx/impl/EdgeRDDImpl.scala | 2 +- .../org/apache/spark/graphx/impl/VertexRDDImpl.scala | 2 +- .../org/apache/spark/graphx/lib/SVDPlusPlus.scala | 2 +- .../scala/org/apache/spark/graphx/EdgeRDDSuite.scala | 10 ++++++++++ .../org/apache/spark/graphx/VertexRDDSuite.scala | 11 +++++++++++ .../apache/spark/graphx/lib/SVDPlusPlusSuite.scala | 9 +++++++++ 6 files changed, 33 insertions(+), 3 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala index 376c7b06f9d2..eb8abd1846d0 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala @@ -87,7 +87,7 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( /** The number of edges in the RDD. */ override def count(): Long = { - partitionsRDD.map(_._2.size.toLong).reduce(_ + _) + partitionsRDD.map(_._2.size.toLong).fold(0)(_ + _) } override def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDDImpl[ED2, VD] = diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 3c6f22d97360..2da9762fb045 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -87,7 +87,7 @@ class VertexRDDImpl[VD] private[graphx] ( /** The number of vertices in the RDD. */ override def count(): Long = { - partitionsRDD.map(_.size.toLong).reduce(_ + _) + partitionsRDD.map(_.size.toLong).fold(0)(_ + _) } override private[graphx] def mapVertexPartitions[VD2: ClassTag]( diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index 59fdd855e6f3..2847a4e172d4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -72,7 +72,7 @@ object SVDPlusPlus { // calculate global rating mean edges.cache() - val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) + val (rs, rc) = edges.map(e => (e.attr, 1L)).fold((0, 0))((a, b) => (a._1 + b._1, a._2 + b._2)) val u = rs / rc // construct graph diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala index 7a24e320c3e0..8fd3e6f5229c 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala @@ -60,4 +60,14 @@ class EdgeRDDSuite extends SparkFunSuite with LocalSparkContext { } } + test("count") { + withSpark { sc => + val empty = EdgeRDD.fromEdges(sc.emptyRDD[Edge[Int]]) + assert(empty.count === 0) + + val edges = List(Edge(0, 1, ()), Edge(1, 2, ()), Edge(2, 0, ())) + val nonempty = EdgeRDD.fromEdges(sc.parallelize(edges)) + assert(nonempty.count === edges.size) + } + } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 8e630435279d..434e6a84edf6 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -223,4 +223,15 @@ class VertexRDDSuite extends SparkFunSuite with LocalSparkContext { assert(verts.collect().toSeq === data) // test checkpointed RDD } } + + test("count") { + withSpark { sc => + val empty = VertexRDD(sc.emptyRDD[(Long, Unit)]) + assert(empty.count === 0) + + val n = 100 + val nonempty = vertices(sc, n) + assert(nonempty.count === n + 1) + } + } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala index 2991438f5e57..da0457c354b5 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala @@ -40,4 +40,13 @@ class SVDPlusPlusSuite extends SparkFunSuite with LocalSparkContext { } } + test("Test SVD++ with no edges") { + withSpark { sc => + val edges = sc.emptyRDD[Edge[Double]] + val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations + val (graph, _) = SVDPlusPlus.run(edges, conf) + assert(graph.vertices.count == 0) + assert(graph.edges.count == 0) + } + } } From 2a8343121e62aabe5c69d1e20fbb2c01e2e520e7 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 1 Feb 2019 10:22:05 +0800 Subject: [PATCH 352/879] [SPARK-26745][SPARK-24959][SQL][BRANCH-2.4] Revert count optimization in JSON datasource by ## What changes were proposed in this pull request? This PR reverts JSON count optimization part of #21909. We cannot distinguish the cases below without parsing: ``` [{...}, {...}] ``` ``` [] ``` ``` {...} ``` ```bash # empty string ``` when we `count()`. One line (input: IN) can be, 0 record, 1 record and multiple records and this is dependent on each input. See also https://github.com/apache/spark/pull/23665#discussion_r251276720. ## How was this patch tested? Manually tested. Closes #23708 from HyukjinKwon/SPARK-26745-backport. Authored-by: Hyukjin Kwon Signed-off-by: Hyukjin Kwon --- .../org/apache/spark/sql/DataFrameReader.scala | 6 ++---- .../datasources/FailureSafeParser.scala | 11 ++--------- .../datasources/csv/UnivocityParser.scala | 16 +++++++++++----- .../datasources/json/JsonDataSource.scala | 6 ++---- .../datasources/json/JsonBenchmarks.scala | 4 ---- 5 files changed, 17 insertions(+), 26 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala index 869c584aed90..e9278a096493 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameReader.scala @@ -450,8 +450,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { input => rawParser.parse(input, createParser, UTF8String.fromString), parsedOptions.parseMode, schema, - parsedOptions.columnNameOfCorruptRecord, - parsedOptions.multiLine) + parsedOptions.columnNameOfCorruptRecord) iter.flatMap(parser.parse) } sparkSession.internalCreateDataFrame(parsed, schema, isStreaming = jsonDataset.isStreaming) @@ -526,8 +525,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { input => Seq(rawParser.parse(input)), parsedOptions.parseMode, schema, - parsedOptions.columnNameOfCorruptRecord, - parsedOptions.multiLine) + parsedOptions.columnNameOfCorruptRecord) iter.flatMap(parser.parse) } sparkSession.internalCreateDataFrame(parsed, schema, isStreaming = csvDataset.isStreaming) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FailureSafeParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FailureSafeParser.scala index 90e81661bae7..e618f1723678 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FailureSafeParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FailureSafeParser.scala @@ -29,8 +29,7 @@ class FailureSafeParser[IN]( rawParser: IN => Seq[InternalRow], mode: ParseMode, schema: StructType, - columnNameOfCorruptRecord: String, - isMultiLine: Boolean) { + columnNameOfCorruptRecord: String) { private val corruptFieldIndex = schema.getFieldIndex(columnNameOfCorruptRecord) private val actualSchema = StructType(schema.filterNot(_.name == columnNameOfCorruptRecord)) @@ -58,15 +57,9 @@ class FailureSafeParser[IN]( } } - private val skipParsing = !isMultiLine && mode == PermissiveMode && schema.isEmpty - def parse(input: IN): Iterator[InternalRow] = { try { - if (skipParsing) { - Iterator.single(InternalRow.empty) - } else { - rawParser.apply(input).toIterator.map(row => toResultRow(Some(row), () => null)) - } + rawParser.apply(input).toIterator.map(row => toResultRow(Some(row), () => null)) } catch { case e: BadRecordException => mode match { case PermissiveMode => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala index 9088d43905e2..42e396452522 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala @@ -203,11 +203,19 @@ class UnivocityParser( } } + private val doParse = if (requiredSchema.nonEmpty) { + (input: String) => convert(tokenizer.parseLine(input)) + } else { + // If `columnPruning` enabled and partition attributes scanned only, + // `schema` gets empty. + (_: String) => InternalRow.empty + } + /** * Parses a single CSV string and turns it into either one resulting row or no row (if the * the record is malformed). */ - def parse(input: String): InternalRow = convert(tokenizer.parseLine(input)) + def parse(input: String): InternalRow = doParse(input) private val getToken = if (options.columnPruning) { (tokens: Array[String], index: Int) => tokens(index) @@ -290,8 +298,7 @@ private[csv] object UnivocityParser { input => Seq(parser.convert(input)), parser.options.parseMode, schema, - parser.options.columnNameOfCorruptRecord, - parser.options.multiLine) + parser.options.columnNameOfCorruptRecord) convertStream(inputStream, shouldDropHeader, tokenizer, checkHeader) { tokens => safeParser.parse(tokens) }.flatten @@ -339,8 +346,7 @@ private[csv] object UnivocityParser { input => Seq(parser.parse(input)), parser.options.parseMode, schema, - parser.options.columnNameOfCorruptRecord, - parser.options.multiLine) + parser.options.columnNameOfCorruptRecord) filteredLines.flatMap(safeParser.parse) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 76f58371ae26..d6c588894d7f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -139,8 +139,7 @@ object TextInputJsonDataSource extends JsonDataSource { input => parser.parse(input, textParser, textToUTF8String), parser.options.parseMode, schema, - parser.options.columnNameOfCorruptRecord, - parser.options.multiLine) + parser.options.columnNameOfCorruptRecord) linesReader.flatMap(safeParser.parse) } @@ -224,8 +223,7 @@ object MultiLineJsonDataSource extends JsonDataSource { input => parser.parse[InputStream](input, streamParser, partitionedFileString), parser.options.parseMode, schema, - parser.options.columnNameOfCorruptRecord, - parser.options.multiLine) + parser.options.columnNameOfCorruptRecord) safeParser.parse( CodecStreams.createInputStreamWithCloseResource(conf, new Path(new URI(file.filePath)))) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala index a2b747eaab41..5592aa6e3346 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala @@ -194,9 +194,6 @@ object JSONBenchmarks { benchmark.addCase(s"Select 1 column + count()", 3) { _ => ds.select($"col1").filter((_: Row) => true).count() } - benchmark.addCase(s"count()", 3) { _ => - ds.count() - } /* Intel(R) Core(TM) i7-7700HQ CPU @ 2.80GHz @@ -205,7 +202,6 @@ object JSONBenchmarks { --------------------------------------------------------------------------------------------- Select 10 columns + count() 9961 / 10006 1.0 996.1 1.0X Select 1 column + count() 8355 / 8470 1.2 835.5 1.2X - count() 2104 / 2156 4.8 210.4 4.7X */ benchmark.run() } From bd4ce51e699da306bc36db0c7b0303b6e6c3d4df Mon Sep 17 00:00:00 2001 From: Shixiong Zhu Date: Fri, 1 Feb 2019 11:15:05 -0800 Subject: [PATCH 353/879] [SPARK-26806][SS] EventTimeStats.merge should handle zeros correctly ## What changes were proposed in this pull request? Right now, EventTimeStats.merge doesn't handle `zero.merge(zero)` correctly. This will make `avg` become `NaN`. And whatever gets merged with the result of `zero.merge(zero)`, `avg` will still be `NaN`. Then finally, we call `NaN.toLong` and get `0`, and the user will see the following incorrect report: ``` "eventTime" : { "avg" : "1970-01-01T00:00:00.000Z", "max" : "2019-01-31T12:57:00.000Z", "min" : "2019-01-30T18:44:04.000Z", "watermark" : "1970-01-01T00:00:00.000Z" } ``` This issue was reported by liancheng . This PR fixes the above issue. ## How was this patch tested? The new unit tests. Closes #23718 from zsxwing/merge-zero. Authored-by: Shixiong Zhu Signed-off-by: Shixiong Zhu (cherry picked from commit 03a928cbecaf38bbbab3e6b957fcbb542771cfbd) Signed-off-by: Shixiong Zhu --- .../streaming/EventTimeWatermarkExec.scala | 17 +++++++--- .../streaming/EventTimeWatermarkSuite.scala | 32 +++++++++++++++++-- 2 files changed, 42 insertions(+), 7 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala index b161651c4e6a..6fa7ee0c3818 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala @@ -36,10 +36,19 @@ case class EventTimeStats(var max: Long, var min: Long, var avg: Double, var cou } def merge(that: EventTimeStats): Unit = { - this.max = math.max(this.max, that.max) - this.min = math.min(this.min, that.min) - this.count += that.count - this.avg += (that.avg - this.avg) * that.count / this.count + if (that.count == 0) { + // no-op + } else if (this.count == 0) { + this.max = that.max + this.min = that.min + this.count = that.count + this.avg = that.avg + } else { + this.max = math.max(this.max, that.max) + this.min = math.min(this.min, that.min) + this.count += that.count + this.avg += (that.avg - this.avg) * that.count / this.count + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala index 026af17c7b23..091b9a10205d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/EventTimeWatermarkSuite.scala @@ -43,9 +43,9 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche sqlContext.streams.active.foreach(_.stop()) } - test("EventTimeStats") { - val epsilon = 10E-6 + private val epsilon = 10E-6 + test("EventTimeStats") { val stats = EventTimeStats(max = 100, min = 10, avg = 20.0, count = 5) stats.add(80L) stats.max should be (100) @@ -62,7 +62,6 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche } test("EventTimeStats: avg on large values") { - val epsilon = 10E-6 val largeValue = 10000000000L // 10B // Make sure `largeValue` will cause overflow if we use a Long sum to calc avg. assert(largeValue * largeValue != BigInt(largeValue) * BigInt(largeValue)) @@ -80,6 +79,33 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche stats.avg should be ((largeValue + 0.5) +- epsilon) } + test("EventTimeStats: zero merge zero") { + val stats = EventTimeStats.zero + val stats2 = EventTimeStats.zero + stats.merge(stats2) + stats should be (EventTimeStats.zero) + } + + test("EventTimeStats: non-zero merge zero") { + val stats = EventTimeStats(max = 10, min = 1, avg = 5.0, count = 3) + val stats2 = EventTimeStats.zero + stats.merge(stats2) + stats.max should be (10L) + stats.min should be (1L) + stats.avg should be (5.0 +- epsilon) + stats.count should be (3L) + } + + test("EventTimeStats: zero merge non-zero") { + val stats = EventTimeStats.zero + val stats2 = EventTimeStats(max = 10, min = 1, avg = 5.0, count = 3) + stats.merge(stats2) + stats.max should be (10L) + stats.min should be (1L) + stats.avg should be (5.0 +- epsilon) + stats.count should be (3L) + } + test("error on bad column") { val inputData = MemoryStream[Int].toDF() val e = intercept[AnalysisException] { From 190e48c1aa68158f3477f00dfd11c541539b3174 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Sat, 2 Feb 2019 09:17:52 -0800 Subject: [PATCH 354/879] [SPARK-26677][BUILD] Update Parquet to 1.10.1 with notEq pushdown fix. ## What changes were proposed in this pull request? Update to Parquet Java 1.10.1. ## How was this patch tested? Added a test from HyukjinKwon that validates the notEq case from SPARK-26677. Closes #23704 from rdblue/SPARK-26677-fix-noteq-parquet-bug. Lead-authored-by: Ryan Blue Co-authored-by: Hyukjin Kwon Co-authored-by: Ryan Blue Signed-off-by: Dongjoon Hyun (cherry picked from commit f72d2177882dc47b043fdc7dec9a46fe65df4ee9) Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.7 | 10 +++++----- dev/deps/spark-deps-hadoop-3.1 | 10 +++++----- pom.xml | 2 +- .../datasources/parquet/ParquetQuerySuite.scala | 15 +++++++++++++++ 4 files changed, 26 insertions(+), 11 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.7 b/dev/deps/spark-deps-hadoop-2.7 index 365a59b4ed03..4a6ad3f6b32d 100644 --- a/dev/deps/spark-deps-hadoop-2.7 +++ b/dev/deps/spark-deps-hadoop-2.7 @@ -160,13 +160,13 @@ orc-shims-1.5.4.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.10.0.jar -parquet-common-1.10.0.jar -parquet-encoding-1.10.0.jar +parquet-column-1.10.1.jar +parquet-common-1.10.1.jar +parquet-encoding-1.10.1.jar parquet-format-2.4.0.jar -parquet-hadoop-1.10.0.jar +parquet-hadoop-1.10.1.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.10.0.jar +parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.7.jar pyrolite-4.13.jar diff --git a/dev/deps/spark-deps-hadoop-3.1 b/dev/deps/spark-deps-hadoop-3.1 index 524e370f854b..83e243b7a01e 100644 --- a/dev/deps/spark-deps-hadoop-3.1 +++ b/dev/deps/spark-deps-hadoop-3.1 @@ -178,13 +178,13 @@ orc-shims-1.5.4.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.10.0.jar -parquet-common-1.10.0.jar -parquet-encoding-1.10.0.jar +parquet-column-1.10.1.jar +parquet-common-1.10.1.jar +parquet-encoding-1.10.1.jar parquet-format-2.4.0.jar -parquet-hadoop-1.10.0.jar +parquet-hadoop-1.10.1.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.10.0.jar +parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.7.jar pyrolite-4.13.jar diff --git a/pom.xml b/pom.xml index 91be9d914b1d..a5a52cdc8537 100644 --- a/pom.xml +++ b/pom.xml @@ -130,7 +130,7 @@ 1.2.1 10.12.1.1 - 1.10.0 + 1.10.1 1.5.4 nohive 1.6.0 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 54c77dddc352..7f8357c2c69c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -891,6 +891,21 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } } } + + test("SPARK-26677: negated null-safe equality comparison should not filter matched row groups") { + (true :: false :: Nil).foreach { vectorized => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + withTempPath { path => + // Repeated values for dictionary encoding. + Seq(Some("A"), Some("A"), None).toDF.repartition(1) + .write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), df) + } + } + } + } + } object TestingUDT { From 3d4aa5bd798a793a10c5d769a31f97242e78ce15 Mon Sep 17 00:00:00 2001 From: zhoukang Date: Sun, 3 Feb 2019 08:45:57 -0600 Subject: [PATCH 355/879] [SPARK-26751][SQL] Fix memory leak when statement run in background and throw exception which is not HiveSQLException ## What changes were proposed in this pull request? When we run in background and we get exception which is not HiveSQLException, we may encounter memory leak since handleToOperation will not removed correctly. The reason is below: 1. When calling operation.run() in HiveSessionImpl#executeStatementInternal we throw an exception which is not HiveSQLException 2. Then the opHandle generated by SparkSQLOperationManager will not be added into opHandleSet of HiveSessionImpl , and operationManager.closeOperation(opHandle) will not be called 3. When we close the session we will also call operationManager.closeOperation(opHandle),since we did not add this opHandle into the opHandleSet. For the reasons above,the opHandled will always in SparkSQLOperationManager#handleToOperation,which will cause memory leak. More details and a case has attached on https://issues.apache.org/jira/browse/SPARK-26751 This patch will always throw HiveSQLException when running in background ## How was this patch tested? Exist UT Closes #23673 from caneGuy/zhoukang/fix-hivesessionimpl-leak. Authored-by: zhoukang Signed-off-by: Sean Owen (cherry picked from commit 255faaf3436e1f41838062ed460f801bb0be40ec) Signed-off-by: Sean Owen --- .../sql/hive/thriftserver/SparkExecuteStatementOperation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 3cfc81b8a957..fd17f5028d46 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -204,7 +204,7 @@ private[hive] class SparkExecuteStatementOperation( case NonFatal(e) => logError(s"Error executing query in background", e) setState(OperationState.ERROR) - throw e + throw new HiveSQLException(e) } } } From 7187c012213eb4f13b5546f60514be5f08c7392a Mon Sep 17 00:00:00 2001 From: sandeep-katta Date: Mon, 4 Feb 2019 20:13:22 -0800 Subject: [PATCH 356/879] [SPARK-26758][CORE] Idle Executors are not getting killed after spark.dynamiAllocation.executorIdleTimeout value ## What changes were proposed in this pull request? **updateAndSyncNumExecutorsTarget** API should be called after **initializing** flag is unset ## How was this patch tested? Added UT and also manually tested After Fix ![afterfix](https://user-images.githubusercontent.com/35216143/51983136-ed4a5000-24bd-11e9-90c8-c4a562c17a4b.png) Closes #23697 from sandeep-katta/executorIssue. Authored-by: sandeep-katta Signed-off-by: Sean Owen (cherry picked from commit 1dd7419702c5bc7e36fee9fa1eec06b66f25806e) Signed-off-by: Sean Owen --- .../spark/ExecutorAllocationManager.scala | 4 +-- .../ExecutorAllocationManagerSuite.scala | 26 ++++++++++++++----- 2 files changed, 22 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index c3e5b96a5588..49fa80ca3fcd 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -306,8 +306,6 @@ private[spark] class ExecutorAllocationManager( private def schedule(): Unit = synchronized { val now = clock.getTimeMillis - updateAndSyncNumExecutorsTarget(now) - val executorIdsToBeRemoved = ArrayBuffer[String]() removeTimes.retain { case (executorId, expireTime) => val expired = now >= expireTime @@ -317,6 +315,8 @@ private[spark] class ExecutorAllocationManager( } !expired } + // Update executor target number only after initializing flag is unset + updateAndSyncNumExecutorsTarget(now) if (executorIdsToBeRemoved.nonEmpty) { removeExecutors(executorIdsToBeRemoved) } diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 5c718cb654ce..f50ad7805400 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -935,12 +935,7 @@ class ExecutorAllocationManagerSuite assert(maxNumExecutorsNeeded(manager) === 0) schedule(manager) - // Verify executor is timeout but numExecutorsTarget is not recalculated - assert(numExecutorsTarget(manager) === 3) - - // Schedule again to recalculate the numExecutorsTarget after executor is timeout - schedule(manager) - // Verify that current number of executors should be ramp down when executor is timeout + // Verify executor is timeout,numExecutorsTarget is recalculated assert(numExecutorsTarget(manager) === 2) } @@ -1147,6 +1142,25 @@ class ExecutorAllocationManagerSuite verify(mockAllocationClient).killExecutors(Seq("executor-1"), false, false, false) } + test("SPARK-26758 check executor target number after idle time out ") { + sc = createSparkContext(1, 5, 3) + val manager = sc.executorAllocationManager.get + val clock = new ManualClock(10000L) + manager.setClock(clock) + assert(numExecutorsTarget(manager) === 3) + manager.listener.onExecutorAdded(SparkListenerExecutorAdded( + clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty))) + manager.listener.onExecutorAdded(SparkListenerExecutorAdded( + clock.getTimeMillis(), "executor-2", new ExecutorInfo("host1", 2, Map.empty))) + manager.listener.onExecutorAdded(SparkListenerExecutorAdded( + clock.getTimeMillis(), "executor-3", new ExecutorInfo("host1", 3, Map.empty))) + // make all the executors as idle, so that it will be killed + clock.advance(executorIdleTimeout * 1000) + schedule(manager) + // once the schedule is run target executor number should be 1 + assert(numExecutorsTarget(manager) === 1) + } + private def createSparkContext( minExecutors: Int = 1, maxExecutors: Int = 5, From 570edc62d3c5020dc7499b0106083f8ba4dea03a Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 5 Feb 2019 17:22:33 -0800 Subject: [PATCH 357/879] [SPARK-26677][FOLLOWUP][BRANCH-2.4] Update Parquet manifest with Hadoop-2.6 ## What changes were proposed in this pull request? During merging Parquet upgrade PR, `hadoop-2.6` profile dependency manifest is missed. ## How was this patch tested? Manual. ``` ./dev/test-dependencies.sh ``` Also, this will recover `branch-2.4` with `hadoop-2.6` build. - https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-branch-2.4-test-sbt-hadoop-2.6/281/ Closes #23738 from dongjoon-hyun/SPARK-26677-2. Authored-by: Dongjoon Hyun Signed-off-by: Dongjoon Hyun --- dev/deps/spark-deps-hadoop-2.6 | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/dev/deps/spark-deps-hadoop-2.6 b/dev/deps/spark-deps-hadoop-2.6 index acbfac97ba27..307040ea4f0b 100644 --- a/dev/deps/spark-deps-hadoop-2.6 +++ b/dev/deps/spark-deps-hadoop-2.6 @@ -159,13 +159,13 @@ orc-shims-1.5.4.jar oro-2.0.8.jar osgi-resource-locator-1.0.1.jar paranamer-2.8.jar -parquet-column-1.10.0.jar -parquet-common-1.10.0.jar -parquet-encoding-1.10.0.jar +parquet-column-1.10.1.jar +parquet-common-1.10.1.jar +parquet-encoding-1.10.1.jar parquet-format-2.4.0.jar -parquet-hadoop-1.10.0.jar +parquet-hadoop-1.10.1.jar parquet-hadoop-bundle-1.6.0.jar -parquet-jackson-1.10.0.jar +parquet-jackson-1.10.1.jar protobuf-java-2.5.0.jar py4j-0.10.7.jar pyrolite-4.13.jar From 9b2eedc5beb32e99eedbdca344278243531b1c52 Mon Sep 17 00:00:00 2001 From: Ross Lodge Date: Wed, 6 Feb 2019 08:43:40 -0800 Subject: [PATCH 358/879] [SPARK-26734][STREAMING] Fix StackOverflowError with large block queue ## What changes were proposed in this pull request? SPARK-23991 introduced a bug in `ReceivedBlockTracker#allocateBlocksToBatch`: when a queue with more than a few thousand blocks are in the queue, serializing the queue throws a StackOverflowError. This change just adds `dequeueAll` to the new `clone` operation on the queue so that the fix in 23991 is preserved but the serialized data comes from an ArrayBuffer which doesn't have the serialization problems that mutable.Queue has. ## How was this patch tested? A unit test was added. Closes #23716 from rlodge/SPARK-26734. Authored-by: Ross Lodge Signed-off-by: Sean Owen (cherry picked from commit 8427e9ba5cae28233d1bdc54208b46889b83a821) Signed-off-by: Sean Owen --- .../scheduler/ReceivedBlockTracker.scala | 6 ++++- .../streaming/ReceivedBlockTrackerSuite.scala | 25 +++++++++++++++++-- 2 files changed, 28 insertions(+), 3 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index cf4324578ea8..a9763cfe0453 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -111,8 +111,12 @@ private[streaming] class ReceivedBlockTracker( */ def allocateBlocksToBatch(batchTime: Time): Unit = synchronized { if (lastAllocatedBatchTime == null || batchTime > lastAllocatedBatchTime) { + // We explicitly create an ArrayBuffer here because at least as of Scala 2.11 and 2.12 + // a mutable.Queue fails serialization with a StackOverflow error if it has more than + // a few thousand elements. So we explicitly allocate a collection for serialization which + // we know doesn't have this issue. (See SPARK-26734). val streamIdToBlocks = streamIds.map { streamId => - (streamId, getReceivedBlockQueue(streamId).clone()) + (streamId, mutable.ArrayBuffer(getReceivedBlockQueue(streamId).clone(): _*)) }.toMap val allocatedBlocks = AllocatedBlocks(streamIdToBlocks) if (writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks))) { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index fd7e00b1de25..c0f5eff597c6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -96,6 +96,27 @@ class ReceivedBlockTrackerSuite receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual blockInfos } + test("block addition, and block to batch allocation with many blocks") { + val receivedBlockTracker = createTracker() + receivedBlockTracker.isWriteAheadLogEnabled should be (true) + + val blockInfos = generateBlockInfos(100000) + blockInfos.map(receivedBlockTracker.addBlock) + receivedBlockTracker.allocateBlocksToBatch(1) + + receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual Seq.empty + receivedBlockTracker.hasUnallocatedReceivedBlocks should be (false) + receivedBlockTracker.getBlocksOfBatch(1) shouldEqual Map(streamId -> blockInfos) + receivedBlockTracker.getBlocksOfBatchAndStream(1, streamId) shouldEqual blockInfos + + val expectedWrittenData1 = blockInfos.map(BlockAdditionEvent) :+ + BatchAllocationEvent(1, AllocatedBlocks(Map(streamId -> blockInfos))) + getWrittenLogData() shouldEqual expectedWrittenData1 + getWriteAheadLogFiles() should have size 1 + + receivedBlockTracker.stop() + } + test("recovery with write ahead logs should remove only allocated blocks from received queue") { val manualClock = new ManualClock val batchTime = manualClock.getTimeMillis() @@ -362,8 +383,8 @@ class ReceivedBlockTrackerSuite } /** Generate blocks infos using random ids */ - def generateBlockInfos(): Seq[ReceivedBlockInfo] = { - List.fill(5)(ReceivedBlockInfo(streamId, Some(0L), None, + def generateBlockInfos(blockCount: Int = 5): Seq[ReceivedBlockInfo] = { + List.fill(blockCount)(ReceivedBlockInfo(streamId, Some(0L), None, BlockManagerBasedStoreResult(StreamBlockId(streamId, math.abs(Random.nextInt)), Some(0L)))) } From c41a5e1d8418ff0ce29d8ca98a42b17833d05238 Mon Sep 17 00:00:00 2001 From: mwlon Date: Thu, 7 Feb 2019 01:21:31 -0800 Subject: [PATCH 359/879] [SPARK-26082][MESOS] Fix mesos fetch cache config name ## What changes were proposed in this pull request? * change MesosClusterScheduler to use correct argument name for Mesos fetch cache (spark.mesos.fetchCache.enable -> spark.mesos.fetcherCache.enable) ## How was this patch tested? Not sure this requires a test, since it's just a string change. Closes #23734 from mwlon/SPARK-26082. Authored-by: mwlon Signed-off-by: Dongjoon Hyun (cherry picked from commit c0811e8b4d11892f60b7032ba4c8e3adc40fe82f) Signed-off-by: Dongjoon Hyun --- .../spark/scheduler/cluster/mesos/MesosClusterScheduler.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index cb1bcba651be..8c9a6b44ec6a 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -129,7 +129,7 @@ private[spark] class MesosClusterScheduler( private val queuedCapacity = conf.getInt("spark.mesos.maxDrivers", 200) private val retainedDrivers = conf.getInt("spark.mesos.retainedDrivers", 200) private val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute - private val useFetchCache = conf.getBoolean("spark.mesos.fetchCache.enable", false) + private val useFetchCache = conf.getBoolean("spark.mesos.fetcherCache.enable", false) private val schedulerState = engineFactory.createEngine("scheduler") private val stateLock = new Object() // Keyed by submission id From e645743ad57978823adac57d95fe02fa6f45dad0 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Thu, 7 Feb 2019 08:51:55 -0800 Subject: [PATCH 360/879] [SPARK-26082][MESOS][FOLLOWUP] Add UT on fetcher cache option on MesosClusterScheduler ## What changes were proposed in this pull request? This patch adds UT on testing SPARK-26082 to avoid regression. While #23743 reduces the possibility to make a similar mistake, the needed lines of code for adding tests are not that huge, so I guess it might be worth to add them. ## How was this patch tested? Newly added UTs. Test "supports setting fetcher cache" fails when #23743 is not applied and succeeds when #23743 is applied. Closes #23744 from HeartSaVioR/SPARK-26082-add-unit-test. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun (cherry picked from commit b4e1d145135445eeed85784dab0c2c088930dd26) Signed-off-by: Dongjoon Hyun --- .../mesos/MesosClusterSchedulerSuite.scala | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala index 082d4bcfdf83..7974120cda78 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala @@ -254,6 +254,54 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi assert(networkInfos.get(0).getLabels.getLabels(1).getValue == "val2") } + test("supports setting fetcher cache") { + setScheduler(Map("spark.mesos.fetcherCache.enable" -> "true")) + + val mem = 1000 + val cpu = 1 + + val response = scheduler.submitDriver( + new MesosDriverDescription("d1", "jar", mem, cpu, true, + command, + Map("spark.mesos.executor.home" -> "test", + "spark.app.name" -> "test"), + "s1", + new Date())) + + assert(response.success) + + val offer = Utils.createOffer("o1", "s1", mem, cpu) + scheduler.resourceOffers(driver, List(offer).asJava) + + val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") + val uris = launchedTasks.head.getCommand.getUrisList + assert(uris.stream().allMatch(_.getCache)) + } + + test("supports disabling fetcher cache") { + setScheduler(Map("spark.mesos.fetcherCache.enable" -> "false")) + + val mem = 1000 + val cpu = 1 + + val response = scheduler.submitDriver( + new MesosDriverDescription("d1", "jar", mem, cpu, true, + command, + Map("spark.mesos.executor.home" -> "test", + "spark.app.name" -> "test"), + "s1", + new Date())) + + assert(response.success) + + val offer = Utils.createOffer("o1", "s1", mem, cpu) + scheduler.resourceOffers(driver, List(offer).asJava) + + val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") + val uris = launchedTasks.head.getCommand.getUrisList + assert(uris.stream().allMatch(!_.getCache)) + } + test("accept/decline offers with driver constraints") { setScheduler() From f691b2cd9d6799ab13045a2e9c66b2cfe0a8db10 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 8 Feb 2019 19:51:25 -0800 Subject: [PATCH 361/879] Revert "[SPARK-26082][MESOS][FOLLOWUP] Add UT on fetcher cache option on MesosClusterScheduler" This reverts commit e645743ad57978823adac57d95fe02fa6f45dad0. --- .../mesos/MesosClusterSchedulerSuite.scala | 48 ------------------- 1 file changed, 48 deletions(-) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala index 7974120cda78..082d4bcfdf83 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala @@ -254,54 +254,6 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi assert(networkInfos.get(0).getLabels.getLabels(1).getValue == "val2") } - test("supports setting fetcher cache") { - setScheduler(Map("spark.mesos.fetcherCache.enable" -> "true")) - - val mem = 1000 - val cpu = 1 - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map("spark.mesos.executor.home" -> "test", - "spark.app.name" -> "test"), - "s1", - new Date())) - - assert(response.success) - - val offer = Utils.createOffer("o1", "s1", mem, cpu) - scheduler.resourceOffers(driver, List(offer).asJava) - - val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") - val uris = launchedTasks.head.getCommand.getUrisList - assert(uris.stream().allMatch(_.getCache)) - } - - test("supports disabling fetcher cache") { - setScheduler(Map("spark.mesos.fetcherCache.enable" -> "false")) - - val mem = 1000 - val cpu = 1 - - val response = scheduler.submitDriver( - new MesosDriverDescription("d1", "jar", mem, cpu, true, - command, - Map("spark.mesos.executor.home" -> "test", - "spark.app.name" -> "test"), - "s1", - new Date())) - - assert(response.success) - - val offer = Utils.createOffer("o1", "s1", mem, cpu) - scheduler.resourceOffers(driver, List(offer).asJava) - - val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") - val uris = launchedTasks.head.getCommand.getUrisList - assert(uris.stream().allMatch(!_.getCache)) - } - test("accept/decline offers with driver constraints") { setScheduler() From af3c7111efd22907976fc8bbd7810fe3cfd92092 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Sun, 10 Feb 2019 14:49:37 -0800 Subject: [PATCH 362/879] [SPARK-26082][MESOS][FOLLOWUP][BRANCH-2.4] Add UT on fetcher cache option on MesosClusterScheduler ## What changes were proposed in this pull request? This patch adds UT on testing SPARK-26082 to avoid regression. While #23743 reduces the possibility to make a similar mistake, the needed lines of code for adding tests are not that huge, so I guess it might be worth to add them. ## How was this patch tested? Newly added UTs. Test "supports setting fetcher cache" fails when #23734 is not applied and succeeds when #23734 is applied. Closes #23753 from HeartSaVioR/SPARK-26082-branch-2.4. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Dongjoon Hyun --- .../mesos/MesosClusterSchedulerSuite.scala | 48 +++++++++++++++++++ 1 file changed, 48 insertions(+) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala index 082d4bcfdf83..580c3a72c039 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala @@ -254,6 +254,54 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi assert(networkInfos.get(0).getLabels.getLabels(1).getValue == "val2") } + test("SPARK-26082 supports setting fetcher cache") { + setScheduler(Map("spark.mesos.fetcherCache.enable" -> "true")) + + val mem = 1000 + val cpu = 1 + + val response = scheduler.submitDriver( + new MesosDriverDescription("d1", "jar", mem, cpu, true, + command, + Map("spark.mesos.executor.home" -> "test", + "spark.app.name" -> "test"), + "s1", + new Date())) + + assert(response.success) + + val offer = Utils.createOffer("o1", "s1", mem, cpu) + scheduler.resourceOffers(driver, List(offer).asJava) + + val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") + val uris = launchedTasks.head.getCommand.getUrisList + assert(uris.asScala.forall(_.getCache)) + } + + test("SPARK-26082 supports disabling fetcher cache") { + setScheduler(Map("spark.mesos.fetcherCache.enable" -> "false")) + + val mem = 1000 + val cpu = 1 + + val response = scheduler.submitDriver( + new MesosDriverDescription("d1", "jar", mem, cpu, true, + command, + Map("spark.mesos.executor.home" -> "test", + "spark.app.name" -> "test"), + "s1", + new Date())) + + assert(response.success) + + val offer = Utils.createOffer("o1", "s1", mem, cpu) + scheduler.resourceOffers(driver, List(offer).asJava) + + val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") + val uris = launchedTasks.head.getCommand.getUrisList + assert(uris.asScala.forall(!_.getCache)) + } + test("accept/decline offers with driver constraints") { setScheduler() From 50eba0e3db55e0b2c22518abba87ea8374809f00 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 12 Feb 2019 10:45:06 -0800 Subject: [PATCH 363/879] Preparing Spark release v2.4.1-rc1 --- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 2 +- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 42 files changed, 42 insertions(+), 42 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index ee0de73f21d5..8e11fd687dd8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index b89e0fed2a66..f0eee076faae 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 3105a8182afc..8c8bdf4796b1 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index f85248b53080..663f41d1cf6d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 3350990c0461..9acade15cf38 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 9a2cb90a2f30..1a31a39c0dc9 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 9b927a9ba525..e3d5dd9004f6 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 718875f88e74..4845c58c1c2a 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index d8b078e419e6..d40baeae2994 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 72473771c2e9..7ddafaf76665 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,7 +14,7 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1-SNAPSHOT +SPARK_VERSION: 2.4.1 SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" diff --git a/examples/pom.xml b/examples/pom.xml index 737ebfd7900c..f5493aa9a287 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 7c6b78ac2b31..cf354864a486 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index 6343a15c43a8..ae87188c9ae9 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 2f4886832b75..244b90a54690 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 9404e2a52d90..8966055eb531 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 73dbc8d99449..b17ee8174fcf 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 912d82b764e5..df20f5911b13 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 8454f5c108dc..e1e8563ef843 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 05f7cf870af8..2c0926ebaef1 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index 0e935ac112af..a2c4dea645f9 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 7977389718cb..dd8693689738 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 7193df69900d..dce3c6859a97 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 4b318b00ff5b..d864996ce24c 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 6fbb1b7db6aa..41ee21cdf293 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 9a42d9a69967..70469df92c12 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 702f60ab9f00..ae997f885420 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 62322da714bd..e8e7b6b9728f 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index c8e45b27c78f..ef2b486de5dd 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 6201e0dd3c9a..75f400265c69 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/pom.xml b/pom.xml index a5a52cdc8537..a28ef6e59ab8 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 87282141fa64..d6654524283b 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1.dev0" +__version__ = "2.4.1" diff --git a/repl/pom.xml b/repl/pom.xml index d09548348eb5..b8844ba601c7 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index a9d9c8de9959..309526a1d461 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 793167103179..e6b9752a2d99 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index e6457289b859..5b4d4ef10a0b 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index e70711ce7257..eb6bea1de3e0 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 65cf33f32972..6aea1c19dea9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index c327b85ee866..fd51867b0160 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index d9d742444e7e..55c7c298f27f 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index c428be67aa3d..d593d1d1e201 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 37e96b807653..5567a55ccbe3 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index dd00c2f6cd2c..73fd3e270c78 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1-SNAPSHOT + 2.4.1 ../pom.xml From 351b44de026345bee25a445365c55854242185c2 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 12 Feb 2019 10:45:14 -0800 Subject: [PATCH 364/879] Preparing development version 2.4.2-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 714b6f18b19d..236128924fd6 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.2 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 8e11fd687dd8..c9999df354ab 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index f0eee076faae..092f85bcc32d 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 8c8bdf4796b1..5236fd605fa8 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 663f41d1cf6d..b70dadf158e2 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 9acade15cf38..e9ae143fd727 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 1a31a39c0dc9..2ae4fcbbcae1 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e3d5dd9004f6..6f756dba8612 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 4845c58c1c2a..6e53256cc011 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index d40baeae2994..629777b7b25b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7ddafaf76665..13b5d8ea4613 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1 -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.2-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.2 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index f5493aa9a287..9ad0eaf53e2a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index cf354864a486..2e6275dd03b6 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ae87188c9ae9..a4d67c8e8c23 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 244b90a54690..a57e9106bf16 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 8966055eb531..d05b4012a5dc 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index b17ee8174fcf..705f95ffbcbd 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index df20f5911b13..0814ab0a807f 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index e1e8563ef843..27bb30690e95 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 2c0926ebaef1..632c355f10d4 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a2c4dea645f9..b11912641d96 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index dd8693689738..8941ee7a6192 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index dce3c6859a97..9256f0bc8333 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index d864996ce24c..188c8f54a9fa 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 41ee21cdf293..e54560e2a586 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 70469df92c12..c5a8721e1527 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index ae997f885420..e66a22564029 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e8e7b6b9728f..4969316c3f4d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index ef2b486de5dd..1bdc105855bb 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 75f400265c69..de85b85e7d2c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index a28ef6e59ab8..69f56b81ae6b 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index d6654524283b..02eb2bdfb6d1 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1" +__version__ = "2.4.2.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index b8844ba601c7..a0d14e7c74a6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 309526a1d461..788e706073aa 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index e6b9752a2d99..47d15afeb74b 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 5b4d4ef10a0b..07e17c84878a 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index eb6bea1de3e0..48ca34fc13b1 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6aea1c19dea9..f8bcadf265b5 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index fd51867b0160..110d4a944604 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 55c7c298f27f..7f2d80969aec 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index d593d1d1e201..cb068ed59dc5 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 5567a55ccbe3..0e7b59fb2657 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 73fd3e270c78..7e210ea7393b 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml From 455a57d55e88f85d9e34555cfcc845fda051cec0 Mon Sep 17 00:00:00 2001 From: Viktor Tarasenko Date: Wed, 13 Feb 2019 08:01:20 -0600 Subject: [PATCH 365/879] [MINOR][DOCS] Fix for contradiction in condition formula of keeping intermediate state of window in structured streaming docs This change solves contradiction in structured streaming documentation in formula which tests if specific window will be updated by calculating watermark and comparing with "T" parameter(intermediate state is cleared as (max event time seen by the engine - late threshold > T), otherwise kept(written as "until")). By further examples the "T" seems to be the end of the window, not start like documentation says firstly. For more information please take a look at my question in stackoverflow https://stackoverflow.com/questions/54599594/understanding-window-with-watermark-in-apache-spark-structured-streaming Can be tested by building documentation. Closes #23765 from vitektarasenko/master. Authored-by: Viktor Tarasenko Signed-off-by: Sean Owen (cherry picked from commit 5894f767d1f159fc05e11d77d61089efcd0c50b4) Signed-off-by: Sean Owen --- docs/structured-streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index 3678bfb8609a..3d91223fc730 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -922,7 +922,7 @@ late data for that aggregate any more. To enable this, in Spark 2.1, we have int **watermarking**, which lets the engine automatically track the current event time in the data and attempt to clean up old state accordingly. You can define the watermark of a query by specifying the event time column and the threshold on how late the data is expected to be in terms of -event time. For a specific window starting at time `T`, the engine will maintain state and allow late +event time. For a specific window ending at time `T`, the engine will maintain state and allow late data to update the state until `(max event time seen by the engine - late threshold > T)`. In other words, late data within the threshold will be aggregated, but data later than the threshold will start getting dropped From 61b47872677b655d75720856bc161417b1346cfb Mon Sep 17 00:00:00 2001 From: Peter Toth Date: Thu, 14 Feb 2019 23:02:56 +0800 Subject: [PATCH 366/879] [SPARK-26572][SQL] fix aggregate codegen result evaluation This PR is a correctness fix in `HashAggregateExec` code generation. It forces evaluation of result expressions before calling `consume()` to avoid multiple executions. This PR fixes a use case where an aggregate is nested into a broadcast join and appears on the "stream" side. The issue is that Broadcast join generates it's own loop. And without forcing evaluation of `resultExpressions` of `HashAggregateExec` before the join's loop these expressions can be executed multiple times giving incorrect results. New UT was added. Closes #23731 from peter-toth/SPARK-26572. Authored-by: Peter Toth Signed-off-by: Wenchen Fan (cherry picked from commit 2228ee51ce3550d7e6740a1833aae21ab8596764) Signed-off-by: Wenchen Fan --- .../sql/execution/WholeStageCodegenExec.scala | 12 ++++++++ .../aggregate/HashAggregateExec.scala | 12 ++++++-- .../execution/WholeStageCodegenSuite.scala | 30 ++++++++++++++++++- 3 files changed, 51 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala index 7b01f619a461..d06380e8657c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala @@ -289,6 +289,18 @@ trait CodegenSupport extends SparkPlan { evaluateVars.toString() } + /** + * Returns source code to evaluate the variables for non-deterministic expressions, and clear the + * code of evaluated variables, to prevent them to be evaluated twice. + */ + protected def evaluateNondeterministicVariables( + attributes: Seq[Attribute], + variables: Seq[ExprCode], + expressions: Seq[NamedExpression]): String = { + val nondeterministicAttrs = expressions.filterNot(_.deterministic).map(_.toAttribute) + evaluateRequiredVariables(attributes, variables, AttributeSet(nondeterministicAttrs)) + } + /** * The subset of inputSet those should be evaluated before this plan. * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 98adba50b297..383877747fee 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -473,10 +473,13 @@ case class HashAggregateExec( val resultVars = resultExpressions.map { e => BindReferences.bindReference(e, inputAttrs).genCode(ctx) } + val evaluateNondeterministicResults = + evaluateNondeterministicVariables(output, resultVars, resultExpressions) s""" $evaluateKeyVars $evaluateBufferVars $evaluateAggResults + $evaluateNondeterministicResults ${consume(ctx, resultVars)} """ } else if (modes.contains(Partial) || modes.contains(PartialMerge)) { @@ -513,10 +516,15 @@ case class HashAggregateExec( // generate result based on grouping key ctx.INPUT_ROW = keyTerm ctx.currentVars = null - val eval = resultExpressions.map{ e => + val resultVars = resultExpressions.map{ e => BindReferences.bindReference(e, groupingAttributes).genCode(ctx) } - consume(ctx, eval) + val evaluateNondeterministicResults = + evaluateNondeterministicVariables(output, resultVars, resultExpressions) + s""" + $evaluateNondeterministicResults + ${consume(ctx, resultVars)} + """ } ctx.addNewFunction(funcName, s""" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala index e03f08417162..3c9a0908147a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.expressions.scalalang.typed -import org.apache.spark.sql.functions.{avg, broadcast, col, lit, max} +import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StringType, StructType} @@ -339,4 +339,32 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { checkAnswer(df, Seq(Row(1, 3), Row(2, 3))) } + + test("SPARK-26572: evaluate non-deterministic expressions for aggregate results") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString, + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + val baseTable = Seq(1, 1).toDF("idx") + + // BroadcastHashJoinExec with a HashAggregateExec child containing no aggregate expressions + val distinctWithId = baseTable.distinct().withColumn("id", monotonically_increasing_id()) + .join(baseTable, "idx") + assert(distinctWithId.queryExecution.executedPlan.collectFirst { + case WholeStageCodegenExec( + ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _))) => true + }.isDefined) + checkAnswer(distinctWithId, Seq(Row(1, 0), Row(1, 0))) + + // BroadcastHashJoinExec with a HashAggregateExec child containing a Final mode aggregate + // expression + val groupByWithId = + baseTable.groupBy("idx").sum().withColumn("id", monotonically_increasing_id()) + .join(baseTable, "idx") + assert(groupByWithId.queryExecution.executedPlan.collectFirst { + case WholeStageCodegenExec( + ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _))) => true + }.isDefined) + checkAnswer(groupByWithId, Seq(Row(1, 2, 0), Row(1, 2, 0))) + } + } } From bc1e960f70eda09d6e2b419f0c414528dcbb1cdf Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Thu, 14 Feb 2019 08:25:33 -0800 Subject: [PATCH 367/879] [SPARK-26873][SQL] Use a consistent timestamp to build Hadoop Job IDs. ## What changes were proposed in this pull request? Updates FileFormatWriter to create a consistent Hadoop Job ID for a write. ## How was this patch tested? Existing tests for regressions. Closes #23777 from rdblue/SPARK-26873-fix-file-format-writer-job-ids. Authored-by: Ryan Blue Signed-off-by: Marcelo Vanzin (cherry picked from commit 33334e2728f8d2e4cf7d542049435b589ed05a5e) Signed-off-by: Marcelo Vanzin --- .../spark/sql/execution/datasources/FileFormatWriter.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 2103a2d7e3d2..538b294fcd09 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -162,12 +162,14 @@ object FileFormatWriter extends Logging { rdd } + val jobIdInstant = new Date().getTime val ret = new Array[WriteTaskResult](rddWithNonEmptyPartitions.partitions.length) sparkSession.sparkContext.runJob( rddWithNonEmptyPartitions, (taskContext: TaskContext, iter: Iterator[InternalRow]) => { executeTask( description = description, + jobIdInstant = jobIdInstant, sparkStageId = taskContext.stageId(), sparkPartitionId = taskContext.partitionId(), sparkAttemptNumber = taskContext.taskAttemptId().toInt & Integer.MAX_VALUE, @@ -200,13 +202,14 @@ object FileFormatWriter extends Logging { /** Writes data out in a single Spark task. */ private def executeTask( description: WriteJobDescription, + jobIdInstant: Long, sparkStageId: Int, sparkPartitionId: Int, sparkAttemptNumber: Int, committer: FileCommitProtocol, iterator: Iterator[InternalRow]): WriteTaskResult = { - val jobId = SparkHadoopWriterUtils.createJobID(new Date, sparkStageId) + val jobId = SparkHadoopWriterUtils.createJobID(new Date(jobIdInstant), sparkStageId) val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber) From fccc6d3266876cfbcfe8815876bbe7e385fc5803 Mon Sep 17 00:00:00 2001 From: suxingfate Date: Fri, 15 Feb 2019 10:08:33 -0800 Subject: [PATCH 368/879] [SPARK-25922][K8S] Spark Driver/Executor "spark-app-selector" label mismatch (branch-2.4) In K8S Cluster mode, the algorithm to generate spark-app-selector/spark.app.id of spark driver is different with spark executor. This patch makes sure spark driver and executor to use the same spark-app-selector/spark.app.id if spark.app.id is set, otherwise it will use superclass applicationId. In K8S Client mode, spark-app-selector/spark.app.id for executors will use superclass applicationId. Manually run. Closes #23779 from vanzin/SPARK-25922. Authored-by: suxingfate Signed-off-by: Marcelo Vanzin --- .../KubernetesClusterSchedulerBackend.scala | 27 ++++++++++++++----- ...bernetesClusterSchedulerBackendSuite.scala | 14 +++++----- 2 files changed, 28 insertions(+), 13 deletions(-) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index fa6dc2c479bb..bdd41343e8af 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -18,9 +18,11 @@ package org.apache.spark.scheduler.cluster.k8s import java.util.concurrent.ExecutorService -import io.fabric8.kubernetes.client.KubernetesClient import scala.concurrent.{ExecutionContext, Future} +import io.fabric8.kubernetes.client.KubernetesClient + +import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.rpc.{RpcAddress, RpcEnv} import org.apache.spark.scheduler.{ExecutorLossReason, TaskSchedulerImpl} @@ -39,8 +41,8 @@ private[spark] class KubernetesClusterSchedulerBackend( pollEvents: ExecutorPodsPollingSnapshotSource) extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { - private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( - requestExecutorsService) + private implicit val requestExecutorContext = + ExecutionContext.fromExecutorService(requestExecutorsService) protected override val minRegisteredRatio = if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { @@ -56,6 +58,17 @@ private[spark] class KubernetesClusterSchedulerBackend( removeExecutor(executorId, reason) } + /** + * Get an application ID associated with the job. + * This returns the string value of spark.app.id if set, otherwise + * the locally-generated ID from the superclass. + * + * @return The application ID + */ + override def applicationId(): String = { + conf.getOption("spark.app.id").map(_.toString).getOrElse(super.applicationId) + } + override def start(): Unit = { super.start() if (!Utils.isDynamicAllocationEnabled(conf)) { @@ -83,7 +96,8 @@ private[spark] class KubernetesClusterSchedulerBackend( } Utils.tryLogNonFatalError { - kubernetesClient.pods() + kubernetesClient + .pods() .withLabel(SPARK_APP_ID_LABEL, applicationId()) .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .delete() @@ -114,7 +128,8 @@ private[spark] class KubernetesClusterSchedulerBackend( } override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { - kubernetesClient.pods() + kubernetesClient + .pods() .withLabel(SPARK_APP_ID_LABEL, applicationId()) .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .withLabelIn(SPARK_EXECUTOR_ID_LABEL, executorIds: _*) @@ -127,7 +142,7 @@ private[spark] class KubernetesClusterSchedulerBackend( } private class KubernetesDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) - extends DriverEndpoint(rpcEnv, sparkProperties) { + extends DriverEndpoint(rpcEnv, sparkProperties) { override def onDisconnected(rpcAddress: RpcAddress): Unit = { // Don't do anything besides disabling the executor - allow the Kubernetes API events to diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 52e7a12dbaf0..fbff1d7a24c8 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -37,6 +37,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn private val requestExecutorsService = new DeterministicScheduler() private val sparkConf = new SparkConf(false) .set("spark.executor.instances", "3") + .set("spark.app.id", TEST_SPARK_APP_ID) @Mock private var sc: SparkContext = _ @@ -82,8 +83,10 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn when(taskScheduler.sc).thenReturn(sc) when(sc.conf).thenReturn(sparkConf) driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint]) - when(rpcEnv.setupEndpoint( - mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) + when( + rpcEnv.setupEndpoint( + mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), + driverEndpoint.capture())) .thenReturn(driverEndpointRef) when(kubernetesClient.pods()).thenReturn(podOperations) schedulerBackendUnderTest = new KubernetesClusterSchedulerBackend( @@ -95,9 +98,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn podAllocator, lifecycleEventHandler, watchEvents, - pollEvents) { - override def applicationId(): String = TEST_SPARK_APP_ID - } + pollEvents) } test("Start all components") { @@ -122,8 +123,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn test("Remove executor") { schedulerBackendUnderTest.start() - schedulerBackendUnderTest.doRemoveExecutor( - "1", ExecutorKilled) + schedulerBackendUnderTest.doRemoveExecutor("1", ExecutorKilled) verify(driverEndpointRef).send(RemoveExecutor("1", ExecutorKilled)) } From 72702830b77ceca8443197f041c64cf43f833bc7 Mon Sep 17 00:00:00 2001 From: Dilip Biswal Date: Sat, 16 Feb 2019 17:05:10 +0800 Subject: [PATCH 369/879] [SPARK-26864][SQL][BACKPORT-2.4] Query may return incorrect result when python udf is used as a join condition and the udf uses attributes from both legs of left semi join ## What changes were proposed in this pull request? n SPARK-25314, we supported the scenario of having a python UDF that refers to attributes from both legs of a join condition by rewriting the plan to convert an inner join or left semi join to a filter over a cross join. In case of left semi join, this transformation may cause incorrect results when the right leg of join condition produces duplicate rows based on the join condition. This fix disallows the rewrite for left semi join and raises an error in the case like we do for other types of join. In future, we should have separate rule in optimizer to convert left semi join to inner join (I am aware of one case we could do it if we leverage informational constraint i.e when we know the right side does not produce duplicates). **Python** ```SQL >>> from pyspark import SparkContext >>> from pyspark.sql import SparkSession, Column, Row >>> from pyspark.sql.functions import UserDefinedFunction, udf >>> from pyspark.sql.types import * >>> from pyspark.sql.utils import AnalysisException >>> >>> spark.conf.set("spark.sql.crossJoin.enabled", "True") >>> left = spark.createDataFrame([Row(lc1=1, lc2=1), Row(lc1=2, lc2=2)]) >>> right = spark.createDataFrame([Row(rc1=1, rc2=1), Row(rc1=1, rc2=1)]) >>> func = udf(lambda a, b: a == b, BooleanType()) >>> df = left.join(right, func("lc1", "rc1"), "leftsemi").show() 19/02/12 16:07:10 WARN PullOutPythonUDFInJoinCondition: The join condition:(lc1#0L, rc1#4L) of the join plan contains PythonUDF only, it will be moved out and the join plan will be turned to cross join. +---+---+ |lc1|lc2| +---+---+ | 1| 1| | 1| 1| +---+---+ ``` **Scala** ```SQL scala> val left = Seq((1, 1), (2, 2)).toDF("lc1", "lc2") left: org.apache.spark.sql.DataFrame = [lc1: int, lc2: int] scala> val right = Seq((1, 1), (1, 1)).toDF("rc1", "rc2") right: org.apache.spark.sql.DataFrame = [rc1: int, rc2: int] scala> val equal = udf((p1: Integer, p2: Integer) => { | p1 == p2 | }) equal: org.apache.spark.sql.expressions.UserDefinedFunction = SparkUserDefinedFunction($Lambda$2141/11016292394666f1b5,BooleanType,List(Some(Schema(IntegerType,true)), Some(Schema(IntegerType,true))),None,false,true) scala> val df = left.join(right, equal(col("lc1"), col("rc1")), "leftsemi") df: org.apache.spark.sql.DataFrame = [lc1: int, lc2: int] scala> df.show() +---+---+ |lc1|lc2| +---+---+ | 1| 1| +---+---+ ``` ## How was this patch tested? Modified existing tests. Closes #23780 from dilipbiswal/dkb_python_udf_2.4_2. Authored-by: Dilip Biswal Signed-off-by: Wenchen Fan --- python/pyspark/sql/tests.py | 26 ++----------------- .../spark/sql/catalyst/optimizer/joins.scala | 8 ++---- ...PullOutPythonUDFInJoinConditionSuite.scala | 23 ++-------------- 3 files changed, 6 insertions(+), 51 deletions(-) diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 2098b43b6bb1..2e6d0159cb25 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -576,18 +576,6 @@ def test_udf_in_left_outer_join_condition(self): with self.sql_conf({"spark.sql.crossJoin.enabled": True}): self.assertEqual(df.collect(), [Row(a=1, b=1)]) - def test_udf_in_left_semi_join_condition(self): - # regression test for SPARK-25314 - from pyspark.sql.functions import udf - left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) - right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1)]) - f = udf(lambda a, b: a == b, BooleanType()) - df = left.join(right, f("a", "b"), "leftsemi") - with self.assertRaisesRegexp(AnalysisException, 'Detected implicit cartesian product'): - df.collect() - with self.sql_conf({"spark.sql.crossJoin.enabled": True}): - self.assertEqual(df.collect(), [Row(a=1, a1=1, a2=1)]) - def test_udf_and_common_filter_in_join_condition(self): # regression test for SPARK-25314 # test the complex scenario with both udf and common filter @@ -599,20 +587,9 @@ def test_udf_and_common_filter_in_join_condition(self): # do not need spark.sql.crossJoin.enabled=true for udf is not the only join condition. self.assertEqual(df.collect(), [Row(a=1, a1=1, a2=1, b=1, b1=1, b2=1)]) - def test_udf_and_common_filter_in_left_semi_join_condition(self): - # regression test for SPARK-25314 - # test the complex scenario with both udf and common filter - from pyspark.sql.functions import udf - left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) - right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1), Row(b=1, b1=3, b2=1)]) - f = udf(lambda a, b: a == b, BooleanType()) - df = left.join(right, [f("a", "b"), left.a1 == right.b1], "left_semi") - # do not need spark.sql.crossJoin.enabled=true for udf is not the only join condition. - self.assertEqual(df.collect(), [Row(a=1, a1=1, a2=1)]) - def test_udf_not_supported_in_join_condition(self): # regression test for SPARK-25314 - # test python udf is not supported in join type besides left_semi and inner join. + # test python udf is not supported in join type except inner join. from pyspark.sql.functions import udf left = self.spark.createDataFrame([Row(a=1, a1=1, a2=1), Row(a=2, a1=2, a2=2)]) right = self.spark.createDataFrame([Row(b=1, b1=1, b2=1), Row(b=1, b1=3, b2=1)]) @@ -627,6 +604,7 @@ def runWithJoinType(join_type, type_string): runWithJoinType("left", "LeftOuter") runWithJoinType("right", "RightOuter") runWithJoinType("leftanti", "LeftAnti") + runWithJoinType("leftsemi", "LeftSemi") def test_udf_without_arguments(self): self.spark.catalog.registerFunction("foo", lambda: "bar") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 2feb4720f9f9..56399e0446b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -192,8 +192,8 @@ object PullOutPythonUDFInJoinCondition extends Rule[LogicalPlan] with PredicateH override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case j @ Join(_, _, joinType, Some(cond)) if hasUnevaluablePythonUDF(cond, j) => - if (!joinType.isInstanceOf[InnerLike] && joinType != LeftSemi) { - // The current strategy only support InnerLike and LeftSemi join because for other type, + if (!joinType.isInstanceOf[InnerLike]) { + // The current strategy supports only InnerLike join because for other types, // it breaks SQL semantic if we run the join condition as a filter after join. If we pass // the plan here, it'll still get a an invalid PythonUDF RuntimeException with message // `requires attributes from more than one child`, we throw firstly here for better @@ -214,10 +214,6 @@ object PullOutPythonUDFInJoinCondition extends Rule[LogicalPlan] with PredicateH val newJoin = j.copy(condition = newCondition) joinType match { case _: InnerLike => Filter(udf.reduceLeft(And), newJoin) - case LeftSemi => - Project( - j.left.output.map(_.toAttribute), - Filter(udf.reduceLeft(And), newJoin.copy(joinType = Inner))) case _ => throw new AnalysisException("Using PythonUDF in join condition of join type" + s" $joinType is not supported.") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala index 3f1c91df7f2e..4a25ddf3ed9e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala @@ -64,7 +64,7 @@ class PullOutPythonUDFInJoinConditionSuite extends PlanTest { PythonEvalType.SQL_BATCHED_UDF, udfDeterministic = true) - val unsupportedJoinTypes = Seq(LeftOuter, RightOuter, FullOuter, LeftAnti) + val unsupportedJoinTypes = Seq(LeftOuter, RightOuter, FullOuter, LeftAnti, LeftSemi) private def comparePlanWithCrossJoinEnable(query: LogicalPlan, expected: LogicalPlan): Unit = { // AnalysisException thrown by CheckCartesianProducts while spark.sql.crossJoin.enabled=false @@ -99,25 +99,6 @@ class PullOutPythonUDFInJoinConditionSuite extends PlanTest { comparePlans(Optimize.execute(query2), query2) } - test("left semi join condition with python udf") { - val query1 = testRelationLeft.join( - testRelationRight, - joinType = LeftSemi, - condition = Some(unevaluableJoinCond)) - val expected1 = testRelationLeft.join( - testRelationRight, - joinType = Inner, - condition = None).where(unevaluableJoinCond).select('a, 'b).analyze - comparePlanWithCrossJoinEnable(query1, expected1) - - // evaluable PythonUDF will not be touched - val query2 = testRelationLeft.join( - testRelationRight, - joinType = LeftSemi, - condition = Some(evaluableJoinCond)) - comparePlans(Optimize.execute(query2), query2) - } - test("unevaluable python udf and common condition") { val query = testRelationLeft.join( testRelationRight, @@ -195,7 +176,7 @@ class PullOutPythonUDFInJoinConditionSuite extends PlanTest { comparePlans(optimized, expected) } - test("throw an exception for not support join type") { + test("throw an exception for not supported join types") { for (joinType <- unsupportedJoinTypes) { val e = intercept[AnalysisException] { val query = testRelationLeft.join( From dfda97a29f1211384503343d27afd752cc98f578 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 18 Feb 2019 08:05:49 +0900 Subject: [PATCH 370/879] [SPARK-26897][SQL][TEST] Update Spark 2.3.x testing from HiveExternalCatalogVersionsSuite ## What changes were proposed in this pull request? The maintenance release of `branch-2.3` (v2.3.3) vote passed, so this issue updates PROCESS_TABLES.testingVersions in HiveExternalCatalogVersionsSuite ## How was this patch tested? Pass the Jenkins. Closes #23807 from maropu/SPARK-26897. Authored-by: Takeshi Yamamuro Signed-off-by: Takeshi Yamamuro (cherry picked from commit dcdbd06b687fafbf29df504949db0a5f77608c8e) Signed-off-by: Takeshi Yamamuro --- .../spark/sql/hive/HiveExternalCatalogVersionsSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index 632a21a2a894..598b08b4f732 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -203,7 +203,7 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { object PROCESS_TABLES extends QueryTest with SQLTestUtils { // Tests the latest version of every release line. - val testingVersions = Seq("2.3.2", "2.4.0") + val testingVersions = Seq("2.3.3", "2.4.0") protected var spark: SparkSession = _ From 094cabc3f72da765cf2b4adab9bae61d05aaef45 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Mon, 18 Feb 2019 11:24:36 +0800 Subject: [PATCH 371/879] [SPARK-26897][SQL][TEST][FOLLOW-UP] Remove workaround for 2.2.0 and 2.1.x in HiveExternalCatalogVersionsSuite ## What changes were proposed in this pull request? This pr just removed workaround for 2.2.0 and 2.1.x in HiveExternalCatalogVersionsSuite. ## How was this patch tested? Pass the Jenkins. Closes #23817 from maropu/SPARK-26607-FOLLOWUP. Authored-by: Takeshi Yamamuro Signed-off-by: Hyukjin Kwon (cherry picked from commit e2b8cc65cd579374ddbd70b93c9fcefe9b8873d9) Signed-off-by: Hyukjin Kwon --- .../hive/HiveExternalCatalogVersionsSuite.scala | 17 ++++------------- 1 file changed, 4 insertions(+), 13 deletions(-) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index 598b08b4f732..0ede33d83990 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -257,19 +257,10 @@ object PROCESS_TABLES extends QueryTest with SQLTestUtils { // SPARK-22356: overlapped columns between data and partition schema in data source tables val tbl_with_col_overlap = s"tbl_with_col_overlap_$index" - // For Spark 2.2.0 and 2.1.x, the behavior is different from Spark 2.0, 2.2.1, 2.3+ - if (testingVersions(index).startsWith("2.1") || testingVersions(index) == "2.2.0") { - spark.sql("msck repair table " + tbl_with_col_overlap) - assert(spark.table(tbl_with_col_overlap).columns === Array("i", "j", "p")) - checkAnswer(spark.table(tbl_with_col_overlap), Row(1, 1, 1) :: Row(1, 1, 1) :: Nil) - assert(sql("desc " + tbl_with_col_overlap).select("col_name") - .as[String].collect().mkString(",").contains("i,j,p")) - } else { - assert(spark.table(tbl_with_col_overlap).columns === Array("i", "p", "j")) - checkAnswer(spark.table(tbl_with_col_overlap), Row(1, 1, 1) :: Row(1, 1, 1) :: Nil) - assert(sql("desc " + tbl_with_col_overlap).select("col_name") - .as[String].collect().mkString(",").contains("i,p,j")) - } + assert(spark.table(tbl_with_col_overlap).columns === Array("i", "p", "j")) + checkAnswer(spark.table(tbl_with_col_overlap), Row(1, 1, 1) :: Row(1, 1, 1) :: Nil) + assert(sql("desc " + tbl_with_col_overlap).select("col_name") + .as[String].collect().mkString(",").contains("i,p,j")) } } } From 633de74b60f2399a68dc6aa2c161dbb5568679e8 Mon Sep 17 00:00:00 2001 From: Maxim Gekk Date: Tue, 19 Feb 2019 11:46:42 +0800 Subject: [PATCH 372/879] [SPARK-26740][SQL][BRANCH-2.4] Read timestamp/date column stats written by Spark 3.0 ## What changes were proposed in this pull request? - Backport of #23662 to `branch-2.4` - Added `Timestamp`/`DateFormatter` - Set version of column stats to `1` to keep backward compatibility with previous versions ## How was this patch tested? The changes were tested by `StatisticsCollectionSuite` and by `StatisticsSuite`. Closes #23809 from MaxGekk/column-stats-time-date-2.4. Lead-authored-by: Maxim Gekk Co-authored-by: Maxim Gekk Signed-off-by: Wenchen Fan --- .../sql/catalyst/catalog/interface.scala | 32 +++-- .../catalyst/plans/logical/Statistics.scala | 7 +- .../sql/catalyst/util/DateFormatter.scala | 62 +++++++++ .../util/DateTimeFormatterHelper.scala | 78 ++++++++++++ .../sql/catalyst/util/DateTimeUtils.scala | 15 ++- .../catalyst/util/TimestampFormatter.scala | 87 +++++++++++++ .../spark/sql/catalyst/plans/SQLHelper.scala | 64 ++++++++++ .../sql/catalyst/util/DateTimeTestUtils.scala | 11 ++ .../spark/sql/util/DateFormatterSuite.scala | 98 ++++++++++++++ .../sql/util/TimestampFormatterSuite.scala | 120 ++++++++++++++++++ 10 files changed, 561 insertions(+), 13 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala index 30ded13410f7..645326408345 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Cast, ExprId, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -413,7 +413,8 @@ case class CatalogColumnStat( nullCount: Option[BigInt] = None, avgLen: Option[Long] = None, maxLen: Option[Long] = None, - histogram: Option[Histogram] = None) { + histogram: Option[Histogram] = None, + version: Int = CatalogColumnStat.VERSION) { /** * Returns a map from string to string that can be used to serialize the column stats. @@ -427,7 +428,7 @@ case class CatalogColumnStat( */ def toMap(colName: String): Map[String, String] = { val map = new scala.collection.mutable.HashMap[String, String] - map.put(s"${colName}.${CatalogColumnStat.KEY_VERSION}", "1") + map.put(s"${colName}.${CatalogColumnStat.KEY_VERSION}", CatalogColumnStat.VERSION.toString) distinctCount.foreach { v => map.put(s"${colName}.${CatalogColumnStat.KEY_DISTINCT_COUNT}", v.toString) } @@ -450,12 +451,13 @@ case class CatalogColumnStat( dataType: DataType): ColumnStat = ColumnStat( distinctCount = distinctCount, - min = min.map(CatalogColumnStat.fromExternalString(_, colName, dataType)), - max = max.map(CatalogColumnStat.fromExternalString(_, colName, dataType)), + min = min.map(CatalogColumnStat.fromExternalString(_, colName, dataType, version)), + max = max.map(CatalogColumnStat.fromExternalString(_, colName, dataType, version)), nullCount = nullCount, avgLen = avgLen, maxLen = maxLen, - histogram = histogram) + histogram = histogram, + version = version) } object CatalogColumnStat extends Logging { @@ -470,14 +472,23 @@ object CatalogColumnStat extends Logging { private val KEY_MAX_LEN = "maxLen" private val KEY_HISTOGRAM = "histogram" + val VERSION = 1 + + private def getTimestampFormatter(): TimestampFormatter = { + TimestampFormatter(format = "yyyy-MM-dd HH:mm:ss.SSSSSS", timeZone = DateTimeUtils.TimeZoneUTC) + } + /** * Converts from string representation of data type to the corresponding Catalyst data type. */ - def fromExternalString(s: String, name: String, dataType: DataType): Any = { + def fromExternalString(s: String, name: String, dataType: DataType, version: Int): Any = { dataType match { case BooleanType => s.toBoolean - case DateType => DateTimeUtils.fromJavaDate(java.sql.Date.valueOf(s)) - case TimestampType => DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf(s)) + case DateType if version == 1 => DateTimeUtils.fromJavaDate(java.sql.Date.valueOf(s)) + case DateType => DateFormatter().parse(s) + case TimestampType if version == 1 => + DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf(s)) + case TimestampType => getTimestampFormatter().parse(s) case ByteType => s.toByte case ShortType => s.toShort case IntegerType => s.toInt @@ -530,7 +541,8 @@ object CatalogColumnStat extends Logging { nullCount = map.get(s"${colName}.${KEY_NULL_COUNT}").map(v => BigInt(v.toLong)), avgLen = map.get(s"${colName}.${KEY_AVG_LEN}").map(_.toLong), maxLen = map.get(s"${colName}.${KEY_MAX_LEN}").map(_.toLong), - histogram = map.get(s"${colName}.${KEY_HISTOGRAM}").map(HistogramSerializer.deserialize) + histogram = map.get(s"${colName}.${KEY_HISTOGRAM}").map(HistogramSerializer.deserialize), + version = map(s"${colName}.${KEY_VERSION}").toInt )) } catch { case NonFatal(e) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala index b3a48860aa63..d0ca9ebdb6b9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala @@ -93,6 +93,7 @@ case class Statistics( * @param avgLen average length of the values. For fixed-length types, this should be a constant. * @param maxLen maximum length of the values. For fixed-length types, this should be a constant. * @param histogram histogram of the values + * @param version version of statistics saved to or retrieved from the catalog */ case class ColumnStat( distinctCount: Option[BigInt] = None, @@ -101,7 +102,8 @@ case class ColumnStat( nullCount: Option[BigInt] = None, avgLen: Option[Long] = None, maxLen: Option[Long] = None, - histogram: Option[Histogram] = None) { + histogram: Option[Histogram] = None, + version: Int = CatalogColumnStat.VERSION) { // Are distinctCount and nullCount statistics defined? val hasCountStats = distinctCount.isDefined && nullCount.isDefined @@ -120,7 +122,8 @@ case class ColumnStat( nullCount = nullCount, avgLen = avgLen, maxLen = maxLen, - histogram = histogram) + histogram = histogram, + version = version) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala new file mode 100644 index 000000000000..9535a369cb2e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import java.time.{Instant, ZoneId} +import java.util.Locale + +import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToDays + +sealed trait DateFormatter extends Serializable { + def parse(s: String): Int // returns days since epoch + def format(days: Int): String +} + +class Iso8601DateFormatter( + pattern: String, + locale: Locale) extends DateFormatter with DateTimeFormatterHelper { + + @transient + private lazy val formatter = getOrCreateFormatter(pattern, locale) + private val UTC = ZoneId.of("UTC") + + private def toInstant(s: String): Instant = { + val temporalAccessor = formatter.parse(s) + toInstantWithZoneId(temporalAccessor, UTC) + } + + override def parse(s: String): Int = instantToDays(toInstant(s)) + + override def format(days: Int): String = { + val instant = Instant.ofEpochSecond(days * DateTimeUtils.SECONDS_PER_DAY) + formatter.withZone(UTC).format(instant) + } +} + +object DateFormatter { + val defaultPattern: String = "yyyy-MM-dd" + val defaultLocale: Locale = Locale.US + + def apply(format: String, locale: Locale): DateFormatter = { + new Iso8601DateFormatter(format, locale) + } + + def apply(format: String): DateFormatter = apply(format, defaultLocale) + + def apply(): DateFormatter = apply(defaultPattern) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala new file mode 100644 index 000000000000..81ad6ad8ca40 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import java.time._ +import java.time.chrono.IsoChronology +import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder, ResolverStyle} +import java.time.temporal.{ChronoField, TemporalAccessor, TemporalQueries} +import java.util.Locale + +import com.google.common.cache.CacheBuilder + +import org.apache.spark.sql.catalyst.util.DateTimeFormatterHelper._ + +trait DateTimeFormatterHelper { + protected def toInstantWithZoneId(temporalAccessor: TemporalAccessor, zoneId: ZoneId): Instant = { + val localTime = if (temporalAccessor.query(TemporalQueries.localTime) == null) { + LocalTime.ofNanoOfDay(0) + } else { + LocalTime.from(temporalAccessor) + } + val localDate = LocalDate.from(temporalAccessor) + val localDateTime = LocalDateTime.of(localDate, localTime) + val zonedDateTime = ZonedDateTime.of(localDateTime, zoneId) + Instant.from(zonedDateTime) + } + + // Gets a formatter from the cache or creates new one. The buildFormatter method can be called + // a few times with the same parameters in parallel if the cache does not contain values + // associated to those parameters. Since the formatter is immutable, it does not matter. + // In this way, synchronised is intentionally omitted in this method to make parallel calls + // less synchronised. + // The Cache.get method is not used here to avoid creation of additional instances of Callable. + protected def getOrCreateFormatter(pattern: String, locale: Locale): DateTimeFormatter = { + val key = (pattern, locale) + var formatter = cache.getIfPresent(key) + if (formatter == null) { + formatter = buildFormatter(pattern, locale) + cache.put(key, formatter) + } + formatter + } +} + +private object DateTimeFormatterHelper { + val cache = CacheBuilder.newBuilder() + .maximumSize(128) + .build[(String, Locale), DateTimeFormatter]() + + def buildFormatter(pattern: String, locale: Locale): DateTimeFormatter = { + new DateTimeFormatterBuilder() + .parseCaseInsensitive() + .appendPattern(pattern) + .parseDefaulting(ChronoField.ERA, 1) + .parseDefaulting(ChronoField.MONTH_OF_YEAR, 1) + .parseDefaulting(ChronoField.DAY_OF_MONTH, 1) + .parseDefaulting(ChronoField.MINUTE_OF_HOUR, 0) + .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0) + .toFormatter(locale) + .withChronology(IsoChronology.INSTANCE) + .withResolverStyle(ResolverStyle.STRICT) + } +} 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 81d7274607ac..f01a7698d1f5 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} +import java.time.Instant import java.util.{Calendar, Locale, TimeZone} import java.util.concurrent.ConcurrentHashMap import java.util.function.{Function => JFunction} @@ -50,7 +51,7 @@ object DateTimeUtils { final val MILLIS_PER_SECOND = 1000L final val NANOS_PER_SECOND = MICROS_PER_SECOND * 1000L final val MICROS_PER_DAY = MICROS_PER_SECOND * SECONDS_PER_DAY - + final val NANOS_PER_MICROS = 1000L final val MILLIS_PER_DAY = SECONDS_PER_DAY * 1000L // number of days in 400 years @@ -440,6 +441,18 @@ object DateTimeUtils { Some(c.getTimeInMillis * 1000 + segments(6)) } + def instantToMicros(instant: Instant): Long = { + val sec = Math.multiplyExact(instant.getEpochSecond, MICROS_PER_SECOND) + val result = Math.addExact(sec, instant.getNano / NANOS_PER_MICROS) + result + } + + def instantToDays(instant: Instant): Int = { + val seconds = instant.getEpochSecond + val days = Math.floorDiv(seconds, SECONDS_PER_DAY) + days.toInt + } + /** * Parses a given UTF8 date string to a corresponding [[Int]] value. * The return type is [[Option]] in order to distinguish between 0 and null. The following diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala new file mode 100644 index 000000000000..4ec61e1ca4a5 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import java.text.ParseException +import java.time._ +import java.time.format.DateTimeParseException +import java.time.temporal.TemporalQueries +import java.util.{Locale, TimeZone} + +import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros + +sealed trait TimestampFormatter extends Serializable { + /** + * Parses a timestamp in a string and converts it to microseconds. + * + * @param s - string with timestamp to parse + * @return microseconds since epoch. + * @throws ParseException can be thrown by legacy parser + * @throws DateTimeParseException can be thrown by new parser + * @throws DateTimeException unable to obtain local date or time + */ + @throws(classOf[ParseException]) + @throws(classOf[DateTimeParseException]) + @throws(classOf[DateTimeException]) + def parse(s: String): Long + def format(us: Long): String +} + +class Iso8601TimestampFormatter( + pattern: String, + timeZone: TimeZone, + locale: Locale) extends TimestampFormatter with DateTimeFormatterHelper { + @transient + private lazy val formatter = getOrCreateFormatter(pattern, locale) + + private def toInstant(s: String): Instant = { + val temporalAccessor = formatter.parse(s) + if (temporalAccessor.query(TemporalQueries.offset()) == null) { + toInstantWithZoneId(temporalAccessor, timeZone.toZoneId) + } else { + Instant.from(temporalAccessor) + } + } + + override def parse(s: String): Long = instantToMicros(toInstant(s)) + + override def format(us: Long): String = { + val secs = Math.floorDiv(us, DateTimeUtils.MICROS_PER_SECOND) + val mos = Math.floorMod(us, DateTimeUtils.MICROS_PER_SECOND) + val instant = Instant.ofEpochSecond(secs, mos * DateTimeUtils.NANOS_PER_MICROS) + + formatter.withZone(timeZone.toZoneId).format(instant) + } +} + +object TimestampFormatter { + val defaultPattern: String = "yyyy-MM-dd HH:mm:ss" + val defaultLocale: Locale = Locale.US + + def apply(format: String, timeZone: TimeZone, locale: Locale): TimestampFormatter = { + new Iso8601TimestampFormatter(format, timeZone, locale) + } + + def apply(format: String, timeZone: TimeZone): TimestampFormatter = { + apply(format, timeZone, defaultLocale) + } + + def apply(timeZone: TimeZone): TimestampFormatter = { + apply(defaultPattern, timeZone, defaultLocale) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala new file mode 100644 index 000000000000..4d869d79ad59 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SQLHelper.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.plans + +import java.io.File + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.Utils + +trait SQLHelper { + + /** + * Sets all SQL configurations specified in `pairs`, calls `f`, and then restores all SQL + * configurations. + */ + protected def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + val conf = SQLConf.get + val (keys, values) = pairs.unzip + val currentValues = keys.map { key => + if (conf.contains(key)) { + Some(conf.getConfString(key)) + } else { + None + } + } + (keys, values).zipped.foreach { (k, v) => + if (SQLConf.staticConfKeys.contains(k)) { + throw new AnalysisException(s"Cannot modify the value of a static config: $k") + } + conf.setConfString(k, v) + } + try f finally { + keys.zip(currentValues).foreach { + case (key, Some(value)) => conf.setConfString(key, value) + case (key, None) => conf.unsetConf(key) + } + } + } + + /** + * Generates a temporary path without creating the actual file/directory, then pass it to `f`. If + * a file/directory is created there by `f`, it will be delete after `f` returns. + */ + protected def withTempPath(f: File => Unit): Unit = { + val path = Utils.createTempDir() + path.delete() + try f(path) finally Utils.deleteRecursively(path) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala index 0c1feb3aa088..66d8d28988f8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/DateTimeTestUtils.scala @@ -26,6 +26,17 @@ object DateTimeTestUtils { val ALL_TIMEZONES: Seq[TimeZone] = TimeZone.getAvailableIDs.toSeq.map(TimeZone.getTimeZone) + val outstandingTimezonesIds: Seq[String] = Seq( + "UTC", + "PST", + "CET", + "Africa/Dakar", + "America/Los_Angeles", + "Antarctica/Vostok", + "Asia/Hong_Kong", + "Europe/Amsterdam") + val outstandingTimezones: Seq[TimeZone] = outstandingTimezonesIds.map(TimeZone.getTimeZone) + def withDefaultTimeZone[T](newDefaultTimeZone: TimeZone)(block: => T): T = { val originalDefaultTimeZone = TimeZone.getDefault try { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala new file mode 100644 index 000000000000..602542fb33db --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala @@ -0,0 +1,98 @@ +/* + * 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.util + +import java.time.LocalDate + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf + +class DateFormatterSuite extends SparkFunSuite with SQLHelper { + test("parsing dates") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter() + val daysSinceEpoch = formatter.parse("2018-12-02") + assert(daysSinceEpoch === 17867) + } + } + } + + test("format dates") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter() + val date = formatter.format(17867) + assert(date === "2018-12-02") + } + } + } + + test("roundtrip date -> days -> date") { + Seq( + "0050-01-01", + "0953-02-02", + "1423-03-08", + "1969-12-31", + "1972-08-25", + "1975-09-26", + "2018-12-12", + "2038-01-01", + "5010-11-17").foreach { date => + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter() + val days = formatter.parse(date) + val formatted = formatter.format(days) + assert(date === formatted) + } + } + } + } + + test("roundtrip days -> date -> days") { + Seq( + -701265, + -371419, + -199722, + -1, + 0, + 967, + 2094, + 17877, + 24837, + 1110657).foreach { days => + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter() + val date = formatter.format(days) + val parsed = formatter.parse(date) + assert(days === parsed) + } + } + } + } + + test("parsing date without explicit day") { + val formatter = DateFormatter("yyyy MMM") + val daysSinceEpoch = formatter.parse("2018 Dec") + assert(daysSinceEpoch === LocalDate.of(2018, 12, 1).toEpochDay) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala new file mode 100644 index 000000000000..192ca13ddaf6 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -0,0 +1,120 @@ +/* + * 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.util + +import java.time.{LocalDateTime, ZoneOffset} +import java.util.TimeZone +import java.util.concurrent.TimeUnit + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, TimestampFormatter} + +class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { + + test("parsing timestamps using time zones") { + val localDate = "2018-12-02T10:11:12.001234" + val expectedMicros = Map( + "UTC" -> 1543745472001234L, + "PST" -> 1543774272001234L, + "CET" -> 1543741872001234L, + "Africa/Dakar" -> 1543745472001234L, + "America/Los_Angeles" -> 1543774272001234L, + "Antarctica/Vostok" -> 1543723872001234L, + "Asia/Hong_Kong" -> 1543716672001234L, + "Europe/Amsterdam" -> 1543741872001234L) + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + val formatter = TimestampFormatter( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", + TimeZone.getTimeZone(timeZone)) + val microsSinceEpoch = formatter.parse(localDate) + assert(microsSinceEpoch === expectedMicros(timeZone)) + } + } + + test("format timestamps using time zones") { + val microsSinceEpoch = 1543745472001234L + val expectedTimestamp = Map( + "UTC" -> "2018-12-02T10:11:12.001234", + "PST" -> "2018-12-02T02:11:12.001234", + "CET" -> "2018-12-02T11:11:12.001234", + "Africa/Dakar" -> "2018-12-02T10:11:12.001234", + "America/Los_Angeles" -> "2018-12-02T02:11:12.001234", + "Antarctica/Vostok" -> "2018-12-02T16:11:12.001234", + "Asia/Hong_Kong" -> "2018-12-02T18:11:12.001234", + "Europe/Amsterdam" -> "2018-12-02T11:11:12.001234") + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + val formatter = TimestampFormatter( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", + TimeZone.getTimeZone(timeZone)) + val timestamp = formatter.format(microsSinceEpoch) + assert(timestamp === expectedTimestamp(timeZone)) + } + } + + test("roundtrip micros -> timestamp -> micros using timezones") { + Seq("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", "yyyy-MM-dd'T'HH:mm:ss.SSSSSSXXXXX").foreach { pattern => + Seq( + -58710115316212000L, + -18926315945345679L, + -9463427405253013L, + -244000001L, + 0L, + 99628200102030L, + 1543749753123456L, + 2177456523456789L, + 11858049903010203L).foreach { micros => + DateTimeTestUtils.outstandingTimezones.foreach { timeZone => + val formatter = TimestampFormatter(pattern, timeZone) + val timestamp = formatter.format(micros) + val parsed = formatter.parse(timestamp) + assert(micros === parsed) + } + } + } + } + + test("roundtrip timestamp -> micros -> timestamp using timezones") { + Seq( + "0109-07-20T18:38:03.788000", + "1370-04-01T10:00:54.654321", + "1670-02-11T14:09:54.746987", + "1969-12-31T23:55:55.999999", + "1970-01-01T00:00:00.000000", + "1973-02-27T02:30:00.102030", + "2018-12-02T11:22:33.123456", + "2039-01-01T01:02:03.456789", + "2345-10-07T22:45:03.010203").foreach { timestamp => + DateTimeTestUtils.outstandingTimezones.foreach { timeZone => + val formatter = TimestampFormatter("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", timeZone) + val micros = formatter.parse(timestamp) + val formatted = formatter.format(micros) + assert(timestamp === formatted) + } + } + } + + test(" case insensitive parsing of am and pm") { + val formatter = TimestampFormatter( + "yyyy MMM dd hh:mm:ss a", + TimeZone.getTimeZone("UTC")) + val micros = formatter.parse("2009 Mar 20 11:30:01 am") + assert(micros === TimeUnit.SECONDS.toMicros( + LocalDateTime.of(2009, 3, 20, 11, 30, 1).toEpochSecond(ZoneOffset.UTC))) + } +} From 383b66264aa2bbadb2b87c7fb4096423951b78f1 Mon Sep 17 00:00:00 2001 From: joelgenter Date: Tue, 19 Feb 2019 08:40:59 -0600 Subject: [PATCH 373/879] [MINOR][DOCS] Fix the update rule in StreamingKMeansModel documentation ## What changes were proposed in this pull request? The formatting for the update rule (in the documentation) now appears as ![image](https://user-images.githubusercontent.com/14948437/52933807-5a0c7980-3309-11e9-8573-642a73e77c26.png) instead of ![image](https://user-images.githubusercontent.com/14948437/52933897-a8ba1380-3309-11e9-8e16-e47c27b4a044.png) Closes #23819 from joelgenter/patch-1. Authored-by: joelgenter Signed-off-by: Sean Owen (cherry picked from commit 885aa553c5e8f478b370f8a733102b67f6cd2d99) Signed-off-by: Sean Owen --- .../org/apache/spark/mllib/clustering/StreamingKMeans.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index ed8543da4d4c..ff4ca0ac40fe 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -42,8 +42,8 @@ import org.apache.spark.util.random.XORShiftRandom *
    * $$ * \begin{align} - * c_t+1 &= [(c_t * n_t * a) + (x_t * m_t)] / [n_t + m_t] \\ - * n_t+1 &= n_t * a + m_t + * c_{t+1} &= [(c_t * n_t * a) + (x_t * m_t)] / [n_t + m_t] \\ + * n_{t+1} &= n_t * a + m_t * \end{align} * $$ *
    From 229ad524cfd3f74dd7aa5fc9ba841ae223caa960 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 19 Feb 2019 21:54:26 +0000 Subject: [PATCH 374/879] Preparing Spark release v2.4.1-rc2 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 236128924fd6..714b6f18b19d 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.2 +Version: 2.4.1 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index c9999df354ab..8e11fd687dd8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 092f85bcc32d..f0eee076faae 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 5236fd605fa8..8c8bdf4796b1 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index b70dadf158e2..663f41d1cf6d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index e9ae143fd727..9acade15cf38 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 2ae4fcbbcae1..1a31a39c0dc9 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 6f756dba8612..e3d5dd9004f6 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 6e53256cc011..4845c58c1c2a 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 629777b7b25b..d40baeae2994 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 13b5d8ea4613..7ddafaf76665 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.2-SNAPSHOT -SPARK_VERSION_SHORT: 2.4.2 +SPARK_VERSION: 2.4.1 +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 9ad0eaf53e2a..f5493aa9a287 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 2e6275dd03b6..cf354864a486 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index a4d67c8e8c23..ae87188c9ae9 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index a57e9106bf16..244b90a54690 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index d05b4012a5dc..8966055eb531 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 705f95ffbcbd..b17ee8174fcf 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 0814ab0a807f..df20f5911b13 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 27bb30690e95..e1e8563ef843 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 632c355f10d4..2c0926ebaef1 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index b11912641d96..a2c4dea645f9 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 8941ee7a6192..dd8693689738 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 9256f0bc8333..dce3c6859a97 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 188c8f54a9fa..d864996ce24c 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index e54560e2a586..41ee21cdf293 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c5a8721e1527..70469df92c12 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index e66a22564029..ae997f885420 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 4969316c3f4d..e8e7b6b9728f 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1bdc105855bb..ef2b486de5dd 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index de85b85e7d2c..75f400265c69 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/pom.xml b/pom.xml index 69f56b81ae6b..a28ef6e59ab8 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 02eb2bdfb6d1..d6654524283b 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.2.dev0" +__version__ = "2.4.1" diff --git a/repl/pom.xml b/repl/pom.xml index a0d14e7c74a6..b8844ba601c7 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 788e706073aa..309526a1d461 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 47d15afeb74b..e6b9752a2d99 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 07e17c84878a..5b4d4ef10a0b 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 48ca34fc13b1..eb6bea1de3e0 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index f8bcadf265b5..6aea1c19dea9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 110d4a944604..fd51867b0160 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 7f2d80969aec..55c7c298f27f 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index cb068ed59dc5..d593d1d1e201 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 0e7b59fb2657..5567a55ccbe3 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 7e210ea7393b..73fd3e270c78 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml From 4c60056749fd3892ba3171a268b2432a42ba2966 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 19 Feb 2019 21:54:45 +0000 Subject: [PATCH 375/879] Preparing development version 2.4.2-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 714b6f18b19d..236128924fd6 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.2 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 8e11fd687dd8..c9999df354ab 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index f0eee076faae..092f85bcc32d 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 8c8bdf4796b1..5236fd605fa8 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 663f41d1cf6d..b70dadf158e2 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 9acade15cf38..e9ae143fd727 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 1a31a39c0dc9..2ae4fcbbcae1 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e3d5dd9004f6..6f756dba8612 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 4845c58c1c2a..6e53256cc011 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index d40baeae2994..629777b7b25b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7ddafaf76665..13b5d8ea4613 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1 -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.2-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.2 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index f5493aa9a287..9ad0eaf53e2a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index cf354864a486..2e6275dd03b6 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ae87188c9ae9..a4d67c8e8c23 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 244b90a54690..a57e9106bf16 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 8966055eb531..d05b4012a5dc 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index b17ee8174fcf..705f95ffbcbd 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index df20f5911b13..0814ab0a807f 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index e1e8563ef843..27bb30690e95 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 2c0926ebaef1..632c355f10d4 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a2c4dea645f9..b11912641d96 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index dd8693689738..8941ee7a6192 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index dce3c6859a97..9256f0bc8333 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index d864996ce24c..188c8f54a9fa 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 41ee21cdf293..e54560e2a586 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 70469df92c12..c5a8721e1527 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index ae997f885420..e66a22564029 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e8e7b6b9728f..4969316c3f4d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index ef2b486de5dd..1bdc105855bb 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 75f400265c69..de85b85e7d2c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index a28ef6e59ab8..69f56b81ae6b 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index d6654524283b..02eb2bdfb6d1 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1" +__version__ = "2.4.2.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index b8844ba601c7..a0d14e7c74a6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 309526a1d461..788e706073aa 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index e6b9752a2d99..47d15afeb74b 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 5b4d4ef10a0b..07e17c84878a 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index eb6bea1de3e0..48ca34fc13b1 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6aea1c19dea9..f8bcadf265b5 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index fd51867b0160..110d4a944604 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 55c7c298f27f..7f2d80969aec 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index d593d1d1e201..cb068ed59dc5 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 5567a55ccbe3..0e7b59fb2657 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 73fd3e270c78..7e210ea7393b 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml From 274142be08eb3a4239046d7f7260c7284ed041c2 Mon Sep 17 00:00:00 2001 From: Ivan Vergiliev Date: Wed, 20 Feb 2019 21:49:38 +0800 Subject: [PATCH 376/879] [SPARK-26859][SQL] Fix field writer index bug in non-vectorized ORC deserializer ## What changes were proposed in this pull request? This happens in a schema evolution use case only when a user specifies the schema manually and use non-vectorized ORC deserializer code path. There is a bug in `OrcDeserializer.scala` that results in `null`s being set at the wrong column position, and for state from previous records to remain uncleared in next records. There are more details for when exactly the bug gets triggered and what the outcome is in the [JIRA issue](https://jira.apache.org/jira/browse/SPARK-26859). The high-level summary is that this bug results in severe data correctness issues, but fortunately the set of conditions to expose the bug are complicated and make the surface area somewhat small. This change fixes the problem and adds a respective test. ## How was this patch tested? Pass the Jenkins with the newly added test cases. Closes #23766 from IvanVergiliev/fix-orc-deserializer. Lead-authored-by: Ivan Vergiliev Co-authored-by: Dongjoon Hyun Signed-off-by: Wenchen Fan (cherry picked from commit 096552ae4d6fcef5e20c54384a2687db41ba2fa1) Signed-off-by: Wenchen Fan --- .../datasources/orc/OrcDeserializer.scala | 34 +++++++++------- .../datasources/ReadSchemaSuite.scala | 6 +++ .../datasources/ReadSchemaTest.scala | 39 ++++++++++++++++++- 3 files changed, 64 insertions(+), 15 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala index 4ecc54bd2fd9..decd5c5f1925 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcDeserializer.scala @@ -37,28 +37,34 @@ class OrcDeserializer( private val resultRow = new SpecificInternalRow(requiredSchema.map(_.dataType)) + // `fieldWriters(index)` is + // - null if the respective source column is missing, since the output value + // is always null in this case + // - a function that updates target column `index` otherwise. private val fieldWriters: Array[WritableComparable[_] => Unit] = { requiredSchema.zipWithIndex - // The value of missing columns are always null, do not need writers. - .filterNot { case (_, index) => requestedColIds(index) == -1 } .map { case (f, index) => - val writer = newWriter(f.dataType, new RowUpdater(resultRow)) - (value: WritableComparable[_]) => writer(index, value) + if (requestedColIds(index) == -1) { + null + } else { + val writer = newWriter(f.dataType, new RowUpdater(resultRow)) + (value: WritableComparable[_]) => writer(index, value) + } }.toArray } - private val validColIds = requestedColIds.filterNot(_ == -1) - def deserialize(orcStruct: OrcStruct): InternalRow = { - var i = 0 - while (i < validColIds.length) { - val value = orcStruct.getFieldValue(validColIds(i)) - if (value == null) { - resultRow.setNullAt(i) - } else { - fieldWriters(i)(value) + var targetColumnIndex = 0 + while (targetColumnIndex < fieldWriters.length) { + if (fieldWriters(targetColumnIndex) != null) { + val value = orcStruct.getFieldValue(requestedColIds(targetColumnIndex)) + if (value == null) { + resultRow.setNullAt(targetColumnIndex) + } else { + fieldWriters(targetColumnIndex)(value) + } } - i += 1 + targetColumnIndex += 1 } resultRow } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala index 23c58e175fe5..de234c14c7b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala @@ -72,6 +72,7 @@ class HeaderCSVReadSchemaSuite class JsonReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest with ChangePositionTest with IntegralTypeTest @@ -84,6 +85,7 @@ class JsonReadSchemaSuite class OrcReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest with ChangePositionTest { @@ -103,6 +105,7 @@ class OrcReadSchemaSuite class VectorizedOrcReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest with ChangePositionTest with BooleanTypeTest @@ -125,6 +128,7 @@ class VectorizedOrcReadSchemaSuite class ParquetReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest with ChangePositionTest { @@ -144,6 +148,7 @@ class ParquetReadSchemaSuite class VectorizedParquetReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest with ChangePositionTest { @@ -163,6 +168,7 @@ class VectorizedParquetReadSchemaSuite class MergedParquetReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest with ChangePositionTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala index 2a5457e00b4e..17d9d43a3e08 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala @@ -69,7 +69,7 @@ trait ReadSchemaTest extends QueryTest with SQLTestUtils with SharedSQLContext { } /** - * Add column (Case 1). + * Add column (Case 1-1). * This test suite assumes that the missing column should be `null`. */ trait AddColumnTest extends ReadSchemaTest { @@ -108,6 +108,43 @@ trait AddColumnTest extends ReadSchemaTest { } } +/** + * Add column into the middle (Case 1-2). + */ +trait AddColumnIntoTheMiddleTest extends ReadSchemaTest { + import testImplicits._ + + test("append column into middle") { + withTempPath { dir => + val path = dir.getCanonicalPath + + val df1 = Seq((1, 2, "abc"), (4, 5, "def"), (8, 9, null)).toDF("col1", "col2", "col3") + val df2 = Seq((10, null, 20, null), (40, "uvw", 50, "xyz"), (80, null, 90, null)) + .toDF("col1", "col4", "col2", "col3") + + val dir1 = s"$path${File.separator}part=one" + val dir2 = s"$path${File.separator}part=two" + + df1.write.format(format).options(options).save(dir1) + df2.write.format(format).options(options).save(dir2) + + val df = spark.read + .schema(df2.schema) + .format(format) + .options(options) + .load(path) + + checkAnswer(df, Seq( + Row(1, null, 2, "abc", "one"), + Row(4, null, 5, "def", "one"), + Row(8, null, 9, null, "one"), + Row(10, null, 20, null, "two"), + Row(40, "uvw", 50, "xyz", "two"), + Row(80, null, 90, null, "two"))) + } + } +} + /** * Hide column (Case 2-1). */ From 061185b9b872a672e3d58f8bbe819f8f70b33f91 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 21 Feb 2019 00:45:49 +0000 Subject: [PATCH 377/879] Preparing Spark release v2.4.1-rc3 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 236128924fd6..714b6f18b19d 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.2 +Version: 2.4.1 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index c9999df354ab..8e11fd687dd8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 092f85bcc32d..f0eee076faae 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 5236fd605fa8..8c8bdf4796b1 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index b70dadf158e2..663f41d1cf6d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index e9ae143fd727..9acade15cf38 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 2ae4fcbbcae1..1a31a39c0dc9 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 6f756dba8612..e3d5dd9004f6 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 6e53256cc011..4845c58c1c2a 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 629777b7b25b..d40baeae2994 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 13b5d8ea4613..7ddafaf76665 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.2-SNAPSHOT -SPARK_VERSION_SHORT: 2.4.2 +SPARK_VERSION: 2.4.1 +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 9ad0eaf53e2a..f5493aa9a287 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 2e6275dd03b6..cf354864a486 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index a4d67c8e8c23..ae87188c9ae9 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index a57e9106bf16..244b90a54690 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index d05b4012a5dc..8966055eb531 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 705f95ffbcbd..b17ee8174fcf 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 0814ab0a807f..df20f5911b13 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 27bb30690e95..e1e8563ef843 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 632c355f10d4..2c0926ebaef1 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index b11912641d96..a2c4dea645f9 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 8941ee7a6192..dd8693689738 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 9256f0bc8333..dce3c6859a97 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 188c8f54a9fa..d864996ce24c 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index e54560e2a586..41ee21cdf293 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c5a8721e1527..70469df92c12 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index e66a22564029..ae997f885420 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 4969316c3f4d..e8e7b6b9728f 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1bdc105855bb..ef2b486de5dd 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index de85b85e7d2c..75f400265c69 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/pom.xml b/pom.xml index 69f56b81ae6b..a28ef6e59ab8 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 02eb2bdfb6d1..d6654524283b 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.2.dev0" +__version__ = "2.4.1" diff --git a/repl/pom.xml b/repl/pom.xml index a0d14e7c74a6..b8844ba601c7 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 788e706073aa..309526a1d461 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 47d15afeb74b..e6b9752a2d99 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 07e17c84878a..5b4d4ef10a0b 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 48ca34fc13b1..eb6bea1de3e0 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index f8bcadf265b5..6aea1c19dea9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 110d4a944604..fd51867b0160 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 7f2d80969aec..55c7c298f27f 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index cb068ed59dc5..d593d1d1e201 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 0e7b59fb2657..5567a55ccbe3 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 7e210ea7393b..73fd3e270c78 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml From 0926f49f407201384dd020073b41e7023465b7e9 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 21 Feb 2019 00:46:07 +0000 Subject: [PATCH 378/879] Preparing development version 2.4.2-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 714b6f18b19d..236128924fd6 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.2 Title: R Frontend for Apache Spark Description: Provides an R Frontend for Apache Spark. Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 8e11fd687dd8..c9999df354ab 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index f0eee076faae..092f85bcc32d 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 8c8bdf4796b1..5236fd605fa8 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 663f41d1cf6d..b70dadf158e2 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 9acade15cf38..e9ae143fd727 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 1a31a39c0dc9..2ae4fcbbcae1 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e3d5dd9004f6..6f756dba8612 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 4845c58c1c2a..6e53256cc011 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index d40baeae2994..629777b7b25b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7ddafaf76665..13b5d8ea4613 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1 -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.2-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.2 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index f5493aa9a287..9ad0eaf53e2a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index cf354864a486..2e6275dd03b6 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ae87188c9ae9..a4d67c8e8c23 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 244b90a54690..a57e9106bf16 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 8966055eb531..d05b4012a5dc 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index b17ee8174fcf..705f95ffbcbd 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index df20f5911b13..0814ab0a807f 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index e1e8563ef843..27bb30690e95 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 2c0926ebaef1..632c355f10d4 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a2c4dea645f9..b11912641d96 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index dd8693689738..8941ee7a6192 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index dce3c6859a97..9256f0bc8333 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index d864996ce24c..188c8f54a9fa 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 41ee21cdf293..e54560e2a586 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 70469df92c12..c5a8721e1527 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index ae997f885420..e66a22564029 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e8e7b6b9728f..4969316c3f4d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index ef2b486de5dd..1bdc105855bb 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 75f400265c69..de85b85e7d2c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index a28ef6e59ab8..69f56b81ae6b 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index d6654524283b..02eb2bdfb6d1 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1" +__version__ = "2.4.2.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index b8844ba601c7..a0d14e7c74a6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 309526a1d461..788e706073aa 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index e6b9752a2d99..47d15afeb74b 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 5b4d4ef10a0b..07e17c84878a 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index eb6bea1de3e0..48ca34fc13b1 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6aea1c19dea9..f8bcadf265b5 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index fd51867b0160..110d4a944604 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 55c7c298f27f..7f2d80969aec 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index d593d1d1e201..cb068ed59dc5 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 5567a55ccbe3..0e7b59fb2657 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 73fd3e270c78..7e210ea7393b 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml From d8576301fd1d33675a9542791e58e7963081ce04 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Thu, 21 Feb 2019 08:42:15 -0800 Subject: [PATCH 379/879] [R][BACKPORT-2.4] update package description #23852 doesn't port cleanly to 2.4. we need this in branch-2.4 and branch-2.3 Author: Felix Cheung Closes #23860 from felixcheung/2.4rdesc. --- R/pkg/DESCRIPTION | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 236128924fd6..5e3d18629432 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,8 +1,8 @@ Package: SparkR Type: Package Version: 2.4.2 -Title: R Frontend for Apache Spark -Description: Provides an R Frontend for Apache Spark. +Title: R Front end for 'Apache Spark' +Description: Provides an R Front end for 'Apache Spark' . Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "shivaram@cs.berkeley.edu"), person("Xiangrui", "Meng", role = "aut", @@ -11,8 +11,8 @@ Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "felixcheung@apache.org"), person(family = "The Apache Software Foundation", role = c("aut", "cph"))) License: Apache License (== 2.0) -URL: http://www.apache.org/ http://spark.apache.org/ -BugReports: http://spark.apache.org/contributing.html +URL: https://www.apache.org/ https://spark.apache.org/ +BugReports: https://spark.apache.org/contributing.html SystemRequirements: Java (== 8) Depends: R (>= 3.0), From 79c1f7e89c6c15704d046fa4d334cacce3d19217 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 21 Feb 2019 23:01:58 +0000 Subject: [PATCH 380/879] Preparing Spark release v2.4.1-rc4 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 5e3d18629432..be924c938c85 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.2 +Version: 2.4.1 Title: R Front end for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index c9999df354ab..8e11fd687dd8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 092f85bcc32d..f0eee076faae 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 5236fd605fa8..8c8bdf4796b1 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index b70dadf158e2..663f41d1cf6d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index e9ae143fd727..9acade15cf38 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 2ae4fcbbcae1..1a31a39c0dc9 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 6f756dba8612..e3d5dd9004f6 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 6e53256cc011..4845c58c1c2a 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 629777b7b25b..d40baeae2994 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 13b5d8ea4613..7ddafaf76665 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.2-SNAPSHOT -SPARK_VERSION_SHORT: 2.4.2 +SPARK_VERSION: 2.4.1 +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 9ad0eaf53e2a..f5493aa9a287 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 2e6275dd03b6..cf354864a486 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index a4d67c8e8c23..ae87188c9ae9 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index a57e9106bf16..244b90a54690 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index d05b4012a5dc..8966055eb531 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 705f95ffbcbd..b17ee8174fcf 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 0814ab0a807f..df20f5911b13 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 27bb30690e95..e1e8563ef843 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 632c355f10d4..2c0926ebaef1 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index b11912641d96..a2c4dea645f9 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 8941ee7a6192..dd8693689738 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 9256f0bc8333..dce3c6859a97 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 188c8f54a9fa..d864996ce24c 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index e54560e2a586..41ee21cdf293 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c5a8721e1527..70469df92c12 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index e66a22564029..ae997f885420 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 4969316c3f4d..e8e7b6b9728f 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1bdc105855bb..ef2b486de5dd 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index de85b85e7d2c..75f400265c69 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/pom.xml b/pom.xml index 69f56b81ae6b..a28ef6e59ab8 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 02eb2bdfb6d1..d6654524283b 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.2.dev0" +__version__ = "2.4.1" diff --git a/repl/pom.xml b/repl/pom.xml index a0d14e7c74a6..b8844ba601c7 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 788e706073aa..309526a1d461 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 47d15afeb74b..e6b9752a2d99 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 07e17c84878a..5b4d4ef10a0b 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 48ca34fc13b1..eb6bea1de3e0 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index f8bcadf265b5..6aea1c19dea9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 110d4a944604..fd51867b0160 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 7f2d80969aec..55c7c298f27f 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index cb068ed59dc5..d593d1d1e201 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 0e7b59fb2657..5567a55ccbe3 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 7e210ea7393b..73fd3e270c78 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml From 32825442283fcd4c244c59ad7c4c9331538e790d Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Thu, 21 Feb 2019 23:02:17 +0000 Subject: [PATCH 381/879] Preparing development version 2.4.2-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index be924c938c85..5e3d18629432 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.2 Title: R Front end for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 8e11fd687dd8..c9999df354ab 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index f0eee076faae..092f85bcc32d 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 8c8bdf4796b1..5236fd605fa8 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 663f41d1cf6d..b70dadf158e2 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 9acade15cf38..e9ae143fd727 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 1a31a39c0dc9..2ae4fcbbcae1 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e3d5dd9004f6..6f756dba8612 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 4845c58c1c2a..6e53256cc011 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index d40baeae2994..629777b7b25b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7ddafaf76665..13b5d8ea4613 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1 -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.2-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.2 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index f5493aa9a287..9ad0eaf53e2a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index cf354864a486..2e6275dd03b6 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ae87188c9ae9..a4d67c8e8c23 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 244b90a54690..a57e9106bf16 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 8966055eb531..d05b4012a5dc 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index b17ee8174fcf..705f95ffbcbd 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index df20f5911b13..0814ab0a807f 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index e1e8563ef843..27bb30690e95 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 2c0926ebaef1..632c355f10d4 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a2c4dea645f9..b11912641d96 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index dd8693689738..8941ee7a6192 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index dce3c6859a97..9256f0bc8333 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index d864996ce24c..188c8f54a9fa 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 41ee21cdf293..e54560e2a586 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 70469df92c12..c5a8721e1527 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index ae997f885420..e66a22564029 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e8e7b6b9728f..4969316c3f4d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index ef2b486de5dd..1bdc105855bb 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 75f400265c69..de85b85e7d2c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index a28ef6e59ab8..69f56b81ae6b 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index d6654524283b..02eb2bdfb6d1 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1" +__version__ = "2.4.2.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index b8844ba601c7..a0d14e7c74a6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 309526a1d461..788e706073aa 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index e6b9752a2d99..47d15afeb74b 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 5b4d4ef10a0b..07e17c84878a 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index eb6bea1de3e0..48ca34fc13b1 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6aea1c19dea9..f8bcadf265b5 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index fd51867b0160..110d4a944604 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 55c7c298f27f..7f2d80969aec 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index d593d1d1e201..cb068ed59dc5 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 5567a55ccbe3..0e7b59fb2657 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 73fd3e270c78..7e210ea7393b 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml From 8d68d54f2e2cbbe55a4bb87c2216cff896add517 Mon Sep 17 00:00:00 2001 From: Felix Cheung Date: Fri, 22 Feb 2019 10:12:38 +0800 Subject: [PATCH 382/879] [R][BACKPORT-2.3] update package description doesn't port cleanly to 2.3. we need this in branch-2.4 and branch-2.3 Closes #23861 from felixcheung/2.3rdesc. Authored-by: Felix Cheung Signed-off-by: Hyukjin Kwon (cherry picked from commit 36db45d5b90ddc3ce54febff2ed41cd29c0a8a04) Signed-off-by: Hyukjin Kwon --- R/append/commits/0 | 2 ++ R/append/commits/1 | 2 ++ R/append/metadata | 1 + R/append/offsets/0 | 3 +++ R/append/offsets/1 | 3 +++ R/append/sources/0/0 | 2 ++ R/append/sources/0/1 | 2 ++ 7 files changed, 15 insertions(+) create mode 100644 R/append/commits/0 create mode 100644 R/append/commits/1 create mode 100644 R/append/metadata create mode 100644 R/append/offsets/0 create mode 100644 R/append/offsets/1 create mode 100644 R/append/sources/0/0 create mode 100644 R/append/sources/0/1 diff --git a/R/append/commits/0 b/R/append/commits/0 new file mode 100644 index 000000000000..9c1e3021c3ea --- /dev/null +++ b/R/append/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/R/append/commits/1 b/R/append/commits/1 new file mode 100644 index 000000000000..9c1e3021c3ea --- /dev/null +++ b/R/append/commits/1 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/R/append/metadata b/R/append/metadata new file mode 100644 index 000000000000..e10d274a2650 --- /dev/null +++ b/R/append/metadata @@ -0,0 +1 @@ +{"id":"816b9eb3-4e0e-4419-aa6b-042fe770fe9e"} \ No newline at end of file diff --git a/R/append/offsets/0 b/R/append/offsets/0 new file mode 100644 index 000000000000..f725b7ed4ef4 --- /dev/null +++ b/R/append/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1550545145189,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"200"}} +{"logOffset":0} \ No newline at end of file diff --git a/R/append/offsets/1 b/R/append/offsets/1 new file mode 100644 index 000000000000..6a8b0cf3f26c --- /dev/null +++ b/R/append/offsets/1 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1550546700082,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"200"}} +{"logOffset":1} \ No newline at end of file diff --git a/R/append/sources/0/0 b/R/append/sources/0/0 new file mode 100644 index 000000000000..72abd6ca2c8c --- /dev/null +++ b/R/append/sources/0/0 @@ -0,0 +1,2 @@ +v1 +{"path":"file:///var/folders/71/484zt4z10ks1vydt03bhp6hr0000gp/T/RtmpYrC5NR/sparkr-test403b46ee34f0.parquet/part-00000-b8e0fa75-2067-4518-abc9-9f187ef289c4-c000.snappy.parquet","timestamp":1550545144000,"batchId":0} \ No newline at end of file diff --git a/R/append/sources/0/1 b/R/append/sources/0/1 new file mode 100644 index 000000000000..b336c6bcbf67 --- /dev/null +++ b/R/append/sources/0/1 @@ -0,0 +1,2 @@ +v1 +{"path":"file:///var/folders/71/484zt4z10ks1vydt03bhp6hr0000gp/T/RtmpDDmJpK/sparkr-testb1994d9aae56.parquet/part-00000-9f3a8856-ef41-47d7-86a1-6f5a9ae8501d-c000.snappy.parquet","timestamp":1550546699000,"batchId":1} \ No newline at end of file From b40361249f8b9dae2ade1e8579c9a271e248b5a9 Mon Sep 17 00:00:00 2001 From: Hyukjin Kwon Date: Fri, 22 Feb 2019 10:14:56 +0800 Subject: [PATCH 383/879] Revert "[R][BACKPORT-2.3] update package description" This reverts commit 8d68d54f2e2cbbe55a4bb87c2216cff896add517. --- R/append/commits/0 | 2 -- R/append/commits/1 | 2 -- R/append/metadata | 1 - R/append/offsets/0 | 3 --- R/append/offsets/1 | 3 --- R/append/sources/0/0 | 2 -- R/append/sources/0/1 | 2 -- 7 files changed, 15 deletions(-) delete mode 100644 R/append/commits/0 delete mode 100644 R/append/commits/1 delete mode 100644 R/append/metadata delete mode 100644 R/append/offsets/0 delete mode 100644 R/append/offsets/1 delete mode 100644 R/append/sources/0/0 delete mode 100644 R/append/sources/0/1 diff --git a/R/append/commits/0 b/R/append/commits/0 deleted file mode 100644 index 9c1e3021c3ea..000000000000 --- a/R/append/commits/0 +++ /dev/null @@ -1,2 +0,0 @@ -v1 -{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/R/append/commits/1 b/R/append/commits/1 deleted file mode 100644 index 9c1e3021c3ea..000000000000 --- a/R/append/commits/1 +++ /dev/null @@ -1,2 +0,0 @@ -v1 -{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/R/append/metadata b/R/append/metadata deleted file mode 100644 index e10d274a2650..000000000000 --- a/R/append/metadata +++ /dev/null @@ -1 +0,0 @@ -{"id":"816b9eb3-4e0e-4419-aa6b-042fe770fe9e"} \ No newline at end of file diff --git a/R/append/offsets/0 b/R/append/offsets/0 deleted file mode 100644 index f725b7ed4ef4..000000000000 --- a/R/append/offsets/0 +++ /dev/null @@ -1,3 +0,0 @@ -v1 -{"batchWatermarkMs":0,"batchTimestampMs":1550545145189,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"200"}} -{"logOffset":0} \ No newline at end of file diff --git a/R/append/offsets/1 b/R/append/offsets/1 deleted file mode 100644 index 6a8b0cf3f26c..000000000000 --- a/R/append/offsets/1 +++ /dev/null @@ -1,3 +0,0 @@ -v1 -{"batchWatermarkMs":0,"batchTimestampMs":1550546700082,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"200"}} -{"logOffset":1} \ No newline at end of file diff --git a/R/append/sources/0/0 b/R/append/sources/0/0 deleted file mode 100644 index 72abd6ca2c8c..000000000000 --- a/R/append/sources/0/0 +++ /dev/null @@ -1,2 +0,0 @@ -v1 -{"path":"file:///var/folders/71/484zt4z10ks1vydt03bhp6hr0000gp/T/RtmpYrC5NR/sparkr-test403b46ee34f0.parquet/part-00000-b8e0fa75-2067-4518-abc9-9f187ef289c4-c000.snappy.parquet","timestamp":1550545144000,"batchId":0} \ No newline at end of file diff --git a/R/append/sources/0/1 b/R/append/sources/0/1 deleted file mode 100644 index b336c6bcbf67..000000000000 --- a/R/append/sources/0/1 +++ /dev/null @@ -1,2 +0,0 @@ -v1 -{"path":"file:///var/folders/71/484zt4z10ks1vydt03bhp6hr0000gp/T/RtmpDDmJpK/sparkr-testb1994d9aae56.parquet/part-00000-9f3a8856-ef41-47d7-86a1-6f5a9ae8501d-c000.snappy.parquet","timestamp":1550546699000,"batchId":1} \ No newline at end of file From ef67be363be6d6b6954b55ef1c243a0672b84abb Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Fri, 22 Feb 2019 12:25:26 +0800 Subject: [PATCH 384/879] [SPARK-26950][SQL][TEST] Make RandomDataGenerator use Float.NaN or Double.NaN for all NaN values ## What changes were proposed in this pull request? Apache Spark uses the predefined `Float.NaN` and `Double.NaN` for NaN values, but there exists more NaN values with different binary presentations. ```scala scala> java.nio.ByteBuffer.allocate(4).putFloat(Float.NaN).array res1: Array[Byte] = Array(127, -64, 0, 0) scala> val x = java.lang.Float.intBitsToFloat(-6966608) x: Float = NaN scala> java.nio.ByteBuffer.allocate(4).putFloat(x).array res2: Array[Byte] = Array(-1, -107, -78, -80) ``` Since users can have these values, `RandomDataGenerator` generates these NaN values. However, this causes `checkEvaluationWithUnsafeProjection` failures due to the difference between `UnsafeRow` binary presentation. The following is the UT failure instance. This PR aims to fix this UT flakiness. - https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/102528/testReport/ ## How was this patch tested? Pass the Jenkins with the newly added test cases. Closes #23851 from dongjoon-hyun/SPARK-26950. Authored-by: Dongjoon Hyun Signed-off-by: Wenchen Fan (cherry picked from commit ffef3d40741b0be321421aa52a6e17a26d89f541) Signed-off-by: Wenchen Fan --- .../spark/sql/RandomDataGenerator.scala | 24 ++++++++++++-- .../spark/sql/RandomDataGeneratorSuite.scala | 31 +++++++++++++++++++ 2 files changed, 53 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala index 8ae3ff5043e6..d361e6248e2f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql -import java.lang.Double.longBitsToDouble -import java.lang.Float.intBitsToFloat import java.math.MathContext import scala.collection.mutable @@ -69,6 +67,28 @@ object RandomDataGenerator { Some(f) } + /** + * A wrapper of Float.intBitsToFloat to use a unique NaN value for all NaN values. + * This prevents `checkEvaluationWithUnsafeProjection` from failing due to + * the difference between `UnsafeRow` binary presentation for NaN. + * This is visible for testing. + */ + def intBitsToFloat(bits: Int): Float = { + val value = java.lang.Float.intBitsToFloat(bits) + if (value.isNaN) Float.NaN else value + } + + /** + * A wrapper of Double.longBitsToDouble to use a unique NaN value for all NaN values. + * This prevents `checkEvaluationWithUnsafeProjection` from failing due to + * the difference between `UnsafeRow` binary presentation for NaN. + * This is visible for testing. + */ + def longBitsToDouble(bits: Long): Double = { + val value = java.lang.Double.longBitsToDouble(bits) + if (value.isNaN) Double.NaN else value + } + /** * Returns a randomly generated schema, based on the given accepted types. * diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala index 3c2f8a28875f..3e62ca069e9e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql +import java.nio.ByteBuffer +import java.util.Arrays + import scala.util.Random import org.apache.spark.SparkFunSuite @@ -106,4 +109,32 @@ class RandomDataGeneratorSuite extends SparkFunSuite { assert(deviation.toDouble / expectedTotalElements < 2e-1) } } + + test("Use Float.NaN for all NaN values") { + val bits = -6966608 + val nan1 = java.lang.Float.intBitsToFloat(bits) + val nan2 = RandomDataGenerator.intBitsToFloat(bits) + assert(nan1.isNaN) + assert(nan2.isNaN) + + val arrayExpected = ByteBuffer.allocate(4).putFloat(Float.NaN).array + val array1 = ByteBuffer.allocate(4).putFloat(nan1).array + val array2 = ByteBuffer.allocate(4).putFloat(nan2).array + assert(!Arrays.equals(array1, arrayExpected)) + assert(Arrays.equals(array2, arrayExpected)) + } + + test("Use Double.NaN for all NaN values") { + val bits = -6966608 + val nan1 = java.lang.Double.longBitsToDouble(bits) + val nan2 = RandomDataGenerator.longBitsToDouble(bits) + assert(nan1.isNaN) + assert(nan2.isNaN) + + val arrayExpected = ByteBuffer.allocate(8).putDouble(Double.NaN).array + val array1 = ByteBuffer.allocate(8).putDouble(nan1).array + val array2 = ByteBuffer.allocate(8).putDouble(nan2).array + assert(!Arrays.equals(array1, arrayExpected)) + assert(Arrays.equals(array2, arrayExpected)) + } } From eb2af242260e6f6f2c4ac8fdf11d171333101a4f Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Fri, 22 Feb 2019 22:54:15 +0000 Subject: [PATCH 385/879] Preparing Spark release v2.4.1-rc5 --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 5e3d18629432..be924c938c85 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.2 +Version: 2.4.1 Title: R Front end for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index c9999df354ab..8e11fd687dd8 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 092f85bcc32d..f0eee076faae 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 5236fd605fa8..8c8bdf4796b1 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index b70dadf158e2..663f41d1cf6d 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index e9ae143fd727..9acade15cf38 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 2ae4fcbbcae1..1a31a39c0dc9 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index 6f756dba8612..e3d5dd9004f6 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 6e53256cc011..4845c58c1c2a 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index 629777b7b25b..d40baeae2994 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 13b5d8ea4613..7ddafaf76665 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.2-SNAPSHOT -SPARK_VERSION_SHORT: 2.4.2 +SPARK_VERSION: 2.4.1 +SPARK_VERSION_SHORT: 2.4.1 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index 9ad0eaf53e2a..f5493aa9a287 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 2e6275dd03b6..cf354864a486 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index a4d67c8e8c23..ae87188c9ae9 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index a57e9106bf16..244b90a54690 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index d05b4012a5dc..8966055eb531 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 705f95ffbcbd..b17ee8174fcf 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 0814ab0a807f..df20f5911b13 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index 27bb30690e95..e1e8563ef843 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 632c355f10d4..2c0926ebaef1 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index b11912641d96..a2c4dea645f9 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index 8941ee7a6192..dd8693689738 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 9256f0bc8333..dce3c6859a97 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 188c8f54a9fa..d864996ce24c 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index e54560e2a586..41ee21cdf293 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index c5a8721e1527..70469df92c12 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index e66a22564029..ae997f885420 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index 4969316c3f4d..e8e7b6b9728f 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index 1bdc105855bb..ef2b486de5dd 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index de85b85e7d2c..75f400265c69 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/pom.xml b/pom.xml index 69f56b81ae6b..a28ef6e59ab8 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index 02eb2bdfb6d1..d6654524283b 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.2.dev0" +__version__ = "2.4.1" diff --git a/repl/pom.xml b/repl/pom.xml index a0d14e7c74a6..b8844ba601c7 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 788e706073aa..309526a1d461 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 47d15afeb74b..e6b9752a2d99 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 07e17c84878a..5b4d4ef10a0b 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index 48ca34fc13b1..eb6bea1de3e0 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index f8bcadf265b5..6aea1c19dea9 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 110d4a944604..fd51867b0160 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 7f2d80969aec..55c7c298f27f 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index cb068ed59dc5..d593d1d1e201 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 0e7b59fb2657..5567a55ccbe3 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 7e210ea7393b..73fd3e270c78 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.2-SNAPSHOT + 2.4.1 ../pom.xml From 073c47beeb468328dd88b7559d584d098ab6661f Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Fri, 22 Feb 2019 22:54:37 +0000 Subject: [PATCH 386/879] Preparing development version 2.4.2-SNAPSHOT --- R/pkg/DESCRIPTION | 2 +- assembly/pom.xml | 2 +- common/kvstore/pom.xml | 2 +- common/network-common/pom.xml | 2 +- common/network-shuffle/pom.xml | 2 +- common/network-yarn/pom.xml | 2 +- common/sketch/pom.xml | 2 +- common/tags/pom.xml | 2 +- common/unsafe/pom.xml | 2 +- core/pom.xml | 2 +- docs/_config.yml | 4 ++-- examples/pom.xml | 2 +- external/avro/pom.xml | 2 +- external/docker-integration-tests/pom.xml | 2 +- external/flume-assembly/pom.xml | 2 +- external/flume-sink/pom.xml | 2 +- external/flume/pom.xml | 2 +- external/kafka-0-10-assembly/pom.xml | 2 +- external/kafka-0-10-sql/pom.xml | 2 +- external/kafka-0-10/pom.xml | 2 +- external/kafka-0-8-assembly/pom.xml | 2 +- external/kafka-0-8/pom.xml | 2 +- external/kinesis-asl-assembly/pom.xml | 2 +- external/kinesis-asl/pom.xml | 2 +- external/spark-ganglia-lgpl/pom.xml | 2 +- graphx/pom.xml | 2 +- hadoop-cloud/pom.xml | 2 +- launcher/pom.xml | 2 +- mllib-local/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 2 +- python/pyspark/version.py | 2 +- repl/pom.xml | 2 +- resource-managers/kubernetes/core/pom.xml | 2 +- resource-managers/kubernetes/integration-tests/pom.xml | 2 +- resource-managers/mesos/pom.xml | 2 +- resource-managers/yarn/pom.xml | 2 +- sql/catalyst/pom.xml | 2 +- sql/core/pom.xml | 2 +- sql/hive-thriftserver/pom.xml | 2 +- sql/hive/pom.xml | 2 +- streaming/pom.xml | 2 +- tools/pom.xml | 2 +- 43 files changed, 44 insertions(+), 44 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index be924c938c85..5e3d18629432 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,6 +1,6 @@ Package: SparkR Type: Package -Version: 2.4.1 +Version: 2.4.2 Title: R Front end for 'Apache Spark' Description: Provides an R Front end for 'Apache Spark' . Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), diff --git a/assembly/pom.xml b/assembly/pom.xml index 8e11fd687dd8..c9999df354ab 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index f0eee076faae..092f85bcc32d 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 8c8bdf4796b1..5236fd605fa8 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index 663f41d1cf6d..b70dadf158e2 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index 9acade15cf38..e9ae143fd727 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index 1a31a39c0dc9..2ae4fcbbcae1 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/tags/pom.xml b/common/tags/pom.xml index e3d5dd9004f6..6f756dba8612 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 4845c58c1c2a..6e53256cc011 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/core/pom.xml b/core/pom.xml index d40baeae2994..629777b7b25b 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/docs/_config.yml b/docs/_config.yml index 7ddafaf76665..13b5d8ea4613 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -14,8 +14,8 @@ include: # These allow the documentation to be updated with newer releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 2.4.1 -SPARK_VERSION_SHORT: 2.4.1 +SPARK_VERSION: 2.4.2-SNAPSHOT +SPARK_VERSION_SHORT: 2.4.2 SCALA_BINARY_VERSION: "2.11" SCALA_VERSION: "2.11.12" MESOS_VERSION: 1.0.0 diff --git a/examples/pom.xml b/examples/pom.xml index f5493aa9a287..9ad0eaf53e2a 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/external/avro/pom.xml b/external/avro/pom.xml index cf354864a486..2e6275dd03b6 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index ae87188c9ae9..a4d67c8e8c23 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-assembly/pom.xml b/external/flume-assembly/pom.xml index 244b90a54690..a57e9106bf16 100644 --- a/external/flume-assembly/pom.xml +++ b/external/flume-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index 8966055eb531..d05b4012a5dc 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/flume/pom.xml b/external/flume/pom.xml index b17ee8174fcf..705f95ffbcbd 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index df20f5911b13..0814ab0a807f 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index e1e8563ef843..27bb30690e95 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index 2c0926ebaef1..632c355f10d4 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml index a2c4dea645f9..b11912641d96 100644 --- a/external/kafka-0-8-assembly/pom.xml +++ b/external/kafka-0-8-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-8/pom.xml index dd8693689738..8941ee7a6192 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-8/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index dce3c6859a97..9256f0bc8333 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index d864996ce24c..188c8f54a9fa 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 41ee21cdf293..e54560e2a586 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/graphx/pom.xml b/graphx/pom.xml index 70469df92c12..c5a8721e1527 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index ae997f885420..e66a22564029 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/launcher/pom.xml b/launcher/pom.xml index e8e7b6b9728f..4969316c3f4d 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/mllib-local/pom.xml b/mllib-local/pom.xml index ef2b486de5dd..1bdc105855bb 100644 --- a/mllib-local/pom.xml +++ b/mllib-local/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 75f400265c69..de85b85e7d2c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/pom.xml b/pom.xml index a28ef6e59ab8..69f56b81ae6b 100644 --- a/pom.xml +++ b/pom.xml @@ -26,7 +26,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT pom Spark Project Parent POM http://spark.apache.org/ diff --git a/python/pyspark/version.py b/python/pyspark/version.py index d6654524283b..02eb2bdfb6d1 100644 --- a/python/pyspark/version.py +++ b/python/pyspark/version.py @@ -16,4 +16,4 @@ # See the License for the specific language governing permissions and # limitations under the License. -__version__ = "2.4.1" +__version__ = "2.4.2.dev0" diff --git a/repl/pom.xml b/repl/pom.xml index b8844ba601c7..a0d14e7c74a6 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 309526a1d461..788e706073aa 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index e6b9752a2d99..47d15afeb74b 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../../pom.xml diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 5b4d4ef10a0b..07e17c84878a 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index eb6bea1de3e0..48ca34fc13b1 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 6aea1c19dea9..f8bcadf265b5 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/core/pom.xml b/sql/core/pom.xml index fd51867b0160..110d4a944604 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 55c7c298f27f..7f2d80969aec 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index d593d1d1e201..cb068ed59dc5 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -22,7 +22,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../../pom.xml diff --git a/streaming/pom.xml b/streaming/pom.xml index 5567a55ccbe3..0e7b59fb2657 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml diff --git a/tools/pom.xml b/tools/pom.xml index 73fd3e270c78..7e210ea7393b 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent_2.11 - 2.4.1 + 2.4.2-SNAPSHOT ../pom.xml From b031f4a481f42e70c3cc00137430c04f50e8acd1 Mon Sep 17 00:00:00 2001 From: "Jungtaek Lim (HeartSaVioR)" Date: Mon, 25 Feb 2019 11:25:53 -0800 Subject: [PATCH 387/879] [MINOR][BUILD] Update all checkstyle dtd to use "https://checkstyle.org" ## What changes were proposed in this pull request? Below build failed with Java checkstyle test, but instead of violation it shows FileNotFound on dtd file. https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/102751/ Looks like the link of dtd file is dead `http://www.puppycrawl.com/dtds/configuration_1_3.dtd`. This patch updates the dtd link to "https://checkstyle.org/dtds/" given checkstyle repository also updated the URL path. https://github.com/checkstyle/checkstyle/issues/5601 ## How was this patch tested? Checked the new links. Closes #23887 from HeartSaVioR/java-checkstyle-dtd-change-url. Authored-by: Jungtaek Lim (HeartSaVioR) Signed-off-by: Marcelo Vanzin (cherry picked from commit c5de804093540509929f6de211dbbe644b33e6db) Signed-off-by: Marcelo Vanzin --- dev/checkstyle-suppressions.xml | 2 +- dev/checkstyle.xml | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/dev/checkstyle-suppressions.xml b/dev/checkstyle-suppressions.xml index bbda824dd13b..945686de4996 100644 --- a/dev/checkstyle-suppressions.xml +++ b/dev/checkstyle-suppressions.xml @@ -17,7 +17,7 @@ +"https://checkstyle.org/dtds/suppressions_1_1.dtd"> - + diff --git a/docs/js/vendor/jquery-1.12.4.min.js b/docs/js/vendor/jquery-1.12.4.min.js new file mode 100755 index 000000000000..e836475870da --- /dev/null +++ b/docs/js/vendor/jquery-1.12.4.min.js @@ -0,0 +1,5 @@ +/*! jQuery v1.12.4 | (c) jQuery Foundation | jquery.org/license */ +!function(a,b){"object"==typeof module&&"object"==typeof module.exports?module.exports=a.document?b(a,!0):function(a){if(!a.document)throw new Error("jQuery requires a window with a document");return b(a)}:b(a)}("undefined"!=typeof window?window:this,function(a,b){var c=[],d=a.document,e=c.slice,f=c.concat,g=c.push,h=c.indexOf,i={},j=i.toString,k=i.hasOwnProperty,l={},m="1.12.4",n=function(a,b){return new n.fn.init(a,b)},o=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g,p=/^-ms-/,q=/-([\da-z])/gi,r=function(a,b){return b.toUpperCase()};n.fn=n.prototype={jquery:m,constructor:n,selector:"",length:0,toArray:function(){return e.call(this)},get:function(a){return null!=a?0>a?this[a+this.length]:this[a]:e.call(this)},pushStack:function(a){var b=n.merge(this.constructor(),a);return b.prevObject=this,b.context=this.context,b},each:function(a){return n.each(this,a)},map:function(a){return this.pushStack(n.map(this,function(b,c){return a.call(b,c,b)}))},slice:function(){return this.pushStack(e.apply(this,arguments))},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},eq:function(a){var b=this.length,c=+a+(0>a?b:0);return this.pushStack(c>=0&&b>c?[this[c]]:[])},end:function(){return this.prevObject||this.constructor()},push:g,sort:c.sort,splice:c.splice},n.extend=n.fn.extend=function(){var a,b,c,d,e,f,g=arguments[0]||{},h=1,i=arguments.length,j=!1;for("boolean"==typeof g&&(j=g,g=arguments[h]||{},h++),"object"==typeof g||n.isFunction(g)||(g={}),h===i&&(g=this,h--);i>h;h++)if(null!=(e=arguments[h]))for(d in e)a=g[d],c=e[d],g!==c&&(j&&c&&(n.isPlainObject(c)||(b=n.isArray(c)))?(b?(b=!1,f=a&&n.isArray(a)?a:[]):f=a&&n.isPlainObject(a)?a:{},g[d]=n.extend(j,f,c)):void 0!==c&&(g[d]=c));return g},n.extend({expando:"jQuery"+(m+Math.random()).replace(/\D/g,""),isReady:!0,error:function(a){throw new Error(a)},noop:function(){},isFunction:function(a){return"function"===n.type(a)},isArray:Array.isArray||function(a){return"array"===n.type(a)},isWindow:function(a){return null!=a&&a==a.window},isNumeric:function(a){var b=a&&a.toString();return!n.isArray(a)&&b-parseFloat(b)+1>=0},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},isPlainObject:function(a){var b;if(!a||"object"!==n.type(a)||a.nodeType||n.isWindow(a))return!1;try{if(a.constructor&&!k.call(a,"constructor")&&!k.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}if(!l.ownFirst)for(b in a)return k.call(a,b);for(b in a);return void 0===b||k.call(a,b)},type:function(a){return null==a?a+"":"object"==typeof a||"function"==typeof a?i[j.call(a)]||"object":typeof a},globalEval:function(b){b&&n.trim(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(p,"ms-").replace(q,r)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toLowerCase()===b.toLowerCase()},each:function(a,b){var c,d=0;if(s(a)){for(c=a.length;c>d;d++)if(b.call(a[d],d,a[d])===!1)break}else for(d in a)if(b.call(a[d],d,a[d])===!1)break;return a},trim:function(a){return null==a?"":(a+"").replace(o,"")},makeArray:function(a,b){var c=b||[];return null!=a&&(s(Object(a))?n.merge(c,"string"==typeof a?[a]:a):g.call(c,a)),c},inArray:function(a,b,c){var d;if(b){if(h)return h.call(b,a,c);for(d=b.length,c=c?0>c?Math.max(0,d+c):c:0;d>c;c++)if(c in b&&b[c]===a)return c}return-1},merge:function(a,b){var c=+b.length,d=0,e=a.length;while(c>d)a[e++]=b[d++];if(c!==c)while(void 0!==b[d])a[e++]=b[d++];return a.length=e,a},grep:function(a,b,c){for(var d,e=[],f=0,g=a.length,h=!c;g>f;f++)d=!b(a[f],f),d!==h&&e.push(a[f]);return e},map:function(a,b,c){var d,e,g=0,h=[];if(s(a))for(d=a.length;d>g;g++)e=b(a[g],g,c),null!=e&&h.push(e);else for(g in a)e=b(a[g],g,c),null!=e&&h.push(e);return f.apply([],h)},guid:1,proxy:function(a,b){var c,d,f;return"string"==typeof b&&(f=a[b],b=a,a=f),n.isFunction(a)?(c=e.call(arguments,2),d=function(){return a.apply(b||this,c.concat(e.call(arguments)))},d.guid=a.guid=a.guid||n.guid++,d):void 0},now:function(){return+new Date},support:l}),"function"==typeof Symbol&&(n.fn[Symbol.iterator]=c[Symbol.iterator]),n.each("Boolean Number String Function Array Date RegExp Object Error Symbol".split(" "),function(a,b){i["[object "+b+"]"]=b.toLowerCase()});function s(a){var b=!!a&&"length"in a&&a.length,c=n.type(a);return"function"===c||n.isWindow(a)?!1:"array"===c||0===b||"number"==typeof b&&b>0&&b-1 in a}var t=function(a){var b,c,d,e,f,g,h,i,j,k,l,m,n,o,p,q,r,s,t,u="sizzle"+1*new Date,v=a.document,w=0,x=0,y=ga(),z=ga(),A=ga(),B=function(a,b){return a===b&&(l=!0),0},C=1<<31,D={}.hasOwnProperty,E=[],F=E.pop,G=E.push,H=E.push,I=E.slice,J=function(a,b){for(var c=0,d=a.length;d>c;c++)if(a[c]===b)return c;return-1},K="checked|selected|async|autofocus|autoplay|controls|defer|disabled|hidden|ismap|loop|multiple|open|readonly|required|scoped",L="[\\x20\\t\\r\\n\\f]",M="(?:\\\\.|[\\w-]|[^\\x00-\\xa0])+",N="\\["+L+"*("+M+")(?:"+L+"*([*^$|!~]?=)"+L+"*(?:'((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\"|("+M+"))|)"+L+"*\\]",O=":("+M+")(?:\\((('((?:\\\\.|[^\\\\'])*)'|\"((?:\\\\.|[^\\\\\"])*)\")|((?:\\\\.|[^\\\\()[\\]]|"+N+")*)|.*)\\)|)",P=new RegExp(L+"+","g"),Q=new RegExp("^"+L+"+|((?:^|[^\\\\])(?:\\\\.)*)"+L+"+$","g"),R=new RegExp("^"+L+"*,"+L+"*"),S=new RegExp("^"+L+"*([>+~]|"+L+")"+L+"*"),T=new RegExp("="+L+"*([^\\]'\"]*?)"+L+"*\\]","g"),U=new RegExp(O),V=new RegExp("^"+M+"$"),W={ID:new RegExp("^#("+M+")"),CLASS:new RegExp("^\\.("+M+")"),TAG:new RegExp("^("+M+"|[*])"),ATTR:new RegExp("^"+N),PSEUDO:new RegExp("^"+O),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+L+"*(even|odd|(([+-]|)(\\d*)n|)"+L+"*(?:([+-]|)"+L+"*(\\d+)|))"+L+"*\\)|)","i"),bool:new RegExp("^(?:"+K+")$","i"),needsContext:new RegExp("^"+L+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+L+"*((?:-\\d)?\\d*)"+L+"*\\)|)(?=[^-]|$)","i")},X=/^(?:input|select|textarea|button)$/i,Y=/^h\d$/i,Z=/^[^{]+\{\s*\[native \w/,$=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,_=/[+~]/,aa=/'|\\/g,ba=new RegExp("\\\\([\\da-f]{1,6}"+L+"?|("+L+")|.)","ig"),ca=function(a,b,c){var d="0x"+b-65536;return d!==d||c?b:0>d?String.fromCharCode(d+65536):String.fromCharCode(d>>10|55296,1023&d|56320)},da=function(){m()};try{H.apply(E=I.call(v.childNodes),v.childNodes),E[v.childNodes.length].nodeType}catch(ea){H={apply:E.length?function(a,b){G.apply(a,I.call(b))}:function(a,b){var c=a.length,d=0;while(a[c++]=b[d++]);a.length=c-1}}}function fa(a,b,d,e){var f,h,j,k,l,o,r,s,w=b&&b.ownerDocument,x=b?b.nodeType:9;if(d=d||[],"string"!=typeof a||!a||1!==x&&9!==x&&11!==x)return d;if(!e&&((b?b.ownerDocument||b:v)!==n&&m(b),b=b||n,p)){if(11!==x&&(o=$.exec(a)))if(f=o[1]){if(9===x){if(!(j=b.getElementById(f)))return d;if(j.id===f)return d.push(j),d}else if(w&&(j=w.getElementById(f))&&t(b,j)&&j.id===f)return d.push(j),d}else{if(o[2])return H.apply(d,b.getElementsByTagName(a)),d;if((f=o[3])&&c.getElementsByClassName&&b.getElementsByClassName)return H.apply(d,b.getElementsByClassName(f)),d}if(c.qsa&&!A[a+" "]&&(!q||!q.test(a))){if(1!==x)w=b,s=a;else if("object"!==b.nodeName.toLowerCase()){(k=b.getAttribute("id"))?k=k.replace(aa,"\\$&"):b.setAttribute("id",k=u),r=g(a),h=r.length,l=V.test(k)?"#"+k:"[id='"+k+"']";while(h--)r[h]=l+" "+qa(r[h]);s=r.join(","),w=_.test(a)&&oa(b.parentNode)||b}if(s)try{return H.apply(d,w.querySelectorAll(s)),d}catch(y){}finally{k===u&&b.removeAttribute("id")}}}return i(a.replace(Q,"$1"),b,d,e)}function ga(){var a=[];function b(c,e){return a.push(c+" ")>d.cacheLength&&delete b[a.shift()],b[c+" "]=e}return b}function ha(a){return a[u]=!0,a}function ia(a){var b=n.createElement("div");try{return!!a(b)}catch(c){return!1}finally{b.parentNode&&b.parentNode.removeChild(b),b=null}}function ja(a,b){var c=a.split("|"),e=c.length;while(e--)d.attrHandle[c[e]]=b}function ka(a,b){var c=b&&a,d=c&&1===a.nodeType&&1===b.nodeType&&(~b.sourceIndex||C)-(~a.sourceIndex||C);if(d)return d;if(c)while(c=c.nextSibling)if(c===b)return-1;return a?1:-1}function la(a){return function(b){var c=b.nodeName.toLowerCase();return"input"===c&&b.type===a}}function ma(a){return function(b){var c=b.nodeName.toLowerCase();return("input"===c||"button"===c)&&b.type===a}}function na(a){return ha(function(b){return b=+b,ha(function(c,d){var e,f=a([],c.length,b),g=f.length;while(g--)c[e=f[g]]&&(c[e]=!(d[e]=c[e]))})})}function oa(a){return a&&"undefined"!=typeof a.getElementsByTagName&&a}c=fa.support={},f=fa.isXML=function(a){var b=a&&(a.ownerDocument||a).documentElement;return b?"HTML"!==b.nodeName:!1},m=fa.setDocument=function(a){var b,e,g=a?a.ownerDocument||a:v;return g!==n&&9===g.nodeType&&g.documentElement?(n=g,o=n.documentElement,p=!f(n),(e=n.defaultView)&&e.top!==e&&(e.addEventListener?e.addEventListener("unload",da,!1):e.attachEvent&&e.attachEvent("onunload",da)),c.attributes=ia(function(a){return a.className="i",!a.getAttribute("className")}),c.getElementsByTagName=ia(function(a){return a.appendChild(n.createComment("")),!a.getElementsByTagName("*").length}),c.getElementsByClassName=Z.test(n.getElementsByClassName),c.getById=ia(function(a){return o.appendChild(a).id=u,!n.getElementsByName||!n.getElementsByName(u).length}),c.getById?(d.find.ID=function(a,b){if("undefined"!=typeof b.getElementById&&p){var c=b.getElementById(a);return c?[c]:[]}},d.filter.ID=function(a){var b=a.replace(ba,ca);return function(a){return a.getAttribute("id")===b}}):(delete d.find.ID,d.filter.ID=function(a){var b=a.replace(ba,ca);return function(a){var c="undefined"!=typeof a.getAttributeNode&&a.getAttributeNode("id");return c&&c.value===b}}),d.find.TAG=c.getElementsByTagName?function(a,b){return"undefined"!=typeof b.getElementsByTagName?b.getElementsByTagName(a):c.qsa?b.querySelectorAll(a):void 0}:function(a,b){var c,d=[],e=0,f=b.getElementsByTagName(a);if("*"===a){while(c=f[e++])1===c.nodeType&&d.push(c);return d}return f},d.find.CLASS=c.getElementsByClassName&&function(a,b){return"undefined"!=typeof b.getElementsByClassName&&p?b.getElementsByClassName(a):void 0},r=[],q=[],(c.qsa=Z.test(n.querySelectorAll))&&(ia(function(a){o.appendChild(a).innerHTML="",a.querySelectorAll("[msallowcapture^='']").length&&q.push("[*^$]="+L+"*(?:''|\"\")"),a.querySelectorAll("[selected]").length||q.push("\\["+L+"*(?:value|"+K+")"),a.querySelectorAll("[id~="+u+"-]").length||q.push("~="),a.querySelectorAll(":checked").length||q.push(":checked"),a.querySelectorAll("a#"+u+"+*").length||q.push(".#.+[+~]")}),ia(function(a){var b=n.createElement("input");b.setAttribute("type","hidden"),a.appendChild(b).setAttribute("name","D"),a.querySelectorAll("[name=d]").length&&q.push("name"+L+"*[*^$|!~]?="),a.querySelectorAll(":enabled").length||q.push(":enabled",":disabled"),a.querySelectorAll("*,:x"),q.push(",.*:")})),(c.matchesSelector=Z.test(s=o.matches||o.webkitMatchesSelector||o.mozMatchesSelector||o.oMatchesSelector||o.msMatchesSelector))&&ia(function(a){c.disconnectedMatch=s.call(a,"div"),s.call(a,"[s!='']:x"),r.push("!=",O)}),q=q.length&&new RegExp(q.join("|")),r=r.length&&new RegExp(r.join("|")),b=Z.test(o.compareDocumentPosition),t=b||Z.test(o.contains)?function(a,b){var c=9===a.nodeType?a.documentElement:a,d=b&&b.parentNode;return a===d||!(!d||1!==d.nodeType||!(c.contains?c.contains(d):a.compareDocumentPosition&&16&a.compareDocumentPosition(d)))}:function(a,b){if(b)while(b=b.parentNode)if(b===a)return!0;return!1},B=b?function(a,b){if(a===b)return l=!0,0;var d=!a.compareDocumentPosition-!b.compareDocumentPosition;return d?d:(d=(a.ownerDocument||a)===(b.ownerDocument||b)?a.compareDocumentPosition(b):1,1&d||!c.sortDetached&&b.compareDocumentPosition(a)===d?a===n||a.ownerDocument===v&&t(v,a)?-1:b===n||b.ownerDocument===v&&t(v,b)?1:k?J(k,a)-J(k,b):0:4&d?-1:1)}:function(a,b){if(a===b)return l=!0,0;var c,d=0,e=a.parentNode,f=b.parentNode,g=[a],h=[b];if(!e||!f)return a===n?-1:b===n?1:e?-1:f?1:k?J(k,a)-J(k,b):0;if(e===f)return ka(a,b);c=a;while(c=c.parentNode)g.unshift(c);c=b;while(c=c.parentNode)h.unshift(c);while(g[d]===h[d])d++;return d?ka(g[d],h[d]):g[d]===v?-1:h[d]===v?1:0},n):n},fa.matches=function(a,b){return fa(a,null,null,b)},fa.matchesSelector=function(a,b){if((a.ownerDocument||a)!==n&&m(a),b=b.replace(T,"='$1']"),c.matchesSelector&&p&&!A[b+" "]&&(!r||!r.test(b))&&(!q||!q.test(b)))try{var d=s.call(a,b);if(d||c.disconnectedMatch||a.document&&11!==a.document.nodeType)return d}catch(e){}return fa(b,n,null,[a]).length>0},fa.contains=function(a,b){return(a.ownerDocument||a)!==n&&m(a),t(a,b)},fa.attr=function(a,b){(a.ownerDocument||a)!==n&&m(a);var e=d.attrHandle[b.toLowerCase()],f=e&&D.call(d.attrHandle,b.toLowerCase())?e(a,b,!p):void 0;return void 0!==f?f:c.attributes||!p?a.getAttribute(b):(f=a.getAttributeNode(b))&&f.specified?f.value:null},fa.error=function(a){throw new Error("Syntax error, unrecognized expression: "+a)},fa.uniqueSort=function(a){var b,d=[],e=0,f=0;if(l=!c.detectDuplicates,k=!c.sortStable&&a.slice(0),a.sort(B),l){while(b=a[f++])b===a[f]&&(e=d.push(f));while(e--)a.splice(d[e],1)}return k=null,a},e=fa.getText=function(a){var b,c="",d=0,f=a.nodeType;if(f){if(1===f||9===f||11===f){if("string"==typeof a.textContent)return a.textContent;for(a=a.firstChild;a;a=a.nextSibling)c+=e(a)}else if(3===f||4===f)return a.nodeValue}else while(b=a[d++])c+=e(b);return c},d=fa.selectors={cacheLength:50,createPseudo:ha,match:W,attrHandle:{},find:{},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(ba,ca),a[3]=(a[3]||a[4]||a[5]||"").replace(ba,ca),"~="===a[2]&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),"nth"===a[1].slice(0,3)?(a[3]||fa.error(a[0]),a[4]=+(a[4]?a[5]+(a[6]||1):2*("even"===a[3]||"odd"===a[3])),a[5]=+(a[7]+a[8]||"odd"===a[3])):a[3]&&fa.error(a[0]),a},PSEUDO:function(a){var b,c=!a[6]&&a[2];return W.CHILD.test(a[0])?null:(a[3]?a[2]=a[4]||a[5]||"":c&&U.test(c)&&(b=g(c,!0))&&(b=c.indexOf(")",c.length-b)-c.length)&&(a[0]=a[0].slice(0,b),a[2]=c.slice(0,b)),a.slice(0,3))}},filter:{TAG:function(a){var b=a.replace(ba,ca).toLowerCase();return"*"===a?function(){return!0}:function(a){return a.nodeName&&a.nodeName.toLowerCase()===b}},CLASS:function(a){var b=y[a+" "];return b||(b=new RegExp("(^|"+L+")"+a+"("+L+"|$)"))&&y(a,function(a){return b.test("string"==typeof a.className&&a.className||"undefined"!=typeof a.getAttribute&&a.getAttribute("class")||"")})},ATTR:function(a,b,c){return function(d){var e=fa.attr(d,a);return null==e?"!="===b:b?(e+="","="===b?e===c:"!="===b?e!==c:"^="===b?c&&0===e.indexOf(c):"*="===b?c&&e.indexOf(c)>-1:"$="===b?c&&e.slice(-c.length)===c:"~="===b?(" "+e.replace(P," ")+" ").indexOf(c)>-1:"|="===b?e===c||e.slice(0,c.length+1)===c+"-":!1):!0}},CHILD:function(a,b,c,d,e){var f="nth"!==a.slice(0,3),g="last"!==a.slice(-4),h="of-type"===b;return 1===d&&0===e?function(a){return!!a.parentNode}:function(b,c,i){var j,k,l,m,n,o,p=f!==g?"nextSibling":"previousSibling",q=b.parentNode,r=h&&b.nodeName.toLowerCase(),s=!i&&!h,t=!1;if(q){if(f){while(p){m=b;while(m=m[p])if(h?m.nodeName.toLowerCase()===r:1===m.nodeType)return!1;o=p="only"===a&&!o&&"nextSibling"}return!0}if(o=[g?q.firstChild:q.lastChild],g&&s){m=q,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n&&j[2],m=n&&q.childNodes[n];while(m=++n&&m&&m[p]||(t=n=0)||o.pop())if(1===m.nodeType&&++t&&m===b){k[a]=[w,n,t];break}}else if(s&&(m=b,l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),j=k[a]||[],n=j[0]===w&&j[1],t=n),t===!1)while(m=++n&&m&&m[p]||(t=n=0)||o.pop())if((h?m.nodeName.toLowerCase()===r:1===m.nodeType)&&++t&&(s&&(l=m[u]||(m[u]={}),k=l[m.uniqueID]||(l[m.uniqueID]={}),k[a]=[w,t]),m===b))break;return t-=e,t===d||t%d===0&&t/d>=0}}},PSEUDO:function(a,b){var c,e=d.pseudos[a]||d.setFilters[a.toLowerCase()]||fa.error("unsupported pseudo: "+a);return e[u]?e(b):e.length>1?(c=[a,a,"",b],d.setFilters.hasOwnProperty(a.toLowerCase())?ha(function(a,c){var d,f=e(a,b),g=f.length;while(g--)d=J(a,f[g]),a[d]=!(c[d]=f[g])}):function(a){return e(a,0,c)}):e}},pseudos:{not:ha(function(a){var b=[],c=[],d=h(a.replace(Q,"$1"));return d[u]?ha(function(a,b,c,e){var f,g=d(a,null,e,[]),h=a.length;while(h--)(f=g[h])&&(a[h]=!(b[h]=f))}):function(a,e,f){return b[0]=a,d(b,null,f,c),b[0]=null,!c.pop()}}),has:ha(function(a){return function(b){return fa(a,b).length>0}}),contains:ha(function(a){return a=a.replace(ba,ca),function(b){return(b.textContent||b.innerText||e(b)).indexOf(a)>-1}}),lang:ha(function(a){return V.test(a||"")||fa.error("unsupported lang: "+a),a=a.replace(ba,ca).toLowerCase(),function(b){var c;do if(c=p?b.lang:b.getAttribute("xml:lang")||b.getAttribute("lang"))return c=c.toLowerCase(),c===a||0===c.indexOf(a+"-");while((b=b.parentNode)&&1===b.nodeType);return!1}}),target:function(b){var c=a.location&&a.location.hash;return c&&c.slice(1)===b.id},root:function(a){return a===o},focus:function(a){return a===n.activeElement&&(!n.hasFocus||n.hasFocus())&&!!(a.type||a.href||~a.tabIndex)},enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&!!a.checked||"option"===b&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},empty:function(a){for(a=a.firstChild;a;a=a.nextSibling)if(a.nodeType<6)return!1;return!0},parent:function(a){return!d.pseudos.empty(a)},header:function(a){return Y.test(a.nodeName)},input:function(a){return X.test(a.nodeName)},button:function(a){var b=a.nodeName.toLowerCase();return"input"===b&&"button"===a.type||"button"===b},text:function(a){var b;return"input"===a.nodeName.toLowerCase()&&"text"===a.type&&(null==(b=a.getAttribute("type"))||"text"===b.toLowerCase())},first:na(function(){return[0]}),last:na(function(a,b){return[b-1]}),eq:na(function(a,b,c){return[0>c?c+b:c]}),even:na(function(a,b){for(var c=0;b>c;c+=2)a.push(c);return a}),odd:na(function(a,b){for(var c=1;b>c;c+=2)a.push(c);return a}),lt:na(function(a,b,c){for(var d=0>c?c+b:c;--d>=0;)a.push(d);return a}),gt:na(function(a,b,c){for(var d=0>c?c+b:c;++db;b++)d+=a[b].value;return d}function ra(a,b,c){var d=b.dir,e=c&&"parentNode"===d,f=x++;return b.first?function(b,c,f){while(b=b[d])if(1===b.nodeType||e)return a(b,c,f)}:function(b,c,g){var h,i,j,k=[w,f];if(g){while(b=b[d])if((1===b.nodeType||e)&&a(b,c,g))return!0}else while(b=b[d])if(1===b.nodeType||e){if(j=b[u]||(b[u]={}),i=j[b.uniqueID]||(j[b.uniqueID]={}),(h=i[d])&&h[0]===w&&h[1]===f)return k[2]=h[2];if(i[d]=k,k[2]=a(b,c,g))return!0}}}function sa(a){return a.length>1?function(b,c,d){var e=a.length;while(e--)if(!a[e](b,c,d))return!1;return!0}:a[0]}function ta(a,b,c){for(var d=0,e=b.length;e>d;d++)fa(a,b[d],c);return c}function ua(a,b,c,d,e){for(var f,g=[],h=0,i=a.length,j=null!=b;i>h;h++)(f=a[h])&&(c&&!c(f,d,e)||(g.push(f),j&&b.push(h)));return g}function va(a,b,c,d,e,f){return d&&!d[u]&&(d=va(d)),e&&!e[u]&&(e=va(e,f)),ha(function(f,g,h,i){var j,k,l,m=[],n=[],o=g.length,p=f||ta(b||"*",h.nodeType?[h]:h,[]),q=!a||!f&&b?p:ua(p,m,a,h,i),r=c?e||(f?a:o||d)?[]:g:q;if(c&&c(q,r,h,i),d){j=ua(r,n),d(j,[],h,i),k=j.length;while(k--)(l=j[k])&&(r[n[k]]=!(q[n[k]]=l))}if(f){if(e||a){if(e){j=[],k=r.length;while(k--)(l=r[k])&&j.push(q[k]=l);e(null,r=[],j,i)}k=r.length;while(k--)(l=r[k])&&(j=e?J(f,l):m[k])>-1&&(f[j]=!(g[j]=l))}}else r=ua(r===g?r.splice(o,r.length):r),e?e(null,g,r,i):H.apply(g,r)})}function wa(a){for(var b,c,e,f=a.length,g=d.relative[a[0].type],h=g||d.relative[" "],i=g?1:0,k=ra(function(a){return a===b},h,!0),l=ra(function(a){return J(b,a)>-1},h,!0),m=[function(a,c,d){var e=!g&&(d||c!==j)||((b=c).nodeType?k(a,c,d):l(a,c,d));return b=null,e}];f>i;i++)if(c=d.relative[a[i].type])m=[ra(sa(m),c)];else{if(c=d.filter[a[i].type].apply(null,a[i].matches),c[u]){for(e=++i;f>e;e++)if(d.relative[a[e].type])break;return va(i>1&&sa(m),i>1&&qa(a.slice(0,i-1).concat({value:" "===a[i-2].type?"*":""})).replace(Q,"$1"),c,e>i&&wa(a.slice(i,e)),f>e&&wa(a=a.slice(e)),f>e&&qa(a))}m.push(c)}return sa(m)}function xa(a,b){var c=b.length>0,e=a.length>0,f=function(f,g,h,i,k){var l,o,q,r=0,s="0",t=f&&[],u=[],v=j,x=f||e&&d.find.TAG("*",k),y=w+=null==v?1:Math.random()||.1,z=x.length;for(k&&(j=g===n||g||k);s!==z&&null!=(l=x[s]);s++){if(e&&l){o=0,g||l.ownerDocument===n||(m(l),h=!p);while(q=a[o++])if(q(l,g||n,h)){i.push(l);break}k&&(w=y)}c&&((l=!q&&l)&&r--,f&&t.push(l))}if(r+=s,c&&s!==r){o=0;while(q=b[o++])q(t,u,g,h);if(f){if(r>0)while(s--)t[s]||u[s]||(u[s]=F.call(i));u=ua(u)}H.apply(i,u),k&&!f&&u.length>0&&r+b.length>1&&fa.uniqueSort(i)}return k&&(w=y,j=v),t};return c?ha(f):f}return h=fa.compile=function(a,b){var c,d=[],e=[],f=A[a+" "];if(!f){b||(b=g(a)),c=b.length;while(c--)f=wa(b[c]),f[u]?d.push(f):e.push(f);f=A(a,xa(e,d)),f.selector=a}return f},i=fa.select=function(a,b,e,f){var i,j,k,l,m,n="function"==typeof a&&a,o=!f&&g(a=n.selector||a);if(e=e||[],1===o.length){if(j=o[0]=o[0].slice(0),j.length>2&&"ID"===(k=j[0]).type&&c.getById&&9===b.nodeType&&p&&d.relative[j[1].type]){if(b=(d.find.ID(k.matches[0].replace(ba,ca),b)||[])[0],!b)return e;n&&(b=b.parentNode),a=a.slice(j.shift().value.length)}i=W.needsContext.test(a)?0:j.length;while(i--){if(k=j[i],d.relative[l=k.type])break;if((m=d.find[l])&&(f=m(k.matches[0].replace(ba,ca),_.test(j[0].type)&&oa(b.parentNode)||b))){if(j.splice(i,1),a=f.length&&qa(j),!a)return H.apply(e,f),e;break}}}return(n||h(a,o))(f,b,!p,e,!b||_.test(a)&&oa(b.parentNode)||b),e},c.sortStable=u.split("").sort(B).join("")===u,c.detectDuplicates=!!l,m(),c.sortDetached=ia(function(a){return 1&a.compareDocumentPosition(n.createElement("div"))}),ia(function(a){return a.innerHTML="","#"===a.firstChild.getAttribute("href")})||ja("type|href|height|width",function(a,b,c){return c?void 0:a.getAttribute(b,"type"===b.toLowerCase()?1:2)}),c.attributes&&ia(function(a){return a.innerHTML="",a.firstChild.setAttribute("value",""),""===a.firstChild.getAttribute("value")})||ja("value",function(a,b,c){return c||"input"!==a.nodeName.toLowerCase()?void 0:a.defaultValue}),ia(function(a){return null==a.getAttribute("disabled")})||ja(K,function(a,b,c){var d;return c?void 0:a[b]===!0?b.toLowerCase():(d=a.getAttributeNode(b))&&d.specified?d.value:null}),fa}(a);n.find=t,n.expr=t.selectors,n.expr[":"]=n.expr.pseudos,n.uniqueSort=n.unique=t.uniqueSort,n.text=t.getText,n.isXMLDoc=t.isXML,n.contains=t.contains;var u=function(a,b,c){var d=[],e=void 0!==c;while((a=a[b])&&9!==a.nodeType)if(1===a.nodeType){if(e&&n(a).is(c))break;d.push(a)}return d},v=function(a,b){for(var c=[];a;a=a.nextSibling)1===a.nodeType&&a!==b&&c.push(a);return c},w=n.expr.match.needsContext,x=/^<([\w-]+)\s*\/?>(?:<\/\1>|)$/,y=/^.[^:#\[\.,]*$/;function z(a,b,c){if(n.isFunction(b))return n.grep(a,function(a,d){return!!b.call(a,d,a)!==c});if(b.nodeType)return n.grep(a,function(a){return a===b!==c});if("string"==typeof b){if(y.test(b))return n.filter(b,a,c);b=n.filter(b,a)}return n.grep(a,function(a){return n.inArray(a,b)>-1!==c})}n.filter=function(a,b,c){var d=b[0];return c&&(a=":not("+a+")"),1===b.length&&1===d.nodeType?n.find.matchesSelector(d,a)?[d]:[]:n.find.matches(a,n.grep(b,function(a){return 1===a.nodeType}))},n.fn.extend({find:function(a){var b,c=[],d=this,e=d.length;if("string"!=typeof a)return this.pushStack(n(a).filter(function(){for(b=0;e>b;b++)if(n.contains(d[b],this))return!0}));for(b=0;e>b;b++)n.find(a,d[b],c);return c=this.pushStack(e>1?n.unique(c):c),c.selector=this.selector?this.selector+" "+a:a,c},filter:function(a){return this.pushStack(z(this,a||[],!1))},not:function(a){return this.pushStack(z(this,a||[],!0))},is:function(a){return!!z(this,"string"==typeof a&&w.test(a)?n(a):a||[],!1).length}});var A,B=/^(?:\s*(<[\w\W]+>)[^>]*|#([\w-]*))$/,C=n.fn.init=function(a,b,c){var e,f;if(!a)return this;if(c=c||A,"string"==typeof a){if(e="<"===a.charAt(0)&&">"===a.charAt(a.length-1)&&a.length>=3?[null,a,null]:B.exec(a),!e||!e[1]&&b)return!b||b.jquery?(b||c).find(a):this.constructor(b).find(a);if(e[1]){if(b=b instanceof n?b[0]:b,n.merge(this,n.parseHTML(e[1],b&&b.nodeType?b.ownerDocument||b:d,!0)),x.test(e[1])&&n.isPlainObject(b))for(e in b)n.isFunction(this[e])?this[e](b[e]):this.attr(e,b[e]);return this}if(f=d.getElementById(e[2]),f&&f.parentNode){if(f.id!==e[2])return A.find(a);this.length=1,this[0]=f}return this.context=d,this.selector=a,this}return a.nodeType?(this.context=this[0]=a,this.length=1,this):n.isFunction(a)?"undefined"!=typeof c.ready?c.ready(a):a(n):(void 0!==a.selector&&(this.selector=a.selector,this.context=a.context),n.makeArray(a,this))};C.prototype=n.fn,A=n(d);var D=/^(?:parents|prev(?:Until|All))/,E={children:!0,contents:!0,next:!0,prev:!0};n.fn.extend({has:function(a){var b,c=n(a,this),d=c.length;return this.filter(function(){for(b=0;d>b;b++)if(n.contains(this,c[b]))return!0})},closest:function(a,b){for(var c,d=0,e=this.length,f=[],g=w.test(a)||"string"!=typeof a?n(a,b||this.context):0;e>d;d++)for(c=this[d];c&&c!==b;c=c.parentNode)if(c.nodeType<11&&(g?g.index(c)>-1:1===c.nodeType&&n.find.matchesSelector(c,a))){f.push(c);break}return this.pushStack(f.length>1?n.uniqueSort(f):f)},index:function(a){return a?"string"==typeof a?n.inArray(this[0],n(a)):n.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.first().prevAll().length:-1},add:function(a,b){return this.pushStack(n.uniqueSort(n.merge(this.get(),n(a,b))))},addBack:function(a){return this.add(null==a?this.prevObject:this.prevObject.filter(a))}});function F(a,b){do a=a[b];while(a&&1!==a.nodeType);return a}n.each({parent:function(a){var b=a.parentNode;return b&&11!==b.nodeType?b:null},parents:function(a){return u(a,"parentNode")},parentsUntil:function(a,b,c){return u(a,"parentNode",c)},next:function(a){return F(a,"nextSibling")},prev:function(a){return F(a,"previousSibling")},nextAll:function(a){return u(a,"nextSibling")},prevAll:function(a){return u(a,"previousSibling")},nextUntil:function(a,b,c){return u(a,"nextSibling",c)},prevUntil:function(a,b,c){return u(a,"previousSibling",c)},siblings:function(a){return v((a.parentNode||{}).firstChild,a)},children:function(a){return v(a.firstChild)},contents:function(a){return n.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:n.merge([],a.childNodes)}},function(a,b){n.fn[a]=function(c,d){var e=n.map(this,b,c);return"Until"!==a.slice(-5)&&(d=c),d&&"string"==typeof d&&(e=n.filter(d,e)),this.length>1&&(E[a]||(e=n.uniqueSort(e)),D.test(a)&&(e=e.reverse())),this.pushStack(e)}});var G=/\S+/g;function H(a){var b={};return n.each(a.match(G)||[],function(a,c){b[c]=!0}),b}n.Callbacks=function(a){a="string"==typeof a?H(a):n.extend({},a);var b,c,d,e,f=[],g=[],h=-1,i=function(){for(e=a.once,d=b=!0;g.length;h=-1){c=g.shift();while(++h-1)f.splice(c,1),h>=c&&h--}),this},has:function(a){return a?n.inArray(a,f)>-1:f.length>0},empty:function(){return f&&(f=[]),this},disable:function(){return e=g=[],f=c="",this},disabled:function(){return!f},lock:function(){return e=!0,c||j.disable(),this},locked:function(){return!!e},fireWith:function(a,c){return e||(c=c||[],c=[a,c.slice?c.slice():c],g.push(c),b||i()),this},fire:function(){return j.fireWith(this,arguments),this},fired:function(){return!!d}};return j},n.extend({Deferred:function(a){var b=[["resolve","done",n.Callbacks("once memory"),"resolved"],["reject","fail",n.Callbacks("once memory"),"rejected"],["notify","progress",n.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return n.Deferred(function(c){n.each(b,function(b,f){var g=n.isFunction(a[b])&&a[b];e[f[1]](function(){var a=g&&g.apply(this,arguments);a&&n.isFunction(a.promise)?a.promise().progress(c.notify).done(c.resolve).fail(c.reject):c[f[0]+"With"](this===d?c.promise():this,g?[a]:arguments)})}),a=null}).promise()},promise:function(a){return null!=a?n.extend(a,d):d}},e={};return d.pipe=d.then,n.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[1^a][2].disable,b[2][2].lock),e[f[0]]=function(){return e[f[0]+"With"](this===e?d:this,arguments),this},e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=e.call(arguments),d=c.length,f=1!==d||a&&n.isFunction(a.promise)?d:0,g=1===f?a:n.Deferred(),h=function(a,b,c){return function(d){b[a]=this,c[a]=arguments.length>1?e.call(arguments):d,c===i?g.notifyWith(b,c):--f||g.resolveWith(b,c)}},i,j,k;if(d>1)for(i=new Array(d),j=new Array(d),k=new Array(d);d>b;b++)c[b]&&n.isFunction(c[b].promise)?c[b].promise().progress(h(b,j,i)).done(h(b,k,c)).fail(g.reject):--f;return f||g.resolveWith(k,c),g.promise()}});var I;n.fn.ready=function(a){return n.ready.promise().done(a),this},n.extend({isReady:!1,readyWait:1,holdReady:function(a){a?n.readyWait++:n.ready(!0)},ready:function(a){(a===!0?--n.readyWait:n.isReady)||(n.isReady=!0,a!==!0&&--n.readyWait>0||(I.resolveWith(d,[n]),n.fn.triggerHandler&&(n(d).triggerHandler("ready"),n(d).off("ready"))))}});function J(){d.addEventListener?(d.removeEventListener("DOMContentLoaded",K),a.removeEventListener("load",K)):(d.detachEvent("onreadystatechange",K),a.detachEvent("onload",K))}function K(){(d.addEventListener||"load"===a.event.type||"complete"===d.readyState)&&(J(),n.ready())}n.ready.promise=function(b){if(!I)if(I=n.Deferred(),"complete"===d.readyState||"loading"!==d.readyState&&!d.documentElement.doScroll)a.setTimeout(n.ready);else if(d.addEventListener)d.addEventListener("DOMContentLoaded",K),a.addEventListener("load",K);else{d.attachEvent("onreadystatechange",K),a.attachEvent("onload",K);var c=!1;try{c=null==a.frameElement&&d.documentElement}catch(e){}c&&c.doScroll&&!function f(){if(!n.isReady){try{c.doScroll("left")}catch(b){return a.setTimeout(f,50)}J(),n.ready()}}()}return I.promise(b)},n.ready.promise();var L;for(L in n(l))break;l.ownFirst="0"===L,l.inlineBlockNeedsLayout=!1,n(function(){var a,b,c,e;c=d.getElementsByTagName("body")[0],c&&c.style&&(b=d.createElement("div"),e=d.createElement("div"),e.style.cssText="position:absolute;border:0;width:0;height:0;top:0;left:-9999px",c.appendChild(e).appendChild(b),"undefined"!=typeof b.style.zoom&&(b.style.cssText="display:inline;margin:0;border:0;padding:1px;width:1px;zoom:1",l.inlineBlockNeedsLayout=a=3===b.offsetWidth,a&&(c.style.zoom=1)),c.removeChild(e))}),function(){var a=d.createElement("div");l.deleteExpando=!0;try{delete a.test}catch(b){l.deleteExpando=!1}a=null}();var M=function(a){var b=n.noData[(a.nodeName+" ").toLowerCase()],c=+a.nodeType||1;return 1!==c&&9!==c?!1:!b||b!==!0&&a.getAttribute("classid")===b},N=/^(?:\{[\w\W]*\}|\[[\w\W]*\])$/,O=/([A-Z])/g;function P(a,b,c){if(void 0===c&&1===a.nodeType){var d="data-"+b.replace(O,"-$1").toLowerCase();if(c=a.getAttribute(d),"string"==typeof c){try{c="true"===c?!0:"false"===c?!1:"null"===c?null:+c+""===c?+c:N.test(c)?n.parseJSON(c):c}catch(e){}n.data(a,b,c)}else c=void 0; +}return c}function Q(a){var b;for(b in a)if(("data"!==b||!n.isEmptyObject(a[b]))&&"toJSON"!==b)return!1;return!0}function R(a,b,d,e){if(M(a)){var f,g,h=n.expando,i=a.nodeType,j=i?n.cache:a,k=i?a[h]:a[h]&&h;if(k&&j[k]&&(e||j[k].data)||void 0!==d||"string"!=typeof b)return k||(k=i?a[h]=c.pop()||n.guid++:h),j[k]||(j[k]=i?{}:{toJSON:n.noop}),"object"!=typeof b&&"function"!=typeof b||(e?j[k]=n.extend(j[k],b):j[k].data=n.extend(j[k].data,b)),g=j[k],e||(g.data||(g.data={}),g=g.data),void 0!==d&&(g[n.camelCase(b)]=d),"string"==typeof b?(f=g[b],null==f&&(f=g[n.camelCase(b)])):f=g,f}}function S(a,b,c){if(M(a)){var d,e,f=a.nodeType,g=f?n.cache:a,h=f?a[n.expando]:n.expando;if(g[h]){if(b&&(d=c?g[h]:g[h].data)){n.isArray(b)?b=b.concat(n.map(b,n.camelCase)):b in d?b=[b]:(b=n.camelCase(b),b=b in d?[b]:b.split(" ")),e=b.length;while(e--)delete d[b[e]];if(c?!Q(d):!n.isEmptyObject(d))return}(c||(delete g[h].data,Q(g[h])))&&(f?n.cleanData([a],!0):l.deleteExpando||g!=g.window?delete g[h]:g[h]=void 0)}}}n.extend({cache:{},noData:{"applet ":!0,"embed ":!0,"object ":"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000"},hasData:function(a){return a=a.nodeType?n.cache[a[n.expando]]:a[n.expando],!!a&&!Q(a)},data:function(a,b,c){return R(a,b,c)},removeData:function(a,b){return S(a,b)},_data:function(a,b,c){return R(a,b,c,!0)},_removeData:function(a,b){return S(a,b,!0)}}),n.fn.extend({data:function(a,b){var c,d,e,f=this[0],g=f&&f.attributes;if(void 0===a){if(this.length&&(e=n.data(f),1===f.nodeType&&!n._data(f,"parsedAttrs"))){c=g.length;while(c--)g[c]&&(d=g[c].name,0===d.indexOf("data-")&&(d=n.camelCase(d.slice(5)),P(f,d,e[d])));n._data(f,"parsedAttrs",!0)}return e}return"object"==typeof a?this.each(function(){n.data(this,a)}):arguments.length>1?this.each(function(){n.data(this,a,b)}):f?P(f,a,n.data(f,a)):void 0},removeData:function(a){return this.each(function(){n.removeData(this,a)})}}),n.extend({queue:function(a,b,c){var d;return a?(b=(b||"fx")+"queue",d=n._data(a,b),c&&(!d||n.isArray(c)?d=n._data(a,b,n.makeArray(c)):d.push(c)),d||[]):void 0},dequeue:function(a,b){b=b||"fx";var c=n.queue(a,b),d=c.length,e=c.shift(),f=n._queueHooks(a,b),g=function(){n.dequeue(a,b)};"inprogress"===e&&(e=c.shift(),d--),e&&("fx"===b&&c.unshift("inprogress"),delete f.stop,e.call(a,g,f)),!d&&f&&f.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return n._data(a,c)||n._data(a,c,{empty:n.Callbacks("once memory").add(function(){n._removeData(a,b+"queue"),n._removeData(a,c)})})}}),n.fn.extend({queue:function(a,b){var c=2;return"string"!=typeof a&&(b=a,a="fx",c--),arguments.lengthh;h++)b(a[h],c,g?d:d.call(a[h],h,b(a[h],c)));return e?a:j?b.call(a):i?b(a[0],c):f},Z=/^(?:checkbox|radio)$/i,$=/<([\w:-]+)/,_=/^$|\/(?:java|ecma)script/i,aa=/^\s+/,ba="abbr|article|aside|audio|bdi|canvas|data|datalist|details|dialog|figcaption|figure|footer|header|hgroup|main|mark|meter|nav|output|picture|progress|section|summary|template|time|video";function ca(a){var b=ba.split("|"),c=a.createDocumentFragment();if(c.createElement)while(b.length)c.createElement(b.pop());return c}!function(){var a=d.createElement("div"),b=d.createDocumentFragment(),c=d.createElement("input");a.innerHTML="
    a",l.leadingWhitespace=3===a.firstChild.nodeType,l.tbody=!a.getElementsByTagName("tbody").length,l.htmlSerialize=!!a.getElementsByTagName("link").length,l.html5Clone="<:nav>"!==d.createElement("nav").cloneNode(!0).outerHTML,c.type="checkbox",c.checked=!0,b.appendChild(c),l.appendChecked=c.checked,a.innerHTML="",l.noCloneChecked=!!a.cloneNode(!0).lastChild.defaultValue,b.appendChild(a),c=d.createElement("input"),c.setAttribute("type","radio"),c.setAttribute("checked","checked"),c.setAttribute("name","t"),a.appendChild(c),l.checkClone=a.cloneNode(!0).cloneNode(!0).lastChild.checked,l.noCloneEvent=!!a.addEventListener,a[n.expando]=1,l.attributes=!a.getAttribute(n.expando)}();var da={option:[1,""],legend:[1,"
    ","
    "],area:[1,"",""],param:[1,"",""],thead:[1,"","
    "],tr:[2,"","
    "],col:[2,"","
    "],td:[3,"","
    "],_default:l.htmlSerialize?[0,"",""]:[1,"X
    ","
    "]};da.optgroup=da.option,da.tbody=da.tfoot=da.colgroup=da.caption=da.thead,da.th=da.td;function ea(a,b){var c,d,e=0,f="undefined"!=typeof a.getElementsByTagName?a.getElementsByTagName(b||"*"):"undefined"!=typeof a.querySelectorAll?a.querySelectorAll(b||"*"):void 0;if(!f)for(f=[],c=a.childNodes||a;null!=(d=c[e]);e++)!b||n.nodeName(d,b)?f.push(d):n.merge(f,ea(d,b));return void 0===b||b&&n.nodeName(a,b)?n.merge([a],f):f}function fa(a,b){for(var c,d=0;null!=(c=a[d]);d++)n._data(c,"globalEval",!b||n._data(b[d],"globalEval"))}var ga=/<|&#?\w+;/,ha=/r;r++)if(g=a[r],g||0===g)if("object"===n.type(g))n.merge(q,g.nodeType?[g]:g);else if(ga.test(g)){i=i||p.appendChild(b.createElement("div")),j=($.exec(g)||["",""])[1].toLowerCase(),m=da[j]||da._default,i.innerHTML=m[1]+n.htmlPrefilter(g)+m[2],f=m[0];while(f--)i=i.lastChild;if(!l.leadingWhitespace&&aa.test(g)&&q.push(b.createTextNode(aa.exec(g)[0])),!l.tbody){g="table"!==j||ha.test(g)?""!==m[1]||ha.test(g)?0:i:i.firstChild,f=g&&g.childNodes.length;while(f--)n.nodeName(k=g.childNodes[f],"tbody")&&!k.childNodes.length&&g.removeChild(k)}n.merge(q,i.childNodes),i.textContent="";while(i.firstChild)i.removeChild(i.firstChild);i=p.lastChild}else q.push(b.createTextNode(g));i&&p.removeChild(i),l.appendChecked||n.grep(ea(q,"input"),ia),r=0;while(g=q[r++])if(d&&n.inArray(g,d)>-1)e&&e.push(g);else if(h=n.contains(g.ownerDocument,g),i=ea(p.appendChild(g),"script"),h&&fa(i),c){f=0;while(g=i[f++])_.test(g.type||"")&&c.push(g)}return i=null,p}!function(){var b,c,e=d.createElement("div");for(b in{submit:!0,change:!0,focusin:!0})c="on"+b,(l[b]=c in a)||(e.setAttribute(c,"t"),l[b]=e.attributes[c].expando===!1);e=null}();var ka=/^(?:input|select|textarea)$/i,la=/^key/,ma=/^(?:mouse|pointer|contextmenu|drag|drop)|click/,na=/^(?:focusinfocus|focusoutblur)$/,oa=/^([^.]*)(?:\.(.+)|)/;function pa(){return!0}function qa(){return!1}function ra(){try{return d.activeElement}catch(a){}}function sa(a,b,c,d,e,f){var g,h;if("object"==typeof b){"string"!=typeof c&&(d=d||c,c=void 0);for(h in b)sa(a,h,c,d,b[h],f);return a}if(null==d&&null==e?(e=c,d=c=void 0):null==e&&("string"==typeof c?(e=d,d=void 0):(e=d,d=c,c=void 0)),e===!1)e=qa;else if(!e)return a;return 1===f&&(g=e,e=function(a){return n().off(a),g.apply(this,arguments)},e.guid=g.guid||(g.guid=n.guid++)),a.each(function(){n.event.add(this,b,e,d,c)})}n.event={global:{},add:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n._data(a);if(r){c.handler&&(i=c,c=i.handler,e=i.selector),c.guid||(c.guid=n.guid++),(g=r.events)||(g=r.events={}),(k=r.handle)||(k=r.handle=function(a){return"undefined"==typeof n||a&&n.event.triggered===a.type?void 0:n.event.dispatch.apply(k.elem,arguments)},k.elem=a),b=(b||"").match(G)||[""],h=b.length;while(h--)f=oa.exec(b[h])||[],o=q=f[1],p=(f[2]||"").split(".").sort(),o&&(j=n.event.special[o]||{},o=(e?j.delegateType:j.bindType)||o,j=n.event.special[o]||{},l=n.extend({type:o,origType:q,data:d,handler:c,guid:c.guid,selector:e,needsContext:e&&n.expr.match.needsContext.test(e),namespace:p.join(".")},i),(m=g[o])||(m=g[o]=[],m.delegateCount=0,j.setup&&j.setup.call(a,d,p,k)!==!1||(a.addEventListener?a.addEventListener(o,k,!1):a.attachEvent&&a.attachEvent("on"+o,k))),j.add&&(j.add.call(a,l),l.handler.guid||(l.handler.guid=c.guid)),e?m.splice(m.delegateCount++,0,l):m.push(l),n.event.global[o]=!0);a=null}},remove:function(a,b,c,d,e){var f,g,h,i,j,k,l,m,o,p,q,r=n.hasData(a)&&n._data(a);if(r&&(k=r.events)){b=(b||"").match(G)||[""],j=b.length;while(j--)if(h=oa.exec(b[j])||[],o=q=h[1],p=(h[2]||"").split(".").sort(),o){l=n.event.special[o]||{},o=(d?l.delegateType:l.bindType)||o,m=k[o]||[],h=h[2]&&new RegExp("(^|\\.)"+p.join("\\.(?:.*\\.|)")+"(\\.|$)"),i=f=m.length;while(f--)g=m[f],!e&&q!==g.origType||c&&c.guid!==g.guid||h&&!h.test(g.namespace)||d&&d!==g.selector&&("**"!==d||!g.selector)||(m.splice(f,1),g.selector&&m.delegateCount--,l.remove&&l.remove.call(a,g));i&&!m.length&&(l.teardown&&l.teardown.call(a,p,r.handle)!==!1||n.removeEvent(a,o,r.handle),delete k[o])}else for(o in k)n.event.remove(a,o+b[j],c,d,!0);n.isEmptyObject(k)&&(delete r.handle,n._removeData(a,"events"))}},trigger:function(b,c,e,f){var g,h,i,j,l,m,o,p=[e||d],q=k.call(b,"type")?b.type:b,r=k.call(b,"namespace")?b.namespace.split("."):[];if(i=m=e=e||d,3!==e.nodeType&&8!==e.nodeType&&!na.test(q+n.event.triggered)&&(q.indexOf(".")>-1&&(r=q.split("."),q=r.shift(),r.sort()),h=q.indexOf(":")<0&&"on"+q,b=b[n.expando]?b:new n.Event(q,"object"==typeof b&&b),b.isTrigger=f?2:3,b.namespace=r.join("."),b.rnamespace=b.namespace?new RegExp("(^|\\.)"+r.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,b.result=void 0,b.target||(b.target=e),c=null==c?[b]:n.makeArray(c,[b]),l=n.event.special[q]||{},f||!l.trigger||l.trigger.apply(e,c)!==!1)){if(!f&&!l.noBubble&&!n.isWindow(e)){for(j=l.delegateType||q,na.test(j+q)||(i=i.parentNode);i;i=i.parentNode)p.push(i),m=i;m===(e.ownerDocument||d)&&p.push(m.defaultView||m.parentWindow||a)}o=0;while((i=p[o++])&&!b.isPropagationStopped())b.type=o>1?j:l.bindType||q,g=(n._data(i,"events")||{})[b.type]&&n._data(i,"handle"),g&&g.apply(i,c),g=h&&i[h],g&&g.apply&&M(i)&&(b.result=g.apply(i,c),b.result===!1&&b.preventDefault());if(b.type=q,!f&&!b.isDefaultPrevented()&&(!l._default||l._default.apply(p.pop(),c)===!1)&&M(e)&&h&&e[q]&&!n.isWindow(e)){m=e[h],m&&(e[h]=null),n.event.triggered=q;try{e[q]()}catch(s){}n.event.triggered=void 0,m&&(e[h]=m)}return b.result}},dispatch:function(a){a=n.event.fix(a);var b,c,d,f,g,h=[],i=e.call(arguments),j=(n._data(this,"events")||{})[a.type]||[],k=n.event.special[a.type]||{};if(i[0]=a,a.delegateTarget=this,!k.preDispatch||k.preDispatch.call(this,a)!==!1){h=n.event.handlers.call(this,a,j),b=0;while((f=h[b++])&&!a.isPropagationStopped()){a.currentTarget=f.elem,c=0;while((g=f.handlers[c++])&&!a.isImmediatePropagationStopped())a.rnamespace&&!a.rnamespace.test(g.namespace)||(a.handleObj=g,a.data=g.data,d=((n.event.special[g.origType]||{}).handle||g.handler).apply(f.elem,i),void 0!==d&&(a.result=d)===!1&&(a.preventDefault(),a.stopPropagation()))}return k.postDispatch&&k.postDispatch.call(this,a),a.result}},handlers:function(a,b){var c,d,e,f,g=[],h=b.delegateCount,i=a.target;if(h&&i.nodeType&&("click"!==a.type||isNaN(a.button)||a.button<1))for(;i!=this;i=i.parentNode||this)if(1===i.nodeType&&(i.disabled!==!0||"click"!==a.type)){for(d=[],c=0;h>c;c++)f=b[c],e=f.selector+" ",void 0===d[e]&&(d[e]=f.needsContext?n(e,this).index(i)>-1:n.find(e,this,null,[i]).length),d[e]&&d.push(f);d.length&&g.push({elem:i,handlers:d})}return h]","i"),va=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:-]+)[^>]*)\/>/gi,wa=/\s*$/g,Aa=ca(d),Ba=Aa.appendChild(d.createElement("div"));function Ca(a,b){return n.nodeName(a,"table")&&n.nodeName(11!==b.nodeType?b:b.firstChild,"tr")?a.getElementsByTagName("tbody")[0]||a.appendChild(a.ownerDocument.createElement("tbody")):a}function Da(a){return a.type=(null!==n.find.attr(a,"type"))+"/"+a.type,a}function Ea(a){var b=ya.exec(a.type);return b?a.type=b[1]:a.removeAttribute("type"),a}function Fa(a,b){if(1===b.nodeType&&n.hasData(a)){var c,d,e,f=n._data(a),g=n._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;e>d;d++)n.event.add(b,c,h[c][d])}g.data&&(g.data=n.extend({},g.data))}}function Ga(a,b){var c,d,e;if(1===b.nodeType){if(c=b.nodeName.toLowerCase(),!l.noCloneEvent&&b[n.expando]){e=n._data(b);for(d in e.events)n.removeEvent(b,d,e.handle);b.removeAttribute(n.expando)}"script"===c&&b.text!==a.text?(Da(b).text=a.text,Ea(b)):"object"===c?(b.parentNode&&(b.outerHTML=a.outerHTML),l.html5Clone&&a.innerHTML&&!n.trim(b.innerHTML)&&(b.innerHTML=a.innerHTML)):"input"===c&&Z.test(a.type)?(b.defaultChecked=b.checked=a.checked,b.value!==a.value&&(b.value=a.value)):"option"===c?b.defaultSelected=b.selected=a.defaultSelected:"input"!==c&&"textarea"!==c||(b.defaultValue=a.defaultValue)}}function Ha(a,b,c,d){b=f.apply([],b);var e,g,h,i,j,k,m=0,o=a.length,p=o-1,q=b[0],r=n.isFunction(q);if(r||o>1&&"string"==typeof q&&!l.checkClone&&xa.test(q))return a.each(function(e){var f=a.eq(e);r&&(b[0]=q.call(this,e,f.html())),Ha(f,b,c,d)});if(o&&(k=ja(b,a[0].ownerDocument,!1,a,d),e=k.firstChild,1===k.childNodes.length&&(k=e),e||d)){for(i=n.map(ea(k,"script"),Da),h=i.length;o>m;m++)g=k,m!==p&&(g=n.clone(g,!0,!0),h&&n.merge(i,ea(g,"script"))),c.call(a[m],g,m);if(h)for(j=i[i.length-1].ownerDocument,n.map(i,Ea),m=0;h>m;m++)g=i[m],_.test(g.type||"")&&!n._data(g,"globalEval")&&n.contains(j,g)&&(g.src?n._evalUrl&&n._evalUrl(g.src):n.globalEval((g.text||g.textContent||g.innerHTML||"").replace(za,"")));k=e=null}return a}function Ia(a,b,c){for(var d,e=b?n.filter(b,a):a,f=0;null!=(d=e[f]);f++)c||1!==d.nodeType||n.cleanData(ea(d)),d.parentNode&&(c&&n.contains(d.ownerDocument,d)&&fa(ea(d,"script")),d.parentNode.removeChild(d));return a}n.extend({htmlPrefilter:function(a){return a.replace(va,"<$1>")},clone:function(a,b,c){var d,e,f,g,h,i=n.contains(a.ownerDocument,a);if(l.html5Clone||n.isXMLDoc(a)||!ua.test("<"+a.nodeName+">")?f=a.cloneNode(!0):(Ba.innerHTML=a.outerHTML,Ba.removeChild(f=Ba.firstChild)),!(l.noCloneEvent&&l.noCloneChecked||1!==a.nodeType&&11!==a.nodeType||n.isXMLDoc(a)))for(d=ea(f),h=ea(a),g=0;null!=(e=h[g]);++g)d[g]&&Ga(e,d[g]);if(b)if(c)for(h=h||ea(a),d=d||ea(f),g=0;null!=(e=h[g]);g++)Fa(e,d[g]);else Fa(a,f);return d=ea(f,"script"),d.length>0&&fa(d,!i&&ea(a,"script")),d=h=e=null,f},cleanData:function(a,b){for(var d,e,f,g,h=0,i=n.expando,j=n.cache,k=l.attributes,m=n.event.special;null!=(d=a[h]);h++)if((b||M(d))&&(f=d[i],g=f&&j[f])){if(g.events)for(e in g.events)m[e]?n.event.remove(d,e):n.removeEvent(d,e,g.handle);j[f]&&(delete j[f],k||"undefined"==typeof d.removeAttribute?d[i]=void 0:d.removeAttribute(i),c.push(f))}}}),n.fn.extend({domManip:Ha,detach:function(a){return Ia(this,a,!0)},remove:function(a){return Ia(this,a)},text:function(a){return Y(this,function(a){return void 0===a?n.text(this):this.empty().append((this[0]&&this[0].ownerDocument||d).createTextNode(a))},null,a,arguments.length)},append:function(){return Ha(this,arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=Ca(this,a);b.appendChild(a)}})},prepend:function(){return Ha(this,arguments,function(a){if(1===this.nodeType||11===this.nodeType||9===this.nodeType){var b=Ca(this,a);b.insertBefore(a,b.firstChild)}})},before:function(){return Ha(this,arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this)})},after:function(){return Ha(this,arguments,function(a){this.parentNode&&this.parentNode.insertBefore(a,this.nextSibling)})},empty:function(){for(var a,b=0;null!=(a=this[b]);b++){1===a.nodeType&&n.cleanData(ea(a,!1));while(a.firstChild)a.removeChild(a.firstChild);a.options&&n.nodeName(a,"select")&&(a.options.length=0)}return this},clone:function(a,b){return a=null==a?!1:a,b=null==b?a:b,this.map(function(){return n.clone(this,a,b)})},html:function(a){return Y(this,function(a){var b=this[0]||{},c=0,d=this.length;if(void 0===a)return 1===b.nodeType?b.innerHTML.replace(ta,""):void 0;if("string"==typeof a&&!wa.test(a)&&(l.htmlSerialize||!ua.test(a))&&(l.leadingWhitespace||!aa.test(a))&&!da[($.exec(a)||["",""])[1].toLowerCase()]){a=n.htmlPrefilter(a);try{for(;d>c;c++)b=this[c]||{},1===b.nodeType&&(n.cleanData(ea(b,!1)),b.innerHTML=a);b=0}catch(e){}}b&&this.empty().append(a)},null,a,arguments.length)},replaceWith:function(){var a=[];return Ha(this,arguments,function(b){var c=this.parentNode;n.inArray(this,a)<0&&(n.cleanData(ea(this)),c&&c.replaceChild(b,this))},a)}}),n.each({appendTo:"append",prependTo:"prepend",insertBefore:"before",insertAfter:"after",replaceAll:"replaceWith"},function(a,b){n.fn[a]=function(a){for(var c,d=0,e=[],f=n(a),h=f.length-1;h>=d;d++)c=d===h?this:this.clone(!0),n(f[d])[b](c),g.apply(e,c.get());return this.pushStack(e)}});var Ja,Ka={HTML:"block",BODY:"block"};function La(a,b){var c=n(b.createElement(a)).appendTo(b.body),d=n.css(c[0],"display");return c.detach(),d}function Ma(a){var b=d,c=Ka[a];return c||(c=La(a,b),"none"!==c&&c||(Ja=(Ja||n("