From 8856c3d86009295be871989a5dc7270f31b420cd Mon Sep 17 00:00:00 2001 From: chutium Date: Tue, 26 Aug 2014 11:51:26 -0700 Subject: [PATCH 01/51] [SPARK-3131][SQL] Allow user to set parquet compression codec for writing ParquetFile in SQLContext There are 4 different compression codec available for ```ParquetOutputFormat``` in Spark SQL, it was set as a hard-coded value in ```ParquetRelation.defaultCompression``` original discuss: https://github.com/apache/spark/pull/195#discussion-diff-11002083 i added a new config property in SQLConf to allow user to change this compression codec, and i used similar short names syntax as described in SPARK-2953 #1873 (https://github.com/apache/spark/pull/1873/files#diff-0) btw, which codec should we use as default? it was set to GZIP (https://github.com/apache/spark/pull/195/files#diff-4), but i think maybe we should change this to SNAPPY, since SNAPPY is already the default codec for shuffling in spark-core (SPARK-2469, #1415), and parquet-mr supports Snappy codec natively (https://github.com/Parquet/parquet-mr/commit/e440108de57199c12d66801ca93804086e7f7632). Author: chutium Closes #2039 from chutium/parquet-compression and squashes the following commits: 2f44964 [chutium] [SPARK-3131][SQL] parquet compression default codec set to snappy, also in test suite e578e21 [chutium] [SPARK-3131][SQL] compression codec config property name and default codec set to snappy 21235dc [chutium] [SPARK-3131][SQL] Allow user to set parquet compression codec for writing ParquetFile in SQLContext --- .../scala/org/apache/spark/sql/SQLConf.scala | 4 + .../spark/sql/parquet/ParquetRelation.scala | 14 ++- .../spark/sql/parquet/ParquetQuerySuite.scala | 94 +++++++++++++++++++ 3 files changed, 107 insertions(+), 5 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 5cc41a83cc792..f0df19112ae37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -33,6 +33,7 @@ private[spark] object SQLConf { val DIALECT = "spark.sql.dialect" val PARQUET_BINARY_AS_STRING = "spark.sql.parquet.binaryAsString" val PARQUET_CACHE_METADATA = "spark.sql.parquet.cacheMetadata" + val PARQUET_COMPRESSION = "spark.sql.parquet.compression.codec" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" @@ -78,6 +79,9 @@ trait SQLConf { /** When true tables cached using the in-memory columnar caching will be compressed. */ private[spark] def useCompression: Boolean = getConf(COMPRESS_CACHED, "false").toBoolean + /** The compression codec for writing to a Parquetfile */ + private[spark] def parquetCompressionCodec: String = getConf(PARQUET_COMPRESSION, "snappy") + /** The number of rows that will be */ private[spark] def columnBatchSize: Int = getConf(COLUMN_BATCH_SIZE, "1000").toInt diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala index 1713ae6fb5d93..5ae768293a22e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -100,8 +100,13 @@ private[sql] object ParquetRelation { // The compression type type CompressionType = parquet.hadoop.metadata.CompressionCodecName - // The default compression - val defaultCompression = CompressionCodecName.GZIP + // The parquet compression short names + val shortParquetCompressionCodecNames = Map( + "NONE" -> CompressionCodecName.UNCOMPRESSED, + "UNCOMPRESSED" -> CompressionCodecName.UNCOMPRESSED, + "SNAPPY" -> CompressionCodecName.SNAPPY, + "GZIP" -> CompressionCodecName.GZIP, + "LZO" -> CompressionCodecName.LZO) /** * Creates a new ParquetRelation and underlying Parquetfile for the given LogicalPlan. Note that @@ -141,9 +146,8 @@ private[sql] object ParquetRelation { conf: Configuration, sqlContext: SQLContext): ParquetRelation = { val path = checkPath(pathString, allowExisting, conf) - if (conf.get(ParquetOutputFormat.COMPRESSION) == null) { - conf.set(ParquetOutputFormat.COMPRESSION, ParquetRelation.defaultCompression.name()) - } + conf.set(ParquetOutputFormat.COMPRESSION, shortParquetCompressionCodecNames.getOrElse( + sqlContext.parquetCompressionCodec.toUpperCase, CompressionCodecName.UNCOMPRESSED).name()) ParquetRelation.enableLogForwarding() ParquetTypesConverter.writeMetaData(attributes, path, conf) new ParquetRelation(path.toString, Some(conf), sqlContext) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 172dcd6aa0ee3..28f43b36832ac 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -186,6 +186,100 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA TestSQLContext.setConf(SQLConf.PARQUET_BINARY_AS_STRING, oldIsParquetBinaryAsString.toString) } + test("Compression options for writing to a Parquetfile") { + val defaultParquetCompressionCodec = TestSQLContext.parquetCompressionCodec + import scala.collection.JavaConversions._ + + val file = getTempFilePath("parquet") + val path = file.toString + val rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + + // test default compression codec + rdd.saveAsParquetFile(path) + var actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test uncompressed parquet file with property value "UNCOMPRESSED" + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "UNCOMPRESSED") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test uncompressed parquet file with property value "none" + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "none") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === "UNCOMPRESSED" :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test gzip compression codec + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "gzip") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // test snappy compression codec + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, "snappy") + + rdd.saveAsParquetFile(path) + actualCodec = ParquetTypesConverter.readMetaData(new Path(path), Some(TestSQLContext.sparkContext.hadoopConfiguration)) + .getBlocks.flatMap(block => block.getColumns).map(column => column.getCodec.name()).distinct + assert(actualCodec === TestSQLContext.parquetCompressionCodec.toUpperCase :: Nil) + + parquetFile(path).registerTempTable("tmp") + checkAnswer( + sql("SELECT key, value FROM tmp WHERE value = 'val_5' OR value = 'val_7'"), + (5, "val_5") :: + (7, "val_7") :: Nil) + + Utils.deleteRecursively(file) + + // TODO: Lzo requires additional external setup steps so leave it out for now + // ref.: https://github.com/Parquet/parquet-mr/blob/parquet-1.5.0/parquet-hadoop/src/test/java/parquet/hadoop/example/TestInputOutputFormat.java#L169 + + // Set it back. + TestSQLContext.setConf(SQLConf.PARQUET_COMPRESSION, defaultParquetCompressionCodec) + } + test("Read/Write All Types with non-primitive type") { val tempDir = getTempFilePath("parquetTest").getCanonicalPath val range = (0 to 255) From 3cedc4f4d78e093fd362085e0a077bb9e4f28ca5 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 26 Aug 2014 13:04:30 -0700 Subject: [PATCH 02/51] [SPARK-2871] [PySpark] add histgram() API RDD.histogram(buckets) Compute a histogram using the provided buckets. The buckets are all open to the right except for the last which is closed. e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50], which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1 and 50 we would have a histogram of 1,0,1. If your histogram is evenly spaced (e.g. [0, 10, 20, 30]), this can be switched from an O(log n) inseration to O(1) per element(where n = # buckets). Buckets must be sorted and not contain any duplicates, must be at least two elements. If `buckets` is a number, it will generates buckets which is evenly spaced between the minimum and maximum of the RDD. For example, if the min value is 0 and the max is 100, given buckets as 2, the resulting buckets will be [0,50) [50,100]. buckets must be at least 1 If the RDD contains infinity, NaN throws an exception If the elements in RDD do not vary (max == min) always returns a single bucket. It will return an tuple of buckets and histogram. >>> rdd = sc.parallelize(range(51)) >>> rdd.histogram(2) ([0, 25, 50], [25, 26]) >>> rdd.histogram([0, 5, 25, 50]) ([0, 5, 25, 50], [5, 20, 26]) >>> rdd.histogram([0, 15, 30, 45, 60], True) ([0, 15, 30, 45, 60], [15, 15, 15, 6]) >>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"]) >>> rdd.histogram(("a", "b", "c")) (('a', 'b', 'c'), [2, 2]) closes #122, it's duplicated. Author: Davies Liu Closes #2091 from davies/histgram and squashes the following commits: a322f8a [Davies Liu] fix deprecation of e.message 84e85fa [Davies Liu] remove evenBuckets, add more tests (including str) d9a0722 [Davies Liu] address comments 0e18a2d [Davies Liu] add histgram() API --- python/pyspark/rdd.py | 129 +++++++++++++++++++++++++++++++++++++++- python/pyspark/tests.py | 104 ++++++++++++++++++++++++++++++++ 2 files changed, 232 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1374f74968c9e..3a2e7649e6827 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -32,7 +32,7 @@ import heapq import bisect from random import Random -from math import sqrt, log +from math import sqrt, log, isinf, isnan from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ @@ -886,6 +886,133 @@ def redFunc(left_counter, right_counter): return self.mapPartitions(lambda i: [StatCounter(i)]).reduce(redFunc) + def histogram(self, buckets): + """ + Compute a histogram using the provided buckets. The buckets + are all open to the right except for the last which is closed. + e.g. [1,10,20,50] means the buckets are [1,10) [10,20) [20,50], + which means 1<=x<10, 10<=x<20, 20<=x<=50. And on the input of 1 + and 50 we would have a histogram of 1,0,1. + + If your histogram is evenly spaced (e.g. [0, 10, 20, 30]), + this can be switched from an O(log n) inseration to O(1) per + element(where n = # buckets). + + Buckets must be sorted and not contain any duplicates, must be + at least two elements. + + If `buckets` is a number, it will generates buckets which are + evenly spaced between the minimum and maximum of the RDD. For + example, if the min value is 0 and the max is 100, given buckets + as 2, the resulting buckets will be [0,50) [50,100]. buckets must + be at least 1 If the RDD contains infinity, NaN throws an exception + If the elements in RDD do not vary (max == min) always returns + a single bucket. + + It will return an tuple of buckets and histogram. + + >>> rdd = sc.parallelize(range(51)) + >>> rdd.histogram(2) + ([0, 25, 50], [25, 26]) + >>> rdd.histogram([0, 5, 25, 50]) + ([0, 5, 25, 50], [5, 20, 26]) + >>> rdd.histogram([0, 15, 30, 45, 60]) # evenly spaced buckets + ([0, 15, 30, 45, 60], [15, 15, 15, 6]) + >>> rdd = sc.parallelize(["ab", "ac", "b", "bd", "ef"]) + >>> rdd.histogram(("a", "b", "c")) + (('a', 'b', 'c'), [2, 2]) + """ + + if isinstance(buckets, (int, long)): + if buckets < 1: + raise ValueError("number of buckets must be >= 1") + + # filter out non-comparable elements + def comparable(x): + if x is None: + return False + if type(x) is float and isnan(x): + return False + return True + + filtered = self.filter(comparable) + + # faster than stats() + def minmax(a, b): + return min(a[0], b[0]), max(a[1], b[1]) + try: + minv, maxv = filtered.map(lambda x: (x, x)).reduce(minmax) + except TypeError as e: + if " empty " in str(e): + raise ValueError("can not generate buckets from empty RDD") + raise + + if minv == maxv or buckets == 1: + return [minv, maxv], [filtered.count()] + + try: + inc = (maxv - minv) / buckets + except TypeError: + raise TypeError("Can not generate buckets with non-number in RDD") + + if isinf(inc): + raise ValueError("Can not generate buckets with infinite value") + + # keep them as integer if possible + if inc * buckets != maxv - minv: + inc = (maxv - minv) * 1.0 / buckets + + buckets = [i * inc + minv for i in range(buckets)] + buckets.append(maxv) # fix accumulated error + even = True + + elif isinstance(buckets, (list, tuple)): + if len(buckets) < 2: + raise ValueError("buckets should have more than one value") + + if any(i is None or isinstance(i, float) and isnan(i) for i in buckets): + raise ValueError("can not have None or NaN in buckets") + + if sorted(buckets) != list(buckets): + raise ValueError("buckets should be sorted") + + if len(set(buckets)) != len(buckets): + raise ValueError("buckets should not contain duplicated values") + + minv = buckets[0] + maxv = buckets[-1] + even = False + inc = None + try: + steps = [buckets[i + 1] - buckets[i] for i in range(len(buckets) - 1)] + except TypeError: + pass # objects in buckets do not support '-' + else: + if max(steps) - min(steps) < 1e-10: # handle precision errors + even = True + inc = (maxv - minv) / (len(buckets) - 1) + + else: + raise TypeError("buckets should be a list or tuple or number(int or long)") + + def histogram(iterator): + counters = [0] * len(buckets) + for i in iterator: + if i is None or (type(i) is float and isnan(i)) or i > maxv or i < minv: + continue + t = (int((i - minv) / inc) if even + else bisect.bisect_right(buckets, i) - 1) + counters[t] += 1 + # add last two together + last = counters.pop() + counters[-1] += last + return [counters] + + def mergeCounters(a, b): + return [i + j for i, j in zip(a, b)] + + return buckets, self.mapPartitions(histogram).reduce(mergeCounters) + def mean(self): """ Compute the mean of this RDD's elements. diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 51bfbb47e53c2..1db922f513743 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -364,6 +364,110 @@ def test_zip_with_different_number_of_items(self): self.assertEquals(a.count(), b.count()) self.assertRaises(Exception, lambda: a.zip(b).count()) + def test_histogram(self): + # empty + rdd = self.sc.parallelize([]) + self.assertEquals([0], rdd.histogram([0, 10])[1]) + self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertRaises(ValueError, lambda: rdd.histogram(1)) + + # out of range + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0], rdd.histogram([0, 10])[1]) + self.assertEquals([0, 0], rdd.histogram((0, 4, 10))[1]) + + # in range with one bucket + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals([4], rdd.histogram([0, 10])[1]) + self.assertEquals([3, 1], rdd.histogram([0, 4, 10])[1]) + + # in range with one bucket exact match + self.assertEquals([4], rdd.histogram([1, 4])[1]) + + # out of range with two buckets + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0, 0], rdd.histogram([0, 5, 10])[1]) + + # out of range with two uneven buckets + rdd = self.sc.parallelize([10.01, -0.01]) + self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + + # in range with two buckets + rdd = self.sc.parallelize([1, 2, 3, 5, 6]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + + # in range with two bucket and None + rdd = self.sc.parallelize([1, 2, 3, 5, 6, None, float('nan')]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + + # in range with two uneven buckets + rdd = self.sc.parallelize([1, 2, 3, 5, 6]) + self.assertEquals([3, 2], rdd.histogram([0, 5, 11])[1]) + + # mixed range with two uneven buckets + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01]) + self.assertEquals([4, 3], rdd.histogram([0, 5, 11])[1]) + + # mixed range with four uneven buckets + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1]) + self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + + # mixed range with uneven buckets and NaN + rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, + 199.0, 200.0, 200.1, None, float('nan')]) + self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + + # out of range with infinite buckets + rdd = self.sc.parallelize([10.01, -0.01, float('nan'), float("inf")]) + self.assertEquals([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) + + # invalid buckets + self.assertRaises(ValueError, lambda: rdd.histogram([])) + self.assertRaises(ValueError, lambda: rdd.histogram([1])) + self.assertRaises(ValueError, lambda: rdd.histogram(0)) + self.assertRaises(TypeError, lambda: rdd.histogram({})) + + # without buckets + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals(([1, 4], [4]), rdd.histogram(1)) + + # without buckets single element + rdd = self.sc.parallelize([1]) + self.assertEquals(([1, 1], [1]), rdd.histogram(1)) + + # without bucket no range + rdd = self.sc.parallelize([1] * 4) + self.assertEquals(([1, 1], [4]), rdd.histogram(1)) + + # without buckets basic two + rdd = self.sc.parallelize(range(1, 5)) + self.assertEquals(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) + + # without buckets with more requested than elements + rdd = self.sc.parallelize([1, 2]) + buckets = [1 + 0.2 * i for i in range(6)] + hist = [1, 0, 0, 0, 1] + self.assertEquals((buckets, hist), rdd.histogram(5)) + + # invalid RDDs + rdd = self.sc.parallelize([1, float('inf')]) + self.assertRaises(ValueError, lambda: rdd.histogram(2)) + rdd = self.sc.parallelize([float('nan')]) + self.assertRaises(ValueError, lambda: rdd.histogram(2)) + + # string + rdd = self.sc.parallelize(["ab", "ac", "b", "bd", "ef"], 2) + self.assertEquals([2, 2], rdd.histogram(["a", "b", "c"])[1]) + self.assertEquals((["ab", "ef"], [5]), rdd.histogram(1)) + self.assertRaises(TypeError, lambda: rdd.histogram(2)) + + # mixed RDD + rdd = self.sc.parallelize([1, 4, "ab", "ac", "b"], 2) + self.assertEquals([1, 1], rdd.histogram([0, 4, 10])[1]) + self.assertEquals([2, 1], rdd.histogram(["a", "b", "c"])[1]) + self.assertEquals(([1, "b"], [5]), rdd.histogram(1)) + self.assertRaises(TypeError, lambda: rdd.histogram(2)) + class TestIO(PySparkTestCase): From 98c2bb0bbde6fb2b6f64af3efffefcb0dae94c12 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 26 Aug 2014 13:22:55 -0700 Subject: [PATCH 03/51] [SPARK-2969][SQL] Make ScalaReflection be able to handle ArrayType.containsNull and MapType.valueContainsNull. Make `ScalaReflection` be able to handle like: - `Seq[Int]` as `ArrayType(IntegerType, containsNull = false)` - `Seq[java.lang.Integer]` as `ArrayType(IntegerType, containsNull = true)` - `Map[Int, Long]` as `MapType(IntegerType, LongType, valueContainsNull = false)` - `Map[Int, java.lang.Long]` as `MapType(IntegerType, LongType, valueContainsNull = true)` Author: Takuya UESHIN Closes #1889 from ueshin/issues/SPARK-2969 and squashes the following commits: 24f1c5c [Takuya UESHIN] Change the default value of ArrayType.containsNull to true in Python API. 79f5b65 [Takuya UESHIN] Change the default value of ArrayType.containsNull to true in Java API. 7cd1a7a [Takuya UESHIN] Fix json test failures. 2cfb862 [Takuya UESHIN] Change the default value of ArrayType.containsNull to true. 2f38e61 [Takuya UESHIN] Revert the default value of MapTypes.valueContainsNull. 9fa02f5 [Takuya UESHIN] Fix a test failure. 1a9a96b [Takuya UESHIN] Modify ScalaReflection to handle ArrayType.containsNull and MapType.valueContainsNull. --- python/pyspark/sql.py | 6 ++-- .../spark/sql/catalyst/ScalaReflection.scala | 9 ++++-- .../spark/sql/catalyst/types/dataTypes.scala | 4 +-- .../sql/catalyst/ScalaReflectionSuite.scala | 22 +++++++++++-- .../apache/spark/sql/api/java/DataType.java | 4 +-- .../org/apache/spark/sql/DataTypeSuite.scala | 2 +- .../org/apache/spark/sql/json/JsonSuite.scala | 32 +++++++++---------- 7 files changed, 49 insertions(+), 30 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index d4ca0cc8f336e..0ff6a548a85f1 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -186,15 +186,15 @@ class ArrayType(DataType): """ - def __init__(self, elementType, containsNull=False): + def __init__(self, elementType, containsNull=True): """Creates an ArrayType :param elementType: the data type of elements. :param containsNull: indicates whether the list contains None values. - >>> ArrayType(StringType) == ArrayType(StringType, False) + >>> ArrayType(StringType) == ArrayType(StringType, True) True - >>> ArrayType(StringType, True) == ArrayType(StringType) + >>> ArrayType(StringType, False) == ArrayType(StringType) False """ self.elementType = elementType 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 0d26b52a84695..6b6b636cd96dc 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 @@ -62,11 +62,14 @@ object ScalaReflection { sys.error(s"Only Array[Byte] supported now, use Seq instead of $t") case t if t <:< typeOf[Seq[_]] => val TypeRef(_, _, Seq(elementType)) = t - Schema(ArrayType(schemaFor(elementType).dataType), nullable = true) + val Schema(dataType, nullable) = schemaFor(elementType) + Schema(ArrayType(dataType, containsNull = nullable), nullable = true) case t if t <:< typeOf[Map[_,_]] => val TypeRef(_, _, Seq(keyType, valueType)) = t - Schema(MapType(schemaFor(keyType).dataType, schemaFor(valueType).dataType), nullable = true) - case t if t <:< typeOf[String] => Schema(StringType, nullable = true) + val Schema(valueDataType, valueNullable) = schemaFor(valueType) + Schema(MapType(schemaFor(keyType).dataType, + valueDataType, valueContainsNull = valueNullable), nullable = true) + case t if t <:< typeOf[String] => Schema(StringType, nullable = true) case t if t <:< typeOf[Timestamp] => Schema(TimestampType, nullable = true) case t if t <:< typeOf[BigDecimal] => Schema(DecimalType, nullable = true) case t if t <:< typeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala index b52ee6d3378a3..70c6d06cf2534 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -270,8 +270,8 @@ case object FloatType extends FractionalType { } object ArrayType { - /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is false. */ - def apply(elementType: DataType): ArrayType = ArrayType(elementType, false) + /** Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. */ + def apply(elementType: DataType): ArrayType = ArrayType(elementType, true) } /** diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala index e75373d5a74a7..428607d8c8253 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ScalaReflectionSuite.scala @@ -57,7 +57,9 @@ case class OptionalData( case class ComplexData( arrayField: Seq[Int], - mapField: Map[Int, String], + arrayFieldContainsNull: Seq[java.lang.Integer], + mapField: Map[Int, Long], + mapFieldValueContainsNull: Map[Int, java.lang.Long], structField: PrimitiveData) case class GenericData[A]( @@ -116,8 +118,22 @@ class ScalaReflectionSuite extends FunSuite { val schema = schemaFor[ComplexData] assert(schema === Schema( StructType(Seq( - StructField("arrayField", ArrayType(IntegerType), nullable = true), - StructField("mapField", MapType(IntegerType, StringType), nullable = true), + StructField( + "arrayField", + ArrayType(IntegerType, containsNull = false), + nullable = true), + StructField( + "arrayFieldContainsNull", + ArrayType(IntegerType, containsNull = true), + nullable = true), + StructField( + "mapField", + MapType(IntegerType, LongType, valueContainsNull = false), + nullable = true), + StructField( + "mapFieldValueContainsNull", + MapType(IntegerType, LongType, valueContainsNull = true), + nullable = true), StructField( "structField", StructType(Seq( diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java index 3eccddef88134..37b4c8ffcba0b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/DataType.java @@ -86,14 +86,14 @@ public abstract class DataType { /** * Creates an ArrayType by specifying the data type of elements ({@code elementType}). - * The field of {@code containsNull} is set to {@code false}. + * The field of {@code containsNull} is set to {@code true}. */ public static ArrayType createArrayType(DataType elementType) { if (elementType == null) { throw new IllegalArgumentException("elementType should not be null."); } - return new ArrayType(elementType, false); + return new ArrayType(elementType, true); } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala index cf7d79f42db1d..8fb59c5830f6d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataTypeSuite.scala @@ -24,7 +24,7 @@ class DataTypeSuite extends FunSuite { test("construct an ArrayType") { val array = ArrayType(StringType) - assert(ArrayType(StringType, false) === array) + assert(ArrayType(StringType, true) === array) } test("construct an MapType") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 58b1e23891a3b..05513a127150c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -130,11 +130,11 @@ class JsonSuite extends QueryTest { checkDataType( ArrayType(IntegerType, true), ArrayType(IntegerType, true), ArrayType(IntegerType, true)) checkDataType( - ArrayType(IntegerType, false), ArrayType(IntegerType), ArrayType(IntegerType, false)) + ArrayType(IntegerType, false), ArrayType(IntegerType), ArrayType(IntegerType, true)) checkDataType( ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType, false)) checkDataType( - ArrayType(IntegerType, false), ArrayType(IntegerType, false), ArrayType(IntegerType)) + ArrayType(IntegerType, false), ArrayType(IntegerType, true), ArrayType(IntegerType, true)) // StructType checkDataType(StructType(Nil), StructType(Nil), StructType(Nil)) @@ -201,26 +201,26 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonRDD(complexFieldAndType) val expectedSchema = StructType( - StructField("arrayOfArray1", ArrayType(ArrayType(StringType)), true) :: - StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType)), true) :: - StructField("arrayOfBigInteger", ArrayType(DecimalType), true) :: - StructField("arrayOfBoolean", ArrayType(BooleanType), true) :: - StructField("arrayOfDouble", ArrayType(DoubleType), true) :: - StructField("arrayOfInteger", ArrayType(IntegerType), true) :: - StructField("arrayOfLong", ArrayType(LongType), true) :: + StructField("arrayOfArray1", ArrayType(ArrayType(StringType, false), false), true) :: + StructField("arrayOfArray2", ArrayType(ArrayType(DoubleType, false), false), true) :: + StructField("arrayOfBigInteger", ArrayType(DecimalType, false), true) :: + StructField("arrayOfBoolean", ArrayType(BooleanType, false), true) :: + StructField("arrayOfDouble", ArrayType(DoubleType, false), true) :: + StructField("arrayOfInteger", ArrayType(IntegerType, false), true) :: + StructField("arrayOfLong", ArrayType(LongType, false), true) :: StructField("arrayOfNull", ArrayType(StringType, true), true) :: - StructField("arrayOfString", ArrayType(StringType), true) :: + StructField("arrayOfString", ArrayType(StringType, false), true) :: StructField("arrayOfStruct", ArrayType( StructType( StructField("field1", BooleanType, true) :: StructField("field2", StringType, true) :: - StructField("field3", StringType, true) :: Nil)), true) :: + StructField("field3", StringType, true) :: Nil), false), true) :: StructField("struct", StructType( StructField("field1", BooleanType, true) :: StructField("field2", DecimalType, true) :: Nil), true) :: StructField("structWithArrayFields", StructType( - StructField("field1", ArrayType(IntegerType), true) :: - StructField("field2", ArrayType(StringType), true) :: Nil), true) :: Nil) + StructField("field1", ArrayType(IntegerType, false), true) :: + StructField("field2", ArrayType(StringType, false), true) :: Nil), true) :: Nil) assert(expectedSchema === jsonSchemaRDD.schema) @@ -441,7 +441,7 @@ class JsonSuite extends QueryTest { val jsonSchemaRDD = jsonRDD(complexFieldValueTypeConflict) val expectedSchema = StructType( - StructField("array", ArrayType(IntegerType), true) :: + StructField("array", ArrayType(IntegerType, false), true) :: StructField("num_struct", StringType, true) :: StructField("str_array", StringType, true) :: StructField("struct", StructType( @@ -467,7 +467,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("array1", ArrayType(StringType, true), true) :: StructField("array2", ArrayType(StructType( - StructField("field", LongType, true) :: Nil)), true) :: Nil) + StructField("field", LongType, true) :: Nil), false), true) :: Nil) assert(expectedSchema === jsonSchemaRDD.schema) @@ -492,7 +492,7 @@ class JsonSuite extends QueryTest { val expectedSchema = StructType( StructField("a", BooleanType, true) :: StructField("b", LongType, true) :: - StructField("c", ArrayType(IntegerType), true) :: + StructField("c", ArrayType(IntegerType, false), true) :: StructField("d", StructType( StructField("field", BooleanType, true) :: Nil), true) :: StructField("e", StringType, true) :: Nil) From 6b5584ef1c605cd30f25dbe7099ab32aea1746fb Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 26 Aug 2014 15:04:08 -0700 Subject: [PATCH 04/51] [SPARK-3063][SQL] ExistingRdd should convert Map to catalyst Map. Currently `ExistingRdd.convertToCatalyst` doesn't convert `Map` value. Author: Takuya UESHIN Closes #1963 from ueshin/issues/SPARK-3063 and squashes the following commits: 3ba41f2 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063 4d7bae2 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063 9321379 [Takuya UESHIN] Merge branch 'master' into issues/SPARK-3063 d8a900a [Takuya UESHIN] Make ExistingRdd.convertToCatalyst be able to convert Map value. --- .../spark/sql/execution/basicOperators.scala | 3 +- .../sql/ScalaReflectionRelationSuite.scala | 46 +++++++++++++++++++ 2 files changed, 48 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index f9dfa3c92f1eb..374af48b820c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -206,7 +206,8 @@ case class Sort( object ExistingRdd { def convertToCatalyst(a: Any): Any = a match { case o: Option[_] => o.orNull - case s: Seq[Any] => s.map(convertToCatalyst) + case s: Seq[_] => s.map(convertToCatalyst) + case m: Map[_, _] => m.map { case (k, v) => convertToCatalyst(k) -> convertToCatalyst(v) } case p: Product => new GenericRow(p.productIterator.map(convertToCatalyst).toArray) case other => other } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 5b84c658db942..e24c521d24c7a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -21,6 +21,7 @@ import java.sql.Timestamp import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.test.TestSQLContext._ case class ReflectData( @@ -56,6 +57,22 @@ case class OptionalReflectData( case class ReflectBinary(data: Array[Byte]) +case class Nested(i: Option[Int], s: String) + +case class Data( + array: Seq[Int], + arrayContainsNull: Seq[Option[Int]], + map: Map[Int, Long], + mapContainsNul: Map[Int, Option[Long]], + nested: Nested) + +case class ComplexReflectData( + arrayField: Seq[Int], + arrayFieldContainsNull: Seq[Option[Int]], + mapField: Map[Int, Long], + mapFieldContainsNull: Map[Int, Option[Long]], + dataField: Data) + class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, @@ -90,4 +107,33 @@ class ScalaReflectionRelationSuite extends FunSuite { val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] assert(result.toSeq === Seq[Byte](1)) } + + test("query complex data") { + val data = ComplexReflectData( + Seq(1, 2, 3), + Seq(Some(1), Some(2), None), + Map(1 -> 10L, 2 -> 20L), + Map(1 -> Some(10L), 2 -> Some(20L), 3 -> None), + Data( + Seq(10, 20, 30), + Seq(Some(10), Some(20), None), + Map(10 -> 100L, 20 -> 200L), + Map(10 -> Some(100L), 20 -> Some(200L), 30 -> None), + Nested(None, "abc"))) + val rdd = sparkContext.parallelize(data :: Nil) + rdd.registerTempTable("reflectComplexData") + + assert(sql("SELECT * FROM reflectComplexData").collect().head === + new GenericRow(Array[Any]( + Seq(1, 2, 3), + Seq(1, 2, null), + Map(1 -> 10L, 2 -> 20L), + Map(1 -> 10L, 2 -> 20L, 3 -> null), + new GenericRow(Array[Any]( + Seq(10, 20, 30), + Seq(10, 20, null), + Map(10 -> 100L, 20 -> 200L), + Map(10 -> 100L, 20 -> 200L, 30 -> null), + new GenericRow(Array[Any](null, "abc"))))))) + } } From adbd5c1636669fc474ab02b54cd1ced353f68712 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 26 Aug 2014 15:12:27 -0700 Subject: [PATCH 05/51] [SPARK-3226][MLLIB] doc update for native libraries to mention `-Pnetlib-lgpl` option. atalwalkar Author: Xiangrui Meng Closes #2128 from mengxr/mllib-native and squashes the following commits: 4cbba57 [Xiangrui Meng] update mllib dependencies --- docs/mllib-guide.md | 25 +++++++++++++++---------- 1 file changed, 15 insertions(+), 10 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index ca0a84a8c53fd..4d4198b9e0452 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -35,18 +35,23 @@ and the migration guide below will explain all changes between releases. # Dependencies -MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), which depends on -[netlib-java](https://github.com/fommil/netlib-java), and -[jblas](https://github.com/mikiobraun/jblas). +MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), +which depends on [netlib-java](https://github.com/fommil/netlib-java), +and [jblas](https://github.com/mikiobraun/jblas). `netlib-java` and `jblas` depend on native Fortran routines. You need to install the -[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) if it is not -already present on your nodes. MLlib will throw a linking error if it cannot detect these libraries -automatically. Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's -dependency set. If no native library is available at runtime, you will see a warning message. To -use native libraries from `netlib-java`, please include artifact -`com.github.fommil.netlib:all:1.1.2` as a dependency of your project or build your own (see -[instructions](https://github.com/fommil/netlib-java/blob/master/README.md#machine-optimised-system-libraries)). +[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) +if it is not already present on your nodes. +MLlib will throw a linking error if it cannot detect these libraries automatically. +Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's +dependency set under default settings. +If no native library is available at runtime, you will see a warning message. +To use native libraries from `netlib-java`, please build Spark with `-Pnetlib-lgpl` or +include `com.github.fommil.netlib:all:1.1.2` as a dependency of your project. +If you want to use optimized BLAS/LAPACK libraries such as +[OpenBLAS](http://www.openblas.net/), please link its shared libraries to +`/usr/lib/libblas.so.3` and `/usr/lib/liblapack.so.3`, respectively. +BLAS/LAPACK libraries on worker nodes should be built without multithreading. To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. From 1208f72ac78960fe5060187761479b2a9a417c1b Mon Sep 17 00:00:00 2001 From: Burak Date: Tue, 26 Aug 2014 15:18:42 -0700 Subject: [PATCH 06/51] [SPARK-2839][MLlib] Stats Toolkit documentation updated Documentation updated for the Statistics Toolkit of MLlib. mengxr atalwalkar https://issues.apache.org/jira/browse/SPARK-2839 P.S. Accidentally closed #2123. New commits didn't show up after I reopened the PR. I've opened this instead and closed the old one. Author: Burak Closes #2130 from brkyvz/StatsLib-Docs and squashes the following commits: a54a855 [Burak] [SPARK-2839][MLlib] Addressed comments bfc6896 [Burak] [SPARK-2839][MLlib] Added a more specific link to colStats() for pyspark 213fe3f [Burak] [SPARK-2839][MLlib] Modifications made according to review fec4d9d [Burak] [SPARK-2830][MLlib] Stats Toolkit documentation updated --- docs/mllib-stats.md | 372 +++++++++++++++++++++++++++++++++++++++----- 1 file changed, 331 insertions(+), 41 deletions(-) diff --git a/docs/mllib-stats.md b/docs/mllib-stats.md index f25dca746ba3a..511a9fbf710cc 100644 --- a/docs/mllib-stats.md +++ b/docs/mllib-stats.md @@ -25,6 +25,85 @@ displayTitle: MLlib - Statistics Functionality \newcommand{\zero}{\mathbf{0}} \]` +## Summary Statistics + +We provide column summary statistics for `RDD[Vector]` through the function `colStats` +available in `Statistics`. + +
+
+ +[`colStats()`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) returns an instance of +[`MultivariateStatisticalSummary`](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} + +val observations: RDD[Vector] = ... // an RDD of Vectors + +// Compute column summary statistics. +val summary: MultivariateStatisticalSummary = Statistics.colStats(observations) +println(summary.mean) // a dense vector containing the mean value for each column +println(summary.variance) // column-wise variance +println(summary.numNonzeros) // number of nonzeros in each column + +{% endhighlight %} +
+ +
+ +[`colStats()`](api/java/org/apache/spark/mllib/stat/Statistics.html) returns an instance of +[`MultivariateStatisticalSummary`](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight java %} +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; +import org.apache.spark.mllib.stat.Statistics; + +JavaSparkContext jsc = ... + +JavaRDD mat = ... // an RDD of Vectors + +// Compute column summary statistics. +MultivariateStatisticalSummary summary = Statistics.colStats(mat.rdd()); +System.out.println(summary.mean()); // a dense vector containing the mean value for each column +System.out.println(summary.variance()); // column-wise variance +System.out.println(summary.numNonzeros()); // number of nonzeros in each column + +{% endhighlight %} +
+ +
+[`colStats()`](api/python/pyspark.mllib.stat.Statistics-class.html#colStats) returns an instance of +[`MultivariateStatisticalSummary`](api/python/pyspark.mllib.stat.MultivariateStatisticalSummary-class.html), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight python %} +from pyspark.mllib.stat import Statistics + +sc = ... # SparkContext + +mat = ... # an RDD of Vectors + +# Compute column summary statistics. +summary = Statistics.colStats(mat) +print summary.mean() +print summary.variance() +print summary.numNonzeros() + +{% endhighlight %} +
+ +
+ ## Random data generation Random data generation is useful for randomized algorithms, prototyping, and performance testing. @@ -99,69 +178,280 @@ v = u.map(lambda x: 1.0 + 2.0 * x) -## Stratified Sampling +## Correlations calculation -## Summary Statistics +Calculating the correlation between two series of data is a common operation in Statistics. In MLlib +we provide the flexibility to calculate pairwise correlations among many series. The supported +correlation methods are currently Pearson's and Spearman's correlation. + +
+
+[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to +calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or +an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.stat.Statistics + +val sc: SparkContext = ... + +val seriesX: RDD[Double] = ... // a series +val seriesY: RDD[Double] = ... // must have the same number of partitions and cardinality as seriesX + +// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +// method is not specified, Pearson's method will be used by default. +val correlation: Double = Statistics.corr(seriesX, seriesY, "pearson") + +val data: RDD[Vector] = ... // note that each Vector is a row and not a column + +// calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +// If a method is not specified, Pearson's method will be used by default. +val correlMatrix: Matrix = Statistics.corr(data, "pearson") + +{% endhighlight %} +
+ +
+[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to +calculate correlations between series. Depending on the type of input, two `JavaDoubleRDD`s or +a `JavaRDD`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight java %} +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.*; +import org.apache.spark.mllib.stat.Statistics; + +JavaSparkContext jsc = ... + +JavaDoubleRDD seriesX = ... // a series +JavaDoubleRDD seriesY = ... // must have the same number of partitions and cardinality as seriesX + +// compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +// method is not specified, Pearson's method will be used by default. +Double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson"); + +JavaRDD data = ... // note that each Vector is a row and not a column + +// calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +// If a method is not specified, Pearson's method will be used by default. +Matrix correlMatrix = Statistics.corr(data.rdd(), "pearson"); + +{% endhighlight %} +
-### Multivariate summary statistics +
+[`Statistics`](api/python/pyspark.mllib.stat.Statistics-class.html) provides methods to +calculate correlations between series. Depending on the type of input, two `RDD[Double]`s or +an `RDD[Vector]`, the output will be a `Double` or the correlation `Matrix` respectively. + +{% highlight python %} +from pyspark.mllib.stat import Statistics + +sc = ... # SparkContext + +seriesX = ... # a series +seriesY = ... # must have the same number of partitions and cardinality as seriesX -We provide column summary statistics for `RowMatrix` (note: this functionality is not currently supported in `IndexedRowMatrix` or `CoordinateMatrix`). -If the number of columns is not large, e.g., on the order of thousands, then the -covariance matrix can also be computed as a local matrix, which requires $\mathcal{O}(n^2)$ storage where $n$ is the -number of columns. The total CPU time is $\mathcal{O}(m n^2)$, where $m$ is the number of rows, -and is faster if the rows are sparse. +# Compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. If a +# method is not specified, Pearson's method will be used by default. +print Statistics.corr(seriesX, seriesY, method="pearson") + +data = ... # an RDD of Vectors +# calculate the correlation matrix using Pearson's method. Use "spearman" for Spearman's method. +# If a method is not specified, Pearson's method will be used by default. +print Statistics.corr(data, method="pearson") + +{% endhighlight %} +
+ +
+ +## Stratified sampling + +Unlike the other statistics functions, which reside in MLLib, stratified sampling methods, +`sampleByKey` and `sampleByKeyExact`, can be performed on RDD's of key-value pairs. For stratified +sampling, the keys can be thought of as a label and the value as a specific attribute. For example +the key can be man or woman, or document ids, and the respective values can be the list of ages +of the people in the population or the list of words in the documents. The `sampleByKey` method +will flip a coin to decide whether an observation will be sampled or not, therefore requires one +pass over the data, and provides an *expected* sample size. `sampleByKeyExact` requires significant +more resources than the per-stratum simple random sampling used in `sampleByKey`, but will provide +the exact sampling size with 99.99% confidence. `sampleByKeyExact` is currently not supported in +python.
- -[`computeColumnSummaryStatistics()`](api/scala/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) returns an instance of -[`MultivariateStatisticalSummary`](api/scala/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), -which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the -total count. +[`sampleByKeyExact()`](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions) allows users to +sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired +fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of +keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample +size, whereas sampling with replacement requires two additional passes. {% highlight scala %} -import org.apache.spark.mllib.linalg.Matrix -import org.apache.spark.mllib.linalg.distributed.RowMatrix -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.PairRDDFunctions -val mat: RowMatrix = ... // a RowMatrix +val sc: SparkContext = ... -// Compute column summary statistics. -val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() -println(summary.mean) // a dense vector containing the mean value for each column -println(summary.variance) // column-wise variance -println(summary.numNonzeros) // number of nonzeros in each column +val data = ... // an RDD[(K, V)] of any key value pairs +val fractions: Map[K, Double] = ... // specify the exact fraction desired from each key + +// Get an exact sample from each stratum +val approxSample = data.sampleByKey(withReplacement = false, fractions) +val exactSample = data.sampleByKeyExact(withReplacement = false, fractions) -// Compute the covariance matrix. -val cov: Matrix = mat.computeCovariance() {% endhighlight %}
- -[`RowMatrix#computeColumnSummaryStatistics`](api/java/org/apache/spark/mllib/linalg/distributed/RowMatrix.html#computeColumnSummaryStatistics()) returns an instance of -[`MultivariateStatisticalSummary`](api/java/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.html), -which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the -total count. +[`sampleByKeyExact()`](api/java/org/apache/spark/api/java/JavaPairRDD.html) allows users to +sample exactly $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the desired +fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the set of +keys. Sampling without replacement requires one additional pass over the RDD to guarantee sample +size, whereas sampling with replacement requires two additional passes. {% highlight java %} -import org.apache.spark.mllib.linalg.Matrix; -import org.apache.spark.mllib.linalg.distributed.RowMatrix; -import org.apache.spark.mllib.stat.MultivariateStatisticalSummary; +import java.util.Map; -RowMatrix mat = ... // a RowMatrix +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaSparkContext; -// Compute column summary statistics. -MultivariateStatisticalSummary summary = mat.computeColumnSummaryStatistics(); -System.out.println(summary.mean()); // a dense vector containing the mean value for each column -System.out.println(summary.variance()); // column-wise variance -System.out.println(summary.numNonzeros()); // number of nonzeros in each column +JavaSparkContext jsc = ... + +JavaPairRDD data = ... // an RDD of any key value pairs +Map fractions = ... // specify the exact fraction desired from each key + +// Get an exact sample from each stratum +JavaPairRDD approxSample = data.sampleByKey(false, fractions); +JavaPairRDD exactSample = data.sampleByKeyExact(false, fractions); + +{% endhighlight %} +
+
+[`sampleByKey()`](api/python/pyspark.rdd.RDD-class.html#sampleByKey) allows users to +sample approximately $\lceil f_k \cdot n_k \rceil \, \forall k \in K$ items, where $f_k$ is the +desired fraction for key $k$, $n_k$ is the number of key-value pairs for key $k$, and $K$ is the +set of keys. + +*Note:* `sampleByKeyExact()` is currently not supported in Python. + +{% highlight python %} + +sc = ... # SparkContext + +data = ... # an RDD of any key value pairs +fractions = ... # specify the exact fraction desired from each key as a dictionary + +approxSample = data.sampleByKey(False, fractions); -// Compute the covariance matrix. -Matrix cov = mat.computeCovariance(); {% endhighlight %}
+ +
+ +## Hypothesis testing + +Hypothesis testing is a powerful tool in statistics to determine whether a result is statistically +significant, whether this result occurred by chance or not. MLlib currently supports Pearson's +chi-squared ( $\chi^2$) tests for goodness of fit and independence. The input data types determine +whether the goodness of fit or the independence test is conducted. The goodness of fit test requires +an input type of `Vector`, whereas the independence test requires a `Matrix` as input. + +MLlib also supports the input type `RDD[LabeledPoint]` to enable feature selection via chi-squared +independence tests. + +
+
+[`Statistics`](api/scala/index.html#org.apache.spark.mllib.stat.Statistics$) provides methods to +run Pearson's chi-squared tests. The following example demonstrates how to run and interpret +hypothesis tests. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.stat.Statistics._ + +val sc: SparkContext = ... + +val vec: Vector = ... // a vector composed of the frequencies of events + +// compute the goodness of fit. If a second vector to test against is not supplied as a parameter, +// the test runs against a uniform distribution. +val goodnessOfFitTestResult = Statistics.chiSqTest(vec) +println(goodnessOfFitTestResult) // summary of the test including the p-value, degrees of freedom, + // test statistic, the method used, and the null hypothesis. + +val mat: Matrix = ... // a contingency matrix + +// conduct Pearson's independence test on the input contingency matrix +val independenceTestResult = Statistics.chiSqTest(mat) +println(independenceTestResult) // summary of the test including the p-value, degrees of freedom... + +val obs: RDD[LabeledPoint] = ... // (feature, label) pairs. + +// The contingency table is constructed from the raw (feature, label) pairs and used to conduct +// the independence test. Returns an array containing the ChiSquaredTestResult for every feature +// against the label. +val featureTestResults: Array[ChiSqTestResult] = Statistics.chiSqTest(obs) +var i = 1 +featureTestResults.foreach { result => + println(s"Column $i:\n$result") + i += 1 +} // summary of the test + +{% endhighlight %}
+
+[`Statistics`](api/java/org/apache/spark/mllib/stat/Statistics.html) provides methods to +run Pearson's chi-squared tests. The following example demonstrates how to run and interpret +hypothesis tests. + +{% highlight java %} +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.*; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.stat.Statistics; +import org.apache.spark.mllib.stat.test.ChiSqTestResult; + +JavaSparkContext jsc = ... -## Hypothesis Testing +Vector vec = ... // a vector composed of the frequencies of events + +// compute the goodness of fit. If a second vector to test against is not supplied as a parameter, +// the test runs against a uniform distribution. +ChiSqTestResult goodnessOfFitTestResult = Statistics.chiSqTest(vec); +// summary of the test including the p-value, degrees of freedom, test statistic, the method used, +// and the null hypothesis. +System.out.println(goodnessOfFitTestResult); + +Matrix mat = ... // a contingency matrix + +// conduct Pearson's independence test on the input contingency matrix +ChiSqTestResult independenceTestResult = Statistics.chiSqTest(mat); +// summary of the test including the p-value, degrees of freedom... +System.out.println(independenceTestResult); + +JavaRDD obs = ... // an RDD of labeled points + +// The contingency table is constructed from the raw (feature, label) pairs and used to conduct +// the independence test. Returns an array containing the ChiSquaredTestResult for every feature +// against the label. +ChiSqTestResult[] featureTestResults = Statistics.chiSqTest(obs.rdd()); +int i = 1; +for (ChiSqTestResult result : featureTestResults) { + System.out.println("Column " + i + ":"); + System.out.println(result); // summary of the test + i++; +} + +{% endhighlight %} +
+ +
From c4787a3690a9ed3b8b2c6c294fc4a6915436b6f7 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 26 Aug 2014 16:29:14 -0700 Subject: [PATCH 07/51] [SPARK-3194][SQL] Add AttributeSet to fix bugs with invalid comparisons of AttributeReferences It is common to want to describe sets of attributes that are in various parts of a query plan. However, the semantics of putting `AttributeReference` objects into a standard Scala `Set` result in subtle bugs when references differ cosmetically. For example, with case insensitive resolution it is possible to have two references to the same attribute whose names are not equal. In this PR I introduce a new abstraction, an `AttributeSet`, which performs all comparisons using the globally unique `ExpressionId` instead of case class equality. (There is already a related class, [`AttributeMap`](https://github.com/marmbrus/spark/blob/inMemStats/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeMap.scala#L32)) This new type of set is used to fix a bug in the optimizer where needed attributes were getting projected away underneath join operators. I also took this opportunity to refactor the expression and query plan base classes. In all but one instance the logic for computing the `references` of an `Expression` were the same. Thus, I moved this logic into the base class. For query plans the semantics of the `references` method were ill defined (is it the references output? or is it those used by expression evaluation? or what?). As a result, this method wasn't really used very much. So, I removed it. TODO: - [x] Finish scala doc for `AttributeSet` - [x] Scan the code for other instances of `Set[Attribute]` and refactor them. - [x] Finish removing `references` from `QueryPlan` Author: Michael Armbrust Closes #2109 from marmbrus/attributeSets and squashes the following commits: 1c0dae5 [Michael Armbrust] work on serialization bug. 9ba868d [Michael Armbrust] Merge remote-tracking branch 'origin/master' into attributeSets 3ae5288 [Michael Armbrust] review comments 40ce7f6 [Michael Armbrust] style d577cc7 [Michael Armbrust] Scaladoc cae5d22 [Michael Armbrust] remove more references implementations d6e16be [Michael Armbrust] Remove more instances of "def references" and normal sets of attributes. fc26b49 [Michael Armbrust] Add AttributeSet class, remove references from Expression. --- .../sql/catalyst/analysis/Analyzer.scala | 6 +- .../sql/catalyst/analysis/unresolved.scala | 1 - .../catalyst/expressions/AttributeSet.scala | 106 ++++++++++++++++++ .../catalyst/expressions/BoundAttribute.scala | 2 - .../sql/catalyst/expressions/Expression.scala | 6 +- .../spark/sql/catalyst/expressions/Rand.scala | 1 - .../sql/catalyst/expressions/ScalaUdf.scala | 1 - .../sql/catalyst/expressions/SortOrder.scala | 1 - .../catalyst/expressions/WrapDynamic.scala | 2 +- .../sql/catalyst/expressions/aggregates.scala | 25 ++--- .../sql/catalyst/expressions/arithmetic.scala | 2 - .../catalyst/expressions/complexTypes.scala | 2 +- .../sql/catalyst/expressions/generators.scala | 2 - .../sql/catalyst/expressions/literals.scala | 4 +- .../expressions/namedExpressions.scala | 6 +- .../catalyst/expressions/nullFunctions.scala | 3 - .../sql/catalyst/expressions/predicates.scala | 6 +- .../spark/sql/catalyst/expressions/sets.scala | 5 - .../expressions/stringOperations.scala | 2 - .../sql/catalyst/optimizer/Optimizer.scala | 12 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 4 +- .../catalyst/plans/logical/LogicalPlan.scala | 11 +- .../plans/logical/ScriptTransformation.scala | 4 +- .../plans/logical/basicOperators.scala | 29 +---- .../catalyst/plans/logical/partitioning.scala | 4 - .../plans/physical/partitioning.scala | 3 +- .../sql/catalyst/trees/TreeNodeSuite.scala | 1 - .../org/apache/spark/sql/SQLContext.scala | 7 +- .../columnar/InMemoryColumnarTableScan.scala | 2 - .../spark/sql/execution/SparkPlan.scala | 3 +- .../spark/sql/execution/debug/package.scala | 2 - .../spark/sql/execution/pythonUdfs.scala | 2 - .../spark/sql/hive/HiveStrategies.scala | 8 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 5 - .../hive/execution/HiveResolutionSuite.scala | 9 +- 35 files changed, 166 insertions(+), 123 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala 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 c18d7858f0a43..4a9524074132e 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 @@ -132,7 +132,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case s @ Sort(ordering, p @ Project(projectList, child)) if !s.resolved && p.resolved => val unresolved = ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) val resolved = unresolved.flatMap(child.resolveChildren) - val requiredAttributes = resolved.collect { case a: Attribute => a }.toSet + val requiredAttributes = AttributeSet(resolved.collect { case a: Attribute => a }) val missingInProject = requiredAttributes -- p.output if (missingInProject.nonEmpty) { @@ -152,8 +152,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool ) logDebug(s"Grouping expressions: $groupingRelation") - val resolved = unresolved.flatMap(groupingRelation.resolve).toSet - val missingInAggs = resolved -- a.outputSet + val resolved = unresolved.flatMap(groupingRelation.resolve) + val missingInAggs = resolved.filterNot(a.outputSet.contains) logDebug(s"Resolved: $resolved Missing in aggs: $missingInAggs") if (missingInAggs.nonEmpty) { // Add missing grouping exprs and then project them away after the sort. 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 a0e25775da6dd..a2c61c65487cb 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 @@ -66,7 +66,6 @@ case class UnresolvedFunction(name: String, children: Seq[Expression]) extends E override def dataType = throw new UnresolvedException(this, "dataType") override def foldable = throw new UnresolvedException(this, "foldable") override def nullable = throw new UnresolvedException(this, "nullable") - override def references = children.flatMap(_.references).toSet override lazy val resolved = false // Unresolved functions are transient at compile time and don't get evaluated during execution. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala new file mode 100644 index 0000000000000..c3a08bbdb6bc7 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -0,0 +1,106 @@ +/* + * 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 + +protected class AttributeEquals(val a: Attribute) { + override def hashCode() = a.exprId.hashCode() + override def equals(other: Any) = other match { + case otherReference: AttributeEquals => a.exprId == otherReference.a.exprId + case otherAttribute => false + } +} + +object AttributeSet { + /** Constructs a new [[AttributeSet]] given a sequence of [[Attribute Attributes]]. */ + def apply(baseSet: Seq[Attribute]) = { + new AttributeSet(baseSet.map(new AttributeEquals(_)).toSet) + } +} + +/** + * A Set designed to hold [[AttributeReference]] objects, that performs equality checking using + * expression id instead of standard java equality. Using expression id means that these + * sets will correctly test for membership, even when the AttributeReferences in question differ + * cosmetically (e.g., the names have different capitalizations). + * + * Note that we do not override equality for Attribute references as it is really weird when + * `AttributeReference("a"...) == AttrributeReference("b", ...)`. This tactic leads to broken tests, + * and also makes doing transformations hard (we always try keep older trees instead of new ones + * when the transformation was a no-op). + */ +class AttributeSet private (val baseSet: Set[AttributeEquals]) + extends Traversable[Attribute] with Serializable { + + /** Returns true if the members of this AttributeSet and other are the same. */ + override def equals(other: Any) = other match { + case otherSet: AttributeSet => baseSet.map(_.a).forall(otherSet.contains) + case _ => false + } + + /** Returns true if this set contains an Attribute with the same expression id as `elem` */ + def contains(elem: NamedExpression): Boolean = + baseSet.contains(new AttributeEquals(elem.toAttribute)) + + /** Returns a new [[AttributeSet]] that contains `elem` in addition to the current elements. */ + def +(elem: Attribute): AttributeSet = // scalastyle:ignore + new AttributeSet(baseSet + new AttributeEquals(elem)) + + /** Returns a new [[AttributeSet]] that does not contain `elem`. */ + def -(elem: Attribute): AttributeSet = + new AttributeSet(baseSet - new AttributeEquals(elem)) + + /** Returns an iterator containing all of the attributes in the set. */ + def iterator: Iterator[Attribute] = baseSet.map(_.a).iterator + + /** + * Returns true if the [[Attribute Attributes]] in this set are a subset of the Attributes in + * `other`. + */ + def subsetOf(other: AttributeSet) = baseSet.subsetOf(other.baseSet) + + /** + * Returns a new [[AttributeSet]] that does not contain any of the [[Attribute Attributes]] found + * in `other`. + */ + def --(other: Traversable[NamedExpression]) = + new AttributeSet(baseSet -- other.map(a => new AttributeEquals(a.toAttribute))) + + /** + * Returns a new [[AttributeSet]] that contains all of the [[Attribute Attributes]] found + * in `other`. + */ + def ++(other: AttributeSet) = new AttributeSet(baseSet ++ other.baseSet) + + /** + * Returns a new [[AttributeSet]] contain only the [[Attribute Attributes]] where `f` evaluates to + * true. + */ + override def filter(f: Attribute => Boolean) = new AttributeSet(baseSet.filter(ae => f(ae.a))) + + /** + * Returns a new [[AttributeSet]] that only contains [[Attribute Attributes]] that are found in + * `this` and `other`. + */ + def intersect(other: AttributeSet) = new AttributeSet(baseSet.intersect(other.baseSet)) + + override def foreach[U](f: (Attribute) => U): Unit = baseSet.map(_.a).foreach(f) + + // We must force toSeq to not be strict otherwise we end up with a [[Stream]] that captures all + // sorts of things in its closure. + override def toSeq: Seq[Attribute] = baseSet.map(_.a).toArray.toSeq +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 0913f15888780..54c6baf1af3bf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -32,8 +32,6 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) type EvaluatedType = Any - override def references = Set.empty - override def toString = s"input[$ordinal]" override def eval(input: Row): Any = input(ordinal) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala index ba62dabe3dd6a..70507e7ee2be8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -41,7 +41,7 @@ abstract class Expression extends TreeNode[Expression] { */ def foldable: Boolean = false def nullable: Boolean - def references: Set[Attribute] + def references: AttributeSet = AttributeSet(children.flatMap(_.references.iterator)) /** Returns the result of evaluating this expression on a given input Row */ def eval(input: Row = null): EvaluatedType @@ -230,8 +230,6 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express override def foldable = left.foldable && right.foldable - override def references = left.references ++ right.references - override def toString = s"($left $symbol $right)" } @@ -242,5 +240,5 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression] abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] { self: Product => - override def references = child.references + } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala index 38f836f0a1a0e..851db95b9177e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala @@ -24,7 +24,6 @@ import org.apache.spark.sql.catalyst.types.DoubleType case object Rand extends LeafExpression { override def dataType = DoubleType override def nullable = false - override def references = Set.empty private[this] lazy val rand = new Random 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 95633dd0c9870..63ac2a608b6ff 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 @@ -24,7 +24,6 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi type EvaluatedType = Any - def references = children.flatMap(_.references).toSet def nullable = true /** This method has been generated by this script diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala index d2b7685e73065..d00b2ac09745c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala @@ -31,7 +31,6 @@ case object Descending extends SortDirection case class SortOrder(child: Expression, direction: SortDirection) extends Expression with trees.UnaryNode[Expression] { - override def references = child.references override def dataType = child.dataType override def nullable = child.nullable diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala index eb8898900d6a5..1eb55715794a7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -35,7 +35,7 @@ case class WrapDynamic(children: Seq[Attribute]) extends Expression { type EvaluatedType = DynamicRow def nullable = false - def references = children.toSet + def dataType = DynamicType override def eval(input: Row): DynamicRow = input match { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 613b87ca98d97..dbc0c2965a805 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -78,7 +78,7 @@ abstract class AggregateFunction /** Base should return the generic aggregate expression that this function is computing */ val base: AggregateExpression - override def references = base.references + override def nullable = base.nullable override def dataType = base.dataType @@ -89,7 +89,7 @@ abstract class AggregateFunction } case class Min(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = true override def dataType = child.dataType override def toString = s"MIN($child)" @@ -119,7 +119,7 @@ case class MinFunction(expr: Expression, base: AggregateExpression) extends Aggr } case class Max(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = true override def dataType = child.dataType override def toString = s"MAX($child)" @@ -149,7 +149,7 @@ case class MaxFunction(expr: Expression, base: AggregateExpression) extends Aggr } case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"COUNT($child)" @@ -166,7 +166,7 @@ case class CountDistinct(expressions: Seq[Expression]) extends PartialAggregate def this() = this(null) override def children = expressions - override def references = expressions.flatMap(_.references).toSet + override def nullable = false override def dataType = LongType override def toString = s"COUNT(DISTINCT ${expressions.mkString(",")})" @@ -184,7 +184,6 @@ case class CollectHashSet(expressions: Seq[Expression]) extends AggregateExpress def this() = this(null) override def children = expressions - override def references = expressions.flatMap(_.references).toSet override def nullable = false override def dataType = ArrayType(expressions.head.dataType) override def toString = s"AddToHashSet(${expressions.mkString(",")})" @@ -219,7 +218,6 @@ case class CombineSetsAndCount(inputSet: Expression) extends AggregateExpression def this() = this(null) override def children = inputSet :: Nil - override def references = inputSet.references override def nullable = false override def dataType = LongType override def toString = s"CombineAndCount($inputSet)" @@ -248,7 +246,7 @@ case class CombineSetsAndCountFunction( case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -257,7 +255,7 @@ case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -266,7 +264,7 @@ case class ApproxCountDistinctMerge(child: Expression, relativeSD: Double) case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = LongType override def toString = s"APPROXIMATE COUNT(DISTINCT $child)" @@ -284,7 +282,7 @@ case class ApproxCountDistinct(child: Expression, relativeSD: Double = 0.05) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = DoubleType override def toString = s"AVG($child)" @@ -304,7 +302,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"SUM($child)" @@ -322,7 +320,7 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ case class SumDistinct(child: Expression) extends AggregateExpression with trees.UnaryNode[Expression] { - override def references = child.references + override def nullable = false override def dataType = child.dataType override def toString = s"SUM(DISTINCT $child)" @@ -331,7 +329,6 @@ case class SumDistinct(child: Expression) } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - override def references = child.references override def nullable = true override def dataType = child.dataType override def toString = s"FIRST($child)" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 5f8b6ae10f0c4..aae86a3628be1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -95,8 +95,6 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { override def children = left :: right :: Nil - override def references = left.references ++ right.references - override def dataType = left.dataType override def eval(input: Row): Any = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index c1154eb81c319..dafd745ec96c6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -31,7 +31,7 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { /** `Null` is returned for invalid ordinals. */ override def nullable = true override def foldable = child.foldable && ordinal.foldable - override def references = children.flatMap(_.references).toSet + def dataType = child.dataType match { case ArrayType(dt, _) => dt case MapType(_, vt, _) => vt diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index e99c5b452d183..9c865254e0be9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -47,8 +47,6 @@ abstract class Generator extends Expression { override def nullable = false - override def references = children.flatMap(_.references).toSet - /** * Should be overridden by specific generators. Called only once for each instance to ensure * that rule application does not change the output schema of a generator. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index e15e16d633365..a8c2396d62632 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -52,7 +52,7 @@ case class Literal(value: Any, dataType: DataType) extends LeafExpression { override def foldable = true def nullable = value == null - def references = Set.empty + override def toString = if (value != null) value.toString else "null" @@ -66,8 +66,6 @@ case class MutableLiteral(var value: Any, nullable: Boolean = true) extends Leaf val dataType = Literal(value).dataType - def references = Set.empty - def update(expression: Expression, input: Row) = { value = expression.eval(input) } 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 02d04762629f5..7c4b9d4847e26 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 @@ -62,7 +62,7 @@ abstract class Attribute extends NamedExpression { def toAttribute = this def newInstance: Attribute - override def references = Set(this) + } /** @@ -85,7 +85,7 @@ case class Alias(child: Expression, name: String) override def dataType = child.dataType override def nullable = child.nullable - override def references = child.references + override def toAttribute = { if (resolved) { @@ -116,6 +116,8 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea (val exprId: ExprId = NamedExpression.newExprId, val qualifiers: Seq[String] = Nil) extends Attribute with trees.LeafNode[Expression] { + override def references = AttributeSet(this :: Nil) + override def equals(other: Any) = other match { case ar: AttributeReference => exprId == ar.exprId && dataType == ar.dataType case _ => false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala index e88c5d4fa178a..086d0a3e073e5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -26,7 +26,6 @@ case class Coalesce(children: Seq[Expression]) extends Expression { /** Coalesce is nullable if all of its children are nullable, or if it has no children. */ def nullable = !children.exists(!_.nullable) - def references = children.flatMap(_.references).toSet // Coalesce is foldable if all children are foldable. override def foldable = !children.exists(!_.foldable) @@ -53,7 +52,6 @@ case class Coalesce(children: Seq[Expression]) extends Expression { } case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references override def foldable = child.foldable def nullable = false @@ -65,7 +63,6 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr } case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[Expression] { - def references = child.references override def foldable = child.foldable def nullable = false override def toString = s"IS NOT NULL $child" 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 5976b0ddf3e03..1313ccd120c1f 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 @@ -85,7 +85,7 @@ case class Not(child: Expression) extends UnaryExpression with Predicate { */ case class In(value: Expression, list: Seq[Expression]) extends Predicate { def children = value +: list - def references = children.flatMap(_.references).toSet + def nullable = true // TODO: Figure out correct nullability semantics of IN. override def toString = s"$value IN ${list.mkString("(", ",", ")")}" @@ -197,7 +197,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi def children = predicate :: trueValue :: falseValue :: Nil override def nullable = trueValue.nullable || falseValue.nullable - def references = children.flatMap(_.references).toSet + override lazy val resolved = childrenResolved && trueValue.dataType == falseValue.dataType def dataType = { if (!resolved) { @@ -239,7 +239,7 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi case class CaseWhen(branches: Seq[Expression]) extends Expression { type EvaluatedType = Any def children = branches - def references = children.flatMap(_.references).toSet + def dataType = { if (!resolved) { throw new UnresolvedException(this, "cannot resolve due to differing types in some branches") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala index e6c570b47bee2..3d4c4a8853c12 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -26,8 +26,6 @@ import org.apache.spark.util.collection.OpenHashSet case class NewSet(elementType: DataType) extends LeafExpression { type EvaluatedType = Any - def references = Set.empty - def nullable = false // We are currently only using these Expressions internally for aggregation. However, if we ever @@ -53,9 +51,6 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression { def nullable = set.nullable def dataType = set.dataType - - def references = (item.flatMap(_.references) ++ set.flatMap(_.references)).toSet - def eval(input: Row): Any = { val itemEval = item.eval(input) val setEval = set.eval(input).asInstanceOf[OpenHashSet[Any]] diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index 97fc3a3b14b88..c2a3a5ca3ca8b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -226,8 +226,6 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends if (str.dataType == BinaryType) str.dataType else StringType } - def references = children.flatMap(_.references).toSet - override def children = str :: pos :: len :: Nil @inline 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 5f86d6047cb9c..ddd4b3755d629 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 @@ -65,8 +65,10 @@ object ColumnPruning extends Rule[LogicalPlan] { // Eliminate unneeded attributes from either side of a Join. case Project(projectList, Join(left, right, joinType, condition)) => // Collect the list of all references required either above or to evaluate the condition. - val allReferences: Set[Attribute] = - projectList.flatMap(_.references).toSet ++ condition.map(_.references).getOrElse(Set.empty) + val allReferences: AttributeSet = + AttributeSet( + projectList.flatMap(_.references.iterator)) ++ + condition.map(_.references).getOrElse(AttributeSet(Seq.empty)) /** Applies a projection only when the child is producing unnecessary attributes */ def pruneJoinChild(c: LogicalPlan) = prunedChild(c, allReferences) @@ -76,8 +78,8 @@ object ColumnPruning extends Rule[LogicalPlan] { // Eliminate unneeded attributes from right side of a LeftSemiJoin. case Join(left, right, LeftSemi, condition) => // Collect the list of all references required to evaluate the condition. - val allReferences: Set[Attribute] = - condition.map(_.references).getOrElse(Set.empty) + val allReferences: AttributeSet = + condition.map(_.references).getOrElse(AttributeSet(Seq.empty)) Join(left, prunedChild(right, allReferences), LeftSemi, condition) @@ -104,7 +106,7 @@ object ColumnPruning extends Rule[LogicalPlan] { } /** Applies a projection only when the child is producing unnecessary attributes */ - private def prunedChild(c: LogicalPlan, allReferences: Set[Attribute]) = + private def prunedChild(c: LogicalPlan, allReferences: AttributeSet) = if ((c.outputSet -- allReferences.filter(c.outputSet.contains)).nonEmpty) { Project(allReferences.filter(c.outputSet.contains).toSeq, c) } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala index 0988b0c6d990c..1e177e28f80b3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression} import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.{ArrayType, DataType, StructField, StructType} @@ -29,7 +29,7 @@ abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanTy /** * Returns the set of attributes that are output by this node. */ - def outputSet: Set[Attribute] = output.toSet + def outputSet: AttributeSet = AttributeSet(output) /** * Runs [[transform]] with `rule` on all expressions present in this query operator. 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 278569f0cb14a..8616ac45b0e95 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 @@ -45,17 +45,11 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] { sizeInBytes = children.map(_.statistics).map(_.sizeInBytes).product ) - /** - * Returns the set of attributes that are referenced by this node - * during evaluation. - */ - def references: Set[Attribute] - /** * Returns the set of attributes that this node takes as * input from its children. */ - lazy val inputSet: Set[Attribute] = children.flatMap(_.output).toSet + lazy val inputSet: AttributeSet = AttributeSet(children.flatMap(_.output)) /** * Returns true if this expression and all its children have been resolved to a specific schema @@ -126,9 +120,6 @@ abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { override lazy val statistics: Statistics = throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.") - - // Leaf nodes by definition cannot reference any input attributes. - override def references = Set.empty } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index d3f9d0fb93237..4460c86ed9026 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -30,6 +30,4 @@ case class ScriptTransformation( input: Seq[Expression], script: String, output: Seq[Attribute], - child: LogicalPlan) extends UnaryNode { - def references = input.flatMap(_.references).toSet -} + child: LogicalPlan) extends UnaryNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 3cb407217c4c3..4adfb189372d6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -23,7 +23,6 @@ import org.apache.spark.sql.catalyst.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) - def references = projectList.flatMap(_.references).toSet } /** @@ -59,14 +58,10 @@ case class Generate( override def output = if (join) child.output ++ generatorOutput else generatorOutput - - override def references = - if (join) child.outputSet else generator.references } case class Filter(condition: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = condition.references } case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { @@ -76,8 +71,6 @@ case class Union(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override lazy val resolved = childrenResolved && !left.output.zip(right.output).exists { case (l,r) => l.dataType != r.dataType } - - override def references = Set.empty } case class Join( @@ -86,8 +79,6 @@ case class Join( joinType: JoinType, condition: Option[Expression]) extends BinaryNode { - override def references = condition.map(_.references).getOrElse(Set.empty) - override def output = { joinType match { case LeftSemi => @@ -106,8 +97,6 @@ case class Join( case class Except(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { def output = left.output - - def references = Set.empty } case class InsertIntoTable( @@ -118,7 +107,6 @@ case class InsertIntoTable( extends LogicalPlan { // The table being inserted into is a child for the purposes of transformations. override def children = table :: child :: Nil - override def references = Set.empty override def output = child.output override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { @@ -130,20 +118,17 @@ case class InsertIntoCreatedTable( databaseName: Option[String], tableName: String, child: LogicalPlan) extends UnaryNode { - override def references = Set.empty override def output = child.output } case class WriteToFile( path: String, child: LogicalPlan) extends UnaryNode { - override def references = Set.empty override def output = child.output } case class Sort(order: Seq[SortOrder], child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = order.flatMap(_.references).toSet } case class Aggregate( @@ -152,19 +137,20 @@ case class Aggregate( child: LogicalPlan) extends UnaryNode { + /** The set of all AttributeReferences required for this aggregation. */ + def references = + AttributeSet( + groupingExpressions.flatMap(_.references) ++ aggregateExpressions.flatMap(_.references)) + override def output = aggregateExpressions.map(_.toAttribute) - override def references = - (groupingExpressions ++ aggregateExpressions).flatMap(_.references).toSet } case class Limit(limitExpr: Expression, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = limitExpr.references } case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { override def output = child.output.map(_.withQualifiers(alias :: Nil)) - override def references = Set.empty } /** @@ -191,20 +177,16 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { a.qualifiers) case other => other } - - override def references = Set.empty } case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = Set.empty } case class Distinct(child: LogicalPlan) extends UnaryNode { override def output = child.output - override def references = child.outputSet } case object NoRelation extends LeafNode { @@ -213,5 +195,4 @@ case object NoRelation extends LeafNode { case class Intersect(left: LogicalPlan, right: LogicalPlan) extends BinaryNode { override def output = left.output - override def references = Set.empty } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala index 7146fbd540f29..72b0c5c8e7a26 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala @@ -31,13 +31,9 @@ abstract class RedistributeData extends UnaryNode { case class SortPartitions(sortExpressions: Seq[SortOrder], child: LogicalPlan) extends RedistributeData { - - def references = sortExpressions.flatMap(_.references).toSet } case class Repartition(partitionExpressions: Seq[Expression], child: LogicalPlan) extends RedistributeData { - - def references = partitionExpressions.flatMap(_.references).toSet } 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 4bb022cf238af..ccb0df113c063 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 @@ -71,6 +71,7 @@ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { "An AllTuples should be used to represent a distribution that only has " + "a single partition.") + // TODO: This is not really valid... def clustering = ordering.map(_.child).toSet } @@ -139,7 +140,6 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) with Partitioning { override def children = expressions - override def references = expressions.flatMap(_.references).toSet override def nullable = false override def dataType = IntegerType @@ -179,7 +179,6 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) with Partitioning { override def children = ordering - override def references = ordering.flatMap(_.references).toSet override def nullable = false override def dataType = IntegerType 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 6344874538d67..296202543e2ca 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 @@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.types.{StringType, NullType} case class Dummy(optKey: Option[Expression]) extends Expression { def children = optKey.toSeq - def references = Set.empty[Attribute] def nullable = true def dataType = NullType override lazy val resolved = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 8a9f4deb6a19e..6f0eed3f63c41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -344,8 +344,8 @@ class SQLContext(@transient val sparkContext: SparkContext) prunePushedDownFilters: Seq[Expression] => Seq[Expression], scanBuilder: Seq[Attribute] => SparkPlan): SparkPlan = { - val projectSet = projectList.flatMap(_.references).toSet - val filterSet = filterPredicates.flatMap(_.references).toSet + val projectSet = AttributeSet(projectList.flatMap(_.references)) + val filterSet = AttributeSet(filterPredicates.flatMap(_.references)) val filterCondition = prunePushedDownFilters(filterPredicates).reduceLeftOption(And) // Right now we still use a projection even if the only evaluation is applying an alias @@ -354,7 +354,8 @@ class SQLContext(@transient val sparkContext: SparkContext) // TODO: Decouple final output schema from expression evaluation so this copy can be // avoided safely. - if (projectList.toSet == projectSet && filterSet.subsetOf(projectSet)) { + if (AttributeSet(projectList.map(_.toAttribute)) == projectSet && + filterSet.subsetOf(projectSet)) { // When it is possible to just use column pruning to get the right projection and // when the columns of this projection are enough to evaluate all filter conditions, // just do a scan followed by a filter, with no extra project. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index e63b4903041f6..24e88eea3189e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -79,8 +79,6 @@ private[sql] case class InMemoryRelation( override def children = Seq.empty - override def references = Set.empty - override def newInstance() = { new InMemoryRelation( output.map(_.newInstance), 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 21cbbc9772a00..7d33ea5b021e2 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 @@ -141,10 +141,9 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan)(@transient sqlContext: SQ extends LogicalPlan with MultiInstanceRelation { def output = alreadyPlanned.output - override def references = Set.empty override def children = Nil - override final def newInstance: this.type = { + override final def newInstance(): this.type = { SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index f31df051824d7..5b896c55b7393 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -58,8 +58,6 @@ package object debug { } private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { - def references = Set.empty - def output = child.output implicit object SetAccumulatorParam extends AccumulatorParam[HashSet[String]] { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala index b92091b560b1c..aef6ebf86b1eb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/pythonUdfs.scala @@ -49,7 +49,6 @@ private[spark] case class PythonUDF( override def toString = s"PythonUDF#$name(${children.mkString(",")})" def nullable: Boolean = true - def references: Set[Attribute] = children.flatMap(_.references).toSet override def eval(input: Row) = sys.error("PythonUDFs can not be directly evaluated.") } @@ -113,7 +112,6 @@ private[spark] object ExtractPythonUdfs extends Rule[LogicalPlan] { case class EvaluatePython(udf: PythonUDF, child: LogicalPlan) extends logical.UnaryNode { val resultAttribute = AttributeReference("pythonUDF", udf.dataType, nullable=true)() - def references = Set.empty def output = child.output :+ resultAttribute } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 389ace726d205..10fa8314c9156 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -79,9 +79,9 @@ private[hive] trait HiveStrategies { hiveContext.convertMetastoreParquet => // Filter out all predicates that only deal with partition keys - val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet + val partitionsKeys = AttributeSet(relation.partitionKeys) val (pruningPredicates, otherPredicates) = predicates.partition { - _.references.map(_.exprId).subsetOf(partitionKeyIds) + _.references.subsetOf(partitionsKeys) } // We are going to throw the predicates and projection back at the whole optimization @@ -176,9 +176,9 @@ private[hive] trait HiveStrategies { case PhysicalOperation(projectList, predicates, relation: MetastoreRelation) => // Filter out all predicates that only deal with partition keys, these are given to the // hive table scan operator to be used for partition pruning. - val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet + val partitionKeyIds = AttributeSet(relation.partitionKeys) val (pruningPredicates, otherPredicates) = predicates.partition { - _.references.map(_.exprId).subsetOf(partitionKeyIds) + _.references.subsetOf(partitionKeyIds) } pruneFilterProject( 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 c6497a15efa0c..7d1ad53d8bdb3 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 @@ -88,7 +88,6 @@ private[hive] abstract class HiveUdf extends Expression with Logging with HiveFu type EvaluatedType = Any def nullable = true - def references = children.flatMap(_.references).toSet lazy val function = createFunction[UDFType]() @@ -229,8 +228,6 @@ private[hive] case class HiveGenericUdaf( def nullable: Boolean = true - def references: Set[Attribute] = children.map(_.references).flatten.toSet - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" def newInstance() = new HiveUdafFunction(functionClassName, children, this) @@ -253,8 +250,6 @@ private[hive] case class HiveGenericUdtf( children: Seq[Expression]) extends Generator with HiveInspectors with HiveFunctionFactory { - override def references = children.flatMap(_.references).toSet - @transient protected lazy val function: GenericUDTF = createFunction() diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 6b3ffd1c0ffe2..b6be6bc1bfefe 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ -case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) case class Nested(a: Int, B: Int) +case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. @@ -57,6 +57,13 @@ class HiveResolutionSuite extends HiveComparisonTest { .registerTempTable("caseSensitivityTest") sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + + println(sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").queryExecution) + + sql("SELECT * FROM casesensitivitytest one JOIN casesensitivitytest two ON one.a = two.a").collect() + + // TODO: sql("SELECT * FROM casesensitivitytest a JOIN casesensitivitytest b ON a.a = b.a") + } test("nested repeated resolution") { From f1e71d4c3ba678fc108effb05cf2d6101dadc0ce Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Tue, 26 Aug 2014 16:57:40 -0700 Subject: [PATCH 08/51] [SPARK-3073] [PySpark] use external sort in sortBy() and sortByKey() Using external sort to support sort large datasets in reduce stage. Author: Davies Liu Closes #1978 from davies/sort and squashes the following commits: bbcd9ba [Davies Liu] check spilled bytes in tests b125d2f [Davies Liu] add test for external sort in rdd eae0176 [Davies Liu] choose different disks from different processes and instances 1f075ed [Davies Liu] Merge branch 'master' into sort eb53ca6 [Davies Liu] Merge branch 'master' into sort 644abaf [Davies Liu] add license in LICENSE 19f7873 [Davies Liu] improve tests 55602ee [Davies Liu] use external sort in sortBy() and sortByKey() --- .rat-excludes | 1 + LICENSE | 283 ++++++++++++ python/pyspark/heapq3.py | 890 ++++++++++++++++++++++++++++++++++++++ python/pyspark/rdd.py | 9 +- python/pyspark/shuffle.py | 91 +++- python/pyspark/tests.py | 42 +- tox.ini | 2 +- 7 files changed, 1306 insertions(+), 12 deletions(-) create mode 100644 python/pyspark/heapq3.py diff --git a/.rat-excludes b/.rat-excludes index eaefef1b0aa2e..fb6323daf9211 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -31,6 +31,7 @@ sorttable.js .*data .*log cloudpickle.py +heapq3.py join.py SparkExprTyper.scala SparkILoop.scala diff --git a/LICENSE b/LICENSE index e9a1153fdc5db..a7eee041129cb 100644 --- a/LICENSE +++ b/LICENSE @@ -338,6 +338,289 @@ THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. +======================================================================== +For heapq (pyspark/heapq3.py): +======================================================================== + +# A. HISTORY OF THE SOFTWARE +# ========================== +# +# Python was created in the early 1990s by Guido van Rossum at Stichting +# Mathematisch Centrum (CWI, see http://www.cwi.nl) in the Netherlands +# as a successor of a language called ABC. Guido remains Python's +# principal author, although it includes many contributions from others. +# +# In 1995, Guido continued his work on Python at the Corporation for +# National Research Initiatives (CNRI, see http://www.cnri.reston.va.us) +# in Reston, Virginia where he released several versions of the +# software. +# +# In May 2000, Guido and the Python core development team moved to +# BeOpen.com to form the BeOpen PythonLabs team. In October of the same +# year, the PythonLabs team moved to Digital Creations (now Zope +# Corporation, see http://www.zope.com). In 2001, the Python Software +# Foundation (PSF, see http://www.python.org/psf/) was formed, a +# non-profit organization created specifically to own Python-related +# Intellectual Property. Zope Corporation is a sponsoring member of +# the PSF. +# +# All Python releases are Open Source (see http://www.opensource.org for +# the Open Source Definition). Historically, most, but not all, Python +# releases have also been GPL-compatible; the table below summarizes +# the various releases. +# +# Release Derived Year Owner GPL- +# from compatible? (1) +# +# 0.9.0 thru 1.2 1991-1995 CWI yes +# 1.3 thru 1.5.2 1.2 1995-1999 CNRI yes +# 1.6 1.5.2 2000 CNRI no +# 2.0 1.6 2000 BeOpen.com no +# 1.6.1 1.6 2001 CNRI yes (2) +# 2.1 2.0+1.6.1 2001 PSF no +# 2.0.1 2.0+1.6.1 2001 PSF yes +# 2.1.1 2.1+2.0.1 2001 PSF yes +# 2.2 2.1.1 2001 PSF yes +# 2.1.2 2.1.1 2002 PSF yes +# 2.1.3 2.1.2 2002 PSF yes +# 2.2.1 2.2 2002 PSF yes +# 2.2.2 2.2.1 2002 PSF yes +# 2.2.3 2.2.2 2003 PSF yes +# 2.3 2.2.2 2002-2003 PSF yes +# 2.3.1 2.3 2002-2003 PSF yes +# 2.3.2 2.3.1 2002-2003 PSF yes +# 2.3.3 2.3.2 2002-2003 PSF yes +# 2.3.4 2.3.3 2004 PSF yes +# 2.3.5 2.3.4 2005 PSF yes +# 2.4 2.3 2004 PSF yes +# 2.4.1 2.4 2005 PSF yes +# 2.4.2 2.4.1 2005 PSF yes +# 2.4.3 2.4.2 2006 PSF yes +# 2.4.4 2.4.3 2006 PSF yes +# 2.5 2.4 2006 PSF yes +# 2.5.1 2.5 2007 PSF yes +# 2.5.2 2.5.1 2008 PSF yes +# 2.5.3 2.5.2 2008 PSF yes +# 2.6 2.5 2008 PSF yes +# 2.6.1 2.6 2008 PSF yes +# 2.6.2 2.6.1 2009 PSF yes +# 2.6.3 2.6.2 2009 PSF yes +# 2.6.4 2.6.3 2009 PSF yes +# 2.6.5 2.6.4 2010 PSF yes +# 2.7 2.6 2010 PSF yes +# +# Footnotes: +# +# (1) GPL-compatible doesn't mean that we're distributing Python under +# the GPL. All Python licenses, unlike the GPL, let you distribute +# a modified version without making your changes open source. The +# GPL-compatible licenses make it possible to combine Python with +# other software that is released under the GPL; the others don't. +# +# (2) According to Richard Stallman, 1.6.1 is not GPL-compatible, +# because its license has a choice of law clause. According to +# CNRI, however, Stallman's lawyer has told CNRI's lawyer that 1.6.1 +# is "not incompatible" with the GPL. +# +# Thanks to the many outside volunteers who have worked under Guido's +# direction to make these releases possible. +# +# +# B. TERMS AND CONDITIONS FOR ACCESSING OR OTHERWISE USING PYTHON +# =============================================================== +# +# PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 +# -------------------------------------------- +# +# 1. This LICENSE AGREEMENT is between the Python Software Foundation +# ("PSF"), and the Individual or Organization ("Licensee") accessing and +# otherwise using this software ("Python") in source or binary form and +# its associated documentation. +# +# 2. Subject to the terms and conditions of this License Agreement, PSF hereby +# grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, +# analyze, test, perform and/or display publicly, prepare derivative works, +# distribute, and otherwise use Python alone or in any derivative version, +# provided, however, that PSF's License Agreement and PSF's notice of copyright, +# i.e., "Copyright (c) 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, +# 2011, 2012, 2013 Python Software Foundation; All Rights Reserved" are retained +# in Python alone or in any derivative version prepared by Licensee. +# +# 3. In the event Licensee prepares a derivative work that is based on +# or incorporates Python or any part thereof, and wants to make +# the derivative work available to others as provided herein, then +# Licensee hereby agrees to include in any such work a brief summary of +# the changes made to Python. +# +# 4. PSF is making Python available to Licensee on an "AS IS" +# basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON +# FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS +# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON, +# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 6. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 7. Nothing in this License Agreement shall be deemed to create any +# relationship of agency, partnership, or joint venture between PSF and +# Licensee. This License Agreement does not grant permission to use PSF +# trademarks or trade name in a trademark sense to endorse or promote +# products or services of Licensee, or any third party. +# +# 8. By copying, installing or otherwise using Python, Licensee +# agrees to be bound by the terms and conditions of this License +# Agreement. +# +# +# BEOPEN.COM LICENSE AGREEMENT FOR PYTHON 2.0 +# ------------------------------------------- +# +# BEOPEN PYTHON OPEN SOURCE LICENSE AGREEMENT VERSION 1 +# +# 1. This LICENSE AGREEMENT is between BeOpen.com ("BeOpen"), having an +# office at 160 Saratoga Avenue, Santa Clara, CA 95051, and the +# Individual or Organization ("Licensee") accessing and otherwise using +# this software in source or binary form and its associated +# documentation ("the Software"). +# +# 2. Subject to the terms and conditions of this BeOpen Python License +# Agreement, BeOpen hereby grants Licensee a non-exclusive, +# royalty-free, world-wide license to reproduce, analyze, test, perform +# and/or display publicly, prepare derivative works, distribute, and +# otherwise use the Software alone or in any derivative version, +# provided, however, that the BeOpen Python License is retained in the +# Software, alone or in any derivative version prepared by Licensee. +# +# 3. BeOpen is making the Software available to Licensee on an "AS IS" +# basis. BEOPEN MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, BEOPEN MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF THE SOFTWARE WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 4. BEOPEN SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF THE +# SOFTWARE FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS +# AS A RESULT OF USING, MODIFYING OR DISTRIBUTING THE SOFTWARE, OR ANY +# DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 5. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 6. This License Agreement shall be governed by and interpreted in all +# respects by the law of the State of California, excluding conflict of +# law provisions. Nothing in this License Agreement shall be deemed to +# create any relationship of agency, partnership, or joint venture +# between BeOpen and Licensee. This License Agreement does not grant +# permission to use BeOpen trademarks or trade names in a trademark +# sense to endorse or promote products or services of Licensee, or any +# third party. As an exception, the "BeOpen Python" logos available at +# http://www.pythonlabs.com/logos.html may be used according to the +# permissions granted on that web page. +# +# 7. By copying, installing or otherwise using the software, Licensee +# agrees to be bound by the terms and conditions of this License +# Agreement. +# +# +# CNRI LICENSE AGREEMENT FOR PYTHON 1.6.1 +# --------------------------------------- +# +# 1. This LICENSE AGREEMENT is between the Corporation for National +# Research Initiatives, having an office at 1895 Preston White Drive, +# Reston, VA 20191 ("CNRI"), and the Individual or Organization +# ("Licensee") accessing and otherwise using Python 1.6.1 software in +# source or binary form and its associated documentation. +# +# 2. Subject to the terms and conditions of this License Agreement, CNRI +# hereby grants Licensee a nonexclusive, royalty-free, world-wide +# license to reproduce, analyze, test, perform and/or display publicly, +# prepare derivative works, distribute, and otherwise use Python 1.6.1 +# alone or in any derivative version, provided, however, that CNRI's +# License Agreement and CNRI's notice of copyright, i.e., "Copyright (c) +# 1995-2001 Corporation for National Research Initiatives; All Rights +# Reserved" are retained in Python 1.6.1 alone or in any derivative +# version prepared by Licensee. Alternately, in lieu of CNRI's License +# Agreement, Licensee may substitute the following text (omitting the +# quotes): "Python 1.6.1 is made available subject to the terms and +# conditions in CNRI's License Agreement. This Agreement together with +# Python 1.6.1 may be located on the Internet using the following +# unique, persistent identifier (known as a handle): 1895.22/1013. This +# Agreement may also be obtained from a proxy server on the Internet +# using the following URL: http://hdl.handle.net/1895.22/1013". +# +# 3. In the event Licensee prepares a derivative work that is based on +# or incorporates Python 1.6.1 or any part thereof, and wants to make +# the derivative work available to others as provided herein, then +# Licensee hereby agrees to include in any such work a brief summary of +# the changes made to Python 1.6.1. +# +# 4. CNRI is making Python 1.6.1 available to Licensee on an "AS IS" +# basis. CNRI MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, CNRI MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON 1.6.1 WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 5. CNRI SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON +# 1.6.1 FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS +# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON 1.6.1, +# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 6. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 7. This License Agreement shall be governed by the federal +# intellectual property law of the United States, including without +# limitation the federal copyright law, and, to the extent such +# U.S. federal law does not apply, by the law of the Commonwealth of +# Virginia, excluding Virginia's conflict of law provisions. +# Notwithstanding the foregoing, with regard to derivative works based +# on Python 1.6.1 that incorporate non-separable material that was +# previously distributed under the GNU General Public License (GPL), the +# law of the Commonwealth of Virginia shall govern this License +# Agreement only as to issues arising under or with respect to +# Paragraphs 4, 5, and 7 of this License Agreement. Nothing in this +# License Agreement shall be deemed to create any relationship of +# agency, partnership, or joint venture between CNRI and Licensee. This +# License Agreement does not grant permission to use CNRI trademarks or +# trade name in a trademark sense to endorse or promote products or +# services of Licensee, or any third party. +# +# 8. By clicking on the "ACCEPT" button where indicated, or by copying, +# installing or otherwise using Python 1.6.1, Licensee agrees to be +# bound by the terms and conditions of this License Agreement. +# +# ACCEPT +# +# +# CWI LICENSE AGREEMENT FOR PYTHON 0.9.0 THROUGH 1.2 +# -------------------------------------------------- +# +# Copyright (c) 1991 - 1995, Stichting Mathematisch Centrum Amsterdam, +# The Netherlands. All rights reserved. +# +# Permission to use, copy, modify, and distribute this software and its +# documentation for any purpose and without fee is hereby granted, +# provided that the above copyright notice appear in all copies and that +# both that copyright notice and this permission notice appear in +# supporting documentation, and that the name of Stichting Mathematisch +# Centrum or CWI not be used in advertising or publicity pertaining to +# distribution of the software without specific, written prior +# permission. +# +# STICHTING MATHEMATISCH CENTRUM DISCLAIMS ALL WARRANTIES WITH REGARD TO +# THIS SOFTWARE, INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND +# FITNESS, IN NO EVENT SHALL STICHTING MATHEMATISCH CENTRUM BE LIABLE +# FOR ANY SPECIAL, INDIRECT OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES +# WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN +# ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT +# OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. ======================================================================== For sorttable (core/src/main/resources/org/apache/spark/ui/static/sorttable.js): diff --git a/python/pyspark/heapq3.py b/python/pyspark/heapq3.py new file mode 100644 index 0000000000000..bc441f138f7fc --- /dev/null +++ b/python/pyspark/heapq3.py @@ -0,0 +1,890 @@ +# -*- encoding: utf-8 -*- +# back ported from CPython 3 +# A. HISTORY OF THE SOFTWARE +# ========================== +# +# Python was created in the early 1990s by Guido van Rossum at Stichting +# Mathematisch Centrum (CWI, see http://www.cwi.nl) in the Netherlands +# as a successor of a language called ABC. Guido remains Python's +# principal author, although it includes many contributions from others. +# +# In 1995, Guido continued his work on Python at the Corporation for +# National Research Initiatives (CNRI, see http://www.cnri.reston.va.us) +# in Reston, Virginia where he released several versions of the +# software. +# +# In May 2000, Guido and the Python core development team moved to +# BeOpen.com to form the BeOpen PythonLabs team. In October of the same +# year, the PythonLabs team moved to Digital Creations (now Zope +# Corporation, see http://www.zope.com). In 2001, the Python Software +# Foundation (PSF, see http://www.python.org/psf/) was formed, a +# non-profit organization created specifically to own Python-related +# Intellectual Property. Zope Corporation is a sponsoring member of +# the PSF. +# +# All Python releases are Open Source (see http://www.opensource.org for +# the Open Source Definition). Historically, most, but not all, Python +# releases have also been GPL-compatible; the table below summarizes +# the various releases. +# +# Release Derived Year Owner GPL- +# from compatible? (1) +# +# 0.9.0 thru 1.2 1991-1995 CWI yes +# 1.3 thru 1.5.2 1.2 1995-1999 CNRI yes +# 1.6 1.5.2 2000 CNRI no +# 2.0 1.6 2000 BeOpen.com no +# 1.6.1 1.6 2001 CNRI yes (2) +# 2.1 2.0+1.6.1 2001 PSF no +# 2.0.1 2.0+1.6.1 2001 PSF yes +# 2.1.1 2.1+2.0.1 2001 PSF yes +# 2.2 2.1.1 2001 PSF yes +# 2.1.2 2.1.1 2002 PSF yes +# 2.1.3 2.1.2 2002 PSF yes +# 2.2.1 2.2 2002 PSF yes +# 2.2.2 2.2.1 2002 PSF yes +# 2.2.3 2.2.2 2003 PSF yes +# 2.3 2.2.2 2002-2003 PSF yes +# 2.3.1 2.3 2002-2003 PSF yes +# 2.3.2 2.3.1 2002-2003 PSF yes +# 2.3.3 2.3.2 2002-2003 PSF yes +# 2.3.4 2.3.3 2004 PSF yes +# 2.3.5 2.3.4 2005 PSF yes +# 2.4 2.3 2004 PSF yes +# 2.4.1 2.4 2005 PSF yes +# 2.4.2 2.4.1 2005 PSF yes +# 2.4.3 2.4.2 2006 PSF yes +# 2.4.4 2.4.3 2006 PSF yes +# 2.5 2.4 2006 PSF yes +# 2.5.1 2.5 2007 PSF yes +# 2.5.2 2.5.1 2008 PSF yes +# 2.5.3 2.5.2 2008 PSF yes +# 2.6 2.5 2008 PSF yes +# 2.6.1 2.6 2008 PSF yes +# 2.6.2 2.6.1 2009 PSF yes +# 2.6.3 2.6.2 2009 PSF yes +# 2.6.4 2.6.3 2009 PSF yes +# 2.6.5 2.6.4 2010 PSF yes +# 2.7 2.6 2010 PSF yes +# +# Footnotes: +# +# (1) GPL-compatible doesn't mean that we're distributing Python under +# the GPL. All Python licenses, unlike the GPL, let you distribute +# a modified version without making your changes open source. The +# GPL-compatible licenses make it possible to combine Python with +# other software that is released under the GPL; the others don't. +# +# (2) According to Richard Stallman, 1.6.1 is not GPL-compatible, +# because its license has a choice of law clause. According to +# CNRI, however, Stallman's lawyer has told CNRI's lawyer that 1.6.1 +# is "not incompatible" with the GPL. +# +# Thanks to the many outside volunteers who have worked under Guido's +# direction to make these releases possible. +# +# +# B. TERMS AND CONDITIONS FOR ACCESSING OR OTHERWISE USING PYTHON +# =============================================================== +# +# PYTHON SOFTWARE FOUNDATION LICENSE VERSION 2 +# -------------------------------------------- +# +# 1. This LICENSE AGREEMENT is between the Python Software Foundation +# ("PSF"), and the Individual or Organization ("Licensee") accessing and +# otherwise using this software ("Python") in source or binary form and +# its associated documentation. +# +# 2. Subject to the terms and conditions of this License Agreement, PSF hereby +# grants Licensee a nonexclusive, royalty-free, world-wide license to reproduce, +# analyze, test, perform and/or display publicly, prepare derivative works, +# distribute, and otherwise use Python alone or in any derivative version, +# provided, however, that PSF's License Agreement and PSF's notice of copyright, +# i.e., "Copyright (c) 2001, 2002, 2003, 2004, 2005, 2006, 2007, 2008, 2009, 2010, +# 2011, 2012, 2013 Python Software Foundation; All Rights Reserved" are retained +# in Python alone or in any derivative version prepared by Licensee. +# +# 3. In the event Licensee prepares a derivative work that is based on +# or incorporates Python or any part thereof, and wants to make +# the derivative work available to others as provided herein, then +# Licensee hereby agrees to include in any such work a brief summary of +# the changes made to Python. +# +# 4. PSF is making Python available to Licensee on an "AS IS" +# basis. PSF MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, PSF MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 5. PSF SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON +# FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS +# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON, +# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 6. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 7. Nothing in this License Agreement shall be deemed to create any +# relationship of agency, partnership, or joint venture between PSF and +# Licensee. This License Agreement does not grant permission to use PSF +# trademarks or trade name in a trademark sense to endorse or promote +# products or services of Licensee, or any third party. +# +# 8. By copying, installing or otherwise using Python, Licensee +# agrees to be bound by the terms and conditions of this License +# Agreement. +# +# +# BEOPEN.COM LICENSE AGREEMENT FOR PYTHON 2.0 +# ------------------------------------------- +# +# BEOPEN PYTHON OPEN SOURCE LICENSE AGREEMENT VERSION 1 +# +# 1. This LICENSE AGREEMENT is between BeOpen.com ("BeOpen"), having an +# office at 160 Saratoga Avenue, Santa Clara, CA 95051, and the +# Individual or Organization ("Licensee") accessing and otherwise using +# this software in source or binary form and its associated +# documentation ("the Software"). +# +# 2. Subject to the terms and conditions of this BeOpen Python License +# Agreement, BeOpen hereby grants Licensee a non-exclusive, +# royalty-free, world-wide license to reproduce, analyze, test, perform +# and/or display publicly, prepare derivative works, distribute, and +# otherwise use the Software alone or in any derivative version, +# provided, however, that the BeOpen Python License is retained in the +# Software, alone or in any derivative version prepared by Licensee. +# +# 3. BeOpen is making the Software available to Licensee on an "AS IS" +# basis. BEOPEN MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, BEOPEN MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF THE SOFTWARE WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 4. BEOPEN SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF THE +# SOFTWARE FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS +# AS A RESULT OF USING, MODIFYING OR DISTRIBUTING THE SOFTWARE, OR ANY +# DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 5. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 6. This License Agreement shall be governed by and interpreted in all +# respects by the law of the State of California, excluding conflict of +# law provisions. Nothing in this License Agreement shall be deemed to +# create any relationship of agency, partnership, or joint venture +# between BeOpen and Licensee. This License Agreement does not grant +# permission to use BeOpen trademarks or trade names in a trademark +# sense to endorse or promote products or services of Licensee, or any +# third party. As an exception, the "BeOpen Python" logos available at +# http://www.pythonlabs.com/logos.html may be used according to the +# permissions granted on that web page. +# +# 7. By copying, installing or otherwise using the software, Licensee +# agrees to be bound by the terms and conditions of this License +# Agreement. +# +# +# CNRI LICENSE AGREEMENT FOR PYTHON 1.6.1 +# --------------------------------------- +# +# 1. This LICENSE AGREEMENT is between the Corporation for National +# Research Initiatives, having an office at 1895 Preston White Drive, +# Reston, VA 20191 ("CNRI"), and the Individual or Organization +# ("Licensee") accessing and otherwise using Python 1.6.1 software in +# source or binary form and its associated documentation. +# +# 2. Subject to the terms and conditions of this License Agreement, CNRI +# hereby grants Licensee a nonexclusive, royalty-free, world-wide +# license to reproduce, analyze, test, perform and/or display publicly, +# prepare derivative works, distribute, and otherwise use Python 1.6.1 +# alone or in any derivative version, provided, however, that CNRI's +# License Agreement and CNRI's notice of copyright, i.e., "Copyright (c) +# 1995-2001 Corporation for National Research Initiatives; All Rights +# Reserved" are retained in Python 1.6.1 alone or in any derivative +# version prepared by Licensee. Alternately, in lieu of CNRI's License +# Agreement, Licensee may substitute the following text (omitting the +# quotes): "Python 1.6.1 is made available subject to the terms and +# conditions in CNRI's License Agreement. This Agreement together with +# Python 1.6.1 may be located on the Internet using the following +# unique, persistent identifier (known as a handle): 1895.22/1013. This +# Agreement may also be obtained from a proxy server on the Internet +# using the following URL: http://hdl.handle.net/1895.22/1013". +# +# 3. In the event Licensee prepares a derivative work that is based on +# or incorporates Python 1.6.1 or any part thereof, and wants to make +# the derivative work available to others as provided herein, then +# Licensee hereby agrees to include in any such work a brief summary of +# the changes made to Python 1.6.1. +# +# 4. CNRI is making Python 1.6.1 available to Licensee on an "AS IS" +# basis. CNRI MAKES NO REPRESENTATIONS OR WARRANTIES, EXPRESS OR +# IMPLIED. BY WAY OF EXAMPLE, BUT NOT LIMITATION, CNRI MAKES NO AND +# DISCLAIMS ANY REPRESENTATION OR WARRANTY OF MERCHANTABILITY OR FITNESS +# FOR ANY PARTICULAR PURPOSE OR THAT THE USE OF PYTHON 1.6.1 WILL NOT +# INFRINGE ANY THIRD PARTY RIGHTS. +# +# 5. CNRI SHALL NOT BE LIABLE TO LICENSEE OR ANY OTHER USERS OF PYTHON +# 1.6.1 FOR ANY INCIDENTAL, SPECIAL, OR CONSEQUENTIAL DAMAGES OR LOSS AS +# A RESULT OF MODIFYING, DISTRIBUTING, OR OTHERWISE USING PYTHON 1.6.1, +# OR ANY DERIVATIVE THEREOF, EVEN IF ADVISED OF THE POSSIBILITY THEREOF. +# +# 6. This License Agreement will automatically terminate upon a material +# breach of its terms and conditions. +# +# 7. This License Agreement shall be governed by the federal +# intellectual property law of the United States, including without +# limitation the federal copyright law, and, to the extent such +# U.S. federal law does not apply, by the law of the Commonwealth of +# Virginia, excluding Virginia's conflict of law provisions. +# Notwithstanding the foregoing, with regard to derivative works based +# on Python 1.6.1 that incorporate non-separable material that was +# previously distributed under the GNU General Public License (GPL), the +# law of the Commonwealth of Virginia shall govern this License +# Agreement only as to issues arising under or with respect to +# Paragraphs 4, 5, and 7 of this License Agreement. Nothing in this +# License Agreement shall be deemed to create any relationship of +# agency, partnership, or joint venture between CNRI and Licensee. This +# License Agreement does not grant permission to use CNRI trademarks or +# trade name in a trademark sense to endorse or promote products or +# services of Licensee, or any third party. +# +# 8. By clicking on the "ACCEPT" button where indicated, or by copying, +# installing or otherwise using Python 1.6.1, Licensee agrees to be +# bound by the terms and conditions of this License Agreement. +# +# ACCEPT +# +# +# CWI LICENSE AGREEMENT FOR PYTHON 0.9.0 THROUGH 1.2 +# -------------------------------------------------- +# +# Copyright (c) 1991 - 1995, Stichting Mathematisch Centrum Amsterdam, +# The Netherlands. All rights reserved. +# +# Permission to use, copy, modify, and distribute this software and its +# documentation for any purpose and without fee is hereby granted, +# provided that the above copyright notice appear in all copies and that +# both that copyright notice and this permission notice appear in +# supporting documentation, and that the name of Stichting Mathematisch +# Centrum or CWI not be used in advertising or publicity pertaining to +# distribution of the software without specific, written prior +# permission. +# +# STICHTING MATHEMATISCH CENTRUM DISCLAIMS ALL WARRANTIES WITH REGARD TO +# THIS SOFTWARE, INCLUDING ALL IMPLIED WARRANTIES OF MERCHANTABILITY AND +# FITNESS, IN NO EVENT SHALL STICHTING MATHEMATISCH CENTRUM BE LIABLE +# FOR ANY SPECIAL, INDIRECT OR CONSEQUENTIAL DAMAGES OR ANY DAMAGES +# WHATSOEVER RESULTING FROM LOSS OF USE, DATA OR PROFITS, WHETHER IN AN +# ACTION OF CONTRACT, NEGLIGENCE OR OTHER TORTIOUS ACTION, ARISING OUT +# OF OR IN CONNECTION WITH THE USE OR PERFORMANCE OF THIS SOFTWARE. +"""Heap queue algorithm (a.k.a. priority queue). + +Heaps are arrays for which a[k] <= a[2*k+1] and a[k] <= a[2*k+2] for +all k, counting elements from 0. For the sake of comparison, +non-existing elements are considered to be infinite. The interesting +property of a heap is that a[0] is always its smallest element. + +Usage: + +heap = [] # creates an empty heap +heappush(heap, item) # pushes a new item on the heap +item = heappop(heap) # pops the smallest item from the heap +item = heap[0] # smallest item on the heap without popping it +heapify(x) # transforms list into a heap, in-place, in linear time +item = heapreplace(heap, item) # pops and returns smallest item, and adds + # new item; the heap size is unchanged + +Our API differs from textbook heap algorithms as follows: + +- We use 0-based indexing. This makes the relationship between the + index for a node and the indexes for its children slightly less + obvious, but is more suitable since Python uses 0-based indexing. + +- Our heappop() method returns the smallest item, not the largest. + +These two make it possible to view the heap as a regular Python list +without surprises: heap[0] is the smallest item, and heap.sort() +maintains the heap invariant! +""" + +# Original code by Kevin O'Connor, augmented by Tim Peters and Raymond Hettinger + +__about__ = """Heap queues + +[explanation by François Pinard] + +Heaps are arrays for which a[k] <= a[2*k+1] and a[k] <= a[2*k+2] for +all k, counting elements from 0. For the sake of comparison, +non-existing elements are considered to be infinite. The interesting +property of a heap is that a[0] is always its smallest element. + +The strange invariant above is meant to be an efficient memory +representation for a tournament. The numbers below are `k', not a[k]: + + 0 + + 1 2 + + 3 4 5 6 + + 7 8 9 10 11 12 13 14 + + 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 + + +In the tree above, each cell `k' is topping `2*k+1' and `2*k+2'. In +an usual binary tournament we see in sports, each cell is the winner +over the two cells it tops, and we can trace the winner down the tree +to see all opponents s/he had. However, in many computer applications +of such tournaments, we do not need to trace the history of a winner. +To be more memory efficient, when a winner is promoted, we try to +replace it by something else at a lower level, and the rule becomes +that a cell and the two cells it tops contain three different items, +but the top cell "wins" over the two topped cells. + +If this heap invariant is protected at all time, index 0 is clearly +the overall winner. The simplest algorithmic way to remove it and +find the "next" winner is to move some loser (let's say cell 30 in the +diagram above) into the 0 position, and then percolate this new 0 down +the tree, exchanging values, until the invariant is re-established. +This is clearly logarithmic on the total number of items in the tree. +By iterating over all items, you get an O(n ln n) sort. + +A nice feature of this sort is that you can efficiently insert new +items while the sort is going on, provided that the inserted items are +not "better" than the last 0'th element you extracted. This is +especially useful in simulation contexts, where the tree holds all +incoming events, and the "win" condition means the smallest scheduled +time. When an event schedule other events for execution, they are +scheduled into the future, so they can easily go into the heap. So, a +heap is a good structure for implementing schedulers (this is what I +used for my MIDI sequencer :-). + +Various structures for implementing schedulers have been extensively +studied, and heaps are good for this, as they are reasonably speedy, +the speed is almost constant, and the worst case is not much different +than the average case. However, there are other representations which +are more efficient overall, yet the worst cases might be terrible. + +Heaps are also very useful in big disk sorts. You most probably all +know that a big sort implies producing "runs" (which are pre-sorted +sequences, which size is usually related to the amount of CPU memory), +followed by a merging passes for these runs, which merging is often +very cleverly organised[1]. It is very important that the initial +sort produces the longest runs possible. Tournaments are a good way +to that. If, using all the memory available to hold a tournament, you +replace and percolate items that happen to fit the current run, you'll +produce runs which are twice the size of the memory for random input, +and much better for input fuzzily ordered. + +Moreover, if you output the 0'th item on disk and get an input which +may not fit in the current tournament (because the value "wins" over +the last output value), it cannot fit in the heap, so the size of the +heap decreases. The freed memory could be cleverly reused immediately +for progressively building a second heap, which grows at exactly the +same rate the first heap is melting. When the first heap completely +vanishes, you switch heaps and start a new run. Clever and quite +effective! + +In a word, heaps are useful memory structures to know. I use them in +a few applications, and I think it is good to keep a `heap' module +around. :-) + +-------------------- +[1] The disk balancing algorithms which are current, nowadays, are +more annoying than clever, and this is a consequence of the seeking +capabilities of the disks. On devices which cannot seek, like big +tape drives, the story was quite different, and one had to be very +clever to ensure (far in advance) that each tape movement will be the +most effective possible (that is, will best participate at +"progressing" the merge). Some tapes were even able to read +backwards, and this was also used to avoid the rewinding time. +Believe me, real good tape sorts were quite spectacular to watch! +From all times, sorting has always been a Great Art! :-) +""" + +__all__ = ['heappush', 'heappop', 'heapify', 'heapreplace', 'merge', + 'nlargest', 'nsmallest', 'heappushpop'] + +def heappush(heap, item): + """Push item onto heap, maintaining the heap invariant.""" + heap.append(item) + _siftdown(heap, 0, len(heap)-1) + +def heappop(heap): + """Pop the smallest item off the heap, maintaining the heap invariant.""" + lastelt = heap.pop() # raises appropriate IndexError if heap is empty + if heap: + returnitem = heap[0] + heap[0] = lastelt + _siftup(heap, 0) + return returnitem + return lastelt + +def heapreplace(heap, item): + """Pop and return the current smallest value, and add the new item. + + This is more efficient than heappop() followed by heappush(), and can be + more appropriate when using a fixed-size heap. Note that the value + returned may be larger than item! That constrains reasonable uses of + this routine unless written as part of a conditional replacement: + + if item > heap[0]: + item = heapreplace(heap, item) + """ + returnitem = heap[0] # raises appropriate IndexError if heap is empty + heap[0] = item + _siftup(heap, 0) + return returnitem + +def heappushpop(heap, item): + """Fast version of a heappush followed by a heappop.""" + if heap and heap[0] < item: + item, heap[0] = heap[0], item + _siftup(heap, 0) + return item + +def heapify(x): + """Transform list into a heap, in-place, in O(len(x)) time.""" + n = len(x) + # Transform bottom-up. The largest index there's any point to looking at + # is the largest with a child index in-range, so must have 2*i + 1 < n, + # or i < (n-1)/2. If n is even = 2*j, this is (2*j-1)/2 = j-1/2 so + # j-1 is the largest, which is n//2 - 1. If n is odd = 2*j+1, this is + # (2*j+1-1)/2 = j so j-1 is the largest, and that's again n//2-1. + for i in reversed(range(n//2)): + _siftup(x, i) + +def _heappop_max(heap): + """Maxheap version of a heappop.""" + lastelt = heap.pop() # raises appropriate IndexError if heap is empty + if heap: + returnitem = heap[0] + heap[0] = lastelt + _siftup_max(heap, 0) + return returnitem + return lastelt + +def _heapreplace_max(heap, item): + """Maxheap version of a heappop followed by a heappush.""" + returnitem = heap[0] # raises appropriate IndexError if heap is empty + heap[0] = item + _siftup_max(heap, 0) + return returnitem + +def _heapify_max(x): + """Transform list into a maxheap, in-place, in O(len(x)) time.""" + n = len(x) + for i in reversed(range(n//2)): + _siftup_max(x, i) + +# 'heap' is a heap at all indices >= startpos, except possibly for pos. pos +# is the index of a leaf with a possibly out-of-order value. Restore the +# heap invariant. +def _siftdown(heap, startpos, pos): + newitem = heap[pos] + # Follow the path to the root, moving parents down until finding a place + # newitem fits. + while pos > startpos: + parentpos = (pos - 1) >> 1 + parent = heap[parentpos] + if newitem < parent: + heap[pos] = parent + pos = parentpos + continue + break + heap[pos] = newitem + +# The child indices of heap index pos are already heaps, and we want to make +# a heap at index pos too. We do this by bubbling the smaller child of +# pos up (and so on with that child's children, etc) until hitting a leaf, +# then using _siftdown to move the oddball originally at index pos into place. +# +# We *could* break out of the loop as soon as we find a pos where newitem <= +# both its children, but turns out that's not a good idea, and despite that +# many books write the algorithm that way. During a heap pop, the last array +# element is sifted in, and that tends to be large, so that comparing it +# against values starting from the root usually doesn't pay (= usually doesn't +# get us out of the loop early). See Knuth, Volume 3, where this is +# explained and quantified in an exercise. +# +# Cutting the # of comparisons is important, since these routines have no +# way to extract "the priority" from an array element, so that intelligence +# is likely to be hiding in custom comparison methods, or in array elements +# storing (priority, record) tuples. Comparisons are thus potentially +# expensive. +# +# On random arrays of length 1000, making this change cut the number of +# comparisons made by heapify() a little, and those made by exhaustive +# heappop() a lot, in accord with theory. Here are typical results from 3 +# runs (3 just to demonstrate how small the variance is): +# +# Compares needed by heapify Compares needed by 1000 heappops +# -------------------------- -------------------------------- +# 1837 cut to 1663 14996 cut to 8680 +# 1855 cut to 1659 14966 cut to 8678 +# 1847 cut to 1660 15024 cut to 8703 +# +# Building the heap by using heappush() 1000 times instead required +# 2198, 2148, and 2219 compares: heapify() is more efficient, when +# you can use it. +# +# The total compares needed by list.sort() on the same lists were 8627, +# 8627, and 8632 (this should be compared to the sum of heapify() and +# heappop() compares): list.sort() is (unsurprisingly!) more efficient +# for sorting. + +def _siftup(heap, pos): + endpos = len(heap) + startpos = pos + newitem = heap[pos] + # Bubble up the smaller child until hitting a leaf. + childpos = 2*pos + 1 # leftmost child position + while childpos < endpos: + # Set childpos to index of smaller child. + rightpos = childpos + 1 + if rightpos < endpos and not heap[childpos] < heap[rightpos]: + childpos = rightpos + # Move the smaller child up. + heap[pos] = heap[childpos] + pos = childpos + childpos = 2*pos + 1 + # The leaf at pos is empty now. Put newitem there, and bubble it up + # to its final resting place (by sifting its parents down). + heap[pos] = newitem + _siftdown(heap, startpos, pos) + +def _siftdown_max(heap, startpos, pos): + 'Maxheap variant of _siftdown' + newitem = heap[pos] + # Follow the path to the root, moving parents down until finding a place + # newitem fits. + while pos > startpos: + parentpos = (pos - 1) >> 1 + parent = heap[parentpos] + if parent < newitem: + heap[pos] = parent + pos = parentpos + continue + break + heap[pos] = newitem + +def _siftup_max(heap, pos): + 'Maxheap variant of _siftup' + endpos = len(heap) + startpos = pos + newitem = heap[pos] + # Bubble up the larger child until hitting a leaf. + childpos = 2*pos + 1 # leftmost child position + while childpos < endpos: + # Set childpos to index of larger child. + rightpos = childpos + 1 + if rightpos < endpos and not heap[rightpos] < heap[childpos]: + childpos = rightpos + # Move the larger child up. + heap[pos] = heap[childpos] + pos = childpos + childpos = 2*pos + 1 + # The leaf at pos is empty now. Put newitem there, and bubble it up + # to its final resting place (by sifting its parents down). + heap[pos] = newitem + _siftdown_max(heap, startpos, pos) + +def merge(iterables, key=None, reverse=False): + '''Merge multiple sorted inputs into a single sorted output. + + Similar to sorted(itertools.chain(*iterables)) but returns a generator, + does not pull the data into memory all at once, and assumes that each of + the input streams is already sorted (smallest to largest). + + >>> list(merge([1,3,5,7], [0,2,4,8], [5,10,15,20], [], [25])) + [0, 1, 2, 3, 4, 5, 5, 7, 8, 10, 15, 20, 25] + + If *key* is not None, applies a key function to each element to determine + its sort order. + + >>> list(merge(['dog', 'horse'], ['cat', 'fish', 'kangaroo'], key=len)) + ['dog', 'cat', 'fish', 'horse', 'kangaroo'] + + ''' + + h = [] + h_append = h.append + + if reverse: + _heapify = _heapify_max + _heappop = _heappop_max + _heapreplace = _heapreplace_max + direction = -1 + else: + _heapify = heapify + _heappop = heappop + _heapreplace = heapreplace + direction = 1 + + if key is None: + for order, it in enumerate(map(iter, iterables)): + try: + next = it.next + h_append([next(), order * direction, next]) + except StopIteration: + pass + _heapify(h) + while len(h) > 1: + try: + while True: + value, order, next = s = h[0] + yield value + s[0] = next() # raises StopIteration when exhausted + _heapreplace(h, s) # restore heap condition + except StopIteration: + _heappop(h) # remove empty iterator + if h: + # fast case when only a single iterator remains + value, order, next = h[0] + yield value + for value in next.__self__: + yield value + return + + for order, it in enumerate(map(iter, iterables)): + try: + next = it.next + value = next() + h_append([key(value), order * direction, value, next]) + except StopIteration: + pass + _heapify(h) + while len(h) > 1: + try: + while True: + key_value, order, value, next = s = h[0] + yield value + value = next() + s[0] = key(value) + s[2] = value + _heapreplace(h, s) + except StopIteration: + _heappop(h) + if h: + key_value, order, value, next = h[0] + yield value + for value in next.__self__: + yield value + + +# Algorithm notes for nlargest() and nsmallest() +# ============================================== +# +# Make a single pass over the data while keeping the k most extreme values +# in a heap. Memory consumption is limited to keeping k values in a list. +# +# Measured performance for random inputs: +# +# number of comparisons +# n inputs k-extreme values (average of 5 trials) % more than min() +# ------------- ---------------- --------------------- ----------------- +# 1,000 100 3,317 231.7% +# 10,000 100 14,046 40.5% +# 100,000 100 105,749 5.7% +# 1,000,000 100 1,007,751 0.8% +# 10,000,000 100 10,009,401 0.1% +# +# Theoretical number of comparisons for k smallest of n random inputs: +# +# Step Comparisons Action +# ---- -------------------------- --------------------------- +# 1 1.66 * k heapify the first k-inputs +# 2 n - k compare remaining elements to top of heap +# 3 k * (1 + lg2(k)) * ln(n/k) replace the topmost value on the heap +# 4 k * lg2(k) - (k/2) final sort of the k most extreme values +# +# Combining and simplifying for a rough estimate gives: +# +# comparisons = n + k * (log(k, 2) * log(n/k) + log(k, 2) + log(n/k)) +# +# Computing the number of comparisons for step 3: +# ----------------------------------------------- +# * For the i-th new value from the iterable, the probability of being in the +# k most extreme values is k/i. For example, the probability of the 101st +# value seen being in the 100 most extreme values is 100/101. +# * If the value is a new extreme value, the cost of inserting it into the +# heap is 1 + log(k, 2). +# * The probabilty times the cost gives: +# (k/i) * (1 + log(k, 2)) +# * Summing across the remaining n-k elements gives: +# sum((k/i) * (1 + log(k, 2)) for i in range(k+1, n+1)) +# * This reduces to: +# (H(n) - H(k)) * k * (1 + log(k, 2)) +# * Where H(n) is the n-th harmonic number estimated by: +# gamma = 0.5772156649 +# H(n) = log(n, e) + gamma + 1 / (2 * n) +# http://en.wikipedia.org/wiki/Harmonic_series_(mathematics)#Rate_of_divergence +# * Substituting the H(n) formula: +# comparisons = k * (1 + log(k, 2)) * (log(n/k, e) + (1/n - 1/k) / 2) +# +# Worst-case for step 3: +# ---------------------- +# In the worst case, the input data is reversed sorted so that every new element +# must be inserted in the heap: +# +# comparisons = 1.66 * k + log(k, 2) * (n - k) +# +# Alternative Algorithms +# ---------------------- +# Other algorithms were not used because they: +# 1) Took much more auxiliary memory, +# 2) Made multiple passes over the data. +# 3) Made more comparisons in common cases (small k, large n, semi-random input). +# See the more detailed comparison of approach at: +# http://code.activestate.com/recipes/577573-compare-algorithms-for-heapqsmallest + +def nsmallest(n, iterable, key=None): + """Find the n smallest elements in a dataset. + + Equivalent to: sorted(iterable, key=key)[:n] + """ + + # Short-cut for n==1 is to use min() + if n == 1: + it = iter(iterable) + sentinel = object() + if key is None: + result = min(it, default=sentinel) + else: + result = min(it, default=sentinel, key=key) + return [] if result is sentinel else [result] + + # When n>=size, it's faster to use sorted() + try: + size = len(iterable) + except (TypeError, AttributeError): + pass + else: + if n >= size: + return sorted(iterable, key=key)[:n] + + # When key is none, use simpler decoration + if key is None: + it = iter(iterable) + # put the range(n) first so that zip() doesn't + # consume one too many elements from the iterator + result = [(elem, i) for i, elem in zip(range(n), it)] + if not result: + return result + _heapify_max(result) + top = result[0][0] + order = n + _heapreplace = _heapreplace_max + for elem in it: + if elem < top: + _heapreplace(result, (elem, order)) + top = result[0][0] + order += 1 + result.sort() + return [r[0] for r in result] + + # General case, slowest method + it = iter(iterable) + result = [(key(elem), i, elem) for i, elem in zip(range(n), it)] + if not result: + return result + _heapify_max(result) + top = result[0][0] + order = n + _heapreplace = _heapreplace_max + for elem in it: + k = key(elem) + if k < top: + _heapreplace(result, (k, order, elem)) + top = result[0][0] + order += 1 + result.sort() + return [r[2] for r in result] + +def nlargest(n, iterable, key=None): + """Find the n largest elements in a dataset. + + Equivalent to: sorted(iterable, key=key, reverse=True)[:n] + """ + + # Short-cut for n==1 is to use max() + if n == 1: + it = iter(iterable) + sentinel = object() + if key is None: + result = max(it, default=sentinel) + else: + result = max(it, default=sentinel, key=key) + return [] if result is sentinel else [result] + + # When n>=size, it's faster to use sorted() + try: + size = len(iterable) + except (TypeError, AttributeError): + pass + else: + if n >= size: + return sorted(iterable, key=key, reverse=True)[:n] + + # When key is none, use simpler decoration + if key is None: + it = iter(iterable) + result = [(elem, i) for i, elem in zip(range(0, -n, -1), it)] + if not result: + return result + heapify(result) + top = result[0][0] + order = -n + _heapreplace = heapreplace + for elem in it: + if top < elem: + _heapreplace(result, (elem, order)) + top = result[0][0] + order -= 1 + result.sort(reverse=True) + return [r[0] for r in result] + + # General case, slowest method + it = iter(iterable) + result = [(key(elem), i, elem) for i, elem in zip(range(0, -n, -1), it)] + if not result: + return result + heapify(result) + top = result[0][0] + order = -n + _heapreplace = heapreplace + for elem in it: + k = key(elem) + if top < k: + _heapreplace(result, (k, order, elem)) + top = result[0][0] + order -= 1 + result.sort(reverse=True) + return [r[2] for r in result] + +# If available, use C implementation +try: + from _heapq import * +except ImportError: + pass +try: + from _heapq import _heapreplace_max +except ImportError: + pass +try: + from _heapq import _heapify_max +except ImportError: + pass +try: + from _heapq import _heappop_max +except ImportError: + pass + + +if __name__ == "__main__": + + import doctest + print(doctest.testmod()) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 3a2e7649e6827..31919741e9d73 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -44,7 +44,7 @@ from pyspark.storagelevel import StorageLevel from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, \ - get_used_memory + get_used_memory, ExternalSorter from py4j.java_collections import ListConverter, MapConverter @@ -605,8 +605,13 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): if numPartitions is None: numPartitions = self._defaultReducePartitions() + spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() == 'true') + memory = _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m")) + serializer = self._jrdd_deserializer + def sortPartition(iterator): - return iter(sorted(iterator, key=lambda (k, v): keyfunc(k), reverse=not ascending)) + sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted + return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) if numPartitions == 1: if self.getNumPartitions() > 1: diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 1ebe7df418327..49829f5280a5f 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -21,7 +21,10 @@ import shutil import warnings import gc +import itertools +import random +import pyspark.heapq3 as heapq from pyspark.serializers import BatchedSerializer, PickleSerializer try: @@ -54,6 +57,17 @@ def get_used_memory(): return 0 +def _get_local_dirs(sub): + """ Get all the directories """ + path = os.environ.get("SPARK_LOCAL_DIRS", "/tmp") + dirs = path.split(",") + if len(dirs) > 1: + # different order in different processes and instances + rnd = random.Random(os.getpid() + id(dirs)) + random.shuffle(dirs, rnd.random) + return [os.path.join(d, "python", str(os.getpid()), sub) for d in dirs] + + class Aggregator(object): """ @@ -196,7 +210,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, # default serializer is only used for tests self.serializer = serializer or \ BatchedSerializer(PickleSerializer(), 1024) - self.localdirs = localdirs or self._get_dirs() + self.localdirs = localdirs or _get_local_dirs(str(id(self))) # number of partitions when spill data into disks self.partitions = partitions # check the memory after # of items merged @@ -212,13 +226,6 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, # randomize the hash of key, id(o) is the address of o (aligned by 8) self._seed = id(self) + 7 - def _get_dirs(self): - """ Get all the directories """ - path = os.environ.get("SPARK_LOCAL_DIRS", "/tmp") - dirs = path.split(",") - return [os.path.join(d, "python", str(os.getpid()), str(id(self))) - for d in dirs] - def _get_spill_dir(self, n): """ Choose one directory for spill by number n """ return os.path.join(self.localdirs[n % len(self.localdirs)], str(n)) @@ -434,6 +441,74 @@ def _recursive_merged_items(self, start): os.remove(os.path.join(path, str(i))) +class ExternalSorter(object): + """ + ExtenalSorter will divide the elements into chunks, sort them in + memory and dump them into disks, finally merge them back. + + The spilling will only happen when the used memory goes above + the limit. + + >>> sorter = ExternalSorter(1) # 1M + >>> import random + >>> l = range(1024) + >>> random.shuffle(l) + >>> sorted(l) == list(sorter.sorted(l)) + True + >>> sorted(l) == list(sorter.sorted(l, key=lambda x: -x, reverse=True)) + True + """ + def __init__(self, memory_limit, serializer=None): + self.memory_limit = memory_limit + self.local_dirs = _get_local_dirs("sort") + self.serializer = serializer or BatchedSerializer(PickleSerializer(), 1024) + self._spilled_bytes = 0 + + def _get_path(self, n): + """ Choose one directory for spill by number n """ + d = self.local_dirs[n % len(self.local_dirs)] + if not os.path.exists(d): + os.makedirs(d) + return os.path.join(d, str(n)) + + def sorted(self, iterator, key=None, reverse=False): + """ + Sort the elements in iterator, do external sort when the memory + goes above the limit. + """ + batch = 10 + chunks, current_chunk = [], [] + iterator = iter(iterator) + while True: + # pick elements in batch + chunk = list(itertools.islice(iterator, batch)) + current_chunk.extend(chunk) + if len(chunk) < batch: + break + + if get_used_memory() > self.memory_limit: + # sort them inplace will save memory + current_chunk.sort(key=key, reverse=reverse) + path = self._get_path(len(chunks)) + with open(path, 'w') as f: + self.serializer.dump_stream(current_chunk, f) + self._spilled_bytes += os.path.getsize(path) + chunks.append(self.serializer.load_stream(open(path))) + current_chunk = [] + + elif not chunks: + batch = min(batch * 2, 10000) + + current_chunk.sort(key=key, reverse=reverse) + if not chunks: + return current_chunk + + if current_chunk: + chunks.append(iter(current_chunk)) + + return heapq.merge(chunks, key=key, reverse=reverse) + + if __name__ == "__main__": import doctest doctest.testmod() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 1db922f513743..3e7040eade1ab 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -30,6 +30,7 @@ import tempfile import time import zipfile +import random if sys.version_info[:2] <= (2, 6): import unittest2 as unittest @@ -37,10 +38,11 @@ import unittest +from pyspark.conf import SparkConf from pyspark.context import SparkContext from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer -from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger +from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter _have_scipy = False _have_numpy = False @@ -117,6 +119,44 @@ def test_huge_dataset(self): m._cleanup() +class TestSorter(unittest.TestCase): + def test_in_memory_sort(self): + l = range(1024) + random.shuffle(l) + sorter = ExternalSorter(1024) + self.assertEquals(sorted(l), list(sorter.sorted(l))) + self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + + def test_external_sort(self): + l = range(1024) + random.shuffle(l) + sorter = ExternalSorter(1) + self.assertEquals(sorted(l), list(sorter.sorted(l))) + self.assertGreater(sorter._spilled_bytes, 0) + last = sorter._spilled_bytes + self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertGreater(sorter._spilled_bytes, last) + last = sorter._spilled_bytes + self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertGreater(sorter._spilled_bytes, last) + last = sorter._spilled_bytes + self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + self.assertGreater(sorter._spilled_bytes, last) + + def test_external_sort_in_rdd(self): + conf = SparkConf().set("spark.python.worker.memory", "1m") + sc = SparkContext(conf=conf) + l = range(10240) + random.shuffle(l) + rdd = sc.parallelize(l, 10) + self.assertEquals(sorted(l), rdd.sortBy(lambda x: x).collect()) + sc.stop() + + class SerializationTestCase(unittest.TestCase): def test_namedtuple(self): diff --git a/tox.ini b/tox.ini index a1fefdd0e176f..b568029a204cc 100644 --- a/tox.ini +++ b/tox.ini @@ -15,4 +15,4 @@ [pep8] max-line-length=100 -exclude=cloudpickle.py +exclude=cloudpickle.py,heapq3.py From 2ffd3290fe30c23df8da1efe153b84c23eb2e1cd Mon Sep 17 00:00:00 2001 From: WangTao Date: Tue, 26 Aug 2014 17:30:59 -0700 Subject: [PATCH 09/51] [SPARK-3225]Typo in script use_conf_dir => user_conf_dir in load-spark-env.sh. Author: WangTao Closes #1926 from WangTaoTheTonic/TypoInScript and squashes the following commits: 0c104ad [WangTao] Typo in script --- bin/load-spark-env.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index d425f9feaac54..493d3785a081b 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -27,12 +27,12 @@ if [ -z "$SPARK_ENV_LOADED" ]; then # Returns the parent of the directory this script lives in. parent_dir="$(cd `dirname $0`/..; pwd)" - use_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} + user_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} - if [ -f "${use_conf_dir}/spark-env.sh" ]; then + if [ -f "${user_conf_dir}/spark-env.sh" ]; then # Promote all variable declarations to environment (exported) variables set -a - . "${use_conf_dir}/spark-env.sh" + . "${user_conf_dir}/spark-env.sh" set +a fi fi From faeb9c0e1440f4af888be0dfc5de7b57efc92b00 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Tue, 26 Aug 2014 17:33:40 -0700 Subject: [PATCH 10/51] [SPARK-2964] [SQL] Remove duplicated code from spark-sql and start-thriftserver.sh Author: Cheng Lian Author: Kousuke Saruta Closes #1886 from sarutak/SPARK-2964 and squashes the following commits: 8ef8751 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-2964 26e7c95 [Kousuke Saruta] Revert "Shorten timeout to more reasonable value" ffb68fa [Kousuke Saruta] Modified spark-sql and start-thriftserver.sh to use bin/utils.sh 8c6f658 [Kousuke Saruta] Merge branch 'spark-3026' of https://github.com/liancheng/spark into SPARK-2964 81b43a8 [Cheng Lian] Shorten timeout to more reasonable value a89e66d [Cheng Lian] Fixed command line options quotation in scripts 9c894d3 [Cheng Lian] Fixed bin/spark-sql -S option typo be4736b [Cheng Lian] Report better error message when running JDBC/CLI without hive-thriftserver profile enabled --- bin/spark-sql | 55 +++++-------------- .../org/apache/spark/deploy/SparkSubmit.scala | 14 ++++- sbin/start-thriftserver.sh | 39 +++++-------- 3 files changed, 39 insertions(+), 69 deletions(-) diff --git a/bin/spark-sql b/bin/spark-sql index 564f1f419060f..2a3cb31f58e8d 100755 --- a/bin/spark-sql +++ b/bin/spark-sql @@ -24,6 +24,7 @@ set -o posix CLASS="org.apache.spark.sql.hive.thriftserver.SparkSQLCLIDriver" +CLASS_NOT_FOUND_EXIT_STATUS=1 # Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" @@ -43,52 +44,22 @@ function usage { $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } -function ensure_arg_number { - arg_number=$1 - at_least=$2 - - if [[ $arg_number -lt $at_least ]]; then - usage - exit 1 - fi -} - -if [[ "$@" = --help ]] || [[ "$@" = -h ]]; then +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage exit 0 fi -CLI_ARGS=() -SUBMISSION_ARGS=() - -while (($#)); do - case $1 in - -d | --define | --database | -f | -h | --hiveconf | --hivevar | -i | -p) - ensure_arg_number $# 2 - CLI_ARGS+=("$1"); shift - CLI_ARGS+=("$1"); shift - ;; +source $FWDIR/bin/utils.sh +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" - -e) - ensure_arg_number $# 2 - CLI_ARGS+=("$1"); shift - CLI_ARGS+=("$1"); shift - ;; +"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" +exit_status=$? - -s | --silent) - CLI_ARGS+=("$1"); shift - ;; - - -v | --verbose) - # Both SparkSubmit and SparkSQLCLIDriver recognizes -v | --verbose - CLI_ARGS+=("$1") - SUBMISSION_ARGS+=("$1"); shift - ;; - - *) - SUBMISSION_ARGS+=("$1"); shift - ;; - esac -done +if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then + echo + echo "Failed to load Spark SQL CLI main class $CLASS." + echo "You need to build Spark with -Phive." +fi -exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${CLI_ARGS[@]}" +exit $exit_status 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 f8cdbc3c392b5..550ee72538900 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -54,6 +54,8 @@ object SparkSubmit { private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" + private val CLASS_NOT_FOUND_EXIT_STATUS = 1 + // Exposed for testing private[spark] var exitFn: () => Unit = () => System.exit(-1) private[spark] var printStream: PrintStream = System.err @@ -311,8 +313,18 @@ object SparkSubmit { System.setProperty(key, value) } - val mainClass = Class.forName(childMainClass, true, loader) + var mainClass: Class[_] = null + + try { + mainClass = Class.forName(childMainClass, true, loader) + } catch { + case e: ClassNotFoundException => + e.printStackTrace(printStream) + System.exit(CLASS_NOT_FOUND_EXIT_STATUS) + } + val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) + try { mainMethod.invoke(null, childArgs.toArray) } catch { diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index 2c4452473ccbc..c519a77df4a14 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -27,6 +27,7 @@ set -o posix FWDIR="$(cd `dirname $0`/..; pwd)" CLASS="org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" +CLASS_NOT_FOUND_EXIT_STATUS=1 function usage { echo "Usage: ./sbin/start-thriftserver [options] [thrift server options]" @@ -43,36 +44,22 @@ function usage { $FWDIR/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 } -function ensure_arg_number { - arg_number=$1 - at_least=$2 - - if [[ $arg_number -lt $at_least ]]; then - usage - exit 1 - fi -} - -if [[ "$@" = --help ]] || [[ "$@" = -h ]]; then +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage exit 0 fi -THRIFT_SERVER_ARGS=() -SUBMISSION_ARGS=() +source $FWDIR/bin/utils.sh +SUBMIT_USAGE_FUNCTION=usage +gatherSparkSubmitOpts "$@" -while (($#)); do - case $1 in - --hiveconf) - ensure_arg_number $# 2 - THRIFT_SERVER_ARGS+=("$1"); shift - THRIFT_SERVER_ARGS+=("$1"); shift - ;; +"$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_OPTS[@]}" spark-internal "${APPLICATION_OPTS[@]}" +exit_status=$? - *) - SUBMISSION_ARGS+=("$1"); shift - ;; - esac -done +if [[ exit_status -eq CLASS_NOT_FOUND_EXIT_STATUS ]]; then + echo + echo "Failed to load Hive Thrift server main class $CLASS." + echo "You need to build Spark with -Phive." +fi -exec "$FWDIR"/bin/spark-submit --class $CLASS "${SUBMISSION_ARGS[@]}" spark-internal "${THRIFT_SERVER_ARGS[@]}" +exit $exit_status From 73b3089b8d2901dab11bb1ef6f46c29625b677fe Mon Sep 17 00:00:00 2001 From: nchammas Date: Tue, 26 Aug 2014 17:50:04 -0700 Subject: [PATCH 11/51] [Docs] Run tests like in contributing guide The Contributing to Spark guide [recommends](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark#ContributingtoSpark-AutomatedTesting) running tests by calling `./dev/run-tests`. The README should, too. `./sbt/sbt test` does not cover Python tests or style tests. Author: nchammas Closes #2149 from nchammas/patch-2 and squashes the following commits: 2b3b132 [nchammas] [Docs] Run tests like in contributing guide --- README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/README.md b/README.md index f75226ce11335..0a683a460ffac 100644 --- a/README.md +++ b/README.md @@ -69,7 +69,7 @@ Many of the example programs print usage help if no params are given. Testing first requires [building Spark](#building-spark). Once Spark is built, tests can be run using: - ./sbt/sbt test + ./dev/run-tests ## A Note About Hadoop Versions From 727cb25bcc29481d6b744abef1ca091e64b5f91f Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Tue, 26 Aug 2014 18:28:41 -0700 Subject: [PATCH 12/51] [SPARK-3036][SPARK-3037][SQL] Add MapType/ArrayType containing null value support to Parquet. JIRA: - https://issues.apache.org/jira/browse/SPARK-3036 - https://issues.apache.org/jira/browse/SPARK-3037 Currently this uses the following Parquet schema for `MapType` when `valueContainsNull` is `true`: ``` message root { optional group a (MAP) { repeated group map (MAP_KEY_VALUE) { required int32 key; optional int32 value; } } } ``` for `ArrayType` when `containsNull` is `true`: ``` message root { optional group a (LIST) { repeated group bag { optional int32 array; } } } ``` We have to think about compatibilities with older version of Spark or Hive or others I mentioned in the JIRA issues. Notice: This PR is based on #1963 and #1889. Please check them first. /cc marmbrus, yhuai Author: Takuya UESHIN Closes #2032 from ueshin/issues/SPARK-3036_3037 and squashes the following commits: 4e8e9e7 [Takuya UESHIN] Add ArrayType containing null value support to Parquet. 013c2ca [Takuya UESHIN] Add MapType containing null value support to Parquet. 62989de [Takuya UESHIN] Merge branch 'issues/SPARK-2969' into issues/SPARK-3036_3037 8e38b53 [Takuya UESHIN] Merge branch 'issues/SPARK-3063' into issues/SPARK-3036_3037 --- .../spark/sql/parquet/ParquetConverter.scala | 83 +++++++++++++++++++ .../sql/parquet/ParquetTableSupport.scala | 54 +++++++----- .../spark/sql/parquet/ParquetTypes.scala | 54 ++++++++---- .../spark/sql/parquet/ParquetQuerySuite.scala | 16 +++- 4 files changed, 167 insertions(+), 40 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index ef4526ec03439..9fd6aed402838 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -58,6 +58,7 @@ private[sql] object CatalystConverter { // This is mostly Parquet convention (see, e.g., `ConversionPatterns`). // Note that "array" for the array elements is chosen by ParquetAvro. // Using a different value will result in Parquet silently dropping columns. + val ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME = "bag" val ARRAY_ELEMENTS_SCHEMA_NAME = "array" val MAP_KEY_SCHEMA_NAME = "key" val MAP_VALUE_SCHEMA_NAME = "value" @@ -82,6 +83,9 @@ private[sql] object CatalystConverter { case ArrayType(elementType: DataType, false) => { new CatalystArrayConverter(elementType, fieldIndex, parent) } + case ArrayType(elementType: DataType, true) => { + new CatalystArrayContainsNullConverter(elementType, fieldIndex, parent) + } case StructType(fields: Seq[StructField]) => { new CatalystStructConverter(fields.toArray, fieldIndex, parent) } @@ -567,6 +571,85 @@ private[parquet] class CatalystNativeArrayConverter( } } +/** + * A `parquet.io.api.GroupConverter` that converts a single-element groups that + * match the characteristics of an array contains null (see + * [[org.apache.spark.sql.parquet.ParquetTypesConverter]]) into an + * [[org.apache.spark.sql.catalyst.types.ArrayType]]. + * + * @param elementType The type of the array elements (complex or primitive) + * @param index The position of this (array) field inside its parent converter + * @param parent The parent converter + * @param buffer A data buffer + */ +private[parquet] class CatalystArrayContainsNullConverter( + val elementType: DataType, + val index: Int, + protected[parquet] val parent: CatalystConverter, + protected[parquet] var buffer: Buffer[Any]) + extends CatalystConverter { + + def this(elementType: DataType, index: Int, parent: CatalystConverter) = + this( + elementType, + index, + parent, + new ArrayBuffer[Any](CatalystArrayConverter.INITIAL_ARRAY_SIZE)) + + protected[parquet] val converter: Converter = new CatalystConverter { + + private var current: Any = null + + val converter = CatalystConverter.createConverter( + new CatalystConverter.FieldType( + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + elementType, + false), + fieldIndex = 0, + parent = this) + + override def getConverter(fieldIndex: Int): Converter = converter + + override def end(): Unit = parent.updateField(index, current) + + override def start(): Unit = { + current = null + } + + override protected[parquet] val size: Int = 1 + override protected[parquet] val index: Int = 0 + override protected[parquet] val parent = CatalystArrayContainsNullConverter.this + + override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { + current = value + } + + override protected[parquet] def clearBuffer(): Unit = {} + } + + override def getConverter(fieldIndex: Int): Converter = converter + + // arrays have only one (repeated) field, which is its elements + override val size = 1 + + override protected[parquet] def updateField(fieldIndex: Int, value: Any): Unit = { + buffer += value + } + + override protected[parquet] def clearBuffer(): Unit = { + buffer.clear() + } + + override def start(): Unit = {} + + override def end(): Unit = { + assert(parent != null) + // here we need to make sure to use ArrayScalaType + parent.updateField(index, buffer.toArray.toSeq) + clearBuffer() + } +} + /** * This converter is for multi-element groups of primitive or complex types * that have repetition level optional or required (so struct fields). diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 6a657c20fe46c..bdf02401b21be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -173,7 +173,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { private[parquet] def writeValue(schema: DataType, value: Any): Unit = { if (value != null) { schema match { - case t @ ArrayType(_, false) => writeArray( + case t @ ArrayType(_, _) => writeArray( t, value.asInstanceOf[CatalystConverter.ArrayScalaType[_]]) case t @ MapType(_, _, _) => writeMap( @@ -228,45 +228,57 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { } } - // TODO: support null values, see - // https://issues.apache.org/jira/browse/SPARK-1649 private[parquet] def writeArray( schema: ArrayType, array: CatalystConverter.ArrayScalaType[_]): Unit = { val elementType = schema.elementType writer.startGroup() if (array.size > 0) { - writer.startField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) - var i = 0 - while(i < array.size) { - writeValue(elementType, array(i)) - i = i + 1 + if (schema.containsNull) { + writer.startField(CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME, 0) + var i = 0 + while (i < array.size) { + writer.startGroup() + if (array(i) != null) { + writer.startField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) + writeValue(elementType, array(i)) + writer.endField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) + } + writer.endGroup() + i = i + 1 + } + writer.endField(CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME, 0) + } else { + writer.startField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) + var i = 0 + while (i < array.size) { + writeValue(elementType, array(i)) + i = i + 1 + } + writer.endField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) } - writer.endField(CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, 0) } writer.endGroup() } - // TODO: support null values, see - // https://issues.apache.org/jira/browse/SPARK-1649 private[parquet] def writeMap( schema: MapType, map: CatalystConverter.MapScalaType[_, _]): Unit = { writer.startGroup() if (map.size > 0) { writer.startField(CatalystConverter.MAP_SCHEMA_NAME, 0) - writer.startGroup() - writer.startField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) - for(key <- map.keys) { + for ((key, value) <- map) { + writer.startGroup() + writer.startField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) writeValue(schema.keyType, key) + writer.endField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) + if (value != null) { + writer.startField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) + writeValue(schema.valueType, value) + writer.endField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) + } + writer.endGroup() } - writer.endField(CatalystConverter.MAP_KEY_SCHEMA_NAME, 0) - writer.startField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) - for(value <- map.values) { - writeValue(schema.valueType, value) - } - writer.endField(CatalystConverter.MAP_VALUE_SCHEMA_NAME, 1) - writer.endGroup() writer.endField(CatalystConverter.MAP_SCHEMA_NAME, 0) } writer.endGroup() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala index af8cd0a73b674..1a52377651737 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTypes.scala @@ -119,7 +119,13 @@ private[parquet] object ParquetTypesConverter extends Logging { case ParquetOriginalType.LIST => { // TODO: check enums! assert(groupType.getFieldCount == 1) val field = groupType.getFields.apply(0) - ArrayType(toDataType(field, isBinaryAsString), containsNull = false) + if (field.getName == CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME) { + val bag = field.asGroupType() + assert(bag.getFieldCount == 1) + ArrayType(toDataType(bag.getFields.apply(0), isBinaryAsString), containsNull = true) + } else { + ArrayType(toDataType(field, isBinaryAsString), containsNull = false) + } } case ParquetOriginalType.MAP => { assert( @@ -129,28 +135,32 @@ private[parquet] object ParquetTypesConverter extends Logging { assert( keyValueGroup.getFieldCount == 2, "Parquet Map type malformatted: nested group should have 2 (key, value) fields!") - val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) + + val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) - assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) - // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true - // at here. - MapType(keyType, valueType) + MapType(keyType, valueType, + keyValueGroup.getFields.apply(1).getRepetition != Repetition.REQUIRED) } case _ => { // Note: the order of these checks is important! if (correspondsToMap(groupType)) { // MapType val keyValueGroup = groupType.getFields.apply(0).asGroupType() - val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) assert(keyValueGroup.getFields.apply(0).getRepetition == Repetition.REQUIRED) + + val keyType = toDataType(keyValueGroup.getFields.apply(0), isBinaryAsString) val valueType = toDataType(keyValueGroup.getFields.apply(1), isBinaryAsString) - assert(keyValueGroup.getFields.apply(1).getRepetition == Repetition.REQUIRED) - // TODO: set valueContainsNull explicitly instead of assuming valueContainsNull is true - // at here. - MapType(keyType, valueType) + MapType(keyType, valueType, + keyValueGroup.getFields.apply(1).getRepetition != Repetition.REQUIRED) } else if (correspondsToArray(groupType)) { // ArrayType - val elementType = toDataType(groupType.getFields.apply(0), isBinaryAsString) - ArrayType(elementType, containsNull = false) + val field = groupType.getFields.apply(0) + if (field.getName == CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME) { + val bag = field.asGroupType() + assert(bag.getFieldCount == 1) + ArrayType(toDataType(bag.getFields.apply(0), isBinaryAsString), containsNull = true) + } else { + ArrayType(toDataType(field, isBinaryAsString), containsNull = false) + } } else { // everything else: StructType val fields = groupType .getFields @@ -249,13 +259,27 @@ private[parquet] object ParquetTypesConverter extends Logging { inArray = true) ConversionPatterns.listType(repetition, name, parquetElementType) } + case ArrayType(elementType, true) => { + val parquetElementType = fromDataType( + elementType, + CatalystConverter.ARRAY_ELEMENTS_SCHEMA_NAME, + nullable = true, + inArray = false) + ConversionPatterns.listType( + repetition, + name, + new ParquetGroupType( + Repetition.REPEATED, + CatalystConverter.ARRAY_CONTAINS_NULL_BAG_SCHEMA_NAME, + parquetElementType)) + } case StructType(structFields) => { val fields = structFields.map { field => fromDataType(field.dataType, field.name, field.nullable, inArray = false) } new ParquetGroupType(repetition, name, fields) } - case MapType(keyType, valueType, _) => { + case MapType(keyType, valueType, valueContainsNull) => { val parquetKeyType = fromDataType( keyType, @@ -266,7 +290,7 @@ private[parquet] object ParquetTypesConverter extends Logging { fromDataType( valueType, CatalystConverter.MAP_VALUE_SCHEMA_NAME, - nullable = false, + nullable = valueContainsNull, inArray = false) ConversionPatterns.mapType( repetition, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 28f43b36832ac..4219cc080000b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -78,7 +78,9 @@ case class AllDataTypesWithNonPrimitiveType( booleanField: Boolean, binaryField: Array[Byte], array: Seq[Int], - map: Map[Int, String], + arrayContainsNull: Seq[Option[Int]], + map: Map[Int, Long], + mapValueContainsNull: Map[Int, Option[Long]], data: Data) class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterAll { @@ -287,7 +289,11 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA .map(x => AllDataTypesWithNonPrimitiveType( s"$x", x, x.toLong, x.toFloat, x.toDouble, x.toShort, x.toByte, x % 2 == 0, (0 to x).map(_.toByte).toArray, - (0 until x), (0 until x).map(i => i -> s"$i").toMap, Data((0 until x), Nested(x, s"$x")))) + (0 until x), + (0 until x).map(Option(_).filter(_ % 3 == 0)), + (0 until x).map(i => i -> i.toLong).toMap, + (0 until x).map(i => i -> Option(i.toLong)).toMap + (x -> None), + Data((0 until x), Nested(x, s"$x")))) .saveAsParquetFile(tempDir) val result = parquetFile(tempDir).collect() range.foreach { @@ -302,8 +308,10 @@ class ParquetQuerySuite extends QueryTest with FunSuiteLike with BeforeAndAfterA assert(result(i).getBoolean(7) === (i % 2 == 0)) assert(result(i)(8) === (0 to i).map(_.toByte).toArray) assert(result(i)(9) === (0 until i)) - assert(result(i)(10) === (0 until i).map(i => i -> s"$i").toMap) - assert(result(i)(11) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) + assert(result(i)(10) === (0 until i).map(i => if (i % 3 == 0) i else null)) + assert(result(i)(11) === (0 until i).map(i => i -> i.toLong).toMap) + assert(result(i)(12) === (0 until i).map(i => i -> i.toLong).toMap + (i -> null)) + assert(result(i)(13) === new GenericRow(Array[Any]((0 until i), new GenericRow(Array[Any](i, s"$i"))))) } } From be043e3f20c6562482f9e4e739d8bb3fc9c1f201 Mon Sep 17 00:00:00 2001 From: Martin Weindel Date: Tue, 26 Aug 2014 18:30:39 -0700 Subject: [PATCH 13/51] [SPARK-3240] Adding known issue for MESOS-1688 When using Mesos with the fine-grained mode, a Spark job can run into a dead lock on low allocatable memory on Mesos slaves. As a work-around 32 MB (= Mesos MIN_MEM) are allocated for each task, to ensure Mesos making new offers after task completion. From my perspective, it would be better to fix this problem in Mesos by dropping the constraint on memory for offers, but as temporary solution this patch helps to avoid the dead lock on current Mesos versions. See [[MESOS-1688] No offers if no memory is allocatable](https://issues.apache.org/jira/browse/MESOS-1688) for details for this problem. Author: Martin Weindel Closes #1860 from MartinWeindel/master and squashes the following commits: 5762030 [Martin Weindel] reverting work-around a6bf837 [Martin Weindel] added known issue for issue MESOS-1688 d9d2ca6 [Martin Weindel] work around for problem with Mesos offering semantic (see [https://issues.apache.org/jira/browse/MESOS-1688]) --- docs/running-on-mesos.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index 9998dddc652a6..1073abb202c56 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -165,6 +165,8 @@ acquire. By default, it will acquire *all* cores in the cluster (that get offere only makes sense if you run just one application at a time. You can cap the maximum number of cores using `conf.set("spark.cores.max", "10")` (for example). +# Known issues +- When using the "fine-grained" mode, make sure that your executors always leave 32 MB free on the slaves. Otherwise it can happen that your Spark job does not proceed anymore. Currently, Apache Mesos only offers resources if there are at least 32 MB memory allocatable. But as Spark allocates memory only for the executor and cpu only for tasks, it can happen on high slave memory usage that no new tasks will be started anymore. More details can be found in [MESOS-1688](https://issues.apache.org/jira/browse/MESOS-1688). Alternatively use the "coarse-gained" mode, which is not affected by this issue. # Running Alongside Hadoop From d8345471ce8e59d693f7f59c480df339953a4d32 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 26 Aug 2014 18:55:00 -0700 Subject: [PATCH 14/51] Fix unclosed HTML tag in Yarn docs. --- docs/running-on-yarn.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 9bc20dbf926b2..943f06b114cb9 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -75,7 +75,7 @@ Most of the configs are the same for Spark on YARN as for other deployment modes (none) Comma-separated list of files to be placed in the working directory of each executor. - + spark.yarn.executor.memoryOverhead From ee91eb8c53da84f8875fb5cec5820a6e189057b9 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Aug 2014 19:29:54 -0700 Subject: [PATCH 15/51] Manually close some old pull requests Closes #530, Closes #223, Closes #738, Closes #546 From e70aff6c2d7f216060def0bd02be6a3d9017cd13 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 26 Aug 2014 19:35:19 -0700 Subject: [PATCH 16/51] Manually close old pull requests Closes #671, Closes #515 From bf719056b71d55e1194554661dfa194ed03d364d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 26 Aug 2014 21:59:48 -0700 Subject: [PATCH 17/51] [SPARK-3224] FetchFailed reduce stages should only show up once in failed stages (in UI) This is a HOTFIX for 1.1. Author: Reynold Xin Author: Kay Ousterhout Closes #2127 from rxin/SPARK-3224 and squashes the following commits: effb1ce [Reynold Xin] Move log message. 49282b3 [Reynold Xin] Kay's feedback. 3f01847 [Reynold Xin] Merge pull request #2 from kayousterhout/SPARK-3224 796d282 [Kay Ousterhout] Added unit test for SPARK-3224 3d3d356 [Reynold Xin] Remove map output loc even for repeated FetchFaileds. 1dd3eb5 [Reynold Xin] [SPARK-3224] FetchFailed reduce stages should only show up once in the failed stages UI. --- .../apache/spark/scheduler/DAGScheduler.scala | 32 +++++++++------ .../spark/scheduler/DAGSchedulerSuite.scala | 41 ++++++++++++++++++- 2 files changed, 59 insertions(+), 14 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 34131984570e4..2ccc27324ac8c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1045,31 +1045,39 @@ class DAGScheduler( stage.pendingTasks += task case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => - // Mark the stage that the reducer was in as unrunnable val failedStage = stageIdToStage(task.stageId) - markStageAsFinished(failedStage, Some("Fetch failure")) - runningStages -= failedStage - // TODO: Cancel running tasks in the stage - logInfo("Marking " + failedStage + " (" + failedStage.name + - ") for resubmision due to a fetch failure") - // Mark the map whose fetch failed as broken in the map stage val mapStage = shuffleToMapStage(shuffleId) - if (mapId != -1) { - mapStage.removeOutputLoc(mapId, bmAddress) - mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + + // It is likely that we receive multiple FetchFailed for a single stage (because we have + // multiple tasks running concurrently on different executors). In that case, it is possible + // the fetch failure has already been handled by the scheduler. + if (runningStages.contains(failedStage)) { + logInfo(s"Marking $failedStage (${failedStage.name}) as failed " + + s"due to a fetch failure from $mapStage (${mapStage.name})") + markStageAsFinished(failedStage, Some("Fetch failure")) + runningStages -= failedStage } - logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + - "); marking it for resubmission") + if (failedStages.isEmpty && eventProcessActor != null) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. eventProcessActor may be // null during unit tests. + // TODO: Cancel running tasks in the stage import env.actorSystem.dispatcher + logInfo(s"Resubmitting $mapStage (${mapStage.name}) and " + + s"$failedStage (${failedStage.name}) due to fetch failure") env.actorSystem.scheduler.scheduleOnce( RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) } failedStages += failedStage failedStages += mapStage + + // Mark the map whose fetch failed as broken in the map stage + if (mapId != -1) { + mapStage.removeOutputLoc(mapId, bmAddress) + mapOutputTracker.unregisterMapOutput(shuffleId, mapId, bmAddress) + } + // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { handleExecutorLost(bmAddress.executorId, Some(task.epoch)) 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 bd829752eb401..f5fed988ade24 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import scala.collection.mutable.{HashSet, HashMap, Map} +import scala.collection.mutable.{ArrayBuffer, HashSet, HashMap, Map} import scala.language.reflectiveCalls import akka.actor._ @@ -98,7 +98,7 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F val WAIT_TIMEOUT_MILLIS = 10000 val sparkListener = new SparkListener() { val successfulStages = new HashSet[Int]() - val failedStages = new HashSet[Int]() + val failedStages = new ArrayBuffer[Int]() override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { val stageInfo = stageCompleted.stageInfo if (stageInfo.failureReason.isEmpty) { @@ -435,6 +435,43 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assertDataStructuresEmpty } + test("trivial shuffle with multiple fetch failures") { + val shuffleMapRdd = new MyRDD(sc, 2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val shuffleId = shuffleDep.shuffleId + val reduceRdd = new MyRDD(sc, 2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) + complete(taskSets(0), Seq( + (Success, makeMapStatus("hostA", 1)), + (Success, makeMapStatus("hostB", 1)))) + // The MapOutputTracker should know about both map output locations. + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === + Array("hostA", "hostB")) + + // The first result task fails, with a fetch failure for the output from the first mapper. + runEvent(CompletionEvent( + taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0), + null, + Map[Long, Any](), + null, + null)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + + // The second ResultTask fails, with a fetch failure for the output from the second mapper. + runEvent(CompletionEvent( + taskSets(1).tasks(0), + FetchFailed(makeBlockManagerId("hostA"), shuffleId, 1, 1), + null, + Map[Long, Any](), + null, + null)) + // The SparkListener should not receive redundant failure events. + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.size == 1) + } + test("ignore late map task completions") { val shuffleMapRdd = new MyRDD(sc, 2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From 7557c4cfef2398d124b00472e2696f0559a36ef7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 26 Aug 2014 22:52:16 -0700 Subject: [PATCH 18/51] [SPARK-3167] Handle special driver configs in Windows This is an effort to bring the Windows scripts up to speed after recent splashing changes in #1845. Author: Andrew Or Closes #2129 from andrewor14/windows-config and squashes the following commits: 881a8f0 [Andrew Or] Add reference to Windows taskkill 92e6047 [Andrew Or] Update a few comments (minor) 22b1acd [Andrew Or] Fix style again (minor) afcffea [Andrew Or] Fix style (minor) 72004c2 [Andrew Or] Actually respect --driver-java-options 803218b [Andrew Or] Actually respect SPARK_*_CLASSPATH eeb34a0 [Andrew Or] Update outdated comment (minor) 35caecc [Andrew Or] In Windows, actually kill Java processes on exit f97daa2 [Andrew Or] Fix Windows spark shell stdin issue 83ebe60 [Andrew Or] Parse special driver configs in Windows (broken) --- bin/compute-classpath.cmd | 3 +- bin/spark-class2.cmd | 46 ++++++++++++++++--- bin/spark-submit | 2 +- bin/spark-submit.cmd | 34 +++++++++----- .../SparkSubmitDriverBootstrapper.scala | 19 +++++--- python/pyspark/java_gateway.py | 17 +++++++ 6 files changed, 95 insertions(+), 26 deletions(-) mode change 100755 => 100644 bin/spark-class2.cmd diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 58710cd1bd548..5ad52452a5c98 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -36,7 +36,8 @@ rem Load environment variables from conf\spark-env.cmd, if it exists if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" rem Build up classpath -set CLASSPATH=%FWDIR%conf +set CLASSPATH=%SPARK_CLASSPATH%;%SPARK_SUBMIT_CLASSPATH%;%FWDIR%conf + if exist "%FWDIR%RELEASE" ( for %%d in ("%FWDIR%lib\spark-assembly*.jar") do ( set ASSEMBLY_JAR=%%d diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd old mode 100755 new mode 100644 index e2c5f9c385189..6c5672819172b --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -17,6 +17,8 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem +rem Any changes to this file must be reflected in SparkSubmitDriverBootstrapper.scala! + setlocal enabledelayedexpansion set SCALA_VERSION=2.10 @@ -38,7 +40,7 @@ if not "x%1"=="x" goto arg_given if not "x%SPARK_MEM%"=="x" ( echo Warning: SPARK_MEM is deprecated, please use a more specific config option - echo e.g., spark.executor.memory or SPARK_DRIVER_MEMORY. + echo e.g., spark.executor.memory or spark.driver.memory. ) rem Use SPARK_MEM or 512m as the default memory, to be overridden by specific options @@ -67,10 +69,18 @@ rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_EXECUTOR_OPTS% if not "x%SPARK_EXECUTOR_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_EXECUTOR_MEMORY% -rem All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SPARK_REPL_OPTS. -) else if "%1"=="org.apache.spark.repl.Main" ( - set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_REPL_OPTS% +rem Spark submit uses SPARK_JAVA_OPTS + SPARK_SUBMIT_OPTS + +rem SPARK_DRIVER_MEMORY + SPARK_SUBMIT_DRIVER_MEMORY. +rem The repl also uses SPARK_REPL_OPTS. +) else if "%1"=="org.apache.spark.deploy.SparkSubmit" ( + set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_SUBMIT_OPTS% %SPARK_REPL_OPTS% + if not "x%SPARK_SUBMIT_LIBRARY_PATH%"=="x" ( + set OUR_JAVA_OPTS=!OUR_JAVA_OPTS! -Djava.library.path=%SPARK_SUBMIT_LIBRARY_PATH% + ) else if not "x%SPARK_LIBRARY_PATH%"=="x" ( + set OUR_JAVA_OPTS=!OUR_JAVA_OPTS! -Djava.library.path=%SPARK_LIBRARY_PATH% + ) if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% + if not "x%SPARK_SUBMIT_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_SUBMIT_DRIVER_MEMORY% ) else ( set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% @@ -80,9 +90,9 @@ rem Set JAVA_OPTS to be able to load native libraries and to set heap size for /f "tokens=3" %%i in ('java -version 2^>^&1 ^| find "version"') do set jversion=%%i for /f "tokens=1 delims=_" %%i in ("%jversion:~1,-1%") do set jversion=%%i if "%jversion%" geq "1.8.0" ( - set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% + set JAVA_OPTS=%OUR_JAVA_OPTS% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% ) else ( - set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% + set JAVA_OPTS=-XX:MaxPermSize=128m %OUR_JAVA_OPTS% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% ) rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in CommandUtils.scala! @@ -115,5 +125,27 @@ rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java -"%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* +rem In Spark submit client mode, the driver is launched in the same JVM as Spark submit itself. +rem Here we must parse the properties file for relevant "spark.driver.*" configs before launching +rem the driver JVM itself. Instead of handling this complexity here, we launch a separate JVM +rem to prepare the launch environment of this driver JVM. + +rem In this case, leave out the main class (org.apache.spark.deploy.SparkSubmit) and use our own. +rem Leaving out the first argument is surprisingly difficult to do in Windows. Note that this must +rem be done here because the Windows "shift" command does not work in a conditional block. +set BOOTSTRAP_ARGS= +shift +:start_parse +if "%~1" == "" goto end_parse +set BOOTSTRAP_ARGS=%BOOTSTRAP_ARGS% %~1 +shift +goto start_parse +:end_parse + +if not [%SPARK_SUBMIT_BOOTSTRAP_DRIVER%] == [] ( + set SPARK_CLASS=1 + "%RUNNER%" org.apache.spark.deploy.SparkSubmitDriverBootstrapper %BOOTSTRAP_ARGS% +) else ( + "%RUNNER%" -cp "%CLASSPATH%" %JAVA_OPTS% %* +) :exit diff --git a/bin/spark-submit b/bin/spark-submit index 32c911cd0438b..277c4ce571ca2 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -17,7 +17,7 @@ # limitations under the License. # -# NOTE: Any changes in this file must be reflected in SparkClassLauncher.scala! +# NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! export SPARK_HOME="$(cd `dirname $0`/..; pwd)" ORIG_ARGS=("$@") diff --git a/bin/spark-submit.cmd b/bin/spark-submit.cmd index 6eb702ed8c561..cf6046d1547ad 100644 --- a/bin/spark-submit.cmd +++ b/bin/spark-submit.cmd @@ -17,23 +17,28 @@ rem See the License for the specific language governing permissions and rem limitations under the License. rem +rem NOTE: Any changes in this file must be reflected in SparkSubmitDriverBootstrapper.scala! + set SPARK_HOME=%~dp0.. set ORIG_ARGS=%* -rem Clear the values of all variables used -set DEPLOY_MODE= -set DRIVER_MEMORY= +rem Reset the values of all variables used +set SPARK_SUBMIT_DEPLOY_MODE=client +set SPARK_SUBMIT_PROPERTIES_FILE=%SPARK_HOME%\conf\spark-defaults.conf +set SPARK_SUBMIT_DRIVER_MEMORY= set SPARK_SUBMIT_LIBRARY_PATH= set SPARK_SUBMIT_CLASSPATH= set SPARK_SUBMIT_OPTS= -set SPARK_DRIVER_MEMORY= +set SPARK_SUBMIT_BOOTSTRAP_DRIVER= :loop if [%1] == [] goto continue if [%1] == [--deploy-mode] ( - set DEPLOY_MODE=%2 + set SPARK_SUBMIT_DEPLOY_MODE=%2 + ) else if [%1] == [--properties-file] ( + set SPARK_SUBMIT_PROPERTIES_FILE=%2 ) else if [%1] == [--driver-memory] ( - set DRIVER_MEMORY=%2 + set SPARK_SUBMIT_DRIVER_MEMORY=%2 ) else if [%1] == [--driver-library-path] ( set SPARK_SUBMIT_LIBRARY_PATH=%2 ) else if [%1] == [--driver-class-path] ( @@ -45,12 +50,19 @@ if [%1] == [] goto continue goto loop :continue -if [%DEPLOY_MODE%] == [] ( - set DEPLOY_MODE=client -) +rem For client mode, the driver will be launched in the same JVM that launches +rem SparkSubmit, so we may need to read the properties file for any extra class +rem paths, library paths, java options and memory early on. Otherwise, it will +rem be too late by the time the driver JVM has started. -if not [%DRIVER_MEMORY%] == [] if [%DEPLOY_MODE%] == [client] ( - set SPARK_DRIVER_MEMORY=%DRIVER_MEMORY% +if [%SPARK_SUBMIT_DEPLOY_MODE%] == [client] ( + if exist %SPARK_SUBMIT_PROPERTIES_FILE% ( + rem Parse the properties file only if the special configs exist + for /f %%i in ('findstr /r /c:"^[\t ]*spark.driver.memory" /c:"^[\t ]*spark.driver.extra" ^ + %SPARK_SUBMIT_PROPERTIES_FILE%') do ( + set SPARK_SUBMIT_BOOTSTRAP_DRIVER=1 + ) + ) ) cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.spark.deploy.SparkSubmit %ORIG_ARGS% diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala index af607e6a4a065..7ca96ed57c2db 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitDriverBootstrapper.scala @@ -133,17 +133,24 @@ private[spark] object SparkSubmitDriverBootstrapper { val process = builder.start() // Redirect stdin, stdout, and stderr to/from the child JVM - val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") val stdoutThread = new RedirectThread(process.getInputStream, System.out, "redirect stdout") val stderrThread = new RedirectThread(process.getErrorStream, System.err, "redirect stderr") - stdinThread.start() stdoutThread.start() stderrThread.start() - // Terminate on broken pipe, which signals that the parent process has exited. This is - // important for the PySpark shell, where Spark submit itself is a python subprocess. - stdinThread.join() - process.destroy() + // In Windows, the subprocess reads directly from our stdin, so we should avoid spawning + // a thread that contends with the subprocess in reading from System.in. + if (Utils.isWindows) { + // For the PySpark shell, the termination of this process is handled in java_gateway.py + process.waitFor() + } else { + // Terminate on broken pipe, which signals that the parent process has exited. This is + // important for the PySpark shell, where Spark submit itself is a python subprocess. + val stdinThread = new RedirectThread(System.in, process.getOutputStream, "redirect stdin") + stdinThread.start() + stdinThread.join() + process.destroy() + } } } diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 6f4f62f23bc4d..9c70fa5c16d0c 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -15,6 +15,7 @@ # limitations under the License. # +import atexit import os import sys import signal @@ -69,6 +70,22 @@ def preexec_func(): error_msg += "--------------------------------------------------------------\n" raise Exception(error_msg) + # In Windows, ensure the Java child processes do not linger after Python has exited. + # In UNIX-based systems, the child process can kill itself on broken pipe (i.e. when + # the parent process' stdin sends an EOF). In Windows, however, this is not possible + # because java.lang.Process reads directly from the parent process' stdin, contending + # with any opportunity to read an EOF from the parent. Note that this is only best + # effort and will not take effect if the python process is violently terminated. + if on_windows: + # In Windows, the child process here is "spark-submit.cmd", not the JVM itself + # (because the UNIX "exec" command is not available). This means we cannot simply + # call proc.kill(), which kills only the "spark-submit.cmd" process but not the + # JVMs. Instead, we use "taskkill" with the tree-kill option "/t" to terminate all + # child processes in the tree (http://technet.microsoft.com/en-us/library/bb491009.aspx) + def killChild(): + Popen(["cmd", "/c", "taskkill", "/f", "/t", "/pid", str(proc.pid)]) + atexit.register(killChild) + # Create a thread to echo output from the GatewayServer, which is required # for Java log output to show up: class EchoOutputThread(Thread): From 9d65f2712c250a561c9c1f6259aa12e861ed239d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 26 Aug 2014 23:40:50 -0700 Subject: [PATCH 19/51] HOTFIX: Minor typo in conf template --- conf/spark-defaults.conf.template | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template index 94427029b94d7..a48dcc70e1363 100644 --- a/conf/spark-defaults.conf.template +++ b/conf/spark-defaults.conf.template @@ -7,4 +7,4 @@ # spark.eventLog.dir hdfs://namenode:8021/directory # spark.serializer org.apache.spark.serializer.KryoSerializer # spark.driver.memory 5g -# spark.executor.extraJavaOptions -XX:+PrintGCDetail -Dkey=value -Dnumbers="one two three" +# spark.executor.extraJavaOptions -XX:+PrintGCDetails -Dkey=value -Dnumbers="one two three" From 3e2864e40472b32e6a7eec5ba3bc83562d2a1a62 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Wed, 27 Aug 2014 00:13:38 -0700 Subject: [PATCH 20/51] [SPARK-3139] Made ContextCleaner to not block on shuffles As a workaround for SPARK-3015, the ContextCleaner was made "blocking", that is, it cleaned items one-by-one. But shuffles can take a long time to be deleted. Given that the RC for 1.1 is imminent, this PR makes a narrow change in the context cleaner - not wait for shuffle cleanups to complete. Also it changes the error messages on failure to delete to be milder warnings, as exceptions in the delete code path for one item does not really stop the actual functioning of the system. Author: Tathagata Das Closes #2143 from tdas/cleaner-shuffle-fix and squashes the following commits: 9c84202 [Tathagata Das] Restoring default blocking behavior in ContextCleanerSuite, and added docs to identify that spark.cleaner.referenceTracking.blocking does not control shuffle. 2181329 [Tathagata Das] Mark shuffle cleanup as non-blocking. e337cc2 [Tathagata Das] Changed semantics based on PR comments. 387b578 [Tathagata Das] Made ContextCleaner to not block on shuffles --- .../org/apache/spark/ContextCleaner.scala | 18 ++++++++++++++++-- .../spark/storage/BlockManagerMaster.scala | 12 +++++++----- .../org/apache/spark/ContextCleanerSuite.scala | 3 +++ 3 files changed, 26 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 3848734d6f639..ede1e23f4fcc5 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -65,7 +65,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { private val cleaningThread = new Thread() { override def run() { keepCleaning() }} /** - * Whether the cleaning thread will block on cleanup tasks. + * Whether the cleaning thread will block on cleanup tasks (other than shuffle, which + * is controlled by the `spark.cleaner.referenceTracking.blocking.shuffle` parameter). * * Due to SPARK-3015, this is set to true by default. This is intended to be only a temporary * workaround for the issue, which is ultimately caused by the way the BlockManager actors @@ -76,6 +77,19 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { private val blockOnCleanupTasks = sc.conf.getBoolean( "spark.cleaner.referenceTracking.blocking", true) + /** + * Whether the cleaning thread will block on shuffle cleanup tasks. + * + * When context cleaner is configured to block on every delete request, it can throw timeout + * exceptions on cleanup of shuffle blocks, as reported in SPARK-3139. To avoid that, this + * parameter by default disables blocking on shuffle cleanups. Note that this does not affect + * the cleanup of RDDs and broadcasts. This is intended to be a temporary workaround, + * until the real Akka issue (referred to in the comment above `blockOnCleanupTasks`) is + * resolved. + */ + private val blockOnShuffleCleanupTasks = sc.conf.getBoolean( + "spark.cleaner.referenceTracking.blocking.shuffle", false) + @volatile private var stopped = false /** Attach a listener object to get information of when objects are cleaned. */ @@ -128,7 +142,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { case CleanRDD(rddId) => doCleanupRDD(rddId, blocking = blockOnCleanupTasks) case CleanShuffle(shuffleId) => - doCleanupShuffle(shuffleId, blocking = blockOnCleanupTasks) + doCleanupShuffle(shuffleId, blocking = blockOnShuffleCleanupTasks) case CleanBroadcast(broadcastId) => doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 669307765d1fa..e67b3dc5ce02e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -101,7 +101,8 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log def removeRdd(rddId: Int, blocking: Boolean) { val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) future.onFailure { - case e: Throwable => logError("Failed to remove RDD " + rddId, e) + case e: Exception => + logWarning(s"Failed to remove RDD $rddId - ${e.getMessage}}") } if (blocking) { Await.result(future, timeout) @@ -112,7 +113,8 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log def removeShuffle(shuffleId: Int, blocking: Boolean) { val future = askDriverWithReply[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) future.onFailure { - case e: Throwable => logError("Failed to remove shuffle " + shuffleId, e) + case e: Exception => + logWarning(s"Failed to remove shuffle $shuffleId - ${e.getMessage}}") } if (blocking) { Await.result(future, timeout) @@ -124,9 +126,9 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log val future = askDriverWithReply[Future[Seq[Int]]]( RemoveBroadcast(broadcastId, removeFromMaster)) future.onFailure { - case e: Throwable => - logError("Failed to remove broadcast " + broadcastId + - " with removeFromMaster = " + removeFromMaster, e) + case e: Exception => + logWarning(s"Failed to remove broadcast $broadcastId" + + s" with removeFromMaster = $removeFromMaster - ${e.getMessage}}") } if (blocking) { Await.result(future, timeout) diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 4bc4346c0a288..2744894277ae8 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -52,6 +52,7 @@ abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[Ha .setMaster("local[2]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") .set("spark.shuffle.manager", shuffleManager.getName) before { @@ -243,6 +244,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { .setMaster("local-cluster[2, 1, 512]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") .set("spark.shuffle.manager", shuffleManager.getName) sc = new SparkContext(conf2) @@ -319,6 +321,7 @@ class SortShuffleContextCleanerSuite extends ContextCleanerSuiteBase(classOf[Sor .setMaster("local-cluster[2, 1, 512]") .setAppName("ContextCleanerSuite") .set("spark.cleaner.referenceTracking.blocking", "true") + .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") .set("spark.shuffle.manager", shuffleManager.getName) sc = new SparkContext(conf2) From e1139dd60e0692e8adb1337c1f605165ce4b8895 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 27 Aug 2014 00:59:23 -0700 Subject: [PATCH 21/51] [SPARK-3237][SQL] Fix parquet filters with UDFs Author: Michael Armbrust Closes #2153 from marmbrus/parquetFilters and squashes the following commits: 712731a [Michael Armbrust] Use closure serializer for sending filters. 1e83f80 [Michael Armbrust] Clean udf functions. --- .../apache/spark/sql/catalyst/expressions/ScalaUdf.scala | 4 ++++ .../org/apache/spark/sql/parquet/ParquetFilters.scala | 8 ++++++-- 2 files changed, 10 insertions(+), 2 deletions(-) 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 63ac2a608b6ff..0b3c1df453fb8 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,10 +18,14 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.util.ClosureCleaner case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression]) extends Expression { + // Clean function when not called with default no-arg constructor. + if (function != null) { ClosureCleaner.clean(function) } + type EvaluatedType = Any def nullable = true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 2298a9b933df5..fe28e0d7269e0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.parquet +import java.nio.ByteBuffer + import org.apache.hadoop.conf.Configuration import parquet.filter._ @@ -25,6 +27,7 @@ import parquet.column.ColumnReader import com.google.common.io.BaseEncoding +import org.apache.spark.SparkEnv import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.expressions.{Predicate => CatalystPredicate} import org.apache.spark.sql.catalyst.expressions._ @@ -237,7 +240,8 @@ object ParquetFilters { */ def serializeFilterExpressions(filters: Seq[Expression], conf: Configuration): Unit = { if (filters.length > 0) { - val serialized: Array[Byte] = SparkSqlSerializer.serialize(filters) + val serialized: Array[Byte] = + SparkEnv.get.closureSerializer.newInstance().serialize(filters).array() val encoded: String = BaseEncoding.base64().encode(serialized) conf.set(PARQUET_FILTER_DATA, encoded) } @@ -252,7 +256,7 @@ object ParquetFilters { val data = conf.get(PARQUET_FILTER_DATA) if (data != null) { val decoded: Array[Byte] = BaseEncoding.base64().decode(data) - SparkSqlSerializer.deserialize(decoded) + SparkEnv.get.closureSerializer.newInstance().deserialize(ByteBuffer.wrap(decoded)) } else { Seq() } From 43dfc84f883822ea27b6e312d4353bf301c2e7ef Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 27 Aug 2014 01:19:48 -0700 Subject: [PATCH 22/51] [SPARK-2830][MLLIB] doc update for 1.1 1. renamed mllib-basics to mllib-data-types 1. renamed mllib-stats to mllib-statistics 1. moved random data generation to the bottom of mllib-stats 1. updated toc accordingly atalwalkar Author: Xiangrui Meng Closes #2151 from mengxr/mllib-doc-1.1 and squashes the following commits: 0bd79f3 [Xiangrui Meng] add mllib-data-types b64a5d7 [Xiangrui Meng] update the content list of basis statistics in mllib-guide f625cc2 [Xiangrui Meng] move mllib-basics to mllib-data-types 4d69250 [Xiangrui Meng] move random data generation to the bottom of statistics e64f3ce [Xiangrui Meng] move mllib-stats.md to mllib-statistics.md --- docs/{mllib-basics.md => mllib-data-types.md} | 4 +- docs/mllib-dimensionality-reduction.md | 4 +- docs/mllib-guide.md | 9 +- docs/{mllib-stats.md => mllib-statistics.md} | 156 +++++++++--------- 4 files changed, 87 insertions(+), 86 deletions(-) rename docs/{mllib-basics.md => mllib-data-types.md} (99%) rename docs/{mllib-stats.md => mllib-statistics.md} (99%) diff --git a/docs/mllib-basics.md b/docs/mllib-data-types.md similarity index 99% rename from docs/mllib-basics.md rename to docs/mllib-data-types.md index 8752df412950a..101dc2f8695f3 100644 --- a/docs/mllib-basics.md +++ b/docs/mllib-data-types.md @@ -1,7 +1,7 @@ --- layout: global -title: Basics - MLlib -displayTitle: MLlib - Basics +title: Data Types - MLlib +displayTitle: MLlib - Data Types --- * Table of contents diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index 9f2cf6d48ec75..21cb35b4270ca 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -11,7 +11,7 @@ displayTitle: MLlib - Dimensionality Reduction of reducing the number of variables under consideration. It can be used to extract latent features from raw and noisy features or compress data while maintaining the structure. -MLlib provides support for dimensionality reduction on the RowMatrix class. +MLlib provides support for dimensionality reduction on the RowMatrix class. ## Singular value decomposition (SVD) @@ -58,7 +58,7 @@ passes, $O(n)$ storage on each executor, and $O(n k)$ storage on the driver. ### SVD Example MLlib provides SVD functionality to row-oriented matrices, provided in the -RowMatrix class. +RowMatrix class.
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 4d4198b9e0452..d3a510b3c17c6 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -7,12 +7,13 @@ MLlib is Spark's scalable machine learning library consisting of common learning including classification, regression, clustering, collaborative filtering, dimensionality reduction, as well as underlying optimization primitives, as outlined below: -* [Data types](mllib-basics.html) -* [Basic statistics](mllib-stats.html) - * random data generation - * stratified sampling +* [Data types](mllib-data-types.html) +* [Basic statistics](mllib-statistics.html) * summary statistics + * correlations + * stratified sampling * hypothesis testing + * random data generation * [Classification and regression](mllib-classification-regression.html) * [linear models (SVMs, logistic regression, linear regression)](mllib-linear-methods.html) * [decision trees](mllib-decision-tree.html) diff --git a/docs/mllib-stats.md b/docs/mllib-statistics.md similarity index 99% rename from docs/mllib-stats.md rename to docs/mllib-statistics.md index 511a9fbf710cc..c4632413991f1 100644 --- a/docs/mllib-stats.md +++ b/docs/mllib-statistics.md @@ -1,7 +1,7 @@ --- layout: global -title: Statistics Functionality - MLlib -displayTitle: MLlib - Statistics Functionality +title: Basic Statistics - MLlib +displayTitle: MLlib - Basic Statistics --- * Table of contents @@ -25,7 +25,7 @@ displayTitle: MLlib - Statistics Functionality \newcommand{\zero}{\mathbf{0}} \]` -## Summary Statistics +## Summary statistics We provide column summary statistics for `RDD[Vector]` through the function `colStats` available in `Statistics`. @@ -104,81 +104,7 @@ print summary.numNonzeros()
-## Random data generation - -Random data generation is useful for randomized algorithms, prototyping, and performance testing. -MLlib supports generating random RDDs with i.i.d. values drawn from a given distribution: -uniform, standard normal, or Poisson. - -
-
-[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory -methods to generate random double RDDs or vector RDDs. -The following example generates a random double RDD, whose values follows the standard normal -distribution `N(0, 1)`, and then map it to `N(1, 4)`. - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.random.RandomRDDs._ - -val sc: SparkContext = ... - -// Generate a random double RDD that contains 1 million i.i.d. values drawn from the -// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. -val u = normalRDD(sc, 1000000L, 10) -// Apply a transform to get a random double RDD following `N(1, 4)`. -val v = u.map(x => 1.0 + 2.0 * x) -{% endhighlight %} -
- -
-[`RandomRDDs`](api/java/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory -methods to generate random double RDDs or vector RDDs. -The following example generates a random double RDD, whose values follows the standard normal -distribution `N(0, 1)`, and then map it to `N(1, 4)`. - -{% highlight java %} -import org.apache.spark.SparkContext; -import org.apache.spark.api.JavaDoubleRDD; -import static org.apache.spark.mllib.random.RandomRDDs.*; - -JavaSparkContext jsc = ... - -// Generate a random double RDD that contains 1 million i.i.d. values drawn from the -// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. -JavaDoubleRDD u = normalJavaRDD(jsc, 1000000L, 10); -// Apply a transform to get a random double RDD following `N(1, 4)`. -JavaDoubleRDD v = u.map( - new Function() { - public Double call(Double x) { - return 1.0 + 2.0 * x; - } - }); -{% endhighlight %} -
- -
-[`RandomRDDs`](api/python/pyspark.mllib.random.RandomRDDs-class.html) provides factory -methods to generate random double RDDs or vector RDDs. -The following example generates a random double RDD, whose values follows the standard normal -distribution `N(0, 1)`, and then map it to `N(1, 4)`. - -{% highlight python %} -from pyspark.mllib.random import RandomRDDs - -sc = ... # SparkContext - -# Generate a random double RDD that contains 1 million i.i.d. values drawn from the -# standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. -u = RandomRDDs.uniformRDD(sc, 1000000L, 10) -# Apply a transform to get a random double RDD following `N(1, 4)`. -v = u.map(lambda x: 1.0 + 2.0 * x) -{% endhighlight %} -
- -
- -## Correlations calculation +## Correlations Calculating the correlation between two series of data is a common operation in Statistics. In MLlib we provide the flexibility to calculate pairwise correlations among many series. The supported @@ -455,3 +381,77 @@ for (ChiSqTestResult result : featureTestResults) {
+ +## Random data generation + +Random data generation is useful for randomized algorithms, prototyping, and performance testing. +MLlib supports generating random RDDs with i.i.d. values drawn from a given distribution: +uniform, standard normal, or Poisson. + +
+
+[`RandomRDDs`](api/scala/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.random.RandomRDDs._ + +val sc: SparkContext = ... + +// Generate a random double RDD that contains 1 million i.i.d. values drawn from the +// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +val u = normalRDD(sc, 1000000L, 10) +// Apply a transform to get a random double RDD following `N(1, 4)`. +val v = u.map(x => 1.0 + 2.0 * x) +{% endhighlight %} +
+ +
+[`RandomRDDs`](api/java/index.html#org.apache.spark.mllib.random.RandomRDDs) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight java %} +import org.apache.spark.SparkContext; +import org.apache.spark.api.JavaDoubleRDD; +import static org.apache.spark.mllib.random.RandomRDDs.*; + +JavaSparkContext jsc = ... + +// Generate a random double RDD that contains 1 million i.i.d. values drawn from the +// standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +JavaDoubleRDD u = normalJavaRDD(jsc, 1000000L, 10); +// Apply a transform to get a random double RDD following `N(1, 4)`. +JavaDoubleRDD v = u.map( + new Function() { + public Double call(Double x) { + return 1.0 + 2.0 * x; + } + }); +{% endhighlight %} +
+ +
+[`RandomRDDs`](api/python/pyspark.mllib.random.RandomRDDs-class.html) provides factory +methods to generate random double RDDs or vector RDDs. +The following example generates a random double RDD, whose values follows the standard normal +distribution `N(0, 1)`, and then map it to `N(1, 4)`. + +{% highlight python %} +from pyspark.mllib.random import RandomRDDs + +sc = ... # SparkContext + +# Generate a random double RDD that contains 1 million i.i.d. values drawn from the +# standard normal distribution `N(0, 1)`, evenly distributed in 10 partitions. +u = RandomRDDs.uniformRDD(sc, 1000000L, 10) +# Apply a transform to get a random double RDD following `N(1, 4)`. +v = u.map(lambda x: 1.0 + 2.0 * x) +{% endhighlight %} +
+ +
From 171a41cb034f4ea80f6a3c91a6872970de16a14a Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 27 Aug 2014 01:45:59 -0700 Subject: [PATCH 23/51] [SPARK-3227] [mllib] Added migration guide for v1.0 to v1.1 The only updates are in DecisionTree. CC: mengxr Author: Joseph K. Bradley Closes #2146 from jkbradley/mllib-migration and squashes the following commits: 5a1f487 [Joseph K. Bradley] small edit to doc 411d6d9 [Joseph K. Bradley] Added migration guide for v1.0 to v1.1. The only updates are in DecisionTree. --- docs/mllib-guide.md | 28 +++++++++++++++++++++++++++- 1 file changed, 27 insertions(+), 1 deletion(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index d3a510b3c17c6..94fc98ce4fabe 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -60,6 +60,32 @@ To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 # Migration Guide +## From 1.0 to 1.1 + +The only API changes in MLlib v1.1 are in +[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +which continues to be an experimental API in MLlib 1.1: + +1. *(Breaking change)* The meaning of tree depth has been changed by 1 in order to match +the implementations of trees in +[scikit-learn](http://scikit-learn.org/stable/modules/classes.html#module-sklearn.tree) +and in [rpart](http://cran.r-project.org/web/packages/rpart/index.html). +In MLlib v1.0, a depth-1 tree had 1 leaf node, and a depth-2 tree had 1 root node and 2 leaf nodes. +In MLlib v1.1, a depth-0 tree has 1 leaf node, and a depth-1 tree has 1 root node and 2 leaf nodes. +This depth is specified by the `maxDepth` parameter in +[`Strategy`](api/scala/index.html#org.apache.spark.mllib.tree.configuration.Strategy) +or via [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) +static `trainClassifier` and `trainRegressor` methods. + +2. *(Non-breaking change)* We recommend using the newly added `trainClassifier` and `trainRegressor` +methods to build a [`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +rather than using the old parameter class `Strategy`. These new training methods explicitly +separate classification and regression, and they replace specialized parameter types with +simple `String` types. + +Examples of the new, recommended `trainClassifier` and `trainRegressor` are given in the +[Decision Trees Guide](mllib-decision-tree.html#examples). + ## From 0.9 to 1.0 In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few @@ -85,7 +111,7 @@ val vector: Vector = Vectors.dense(array) // a dense vector [`Vectors`](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to create sparse vectors. -*Note*. Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. +*Note*: Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. From 6f671d04fa98f97fd48c5e749b9f47dd4a8b4f44 Mon Sep 17 00:00:00 2001 From: Hari Shreedharan Date: Wed, 27 Aug 2014 02:39:02 -0700 Subject: [PATCH 24/51] [SPARK-3154][STREAMING] Make FlumePollingInputDStream shutdown cleaner. Currently lot of errors get thrown from Avro IPC layer when the dstream or sink is shutdown. This PR cleans it up. Some refactoring is done in the receiver code to put all of the RPC code into a single Try and just recover from that. The sink code has also been cleaned up. Author: Hari Shreedharan Closes #2065 from harishreedharan/clean-flume-shutdown and squashes the following commits: f93a07c [Hari Shreedharan] Formatting fixes. d7427cc [Hari Shreedharan] More fixes! a0a8852 [Hari Shreedharan] Fix race condition, hopefully! Minor other changes. 4c9ed02 [Hari Shreedharan] Remove unneeded list in Callback handler. Other misc changes. 8fee36f [Hari Shreedharan] Scala-library is required, else maven build fails. Also catch InterruptedException in TxnProcessor. 445e700 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into clean-flume-shutdown 87232e0 [Hari Shreedharan] Refactor Flume Input Stream. Clean up code, better error handling. 9001d26 [Hari Shreedharan] Change log level to debug in TransactionProcessor#shutdown method e7b8d82 [Hari Shreedharan] Incorporate review feedback 598efa7 [Hari Shreedharan] Clean up some exception handling code e1027c6 [Hari Shreedharan] Merge remote-tracking branch 'asf/master' into clean-flume-shutdown ed608c8 [Hari Shreedharan] [SPARK-3154][STREAMING] Make FlumePollingInputDStream shutdown cleaner. --- external/flume-sink/pom.xml | 4 + .../flume/sink/SparkAvroCallbackHandler.scala | 56 ++++-- .../flume/sink/TransactionProcessor.scala | 18 +- .../streaming/flume/FlumeBatchFetcher.scala | 167 ++++++++++++++++++ .../flume/FlumePollingInputDStream.scala | 77 ++------ 5 files changed, 236 insertions(+), 86 deletions(-) create mode 100644 external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeBatchFetcher.scala diff --git a/external/flume-sink/pom.xml b/external/flume-sink/pom.xml index c1e8e65464fc1..b345276b08ba3 100644 --- a/external/flume-sink/pom.xml +++ b/external/flume-sink/pom.xml @@ -70,6 +70,10 @@ scalatest_${scala.binary.version} test + + org.scala-lang + scala-library +