From 65b987c3ed79b8362dda53b70434652d2d4840da Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 21 Nov 2014 10:06:30 -0800 Subject: [PATCH 001/305] [SPARK-4397][Core] Reorganize 'implicit's to improve the API convenience This PR moved `implicit`s to `package object` and `companion object` to enable the Scala compiler search them automatically without explicit importing. It should not break any API. A test project for backforward compatibility is [here](https://github.com/zsxwing/SPARK-4397-Backforward-Compatibility). It proves the codes compiled with Spark 1.1.0 can run with this PR. To summarize, the changes are: * Deprecated the old implicit conversion functions: this preserves binary compatibility for code compiled against earlier versions of Spark. * Removed "implicit" from them so they are just normal functions: this made sure the compiler doesn't get confused and warn about multiple implicits in scope. * Created new implicit functions in package rdd object, which is part of the scope that scalac will search when looking for implicit conversions on various RDD objects. The disadvantage is there are duplicated codes in SparkContext for backforward compatibility. Author: zsxwing Closes #3262 from zsxwing/SPARK-4397 and squashes the following commits: fc30314 [zsxwing] Update the comments 9c27aff [zsxwing] Move implicit functions to object RDD and forward old functions to new implicit ones directly 2b5f5a4 [zsxwing] Comments for the deprecated functions 52353de [zsxwing] Remove private[spark] from object WritableConverter 34641d4 [zsxwing] Move ImplicitSuite to org.apache.sparktest 7266218 [zsxwing] Add comments to warn the duplicate codes in SparkContext 185c12f [zsxwing] Remove simpleWritableConverter from SparkContext 3bdcae2 [zsxwing] Move WritableConverter implicits to object WritableConverter 9b73188 [zsxwing] Fix the code style issue 3ac4f07 [zsxwing] Add license header 1eda9e4 [zsxwing] Reorganize 'implicit's to improve the API convenience --- .../scala/org/apache/spark/Accumulators.scala | 30 ++++ .../scala/org/apache/spark/SparkContext.scala | 161 +++++++++++++----- .../apache/spark/api/java/JavaPairRDD.scala | 2 +- .../spark/api/java/JavaSparkContext.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 30 ++++ .../org/apache/spark/SparkContextSuite.scala | 2 +- .../org/apache/sparktest/ImplicitSuite.scala | 128 ++++++++++++++ 7 files changed, 311 insertions(+), 44 deletions(-) create mode 100644 core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 2301caafb07ff..dc1e8f6c21b62 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -244,6 +244,36 @@ trait AccumulatorParam[T] extends AccumulableParam[T, T] { } } +object AccumulatorParam { + + // The following implicit objects were in SparkContext before 1.2 and users had to + // `import SparkContext._` to enable them. Now we move them here to make the compiler find + // them automatically. However, as there are duplicate codes in SparkContext for backward + // compatibility, please update them accordingly if you modify the following implicit objects. + + implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { + def addInPlace(t1: Double, t2: Double): Double = t1 + t2 + def zero(initialValue: Double) = 0.0 + } + + implicit object IntAccumulatorParam extends AccumulatorParam[Int] { + def addInPlace(t1: Int, t2: Int): Int = t1 + t2 + def zero(initialValue: Int) = 0 + } + + implicit object LongAccumulatorParam extends AccumulatorParam[Long] { + def addInPlace(t1: Long, t2: Long) = t1 + t2 + def zero(initialValue: Long) = 0L + } + + implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { + def addInPlace(t1: Float, t2: Float) = t1 + t2 + def zero(initialValue: Float) = 0f + } + + // TODO: Add AccumulatorParams for other types, e.g. lists and strings +} + // TODO: The multi-thread support in accumulators is kind of lame; check // if there's a more intuitive way of doing it right private object Accumulators { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ae8bbfb56f493..586c1ccaca72b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1624,47 +1624,74 @@ object SparkContext extends Logging { private[spark] val DRIVER_IDENTIFIER = "" - implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { + // The following deprecated objects have already been copied to `object AccumulatorParam` to + // make the compiler find them automatically. They are duplicate codes only for backward + // compatibility, please update `object AccumulatorParam` accordingly if you plan to modify the + // following ones. + + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.2.0") + object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 } - implicit object IntAccumulatorParam extends AccumulatorParam[Int] { + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.2.0") + object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 def zero(initialValue: Int) = 0 } - implicit object LongAccumulatorParam extends AccumulatorParam[Long] { + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.2.0") + object LongAccumulatorParam extends AccumulatorParam[Long] { def addInPlace(t1: Long, t2: Long) = t1 + t2 def zero(initialValue: Long) = 0L } - implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { + @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + + "backward compatibility.", "1.2.0") + object FloatAccumulatorParam extends AccumulatorParam[Float] { def addInPlace(t1: Float, t2: Float) = t1 + t2 def zero(initialValue: Float) = 0f } - // TODO: Add AccumulatorParams for other types, e.g. lists and strings + // The following deprecated functions have already been moved to `object RDD` to + // make the compiler find them automatically. They are still kept here for backward compatibility + // and just call the corresponding functions in `object RDD`. - implicit def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { - new PairRDDFunctions(rdd) + RDD.rddToPairRDDFunctions(rdd) } - implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = RDD.rddToAsyncRDDActions(rdd) - implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( rdd: RDD[(K, V)]) = - new SequenceFileRDDFunctions(rdd) + RDD.rddToSequenceFileRDDFunctions(rdd) - implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( rdd: RDD[(K, V)]) = - new OrderedRDDFunctions[K, V, (K, V)](rdd) + RDD.rddToOrderedRDDFunctions(rdd) - implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = new DoubleRDDFunctions(rdd) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = RDD.doubleRDDToDoubleRDDFunctions(rdd) - implicit def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = - new DoubleRDDFunctions(rdd.map(x => num.toDouble(x))) + @deprecated("Replaced by implicit functions in the RDD companion object. This is " + + "kept here only for backward compatibility.", "1.2.0") + def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = + RDD.numericRDDToDoubleRDDFunctions(rdd) // Implicit conversions to common Writable types, for saveAsSequenceFile @@ -1690,40 +1717,49 @@ object SparkContext extends Logging { arr.map(x => anyToWritable(x)).toArray) } - // Helper objects for converting common types to Writable - private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) - : WritableConverter[T] = { - val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]] - new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) - } + // The following deprecated functions have already been moved to `object WritableConverter` to + // make the compiler find them automatically. They are still kept here for backward compatibility + // and just call the corresponding functions in `object WritableConverter`. - implicit def intWritableConverter(): WritableConverter[Int] = - simpleWritableConverter[Int, IntWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def intWritableConverter(): WritableConverter[Int] = + WritableConverter.intWritableConverter() - implicit def longWritableConverter(): WritableConverter[Long] = - simpleWritableConverter[Long, LongWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def longWritableConverter(): WritableConverter[Long] = + WritableConverter.longWritableConverter() - implicit def doubleWritableConverter(): WritableConverter[Double] = - simpleWritableConverter[Double, DoubleWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def doubleWritableConverter(): WritableConverter[Double] = + WritableConverter.doubleWritableConverter() - implicit def floatWritableConverter(): WritableConverter[Float] = - simpleWritableConverter[Float, FloatWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def floatWritableConverter(): WritableConverter[Float] = + WritableConverter.floatWritableConverter() - implicit def booleanWritableConverter(): WritableConverter[Boolean] = - simpleWritableConverter[Boolean, BooleanWritable](_.get) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def booleanWritableConverter(): WritableConverter[Boolean] = + WritableConverter.booleanWritableConverter() - implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = { - simpleWritableConverter[Array[Byte], BytesWritable](bw => - // getBytes method returns array which is longer then data to be returned - Arrays.copyOfRange(bw.getBytes, 0, bw.getLength) - ) - } + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def bytesWritableConverter(): WritableConverter[Array[Byte]] = + WritableConverter.bytesWritableConverter() - implicit def stringWritableConverter(): WritableConverter[String] = - simpleWritableConverter[String, Text](_.toString) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def stringWritableConverter(): WritableConverter[String] = + WritableConverter.stringWritableConverter() - implicit def writableWritableConverter[T <: Writable]() = - new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T]) + @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + + "backward compatibility.", "1.2.0") + def writableWritableConverter[T <: Writable]() = + WritableConverter.writableWritableConverter() /** * Find the JAR from which a given class was loaded, to make it easy for users to pass @@ -1950,3 +1986,46 @@ private[spark] class WritableConverter[T]( val writableClass: ClassTag[T] => Class[_ <: Writable], val convert: Writable => T) extends Serializable + +object WritableConverter { + + // Helper objects for converting common types to Writable + private[spark] def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) + : WritableConverter[T] = { + val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]] + new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) + } + + // The following implicit functions were in SparkContext before 1.2 and users had to + // `import SparkContext._` to enable them. Now we move them here to make the compiler find + // them automatically. However, we still keep the old functions in SparkContext for backward + // compatibility and forward to the following functions directly. + + implicit def intWritableConverter(): WritableConverter[Int] = + simpleWritableConverter[Int, IntWritable](_.get) + + implicit def longWritableConverter(): WritableConverter[Long] = + simpleWritableConverter[Long, LongWritable](_.get) + + implicit def doubleWritableConverter(): WritableConverter[Double] = + simpleWritableConverter[Double, DoubleWritable](_.get) + + implicit def floatWritableConverter(): WritableConverter[Float] = + simpleWritableConverter[Float, FloatWritable](_.get) + + implicit def booleanWritableConverter(): WritableConverter[Boolean] = + simpleWritableConverter[Boolean, BooleanWritable](_.get) + + implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = { + simpleWritableConverter[Array[Byte], BytesWritable](bw => + // getBytes method returns array which is longer then data to be returned + Arrays.copyOfRange(bw.getBytes, 0, bw.getLength) + ) + } + + implicit def stringWritableConverter(): WritableConverter[String] = + simpleWritableConverter[String, Text](_.toString) + + implicit def writableWritableConverter[T <: Writable]() = + new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T]) +} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index e37f3acaf6e30..7af3538262fd6 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -32,13 +32,13 @@ import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ -import org.apache.spark.SparkContext.rddToPairRDDFunctions import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.JavaUtils.mapAsSerializableJavaMap import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} +import org.apache.spark.rdd.RDD.rddToPairRDDFunctions import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 6a6d9bf6857d3..97f5c9f257e09 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -33,7 +33,7 @@ import org.apache.hadoop.mapred.{InputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark._ -import org.apache.spark.SparkContext._ +import org.apache.spark.AccumulatorParam._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index e4025bcf48db6..3add4a76192ca 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -21,6 +21,7 @@ import java.util.{Properties, Random} import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer +import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} import com.clearspring.analytics.stream.cardinality.HyperLogLogPlus @@ -28,6 +29,7 @@ import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable import org.apache.hadoop.io.Text +import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ @@ -1383,3 +1385,31 @@ abstract class RDD[T: ClassTag]( new JavaRDD(this)(elementClassTag) } } + +object RDD { + + // The following implicit functions were in SparkContext before 1.2 and users had to + // `import SparkContext._` to enable them. Now we move them here to make the compiler find + // them automatically. However, we still keep the old functions in SparkContext for backward + // compatibility and forward to the following functions directly. + + implicit def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { + new PairRDDFunctions(rdd) + } + + implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) + + implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( + rdd: RDD[(K, V)]) = + new SequenceFileRDDFunctions(rdd) + + implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( + rdd: RDD[(K, V)]) = + new OrderedRDDFunctions[K, V, (K, V)](rdd) + + implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = new DoubleRDDFunctions(rdd) + + implicit def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = + new DoubleRDDFunctions(rdd.map(x => num.toDouble(x))) +} diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index 9e454ddcc52a6..1362022104195 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -82,7 +82,7 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { bytesWritable.set(inputArray, 0, 10) bytesWritable.set(inputArray, 0, 5) - val converter = SparkContext.bytesWritableConverter() + val converter = WritableConverter.bytesWritableConverter() val byteArray = converter.convert(bytesWritable) assert(byteArray.length === 5) diff --git a/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala new file mode 100644 index 0000000000000..4918e2d92beb4 --- /dev/null +++ b/core/src/test/scala/org/apache/sparktest/ImplicitSuite.scala @@ -0,0 +1,128 @@ +/* + * 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.sparktest + +/** + * A test suite to make sure all `implicit` functions work correctly. + * Please don't `import org.apache.spark.SparkContext._` in this class. + * + * As `implicit` is a compiler feature, we don't need to run this class. + * What we need to do is making the compiler happy. + */ +class ImplicitSuite { + + // We only want to test if `implict` works well with the compiler, so we don't need a real + // SparkContext. + def mockSparkContext[T]: org.apache.spark.SparkContext = null + + // We only want to test if `implict` works well with the compiler, so we don't need a real RDD. + def mockRDD[T]: org.apache.spark.rdd.RDD[T] = null + + def testRddToPairRDDFunctions(): Unit = { + val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD + rdd.groupByKey() + } + + def testRddToAsyncRDDActions(): Unit = { + val rdd: org.apache.spark.rdd.RDD[Int] = mockRDD + rdd.countAsync() + } + + def testRddToSequenceFileRDDFunctions(): Unit = { + // TODO eliminating `import intToIntWritable` needs refactoring SequenceFileRDDFunctions. + // That will be a breaking change. + import org.apache.spark.SparkContext.intToIntWritable + val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD + rdd.saveAsSequenceFile("/a/test/path") + } + + def testRddToOrderedRDDFunctions(): Unit = { + val rdd: org.apache.spark.rdd.RDD[(Int, Int)] = mockRDD + rdd.sortByKey() + } + + def testDoubleRDDToDoubleRDDFunctions(): Unit = { + val rdd: org.apache.spark.rdd.RDD[Double] = mockRDD + rdd.stats() + } + + def testNumericRDDToDoubleRDDFunctions(): Unit = { + val rdd: org.apache.spark.rdd.RDD[Int] = mockRDD + rdd.stats() + } + + def testDoubleAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123.4) + } + + def testIntAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123) + } + + def testLongAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123L) + } + + def testFloatAccumulatorParam(): Unit = { + val sc = mockSparkContext + sc.accumulator(123F) + } + + def testIntWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Int, Int]("/a/test/path") + } + + def testLongWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Long, Long]("/a/test/path") + } + + def testDoubleWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Double, Double]("/a/test/path") + } + + def testFloatWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Float, Float]("/a/test/path") + } + + def testBooleanWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Boolean, Boolean]("/a/test/path") + } + + def testBytesWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[Array[Byte], Array[Byte]]("/a/test/path") + } + + def testStringWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[String, String]("/a/test/path") + } + + def testWritableWritableConverter(): Unit = { + val sc = mockSparkContext + sc.sequenceFile[org.apache.hadoop.io.Text, org.apache.hadoop.io.Text]("/a/test/path") + } +} From a81918c5a66fc6040f9796fc1a9d4e0bfb8d0cbe Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 21 Nov 2014 12:10:04 -0800 Subject: [PATCH 002/305] SPARK-4532: Fix bug in detection of Hive in Spark 1.2 Because the Hive profile is no longer defined in the root pom, we need to check specifically in the sql/hive pom when we perform the check in make-distribtion.sh. Author: Patrick Wendell Closes #3398 from pwendell/make-distribution and squashes the following commits: 8a58279 [Patrick Wendell] Fix bug in detection of Hive in Spark 1.2 --- make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index 2267b1aa08a6c..7c0fb8992a155 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -119,7 +119,7 @@ VERSION=$(mvn help:evaluate -Dexpression=project.version 2>/dev/null | grep -v " SPARK_HADOOP_VERSION=$(mvn help:evaluate -Dexpression=hadoop.version $@ 2>/dev/null\ | grep -v "INFO"\ | tail -n 1) -SPARK_HIVE=$(mvn help:evaluate -Dexpression=project.activeProfiles $@ 2>/dev/null\ +SPARK_HIVE=$(mvn help:evaluate -Dexpression=project.activeProfiles -pl sql/hive $@ 2>/dev/null\ | grep -v "INFO"\ | fgrep --count "hive";\ # Reset exit status to 0, otherwise the script stops here if the last grep finds nothing\ From ce95bd8e130b2c7688b94be40683bdd90d86012d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 21 Nov 2014 15:02:31 -0800 Subject: [PATCH 003/305] [SPARK-4531] [MLlib] cache serialized java object The Pyrolite is pretty slow (comparing to the adhoc serializer in 1.1), it cause much performance regression in 1.2, because we cache the serialized Python object in JVM, deserialize them into Java object in each step. This PR change to cache the deserialized JavaRDD instead of PythonRDD to avoid the deserialization of Pyrolite. It should have similar memory usage as before, but much faster. Author: Davies Liu Closes #3397 from davies/cache and squashes the following commits: 7f6e6ce [Davies Liu] Update -> Updater 4b52edd [Davies Liu] using named argument 63b984e [Davies Liu] fix 7da0332 [Davies Liu] add unpersist() dff33e1 [Davies Liu] address comments c2bdfc2 [Davies Liu] refactor d572f00 [Davies Liu] Merge branch 'master' into cache f1063e1 [Davies Liu] cache serialized java object --- .../mllib/api/python/PythonMLLibAPI.scala | 110 +++++++++--------- .../spark/mllib/clustering/KMeans.scala | 13 +-- .../GeneralizedLinearAlgorithm.scala | 13 +-- python/pyspark/mllib/clustering.py | 8 +- python/pyspark/mllib/common.py | 4 +- python/pyspark/mllib/recommendation.py | 4 +- python/pyspark/mllib/regression.py | 5 +- 7 files changed, 64 insertions(+), 93 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index b6f7618171224..f04df1c156898 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -74,10 +74,28 @@ class PythonMLLibAPI extends Serializable { learner: GeneralizedLinearAlgorithm[_ <: GeneralizedLinearModel], data: JavaRDD[LabeledPoint], initialWeights: Vector): JList[Object] = { - // Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD. - learner.disableUncachedWarning() - val model = learner.run(data.rdd, initialWeights) - List(model.weights, model.intercept).map(_.asInstanceOf[Object]).asJava + try { + val model = learner.run(data.rdd.persist(StorageLevel.MEMORY_AND_DISK), initialWeights) + List(model.weights, model.intercept).map(_.asInstanceOf[Object]).asJava + } finally { + data.rdd.unpersist(blocking = false) + } + } + + /** + * Return the Updater from string + */ + def getUpdaterFromString(regType: String): Updater = { + if (regType == "l2") { + new SquaredL2Updater + } else if (regType == "l1") { + new L1Updater + } else if (regType == null || regType == "none") { + new SimpleUpdater + } else { + throw new IllegalArgumentException("Invalid value for 'regType' parameter." + + " Can only be initialized using the following string values: ['l1', 'l2', None].") + } } /** @@ -99,16 +117,7 @@ class PythonMLLibAPI extends Serializable { .setRegParam(regParam) .setStepSize(stepSize) .setMiniBatchFraction(miniBatchFraction) - if (regType == "l2") { - lrAlg.optimizer.setUpdater(new SquaredL2Updater) - } else if (regType == "l1") { - lrAlg.optimizer.setUpdater(new L1Updater) - } else if (regType == null) { - lrAlg.optimizer.setUpdater(new SimpleUpdater) - } else { - throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." - + " Can only be initialized using the following string values: ['l1', 'l2', None].") - } + lrAlg.optimizer.setUpdater(getUpdaterFromString(regType)) trainRegressionModel( lrAlg, data, @@ -178,16 +187,7 @@ class PythonMLLibAPI extends Serializable { .setRegParam(regParam) .setStepSize(stepSize) .setMiniBatchFraction(miniBatchFraction) - if (regType == "l2") { - SVMAlg.optimizer.setUpdater(new SquaredL2Updater) - } else if (regType == "l1") { - SVMAlg.optimizer.setUpdater(new L1Updater) - } else if (regType == null) { - SVMAlg.optimizer.setUpdater(new SimpleUpdater) - } else { - throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." - + " Can only be initialized using the following string values: ['l1', 'l2', None].") - } + SVMAlg.optimizer.setUpdater(getUpdaterFromString(regType)) trainRegressionModel( SVMAlg, data, @@ -213,16 +213,7 @@ class PythonMLLibAPI extends Serializable { .setRegParam(regParam) .setStepSize(stepSize) .setMiniBatchFraction(miniBatchFraction) - if (regType == "l2") { - LogRegAlg.optimizer.setUpdater(new SquaredL2Updater) - } else if (regType == "l1") { - LogRegAlg.optimizer.setUpdater(new L1Updater) - } else if (regType == null) { - LogRegAlg.optimizer.setUpdater(new SimpleUpdater) - } else { - throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." - + " Can only be initialized using the following string values: ['l1', 'l2', None].") - } + LogRegAlg.optimizer.setUpdater(getUpdaterFromString(regType)) trainRegressionModel( LogRegAlg, data, @@ -248,16 +239,7 @@ class PythonMLLibAPI extends Serializable { .setRegParam(regParam) .setNumCorrections(corrections) .setConvergenceTol(tolerance) - if (regType == "l2") { - LogRegAlg.optimizer.setUpdater(new SquaredL2Updater) - } else if (regType == "l1") { - LogRegAlg.optimizer.setUpdater(new L1Updater) - } else if (regType == null) { - LogRegAlg.optimizer.setUpdater(new SimpleUpdater) - } else { - throw new java.lang.IllegalArgumentException("Invalid value for 'regType' parameter." - + " Can only be initialized using the following string values: ['l1', 'l2', None].") - } + LogRegAlg.optimizer.setUpdater(getUpdaterFromString(regType)) trainRegressionModel( LogRegAlg, data, @@ -289,9 +271,11 @@ class PythonMLLibAPI extends Serializable { .setMaxIterations(maxIterations) .setRuns(runs) .setInitializationMode(initializationMode) - // Disable the uncached input warning because 'data' is a deliberately uncached MappedRDD. - .disableUncachedWarning() - kMeansAlg.run(data.rdd) + try { + kMeansAlg.run(data.rdd.persist(StorageLevel.MEMORY_AND_DISK)) + } finally { + data.rdd.unpersist(blocking = false) + } } /** @@ -425,16 +409,18 @@ class PythonMLLibAPI extends Serializable { numPartitions: Int, numIterations: Int, seed: Long): Word2VecModelWrapper = { - val data = dataJRDD.rdd.persist(StorageLevel.MEMORY_AND_DISK_SER) val word2vec = new Word2Vec() .setVectorSize(vectorSize) .setLearningRate(learningRate) .setNumPartitions(numPartitions) .setNumIterations(numIterations) .setSeed(seed) - val model = word2vec.fit(data) - data.unpersist() - new Word2VecModelWrapper(model) + try { + val model = word2vec.fit(dataJRDD.rdd.persist(StorageLevel.MEMORY_AND_DISK_SER)) + new Word2VecModelWrapper(model) + } finally { + dataJRDD.rdd.unpersist(blocking = false) + } } private[python] class Word2VecModelWrapper(model: Word2VecModel) { @@ -495,8 +481,11 @@ class PythonMLLibAPI extends Serializable { categoricalFeaturesInfo = categoricalFeaturesInfo.asScala.toMap, minInstancesPerNode = minInstancesPerNode, minInfoGain = minInfoGain) - - DecisionTree.train(data.rdd, strategy) + try { + DecisionTree.train(data.rdd.persist(StorageLevel.MEMORY_AND_DISK), strategy) + } finally { + data.rdd.unpersist(blocking = false) + } } /** @@ -526,10 +515,15 @@ class PythonMLLibAPI extends Serializable { numClassesForClassification = numClasses, maxBins = maxBins, categoricalFeaturesInfo = categoricalFeaturesInfo.asScala.toMap) - if (algo == Algo.Classification) { - RandomForest.trainClassifier(data.rdd, strategy, numTrees, featureSubsetStrategy, seed) - } else { - RandomForest.trainRegressor(data.rdd, strategy, numTrees, featureSubsetStrategy, seed) + val cached = data.rdd.persist(StorageLevel.MEMORY_AND_DISK) + try { + if (algo == Algo.Classification) { + RandomForest.trainClassifier(cached, strategy, numTrees, featureSubsetStrategy, seed) + } else { + RandomForest.trainRegressor(cached, strategy, numTrees, featureSubsetStrategy, seed) + } + } finally { + cached.unpersist(blocking = false) } } @@ -711,7 +705,7 @@ private[spark] object SerDe extends Serializable { def pickle(obj: Object, out: OutputStream, pickler: Pickler): Unit = { if (obj == this) { out.write(Opcodes.GLOBAL) - out.write((module + "\n" + name + "\n").getBytes()) + out.write((module + "\n" + name + "\n").getBytes) } else { pickler.save(this) // it will be memorized by Pickler saveState(obj, out, pickler) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 7443f232ec3e7..34ea0de706f08 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -113,22 +113,13 @@ class KMeans private ( this } - /** Whether a warning should be logged if the input RDD is uncached. */ - private var warnOnUncachedInput = true - - /** Disable warnings about uncached input. */ - private[spark] def disableUncachedWarning(): this.type = { - warnOnUncachedInput = false - this - } - /** * Train a K-means model on the given set of points; `data` should be cached for high * performance, because this is an iterative algorithm. */ def run(data: RDD[Vector]): KMeansModel = { - if (warnOnUncachedInput && data.getStorageLevel == StorageLevel.NONE) { + if (data.getStorageLevel == StorageLevel.NONE) { logWarning("The input data is not directly cached, which may hurt performance if its" + " parent RDDs are also uncached.") } @@ -143,7 +134,7 @@ class KMeans private ( norms.unpersist() // Warn at the end of the run as well, for increased visibility. - if (warnOnUncachedInput && data.getStorageLevel == StorageLevel.NONE) { + if (data.getStorageLevel == StorageLevel.NONE) { logWarning("The input data was not directly cached, which may hurt performance if its" + " parent RDDs are also uncached.") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 00dfc86c9e0bd..0287f04e2c777 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -136,15 +136,6 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] this } - /** Whether a warning should be logged if the input RDD is uncached. */ - private var warnOnUncachedInput = true - - /** Disable warnings about uncached input. */ - private[spark] def disableUncachedWarning(): this.type = { - warnOnUncachedInput = false - this - } - /** * Run the algorithm with the configured parameters on an input * RDD of LabeledPoint entries. @@ -161,7 +152,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ def run(input: RDD[LabeledPoint], initialWeights: Vector): M = { - if (warnOnUncachedInput && input.getStorageLevel == StorageLevel.NONE) { + if (input.getStorageLevel == StorageLevel.NONE) { logWarning("The input data is not directly cached, which may hurt performance if its" + " parent RDDs are also uncached.") } @@ -241,7 +232,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] } // Warn at the end of the run as well, for increased visibility. - if (warnOnUncachedInput && input.getStorageLevel == StorageLevel.NONE) { + if (input.getStorageLevel == StorageLevel.NONE) { logWarning("The input data was not directly cached, which may hurt performance if its" + " parent RDDs are also uncached.") } diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index fe4c4cc5094d8..e2492eef5bd6a 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -16,7 +16,7 @@ # from pyspark import SparkContext -from pyspark.mllib.common import callMLlibFunc, callJavaFunc, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc, callJavaFunc from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['KMeansModel', 'KMeans'] @@ -80,10 +80,8 @@ class KMeans(object): @classmethod def train(cls, rdd, k, maxIterations=100, runs=1, initializationMode="k-means||"): """Train a k-means clustering model.""" - # cache serialized data to avoid objects over head in JVM - jcached = _to_java_object_rdd(rdd.map(_convert_to_vector), cache=True) - model = callMLlibFunc("trainKMeansModel", jcached, k, maxIterations, runs, - initializationMode) + model = callMLlibFunc("trainKMeansModel", rdd.map(_convert_to_vector), k, maxIterations, + runs, initializationMode) centers = callJavaFunc(rdd.context, model.clusterCenters) return KMeansModel([c.toArray() for c in centers]) diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index c6149fe391ec8..33c49e2399908 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -54,15 +54,13 @@ def _new_smart_decode(obj): # this will call the MLlib version of pythonToJava() -def _to_java_object_rdd(rdd, cache=False): +def _to_java_object_rdd(rdd): """ Return an JavaRDD of Object by unpickling It will convert each Python object into Java object by Pyrolite, whenever the RDD is serialized in batch or not. """ rdd = rdd._reserialize(AutoBatchedSerializer(PickleSerializer())) - if cache: - rdd.cache() return rdd.ctx._jvm.SerDe.pythonToJava(rdd._jrdd, True) diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index 2bcbf2aaf8e3e..97ec74eda0b71 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -19,7 +19,7 @@ from pyspark import SparkContext from pyspark.rdd import RDD -from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, _to_java_object_rdd +from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc __all__ = ['MatrixFactorizationModel', 'ALS', 'Rating'] @@ -110,7 +110,7 @@ def _prepare(cls, ratings): ratings = ratings.map(lambda x: Rating(*x)) else: raise ValueError("rating should be RDD of Rating or tuple/list") - return _to_java_object_rdd(ratings, True) + return ratings @classmethod def train(cls, ratings, rank, iterations=5, lambda_=0.01, blocks=-1, nonnegative=False, diff --git a/python/pyspark/mllib/regression.py b/python/pyspark/mllib/regression.py index f4f5e615fadc3..210060140fd91 100644 --- a/python/pyspark/mllib/regression.py +++ b/python/pyspark/mllib/regression.py @@ -18,7 +18,7 @@ import numpy as np from numpy import array -from pyspark.mllib.common import callMLlibFunc, _to_java_object_rdd +from pyspark.mllib.common import callMLlibFunc from pyspark.mllib.linalg import SparseVector, _convert_to_vector __all__ = ['LabeledPoint', 'LinearModel', 'LinearRegressionModel', 'RidgeRegressionModel', @@ -129,8 +129,7 @@ def _regression_train_wrapper(train_func, modelClass, data, initial_weights): if not isinstance(first, LabeledPoint): raise ValueError("data should be an RDD of LabeledPoint, but got %s" % first) initial_weights = initial_weights or [0.0] * len(data.first().features) - weights, intercept = train_func(_to_java_object_rdd(data, cache=True), - _convert_to_vector(initial_weights)) + weights, intercept = train_func(data, _convert_to_vector(initial_weights)) return modelClass(weights, intercept) From b5d17ef10e2509d9886c660945449a89750c8116 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Fri, 21 Nov 2014 18:15:07 -0800 Subject: [PATCH 004/305] [SPARK-4431][MLlib] Implement efficient foreachActive for dense and sparse vector Previously, we were using Breeze's activeIterator to access the non-zero elements in dense/sparse vector. Due to the overhead, we switched back to native `while loop` in #SPARK-4129. However, #SPARK-4129 requires de-reference the dv.values/sv.values in each access to the value, which is very expensive. Also, in MultivariateOnlineSummarizer, we're using Breeze's dense vector to store the partial stats, and this is very expensive compared with using primitive scala array. In this PR, efficient foreachActive is implemented to unify the code path for dense and sparse vector operation which makes codebase easier to maintain. Breeze dense vector is replaced by primitive array to reduce the overhead further. Benchmarking with mnist8m dataset on single JVM with first 200 samples loaded in memory, and repeating 5000 times. Before change: Sparse Vector - 30.02 Dense Vector - 38.27 With this PR: Sparse Vector - 6.29 Dense Vector - 11.72 Author: DB Tsai Closes #3288 from dbtsai/activeIterator and squashes the following commits: 844b0e6 [DB Tsai] formating 03dd693 [DB Tsai] futher performance tunning. 1907ae1 [DB Tsai] address feedback 98448bb [DB Tsai] Made the override final, and had a local copy of variables which made the accessing a single step operation. c0cbd5a [DB Tsai] fix a bug 6441f92 [DB Tsai] Finished SPARK-4431 --- .../apache/spark/mllib/linalg/Vectors.scala | 32 +++++ .../stat/MultivariateOnlineSummarizer.scala | 121 +++++++----------- .../spark/mllib/linalg/VectorsSuite.scala | 24 ++++ 3 files changed, 105 insertions(+), 72 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 60ab2aaa8f27a..c6d5fe5bc678c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -76,6 +76,15 @@ sealed trait Vector extends Serializable { def copy: Vector = { throw new NotImplementedError(s"copy is not implemented for ${this.getClass}.") } + + /** + * Applies a function `f` to all the active elements of dense and sparse vector. + * + * @param f the function takes two parameters where the first parameter is the index of + * the vector with type `Int`, and the second parameter is the corresponding value + * with type `Double`. + */ + private[spark] def foreachActive(f: (Int, Double) => Unit) } /** @@ -273,6 +282,17 @@ class DenseVector(val values: Array[Double]) extends Vector { override def copy: DenseVector = { new DenseVector(values.clone()) } + + private[spark] override def foreachActive(f: (Int, Double) => Unit) = { + var i = 0 + val localValuesSize = values.size + val localValues = values + + while (i < localValuesSize) { + f(i, localValues(i)) + i += 1 + } + } } /** @@ -309,4 +329,16 @@ class SparseVector( } private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) + + private[spark] override def foreachActive(f: (Int, Double) => Unit) = { + var i = 0 + val localValuesSize = values.size + val localIndices = indices + val localValues = values + + while (i < localValuesSize) { + f(localIndices(i), localValues(i)) + i += 1 + } + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala index 654479ac2dd4f..fcc2a148791bd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateOnlineSummarizer.scala @@ -17,10 +17,8 @@ package org.apache.spark.mllib.stat -import breeze.linalg.{DenseVector => BDV} - import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors, Vector} +import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: @@ -40,37 +38,14 @@ import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vectors, Vector class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with Serializable { private var n = 0 - private var currMean: BDV[Double] = _ - private var currM2n: BDV[Double] = _ - private var currM2: BDV[Double] = _ - private var currL1: BDV[Double] = _ + private var currMean: Array[Double] = _ + private var currM2n: Array[Double] = _ + private var currM2: Array[Double] = _ + private var currL1: Array[Double] = _ private var totalCnt: Long = 0 - private var nnz: BDV[Double] = _ - private var currMax: BDV[Double] = _ - private var currMin: BDV[Double] = _ - - /** - * Adds input value to position i. - */ - private[this] def add(i: Int, value: Double) = { - if (value != 0.0) { - if (currMax(i) < value) { - currMax(i) = value - } - if (currMin(i) > value) { - currMin(i) = value - } - - val prevMean = currMean(i) - val diff = value - prevMean - currMean(i) = prevMean + diff / (nnz(i) + 1.0) - currM2n(i) += (value - currMean(i)) * diff - currM2(i) += value * value - currL1(i) += math.abs(value) - - nnz(i) += 1.0 - } - } + private var nnz: Array[Double] = _ + private var currMax: Array[Double] = _ + private var currMin: Array[Double] = _ /** * Add a new sample to this summarizer, and update the statistical summary. @@ -83,33 +58,36 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S require(sample.size > 0, s"Vector should have dimension larger than zero.") n = sample.size - currMean = BDV.zeros[Double](n) - currM2n = BDV.zeros[Double](n) - currM2 = BDV.zeros[Double](n) - currL1 = BDV.zeros[Double](n) - nnz = BDV.zeros[Double](n) - currMax = BDV.fill(n)(Double.MinValue) - currMin = BDV.fill(n)(Double.MaxValue) + currMean = Array.ofDim[Double](n) + currM2n = Array.ofDim[Double](n) + currM2 = Array.ofDim[Double](n) + currL1 = Array.ofDim[Double](n) + nnz = Array.ofDim[Double](n) + currMax = Array.fill[Double](n)(Double.MinValue) + currMin = Array.fill[Double](n)(Double.MaxValue) } require(n == sample.size, s"Dimensions mismatch when adding new sample." + s" Expecting $n but got ${sample.size}.") - sample match { - case dv: DenseVector => { - var j = 0 - while (j < dv.size) { - add(j, dv.values(j)) - j += 1 + sample.foreachActive { (index, value) => + if (value != 0.0) { + if (currMax(index) < value) { + currMax(index) = value } - } - case sv: SparseVector => - var j = 0 - while (j < sv.indices.size) { - add(sv.indices(j), sv.values(j)) - j += 1 + if (currMin(index) > value) { + currMin(index) = value } - case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + + val prevMean = currMean(index) + val diff = value - prevMean + currMean(index) = prevMean + diff / (nnz(index) + 1.0) + currM2n(index) += (value - currMean(index)) * diff + currM2(index) += value * value + currL1(index) += math.abs(value) + + nnz(index) += 1.0 + } } totalCnt += 1 @@ -152,14 +130,14 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S } } else if (totalCnt == 0 && other.totalCnt != 0) { this.n = other.n - this.currMean = other.currMean.copy - this.currM2n = other.currM2n.copy - this.currM2 = other.currM2.copy - this.currL1 = other.currL1.copy + this.currMean = other.currMean.clone + this.currM2n = other.currM2n.clone + this.currM2 = other.currM2.clone + this.currL1 = other.currL1.clone this.totalCnt = other.totalCnt - this.nnz = other.nnz.copy - this.currMax = other.currMax.copy - this.currMin = other.currMin.copy + this.nnz = other.nnz.clone + this.currMax = other.currMax.clone + this.currMin = other.currMin.clone } this } @@ -167,19 +145,19 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S override def mean: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") - val realMean = BDV.zeros[Double](n) + val realMean = Array.ofDim[Double](n) var i = 0 while (i < n) { realMean(i) = currMean(i) * (nnz(i) / totalCnt) i += 1 } - Vectors.fromBreeze(realMean) + Vectors.dense(realMean) } override def variance: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") - val realVariance = BDV.zeros[Double](n) + val realVariance = Array.ofDim[Double](n) val denominator = totalCnt - 1.0 @@ -194,8 +172,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S i += 1 } } - - Vectors.fromBreeze(realVariance) + Vectors.dense(realVariance) } override def count: Long = totalCnt @@ -203,7 +180,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S override def numNonzeros: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") - Vectors.fromBreeze(nnz) + Vectors.dense(nnz) } override def max: Vector = { @@ -214,7 +191,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S if ((nnz(i) < totalCnt) && (currMax(i) < 0.0)) currMax(i) = 0.0 i += 1 } - Vectors.fromBreeze(currMax) + Vectors.dense(currMax) } override def min: Vector = { @@ -225,25 +202,25 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S if ((nnz(i) < totalCnt) && (currMin(i) > 0.0)) currMin(i) = 0.0 i += 1 } - Vectors.fromBreeze(currMin) + Vectors.dense(currMin) } override def normL2: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") - val realMagnitude = BDV.zeros[Double](n) + val realMagnitude = Array.ofDim[Double](n) var i = 0 while (i < currM2.size) { realMagnitude(i) = math.sqrt(currM2(i)) i += 1 } - - Vectors.fromBreeze(realMagnitude) + Vectors.dense(realMagnitude) } override def normL1: Vector = { require(totalCnt > 0, s"Nothing has been added to this summarizer.") - Vectors.fromBreeze(currL1) + + Vectors.dense(currL1) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 59cd85eab27d0..9492f604af4d5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -173,4 +173,28 @@ class VectorsSuite extends FunSuite { val v = Vectors.fromBreeze(x(::, 0)) assert(v.size === x.rows) } + + test("foreachActive") { + val dv = Vectors.dense(0.0, 1.2, 3.1, 0.0) + val sv = Vectors.sparse(4, Seq((1, 1.2), (2, 3.1), (3, 0.0))) + + val dvMap = scala.collection.mutable.Map[Int, Double]() + dv.foreachActive { (index, value) => + dvMap.put(index, value) + } + assert(dvMap.size === 4) + assert(dvMap.get(0) === Some(0.0)) + assert(dvMap.get(1) === Some(1.2)) + assert(dvMap.get(2) === Some(3.1)) + assert(dvMap.get(3) === Some(0.0)) + + val svMap = scala.collection.mutable.Map[Int, Double]() + sv.foreachActive { (index, value) => + svMap.put(index, value) + } + assert(svMap.size === 3) + assert(svMap.get(1) === Some(1.2)) + assert(svMap.get(2) === Some(3.1)) + assert(svMap.get(3) === Some(0.0)) + } } From 9b2a3c6126e4fe8485e506f8a56a26cb72509a5f Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 22 Nov 2014 14:05:38 -0800 Subject: [PATCH 005/305] [SPARK-4377] Fixed serialization issue by switching to akka provided serializer. ... - there is no way around this for deserializing actorRef(s). Author: Prashant Sharma Closes #3402 from ScrapCodes/SPARK-4377/troubleDeserializing and squashes the following commits: 77233fd [Prashant Sharma] Style fixes 9b35c6e [Prashant Sharma] Scalastyle fixes 29880da [Prashant Sharma] [SPARK-4377] Fixed serialization issue by switching to akka provided serializer - there is no way around this for deserializing actorRef(s). --- .../master/FileSystemPersistenceEngine.scala | 26 ++++++++++--------- .../apache/spark/deploy/master/Master.scala | 12 ++++++--- .../deploy/master/RecoveryModeFactory.scala | 17 ++++++------ .../master/ZooKeeperPersistenceEngine.scala | 22 ++++++++-------- 4 files changed, 42 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala index 6ff2aa5244847..36a2e2c6a6349 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/FileSystemPersistenceEngine.scala @@ -18,12 +18,13 @@ package org.apache.spark.deploy.master import java.io._ -import java.nio.ByteBuffer + +import scala.reflect.ClassTag + +import akka.serialization.Serialization import org.apache.spark.Logging -import org.apache.spark.serializer.Serializer -import scala.reflect.ClassTag /** * Stores data in a single on-disk directory with one file per application and worker. @@ -34,10 +35,9 @@ import scala.reflect.ClassTag */ private[spark] class FileSystemPersistenceEngine( val dir: String, - val serialization: Serializer) + val serialization: Serialization) extends PersistenceEngine with Logging { - val serializer = serialization.newInstance() new File(dir).mkdir() override def persist(name: String, obj: Object): Unit = { @@ -56,17 +56,17 @@ private[spark] class FileSystemPersistenceEngine( private def serializeIntoFile(file: File, value: AnyRef) { val created = file.createNewFile() if (!created) { throw new IllegalStateException("Could not create file: " + file) } - - val out = serializer.serializeStream(new FileOutputStream(file)) + val serializer = serialization.findSerializerFor(value) + val serialized = serializer.toBinary(value) + val out = new FileOutputStream(file) try { - out.writeObject(value) + out.write(serialized) } finally { out.close() } - } - def deserializeFromFile[T](file: File): T = { + private def deserializeFromFile[T](file: File)(implicit m: ClassTag[T]): T = { val fileData = new Array[Byte](file.length().asInstanceOf[Int]) val dis = new DataInputStream(new FileInputStream(file)) try { @@ -74,7 +74,9 @@ private[spark] class FileSystemPersistenceEngine( } finally { dis.close() } - - serializer.deserializeStream(dis).readObject() + val clazz = m.runtimeClass.asInstanceOf[Class[T]] + val serializer = serialization.serializerFor(clazz) + serializer.fromBinary(fileData).asInstanceOf[T] } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 021454e25804c..7b32c505def9b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -30,6 +30,7 @@ import scala.util.Random import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} +import akka.serialization.Serialization import akka.serialization.SerializationExtension import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} @@ -132,15 +133,18 @@ private[spark] class Master( val (persistenceEngine_, leaderElectionAgent_) = RECOVERY_MODE match { case "ZOOKEEPER" => logInfo("Persisting recovery state to ZooKeeper") - val zkFactory = new ZooKeeperRecoveryModeFactory(conf) + val zkFactory = + new ZooKeeperRecoveryModeFactory(conf, SerializationExtension(context.system)) (zkFactory.createPersistenceEngine(), zkFactory.createLeaderElectionAgent(this)) case "FILESYSTEM" => - val fsFactory = new FileSystemRecoveryModeFactory(conf) + val fsFactory = + new FileSystemRecoveryModeFactory(conf, SerializationExtension(context.system)) (fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this)) case "CUSTOM" => val clazz = Class.forName(conf.get("spark.deploy.recoveryMode.factory")) - val factory = clazz.getConstructor(conf.getClass) - .newInstance(conf).asInstanceOf[StandaloneRecoveryModeFactory] + val factory = clazz.getConstructor(conf.getClass, Serialization.getClass) + .newInstance(conf, SerializationExtension(context.system)) + .asInstanceOf[StandaloneRecoveryModeFactory] (factory.createPersistenceEngine(), factory.createLeaderElectionAgent(this)) case _ => (new BlackHolePersistenceEngine(), new MonarchyLeaderAgent(this)) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index d9d36c1ed5f9f..1096eb0368357 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -17,9 +17,10 @@ package org.apache.spark.deploy.master +import akka.serialization.Serialization + import org.apache.spark.{Logging, SparkConf} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.serializer.JavaSerializer /** * ::DeveloperApi:: @@ -29,7 +30,7 @@ import org.apache.spark.serializer.JavaSerializer * */ @DeveloperApi -abstract class StandaloneRecoveryModeFactory(conf: SparkConf) { +abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serialization) { /** * PersistenceEngine defines how the persistent data(Information about worker, driver etc..) @@ -48,21 +49,21 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf) { * LeaderAgent in this case is a no-op. Since leader is forever leader as the actual * recovery is made by restoring from filesystem. */ -private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf) - extends StandaloneRecoveryModeFactory(conf) with Logging { +private[spark] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serialization) + extends StandaloneRecoveryModeFactory(conf, serializer) with Logging { val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") def createPersistenceEngine() = { logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) - new FileSystemPersistenceEngine(RECOVERY_DIR, new JavaSerializer(conf)) + new FileSystemPersistenceEngine(RECOVERY_DIR, serializer) } def createLeaderElectionAgent(master: LeaderElectable) = new MonarchyLeaderAgent(master) } -private[spark] class ZooKeeperRecoveryModeFactory(conf: SparkConf) - extends StandaloneRecoveryModeFactory(conf) { - def createPersistenceEngine() = new ZooKeeperPersistenceEngine(new JavaSerializer(conf), conf) +private[spark] class ZooKeeperRecoveryModeFactory(conf: SparkConf, serializer: Serialization) + extends StandaloneRecoveryModeFactory(conf, serializer) { + def createPersistenceEngine() = new ZooKeeperPersistenceEngine(conf, serializer) def createLeaderElectionAgent(master: LeaderElectable) = new ZooKeeperLeaderElectionAgent(master, conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 96c2139eb02f0..e11ac031fb9c6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -17,27 +17,24 @@ package org.apache.spark.deploy.master +import akka.serialization.Serialization + import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.serializer.Serializer -import java.nio.ByteBuffer -import scala.reflect.ClassTag - -private[spark] class ZooKeeperPersistenceEngine(val serialization: Serializer, conf: SparkConf) +private[spark] class ZooKeeperPersistenceEngine(conf: SparkConf, val serialization: Serialization) extends PersistenceEngine with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) - val serializer = serialization.newInstance() - SparkCuratorUtil.mkdir(zk, WORKING_DIR) @@ -59,14 +56,17 @@ private[spark] class ZooKeeperPersistenceEngine(val serialization: Serializer, c } private def serializeIntoFile(path: String, value: AnyRef) { - val serialized = serializer.serialize(value) - zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized.array()) + val serializer = serialization.findSerializerFor(value) + val serialized = serializer.toBinary(value) + zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized) } - def deserializeFromFile[T](filename: String): Option[T] = { + def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) + val clazz = m.runtimeClass.asInstanceOf[Class[T]] + val serializer = serialization.serializerFor(clazz) try { - Some(serializer.deserialize(ByteBuffer.wrap(fileData))) + Some(serializer.fromBinary(fileData).asInstanceOf[T]) } catch { case e: Exception => { logWarning("Exception while reading persisted file, deleting", e) From 29372b63185a4a170178b6ec2362d7112f389852 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 24 Nov 2014 13:28:48 -0600 Subject: [PATCH 006/305] SPARK-4457. Document how to build for Hadoop versions greater than 2.4 Author: Sandy Ryza Closes #3322 from sryza/sandy-spark-4457 and squashes the following commits: 5e72b77 [Sandy Ryza] Feedback 0cf05c1 [Sandy Ryza] Caveat be8084b [Sandy Ryza] SPARK-4457. Document how to build for Hadoop versions greater than 2.4 --- docs/building-spark.md | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index bb18414092aae..fee6a8440634c 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -92,8 +92,11 @@ mvn -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package # Apache Hadoop 2.3.X mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -DskipTests clean package -# Apache Hadoop 2.4.X -mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -DskipTests clean package +# Apache Hadoop 2.4.X or 2.5.X +mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=VERSION -DskipTests clean package + +Versions of Hadoop after 2.5.X may or may not work with the -Phadoop-2.4 profile (they were +released after this version of Spark). # Different versions of HDFS and YARN. mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package From a6d7b61f92dc7c1f9632cecb232afa8040ab2b4d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 24 Nov 2014 12:43:45 -0800 Subject: [PATCH 007/305] [SPARK-4479][SQL] Avoids unnecessary defensive copies when sort based shuffle is on This PR is a workaround for SPARK-4479. Two changes are introduced: when merge sort is bypassed in `ExternalSorter`, 1. also bypass RDD elements buffering as buffering is the reason that `MutableRow` backed row objects must be copied, and 2. avoids defensive copies in `Exchange` operator [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3422) Author: Cheng Lian Closes #3422 from liancheng/avoids-defensive-copies and squashes the following commits: 591f2e9 [Cheng Lian] Passes all shuffle suites 0c3c91e [Cheng Lian] Fixes shuffle write metrics when merge sort is bypassed ed5df3c [Cheng Lian] Fixes styling changes f75089b [Cheng Lian] Avoids unnecessary defensive copies when sort based shuffle is on --- .../util/collection/ExternalSorter.scala | 23 ++++++++++++++++--- .../scala/org/apache/spark/ShuffleSuite.scala | 12 +++++----- .../apache/spark/sql/execution/Exchange.scala | 16 ++++++++++++- 3 files changed, 41 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index c617ff5c51d04..15bda1c9cc29c 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -205,6 +205,13 @@ private[spark] class ExternalSorter[K, V, C]( map.changeValue((getPartition(kv._1), kv._1), update) maybeSpillCollection(usingMap = true) } + } else if (bypassMergeSort) { + // SPARK-4479: Also bypass buffering if merge sort is bypassed to avoid defensive copies + if (records.hasNext) { + spillToPartitionFiles(records.map { kv => + ((getPartition(kv._1), kv._1), kv._2.asInstanceOf[C]) + }) + } } else { // Stick values into our buffer while (records.hasNext) { @@ -336,6 +343,10 @@ private[spark] class ExternalSorter[K, V, C]( * @param collection whichever collection we're using (map or buffer) */ private def spillToPartitionFiles(collection: SizeTrackingPairCollection[(Int, K), C]): Unit = { + spillToPartitionFiles(collection.iterator) + } + + private def spillToPartitionFiles(iterator: Iterator[((Int, K), C)]): Unit = { assert(bypassMergeSort) // Create our file writers if we haven't done so yet @@ -350,9 +361,9 @@ private[spark] class ExternalSorter[K, V, C]( } } - val it = collection.iterator // No need to sort stuff, just write each element out - while (it.hasNext) { - val elem = it.next() + // No need to sort stuff, just write each element out + while (iterator.hasNext) { + val elem = iterator.next() val partitionId = elem._1._1 val key = elem._1._2 val value = elem._2 @@ -748,6 +759,12 @@ private[spark] class ExternalSorter[K, V, C]( context.taskMetrics.memoryBytesSpilled += memoryBytesSpilled context.taskMetrics.diskBytesSpilled += diskBytesSpilled + context.taskMetrics.shuffleWriteMetrics.filter(_ => bypassMergeSort).foreach { m => + if (curWriteMetrics != null) { + m.shuffleBytesWritten += curWriteMetrics.shuffleBytesWritten + m.shuffleWriteTime += curWriteMetrics.shuffleWriteTime + } + } lengths } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index cda942e15a704..85e5f9ab444b3 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -95,14 +95,14 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - // 10 partitions from 4 keys - val NUM_BLOCKS = 10 + // 201 partitions (greater than "spark.shuffle.sort.bypassMergeThreshold") from 4 keys + val NUM_BLOCKS = 201 val a = sc.parallelize(1 to 4, NUM_BLOCKS) val b = a.map(x => (x, x*2)) // NOTE: The default Java serializer doesn't create zero-sized blocks. // So, use Kryo - val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(NUM_BLOCKS)) .setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId @@ -122,13 +122,13 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex // Use a local cluster with 2 processes to make sure there are both local and remote blocks sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - // 10 partitions from 4 keys - val NUM_BLOCKS = 10 + // 201 partitions (greater than "spark.shuffle.sort.bypassMergeThreshold") from 4 keys + val NUM_BLOCKS = 201 val a = sc.parallelize(1 to 4, NUM_BLOCKS) val b = a.map(x => (x, x*2)) // NOTE: The default Java serializer should create zero-sized blocks - val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(10)) + val c = new ShuffledRDD[Int, Int, Int](b, new HashPartitioner(NUM_BLOCKS)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleId assert(c.count === 4) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index cff7a012691dc..d7c811ca89022 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -41,11 +41,21 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una /** We must copy rows when sort based shuffle is on */ protected def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] + private val bypassMergeThreshold = + child.sqlContext.sparkContext.conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + override def execute() = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. - val rdd = if (sortBasedShuffleOn) { + // This is a workaround for SPARK-4479. When: + // 1. sort based shuffle is on, and + // 2. the partition number is under the merge threshold, and + // 3. no ordering is required + // we can avoid the defensive copies to improve performance. In the long run, we probably + // want to include information in shuffle dependencies to indicate whether elements in the + // source RDD should be copied. + val rdd = if (sortBasedShuffleOn && numPartitions > bypassMergeThreshold) { child.execute().mapPartitions { iter => val hashExpressions = newMutableProjection(expressions, child.output)() iter.map(r => (hashExpressions(r).copy(), r.copy())) @@ -82,6 +92,10 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una shuffled.map(_._1) case SinglePartition => + // SPARK-4479: Can't turn off defensive copy as what we do for `HashPartitioning`, since + // operators like `TakeOrdered` may require an ordering within the partition, and currently + // `SinglePartition` doesn't include ordering information. + // TODO Add `SingleOrderedPartition` for operators like `TakeOrdered` val rdd = if (sortBasedShuffleOn) { child.execute().mapPartitions { iter => iter.map(r => (null, r.copy())) } } else { From d5834f0732b586731034a7df5402c25454770fc5 Mon Sep 17 00:00:00 2001 From: Daniel Darabos Date: Mon, 24 Nov 2014 12:45:07 -0800 Subject: [PATCH 008/305] [SQL] Fix comment in HiveShim This file is for Hive 0.13.1 I think. Author: Daniel Darabos Closes #3432 from darabos/patch-2 and squashes the following commits: 4fd22ed [Daniel Darabos] Fix comment. This file is for Hive 0.13.1. --- .../scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index 3c7f62af450d9..99c1987158581 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -43,7 +43,7 @@ import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} import org.apache.spark.sql.{SchemaRDD, Row => SparkRow} /** - * A compatibility layer for interacting with Hive version 0.12.0. + * A compatibility layer for interacting with Hive version 0.13.1. */ private[thriftserver] object HiveThriftServerShim { val version = "0.13.1" From b384119304617459592b7ba435368dd6fcc3273e Mon Sep 17 00:00:00 2001 From: scwf Date: Mon, 24 Nov 2014 12:49:08 -0800 Subject: [PATCH 009/305] [SQL] Fix path in HiveFromSpark It require us to run ```HiveFromSpark``` in specified dir because ```HiveFromSpark``` use relative path, this leads to ```run-example``` error(http://apache-spark-developers-list.1001551.n3.nabble.com/src-main-resources-kv1-txt-not-found-in-example-of-HiveFromSpark-td9100.html). Author: scwf Closes #3415 from scwf/HiveFromSpark and squashes the following commits: ed3d6c9 [scwf] revert no need change b00e20c [scwf] fix path usring spark_home dbd321b [scwf] fix path in hivefromspark --- .../org/apache/spark/examples/sql/hive/HiveFromSpark.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 0c52ef8ed96ac..227acc117502d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -27,6 +27,7 @@ object HiveFromSpark { def main(args: Array[String]) { val sparkConf = new SparkConf().setAppName("HiveFromSpark") val sc = new SparkContext(sparkConf) + val path = s"${System.getenv("SPARK_HOME")}/examples/src/main/resources/kv1.txt" // A local hive context creates an instance of the Hive Metastore in process, storing // the warehouse data in the current directory. This location can be overridden by @@ -35,7 +36,7 @@ object HiveFromSpark { import hiveContext._ sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - sql("LOAD DATA LOCAL INPATH 'src/main/resources/kv1.txt' INTO TABLE src") + sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE src") // Queries are expressed in HiveQL println("Result of 'SELECT *': ") From dd1c9cb36cde8202cede8014b5641ae8a0197812 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 24 Nov 2014 12:54:37 -0800 Subject: [PATCH 010/305] [SPARK-4487][SQL] Fix attribute reference resolution error when using ORDER BY. When we use ORDER BY clause, at first, attributes referenced by projection are resolved (1). And then, attributes referenced at ORDER BY clause are resolved (2). But when resolving attributes referenced at ORDER BY clause, the resolution result generated in (1) is discarded so for example, following query fails. SELECT c1 + c2 FROM mytable ORDER BY c1; The query above fails because when resolving the attribute reference 'c1', the resolution result of 'c2' is discarded. Author: Kousuke Saruta Closes #3363 from sarutak/SPARK-4487 and squashes the following commits: fd314f3 [Kousuke Saruta] Fixed attribute resolution logic in Analyzer 6e60c20 [Kousuke Saruta] Fixed conflicts cb5b7e9 [Kousuke Saruta] Added test case for SPARK-4487 282d529 [Kousuke Saruta] Fixed attributes reference resolution error b6123e6 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into concat-feature 317b7fb [Kousuke Saruta] WIP --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index d3b4cf8e34242..facbd8b975f10 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 @@ -179,7 +179,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool val missingInProject = requiredAttributes -- p.output if (missingInProject.nonEmpty) { // Add missing attributes and then project them away after the sort. - Project(projectList, + Project(projectList.map(_.toAttribute), Sort(ordering, Project(projectList ++ missingInProject, child))) } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 0a96831c76f57..84ee3051eb682 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -974,6 +974,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { dropTempTable("data") } + test("SPARK-4432 Fix attribute reference resolution error when using ORDER BY") { + checkAnswer( + sql("SELECT a + b FROM testData2 ORDER BY a"), + Seq(2, 3, 3 ,4 ,4 ,5).map(Seq(_)) + ) + } + test("Supporting relational operator '<=>' in Spark SQL") { val nullCheckData1 = TestData(1,"1") :: TestData(2,null) :: Nil val rdd1 = sparkContext.parallelize((0 to 1).map(i => nullCheckData1(i))) From 4a90276ab22d6989dffb2ee2d8118d9253365646 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Mon, 24 Nov 2014 13:18:14 -0800 Subject: [PATCH 011/305] [SPARK-4145] Web UI job pages MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR adds two new pages to the Spark Web UI: - A jobs overview page, which shows details on running / completed / failed jobs. - A job details page, which displays information on an individual job's stages. The jobs overview page is now the default UI homepage; the old homepage is still accessible at `/stages`. ### Screenshots #### New UI homepage ![image](https://cloud.githubusercontent.com/assets/50748/5119035/fd0a69e6-701f-11e4-89cb-db7e9705714f.png) #### Job details page (This is effectively a per-job version of the stages page that can be extended later with other things, such as DAG visualizations) ![image](https://cloud.githubusercontent.com/assets/50748/5134910/50b340d4-70c7-11e4-88e1-6b73237ea7c8.png) ### Key changes in this PR - Rename `JobProgressPage` to `AllStagesPage` - Expose `StageInfo` objects in the ``SparkListenerJobStart` event; add backwards-compatibility tests to JsonProtocol. - Add additional data structures to `JobProgressListener` to map from stages to jobs. - Add several fields to `JobUIData`. I also added ~150 lines of Selenium tests as I uncovered UI issues while developing this patch. ### Limitations If a job contains stages that aren't run, then its overall job progress bar may be an underestimate of the total job progress; in other words, a completed job may appear to have a progress bar that's not at 100%. If stages or tasks fail, then the progress bar will not go backwards to reflect the true amount of remaining work. Author: Josh Rosen Closes #3009 from JoshRosen/job-page and squashes the following commits: eb05e90 [Josh Rosen] Disable kill button in completed stages tables. f00c851 [Josh Rosen] Fix JsonProtocol compatibility b89c258 [Josh Rosen] More JSON protocol backwards-compatibility fixes. ff804cd [Josh Rosen] Don't write "Stage Ids" field in JobStartEvent JSON. 6f17f3f [Josh Rosen] Only store StageInfos in SparkListenerJobStart event. 2bbf41a [Josh Rosen] Update job progress bar to reflect skipped tasks/stages. 61c265a [Josh Rosen] Add “skipped stages” table; only display non-empty tables. 1f45d44 [Josh Rosen] Incorporate a bunch of minor review feedback. 0b77e3e [Josh Rosen] More bug fixes for phantom stages. 034aa8d [Josh Rosen] Use `.max()` to find result stage for job. eebdc2c [Josh Rosen] Don’t display pending stages for completed jobs. 67080ba [Josh Rosen] Ensure that "phantom stages" don't cause memory leaks. 7d10b97 [Josh Rosen] Merge remote-tracking branch 'apache/master' into job-page d69c775 [Josh Rosen] Fix table sorting on all jobs page. 5eb39dc [Josh Rosen] Add pending stages table to job page. f2a15da [Josh Rosen] Add status field to job details page. 171b53c [Josh Rosen] Move `startTime` to the start of SparkContext. e2f2c43 [Josh Rosen] Fix sorting of stages in job details page. 8955f4c [Josh Rosen] Display information for pending stages on jobs page. 8ab6c28 [Josh Rosen] Compute numTasks from job start stage infos. 5884f91 [Josh Rosen] Add StageInfos to SparkListenerJobStart event. 79793cd [Josh Rosen] Track indices of completed stage to avoid overcounting when failures occur. d62ea7b [Josh Rosen] Add failing Selenium test for stage overcounting issue. 1145c60 [Josh Rosen] Display text instead of progress bar for stages. 3d0a007 [Josh Rosen] Merge remote-tracking branch 'origin/master' into job-page 8a2351b [Josh Rosen] Add help tooltip to Spark Jobs page. b7bf30e [Josh Rosen] Add stages progress bar; fix bug where active stages show as completed. 4846ce4 [Josh Rosen] Hide "(Job Group") if no jobs were submitted in job groups. 4d58e55 [Josh Rosen] Change label to "Tasks (for all stages)" 85e9c85 [Josh Rosen] Extract startTime into separate variable. 1cf4987 [Josh Rosen] Fix broken kill links; add Selenium test to avoid future regressions. 56701fa [Josh Rosen] Move last stage name / description logic out of markup. a475ea1 [Josh Rosen] Add progress bars to jobs page. 45343b8 [Josh Rosen] More comments 4b206fb [Josh Rosen] Merge remote-tracking branch 'origin/master' into job-page bfce2b9 [Josh Rosen] Address review comments, except for progress bar. 4487dcb [Josh Rosen] [SPARK-4145] Web UI job pages 2568a6c [Josh Rosen] Rename JobProgressPage to AllStagesPage: --- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../apache/spark/scheduler/DAGScheduler.scala | 7 +- .../spark/scheduler/SparkListener.scala | 11 +- .../scala/org/apache/spark/ui/SparkUI.scala | 13 +- .../scala/org/apache/spark/ui/UIUtils.scala | 27 +- .../apache/spark/ui/jobs/AllJobsPage.scala | 151 ++++++++++ ...ProgressPage.scala => AllStagesPage.scala} | 13 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 2 +- .../org/apache/spark/ui/jobs/JobPage.scala | 177 +++++++++++ .../spark/ui/jobs/JobProgressListener.scala | 99 ++++++- .../org/apache/spark/ui/jobs/JobsTab.scala | 32 ++ .../org/apache/spark/ui/jobs/PoolPage.scala | 7 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 2 +- .../org/apache/spark/ui/jobs/StageTable.scala | 43 +-- .../{JobProgressTab.scala => StagesTab.scala} | 10 +- .../org/apache/spark/ui/jobs/UIData.scala | 21 +- .../org/apache/spark/util/JsonProtocol.scala | 23 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 201 ++++++++++++- .../ui/jobs/JobProgressListenerSuite.scala | 8 +- .../apache/spark/util/JsonProtocolSuite.scala | 276 +++++++++++++++++- 21 files changed, 1054 insertions(+), 75 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala rename core/src/main/scala/org/apache/spark/ui/jobs/{JobProgressPage.scala => AllStagesPage.scala} (87%) create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala create mode 100644 core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala rename core/src/main/scala/org/apache/spark/ui/jobs/{JobProgressTab.scala => StagesTab.scala} (83%) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 586c1ccaca72b..9b0d5be7a7ab2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -83,6 +83,8 @@ class SparkContext(config: SparkConf) extends Logging { // contains a map from hostname to a list of input format splits on the host. private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() + val startTime = System.currentTimeMillis() + /** * Create a SparkContext that loads settings from system properties (for instance, when * launching with ./bin/spark-submit). @@ -269,8 +271,6 @@ class SparkContext(config: SparkConf) extends Logging { /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) - val startTime = System.currentTimeMillis() - // Add each JAR given through the constructor if (jars != null) { jars.foreach(addJar) 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 22449517d100f..b1222af662e9b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -751,14 +751,15 @@ class DAGScheduler( localExecutionEnabled && allowLocal && finalStage.parents.isEmpty && partitions.length == 1 if (shouldRunLocally) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, Seq.empty, properties)) runLocally(job) } else { jobIdToActiveJob(jobId) = job activeJobs += job finalStage.resultOfJob = Some(job) - listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, - properties)) + val stageIds = jobIdToStageIds(jobId).toArray + val stageInfos = stageIds.flatMap(id => stageIdToStage.get(id).map(_.latestInfo)) + listenerBus.post(SparkListenerJobStart(job.jobId, stageInfos, properties)) submitStage(finalStage) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 86afe3bd5265f..b62b0c1312693 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -56,8 +56,15 @@ case class SparkListenerTaskEnd( extends SparkListenerEvent @DeveloperApi -case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) - extends SparkListenerEvent +case class SparkListenerJobStart( + jobId: Int, + stageInfos: Seq[StageInfo], + properties: Properties = null) + extends SparkListenerEvent { + // Note: this is here for backwards-compatibility with older versions of this event which + // only stored stageIds and not StageInfos: + val stageIds: Seq[Int] = stageInfos.map(_.stageId) +} @DeveloperApi case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 049938f827291..176907dffa46a 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -23,7 +23,7 @@ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.{EnvironmentListener, EnvironmentTab} import org.apache.spark.ui.exec.{ExecutorsListener, ExecutorsTab} -import org.apache.spark.ui.jobs.{JobProgressListener, JobProgressTab} +import org.apache.spark.ui.jobs.{JobsTab, JobProgressListener, StagesTab} import org.apache.spark.ui.storage.{StorageListener, StorageTab} /** @@ -43,17 +43,20 @@ private[spark] class SparkUI private ( extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath, "SparkUI") with Logging { + val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + /** Initialize all components of the server. */ def initialize() { - val jobProgressTab = new JobProgressTab(this) - attachTab(jobProgressTab) + attachTab(new JobsTab(this)) + val stagesTab = new StagesTab(this) + attachTab(stagesTab) attachTab(new StorageTab(this)) attachTab(new EnvironmentTab(this)) attachTab(new ExecutorsTab(this)) attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) - attachHandler(createRedirectHandler("/", "/stages", basePath = basePath)) + attachHandler(createRedirectHandler("/", "/jobs", basePath = basePath)) attachHandler( - createRedirectHandler("/stages/stage/kill", "/stages", jobProgressTab.handleKillRequest)) + createRedirectHandler("/stages/stage/kill", "/stages", stagesTab.handleKillRequest)) // If the UI is live, then serve sc.foreach { _.env.metricsSystem.getServletHandlers.foreach(attachHandler) } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 7bc1e24d58711..0c418beaf7581 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -169,7 +169,8 @@ private[spark] object UIUtils extends Logging { title: String, content: => Seq[Node], activeTab: SparkUITab, - refreshInterval: Option[Int] = None): Seq[Node] = { + refreshInterval: Option[Int] = None, + helpText: Option[String] = None): Seq[Node] = { val appName = activeTab.appName val shortAppName = if (appName.length < 36) appName else appName.take(32) + "..." @@ -178,6 +179,9 @@ private[spark] object UIUtils extends Logging { {tab.name} } + val helpButton: Seq[Node] = helpText.map { helpText => + (?) + }.getOrElse(Seq.empty) @@ -201,6 +205,7 @@ private[spark] object UIUtils extends Logging {

{title} + {helpButton}

@@ -283,4 +288,24 @@ private[spark] object UIUtils extends Logging { } + + def makeProgressBar( + started: Int, + completed: Int, + failed: Int, + skipped:Int, + total: Int): Seq[Node] = { + val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) + val startWidth = "width: %s%%".format((started.toDouble/total)*100) + +
+ + {completed}/{total} + { if (failed > 0) s"($failed failed)" } + { if (skipped > 0) s"($skipped skipped)" } + +
+
+
+ } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala new file mode 100644 index 0000000000000..ea2d187a0e8e4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllJobsPage.scala @@ -0,0 +1,151 @@ +/* + * 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.ui.jobs + +import scala.xml.{Node, NodeSeq} + +import javax.servlet.http.HttpServletRequest + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.ui.jobs.UIData.JobUIData + +/** Page showing list of all ongoing and recently finished jobs */ +private[ui] class AllJobsPage(parent: JobsTab) extends WebUIPage("") { + private val startTime: Option[Long] = parent.sc.map(_.startTime) + private val listener = parent.listener + + private def jobsTable(jobs: Seq[JobUIData]): Seq[Node] = { + val someJobHasJobGroup = jobs.exists(_.jobGroup.isDefined) + + val columns: Seq[Node] = { + {if (someJobHasJobGroup) "Job Id (Job Group)" else "Job Id"} + Description + Submitted + Duration + Stages: Succeeded/Total + Tasks (for all stages): Succeeded/Total + } + + def makeRow(job: JobUIData): Seq[Node] = { + val lastStageInfo = listener.stageIdToInfo.get(job.stageIds.max) + val lastStageData = lastStageInfo.flatMap { s => + listener.stageIdToData.get((s.stageId, s.attemptId)) + } + val isComplete = job.status == JobExecutionStatus.SUCCEEDED + val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") + val lastStageDescription = lastStageData.flatMap(_.description).getOrElse("") + val duration: Option[Long] = { + job.startTime.map { start => + val end = job.endTime.getOrElse(System.currentTimeMillis()) + end - start + } + } + val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") + val formattedSubmissionTime = job.startTime.map(UIUtils.formatDate).getOrElse("Unknown") + val detailUrl = + "%s/jobs/job?id=%s".format(UIUtils.prependBaseUri(parent.basePath), job.jobId) + + + {job.jobId} {job.jobGroup.map(id => s"($id)").getOrElse("")} + + +
{lastStageDescription}
+ {lastStageName} + + + {formattedSubmissionTime} + + {formattedDuration} + + {job.completedStageIndices.size}/{job.stageIds.size - job.numSkippedStages} + {if (job.numFailedStages > 0) s"(${job.numFailedStages} failed)"} + {if (job.numSkippedStages > 0) s"(${job.numSkippedStages} skipped)"} + + + {UIUtils.makeProgressBar(started = job.numActiveTasks, completed = job.numCompletedTasks, + failed = job.numFailedTasks, skipped = job.numSkippedTasks, + total = job.numTasks - job.numSkippedTasks)} + + + } + + + {columns} + + {jobs.map(makeRow)} + +
+ } + + def render(request: HttpServletRequest): Seq[Node] = { + listener.synchronized { + val activeJobs = listener.activeJobs.values.toSeq + val completedJobs = listener.completedJobs.reverse.toSeq + val failedJobs = listener.failedJobs.reverse.toSeq + val now = System.currentTimeMillis + + val activeJobsTable = + jobsTable(activeJobs.sortBy(_.startTime.getOrElse(-1L)).reverse) + val completedJobsTable = + jobsTable(completedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) + val failedJobsTable = + jobsTable(failedJobs.sortBy(_.endTime.getOrElse(-1L)).reverse) + + val summary: NodeSeq = +
+
    + {if (startTime.isDefined) { + // Total duration is not meaningful unless the UI is live +
  • + Total Duration: + {UIUtils.formatDuration(now - startTime.get)} +
  • + }} +
  • + Scheduling Mode: + {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} +
  • +
  • + Active Jobs: + {activeJobs.size} +
  • +
  • + Completed Jobs: + {completedJobs.size} +
  • +
  • + Failed Jobs: + {failedJobs.size} +
  • +
+
+ + val content = summary ++ +

Active Jobs ({activeJobs.size})

++ activeJobsTable ++ +

Completed Jobs ({completedJobs.size})

++ completedJobsTable ++ +

Failed Jobs ({failedJobs.size})

++ failedJobsTable + + val helpText = """A job is triggered by a action, like "count()" or "saveAsTextFile()".""" + + " Click on a job's title to see information about the stages of tasks associated with" + + " the job." + + UIUtils.headerSparkPage("Spark Jobs", content, parent, helpText = Some(helpText)) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala similarity index 87% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala index 83a7898071c9b..b0f8ca2ab0d3f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/AllStagesPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.scheduler.Schedulable import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ -private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") { +private[ui] class AllStagesPage(parent: StagesTab) extends WebUIPage("") { private val sc = parent.sc private val listener = parent.listener private def isFairScheduler = parent.isFairScheduler @@ -41,11 +41,14 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, - parent, parent.killEnabled) + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) val completedStagesTable = - new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent) + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) val failedStagesTable = - new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent) + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getAllPools).getOrElse(Seq.empty[Schedulable]) @@ -93,7 +96,7 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("")

Failed Stages ({numFailedStages})

++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage("Spark Stages", content, parent) + UIUtils.headerSparkPage("Spark Stages (for all jobs)", content, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index fa0f96bff34ff..35bbe8b4f9ac8 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -25,7 +25,7 @@ import org.apache.spark.ui.jobs.UIData.StageUIData import org.apache.spark.util.Utils /** Stage summary grouped by executors. */ -private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: JobProgressTab) { +private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: StagesTab) { private val listener = parent.listener def toNodeSeq: Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala new file mode 100644 index 0000000000000..77d36209c6048 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobPage.scala @@ -0,0 +1,177 @@ +/* + * 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.ui.jobs + +import scala.collection.mutable +import scala.xml.{NodeSeq, Node} + +import javax.servlet.http.HttpServletRequest + +import org.apache.spark.JobExecutionStatus +import org.apache.spark.scheduler.StageInfo +import org.apache.spark.ui.{UIUtils, WebUIPage} + +/** Page showing statistics and stage list for a given job */ +private[ui] class JobPage(parent: JobsTab) extends WebUIPage("job") { + private val listener = parent.listener + + def render(request: HttpServletRequest): Seq[Node] = { + listener.synchronized { + val jobId = request.getParameter("id").toInt + val jobDataOption = listener.jobIdToData.get(jobId) + if (jobDataOption.isEmpty) { + val content = +
+

No information to display for job {jobId}

+
+ return UIUtils.headerSparkPage( + s"Details for Job $jobId", content, parent) + } + val jobData = jobDataOption.get + val isComplete = jobData.status != JobExecutionStatus.RUNNING + val stages = jobData.stageIds.map { stageId => + // This could be empty if the JobProgressListener hasn't received information about the + // stage or if the stage information has been garbage collected + listener.stageIdToInfo.getOrElse(stageId, + new StageInfo(stageId, 0, "Unknown", 0, Seq.empty, "Unknown")) + } + + val activeStages = mutable.Buffer[StageInfo]() + val completedStages = mutable.Buffer[StageInfo]() + // If the job is completed, then any pending stages are displayed as "skipped": + val pendingOrSkippedStages = mutable.Buffer[StageInfo]() + val failedStages = mutable.Buffer[StageInfo]() + for (stage <- stages) { + if (stage.submissionTime.isEmpty) { + pendingOrSkippedStages += stage + } else if (stage.completionTime.isDefined) { + if (stage.failureReason.isDefined) { + failedStages += stage + } else { + completedStages += stage + } + } else { + activeStages += stage + } + } + + val activeStagesTable = + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) + val pendingOrSkippedStagesTable = + new StageTableBase(pendingOrSkippedStages.sortBy(_.stageId).reverse, + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = false) + val completedStagesTable = + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler, killEnabled = false) + val failedStagesTable = + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent.basePath, + parent.listener, isFairScheduler = parent.isFairScheduler) + + val shouldShowActiveStages = activeStages.nonEmpty + val shouldShowPendingStages = !isComplete && pendingOrSkippedStages.nonEmpty + val shouldShowCompletedStages = completedStages.nonEmpty + val shouldShowSkippedStages = isComplete && pendingOrSkippedStages.nonEmpty + val shouldShowFailedStages = failedStages.nonEmpty + + val summary: NodeSeq = +
+
    +
  • + Status: + {jobData.status} +
  • + { + if (jobData.jobGroup.isDefined) { +
  • + Job Group: + {jobData.jobGroup.get} +
  • + } + } + { + if (shouldShowActiveStages) { +
  • + Active Stages: + {activeStages.size} +
  • + } + } + { + if (shouldShowPendingStages) { +
  • + + Pending Stages: + {pendingOrSkippedStages.size} +
  • + } + } + { + if (shouldShowCompletedStages) { +
  • + Completed Stages: + {completedStages.size} +
  • + } + } + { + if (shouldShowSkippedStages) { +
  • + Skipped Stages: + {pendingOrSkippedStages.size} +
  • + } + } + { + if (shouldShowFailedStages) { +
  • + Failed Stages: + {failedStages.size} +
  • + } + } +
+
+ + var content = summary + if (shouldShowActiveStages) { + content ++=

Active Stages ({activeStages.size})

++ + activeStagesTable.toNodeSeq + } + if (shouldShowPendingStages) { + content ++=

Pending Stages ({pendingOrSkippedStages.size})

++ + pendingOrSkippedStagesTable.toNodeSeq + } + if (shouldShowCompletedStages) { + content ++=

Completed Stages ({completedStages.size})

++ + completedStagesTable.toNodeSeq + } + if (shouldShowSkippedStages) { + content ++=

Skipped Stages ({pendingOrSkippedStages.size})

++ + pendingOrSkippedStagesTable.toNodeSeq + } + if (shouldShowFailedStages) { + content ++=

Failed Stages ({failedStages.size})

++ + failedStagesTable.toNodeSeq + } + UIUtils.headerSparkPage(s"Details for Job $jobId", content, parent) + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index ccdcf0e047f48..72935beb3a34a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -17,7 +17,7 @@ package org.apache.spark.ui.jobs -import scala.collection.mutable.{HashMap, ListBuffer} +import scala.collection.mutable.{HashMap, HashSet, ListBuffer} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi @@ -49,8 +49,6 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { type PoolName = String type ExecutorId = String - // Define all of our state: - // Jobs: val activeJobs = new HashMap[JobId, JobUIData] val completedJobs = ListBuffer[JobUIData]() @@ -60,9 +58,11 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { // Stages: val activeStages = new HashMap[StageId, StageInfo] val completedStages = ListBuffer[StageInfo]() + val skippedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() val stageIdToData = new HashMap[(StageId, StageAttemptId), StageUIData] val stageIdToInfo = new HashMap[StageId, StageInfo] + val stageIdToActiveJobIds = new HashMap[StageId, HashSet[JobId]] val poolToActiveStages = HashMap[PoolName, HashMap[StageId, StageInfo]]() // Total of completed and failed stages that have ever been run. These may be greater than // `completedStages.size` and `failedStages.size` if we have run more stages or jobs than @@ -95,7 +95,8 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { Map( "activeStages" -> activeStages.size, "activeJobs" -> activeJobs.size, - "poolToActiveStages" -> poolToActiveStages.values.map(_.size).sum + "poolToActiveStages" -> poolToActiveStages.values.map(_.size).sum, + "stageIdToActiveJobIds" -> stageIdToActiveJobIds.values.map(_.size).sum ) } @@ -106,6 +107,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { "completedJobs" -> completedJobs.size, "failedJobs" -> failedJobs.size, "completedStages" -> completedStages.size, + "skippedStages" -> skippedStages.size, "failedStages" -> failedStages.size ) } @@ -144,11 +146,39 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { } override def onJobStart(jobStart: SparkListenerJobStart) = synchronized { - val jobGroup = Option(jobStart.properties).map(_.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + val jobGroup = for ( + props <- Option(jobStart.properties); + group <- Option(props.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) + ) yield group val jobData: JobUIData = - new JobUIData(jobStart.jobId, jobStart.stageIds, jobGroup, JobExecutionStatus.RUNNING) + new JobUIData( + jobId = jobStart.jobId, + startTime = Some(System.currentTimeMillis), + endTime = None, + stageIds = jobStart.stageIds, + jobGroup = jobGroup, + status = JobExecutionStatus.RUNNING) + // Compute (a potential underestimate of) the number of tasks that will be run by this job. + // This may be an underestimate because the job start event references all of the result + // stages's transitive stage dependencies, but some of these stages might be skipped if their + // output is available from earlier runs. + // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. + jobData.numTasks = { + val allStages = jobStart.stageInfos + val missingStages = allStages.filter(_.completionTime.isEmpty) + missingStages.map(_.numTasks).sum + } jobIdToData(jobStart.jobId) = jobData activeJobs(jobStart.jobId) = jobData + for (stageId <- jobStart.stageIds) { + stageIdToActiveJobIds.getOrElseUpdate(stageId, new HashSet[StageId]).add(jobStart.jobId) + } + // If there's no information for a stage, store the StageInfo received from the scheduler + // so that we can display stage descriptions for pending stages: + for (stageInfo <- jobStart.stageInfos) { + stageIdToInfo.getOrElseUpdate(stageInfo.stageId, stageInfo) + stageIdToData.getOrElseUpdate((stageInfo.stageId, stageInfo.attemptId), new StageUIData) + } } override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { @@ -156,6 +186,7 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { logWarning(s"Job completed for unknown job ${jobEnd.jobId}") new JobUIData(jobId = jobEnd.jobId) } + jobData.endTime = Some(System.currentTimeMillis()) jobEnd.jobResult match { case JobSucceeded => completedJobs += jobData @@ -166,6 +197,20 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { trimJobsIfNecessary(failedJobs) jobData.status = JobExecutionStatus.FAILED } + for (stageId <- jobData.stageIds) { + stageIdToActiveJobIds.get(stageId).foreach { jobsUsingStage => + jobsUsingStage.remove(jobEnd.jobId) + stageIdToInfo.get(stageId).foreach { stageInfo => + if (stageInfo.submissionTime.isEmpty) { + // if this stage is pending, it won't complete, so mark it as "skipped": + skippedStages += stageInfo + trimStagesIfNecessary(skippedStages) + jobData.numSkippedStages += 1 + jobData.numSkippedTasks += stageInfo.numTasks + } + } + } + } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { @@ -193,6 +238,19 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { numFailedStages += 1 trimStagesIfNecessary(failedStages) } + + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(stage.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveStages -= 1 + if (stage.failureReason.isEmpty) { + jobData.completedStageIndices.add(stage.stageId) + } else { + jobData.numFailedStages += 1 + } + } } /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ @@ -214,6 +272,14 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]) stages(stage.stageId) = stage + + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(stage.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveStages += 1 + } } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { @@ -226,6 +292,13 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { stageData.numActiveTasks += 1 stageData.taskData.put(taskInfo.taskId, new TaskUIData(taskInfo)) } + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskStart.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveTasks += 1 + } } override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { @@ -283,6 +356,20 @@ class JobProgressListener(conf: SparkConf) extends SparkListener with Logging { taskData.taskInfo = info taskData.taskMetrics = metrics taskData.errorMessage = errorMessage + + for ( + activeJobsDependentOnStage <- stageIdToActiveJobIds.get(taskEnd.stageId); + jobId <- activeJobsDependentOnStage; + jobData <- jobIdToData.get(jobId) + ) { + jobData.numActiveTasks -= 1 + taskEnd.reason match { + case Success => + jobData.numCompletedTasks += 1 + case _ => + jobData.numFailedTasks += 1 + } + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala new file mode 100644 index 0000000000000..b2bbfdee56946 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -0,0 +1,32 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui.jobs + +import org.apache.spark.scheduler.SchedulingMode +import org.apache.spark.ui.{SparkUI, SparkUITab} + +/** Web UI showing progress status of all jobs in the given SparkContext. */ +private[ui] class JobsTab(parent: SparkUI) extends SparkUITab(parent, "jobs") { + val sc = parent.sc + val killEnabled = parent.killEnabled + def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) + val listener = parent.jobProgressListener + + attachPage(new AllJobsPage(this)) + attachPage(new JobPage(this)) +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 770d99eea1c9d..5fc6cc7533150 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ -private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { +private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { private val sc = parent.sc private val listener = parent.listener @@ -37,8 +37,9 @@ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { case Some(s) => s.values.toSeq case None => Seq[StageInfo]() } - val activeStagesTable = - new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent) + val activeStagesTable = new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent.basePath, parent.listener, isFairScheduler = parent.isFairScheduler, + killEnabled = parent.killEnabled) // For now, pool information is only accessible in live UIs val pools = sc.map(_.getPoolForName(poolName).get).toSeq diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index 64178e1e33d41..df1899e7a9b84 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -24,7 +24,7 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: StagesTab) { private val listener = parent.listener def toNodeSeq: Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 36afc4942e085..40e05f86b661d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.{Utils, Distribution} import org.apache.spark.scheduler.{AccumulableInfo, TaskInfo} /** Page showing statistics and task list for a given stage */ -private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { +private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 2ff561ccc7da0..e7d6244dcd679 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -31,11 +31,10 @@ import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTableBase( stages: Seq[StageInfo], - parent: JobProgressTab, - killEnabled: Boolean = false) { - - private val listener = parent.listener - protected def isFairScheduler = parent.isFairScheduler + basePath: String, + listener: JobProgressListener, + isFairScheduler: Boolean, + killEnabled: Boolean) { protected def columns: Seq[Node] = { Stage Id ++ @@ -73,25 +72,11 @@ private[ui] class StageTableBase( } - private def makeProgressBar(started: Int, completed: Int, failed: Int, total: Int): Seq[Node] = - { - val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) - val startWidth = "width: %s%%".format((started.toDouble/total)*100) - -
- - {completed}/{total} { if (failed > 0) s"($failed failed)" else "" } - -
-
-
- } - private def makeDescription(s: StageInfo): Seq[Node] = { // scalastyle:off val killLink = if (killEnabled) { val killLinkUri = "%s/stages/stage/kill?id=%s&terminate=true" - .format(UIUtils.prependBaseUri(parent.basePath), s.stageId) + .format(UIUtils.prependBaseUri(basePath), s.stageId) val confirm = "return window.confirm('Are you sure you want to kill stage %s ?');" .format(s.stageId) @@ -101,7 +86,7 @@ private[ui] class StageTableBase( // scalastyle:on val nameLinkUri ="%s/stages/stage?id=%s&attempt=%s" - .format(UIUtils.prependBaseUri(parent.basePath), s.stageId, s.attemptId) + .format(UIUtils.prependBaseUri(basePath), s.stageId, s.attemptId) val nameLink = {s.name} val cachedRddInfos = s.rddInfos.filter(_.numCachedPartitions > 0) @@ -115,7 +100,7 @@ private[ui] class StageTableBase( Text("RDD: ") ++ // scalastyle:off cachedRddInfos.map { i => - {i.name} + {i.name} } // scalastyle:on }} @@ -167,7 +152,7 @@ private[ui] class StageTableBase( {if (isFairScheduler) { + .format(UIUtils.prependBaseUri(basePath), stageData.schedulingPool)}> {stageData.schedulingPool} @@ -180,8 +165,9 @@ private[ui] class StageTableBase( {formattedDuration} - {makeProgressBar(stageData.numActiveTasks, stageData.completedIndices.size, - stageData.numFailedTasks, s.numTasks)} + {UIUtils.makeProgressBar(started = stageData.numActiveTasks, + completed = stageData.completedIndices.size, failed = stageData.numFailedTasks, + skipped = 0, total = s.numTasks)} {inputReadWithUnit} {outputWriteWithUnit} @@ -195,9 +181,10 @@ private[ui] class StageTableBase( private[ui] class FailedStageTable( stages: Seq[StageInfo], - parent: JobProgressTab, - killEnabled: Boolean = false) - extends StageTableBase(stages, parent, killEnabled) { + basePath: String, + listener: JobProgressListener, + isFairScheduler: Boolean) + extends StageTableBase(stages, basePath, listener, isFairScheduler, killEnabled = false) { override protected def columns: Seq[Node] = super.columns ++ Failure Reason diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala similarity index 83% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 03ca918e2e8b3..937261de00e3a 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -19,18 +19,16 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.{SparkUI, SparkUITab} -/** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobProgressTab(parent: SparkUI) extends SparkUITab(parent, "stages") { +/** Web UI showing progress status of all stages in the given SparkContext. */ +private[ui] class StagesTab(parent: SparkUI) extends SparkUITab(parent, "stages") { val sc = parent.sc - val conf = sc.map(_.conf).getOrElse(new SparkConf) - val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + val killEnabled = parent.killEnabled val listener = parent.jobProgressListener - attachPage(new JobProgressPage(this)) + attachPage(new AllStagesPage(this)) attachPage(new StagePage(this)) attachPage(new PoolPage(this)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala index 2f7d618df5f6f..48fd7caa1a1ed 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/UIData.scala @@ -40,9 +40,28 @@ private[jobs] object UIData { class JobUIData( var jobId: Int = -1, + var startTime: Option[Long] = None, + var endTime: Option[Long] = None, var stageIds: Seq[Int] = Seq.empty, var jobGroup: Option[String] = None, - var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN + var status: JobExecutionStatus = JobExecutionStatus.UNKNOWN, + /* Tasks */ + // `numTasks` is a potential underestimate of the true number of tasks that this job will run. + // This may be an underestimate because the job start event references all of the result + // stages's transitive stage dependencies, but some of these stages might be skipped if their + // output is available from earlier runs. + // See https://github.com/apache/spark/pull/3009 for a more extensive discussion. + var numTasks: Int = 0, + var numActiveTasks: Int = 0, + var numCompletedTasks: Int = 0, + var numSkippedTasks: Int = 0, + var numFailedTasks: Int = 0, + /* Stages */ + var numActiveStages: Int = 0, + // This needs to be a set instead of a simple count to prevent double-counting of rerun stages: + var completedStageIndices: OpenHashSet[Int] = new OpenHashSet[Int](), + var numSkippedStages: Int = 0, + var numFailedStages: Int = 0 ) class StageUIData { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 7e536edfe807b..7b5db1ed76265 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -31,6 +31,21 @@ import org.apache.spark.scheduler._ import org.apache.spark.storage._ import org.apache.spark._ +/** + * Serializes SparkListener events to/from JSON. This protocol provides strong backwards- + * and forwards-compatibility guarantees: any version of Spark should be able to read JSON output + * written by any other version, including newer versions. + * + * JsonProtocolSuite contains backwards-compatibility tests which check that the current version of + * JsonProtocol is able to read output written by earlier versions. We do not currently have tests + * for reading newer JSON output with older Spark versions. + * + * To ensure that we provide these guarantees, follow these rules when modifying these methods: + * + * - Never delete any JSON fields. + * - Any new JSON fields should be optional; use `Utils.jsonOption` when reading these fields + * in `*FromJson` methods. + */ private[spark] object JsonProtocol { // TODO: Remove this file and put JSON serialization into each individual class. @@ -121,6 +136,7 @@ private[spark] object JsonProtocol { val properties = propertiesToJson(jobStart.properties) ("Event" -> Utils.getFormattedClassName(jobStart)) ~ ("Job ID" -> jobStart.jobId) ~ + ("Stage Infos" -> jobStart.stageInfos.map(stageInfoToJson)) ~ // Added in Spark 1.2.0 ("Stage IDs" -> jobStart.stageIds) ~ ("Properties" -> properties) } @@ -455,7 +471,12 @@ private[spark] object JsonProtocol { val jobId = (json \ "Job ID").extract[Int] val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) val properties = propertiesFromJson(json \ "Properties") - SparkListenerJobStart(jobId, stageIds, properties) + // The "Stage Infos" field was added in Spark 1.2.0 + val stageInfos = Utils.jsonOption(json \ "Stage Infos") + .map(_.extract[Seq[JValue]].map(stageInfoFromJson)).getOrElse { + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) + } + SparkListenerJobStart(jobId, stageInfos, properties) } def jobEndFromJson(json: JValue): SparkListenerJobEnd = { diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index bacf6a16fc233..d2857b8b55664 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -17,16 +17,20 @@ package org.apache.spark.ui -import org.apache.spark.api.java.StorageLevels -import org.apache.spark.{SparkException, SparkConf, SparkContext} -import org.openqa.selenium.WebDriver +import scala.collection.JavaConversions._ + +import org.openqa.selenium.{By, WebDriver} import org.openqa.selenium.htmlunit.HtmlUnitDriver import org.scalatest._ import org.scalatest.concurrent.Eventually._ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ +import org.apache.spark._ +import org.apache.spark.SparkContext._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark.api.java.StorageLevels +import org.apache.spark.shuffle.FetchFailedException /** * Selenium tests for the Spark Web UI. These tests are not run by default @@ -89,7 +93,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { sc.parallelize(1 to 10).map { x => throw new Exception()}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to sc.ui.get.appUIAddress + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") find(id("active")).get.text should be("Active Stages (0)") find(id("failed")).get.text should be("Failed Stages (1)") } @@ -101,7 +105,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { sc.parallelize(1 to 10).map { x => unserializableObject}.collect() } eventually(timeout(5 seconds), interval(50 milliseconds)) { - go to sc.ui.get.appUIAddress + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") find(id("active")).get.text should be("Active Stages (0)") // The failure occurs before the stage becomes active, hence we should still show only one // failed stage, not two: @@ -109,4 +113,191 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers { } } } + + test("spark.ui.killEnabled should properly control kill button display") { + def getSparkContext(killEnabled: Boolean): SparkContext = { + val conf = new SparkConf() + .setMaster("local") + .setAppName("test") + .set("spark.ui.enabled", "true") + .set("spark.ui.killEnabled", killEnabled.toString) + new SparkContext(conf) + } + + def hasKillLink = find(className("kill-link")).isDefined + def runSlowJob(sc: SparkContext) { + sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() + } + + withSpark(getSparkContext(killEnabled = true)) { sc => + runSlowJob(sc) + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + assert(hasKillLink) + } + } + + withSpark(getSparkContext(killEnabled = false)) { sc => + runSlowJob(sc) + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/stages") + assert(!hasKillLink) + } + } + } + + test("jobs page should not display job group name unless some job was submitted in a job group") { + withSpark(newSparkContext()) { sc => + // If no job has been run in a job group, then "(Job Group)" should not appear in the header + sc.parallelize(Seq(1, 2, 3)).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq + tableHeaders should not contain "Job Id (Job Group)" + } + // Once at least one job has been run in a job group, then we should display the group name: + sc.setJobGroup("my-job-group", "my-job-group-description") + sc.parallelize(Seq(1, 2, 3)).count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + val tableHeaders = findAll(cssSelector("th")).map(_.text).toSeq + tableHeaders should contain ("Job Id (Job Group)") + } + } + } + + test("job progress bars should handle stage / task failures") { + withSpark(newSparkContext()) { sc => + val data = sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity) + val shuffleHandle = + data.dependencies.head.asInstanceOf[ShuffleDependency[_, _, _]].shuffleHandle + // Simulate fetch failures: + val mappedData = data.map { x => + val taskContext = TaskContext.get + if (taskContext.attemptId() == 1) { // Cause this stage to fail on its first attempt. + val env = SparkEnv.get + val bmAddress = env.blockManager.blockManagerId + val shuffleId = shuffleHandle.shuffleId + val mapId = 0 + val reduceId = taskContext.partitionId() + val message = "Simulated fetch failure" + throw new FetchFailedException(bmAddress, shuffleId, mapId, reduceId, message) + } else { + x + } + } + mappedData.count() + eventually(timeout(5 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + find(cssSelector(".stage-progress-cell")).get.text should be ("2/2 (1 failed)") + // Ideally, the following test would pass, but currently we overcount completed tasks + // if task recomputations occur: + // find(cssSelector(".progress-cell .progress")).get.text should be ("2/2 (1 failed)") + // Instead, we guarantee that the total number of tasks is always correct, while the number + // of completed tasks may be higher: + find(cssSelector(".progress-cell .progress")).get.text should be ("3/2 (1 failed)") + } + } + } + + test("job details page should display useful information for stages that haven't started") { + withSpark(newSparkContext()) { sc => + // Create a multi-stage job with a long delay in the first stage: + val rdd = sc.parallelize(Seq(1, 2, 3)).map { x => + // This long sleep call won't slow down the tests because we don't actually need to wait + // for the job to finish. + Thread.sleep(20000) + }.groupBy(identity).map(identity).groupBy(identity).map(identity) + // Start the job: + rdd.countAsync() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=0") + find(id("active")).get.text should be ("Active Stages (1)") + find(id("pending")).get.text should be ("Pending Stages (2)") + // Essentially, we want to check that none of the stage rows show + // "No data available for this stage". Checking for the absence of that string is brittle + // because someone could change the error message and cause this test to pass by accident. + // Instead, it's safer to check that each row contains a link to a stage details page. + findAll(cssSelector("tbody tr")).foreach { row => + val link = row.underlying.findElement(By.xpath(".//a")) + link.getAttribute("href") should include ("stage") + } + } + } + } + + test("job progress bars / cells reflect skipped stages / tasks") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = sc.parallelize(1 to 8, 8) + .map(x => x).groupBy((x: Int) => x, numPartitions = 8) + .flatMap(x => x._2).groupBy((x: Int) => x, numPartitions = 8) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + // The completed jobs table should have two rows. The first row will be the most recent job: + val firstRow = find(cssSelector("tbody tr")).get.underlying + val firstRowColumns = firstRow.findElements(By.tagName("td")) + firstRowColumns(0).getText should be ("1") + firstRowColumns(4).getText should be ("1/1 (2 skipped)") + firstRowColumns(5).getText should be ("8/8 (16 skipped)") + // The second row is the first run of the job, where nothing was skipped: + val secondRow = findAll(cssSelector("tbody tr")).toSeq(1).underlying + val secondRowColumns = secondRow.findElements(By.tagName("td")) + secondRowColumns(0).getText should be ("0") + secondRowColumns(4).getText should be ("3/3") + secondRowColumns(5).getText should be ("24/24") + } + } + } + + test("stages that aren't run appear as 'skipped stages' after a job finishes") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = + sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs/job/?id=1") + find(id("pending")) should be (None) + find(id("active")) should be (None) + find(id("failed")) should be (None) + find(id("completed")).get.text should be ("Completed Stages (1)") + find(id("skipped")).get.text should be ("Skipped Stages (2)") + // Essentially, we want to check that none of the stage rows show + // "No data available for this stage". Checking for the absence of that string is brittle + // because someone could change the error message and cause this test to pass by accident. + // Instead, it's safer to check that each row contains a link to a stage details page. + findAll(cssSelector("tbody tr")).foreach { row => + val link = row.underlying.findElement(By.xpath(".//a")) + link.getAttribute("href") should include ("stage") + } + } + } + } + + test("jobs with stages that are skipped should show correct link descriptions on all jobs page") { + withSpark(newSparkContext()) { sc => + // Create an RDD that involves multiple stages: + val rdd = + sc.parallelize(Seq(1, 2, 3)).map(identity).groupBy(identity).map(identity).groupBy(identity) + // Run it twice; this will cause the second job to have two "phantom" stages that were + // mentioned in its job start event but which were never actually executed: + rdd.count() + rdd.count() + eventually(timeout(10 seconds), interval(50 milliseconds)) { + go to (sc.ui.get.appUIAddress.stripSuffix("/") + "/jobs") + findAll(cssSelector("tbody tr a")).foreach { link => + link.text.toLowerCase should include ("count") + link.text.toLowerCase should not include "unknown" + } + } + } + } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 15c5b4e702efa..12af60caf7d54 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -43,7 +43,10 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc } private def createJobStartEvent(jobId: Int, stageIds: Seq[Int]) = { - SparkListenerJobStart(jobId, stageIds) + val stageInfos = stageIds.map { stageId => + new StageInfo(stageId, 0, stageId.toString, 0, null, "") + } + SparkListenerJobStart(jobId, stageInfos) } private def createJobEndEvent(jobId: Int, failed: Boolean = false) = { @@ -52,8 +55,9 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc } private def runJob(listener: SparkListener, jobId: Int, shouldFail: Boolean = false) { + val stagesThatWontBeRun = jobId * 200 to jobId * 200 + 10 val stageIds = jobId * 100 to jobId * 100 + 50 - listener.onJobStart(createJobStartEvent(jobId, stageIds)) + listener.onJobStart(createJobStartEvent(jobId, stageIds ++ stagesThatWontBeRun)) for (stageId <- stageIds) { listener.onStageSubmitted(createStageStartEvent(stageId)) listener.onStageCompleted(createStageEndEvent(stageId, failed = stageId % 2 == 0)) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 50f42054b9296..0bc9492675863 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -47,7 +47,12 @@ class JsonProtocolSuite extends FunSuite { val taskEndWithOutput = SparkListenerTaskEnd(1, 0, "ResultTask", Success, makeTaskInfo(123L, 234, 67, 345L, false), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800, hasHadoopInput = true, hasOutput = true)) - val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties) + val jobStart = { + val stageIds = Seq[Int](1, 2, 3, 4) + val stageInfos = stageIds.map(x => + makeStageInfo(x, x * 200, x * 300, x * 400L, x * 500L)) + SparkListenerJobStart(10, stageInfos, properties) + } val jobEnd = SparkListenerJobEnd(20, JobSucceeded) val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")), @@ -224,6 +229,19 @@ class JsonProtocolSuite extends FunSuite { assert(expectedExecutorLostFailure === JsonProtocol.taskEndReasonFromJson(oldEvent)) } + test("SparkListenerJobStart backward compatibility") { + // Prior to Spark 1.2.0, SparkListenerJobStart did not have a "Stage Infos" property. + val stageIds = Seq[Int](1, 2, 3, 4) + val stageInfos = stageIds.map(x => makeStageInfo(x, x * 200, x * 300, x * 400, x * 500)) + val dummyStageInfos = + stageIds.map(id => new StageInfo(id, 0, "unknown", 0, Seq.empty, "unknown")) + val jobStart = SparkListenerJobStart(10, stageInfos, properties) + val oldEvent = JsonProtocol.jobStartToJson(jobStart).removeField({_._1 == "Stage Infos"}) + val expectedJobStart = + SparkListenerJobStart(10, dummyStageInfos, properties) + assertEquals(expectedJobStart, JsonProtocol.jobStartFromJson(oldEvent)) + } + /** -------------------------- * | Helper test running methods | * --------------------------- */ @@ -306,7 +324,7 @@ class JsonProtocolSuite extends FunSuite { case (e1: SparkListenerJobStart, e2: SparkListenerJobStart) => assert(e1.jobId === e2.jobId) assert(e1.properties === e2.properties) - assertSeqEquals(e1.stageIds, e2.stageIds, (i1: Int, i2: Int) => assert(i1 === i2)) + assert(e1.stageIds === e2.stageIds) case (e1: SparkListenerJobEnd, e2: SparkListenerJobEnd) => assert(e1.jobId === e2.jobId) assertEquals(e1.jobResult, e2.jobResult) @@ -1051,6 +1069,260 @@ class JsonProtocolSuite extends FunSuite { |{ | "Event": "SparkListenerJobStart", | "Job ID": 10, + | "Stage Infos": [ + | { + | "Stage ID": 1, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 200, + | "RDD Info": [ + | { + | "RDD ID": 1, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 200, + | "Number of Cached Partitions": 300, + | "Memory Size": 400, + | "Tachyon Size": 0, + | "Disk Size": 500 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 2, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 400, + | "RDD Info": [ + | { + | "RDD ID": 2, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 400, + | "Number of Cached Partitions": 600, + | "Memory Size": 800, + | "Tachyon Size": 0, + | "Disk Size": 1000 + | }, + | { + | "RDD ID": 3, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 401, + | "Number of Cached Partitions": 601, + | "Memory Size": 801, + | "Tachyon Size": 0, + | "Disk Size": 1001 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 3, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 600, + | "RDD Info": [ + | { + | "RDD ID": 3, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 600, + | "Number of Cached Partitions": 900, + | "Memory Size": 1200, + | "Tachyon Size": 0, + | "Disk Size": 1500 + | }, + | { + | "RDD ID": 4, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 601, + | "Number of Cached Partitions": 901, + | "Memory Size": 1201, + | "Tachyon Size": 0, + | "Disk Size": 1501 + | }, + | { + | "RDD ID": 5, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 602, + | "Number of Cached Partitions": 902, + | "Memory Size": 1202, + | "Tachyon Size": 0, + | "Disk Size": 1502 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | }, + | { + | "Stage ID": 4, + | "Stage Attempt ID": 0, + | "Stage Name": "greetings", + | "Number of Tasks": 800, + | "RDD Info": [ + | { + | "RDD ID": 4, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 800, + | "Number of Cached Partitions": 1200, + | "Memory Size": 1600, + | "Tachyon Size": 0, + | "Disk Size": 2000 + | }, + | { + | "RDD ID": 5, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 801, + | "Number of Cached Partitions": 1201, + | "Memory Size": 1601, + | "Tachyon Size": 0, + | "Disk Size": 2001 + | }, + | { + | "RDD ID": 6, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 802, + | "Number of Cached Partitions": 1202, + | "Memory Size": 1602, + | "Tachyon Size": 0, + | "Disk Size": 2002 + | }, + | { + | "RDD ID": 7, + | "Name": "mayor", + | "Storage Level": { + | "Use Disk": true, + | "Use Memory": true, + | "Use Tachyon": false, + | "Deserialized": true, + | "Replication": 1 + | }, + | "Number of Partitions": 803, + | "Number of Cached Partitions": 1203, + | "Memory Size": 1603, + | "Tachyon Size": 0, + | "Disk Size": 2003 + | } + | ], + | "Details": "details", + | "Accumulables": [ + | { + | "ID": 2, + | "Name": " Accumulable 2", + | "Update": "delta2", + | "Value": "val2" + | }, + | { + | "ID": 1, + | "Name": " Accumulable 1", + | "Update": "delta1", + | "Value": "val1" + | } + | ] + | } + | ], | "Stage IDs": [ | 1, | 2, From cb0e9b0980f38befe88bf52aa037fe33262730f7 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 24 Nov 2014 13:50:20 -0800 Subject: [PATCH 012/305] [SPARK-4518][SPARK-4519][Streaming] Refactored file stream to prevent files from being processed multiple times Because of a corner case, a file already selected for batch t can get considered again for batch t+2. This refactoring fixes it by remembering all the files selected in the last 1 minute, so that this corner case does not arise. Also uses spark context's hadoop configuration to access the file system API for listing directories. pwendell Please take look. I still have not run long-running integration tests, so I cannot say for sure whether this has indeed solved the issue. You could do a first pass on this in the meantime. Author: Tathagata Das Closes #3419 from tdas/filestream-fix2 and squashes the following commits: c19dd8a [Tathagata Das] Addressed PR comments. 513b608 [Tathagata Das] Updated docs. d364faf [Tathagata Das] Added the current time condition back 5526222 [Tathagata Das] Removed unnecessary imports. 38bb736 [Tathagata Das] Fix long line. 203bbc7 [Tathagata Das] Un-ignore tests. eaef4e1 [Tathagata Das] Fixed SPARK-4519 9dbd40a [Tathagata Das] Refactored FileInputDStream to remember last few batches. --- .../spark/streaming/dstream/DStream.scala | 2 +- .../streaming/dstream/FileInputDStream.scala | 291 +++++++++++------- .../spark/streaming/CheckpointSuite.scala | 2 +- .../spark/streaming/InputStreamsSuite.scala | 106 ++++--- 4 files changed, 245 insertions(+), 156 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index eabd61d713e0c..dbf1ebbaf653a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -254,7 +254,7 @@ abstract class DStream[T: ClassTag] ( } private[streaming] def remember(duration: Duration) { - if (duration != null && duration > rememberDuration) { + if (duration != null && (rememberDuration == null || duration > rememberDuration)) { rememberDuration = duration logInfo("Duration for remembering RDDs set to " + rememberDuration + " for " + this) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 55d6cf6a783ea..5f13fdc5579ed 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -17,18 +17,55 @@ package org.apache.spark.streaming.dstream -import java.io.{ObjectInputStream, IOException} -import scala.collection.mutable.{HashSet, HashMap} +import java.io.{IOException, ObjectInputStream} + +import scala.collection.mutable import scala.reflect.ClassTag + import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.UnionRDD -import org.apache.spark.streaming.{StreamingContext, Time} -import org.apache.spark.util.{TimeStampedHashMap, Utils} +import org.apache.spark.rdd.{RDD, UnionRDD} +import org.apache.spark.streaming._ +import org.apache.spark.util.{TimeStampedHashMap, Utils} +/** + * This class represents an input stream that monitors a Hadoop-compatible filesystem for new + * files and creates a stream out of them. The way it works as follows. + * + * At each batch interval, the file system is queried for files in the given directory and + * detected new files are selected for that batch. In this case "new" means files that + * became visible to readers during that time period. Some extra care is needed to deal + * with the fact that files may become visible after they are created. For this purpose, this + * class remembers the information about the files selected in past batches for + * a certain duration (say, "remember window") as shown in the figure below. + * + * |<----- remember window ----->| + * ignore threshold --->| |<--- current batch time + * |____.____.____.____.____.____| + * | | | | | | | + * ---------------------|----|----|----|----|----|----|-----------------------> Time + * |____|____|____|____|____|____| + * remembered batches + * + * The trailing end of the window is the "ignore threshold" and all files whose mod times + * are less than this threshold are assumed to have already been selected and are therefore + * ignored. Files whose mod times are within the "remember window" are checked against files + * that have already been selected. At a high level, this is how new files are identified in + * each batch - files whose mod times are greater than the ignore threshold and + * have not been considered within the remember window. See the documentation on the method + * `isNewFile` for more details. + * + * This makes some assumptions from the underlying file system that the system is monitoring. + * - The clock of the file system is assumed to synchronized with the clock of the machine running + * the streaming app. + * - If a file is to be visible in the directory listings, it must be visible within a certain + * duration of the mod time of the file. This duration is the "remember window", which is set to + * 1 minute (see `FileInputDStream.MIN_REMEMBER_DURATION`). Otherwise, the file will never be + * selected as the mod time will be less than the ignore threshold when it becomes visible. + * - Once a file is visible, the mod time cannot change. If it does due to appends, then the + * processing semantics are undefined. + */ private[streaming] class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : ClassTag]( @transient ssc_ : StreamingContext, @@ -37,22 +74,37 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas newFilesOnly: Boolean = true) extends InputDStream[(K, V)](ssc_) { + // Data to be saved as part of the streaming checkpoints protected[streaming] override val checkpointData = new FileInputDStreamCheckpointData - // files found in the last interval - private val lastFoundFiles = new HashSet[String] + // Initial ignore threshold based on which old, existing files in the directory (at the time of + // starting the streaming application) will be ignored or considered + private val initialModTimeIgnoreThreshold = if (newFilesOnly) System.currentTimeMillis() else 0L + + /* + * Make sure that the information of files selected in the last few batches are remembered. + * This would allow us to filter away not-too-old files which have already been recently + * selected and processed. + */ + private val numBatchesToRemember = FileInputDStream.calculateNumBatchesToRemember(slideDuration) + private val durationToRemember = slideDuration * numBatchesToRemember + remember(durationToRemember) - // Files with mod time earlier than this is ignored. This is updated every interval - // such that in the current interval, files older than any file found in the - // previous interval will be ignored. Obviously this time keeps moving forward. - private var ignoreTime = if (newFilesOnly) System.currentTimeMillis() else 0L + // Map of batch-time to selected file info for the remembered batches + @transient private[streaming] var batchTimeToSelectedFiles = + new mutable.HashMap[Time, Array[String]] + + // Set of files that were selected in the remembered batches + @transient private var recentlySelectedFiles = new mutable.HashSet[String]() + + // Read-through cache of file mod times, used to speed up mod time lookups + @transient private var fileToModTime = new TimeStampedHashMap[String, Long](true) + + // Timestamp of the last round of finding files + @transient private var lastNewFileFindingTime = 0L - // Latest file mod time seen till any point of time @transient private var path_ : Path = null @transient private var fs_ : FileSystem = null - @transient private[streaming] var files = new HashMap[Time, Array[String]] - @transient private var fileModTimes = new TimeStampedHashMap[String, Long](true) - @transient private var lastNewFileFindingTime = 0L override def start() { } @@ -68,54 +120,113 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas * the previous call. */ override def compute(validTime: Time): Option[RDD[(K, V)]] = { - assert(validTime.milliseconds >= ignoreTime, - "Trying to get new files for a really old time [" + validTime + " < " + ignoreTime + "]") - // Find new files - val (newFiles, minNewFileModTime) = findNewFiles(validTime.milliseconds) + val newFiles = findNewFiles(validTime.milliseconds) logInfo("New files at time " + validTime + ":\n" + newFiles.mkString("\n")) - if (!newFiles.isEmpty) { - lastFoundFiles.clear() - lastFoundFiles ++= newFiles - ignoreTime = minNewFileModTime - } - files += ((validTime, newFiles.toArray)) + batchTimeToSelectedFiles += ((validTime, newFiles)) + recentlySelectedFiles ++= newFiles Some(filesToRDD(newFiles)) } /** Clear the old time-to-files mappings along with old RDDs */ protected[streaming] override def clearMetadata(time: Time) { super.clearMetadata(time) - val oldFiles = files.filter(_._1 < (time - rememberDuration)) - files --= oldFiles.keys + val oldFiles = batchTimeToSelectedFiles.filter(_._1 < (time - rememberDuration)) + batchTimeToSelectedFiles --= oldFiles.keys + recentlySelectedFiles --= oldFiles.values.flatten logInfo("Cleared " + oldFiles.size + " old files that were older than " + (time - rememberDuration) + ": " + oldFiles.keys.mkString(", ")) logDebug("Cleared files are:\n" + oldFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n")) // Delete file mod times that weren't accessed in the last round of getting new files - fileModTimes.clearOldValues(lastNewFileFindingTime - 1) + fileToModTime.clearOldValues(lastNewFileFindingTime - 1) } /** - * Find files which have modification timestamp <= current time and return a 3-tuple of - * (new files found, latest modification time among them, files with latest modification time) + * Find new files for the batch of `currentTime`. This is done by first calculating the + * ignore threshold for file mod times, and then getting a list of files filtered based on + * the current batch time and the ignore threshold. The ignore threshold is the max of + * initial ignore threshold and the trailing end of the remember window (that is, which ever + * is later in time). */ - private def findNewFiles(currentTime: Long): (Seq[String], Long) = { - logDebug("Trying to get new files for time " + currentTime) - lastNewFileFindingTime = System.currentTimeMillis - val filter = new CustomPathFilter(currentTime) - val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) - val timeTaken = System.currentTimeMillis - lastNewFileFindingTime - logInfo("Finding new files took " + timeTaken + " ms") - logDebug("# cached file times = " + fileModTimes.size) - if (timeTaken > slideDuration.milliseconds) { - logWarning( - "Time taken to find new files exceeds the batch size. " + - "Consider increasing the batch size or reduceing the number of " + - "files in the monitored directory." + private def findNewFiles(currentTime: Long): Array[String] = { + try { + lastNewFileFindingTime = System.currentTimeMillis + + // Calculate ignore threshold + val modTimeIgnoreThreshold = math.max( + initialModTimeIgnoreThreshold, // initial threshold based on newFilesOnly setting + currentTime - durationToRemember.milliseconds // trailing end of the remember window ) + logDebug(s"Getting new files for time $currentTime, " + + s"ignoring files older than $modTimeIgnoreThreshold") + val filter = new PathFilter { + def accept(path: Path): Boolean = isNewFile(path, currentTime, modTimeIgnoreThreshold) + } + val newFiles = fs.listStatus(directoryPath, filter).map(_.getPath.toString) + val timeTaken = System.currentTimeMillis - lastNewFileFindingTime + logInfo("Finding new files took " + timeTaken + " ms") + logDebug("# cached file times = " + fileToModTime.size) + if (timeTaken > slideDuration.milliseconds) { + logWarning( + "Time taken to find new files exceeds the batch size. " + + "Consider increasing the batch size or reducing the number of " + + "files in the monitored directory." + ) + } + newFiles + } catch { + case e: Exception => + logWarning("Error finding new files", e) + reset() + Array.empty + } + } + + /** + * Identify whether the given `path` is a new file for the batch of `currentTime`. For it to be + * accepted, it has to pass the following criteria. + * - It must pass the user-provided file filter. + * - It must be newer than the ignore threshold. It is assumed that files older than the ignore + * threshold have already been considered or are existing files before start + * (when newFileOnly = true). + * - It must not be present in the recently selected files that this class remembers. + * - It must not be newer than the time of the batch (i.e. `currentTime` for which this + * file is being tested. This can occur if the driver was recovered, and the missing batches + * (during downtime) are being generated. In that case, a batch of time T may be generated + * at time T+x. Say x = 5. If that batch T contains file of mod time T+5, then bad things can + * happen. Let's say the selected files are remembered for 60 seconds. At time t+61, + * the batch of time t is forgotten, and the ignore threshold is still T+1. + * The files with mod time T+5 are not remembered and cannot be ignored (since, t+5 > t+1). + * Hence they can get selected as new files again. To prevent this, files whose mod time is more + * than current batch time are not considered. + */ + private def isNewFile(path: Path, currentTime: Long, modTimeIgnoreThreshold: Long): Boolean = { + val pathStr = path.toString + // Reject file if it does not satisfy filter + if (!filter(path)) { + logDebug(s"$pathStr rejected by filter") + return false + } + // Reject file if it was created before the ignore time + val modTime = getFileModTime(path) + if (modTime <= modTimeIgnoreThreshold) { + // Use <= instead of < to avoid SPARK-4518 + logDebug(s"$pathStr ignored as mod time $modTime <= ignore time $modTimeIgnoreThreshold") + return false } - (newFiles, filter.minNewFileModTime) + // Reject file if mod time > current batch time + if (modTime > currentTime) { + logDebug(s"$pathStr not selected as mod time $modTime > current time $currentTime") + return false + } + // Reject file if it was considered earlier + if (recentlySelectedFiles.contains(pathStr)) { + logDebug(s"$pathStr already considered") + return false + } + logDebug(s"$pathStr accepted with mod time $modTime") + return true } /** Generate one RDD from an array of files */ @@ -132,21 +243,21 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas new UnionRDD(context.sparkContext, fileRDDs) } + /** Get file mod time from cache or fetch it from the file system */ + private def getFileModTime(path: Path) = { + fileToModTime.getOrElseUpdate(path.toString, fs.getFileStatus(path).getModificationTime()) + } + private def directoryPath: Path = { if (path_ == null) path_ = new Path(directory) path_ } private def fs: FileSystem = { - if (fs_ == null) fs_ = directoryPath.getFileSystem(new Configuration()) + if (fs_ == null) fs_ = directoryPath.getFileSystem(ssc.sparkContext.hadoopConfiguration) fs_ } - private def getFileModTime(path: Path) = { - // Get file mod time from cache or fetch it from the file system - fileModTimes.getOrElseUpdate(path.toString, fs.getFileStatus(path).getModificationTime()) - } - private def reset() { fs_ = null } @@ -155,9 +266,10 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas private def readObject(ois: ObjectInputStream): Unit = Utils.tryOrIOException { logDebug(this.getClass().getSimpleName + ".readObject used") ois.defaultReadObject() - generatedRDDs = new HashMap[Time, RDD[(K,V)]] () - files = new HashMap[Time, Array[String]] - fileModTimes = new TimeStampedHashMap[String, Long](true) + generatedRDDs = new mutable.HashMap[Time, RDD[(K,V)]] () + batchTimeToSelectedFiles = new mutable.HashMap[Time, Array[String]]() + recentlySelectedFiles = new mutable.HashSet[String]() + fileToModTime = new TimeStampedHashMap[String, Long](true) } /** @@ -167,11 +279,11 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas private[streaming] class FileInputDStreamCheckpointData extends DStreamCheckpointData(this) { - def hadoopFiles = data.asInstanceOf[HashMap[Time, Array[String]]] + def hadoopFiles = data.asInstanceOf[mutable.HashMap[Time, Array[String]]] override def update(time: Time) { hadoopFiles.clear() - hadoopFiles ++= files + hadoopFiles ++= batchTimeToSelectedFiles } override def cleanup(time: Time) { } @@ -182,7 +294,8 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas // Restore the metadata in both files and generatedRDDs logInfo("Restoring files for time " + t + " - " + f.mkString("[", ", ", "]") ) - files += ((t, f)) + batchTimeToSelectedFiles += ((t, f)) + recentlySelectedFiles ++= f generatedRDDs += ((t, filesToRDD(f))) } } @@ -193,57 +306,25 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas hadoopFiles.map(p => (p._1, p._2.mkString(", "))).mkString("\n") + "\n]" } } +} + +private[streaming] +object FileInputDStream { /** - * Custom PathFilter class to find new files that - * ... have modification time more than ignore time - * ... have not been seen in the last interval - * ... have modification time less than maxModTime + * Minimum duration of remembering the information of selected files. Files with mod times + * older than this "window" of remembering will be ignored. So if new files are visible + * within this window, then the file will get selected in the next batch. */ - private[streaming] - class CustomPathFilter(maxModTime: Long) extends PathFilter { + private val MIN_REMEMBER_DURATION = Minutes(1) - // Minimum of the mod times of new files found in the current interval - var minNewFileModTime = -1L + def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") - def accept(path: Path): Boolean = { - try { - if (!filter(path)) { // Reject file if it does not satisfy filter - logDebug("Rejected by filter " + path) - return false - } - // Reject file if it was found in the last interval - if (lastFoundFiles.contains(path.toString)) { - logDebug("Mod time equal to last mod time, but file considered already") - return false - } - val modTime = getFileModTime(path) - logDebug("Mod time for " + path + " is " + modTime) - if (modTime < ignoreTime) { - // Reject file if it was created before the ignore time (or, before last interval) - logDebug("Mod time " + modTime + " less than ignore time " + ignoreTime) - return false - } else if (modTime > maxModTime) { - // Reject file if it is too new that considering it may give errors - logDebug("Mod time more than ") - return false - } - if (minNewFileModTime < 0 || modTime < minNewFileModTime) { - minNewFileModTime = modTime - } - logDebug("Accepted " + path) - } catch { - case fnfe: java.io.FileNotFoundException => - logWarning("Error finding new files", fnfe) - reset() - return false - } - true - } + /** + * Calculate the number of last batches to remember, such that all the files selected in + * at least last MIN_REMEMBER_DURATION duration can be remembered. + */ + def calculateNumBatchesToRemember(batchDuration: Duration): Int = { + math.ceil(MIN_REMEMBER_DURATION.milliseconds.toDouble / batchDuration.milliseconds).toInt } } - -private[streaming] -object FileInputDStream { - def defaultFilter(path: Path): Boolean = !path.getName().startsWith(".") -} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index e5592e52b0d2d..77ff1ca780a58 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -265,7 +265,7 @@ class CheckpointSuite extends TestSuiteBase { // Verify whether files created have been recorded correctly or not var fileInputDStream = ssc.graph.getInputStreams().head.asInstanceOf[FileInputDStream[_, _, _]] - def recordedFiles = fileInputDStream.files.values.flatMap(x => x) + def recordedFiles = fileInputDStream.batchTimeToSelectedFiles.values.flatten assert(!recordedFiles.filter(_.endsWith("1")).isEmpty) assert(!recordedFiles.filter(_.endsWith("2")).isEmpty) assert(!recordedFiles.filter(_.endsWith("3")).isEmpty) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index fa04fa326e370..307052a4a9cbb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -28,9 +28,12 @@ import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue} import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer, SynchronizedQueue} +import scala.concurrent.duration._ +import scala.language.postfixOps import com.google.common.io.Files import org.scalatest.BeforeAndAfter +import org.scalatest.concurrent.Eventually._ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel @@ -38,6 +41,9 @@ import org.apache.spark.streaming.util.ManualClock import org.apache.spark.util.Utils import org.apache.spark.streaming.receiver.{ActorHelper, Receiver} import org.apache.spark.rdd.RDD +import org.apache.hadoop.io.{Text, LongWritable} +import org.apache.hadoop.mapreduce.lib.input.TextInputFormat +import org.apache.hadoop.fs.Path class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { @@ -91,54 +97,12 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } - test("file input stream") { - // Disable manual clock as FileInputDStream does not work with manual clock - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") - - // Set up the streaming context and input streams - val testDir = Utils.createTempDir() - val ssc = new StreamingContext(conf, batchDuration) - val fileStream = ssc.textFileStream(testDir.toString) - val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] - def output = outputBuffer.flatMap(x => x) - val outputStream = new TestOutputStream(fileStream, outputBuffer) - outputStream.register() - ssc.start() - - // Create files in the temporary directory so that Spark Streaming can read data from it - val input = Seq(1, 2, 3, 4, 5) - val expectedOutput = input.map(_.toString) - Thread.sleep(1000) - for (i <- 0 until input.size) { - val file = new File(testDir, i.toString) - Files.write(input(i) + "\n", file, Charset.forName("UTF-8")) - logInfo("Created file " + file) - Thread.sleep(batchDuration.milliseconds) - Thread.sleep(1000) - } - val startTime = System.currentTimeMillis() - Thread.sleep(1000) - val timeTaken = System.currentTimeMillis() - startTime - assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") - logInfo("Stopping context") - ssc.stop() - - // Verify whether data received by Spark Streaming was as expected - logInfo("--------------------------------") - logInfo("output, size = " + outputBuffer.size) - outputBuffer.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("expected output, size = " + expectedOutput.size) - expectedOutput.foreach(x => logInfo("[" + x.mkString(",") + "]")) - logInfo("--------------------------------") - - // Verify whether all the elements received are as expected - // (whether the elements were received one in each interval is not verified) - assert(output.toList === expectedOutput.toList) - - Utils.deleteRecursively(testDir) + test("file input stream - newFilesOnly = true") { + testFileStream(newFilesOnly = true) + } - // Enable manual clock back again for other tests - conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") + test("file input stream - newFilesOnly = false") { + testFileStream(newFilesOnly = false) } test("multi-thread receiver") { @@ -180,7 +144,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(output.sum === numTotalRecords) } - test("queue input stream - oneAtATime=true") { + test("queue input stream - oneAtATime = true") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val queue = new SynchronizedQueue[RDD[String]]() @@ -223,7 +187,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { } } - test("queue input stream - oneAtATime=false") { + test("queue input stream - oneAtATime = false") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val queue = new SynchronizedQueue[RDD[String]]() @@ -268,6 +232,50 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { assert(output(i) === expectedOutput(i)) } } + + def testFileStream(newFilesOnly: Boolean) { + var ssc: StreamingContext = null + val testDir: File = null + try { + val testDir = Utils.createTempDir() + val existingFile = new File(testDir, "0") + Files.write("0\n", existingFile, Charset.forName("UTF-8")) + + Thread.sleep(1000) + // Set up the streaming context and input streams + val newConf = conf.clone.set( + "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") + ssc = new StreamingContext(newConf, batchDuration) + val fileStream = ssc.fileStream[LongWritable, Text, TextInputFormat]( + testDir.toString, (x: Path) => true, newFilesOnly = newFilesOnly).map(_._2.toString) + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(fileStream, outputBuffer) + outputStream.register() + ssc.start() + + // Create files in the directory + val input = Seq(1, 2, 3, 4, 5) + input.foreach { i => + Thread.sleep(batchDuration.milliseconds) + val file = new File(testDir, i.toString) + Files.write(i + "\n", file, Charset.forName("UTF-8")) + logInfo("Created file " + file) + } + + // Verify that all the files have been read + val expectedOutput = if (newFilesOnly) { + input.map(_.toString).toSet + } else { + (Seq(0) ++ input).map(_.toString).toSet + } + eventually(timeout(maxWaitTimeMillis milliseconds), interval(100 milliseconds)) { + assert(outputBuffer.flatten.toSet === expectedOutput) + } + } finally { + if (ssc != null) ssc.stop() + if (testDir != null) Utils.deleteRecursively(testDir) + } + } } From b660de7a9cbdea3df4a37fbcf60c1c33c71782b8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 24 Nov 2014 16:37:14 -0800 Subject: [PATCH 013/305] [SPARK-4562] [MLlib] speedup vector This PR change the underline array of DenseVector to numpy.ndarray to avoid the conversion, because most of the users will using numpy.array. It also improve the serialization of DenseVector. Before this change: trial | trainingTime | testTime -------|--------|-------- 0 | 5.126 | 1.786 1 |2.698 |1.693 After the change: trial | trainingTime | testTime -------|--------|-------- 0 |4.692 |0.554 1 |2.307 |0.525 This could partially fix the performance regression during test. Author: Davies Liu Closes #3420 from davies/ser2 and squashes the following commits: 0e1e6f3 [Davies Liu] fix tests 426f5db [Davies Liu] impove toArray() 44707ec [Davies Liu] add name for ISO-8859-1 fa7d791 [Davies Liu] address comments 1cfb137 [Davies Liu] handle zero sparse vector 2548ee2 [Davies Liu] fix tests 9e6389d [Davies Liu] bugfix 470f702 [Davies Liu] speed up DenseMatrix f0d3c40 [Davies Liu] speedup SparseVector ef6ce70 [Davies Liu] speed up dense vector --- .../mllib/api/python/PythonMLLibAPI.scala | 73 +++++++++++++++++-- python/pyspark/mllib/linalg.py | 73 ++++++++++++------- python/pyspark/mllib/tests.py | 6 +- 3 files changed, 118 insertions(+), 34 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index f04df1c156898..9f20cd5d00dcd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.api.python import java.io.OutputStream +import java.nio.{ByteBuffer, ByteOrder} import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ @@ -684,6 +685,7 @@ class PythonMLLibAPI extends Serializable { private[spark] object SerDe extends Serializable { val PYSPARK_PACKAGE = "pyspark.mllib" + val LATIN1 = "ISO-8859-1" /** * Base class used for pickle @@ -735,7 +737,16 @@ private[spark] object SerDe extends Serializable { def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { val vector: DenseVector = obj.asInstanceOf[DenseVector] - saveObjects(out, pickler, vector.toArray) + val bytes = new Array[Byte](8 * vector.size) + val bb = ByteBuffer.wrap(bytes) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + db.put(vector.values) + + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.TUPLE1) } def construct(args: Array[Object]): Object = { @@ -743,7 +754,13 @@ private[spark] object SerDe extends Serializable { if (args.length != 1) { throw new PickleException("should be 1") } - new DenseVector(args(0).asInstanceOf[Array[Double]]) + val bytes = args(0).asInstanceOf[String].getBytes(LATIN1) + val bb = ByteBuffer.wrap(bytes, 0, bytes.length) + bb.order(ByteOrder.nativeOrder()) + val db = bb.asDoubleBuffer() + val ans = new Array[Double](bytes.length / 8) + db.get(ans) + Vectors.dense(ans) } } @@ -752,15 +769,30 @@ private[spark] object SerDe extends Serializable { def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { val m: DenseMatrix = obj.asInstanceOf[DenseMatrix] - saveObjects(out, pickler, m.numRows, m.numCols, m.values) + val bytes = new Array[Byte](8 * m.values.size) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().put(m.values) + + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numRows)) + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(m.numCols)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(bytes.length)) + out.write(bytes) + out.write(Opcodes.TUPLE3) } def construct(args: Array[Object]): Object = { if (args.length != 3) { throw new PickleException("should be 3") } - new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], - args(2).asInstanceOf[Array[Double]]) + val bytes = args(2).asInstanceOf[String].getBytes(LATIN1) + val n = bytes.length / 8 + val values = new Array[Double](n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(bytes).order(order).asDoubleBuffer().get(values) + new DenseMatrix(args(0).asInstanceOf[Int], args(1).asInstanceOf[Int], values) } } @@ -769,15 +801,40 @@ private[spark] object SerDe extends Serializable { def saveState(obj: Object, out: OutputStream, pickler: Pickler) = { val v: SparseVector = obj.asInstanceOf[SparseVector] - saveObjects(out, pickler, v.size, v.indices, v.values) + val n = v.indices.size + val indiceBytes = new Array[Byte](4 * n) + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().put(v.indices) + val valueBytes = new Array[Byte](8 * n) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().put(v.values) + + out.write(Opcodes.BININT) + out.write(PickleUtils.integer_to_bytes(v.size)) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(indiceBytes.length)) + out.write(indiceBytes) + out.write(Opcodes.BINSTRING) + out.write(PickleUtils.integer_to_bytes(valueBytes.length)) + out.write(valueBytes) + out.write(Opcodes.TUPLE3) } def construct(args: Array[Object]): Object = { if (args.length != 3) { throw new PickleException("should be 3") } - new SparseVector(args(0).asInstanceOf[Int], args(1).asInstanceOf[Array[Int]], - args(2).asInstanceOf[Array[Double]]) + val size = args(0).asInstanceOf[Int] + val indiceBytes = args(1).asInstanceOf[String].getBytes(LATIN1) + val valueBytes = args(2).asInstanceOf[String].getBytes(LATIN1) + val n = indiceBytes.length / 4 + val indices = new Array[Int](n) + val values = new Array[Double](n) + if (n > 0) { + val order = ByteOrder.nativeOrder() + ByteBuffer.wrap(indiceBytes).order(order).asIntBuffer().get(indices) + ByteBuffer.wrap(valueBytes).order(order).asDoubleBuffer().get(values) + } + new SparseVector(size, indices, values) } } diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 537b17657809c..f7aa2b0cb04b3 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -30,7 +30,7 @@ import numpy as np from pyspark.sql import UserDefinedType, StructField, StructType, ArrayType, DoubleType, \ - IntegerType, ByteType, Row + IntegerType, ByteType __all__ = ['Vector', 'DenseVector', 'SparseVector', 'Vectors', 'DenseMatrix', 'Matrices'] @@ -173,12 +173,16 @@ class DenseVector(Vector): A dense vector represented by a value array. """ def __init__(self, ar): - if not isinstance(ar, array.array): - ar = array.array('d', ar) + if isinstance(ar, basestring): + ar = np.frombuffer(ar, dtype=np.float64) + elif not isinstance(ar, np.ndarray): + ar = np.array(ar, dtype=np.float64) + if ar.dtype != np.float64: + ar.astype(np.float64) self.array = ar def __reduce__(self): - return DenseVector, (self.array,) + return DenseVector, (self.array.tostring(),) def dot(self, other): """ @@ -207,9 +211,10 @@ def dot(self, other): ... AssertionError: dimension mismatch """ - if type(other) == np.ndarray and other.ndim > 1: - assert len(self) == other.shape[0], "dimension mismatch" - return np.dot(self.toArray(), other) + if type(other) == np.ndarray: + if other.ndim > 1: + assert len(self) == other.shape[0], "dimension mismatch" + return np.dot(self.array, other) elif _have_scipy and scipy.sparse.issparse(other): assert len(self) == other.shape[0], "dimension mismatch" return other.transpose().dot(self.toArray()) @@ -261,7 +266,7 @@ def squared_distance(self, other): return np.dot(diff, diff) def toArray(self): - return np.array(self.array) + return self.array def __getitem__(self, item): return self.array[item] @@ -276,7 +281,7 @@ def __repr__(self): return "DenseVector([%s])" % (', '.join(_format_float(i) for i in self.array)) def __eq__(self, other): - return isinstance(other, DenseVector) and self.array == other.array + return isinstance(other, DenseVector) and np.array_equal(self.array, other.array) def __ne__(self, other): return not self == other @@ -314,18 +319,28 @@ def __init__(self, size, *args): if type(pairs) == dict: pairs = pairs.items() pairs = sorted(pairs) - self.indices = array.array('i', [p[0] for p in pairs]) - self.values = array.array('d', [p[1] for p in pairs]) + self.indices = np.array([p[0] for p in pairs], dtype=np.int32) + self.values = np.array([p[1] for p in pairs], dtype=np.float64) else: - assert len(args[0]) == len(args[1]), "index and value arrays not same length" - self.indices = array.array('i', args[0]) - self.values = array.array('d', args[1]) + if isinstance(args[0], basestring): + assert isinstance(args[1], str), "values should be string too" + if args[0]: + self.indices = np.frombuffer(args[0], np.int32) + self.values = np.frombuffer(args[1], np.float64) + else: + # np.frombuffer() doesn't work well with empty string in older version + self.indices = np.array([], dtype=np.int32) + self.values = np.array([], dtype=np.float64) + else: + self.indices = np.array(args[0], dtype=np.int32) + self.values = np.array(args[1], dtype=np.float64) + assert len(self.indices) == len(self.values), "index and value arrays not same length" for i in xrange(len(self.indices) - 1): if self.indices[i] >= self.indices[i + 1]: raise TypeError("indices array must be sorted") def __reduce__(self): - return (SparseVector, (self.size, self.indices, self.values)) + return (SparseVector, (self.size, self.indices.tostring(), self.values.tostring())) def dot(self, other): """ @@ -461,8 +476,7 @@ def toArray(self): Returns a copy of this SparseVector as a 1-dimensional NumPy array. """ arr = np.zeros((self.size,), dtype=np.float64) - for i in xrange(len(self.indices)): - arr[self.indices[i]] = self.values[i] + arr[self.indices] = self.values return arr def __len__(self): @@ -493,8 +507,8 @@ def __eq__(self, other): """ return (isinstance(other, self.__class__) and other.size == self.size - and other.indices == self.indices - and other.values == self.values) + and np.array_equal(other.indices, self.indices) + and np.array_equal(other.values, self.values)) def __ne__(self, other): return not self.__eq__(other) @@ -577,25 +591,34 @@ class DenseMatrix(Matrix): """ def __init__(self, numRows, numCols, values): Matrix.__init__(self, numRows, numCols) + if isinstance(values, basestring): + values = np.frombuffer(values, dtype=np.float64) + elif not isinstance(values, np.ndarray): + values = np.array(values, dtype=np.float64) assert len(values) == numRows * numCols - if not isinstance(values, array.array): - values = array.array('d', values) + if values.dtype != np.float64: + values.astype(np.float64) self.values = values def __reduce__(self): - return DenseMatrix, (self.numRows, self.numCols, self.values) + return DenseMatrix, (self.numRows, self.numCols, self.values.tostring()) def toArray(self): """ Return an numpy.ndarray - >>> arr = array.array('d', [float(i) for i in range(4)]) - >>> m = DenseMatrix(2, 2, arr) + >>> m = DenseMatrix(2, 2, range(4)) >>> m.toArray() array([[ 0., 2.], [ 1., 3.]]) """ - return np.reshape(self.values, (self.numRows, self.numCols), order='F') + return self.values.reshape((self.numRows, self.numCols), order='F') + + def __eq__(self, other): + return (isinstance(other, DenseMatrix) and + self.numRows == other.numRows and + self.numCols == other.numCols and + all(self.values == other.values)) class Matrices(object): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 9fa4d6f6a2f5f..8332f8e061f48 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -33,7 +33,8 @@ else: import unittest -from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector +from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector,\ + DenseMatrix from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.random import RandomRDDs from pyspark.mllib.stat import Statistics @@ -62,6 +63,7 @@ def _squared_distance(a, b): class VectorTests(PySparkTestCase): def _test_serialize(self, v): + self.assertEqual(v, ser.loads(ser.dumps(v))) jvec = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(v))) nv = ser.loads(str(self.sc._jvm.SerDe.dumps(jvec))) self.assertEqual(v, nv) @@ -75,6 +77,8 @@ def test_serialize(self): self._test_serialize(DenseVector(array([1., 2., 3., 4.]))) self._test_serialize(DenseVector(pyarray.array('d', range(10)))) self._test_serialize(SparseVector(4, {1: 1, 3: 2})) + self._test_serialize(SparseVector(3, {})) + self._test_serialize(DenseMatrix(2, 3, range(6))) def test_dot(self): sv = SparseVector(4, {1: 1, 3: 2}) From 050616b408c60eae02256913ceb645912dbff62e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 24 Nov 2014 16:41:23 -0800 Subject: [PATCH 014/305] [SPARK-4578] fix asDict() with nested Row() The Row object is created on the fly once the field is accessed, so we should access them by getattr() in asDict(0 Author: Davies Liu Closes #3434 from davies/fix_asDict and squashes the following commits: b20f1e7 [Davies Liu] fix asDict() with nested Row() --- python/pyspark/sql.py | 2 +- python/pyspark/tests.py | 7 ++++--- 2 files changed, 5 insertions(+), 4 deletions(-) diff --git a/python/pyspark/sql.py b/python/pyspark/sql.py index abb284d1e3dd9..ae288471b0e51 100644 --- a/python/pyspark/sql.py +++ b/python/pyspark/sql.py @@ -1178,7 +1178,7 @@ class Row(tuple): def asDict(self): """ Return as a dict """ - return dict(zip(self.__FIELDS__, self)) + return dict((n, getattr(self, n)) for n in self.__FIELDS__) def __repr__(self): # call collect __repr__ for nested objects diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index a01bd8d415787..29bcd38908d10 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -803,7 +803,7 @@ def setUpClass(cls): @classmethod def tearDownClass(cls): ReusedPySparkTestCase.tearDownClass() - shutil.rmtree(cls.tempdir.name) + shutil.rmtree(cls.tempdir.name, ignore_errors=True) def setUp(self): self.sqlCtx = SQLContext(self.sc) @@ -930,8 +930,9 @@ def test_convert_row_to_dict(self): rdd = self.sc.parallelize([row]) srdd = self.sqlCtx.inferSchema(rdd) srdd.registerTempTable("test") - row = self.sqlCtx.sql("select l[0].a AS la from test").first() - self.assertEqual(1, row.asDict()["la"]) + row = self.sqlCtx.sql("select l, d from test").first() + self.assertEqual(1, row.asDict()["l"][0].a) + self.assertEqual(1.0, row.asDict()['d']['key'].c) def test_infer_schema_with_udt(self): from pyspark.tests import ExamplePoint, ExamplePointUDT From 6cf507685efd01df77d663145ae08e48c7f92948 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Mon, 24 Nov 2014 17:17:03 -0800 Subject: [PATCH 015/305] [SPARK-4548] []SPARK-4517] improve performance of python broadcast Re-implement the Python broadcast using file: 1) serialize the python object using cPickle, write into disks. 2) Create a wrapper in JVM (for the dumped file), it read data from during serialization 3) Using TorrentBroadcast or HttpBroadcast to transfer the data (compressed) into executors 4) During deserialization, writing the data into disk. 5) Passing the path into Python worker, read data from disk and unpickle it into python object, until the first access. It fixes the performance regression introduced in #2659, has similar performance as 1.1, but support object larger than 2G, also improve the memory efficiency (only one compressed copy in driver and executor). Testing with a 500M broadcast and 4 tasks (excluding the benefit from reused worker in 1.2): name | 1.1 | 1.2 with this patch | improvement ---------|--------|---------|-------- python-broadcast-w-bytes | 25.20 | 9.33 | 170.13% | python-broadcast-w-set | 4.13 | 4.50 | -8.35% | Testing with 100 tasks (16 CPUs): name | 1.1 | 1.2 with this patch | improvement ---------|--------|---------|-------- python-broadcast-w-bytes | 38.16 | 8.40 | 353.98% python-broadcast-w-set | 23.29 | 9.59 | 142.80% Author: Davies Liu Closes #3417 from davies/pybroadcast and squashes the following commits: 50a58e0 [Davies Liu] address comments b98de1d [Davies Liu] disable gc while unpickle e5ee6b9 [Davies Liu] support large string 09303b8 [Davies Liu] read all data into memory dde02dd [Davies Liu] improve performance of python broadcast --- .../apache/spark/api/python/PythonRDD.scala | 73 ++++--- python/pyspark/broadcast.py | 95 +++++++--- python/pyspark/context.py | 12 +- python/pyspark/serializers.py | 178 +----------------- python/pyspark/tests.py | 18 +- python/pyspark/worker.py | 10 +- .../apache/spark/sql/UdfRegistration.scala | 3 +- .../spark/sql/execution/pythonUdfs.scala | 4 +- 8 files changed, 135 insertions(+), 258 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index b80c771d58a8f..e0bc00e1eb249 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -19,7 +19,7 @@ package org.apache.spark.api.python import java.io._ import java.net._ -import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} +import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, UUID, Collections} import org.apache.spark.input.PortableDataStream @@ -47,7 +47,7 @@ private[spark] class PythonRDD( pythonIncludes: JList[String], preservePartitoning: Boolean, pythonExec: String, - broadcastVars: JList[Broadcast[Array[Array[Byte]]]], + broadcastVars: JList[Broadcast[PythonBroadcast]], accumulator: Accumulator[JList[Array[Byte]]]) extends RDD[Array[Byte]](parent) { @@ -230,8 +230,7 @@ private[spark] class PythonRDD( if (!oldBids.contains(broadcast.id)) { // send new broadcast dataOut.writeLong(broadcast.id) - dataOut.writeLong(broadcast.value.map(_.length.toLong).sum) - broadcast.value.foreach(dataOut.write) + PythonRDD.writeUTF(broadcast.value.path, dataOut) oldBids.add(broadcast.id) } } @@ -368,24 +367,8 @@ private[spark] object PythonRDD extends Logging { } } - def readBroadcastFromFile( - sc: JavaSparkContext, - filename: String): Broadcast[Array[Array[Byte]]] = { - val size = new File(filename).length() - val file = new DataInputStream(new FileInputStream(filename)) - val blockSize = 1 << 20 - val n = ((size + blockSize - 1) / blockSize).toInt - val obj = new Array[Array[Byte]](n) - try { - for (i <- 0 until n) { - val length = if (i < (n - 1)) blockSize else (size % blockSize).toInt - obj(i) = new Array[Byte](length) - file.readFully(obj(i)) - } - } finally { - file.close() - } - sc.broadcast(obj) + def readBroadcastFromFile(sc: JavaSparkContext, path: String): Broadcast[PythonBroadcast] = { + sc.broadcast(new PythonBroadcast(path)) } def writeIteratorToStream[T](iter: Iterator[T], dataOut: DataOutputStream) { @@ -824,3 +807,49 @@ private class PythonAccumulatorParam(@transient serverHost: String, serverPort: } } } + +/** + * An Wrapper for Python Broadcast, which is written into disk by Python. It also will + * write the data into disk after deserialization, then Python can read it from disks. + */ +private[spark] class PythonBroadcast(@transient var path: String) extends Serializable { + + /** + * Read data from disks, then copy it to `out` + */ + private def writeObject(out: ObjectOutputStream): Unit = Utils.tryOrIOException { + val in = new FileInputStream(new File(path)) + try { + Utils.copyStream(in, out) + } finally { + in.close() + } + } + + /** + * Write data into disk, using randomly generated name. + */ + private def readObject(in: ObjectInputStream): Unit = Utils.tryOrIOException { + val dir = new File(Utils.getLocalDir(SparkEnv.get.conf)) + val file = File.createTempFile("broadcast", "", dir) + path = file.getAbsolutePath + val out = new FileOutputStream(file) + try { + Utils.copyStream(in, out) + } finally { + out.close() + } + } + + /** + * Delete the file once the object is GCed. + */ + override def finalize() { + if (!path.isEmpty) { + val file = new File(path) + if (file.exists()) { + file.delete() + } + } + } +} diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 01cac3c72c690..6b8a8b256a891 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -15,21 +15,10 @@ # limitations under the License. # -""" ->>> from pyspark.context import SparkContext ->>> sc = SparkContext('local', 'test') ->>> b = sc.broadcast([1, 2, 3, 4, 5]) ->>> b.value -[1, 2, 3, 4, 5] ->>> sc.parallelize([0, 0]).flatMap(lambda x: b.value).collect() -[1, 2, 3, 4, 5, 1, 2, 3, 4, 5] ->>> b.unpersist() - ->>> large_broadcast = sc.broadcast(list(range(10000))) -""" import os - -from pyspark.serializers import LargeObjectSerializer +import cPickle +import gc +from tempfile import NamedTemporaryFile __all__ = ['Broadcast'] @@ -49,44 +38,88 @@ def _from_id(bid): class Broadcast(object): """ - A broadcast variable created with - L{SparkContext.broadcast()}. + A broadcast variable created with L{SparkContext.broadcast()}. Access its value through C{.value}. + + Examples: + + >>> from pyspark.context import SparkContext + >>> sc = SparkContext('local', 'test') + >>> b = sc.broadcast([1, 2, 3, 4, 5]) + >>> b.value + [1, 2, 3, 4, 5] + >>> sc.parallelize([0, 0]).flatMap(lambda x: b.value).collect() + [1, 2, 3, 4, 5, 1, 2, 3, 4, 5] + >>> b.unpersist() + + >>> large_broadcast = sc.broadcast(range(10000)) """ - def __init__(self, bid, value, java_broadcast=None, - pickle_registry=None, path=None): + def __init__(self, sc=None, value=None, pickle_registry=None, path=None): """ - Should not be called directly by users -- use - L{SparkContext.broadcast()} + Should not be called directly by users -- use L{SparkContext.broadcast()} instead. """ - self.bid = bid - if path is None: - self._value = value - self._jbroadcast = java_broadcast - self._pickle_registry = pickle_registry - self.path = path + if sc is not None: + f = NamedTemporaryFile(delete=False, dir=sc._temp_dir) + self._path = self.dump(value, f) + self._jbroadcast = sc._jvm.PythonRDD.readBroadcastFromFile(sc._jsc, self._path) + self._pickle_registry = pickle_registry + else: + self._jbroadcast = None + self._path = path + + def dump(self, value, f): + if isinstance(value, basestring): + if isinstance(value, unicode): + f.write('U') + value = value.encode('utf8') + else: + f.write('S') + f.write(value) + else: + f.write('P') + cPickle.dump(value, f, 2) + f.close() + return f.name + + def load(self, path): + with open(path, 'rb', 1 << 20) as f: + flag = f.read(1) + data = f.read() + if flag == 'P': + # cPickle.loads() may create lots of objects, disable GC + # temporary for better performance + gc.disable() + try: + return cPickle.loads(data) + finally: + gc.enable() + else: + return data.decode('utf8') if flag == 'U' else data @property def value(self): """ Return the broadcasted value """ - if not hasattr(self, "_value") and self.path is not None: - ser = LargeObjectSerializer() - self._value = ser.load_stream(open(self.path)).next() + if not hasattr(self, "_value") and self._path is not None: + self._value = self.load(self._path) return self._value def unpersist(self, blocking=False): """ Delete cached copies of this broadcast on the executors. """ + if self._jbroadcast is None: + raise Exception("Broadcast can only be unpersisted in driver") self._jbroadcast.unpersist(blocking) - os.unlink(self.path) + os.unlink(self._path) def __reduce__(self): + if self._jbroadcast is None: + raise Exception("Broadcast can only be serialized in driver") self._pickle_registry.add(self) - return (_from_id, (self.bid, )) + return _from_id, (self._jbroadcast.id(),) if __name__ == "__main__": diff --git a/python/pyspark/context.py b/python/pyspark/context.py index ec67ec8d0f824..ed7351d60cff2 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -29,7 +29,7 @@ from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ - PairDeserializer, AutoBatchedSerializer, NoOpSerializer, LargeObjectSerializer + PairDeserializer, AutoBatchedSerializer, NoOpSerializer from pyspark.storagelevel import StorageLevel from pyspark.rdd import RDD from pyspark.traceback_utils import CallSite, first_spark_call @@ -624,15 +624,7 @@ def broadcast(self, value): object for reading it in distributed functions. The variable will be sent to each cluster only once. """ - ser = LargeObjectSerializer() - - # pass large object by py4j is very slow and need much memory - tempFile = NamedTemporaryFile(delete=False, dir=self._temp_dir) - ser.dump_stream([value], tempFile) - tempFile.close() - jbroadcast = self._jvm.PythonRDD.readBroadcastFromFile(self._jsc, tempFile.name) - return Broadcast(jbroadcast.id(), None, jbroadcast, - self._pickled_broadcast_vars, tempFile.name) + return Broadcast(self, value, self._pickled_broadcast_vars) def accumulator(self, value, accum_param=None): """ diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 760a509f0ef6d..33aa55f7f1429 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -448,184 +448,20 @@ def loads(self, obj): raise ValueError("invalid sevialization type: %s" % _type) -class SizeLimitedStream(object): - """ - Read at most `limit` bytes from underlying stream - - >>> from StringIO import StringIO - >>> io = StringIO() - >>> io.write("Hello world") - >>> io.seek(0) - >>> lio = SizeLimitedStream(io, 5) - >>> lio.read() - 'Hello' - """ - def __init__(self, stream, limit): - self.stream = stream - self.limit = limit - - def read(self, n=0): - if n > self.limit or n == 0: - n = self.limit - buf = self.stream.read(n) - self.limit -= len(buf) - return buf - - -class CompressedStream(object): - """ - Compress the data using zlib - - >>> from StringIO import StringIO - >>> io = StringIO() - >>> wio = CompressedStream(io, 'w') - >>> wio.write("Hello world") - >>> wio.flush() - >>> io.seek(0) - >>> rio = CompressedStream(io, 'r') - >>> rio.read() - 'Hello world' - >>> rio.read() - '' - """ - MAX_BATCH = 1 << 20 # 1MB - - def __init__(self, stream, mode='w', level=1): - self.stream = stream - self.mode = mode - if mode == 'w': - self.compresser = zlib.compressobj(level) - elif mode == 'r': - self.decompresser = zlib.decompressobj() - self.buf = '' - else: - raise ValueError("can only support mode 'w' or 'r' ") - - def write(self, buf): - assert self.mode == 'w', "It's not opened for write" - if len(buf) > self.MAX_BATCH: - # zlib can not compress string larger than 2G - batches = len(buf) / self.MAX_BATCH + 1 # last one may be empty - for i in xrange(batches): - self.write(buf[i * self.MAX_BATCH:(i + 1) * self.MAX_BATCH]) - else: - compressed = self.compresser.compress(buf) - self.stream.write(compressed) - - def flush(self, mode=zlib.Z_FULL_FLUSH): - if self.mode == 'w': - d = self.compresser.flush(mode) - self.stream.write(d) - self.stream.flush() - - def close(self): - if self.mode == 'w': - self.flush(zlib.Z_FINISH) - self.stream.close() - - def read(self, size=0): - assert self.mode == 'r', "It's not opened for read" - if not size: - data = self.stream.read() - result = self.decompresser.decompress(data) - last = self.decompresser.flush() - return self.buf + result + last - - # fast path for small read() - if size <= len(self.buf): - result = self.buf[:size] - self.buf = self.buf[size:] - return result - - result = [self.buf] - size -= len(self.buf) - self.buf = '' - while size: - need = min(size, self.MAX_BATCH) - input = self.stream.read(need) - if input: - buf = self.decompresser.decompress(input) - else: - buf = self.decompresser.flush() - - if len(buf) >= size: - self.buf = buf[size:] - result.append(buf[:size]) - return ''.join(result) - - size -= len(buf) - result.append(buf) - if not input: - return ''.join(result) - - def readline(self): - """ - This is needed for pickle, but not used in protocol 2 - """ - line = [] - b = self.read(1) - while b and b != '\n': - line.append(b) - b = self.read(1) - line.append(b) - return ''.join(line) - - -class LargeObjectSerializer(Serializer): - """ - Serialize large object which could be larger than 2G - - It uses cPickle to serialize the objects - """ - def dump_stream(self, iterator, stream): - stream = CompressedStream(stream, 'w') - for value in iterator: - if isinstance(value, basestring): - if isinstance(value, unicode): - stream.write('U') - value = value.encode("utf-8") - else: - stream.write('S') - write_long(len(value), stream) - stream.write(value) - else: - stream.write('P') - cPickle.dump(value, stream, 2) - stream.flush() - - def load_stream(self, stream): - stream = CompressedStream(stream, 'r') - while True: - type = stream.read(1) - if not type: - return - if type in ('S', 'U'): - length = read_long(stream) - value = stream.read(length) - if type == 'U': - value = value.decode('utf-8') - yield value - elif type == 'P': - yield cPickle.load(stream) - else: - raise ValueError("unknown type: %s" % type) - - -class CompressedSerializer(Serializer): +class CompressedSerializer(FramedSerializer): """ Compress the serialized data """ def __init__(self, serializer): + FramedSerializer.__init__(self) + assert isinstance(serializer, FramedSerializer), "serializer must be a FramedSerializer" self.serializer = serializer - def load_stream(self, stream): - stream = CompressedStream(stream, "r") - return self.serializer.load_stream(stream) + def dumps(self, obj): + return zlib.compress(self.serializer.dumps(obj), 1) - def dump_stream(self, iterator, stream): - stream = CompressedStream(stream, "w") - self.serializer.dump_stream(iterator, stream) - stream.flush() + def loads(self, obj): + return self.serializer.loads(zlib.decompress(obj)) class UTF8Deserializer(Serializer): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 29bcd38908d10..32645778c2b8f 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -48,7 +48,7 @@ from pyspark.context import SparkContext from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ - CloudPickleSerializer, SizeLimitedStream, CompressedSerializer, LargeObjectSerializer + CloudPickleSerializer, CompressedSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark.sql import SQLContext, IntegerType, Row, ArrayType, StructType, StructField, \ UserDefinedType, DoubleType @@ -237,26 +237,16 @@ def foo(): self.assertTrue("exit" in foo.func_code.co_names) ser.dumps(foo) - def _test_serializer(self, ser): + def test_compressed_serializer(self): + ser = CompressedSerializer(PickleSerializer()) from StringIO import StringIO io = StringIO() ser.dump_stream(["abc", u"123", range(5)], io) io.seek(0) self.assertEqual(["abc", u"123", range(5)], list(ser.load_stream(io))) - size = io.tell() ser.dump_stream(range(1000), io) io.seek(0) - first = SizeLimitedStream(io, size) - self.assertEqual(["abc", u"123", range(5)], list(ser.load_stream(first))) - self.assertEqual(range(1000), list(ser.load_stream(io))) - - def test_compressed_serializer(self): - ser = CompressedSerializer(PickleSerializer()) - self._test_serializer(ser) - - def test_large_object_serializer(self): - ser = LargeObjectSerializer() - self._test_serializer(ser) + self.assertEqual(["abc", u"123", range(5)] + range(1000), list(ser.load_stream(io))) class PySparkTestCase(unittest.TestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index e1552a0b0b4ff..7e5343c973dc5 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -30,8 +30,7 @@ from pyspark.broadcast import Broadcast, _broadcastRegistry from pyspark.files import SparkFiles from pyspark.serializers import write_with_length, write_int, read_long, \ - write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer, \ - SizeLimitedStream, LargeObjectSerializer + write_long, read_int, SpecialLengths, UTF8Deserializer, PickleSerializer from pyspark import shuffle pickleSer = PickleSerializer() @@ -78,14 +77,11 @@ def main(infile, outfile): # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) - bser = LargeObjectSerializer() for _ in range(num_broadcast_variables): bid = read_long(infile) if bid >= 0: - size = read_long(infile) - s = SizeLimitedStream(infile, size) - value = list((bser.load_stream(s)))[0] # read out all the bytes - _broadcastRegistry[bid] = Broadcast(bid, value) + path = utf8_deserializer.loads(infile) + _broadcastRegistry[bid] = Broadcast(path=path) else: bid = - bid - 1 _broadcastRegistry.pop(bid) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala index ddcb5db6c3a21..00d6b43a57812 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/UdfRegistration.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import java.util.{List => JList, Map => JMap} import org.apache.spark.Accumulator +import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUdf} @@ -39,7 +40,7 @@ private[sql] trait UDFRegistration { envVars: JMap[String, String], pythonIncludes: JList[String], pythonExec: String, - broadcastVars: JList[Broadcast[Array[Array[Byte]]]], + broadcastVars: JList[Broadcast[PythonBroadcast]], accumulator: Accumulator[JList[Array[Byte]]], stringDataType: String): Unit = { log.debug( 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 f98cae3f17e4a..2b4a88d5e864e 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 @@ -26,7 +26,7 @@ import scala.collection.JavaConverters._ import net.razorvine.pickle.{Pickler, Unpickler} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.api.python.PythonRDD +import org.apache.spark.api.python.{PythonBroadcast, PythonRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.expressions._ @@ -45,7 +45,7 @@ private[spark] case class PythonUDF( envVars: JMap[String, String], pythonIncludes: JList[String], pythonExec: String, - broadcastVars: JList[Broadcast[Array[Array[Byte]]]], + broadcastVars: JList[Broadcast[PythonBroadcast]], accumulator: Accumulator[JList[Array[Byte]]], dataType: DataType, children: Seq[Expression]) extends Expression with SparkLogging { From d24d5bf064572a2319627736b1fbf112b4a78edf Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 24 Nov 2014 18:03:10 -0800 Subject: [PATCH 016/305] [SPARK-4266] [Web-UI] Reduce stage page load time. The commit changes the java script used to show/hide additional metrics in order to reduce page load time. SPARK-4016 significantly increased page load time for the stage page when stages had a lot (thousands or tens of thousands) of tasks, due to the additional Javascript to hide some metrics by default and stripe the tables. This commit reduces page load time in two ways: (1) Now, all of the metrics that are hidden by default are hidden by setting "display: none;" using CSS for the page, rather than hiding them using javascript after the page loads. Without this change, for stages with thousands of tasks, there was a few second delay after page load, where first the additional metrics were shown, and then after a delay were hidden once the relevant JS finished running. (2) CSS is used to stripe all of the tables except for the summary table. The summary table needs javascript to do the striping because some rows are hidden, but the javascript striping is slower, which again resulted in a delay when it was used for the task table (where for a few seconds after page load, all of the rows in the task table would be white, while the browser finished running the JS to stripe the table). cc pwendell This change is intended to be backported to 1.2 to avoid a regression in UI performance when users run large jobs. Author: Kay Ousterhout Closes #3328 from kayousterhout/SPARK-4266 and squashes the following commits: f964091 [Kay Ousterhout] [SPARK-4266] [Web-UI] Reduce stage page load time. --- .../spark/ui/static/additional-metrics.js | 9 ++----- .../org/apache/spark/ui/static/table.js | 24 ++++++++++--------- .../org/apache/spark/ui/static/webui.css | 6 +++++ .../scala/org/apache/spark/ui/UIUtils.scala | 11 ++++----- .../apache/spark/ui/exec/ExecutorsPage.scala | 2 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 2 +- .../org/apache/spark/ui/jobs/StagePage.scala | 9 ++++++- .../spark/ui/jobs/TaskDetailsClassNames.scala | 3 +++ 8 files changed, 39 insertions(+), 27 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js index d33c5c769d683..14ba37d7c9bd9 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js +++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js @@ -28,17 +28,12 @@ $(function() { $(this).find('.expand-additional-metrics-arrow').toggleClass('arrow-closed'); }); - $("input:checkbox:not(:checked)").each(function() { - var column = "table ." + $(this).attr("name"); - $(column).hide(); - }); - // Stripe table rows after rows have been hidden to ensure correct striping. - stripeTables(); + stripeSummaryTable(); $("input:checkbox").click(function() { var column = "table ." + $(this).attr("name"); $(column).toggle(); - stripeTables(); + stripeSummaryTable(); }); $("#select-all-metrics").click(function() { diff --git a/core/src/main/resources/org/apache/spark/ui/static/table.js b/core/src/main/resources/org/apache/spark/ui/static/table.js index 6bb03015abb51..656147e40d13e 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/table.js +++ b/core/src/main/resources/org/apache/spark/ui/static/table.js @@ -15,16 +15,18 @@ * limitations under the License. */ -/* Adds background colors to stripe table rows. This is necessary (instead of using css or the - * table striping provided by bootstrap) to appropriately stripe tables with hidden rows. */ -function stripeTables() { - $("table.table-striped-custom").each(function() { - $(this).find("tr:not(:hidden)").each(function (index) { - if (index % 2 == 1) { - $(this).css("background-color", "#f9f9f9"); - } else { - $(this).css("background-color", "#ffffff"); - } - }); +/* Adds background colors to stripe table rows in the summary table (on the stage page). This is + * necessary (instead of using css or the table striping provided by bootstrap) because the summary + * table has hidden rows. + * + * An ID selector (rather than a class selector) is used to ensure this runs quickly even on pages + * with thousands of task rows (ID selectors are much faster than class selectors). */ +function stripeSummaryTable() { + $("#task-summary-table").find("tr:not(:hidden)").each(function (index) { + if (index % 2 == 1) { + $(this).css("background-color", "#f9f9f9"); + } else { + $(this).css("background-color", "#ffffff"); + } }); } diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index db57712c83503..cdf85bfbf326f 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -168,3 +168,9 @@ span.additional-metric-title { border-left: 5px solid black; display: inline-block; } + +/* Hide all additional metrics by default. This is done here rather than using JavaScript to + * avoid slow page loads for stage pages with large numbers (e.g., thousands) of tasks. */ +.scheduler_delay, .gc_time, .deserialization_time, .serialization_time, .getting_result_time { + display: none; +} diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 0c418beaf7581..09079bbd43f6f 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -26,7 +26,8 @@ import org.apache.spark.Logging /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils extends Logging { - val TABLE_CLASS = "table table-bordered table-striped-custom table-condensed sortable" + val TABLE_CLASS_NOT_STRIPED = "table table-bordered table-condensed sortable" + val TABLE_CLASS_STRIPED = TABLE_CLASS_NOT_STRIPED + " table-striped" // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { @@ -248,12 +249,10 @@ private[spark] object UIUtils extends Logging { data: Iterable[T], fixedWidth: Boolean = false, id: Option[String] = None, - headerClasses: Seq[String] = Seq.empty): Seq[Node] = { + headerClasses: Seq[String] = Seq.empty, + stripeRowsWithCss: Boolean = true): Seq[Node] = { - var listingTableClass = TABLE_CLASS - if (fixedWidth) { - listingTableClass += " table-fixed" - } + val listingTableClass = if (stripeRowsWithCss) TABLE_CLASS_STRIPED else TABLE_CLASS_NOT_STRIPED val colWidth = 100.toDouble / headers.size val colWidthAttr = if (fixedWidth) colWidth + "%" else "" diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 71b59b1d078ca..363cb96de7998 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -57,7 +57,7 @@ private[ui] class ExecutorsPage( val execInfoSorted = execInfo.sortBy(_.id) val execTable = - +
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 35bbe8b4f9ac8..9836d11a6d85f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -36,7 +36,7 @@ private[ui] class ExecutorTable(stageId: Int, stageAttemptId: Int, parent: Stage /** Special table which merges two header cells. */ private def executorTable[T](): Seq[Node] = { -
Executor ID Address
+
diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 40e05f86b661d..bfa54f8492068 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -322,8 +322,15 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", "Max") + // The summary table does not use CSS to stripe rows, which doesn't work with hidden + // rows (instead, JavaScript in table.js is used to stripe the non-hidden rows). Some(UIUtils.listingTable( - quantileHeaders, identity[Seq[Node]], listings, fixedWidth = true)) + quantileHeaders, + identity[Seq[Node]], + listings, + fixedWidth = true, + id = Some("task-summary-table"), + stripeRowsWithCss = false)) } val executorTable = new ExecutorTable(stageId, stageAttemptId, parent) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala index eb371bd0ea7ed..ca942c4051c84 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/TaskDetailsClassNames.scala @@ -20,6 +20,9 @@ package org.apache.spark.ui.jobs /** * Names of the CSS classes corresponding to each type of task detail. Used to allow users * to optionally show/hide columns. + * + * If new optional metrics are added here, they should also be added to the end of webui.css + * to have the style set to "display: none;" by default. */ private object TaskDetailsClassNames { val SCHEDULER_DELAY = "scheduler_delay" From b043c27424d05e3200e7ba99a1a65656b57fa2f0 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Nov 2014 19:14:14 -0800 Subject: [PATCH 017/305] [SPARK-4525] Mesos should decline unused offers Functionally, this is just a small change on top of #3393 (by jongyoul). The issue being addressed is discussed in the comments there. I have not yet added a test for the bug there. I will add one shortly. I've also done some minor renaming/clean-up of variables in this class and tests. Author: Patrick Wendell Author: Jongyoul Lee Closes #3436 from pwendell/mesos-issue and squashes the following commits: 58c35b5 [Patrick Wendell] Adding unit test for this situation c4f0697 [Patrick Wendell] Additional clean-up and fixes on top of existing fix f20f1b3 [Jongyoul Lee] [SPARK-4525] MesosSchedulerBackend.resourceOffers cannot decline unused offers from acceptedOffers - Added code for declining unused offers among acceptedOffers - Edited testCase for checking declining unused offers --- .../cluster/mesos/MesosSchedulerBackend.scala | 25 ++++++-- .../mesos/MesosSchedulerBackendSuite.scala | 61 ++++++++++++++----- 2 files changed, 65 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index d13795186c48e..10e6886c16a4f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -208,10 +208,12 @@ private[spark] class MesosSchedulerBackend( */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { inClassLoader() { - val (acceptedOffers, declinedOffers) = offers.partition { o => + // Fail-fast on offers we know will be rejected + val (usableOffers, unUsableOffers) = offers.partition { o => val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue + // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? (mem >= MemoryUtils.calculateTotalMemory(sc) && // need at least 1 for executor, 1 for task cpus >= 2 * scheduler.CPUS_PER_TASK) || @@ -219,11 +221,12 @@ private[spark] class MesosSchedulerBackend( cpus >= scheduler.CPUS_PER_TASK) } - val offerableWorkers = acceptedOffers.map { o => + val workerOffers = usableOffers.map { o => val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt } else { // If the executor doesn't exist yet, subtract CPU for executor + // TODO(pwendell): Should below just subtract "1"? getResource(o.getResourcesList, "cpus").toInt - scheduler.CPUS_PER_TASK } @@ -233,17 +236,20 @@ private[spark] class MesosSchedulerBackend( cpus) } - val slaveIdToOffer = acceptedOffers.map(o => o.getSlaveId.getValue -> o).toMap + val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] + val slavesIdsOfAcceptedOffers = HashSet[String]() + // Call into the TaskSchedulerImpl - scheduler.resourceOffers(offerableWorkers) - .filter(!_.isEmpty) + val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) + acceptedOffers .foreach { offer => offer.foreach { taskDesc => val slaveId = taskDesc.executorId slaveIdsWithExecutors += slaveId + slavesIdsOfAcceptedOffers += slaveId taskIdToSlaveId(taskDesc.taskId) = slaveId mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) .add(createMesosTask(taskDesc, slaveId)) @@ -257,7 +263,14 @@ private[spark] class MesosSchedulerBackend( d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } - declinedOffers.foreach(o => d.declineOffer(o.getId)) + // Decline offers that weren't used + // NOTE: This logic assumes that we only get a single offer for each host in a given batch + for (o <- usableOffers if !slavesIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) { + d.declineOffer(o.getId) + } + + // Decline offers we ruled out immediately + unUsableOffers.foreach(o => d.declineOffer(o.getId)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index bef8d3a58ba63..e60e70afd3218 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -30,9 +30,11 @@ import java.nio.ByteBuffer import java.util.Collections import java.util import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar { - test("mesos resource offer is launching tasks") { + + test("mesos resource offers result in launching tasks") { def createOffer(id: Int, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() builder.addResourcesBuilder() @@ -43,46 +45,61 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(id.toString).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue("s1")).setHostname("localhost").build() + builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build() } val driver = EasyMock.createMock(classOf[SchedulerDriver]) val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) val sc = EasyMock.createMock(classOf[SparkContext]) - EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes() EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes() EasyMock.replay(sc) + val minMem = MemoryUtils.calculateTotalMemory(sc).toInt val minCpu = 4 - val offers = new java.util.ArrayList[Offer] - offers.add(createOffer(1, minMem, minCpu)) - offers.add(createOffer(1, minMem - 1, minCpu)) + + val mesosOffers = new java.util.ArrayList[Offer] + mesosOffers.add(createOffer(1, minMem, minCpu)) + mesosOffers.add(createOffer(2, minMem - 1, minCpu)) + mesosOffers.add(createOffer(3, minMem, minCpu)) + val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") - val workerOffers = Seq(offers.get(0)).map(o => new WorkerOffer( - o.getSlaveId.getValue, - o.getHostname, + + val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) + expectedWorkerOffers.append(new WorkerOffer( + mesosOffers.get(0).getSlaveId.getValue, + mesosOffers.get(0).getHostname, + 2 + )) + expectedWorkerOffers.append(new WorkerOffer( + mesosOffers.get(2).getSlaveId.getValue, + mesosOffers.get(2).getHostname, 2 )) val taskDesc = new TaskDescription(1L, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(workerOffers))).andReturn(Seq(Seq(taskDesc))) + EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(expectedWorkerOffers))).andReturn(Seq(Seq(taskDesc))) EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() EasyMock.replay(taskScheduler) + val capture = new Capture[util.Collection[TaskInfo]] EasyMock.expect( driver.launchTasks( - EasyMock.eq(Collections.singleton(offers.get(0).getId)), + EasyMock.eq(Collections.singleton(mesosOffers.get(0).getId)), EasyMock.capture(capture), EasyMock.anyObject(classOf[Filters]) ) - ).andReturn(Status.valueOf(1)) - EasyMock.expect(driver.declineOffer(offers.get(1).getId)).andReturn(Status.valueOf(1)) + ).andReturn(Status.valueOf(1)).once + EasyMock.expect(driver.declineOffer(mesosOffers.get(1).getId)).andReturn(Status.valueOf(1)).times(1) + EasyMock.expect(driver.declineOffer(mesosOffers.get(2).getId)).andReturn(Status.valueOf(1)).times(1) EasyMock.replay(driver) - backend.resourceOffers(driver, offers) + + backend.resourceOffers(driver, mesosOffers) + + EasyMock.verify(driver) assert(capture.getValue.size() == 1) val taskInfo = capture.getValue.iterator().next() assert(taskInfo.getName.equals("n1")) @@ -90,5 +107,19 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea assert(cpus.getName.equals("cpus")) assert(cpus.getScalar.getValue.equals(2.0)) assert(taskInfo.getSlaveId.getValue.equals("s1")) + + // Unwanted resources offered on an existing node. Make sure they are declined + val mesosOffers2 = new java.util.ArrayList[Offer] + mesosOffers2.add(createOffer(1, minMem, minCpu)) + EasyMock.reset(taskScheduler) + EasyMock.reset(driver) + EasyMock.expect(taskScheduler.resourceOffers(EasyMock.anyObject(classOf[Seq[WorkerOffer]])).andReturn(Seq(Seq()))) + EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() + EasyMock.replay(taskScheduler) + EasyMock.expect(driver.declineOffer(mesosOffers2.get(0).getId)).andReturn(Status.valueOf(1)).times(1) + EasyMock.replay(driver) + + backend.resourceOffers(driver, mesosOffers2) + EasyMock.verify(driver) } } From a68d442270fa85b5831d201016f3e911af128679 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Nov 2014 19:16:53 -0800 Subject: [PATCH 018/305] Revert "[SPARK-4525] Mesos should decline unused offers" This reverts commit b043c27424d05e3200e7ba99a1a65656b57fa2f0. I accidentally committed this using my own authorship credential. However, I should have given authoriship to the original author: Jongyoul Lee. --- .../cluster/mesos/MesosSchedulerBackend.scala | 25 ++------ .../mesos/MesosSchedulerBackendSuite.scala | 61 +++++-------------- 2 files changed, 21 insertions(+), 65 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 10e6886c16a4f..d13795186c48e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -208,12 +208,10 @@ private[spark] class MesosSchedulerBackend( */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { inClassLoader() { - // Fail-fast on offers we know will be rejected - val (usableOffers, unUsableOffers) = offers.partition { o => + val (acceptedOffers, declinedOffers) = offers.partition { o => val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue - // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? (mem >= MemoryUtils.calculateTotalMemory(sc) && // need at least 1 for executor, 1 for task cpus >= 2 * scheduler.CPUS_PER_TASK) || @@ -221,12 +219,11 @@ private[spark] class MesosSchedulerBackend( cpus >= scheduler.CPUS_PER_TASK) } - val workerOffers = usableOffers.map { o => + val offerableWorkers = acceptedOffers.map { o => val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt } else { // If the executor doesn't exist yet, subtract CPU for executor - // TODO(pwendell): Should below just subtract "1"? getResource(o.getResourcesList, "cpus").toInt - scheduler.CPUS_PER_TASK } @@ -236,20 +233,17 @@ private[spark] class MesosSchedulerBackend( cpus) } - val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap + val slaveIdToOffer = acceptedOffers.map(o => o.getSlaveId.getValue -> o).toMap val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] - val slavesIdsOfAcceptedOffers = HashSet[String]() - // Call into the TaskSchedulerImpl - val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) - acceptedOffers + scheduler.resourceOffers(offerableWorkers) + .filter(!_.isEmpty) .foreach { offer => offer.foreach { taskDesc => val slaveId = taskDesc.executorId slaveIdsWithExecutors += slaveId - slavesIdsOfAcceptedOffers += slaveId taskIdToSlaveId(taskDesc.taskId) = slaveId mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) .add(createMesosTask(taskDesc, slaveId)) @@ -263,14 +257,7 @@ private[spark] class MesosSchedulerBackend( d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } - // Decline offers that weren't used - // NOTE: This logic assumes that we only get a single offer for each host in a given batch - for (o <- usableOffers if !slavesIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) { - d.declineOffer(o.getId) - } - - // Decline offers we ruled out immediately - unUsableOffers.foreach(o => d.declineOffer(o.getId)) + declinedOffers.foreach(o => d.declineOffer(o.getId)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index e60e70afd3218..bef8d3a58ba63 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -30,11 +30,9 @@ import java.nio.ByteBuffer import java.util.Collections import java.util import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar { - - test("mesos resource offers result in launching tasks") { + test("mesos resource offer is launching tasks") { def createOffer(id: Int, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() builder.addResourcesBuilder() @@ -45,61 +43,46 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build() + builder.setId(OfferID.newBuilder().setValue(id.toString).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue("s1")).setHostname("localhost").build() } val driver = EasyMock.createMock(classOf[SchedulerDriver]) val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) val sc = EasyMock.createMock(classOf[SparkContext]) + EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes() EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes() EasyMock.replay(sc) - val minMem = MemoryUtils.calculateTotalMemory(sc).toInt val minCpu = 4 - - val mesosOffers = new java.util.ArrayList[Offer] - mesosOffers.add(createOffer(1, minMem, minCpu)) - mesosOffers.add(createOffer(2, minMem - 1, minCpu)) - mesosOffers.add(createOffer(3, minMem, minCpu)) - + val offers = new java.util.ArrayList[Offer] + offers.add(createOffer(1, minMem, minCpu)) + offers.add(createOffer(1, minMem - 1, minCpu)) val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") - - val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) - expectedWorkerOffers.append(new WorkerOffer( - mesosOffers.get(0).getSlaveId.getValue, - mesosOffers.get(0).getHostname, - 2 - )) - expectedWorkerOffers.append(new WorkerOffer( - mesosOffers.get(2).getSlaveId.getValue, - mesosOffers.get(2).getHostname, + val workerOffers = Seq(offers.get(0)).map(o => new WorkerOffer( + o.getSlaveId.getValue, + o.getHostname, 2 )) val taskDesc = new TaskDescription(1L, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(expectedWorkerOffers))).andReturn(Seq(Seq(taskDesc))) + EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(workerOffers))).andReturn(Seq(Seq(taskDesc))) EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() EasyMock.replay(taskScheduler) - val capture = new Capture[util.Collection[TaskInfo]] EasyMock.expect( driver.launchTasks( - EasyMock.eq(Collections.singleton(mesosOffers.get(0).getId)), + EasyMock.eq(Collections.singleton(offers.get(0).getId)), EasyMock.capture(capture), EasyMock.anyObject(classOf[Filters]) ) - ).andReturn(Status.valueOf(1)).once - EasyMock.expect(driver.declineOffer(mesosOffers.get(1).getId)).andReturn(Status.valueOf(1)).times(1) - EasyMock.expect(driver.declineOffer(mesosOffers.get(2).getId)).andReturn(Status.valueOf(1)).times(1) + ).andReturn(Status.valueOf(1)) + EasyMock.expect(driver.declineOffer(offers.get(1).getId)).andReturn(Status.valueOf(1)) EasyMock.replay(driver) - - backend.resourceOffers(driver, mesosOffers) - - EasyMock.verify(driver) + backend.resourceOffers(driver, offers) assert(capture.getValue.size() == 1) val taskInfo = capture.getValue.iterator().next() assert(taskInfo.getName.equals("n1")) @@ -107,19 +90,5 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea assert(cpus.getName.equals("cpus")) assert(cpus.getScalar.getValue.equals(2.0)) assert(taskInfo.getSlaveId.getValue.equals("s1")) - - // Unwanted resources offered on an existing node. Make sure they are declined - val mesosOffers2 = new java.util.ArrayList[Offer] - mesosOffers2.add(createOffer(1, minMem, minCpu)) - EasyMock.reset(taskScheduler) - EasyMock.reset(driver) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.anyObject(classOf[Seq[WorkerOffer]])).andReturn(Seq(Seq()))) - EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() - EasyMock.replay(taskScheduler) - EasyMock.expect(driver.declineOffer(mesosOffers2.get(0).getId)).andReturn(Status.valueOf(1)).times(1) - EasyMock.replay(driver) - - backend.resourceOffers(driver, mesosOffers2) - EasyMock.verify(driver) } } From f0afb623dc51fd3008bd80496b8d1eaa991323d6 Mon Sep 17 00:00:00 2001 From: Jongyoul Lee Date: Mon, 24 Nov 2014 19:14:14 -0800 Subject: [PATCH 019/305] [SPARK-4525] Mesos should decline unused offers Functionally, this is just a small change on top of #3393 (by jongyoul). The issue being addressed is discussed in the comments there. I have not yet added a test for the bug there. I will add one shortly. I've also done some minor renaming/clean-up of variables in this class and tests. Author: Patrick Wendell Author: Jongyoul Lee Closes #3436 from pwendell/mesos-issue and squashes the following commits: 58c35b5 [Patrick Wendell] Adding unit test for this situation c4f0697 [Patrick Wendell] Additional clean-up and fixes on top of existing fix f20f1b3 [Jongyoul Lee] [SPARK-4525] MesosSchedulerBackend.resourceOffers cannot decline unused offers from acceptedOffers - Added code for declining unused offers among acceptedOffers - Edited testCase for checking declining unused offers --- .../cluster/mesos/MesosSchedulerBackend.scala | 25 ++++++-- .../mesos/MesosSchedulerBackendSuite.scala | 61 ++++++++++++++----- 2 files changed, 65 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index d13795186c48e..10e6886c16a4f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -208,10 +208,12 @@ private[spark] class MesosSchedulerBackend( */ override def resourceOffers(d: SchedulerDriver, offers: JList[Offer]) { inClassLoader() { - val (acceptedOffers, declinedOffers) = offers.partition { o => + // Fail-fast on offers we know will be rejected + val (usableOffers, unUsableOffers) = offers.partition { o => val mem = getResource(o.getResourcesList, "mem") val cpus = getResource(o.getResourcesList, "cpus") val slaveId = o.getSlaveId.getValue + // TODO(pwendell): Should below be 1 + scheduler.CPUS_PER_TASK? (mem >= MemoryUtils.calculateTotalMemory(sc) && // need at least 1 for executor, 1 for task cpus >= 2 * scheduler.CPUS_PER_TASK) || @@ -219,11 +221,12 @@ private[spark] class MesosSchedulerBackend( cpus >= scheduler.CPUS_PER_TASK) } - val offerableWorkers = acceptedOffers.map { o => + val workerOffers = usableOffers.map { o => val cpus = if (slaveIdsWithExecutors.contains(o.getSlaveId.getValue)) { getResource(o.getResourcesList, "cpus").toInt } else { // If the executor doesn't exist yet, subtract CPU for executor + // TODO(pwendell): Should below just subtract "1"? getResource(o.getResourcesList, "cpus").toInt - scheduler.CPUS_PER_TASK } @@ -233,17 +236,20 @@ private[spark] class MesosSchedulerBackend( cpus) } - val slaveIdToOffer = acceptedOffers.map(o => o.getSlaveId.getValue -> o).toMap + val slaveIdToOffer = usableOffers.map(o => o.getSlaveId.getValue -> o).toMap val mesosTasks = new HashMap[String, JArrayList[MesosTaskInfo]] + val slavesIdsOfAcceptedOffers = HashSet[String]() + // Call into the TaskSchedulerImpl - scheduler.resourceOffers(offerableWorkers) - .filter(!_.isEmpty) + val acceptedOffers = scheduler.resourceOffers(workerOffers).filter(!_.isEmpty) + acceptedOffers .foreach { offer => offer.foreach { taskDesc => val slaveId = taskDesc.executorId slaveIdsWithExecutors += slaveId + slavesIdsOfAcceptedOffers += slaveId taskIdToSlaveId(taskDesc.taskId) = slaveId mesosTasks.getOrElseUpdate(slaveId, new JArrayList[MesosTaskInfo]) .add(createMesosTask(taskDesc, slaveId)) @@ -257,7 +263,14 @@ private[spark] class MesosSchedulerBackend( d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) } - declinedOffers.foreach(o => d.declineOffer(o.getId)) + // Decline offers that weren't used + // NOTE: This logic assumes that we only get a single offer for each host in a given batch + for (o <- usableOffers if !slavesIdsOfAcceptedOffers.contains(o.getSlaveId.getValue)) { + d.declineOffer(o.getId) + } + + // Decline offers we ruled out immediately + unUsableOffers.foreach(o => d.declineOffer(o.getId)) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala index bef8d3a58ba63..e60e70afd3218 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/mesos/MesosSchedulerBackendSuite.scala @@ -30,9 +30,11 @@ import java.nio.ByteBuffer import java.util.Collections import java.util import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with EasyMockSugar { - test("mesos resource offer is launching tasks") { + + test("mesos resource offers result in launching tasks") { def createOffer(id: Int, mem: Int, cpu: Int) = { val builder = Offer.newBuilder() builder.addResourcesBuilder() @@ -43,46 +45,61 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea .setName("cpus") .setType(Value.Type.SCALAR) .setScalar(Scalar.newBuilder().setValue(cpu)) - builder.setId(OfferID.newBuilder().setValue(id.toString).build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) - .setSlaveId(SlaveID.newBuilder().setValue("s1")).setHostname("localhost").build() + builder.setId(OfferID.newBuilder().setValue(s"o${id.toString}").build()).setFrameworkId(FrameworkID.newBuilder().setValue("f1")) + .setSlaveId(SlaveID.newBuilder().setValue(s"s${id.toString}")).setHostname(s"host${id.toString}").build() } val driver = EasyMock.createMock(classOf[SchedulerDriver]) val taskScheduler = EasyMock.createMock(classOf[TaskSchedulerImpl]) val sc = EasyMock.createMock(classOf[SparkContext]) - EasyMock.expect(sc.executorMemory).andReturn(100).anyTimes() EasyMock.expect(sc.getSparkHome()).andReturn(Option("/path")).anyTimes() EasyMock.expect(sc.executorEnvs).andReturn(new mutable.HashMap).anyTimes() EasyMock.expect(sc.conf).andReturn(new SparkConf).anyTimes() EasyMock.replay(sc) + val minMem = MemoryUtils.calculateTotalMemory(sc).toInt val minCpu = 4 - val offers = new java.util.ArrayList[Offer] - offers.add(createOffer(1, minMem, minCpu)) - offers.add(createOffer(1, minMem - 1, minCpu)) + + val mesosOffers = new java.util.ArrayList[Offer] + mesosOffers.add(createOffer(1, minMem, minCpu)) + mesosOffers.add(createOffer(2, minMem - 1, minCpu)) + mesosOffers.add(createOffer(3, minMem, minCpu)) + val backend = new MesosSchedulerBackend(taskScheduler, sc, "master") - val workerOffers = Seq(offers.get(0)).map(o => new WorkerOffer( - o.getSlaveId.getValue, - o.getHostname, + + val expectedWorkerOffers = new ArrayBuffer[WorkerOffer](2) + expectedWorkerOffers.append(new WorkerOffer( + mesosOffers.get(0).getSlaveId.getValue, + mesosOffers.get(0).getHostname, + 2 + )) + expectedWorkerOffers.append(new WorkerOffer( + mesosOffers.get(2).getSlaveId.getValue, + mesosOffers.get(2).getHostname, 2 )) val taskDesc = new TaskDescription(1L, "s1", "n1", 0, ByteBuffer.wrap(new Array[Byte](0))) - EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(workerOffers))).andReturn(Seq(Seq(taskDesc))) + EasyMock.expect(taskScheduler.resourceOffers(EasyMock.eq(expectedWorkerOffers))).andReturn(Seq(Seq(taskDesc))) EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() EasyMock.replay(taskScheduler) + val capture = new Capture[util.Collection[TaskInfo]] EasyMock.expect( driver.launchTasks( - EasyMock.eq(Collections.singleton(offers.get(0).getId)), + EasyMock.eq(Collections.singleton(mesosOffers.get(0).getId)), EasyMock.capture(capture), EasyMock.anyObject(classOf[Filters]) ) - ).andReturn(Status.valueOf(1)) - EasyMock.expect(driver.declineOffer(offers.get(1).getId)).andReturn(Status.valueOf(1)) + ).andReturn(Status.valueOf(1)).once + EasyMock.expect(driver.declineOffer(mesosOffers.get(1).getId)).andReturn(Status.valueOf(1)).times(1) + EasyMock.expect(driver.declineOffer(mesosOffers.get(2).getId)).andReturn(Status.valueOf(1)).times(1) EasyMock.replay(driver) - backend.resourceOffers(driver, offers) + + backend.resourceOffers(driver, mesosOffers) + + EasyMock.verify(driver) assert(capture.getValue.size() == 1) val taskInfo = capture.getValue.iterator().next() assert(taskInfo.getName.equals("n1")) @@ -90,5 +107,19 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Ea assert(cpus.getName.equals("cpus")) assert(cpus.getScalar.getValue.equals(2.0)) assert(taskInfo.getSlaveId.getValue.equals("s1")) + + // Unwanted resources offered on an existing node. Make sure they are declined + val mesosOffers2 = new java.util.ArrayList[Offer] + mesosOffers2.add(createOffer(1, minMem, minCpu)) + EasyMock.reset(taskScheduler) + EasyMock.reset(driver) + EasyMock.expect(taskScheduler.resourceOffers(EasyMock.anyObject(classOf[Seq[WorkerOffer]])).andReturn(Seq(Seq()))) + EasyMock.expect(taskScheduler.CPUS_PER_TASK).andReturn(2).anyTimes() + EasyMock.replay(taskScheduler) + EasyMock.expect(driver.declineOffer(mesosOffers2.get(0).getId)).andReturn(Status.valueOf(1)).times(1) + EasyMock.replay(driver) + + backend.resourceOffers(driver, mesosOffers2) + EasyMock.verify(driver) } } From 9ce2bf3821784cded5c2f1e1a75d038f7677be74 Mon Sep 17 00:00:00 2001 From: tkaessmann Date: Mon, 24 Nov 2014 19:58:01 -0800 Subject: [PATCH 020/305] [SPARK-4582][MLLIB] get raw vectors for further processing in Word2Vec This is #3309 for the master branch. e.g. clustering Author: tkaessmann Closes #3309 from tkaessmann/branch-1.2 and squashes the following commits: e3a3142 [tkaessmann] changes the comment for getVectors 58d3d83 [tkaessmann] removes sign from comment a5be213 [tkaessmann] fixes getVectors to fit code guidelines 3782fa9 [tkaessmann] get raw vectors for further processing Author: tkaessmann Closes #3437 from mengxr/SPARK-4582 and squashes the following commits: 6c666b4 [tkaessmann] get raw vectors for further processing in Word2Vec --- .../scala/org/apache/spark/mllib/feature/Word2Vec.scala | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index f5f7ad613d4c4..7960f3cab576f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -461,4 +461,11 @@ class Word2VecModel private[mllib] ( .tail .toArray } + + /** + * Returns a map of words to their vector representations. + */ + def getVectors: Map[String, Array[Float]] = { + model + } } From 723be60e233d0f85944d948efd06845ef546c9f5 Mon Sep 17 00:00:00 2001 From: w00228970 Date: Mon, 24 Nov 2014 21:17:24 -0800 Subject: [PATCH 021/305] [SQL] Compute timeTaken correctly ```timeTaken``` should not count the time of printing result. Author: w00228970 Closes #3423 from scwf/time-taken-bug and squashes the following commits: da7e102 [w00228970] compute time taken correctly --- .../sql/hive/thriftserver/SparkSQLCLIDriver.scala | 11 ++++------- 1 file changed, 4 insertions(+), 7 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala index 2cd02ae9269f5..7385952861ee5 100755 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala @@ -272,8 +272,10 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { if (sessionState.getIsVerbose) { out.println(cmd) } - val rc = driver.run(cmd) + val end = System.currentTimeMillis() + val timeTaken:Double = (end - start) / 1000.0 + ret = rc.getResponseCode if (ret != 0) { console.printError(rc.getErrorMessage()) @@ -309,12 +311,7 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { ret = cret } - val end = System.currentTimeMillis() - if (end > start) { - val timeTaken:Double = (end - start) / 1000.0 - console.printInfo(s"Time taken: $timeTaken seconds", null) - } - + console.printInfo(s"Time taken: $timeTaken seconds", null) // Destroy the driver to release all the locks. driver.destroy() } else { From 0fe54cff19759dad2dc2a0950bd6c1d31c95e858 Mon Sep 17 00:00:00 2001 From: wangfei Date: Mon, 24 Nov 2014 22:32:39 -0800 Subject: [PATCH 022/305] [DOC][Build] Wrong cmd for build spark with apache hadoop 2.4.X and hive 12 Author: wangfei Closes #3335 from scwf/patch-10 and squashes the following commits: d343113 [wangfei] add '-Phive' 60d595e [wangfei] [DOC] Wrong cmd for build spark with apache hadoop 2.4.X and Hive 12 support --- docs/building-spark.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/building-spark.md b/docs/building-spark.md index fee6a8440634c..40a47410e683a 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -112,7 +112,7 @@ Hive 0.12.0 using the `-Phive-0.12.0` profile. mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package # Apache Hadoop 2.4.X with Hive 12 support -mvn -Pyarn -Phive -Phive-thriftserver-0.12.0 -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-thriftserver -DskipTests clean package +mvn -Pyarn -Phadoop-2.4 -Dhadoop.version=2.4.0 -Phive -Phive-0.12.0 -Phive-thriftserver -DskipTests clean package {% endhighlight %} # Building for Scala 2.11 From 89f912264603741c7d980135c26102d63e11791f Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 25 Nov 2014 01:57:34 -0800 Subject: [PATCH 023/305] [SPARK-4596][MLLib] Refactorize Normalizer to make code cleaner In this refactoring, the performance will be slightly increased due to removing the overhead from breeze vector. The bottleneck is still in breeze norm which is implemented by activeIterator. This inefficiency of breeze norm will be addressed in next PR. At least, this PR makes the code more consistent in the codebase. Author: DB Tsai Closes #3446 from dbtsai/normalizer and squashes the following commits: e20a2b9 [DB Tsai] first commit --- .../spark/mllib/feature/Normalizer.scala | 29 ++++++++++++------- 1 file changed, 19 insertions(+), 10 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala index dfad25d57c947..a9c2e23717896 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -17,10 +17,10 @@ package org.apache.spark.mllib.feature -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => brzNorm} +import breeze.linalg.{norm => brzNorm} import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} /** * :: Experimental :: @@ -47,22 +47,31 @@ class Normalizer(p: Double) extends VectorTransformer { * @return normalized vector. If the norm of the input is zero, it will return the input vector. */ override def transform(vector: Vector): Vector = { - var norm = brzNorm(vector.toBreeze, p) + val norm = brzNorm(vector.toBreeze, p) if (norm != 0.0) { // For dense vector, we've to allocate new memory for new output vector. // However, for sparse vector, the `index` array will not be changed, // so we can re-use it to save memory. - vector.toBreeze match { - case dv: BDV[Double] => Vectors.fromBreeze(dv :/ norm) - case sv: BSV[Double] => - val output = new BSV[Double](sv.index, sv.data.clone(), sv.length) + vector match { + case dv: DenseVector => + val values = dv.values.clone() + val size = values.size var i = 0 - while (i < output.data.length) { - output.data(i) /= norm + while (i < size) { + values(i) /= norm i += 1 } - Vectors.fromBreeze(output) + Vectors.dense(values) + case sv: SparseVector => + val values = sv.values.clone() + val nnz = values.size + var i = 0 + while (i < nnz) { + values(i) /= norm + i += 1 + } + Vectors.sparse(sv.size, sv.indices, values) case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) } } else { From f515f9432b05f7e090b651c5536aa706d1cde487 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Tue, 25 Nov 2014 02:01:19 -0800 Subject: [PATCH 024/305] [SPARK-4526][MLLIB]GradientDescent get a wrong gradient value according to the gradient formula. This is caused by the miniBatchSize parameter.The number of `RDD.sample` returns is not fixed. cc mengxr Author: GuoQiang Li Closes #3399 from witgo/GradientDescent and squashes the following commits: 13cb228 [GuoQiang Li] review commit 668ab66 [GuoQiang Li] Double to Long b6aa11a [GuoQiang Li] Check miniBatchSize is greater than 0 0b5c3e3 [GuoQiang Li] Minor fix 12e7424 [GuoQiang Li] GradientDescent get a wrong gradient value according to the gradient formula, which is caused by the miniBatchSize parameter. --- .../mllib/optimization/GradientDescent.scala | 45 +++++++++++-------- 1 file changed, 26 insertions(+), 19 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index a6912056395d7..0857877951c82 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -160,14 +160,15 @@ object GradientDescent extends Logging { val stochasticLossHistory = new ArrayBuffer[Double](numIterations) val numExamples = data.count() - val miniBatchSize = numExamples * miniBatchFraction // if no data, return initial weights to avoid NaNs if (numExamples == 0) { - - logInfo("GradientDescent.runMiniBatchSGD returning initial weights, no data found") + logWarning("GradientDescent.runMiniBatchSGD returning initial weights, no data found") return (initialWeights, stochasticLossHistory.toArray) + } + if (numExamples * miniBatchFraction < 1) { + logWarning("The miniBatchFraction is too small") } // Initialize weights as a column vector @@ -185,25 +186,31 @@ object GradientDescent extends Logging { val bcWeights = data.context.broadcast(weights) // Sample a subset (fraction miniBatchFraction) of the total data // compute and sum up the subgradients on this subset (this is one map-reduce) - val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i) - .treeAggregate((BDV.zeros[Double](n), 0.0))( - seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => - val l = gradient.compute(features, label, bcWeights.value, Vectors.fromBreeze(grad)) - (grad, loss + l) + val (gradientSum, lossSum, miniBatchSize) = data.sample(false, miniBatchFraction, 42 + i) + .treeAggregate((BDV.zeros[Double](n), 0.0, 0L))( + seqOp = (c, v) => { + // c: (grad, loss, count), v: (label, features) + val l = gradient.compute(v._2, v._1, bcWeights.value, Vectors.fromBreeze(c._1)) + (c._1, c._2 + l, c._3 + 1) }, - combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => - (grad1 += grad2, loss1 + loss2) + combOp = (c1, c2) => { + // c: (grad, loss, count) + (c1._1 += c2._1, c1._2 + c2._2, c1._3 + c2._3) }) - /** - * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration - * and regVal is the regularization value computed in the previous iteration as well. - */ - stochasticLossHistory.append(lossSum / miniBatchSize + regVal) - val update = updater.compute( - weights, Vectors.fromBreeze(gradientSum / miniBatchSize), stepSize, i, regParam) - weights = update._1 - regVal = update._2 + if (miniBatchSize > 0) { + /** + * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration + * and regVal is the regularization value computed in the previous iteration as well. + */ + stochasticLossHistory.append(lossSum / miniBatchSize + regVal) + val update = updater.compute( + weights, Vectors.fromBreeze(gradientSum / miniBatchSize.toDouble), stepSize, i, regParam) + weights = update._1 + regVal = update._2 + } else { + logWarning(s"Iteration ($i/$numIterations). The size of sampled batch is zero") + } } logInfo("GradientDescent.runMiniBatchSGD finished. Last 10 stochastic losses %s".format( From a51118a34a4617c07373480c4b021e53124c3c00 Mon Sep 17 00:00:00 2001 From: q00251598 Date: Tue, 25 Nov 2014 04:01:56 -0800 Subject: [PATCH 025/305] [SPARK-4535][Streaming] Fix the error in comments change `NetworkInputDStream` to `ReceiverInputDStream` change `ReceiverInputTracker` to `ReceiverTracker` Author: q00251598 Closes #3400 from watermen/fix-comments and squashes the following commits: 75d795c [q00251598] change 'NetworkInputDStream' to 'ReceiverInputDStream' && change 'ReceiverInputTracker' to 'ReceiverTracker' --- .../examples/streaming/StatefulNetworkWordCount.scala | 2 +- .../examples/streaming/clickstream/PageViewStream.scala | 2 +- .../org/apache/spark/streaming/StreamingContext.scala | 2 +- .../spark/streaming/api/java/JavaStreamingContext.scala | 2 +- .../spark/streaming/dstream/ReceiverInputDStream.scala | 8 ++++---- .../spark/streaming/scheduler/ReceiverTracker.scala | 2 +- 6 files changed, 9 insertions(+), 9 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala index 514252b89e74e..ed186ea5650c4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/StatefulNetworkWordCount.scala @@ -64,7 +64,7 @@ object StatefulNetworkWordCount { // Initial RDD input to updateStateByKey val initialRDD = ssc.sparkContext.parallelize(List(("hello", 1), ("world", 1))) - // Create a NetworkInputDStream on target ip:port and count the + // Create a ReceiverInputDStream on target ip:port and count the // words in input stream of \n delimited test (eg. generated by 'nc') val lines = ssc.socketTextStream(args(0), args(1).toInt) val words = lines.flatMap(_.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala index d9b886eff77cc..55226c0a6df60 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/clickstream/PageViewStream.scala @@ -50,7 +50,7 @@ object PageViewStream { val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) - // Create a NetworkInputDStream on target host:port and convert each line to a PageView + // Create a ReceiverInputDStream on target host:port and convert each line to a PageView val pageViews = ssc.socketTextStream(host, port) .flatMap(_.split("\n")) .map(PageView.fromString(_)) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 54b219711efb9..ec59221459c88 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -187,7 +187,7 @@ class StreamingContext private[streaming] ( /** * Set each DStreams in this context to remember RDDs it generated in the last given duration. * DStreams remember RDDs only for a limited duration of time and releases them for garbage - * collection. This method allows the developer to specify how to long to remember the RDDs ( + * collection. This method allows the developer to specify how long to remember the RDDs ( * if the developer wishes to query old data outside the DStream computation). * @param duration Minimum duration that each DStream should remember its RDDs */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 7db66c69a6d73..d8695b8e05962 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -479,7 +479,7 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { /** * Sets each DStreams in this context to remember RDDs it generated in the last given duration. * DStreams remember RDDs only for a limited duration of duration and releases them for garbage - * collection. This method allows the developer to specify how to long to remember the RDDs ( + * collection. This method allows the developer to specify how long to remember the RDDs ( * if the developer wishes to query old data outside the DStream computation). * @param duration Minimum duration that each DStream should remember its RDDs */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala index 3e67161363e50..c834744631e02 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.scheduler.ReceivedBlockInfo /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] * that has to start a receiver on worker nodes to receive external data. - * Specific implementations of NetworkInputDStream must + * Specific implementations of ReceiverInputDStream must * define `the getReceiver()` function that gets the receiver object of type * [[org.apache.spark.streaming.receiver.Receiver]] that will be sent * to the workers to receive data. @@ -39,17 +39,17 @@ import org.apache.spark.streaming.scheduler.ReceivedBlockInfo abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) extends InputDStream[T](ssc_) { - /** This is an unique identifier for the network input stream. */ + /** This is an unique identifier for the receiver input stream. */ val id = ssc.getNewReceiverStreamId() /** * Gets the receiver object that will be sent to the worker nodes * to receive data. This method needs to defined by any specific implementation - * of a NetworkInputDStream. + * of a ReceiverInputDStream. */ def getReceiver(): Receiver[T] - // Nothing to start or stop as both taken care of by the ReceiverInputTracker. + // Nothing to start or stop as both taken care of by the ReceiverTracker. def start() {} def stop() {} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 1c3984d968d20..32e481dabc8ca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -46,7 +46,7 @@ private[streaming] case class DeregisterReceiver(streamId: Int, msg: String, err extends ReceiverTrackerMessage /** - * This class manages the execution of the receivers of NetworkInputDStreams. Instance of + * This class manages the execution of the receivers of ReceiverInputDStreams. Instance of * this class must be created after all input streams have been added and StreamingContext.start() * has been called because it needs the final set of input streams at the time of instantiation. * From fef27b29431c2adadc17580f26c23afa6a3bd1d2 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Tue, 25 Nov 2014 05:36:29 -0800 Subject: [PATCH 026/305] [SPARK-4381][Streaming]Add warning log when user set spark.master to local in Spark Streaming and there's no job executed Author: jerryshao Closes #3244 from jerryshao/SPARK-4381 and squashes the following commits: d2486c7 [jerryshao] Improve the warning log d726e85 [jerryshao] Add local[1] to the filter condition eca428b [jerryshao] Add warning log --- .../scala/org/apache/spark/streaming/StreamingContext.scala | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index ec59221459c88..ecab5510a8e7b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -121,6 +121,11 @@ class StreamingContext private[streaming] ( } } + if (sc.conf.get("spark.master") == "local" || sc.conf.get("spark.master") == "local[1]") { + logWarning("spark.master should be set as local[n], n > 1 in local mode if you have receivers" + + " to get data, otherwise Spark jobs will not get resources to process the received data.") + } + private[streaming] val conf = sc.conf private[streaming] val env = SparkEnv.get From d240760191f692ee7b88dfc82f06a31a340a88a2 Mon Sep 17 00:00:00 2001 From: arahuja Date: Tue, 25 Nov 2014 08:23:41 -0600 Subject: [PATCH 027/305] [SPARK-4344][DOCS] adding documentation on spark.yarn.user.classpath.first The documentation for the two parameters is the same with a pointer from the standalone parameter to the yarn parameter Author: arahuja Closes #3209 from arahuja/yarn-classpath-first-param and squashes the following commits: 51cb9b2 [arahuja] [SPARK-4344][DOCS] adding documentation for YARN on userClassPathFirst --- docs/configuration.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/configuration.md b/docs/configuration.md index 8839162c3a13e..0b77f5ab645c9 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -224,6 +224,7 @@ Apart from these, the following properties are also available, and may be useful (Experimental) Whether to give user-added jars precedence over Spark's own jars when loading classes in Executors. This feature can be used to mitigate conflicts between Spark's dependencies and user dependencies. It is currently an experimental feature. + (Currently, this setting does not work for YARN, see SPARK-2996 for more details). From 69cd53eae205eb10d52eaf38466db58a23b6ae81 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 25 Nov 2014 06:50:36 -0800 Subject: [PATCH 028/305] [SPARK-4601][Streaming] Set correct call site for streaming jobs so that it is displayed correctly on the Spark UI When running the NetworkWordCount, the description of the word count jobs are set as "getCallsite at DStream:xxx" . This should be set to the line number of the streaming application that has the output operation that led to the job being created. This is because the callsite is incorrectly set in the thread launching the jobs. This PR fixes that. Author: Tathagata Das Closes #3455 from tdas/streaming-callsite-fix and squashes the following commits: 69fc26f [Tathagata Das] Set correct call site for streaming jobs so that it is displayed correctly on the Spark UI --- .../org/apache/spark/streaming/dstream/ForEachDStream.scala | 1 + .../org/apache/spark/streaming/StreamingContextSuite.scala | 6 +++++- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala index 905bc723f69a9..1361c30395b57 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ForEachDStream.scala @@ -38,6 +38,7 @@ class ForEachDStream[T: ClassTag] ( parent.getOrCompute(time) match { case Some(rdd) => val jobFunc = () => { + ssc.sparkContext.setCallSite(creationSite) foreachFunc(rdd, time) } Some(new Job(time, jobFunc)) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 4b49c4d251645..9f352bdcb0893 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -336,16 +336,20 @@ package object testPackage extends Assertions { // Verify creation site of generated RDDs var rddGenerated = false - var rddCreationSiteCorrect = true + var rddCreationSiteCorrect = false + var foreachCallSiteCorrect = false inputStream.foreachRDD { rdd => rddCreationSiteCorrect = rdd.creationSite == creationSite + foreachCallSiteCorrect = + rdd.sparkContext.getCallSite().shortForm.contains("StreamingContextSuite") rddGenerated = true } ssc.start() eventually(timeout(10000 millis), interval(10 millis)) { assert(rddGenerated && rddCreationSiteCorrect, "RDD creation site was not correct") + assert(rddGenerated && foreachCallSiteCorrect, "Call site in foreachRDD was not correct") } } finally { ssc.stop() From bf1a6aaac577757a293a573fe8eae9669697310a Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 25 Nov 2014 11:07:11 -0800 Subject: [PATCH 029/305] [SPARK-4581][MLlib] Refactorize StandardScaler to improve the transformation performance The following optimizations are done to improve the StandardScaler model transformation performance. 1) Covert Breeze dense vector to primitive vector to reduce the overhead. 2) Since mean can be potentially a sparse vector, we explicitly convert it to dense primitive vector. 3) Have a local reference to `shift` and `factor` array so JVM can locate the value with one operation call. 4) In pattern matching part, we use the mllib SparseVector/DenseVector instead of breeze's vector to make the codebase cleaner. Benchmark with mnist8m dataset: Before, DenseVector withMean and withStd: 50.97secs DenseVector withMean and withoutStd: 42.11secs DenseVector withoutMean and withStd: 8.75secs SparseVector withoutMean and withStd: 5.437secs With this PR, DenseVector withMean and withStd: 5.76secs DenseVector withMean and withoutStd: 5.28secs DenseVector withoutMean and withStd: 5.30secs SparseVector withoutMean and withStd: 1.27secs Note that without the local reference copy of `factor` and `shift` arrays, the runtime is almost three time slower. DenseVector withMean and withStd: 18.15secs DenseVector withMean and withoutStd: 18.05secs DenseVector withoutMean and withStd: 18.54secs SparseVector withoutMean and withStd: 2.01secs The following code, ```scala while (i < size) { values(i) = (values(i) - shift(i)) * factor(i) i += 1 } ``` will generate the bytecode ``` L13 LINENUMBER 106 L13 FRAME FULL [org/apache/spark/mllib/feature/StandardScalerModel org/apache/spark/mllib/linalg/Vector org/apache/spark/mllib/linalg/Vector org/apache/spark/mllib/linalg/DenseVector T [D I I] [] ILOAD 7 ILOAD 6 IF_ICMPGE L14 L15 LINENUMBER 107 L15 ALOAD 5 ILOAD 7 ALOAD 5 ILOAD 7 DALOAD ALOAD 0 INVOKESPECIAL org/apache/spark/mllib/feature/StandardScalerModel.shift ()[D ILOAD 7 DALOAD DSUB ALOAD 0 INVOKESPECIAL org/apache/spark/mllib/feature/StandardScalerModel.factor ()[D ILOAD 7 DALOAD DMUL DASTORE L16 LINENUMBER 108 L16 ILOAD 7 ICONST_1 IADD ISTORE 7 GOTO L13 ``` , while with the local reference of the `shift` and `factor` arrays, the bytecode will be ``` L14 LINENUMBER 107 L14 ALOAD 0 INVOKESPECIAL org/apache/spark/mllib/feature/StandardScalerModel.factor ()[D ASTORE 9 L15 LINENUMBER 108 L15 FRAME FULL [org/apache/spark/mllib/feature/StandardScalerModel org/apache/spark/mllib/linalg/Vector [D org/apache/spark/mllib/linalg/Vector org/apache/spark/mllib/linalg/DenseVector T [D I I [D] [] ILOAD 8 ILOAD 7 IF_ICMPGE L16 L17 LINENUMBER 109 L17 ALOAD 6 ILOAD 8 ALOAD 6 ILOAD 8 DALOAD ALOAD 2 ILOAD 8 DALOAD DSUB ALOAD 9 ILOAD 8 DALOAD DMUL DASTORE L18 LINENUMBER 110 L18 ILOAD 8 ICONST_1 IADD ISTORE 8 GOTO L15 ``` You can see that with local reference, the both of the arrays will be in the stack, so JVM can access the value without calling `INVOKESPECIAL`. Author: DB Tsai Closes #3435 from dbtsai/standardscaler and squashes the following commits: 85885a9 [DB Tsai] revert to have lazy in shift array. daf2b06 [DB Tsai] Address the feedback cdb5cef [DB Tsai] small change 9c51eef [DB Tsai] style fc795e4 [DB Tsai] update 5bffd3d [DB Tsai] first commit --- .../spark/mllib/feature/StandardScaler.scala | 70 +++++++++++++------ 1 file changed, 50 insertions(+), 20 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala index 4dfd1f0ab8134..8c4c5db5258d5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/StandardScaler.scala @@ -17,11 +17,9 @@ package org.apache.spark.mllib.feature -import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} - import org.apache.spark.Logging import org.apache.spark.annotation.Experimental -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.rdd.RDD @@ -77,8 +75,8 @@ class StandardScalerModel private[mllib] ( require(mean.size == variance.size) - private lazy val factor: BDV[Double] = { - val f = BDV.zeros[Double](variance.size) + private lazy val factor: Array[Double] = { + val f = Array.ofDim[Double](variance.size) var i = 0 while (i < f.size) { f(i) = if (variance(i) != 0.0) 1.0 / math.sqrt(variance(i)) else 0.0 @@ -87,6 +85,11 @@ class StandardScalerModel private[mllib] ( f } + // Since `shift` will be only used in `withMean` branch, we have it as + // `lazy val` so it will be evaluated in that branch. Note that we don't + // want to create this array multiple times in `transform` function. + private lazy val shift: Array[Double] = mean.toArray + /** * Applies standardization transformation on a vector. * @@ -97,30 +100,57 @@ class StandardScalerModel private[mllib] ( override def transform(vector: Vector): Vector = { require(mean.size == vector.size) if (withMean) { - vector.toBreeze match { - case dv: BDV[Double] => - val output = vector.toBreeze.copy - var i = 0 - while (i < output.length) { - output(i) = (output(i) - mean(i)) * (if (withStd) factor(i) else 1.0) - i += 1 + // By default, Scala generates Java methods for member variables. So every time when + // the member variables are accessed, `invokespecial` will be called which is expensive. + // This can be avoid by having a local reference of `shift`. + val localShift = shift + vector match { + case dv: DenseVector => + val values = dv.values.clone() + val size = values.size + if (withStd) { + // Having a local reference of `factor` to avoid overhead as the comment before. + val localFactor = factor + var i = 0 + while (i < size) { + values(i) = (values(i) - localShift(i)) * localFactor(i) + i += 1 + } + } else { + var i = 0 + while (i < size) { + values(i) -= localShift(i) + i += 1 + } } - Vectors.fromBreeze(output) + Vectors.dense(values) case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) } } else if (withStd) { - vector.toBreeze match { - case dv: BDV[Double] => Vectors.fromBreeze(dv :* factor) - case sv: BSV[Double] => + // Having a local reference of `factor` to avoid overhead as the comment before. + val localFactor = factor + vector match { + case dv: DenseVector => + val values = dv.values.clone() + val size = values.size + var i = 0 + while(i < size) { + values(i) *= localFactor(i) + i += 1 + } + Vectors.dense(values) + case sv: SparseVector => // For sparse vector, the `index` array inside sparse vector object will not be changed, // so we can re-use it to save memory. - val output = new BSV[Double](sv.index, sv.data.clone(), sv.length) + val indices = sv.indices + val values = sv.values.clone() + val nnz = values.size var i = 0 - while (i < output.data.length) { - output.data(i) *= factor(output.index(i)) + while (i < nnz) { + values(i) *= localFactor(indices(i)) i += 1 } - Vectors.fromBreeze(output) + Vectors.sparse(sv.size, indices, values) case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) } } else { From 8838ad7c135a585cde015dc38b5cb23314502dd9 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 25 Nov 2014 14:16:27 -0800 Subject: [PATCH 030/305] [SPARK-4196][SPARK-4602][Streaming] Fix serialization issue in PairDStreamFunctions.saveAsNewAPIHadoopFiles Solves two JIRAs in one shot - Makes the ForechDStream created by saveAsNewAPIHadoopFiles serializable for checkpoints - Makes the default configuration object used saveAsNewAPIHadoopFiles be the Spark's hadoop configuration Author: Tathagata Das Closes #3457 from tdas/savefiles-fix and squashes the following commits: bb4729a [Tathagata Das] Same treatment for saveAsHadoopFiles b382ea9 [Tathagata Das] Fix serialization issue in PairDStreamFunctions.saveAsNewAPIHadoopFiles. --- .../dstream/PairDStreamFunctions.scala | 30 +++++----- .../spark/streaming/CheckpointSuite.scala | 56 ++++++++++++++++++- 2 files changed, 70 insertions(+), 16 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 3f03f42270252..98539e06b4e29 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -17,20 +17,17 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.streaming.StreamingContext._ - -import org.apache.spark.{Partitioner, HashPartitioner} -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} -import org.apache.hadoop.mapred.OutputFormat import org.apache.hadoop.conf.Configuration -import org.apache.spark.streaming.{Time, Duration} +import org.apache.hadoop.mapred.{JobConf, OutputFormat} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} + +import org.apache.spark.{HashPartitioner, Partitioner, SerializableWritable} +import org.apache.spark.rdd.RDD +import org.apache.spark.streaming.{Duration, Time} +import org.apache.spark.streaming.StreamingContext._ /** * Extra functions available on DStream of (key, value) pairs through an implicit conversion. @@ -671,11 +668,13 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: OutputFormat[_, _]], - conf: JobConf = new JobConf + conf: JobConf = new JobConf(ssc.sparkContext.hadoopConfiguration) ) { + // Wrap conf in SerializableWritable so that ForeachDStream can be serialized for checkpoints + val serializableConf = new SerializableWritable(conf) val saveFunc = (rdd: RDD[(K, V)], time: Time) => { val file = rddToFileName(prefix, suffix, time) - rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, conf) + rdd.saveAsHadoopFile(file, keyClass, valueClass, outputFormatClass, serializableConf.value) } self.foreachRDD(saveFunc) } @@ -702,11 +701,14 @@ class PairDStreamFunctions[K, V](self: DStream[(K,V)]) keyClass: Class[_], valueClass: Class[_], outputFormatClass: Class[_ <: NewOutputFormat[_, _]], - conf: Configuration = new Configuration + conf: Configuration = ssc.sparkContext.hadoopConfiguration ) { + // Wrap conf in SerializableWritable so that ForeachDStream can be serialized for checkpoints + val serializableConf = new SerializableWritable(conf) val saveFunc = (rdd: RDD[(K, V)], time: Time) => { val file = rddToFileName(prefix, suffix, time) - rdd.saveAsNewAPIHadoopFile(file, keyClass, valueClass, outputFormatClass, conf) + rdd.saveAsNewAPIHadoopFile( + file, keyClass, valueClass, outputFormatClass, serializableConf.value) } self.foreachRDD(saveFunc) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 77ff1ca780a58..c97998add8ffa 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -22,9 +22,14 @@ import java.nio.charset.Charset import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag + import com.google.common.io.Files -import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.io.{IntWritable, Text} +import org.apache.hadoop.mapred.TextOutputFormat +import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} + import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.{DStream, FileInputDStream} import org.apache.spark.streaming.util.ManualClock @@ -205,6 +210,51 @@ class CheckpointSuite extends TestSuiteBase { testCheckpointedOperation(input, operation, output, 7) } + test("recovery with saveAsHadoopFiles operation") { + val tempDir = Files.createTempDir() + try { + testCheckpointedOperation( + Seq(Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq()), + (s: DStream[String]) => { + val output = s.map(x => (x, 1)).reduceByKey(_ + _) + output.saveAsHadoopFiles( + tempDir.toURI.toString, + "result", + classOf[Text], + classOf[IntWritable], + classOf[TextOutputFormat[Text, IntWritable]]) + output + }, + Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + 3 + ) + } finally { + Utils.deleteRecursively(tempDir) + } + } + + test("recovery with saveAsNewAPIHadoopFiles operation") { + val tempDir = Files.createTempDir() + try { + testCheckpointedOperation( + Seq(Seq("a", "a", "b"), Seq("", ""), Seq(), Seq("a", "a", "b"), Seq("", ""), Seq()), + (s: DStream[String]) => { + val output = s.map(x => (x, 1)).reduceByKey(_ + _) + output.saveAsNewAPIHadoopFiles( + tempDir.toURI.toString, + "result", + classOf[Text], + classOf[IntWritable], + classOf[NewTextOutputFormat[Text, IntWritable]]) + output + }, + Seq(Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq(), Seq(("a", 2), ("b", 1)), Seq(("", 2)), Seq()), + 3 + ) + } finally { + Utils.deleteRecursively(tempDir) + } + } // This tests whether the StateDStream's RDD checkpoints works correctly such // that the system can recover from a master failure. This assumes as reliable, @@ -391,7 +441,9 @@ class CheckpointSuite extends TestSuiteBase { logInfo("Manual clock after advancing = " + clock.time) Thread.sleep(batchDuration.milliseconds) - val outputStream = ssc.graph.getOutputStreams.head.asInstanceOf[TestOutputStreamWithPartitions[V]] + val outputStream = ssc.graph.getOutputStreams.filter { dstream => + dstream.isInstanceOf[TestOutputStreamWithPartitions[V]] + }.head.asInstanceOf[TestOutputStreamWithPartitions[V]] outputStream.output.map(_.flatten) } } From 1b2ab1cd1b7cab9076f3c511188a610eda935701 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 25 Nov 2014 15:46:26 -0800 Subject: [PATCH 031/305] [SPARK-4592] Avoid duplicate worker registrations in standalone mode **Summary.** On failover, the Master may receive duplicate registrations from the same worker, causing the worker to exit. This is caused by this commit https://github.com/apache/spark/commit/4afe9a4852ebeb4cc77322a14225cd3dec165f3f, which adds logic for the worker to re-register with the master in case of failures. However, the following race condition may occur: (1) Master A fails and Worker attempts to reconnect to all masters (2) Master B takes over and notifies Worker (3) Worker responds by registering with Master B (4) Meanwhile, Worker's previous reconnection attempt reaches Master B, causing the same Worker to register with Master B twice **Fix.** Instead of attempting to register with all known masters, the worker should re-register with only the one that it has been communicating with. This is safe because the fact that a failover has occurred means the old master must have died. Then, when the worker is finally notified of a new master, it gives up on the old one in favor of the new one. **Caveat.** Even this fix is subject to more obscure race conditions. For instance, if Master B fails and Master A recovers immediately, then Master A may still observe duplicate worker registrations. However, this and other potential race conditions summarized in [SPARK-4592](https://issues.apache.org/jira/browse/SPARK-4592), are much, much less likely than the one described above, which is deterministically reproducible. Author: Andrew Or Closes #3447 from andrewor14/standalone-failover and squashes the following commits: 0d9716c [Andrew Or] Move re-registration logic to actor for thread-safety 79286dc [Andrew Or] Preserve old behavior for initial retries 83b321c [Andrew Or] Tweak wording 1fce6a9 [Andrew Or] Active master actor could be null in the beginning b6f269e [Andrew Or] Avoid duplicate worker registrations --- .../apache/spark/deploy/DeployMessage.scala | 2 + .../apache/spark/deploy/worker/Worker.scala | 52 ++++++++++++++++--- 2 files changed, 47 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index b9dd8557ee904..c46f84de8444a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -92,6 +92,8 @@ private[deploy] object DeployMessages { case object WorkDirCleanup // Sent to Worker actor periodically for cleaning up app folders + case object ReregisterWithMaster // used when a worker attempts to reconnect to a master + // AppClient to Master case class RegisterApplication(appDescription: ApplicationDescription) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index ca262de832e25..eb11163538b20 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -21,7 +21,6 @@ import java.io.File import java.io.IOException import java.text.SimpleDateFormat import java.util.{UUID, Date} -import java.util.concurrent.TimeUnit import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap @@ -177,6 +176,9 @@ private[spark] class Worker( throw new SparkException("Invalid spark URL: " + x) } connected = true + // Cancel any outstanding re-registration attempts because we found a new master + registrationRetryTimer.foreach(_.cancel()) + registrationRetryTimer = None } private def tryRegisterAllMasters() { @@ -187,7 +189,12 @@ private[spark] class Worker( } } - private def retryConnectToMaster() { + /** + * Re-register with the master because a network failure or a master failure has occurred. + * If the re-registration attempt threshold is exceeded, the worker exits with error. + * Note that for thread-safety this should only be called from the actor. + */ + private def reregisterWithMaster(): Unit = { Utils.tryOrExit { connectionAttemptCount += 1 if (registered) { @@ -195,12 +202,40 @@ private[spark] class Worker( registrationRetryTimer = None } else if (connectionAttemptCount <= TOTAL_REGISTRATION_RETRIES) { logInfo(s"Retrying connection to master (attempt # $connectionAttemptCount)") - tryRegisterAllMasters() + /** + * Re-register with the active master this worker has been communicating with. If there + * is none, then it means this worker is still bootstrapping and hasn't established a + * connection with a master yet, in which case we should re-register with all masters. + * + * It is important to re-register only with the active master during failures. Otherwise, + * if the worker unconditionally attempts to re-register with all masters, the following + * race condition may arise and cause a "duplicate worker" error detailed in SPARK-4592: + * + * (1) Master A fails and Worker attempts to reconnect to all masters + * (2) Master B takes over and notifies Worker + * (3) Worker responds by registering with Master B + * (4) Meanwhile, Worker's previous reconnection attempt reaches Master B, + * causing the same Worker to register with Master B twice + * + * Instead, if we only register with the known active master, we can assume that the + * old master must have died because another master has taken over. Note that this is + * still not safe if the old master recovers within this interval, but this is a much + * less likely scenario. + */ + if (master != null) { + master ! RegisterWorker( + workerId, host, port, cores, memory, webUi.boundPort, publicAddress) + } else { + // We are retrying the initial registration + tryRegisterAllMasters() + } + // We have exceeded the initial registration retry threshold + // All retries from now on should use a higher interval if (connectionAttemptCount == INITIAL_REGISTRATION_RETRIES) { registrationRetryTimer.foreach(_.cancel()) registrationRetryTimer = Some { context.system.scheduler.schedule(PROLONGED_REGISTRATION_RETRY_INTERVAL, - PROLONGED_REGISTRATION_RETRY_INTERVAL)(retryConnectToMaster) + PROLONGED_REGISTRATION_RETRY_INTERVAL, self, ReregisterWithMaster) } } } else { @@ -220,7 +255,7 @@ private[spark] class Worker( connectionAttemptCount = 0 registrationRetryTimer = Some { context.system.scheduler.schedule(INITIAL_REGISTRATION_RETRY_INTERVAL, - INITIAL_REGISTRATION_RETRY_INTERVAL)(retryConnectToMaster) + INITIAL_REGISTRATION_RETRY_INTERVAL, self, ReregisterWithMaster) } case Some(_) => logInfo("Not spawning another attempt to register with the master, since there is an" + @@ -400,12 +435,15 @@ private[spark] class Worker( logInfo(s"$x Disassociated !") masterDisconnected() - case RequestWorkerState => { + case RequestWorkerState => sender ! WorkerStateResponse(host, port, workerId, executors.values.toList, finishedExecutors.values.toList, drivers.values.toList, finishedDrivers.values.toList, activeMasterUrl, cores, memory, coresUsed, memoryUsed, activeMasterWebUiUrl) - } + + case ReregisterWithMaster => + reregisterWithMaster() + } private def masterDisconnected() { From 9afcbe494a3535a9bf7958429b72e989972f82d9 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 25 Nov 2014 15:48:02 -0800 Subject: [PATCH 032/305] [SPARK-4546] Improve HistoryServer first time user experience The documentation points the user to run the following ``` sbin/start-history-server.sh ``` The first thing this does is throw an exception that complains a log directory is not specified. The exception message itself does not say anything about what to set. Instead we should have a default and a landing page with a better message. The new default log directory is `file:/tmp/spark-events`. This is what it looks like as of this PR: ![after](https://issues.apache.org/jira/secure/attachment/12682985/after.png) Author: Andrew Or Closes #3411 from andrewor14/minor-history-improvements and squashes the following commits: f33d6b3 [Andrew Or] Point user to set config if default log dir does not exist fc4c17a [Andrew Or] Improve HistoryServer UX --- .../deploy/history/FsHistoryProvider.scala | 37 ++++++++++++------- .../spark/deploy/history/HistoryPage.scala | 8 +++- .../history/HistoryServerArguments.scala | 15 ++++---- docs/monitoring.md | 2 +- 4 files changed, 40 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index 2d1609b973607..82a54dbfb5330 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -29,22 +29,27 @@ import org.apache.spark.scheduler._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils +/** + * A class that provides application history from event logs stored in the file system. + * This provider checks for new finished applications in the background periodically and + * renders the history application UI by parsing the associated event logs. + */ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHistoryProvider with Logging { + import FsHistoryProvider._ + private val NOT_STARTED = "" // Interval between each check for event log updates private val UPDATE_INTERVAL_MS = conf.getInt("spark.history.fs.updateInterval", conf.getInt("spark.history.updateInterval", 10)) * 1000 - private val logDir = conf.get("spark.history.fs.logDirectory", null) - private val resolvedLogDir = Option(logDir) - .map { d => Utils.resolveURI(d) } - .getOrElse { throw new IllegalArgumentException("Logging directory must be specified.") } + private val logDir = conf.getOption("spark.history.fs.logDirectory") + .map { d => Utils.resolveURI(d).toString } + .getOrElse(DEFAULT_LOG_DIR) - private val fs = Utils.getHadoopFileSystem(resolvedLogDir, - SparkHadoopUtil.get.newConfiguration(conf)) + private val fs = Utils.getHadoopFileSystem(logDir, SparkHadoopUtil.get.newConfiguration(conf)) // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTimeMs = -1L @@ -87,14 +92,17 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis private def initialize() { // Validate the log directory. - val path = new Path(resolvedLogDir) + val path = new Path(logDir) if (!fs.exists(path)) { - throw new IllegalArgumentException( - "Logging directory specified does not exist: %s".format(resolvedLogDir)) + var msg = s"Log directory specified does not exist: $logDir." + if (logDir == DEFAULT_LOG_DIR) { + msg += " Did you configure the correct one through spark.fs.history.logDirectory?" + } + throw new IllegalArgumentException(msg) } if (!fs.getFileStatus(path).isDir) { throw new IllegalArgumentException( - "Logging directory specified is not a directory: %s".format(resolvedLogDir)) + "Logging directory specified is not a directory: %s".format(logDir)) } checkForLogs() @@ -134,8 +142,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } } - override def getConfig(): Map[String, String] = - Map("Event Log Location" -> resolvedLogDir.toString) + override def getConfig(): Map[String, String] = Map("Event log directory" -> logDir.toString) /** * Builds the application list based on the current contents of the log directory. @@ -146,7 +153,7 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis lastLogCheckTimeMs = getMonotonicTimeMs() logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTimeMs)) try { - val logStatus = fs.listStatus(new Path(resolvedLogDir)) + val logStatus = fs.listStatus(new Path(logDir)) val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() // Load all new logs from the log directory. Only directories that have a modification time @@ -244,6 +251,10 @@ private[history] class FsHistoryProvider(conf: SparkConf) extends ApplicationHis } +private object FsHistoryProvider { + val DEFAULT_LOG_DIR = "file:/tmp/spark-events" +} + private class FsApplicationHistoryInfo( val logDir: String, id: String, diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 0e249e51a77d8..5fdc350cd8512 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -58,7 +58,13 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { ++ appTable } else { -

No Completed Applications Found

+

No completed applications found!

++ +

Did you specify the correct logging directory? + Please verify your setting of + spark.history.fs.logDirectory and whether you have the permissions to + access it.
It is also possible that your application did not run to + completion or did not stop the SparkContext. +

} } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index 5bce32a04d16d..b1270ade9f750 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -17,14 +17,13 @@ package org.apache.spark.deploy.history -import org.apache.spark.SparkConf +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.util.Utils /** * Command-line parser for the master. */ -private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]) { - private var logDir: String = null +private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String]) extends Logging { private var propertiesFile: String = null parse(args.toList) @@ -32,7 +31,8 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] private def parse(args: List[String]): Unit = { args match { case ("--dir" | "-d") :: value :: tail => - logDir = value + logWarning("Setting log directory through the command line is deprecated as of " + + "Spark 1.1.0. Please set this through spark.history.fs.logDirectory instead.") conf.set("spark.history.fs.logDirectory", value) System.setProperty("spark.history.fs.logDirectory", value) parse(tail) @@ -78,9 +78,10 @@ private[spark] class HistoryServerArguments(conf: SparkConf, args: Array[String] | (default 50) |FsHistoryProvider options: | - | spark.history.fs.logDirectory Directory where app logs are stored (required) - | spark.history.fs.updateInterval How often to reload log data from storage (in seconds, - | default 10) + | spark.history.fs.logDirectory Directory where app logs are stored + | (default: file:/tmp/spark-events) + | spark.history.fs.updateInterval How often to reload log data from storage + | (in seconds, default: 10) |""".stripMargin) System.exit(exitCode) } diff --git a/docs/monitoring.md b/docs/monitoring.md index e3f81a76acdbb..f32cdef240d31 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -79,7 +79,7 @@ follows:
- + From 9bdf5da59036c0b052df756fc4a28d64677072e7 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?hushan=5B=E8=83=A1=E7=8F=8A=5D?= Date: Tue, 25 Nov 2014 15:51:08 -0800 Subject: [PATCH 033/305] Fix SPARK-4471: blockManagerIdFromJson function throws exception while B... MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fix [SPARK-4471](https://issues.apache.org/jira/browse/SPARK-4471): blockManagerIdFromJson function throws exception while BlockManagerId be null in MetadataFetchFailedException Author: hushan[胡珊] Closes #3340 from suyanNone/fix-blockmanagerId-jnothing-2 and squashes the following commits: 159f9a3 [hushan[胡珊]] Refine test code for blockmanager is null 4380d73 [hushan[胡珊]] remove useless blank line 3ccf651 [hushan[胡珊]] Fix SPARK-4471: blockManagerIdFromJson function throws exception while metadata fetch failed --- .../org/apache/spark/util/JsonProtocol.scala | 4 ++++ .../org/apache/spark/util/JsonProtocolSuite.scala | 15 ++++++++++++--- 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 7b5db1ed76265..e7b80e8774b9c 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -688,6 +688,10 @@ private[spark] object JsonProtocol { } def blockManagerIdFromJson(json: JValue): BlockManagerId = { + // On metadata fetch fail, block manager ID can be null (SPARK-4471) + if (json == JNothing) { + return null + } val executorId = (json \ "Executor ID").extract[String] val host = (json \ "Host").extract[String] val port = (json \ "Port").extract[Int] diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 0bc9492675863..593d6dd8c3794 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.util import java.util.Properties +import org.apache.spark.shuffle.MetadataFetchFailedException + import scala.collection.Map import org.json4s.jackson.JsonMethods._ @@ -116,10 +118,13 @@ class JsonProtocolSuite extends FunSuite { // TaskEndReason val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15), 17, 18, 19, "Some exception") + val fetchMetadataFailed = new MetadataFetchFailedException(17, + 19, "metadata Fetch failed exception").toTaskEndReason val exceptionFailure = new ExceptionFailure(exception, None) testTaskEndReason(Success) testTaskEndReason(Resubmitted) testTaskEndReason(fetchFailed) + testTaskEndReason(fetchMetadataFailed) testTaskEndReason(exceptionFailure) testTaskEndReason(TaskResultLost) testTaskEndReason(TaskKilled) @@ -431,9 +436,13 @@ class JsonProtocolSuite extends FunSuite { } private def assertEquals(bm1: BlockManagerId, bm2: BlockManagerId) { - assert(bm1.executorId === bm2.executorId) - assert(bm1.host === bm2.host) - assert(bm1.port === bm2.port) + if (bm1 == null || bm2 == null) { + assert(bm1 === bm2) + } else { + assert(bm1.executorId === bm2.executorId) + assert(bm1.host === bm2.host) + assert(bm1.port === bm2.port) + } } private def assertEquals(result1: JobResult, result2: JobResult) { From 7eba0fbe456c451122d7a2353ff0beca00f15223 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Nov 2014 16:07:09 -0800 Subject: [PATCH 034/305] [Spark-4509] Revert EC2 tag-based cluster membership patch This PR reverts changes related to tag-based cluster membership. As discussed in SPARK-3332, we didn't figure out a safe strategy to use tags to determine cluster membership, because tagging is not atomic. The following changes are reverted: SPARK-2333: 94053a7b766788bb62e2dbbf352ccbcc75f71fc0 SPARK-3213: 7faf755ae4f0cf510048e432340260a6e609066d SPARK-3608: 78d4220fa0bf2f9ee663e34bbf3544a5313b02f0. I tested launch, login, and destroy. It is easy to check the diff by comparing it to Josh's patch for branch-1.1: https://github.com/apache/spark/pull/2225/files JoshRosen I sent the PR to master. It might be easier for us to keep master and branch-1.2 the same at this time. We can always re-apply the patch once we figure out a stable solution. Author: Xiangrui Meng Closes #3453 from mengxr/SPARK-4509 and squashes the following commits: f0b708b [Xiangrui Meng] revert 94053a7b766788bb62e2dbbf352ccbcc75f71fc0 4298ea5 [Xiangrui Meng] revert 7faf755ae4f0cf510048e432340260a6e609066d 35963a1 [Xiangrui Meng] Revert "SPARK-3608 Break if the instance tag naming succeeds" --- docs/ec2-scripts.md | 14 ++++---- ec2/spark_ec2.py | 83 ++++++++++++--------------------------------- 2 files changed, 28 insertions(+), 69 deletions(-) diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index 530798f2b8022..66bf5f1a855ed 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -12,16 +12,14 @@ on the [Amazon Web Services site](http://aws.amazon.com/). `spark-ec2` is designed to manage multiple named clusters. You can launch a new cluster (telling the script its size and giving it a name), -shutdown an existing cluster, or log into a cluster. Each cluster -launches a set of instances, which are tagged with the cluster name, -and placed into EC2 security groups. If you don't specify a security -group, the `spark-ec2` script will create security groups based on the -cluster name you request. For example, a cluster named +shutdown an existing cluster, or log into a cluster. Each cluster is +identified by placing its machines into EC2 security groups whose names +are derived from the name of the cluster. For example, a cluster named `test` will contain a master node in a security group called `test-master`, and a number of slave nodes in a security group called -`test-slaves`. You can also specify a security group prefix to be used -in place of the cluster name. Machines in a cluster can be identified -by looking for the "Name" tag of the instance in the Amazon EC2 Console. +`test-slaves`. The `spark-ec2` script will create these security groups +for you based on the cluster name you request. You can also use them to +identify machines belonging to each cluster in the Amazon EC2 Console. # Before You Start diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index a5396c2375915..a4ab844a56adf 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -138,7 +138,7 @@ def parse_args(): help="The SSH user you want to connect as (default: %default)") parser.add_option( "--delete-groups", action="store_true", default=False, - help="When destroying a cluster, delete the security groups that were created.") + help="When destroying a cluster, delete the security groups that were created") parser.add_option( "--use-existing-master", action="store_true", default=False, help="Launch fresh slaves, but use an existing stopped master if possible") @@ -152,9 +152,6 @@ def parse_args(): parser.add_option( "--user-data", type="string", default="", help="Path to a user-data file (most AMI's interpret this as an initialization script)") - parser.add_option( - "--security-group-prefix", type="string", default=None, - help="Use this prefix for the security group rather than the cluster name.") parser.add_option( "--authorized-address", type="string", default="0.0.0.0/0", help="Address to authorize on created security groups (default: %default)") @@ -305,12 +302,8 @@ def launch_cluster(conn, opts, cluster_name): user_data_content = user_data_file.read() print "Setting up security groups..." - if opts.security_group_prefix is None: - master_group = get_or_make_group(conn, cluster_name + "-master") - slave_group = get_or_make_group(conn, cluster_name + "-slaves") - else: - master_group = get_or_make_group(conn, opts.security_group_prefix + "-master") - slave_group = get_or_make_group(conn, opts.security_group_prefix + "-slaves") + master_group = get_or_make_group(conn, cluster_name + "-master") + slave_group = get_or_make_group(conn, cluster_name + "-slaves") authorized_address = opts.authorized_address if master_group.rules == []: # Group was just now created master_group.authorize(src_group=master_group) @@ -335,11 +328,12 @@ def launch_cluster(conn, opts, cluster_name): slave_group.authorize('tcp', 60060, 60060, authorized_address) slave_group.authorize('tcp', 60075, 60075, authorized_address) - # Check if instances are already running with the cluster name + # Check if instances are already running in our groups existing_masters, existing_slaves = get_existing_cluster(conn, opts, cluster_name, die_on_error=False) if existing_slaves or (existing_masters and not opts.use_existing_master): - print >> stderr, ("ERROR: There are already instances for name: %s " % cluster_name) + print >> stderr, ("ERROR: There are already instances running in " + + "group %s or %s" % (master_group.name, slave_group.name)) sys.exit(1) # Figure out Spark AMI @@ -413,13 +407,9 @@ def launch_cluster(conn, opts, cluster_name): for r in reqs: id_to_req[r.id] = r active_instance_ids = [] - outstanding_request_ids = [] for i in my_req_ids: - if i in id_to_req: - if id_to_req[i].state == "active": - active_instance_ids.append(id_to_req[i].instance_id) - else: - outstanding_request_ids.append(i) + if i in id_to_req and id_to_req[i].state == "active": + active_instance_ids.append(id_to_req[i].instance_id) if len(active_instance_ids) == opts.slaves: print "All %d slaves granted" % opts.slaves reservations = conn.get_all_instances(active_instance_ids) @@ -428,8 +418,8 @@ def launch_cluster(conn, opts, cluster_name): slave_nodes += r.instances break else: - print "%d of %d slaves granted, waiting longer for request ids including %s" % ( - len(active_instance_ids), opts.slaves, outstanding_request_ids[0:10]) + print "%d of %d slaves granted, waiting longer" % ( + len(active_instance_ids), opts.slaves) except: print "Canceling spot instance requests" conn.cancel_spot_instance_requests(my_req_ids) @@ -488,59 +478,34 @@ def launch_cluster(conn, opts, cluster_name): # Give the instances descriptive names for master in master_nodes: - name = '{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id) - tag_instance(master, name) - + master.add_tag( + key='Name', + value='{cn}-master-{iid}'.format(cn=cluster_name, iid=master.id)) for slave in slave_nodes: - name = '{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id) - tag_instance(slave, name) + slave.add_tag( + key='Name', + value='{cn}-slave-{iid}'.format(cn=cluster_name, iid=slave.id)) # Return all the instances return (master_nodes, slave_nodes) -def tag_instance(instance, name): - for i in range(0, 5): - try: - instance.add_tag(key='Name', value=name) - break - except: - print "Failed attempt %i of 5 to tag %s" % ((i + 1), name) - if i == 5: - raise "Error - failed max attempts to add name tag" - time.sleep(5) - # Get the EC2 instances in an existing cluster if available. # Returns a tuple of lists of EC2 instance objects for the masters and slaves def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): print "Searching for existing cluster " + cluster_name + "..." - # Search all the spot instance requests, and copy any tags from the spot - # instance request to the cluster. - spot_instance_requests = conn.get_all_spot_instance_requests() - for req in spot_instance_requests: - if req.state != u'active': - continue - name = req.tags.get(u'Name', "") - if name.startswith(cluster_name): - reservations = conn.get_all_instances(instance_ids=[req.instance_id]) - for res in reservations: - active = [i for i in res.instances if is_active(i)] - for instance in active: - if instance.tags.get(u'Name') is None: - tag_instance(instance, name) - # Now proceed to detect master and slaves instances. reservations = conn.get_all_instances() master_nodes = [] slave_nodes = [] for res in reservations: active = [i for i in res.instances if is_active(i)] for inst in active: - name = inst.tags.get(u'Name', "") - if name.startswith(cluster_name + "-master"): + group_names = [g.name for g in inst.groups] + if group_names == [cluster_name + "-master"]: master_nodes.append(inst) - elif name.startswith(cluster_name + "-slave"): + elif group_names == [cluster_name + "-slaves"]: slave_nodes.append(inst) if any((master_nodes, slave_nodes)): print "Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes)) @@ -548,12 +513,12 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): return (master_nodes, slave_nodes) else: if master_nodes == [] and slave_nodes != []: - print >> sys.stderr, "ERROR: Could not find master in with name " + \ - cluster_name + "-master" + print >> sys.stderr, "ERROR: Could not find master in group " + cluster_name + "-master" else: print >> sys.stderr, "ERROR: Could not find any existing cluster" sys.exit(1) + # Deploy configuration files and run setup scripts on a newly launched # or started EC2 cluster. @@ -984,11 +949,7 @@ def real_main(): # Delete security groups as well if opts.delete_groups: print "Deleting security groups (this will take some time)..." - if opts.security_group_prefix is None: - group_names = [cluster_name + "-master", cluster_name + "-slaves"] - else: - group_names = [opts.security_group_prefix + "-master", - opts.security_group_prefix + "-slaves"] + group_names = [cluster_name + "-master", cluster_name + "-slaves"] wait_for_cluster_state( cluster_instances=(master_nodes + slave_nodes), cluster_state='terminated', From c251fd7405db57d3ab2686c38712601fd8f13ccd Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Tue, 25 Nov 2014 20:10:15 -0800 Subject: [PATCH 035/305] [SPARK-4583] [mllib] LogLoss for GradientBoostedTrees fix + doc updates Currently, the LogLoss used by GradientBoostedTrees has 2 issues: * the gradient (and therefore loss) does not match that used by Friedman (1999) * the error computation uses 0/1 accuracy, not log loss This PR updates LogLoss. It also adds some doc for boosting and forests. I tested it on sample data and made sure the log loss is monotonically decreasing with each boosting iteration. CC: mengxr manishamde codedeft Author: Joseph K. Bradley Closes #3439 from jkbradley/gbt-loss-fix and squashes the following commits: cfec17e [Joseph K. Bradley] removed forgotten temp comments a27eb6d [Joseph K. Bradley] corrections to last log loss commit ed5da2c [Joseph K. Bradley] updated LogLoss (boosting) for numerical stability 5e52bff [Joseph K. Bradley] * Removed the 1/2 from SquaredError. This also required updating the test suite since it effectively doubles the gradient and loss. * Added doc for developers within RandomForest. * Small cleanup in test suite (generating data only once) e57897a [Joseph K. Bradley] Fixed LogLoss for GradientBoostedTrees, and updated doc for losses, forests, and boosting --- .../mllib/tree/GradientBoostedTrees.scala | 18 +++-- .../spark/mllib/tree/RandomForest.scala | 44 ++++++++++- .../spark/mllib/tree/loss/AbsoluteError.scala | 26 +++---- .../spark/mllib/tree/loss/LogLoss.scala | 34 ++++++--- .../spark/mllib/tree/loss/SquaredError.scala | 22 +++--- .../tree/GradientBoostedTreesSuite.scala | 74 ++++++++++++------- 6 files changed, 146 insertions(+), 72 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala index cb4ddfc814f91..61f6b1313f82e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/GradientBoostedTrees.scala @@ -31,18 +31,20 @@ import org.apache.spark.storage.StorageLevel /** * :: Experimental :: - * A class that implements Stochastic Gradient Boosting for regression and binary classification. + * A class that implements + * [[http://en.wikipedia.org/wiki/Gradient_boosting Stochastic Gradient Boosting]] + * for regression and binary classification. * * The implementation is based upon: * J.H. Friedman. "Stochastic Gradient Boosting." 1999. * - * Notes: - * - This currently can be run with several loss functions. However, only SquaredError is - * fully supported. Specifically, the loss function should be used to compute the gradient - * (to re-label training instances on each iteration) and to weight weak hypotheses. - * Currently, gradients are computed correctly for the available loss functions, - * but weak hypothesis weights are not computed correctly for LogLoss or AbsoluteError. - * Running with those losses will likely behave reasonably, but lacks the same guarantees. + * Notes on Gradient Boosting vs. TreeBoost: + * - This implementation is for Stochastic Gradient Boosting, not for TreeBoost. + * - Both algorithms learn tree ensembles by minimizing loss functions. + * - TreeBoost (Friedman, 1999) additionally modifies the outputs at tree leaf nodes + * based on the loss function, whereas the original gradient boosting method does not. + * - When the loss is SquaredError, these methods give the same result, but they could differ + * for other loss functions. * * @param boostingStrategy Parameters for the gradient boosting algorithm. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index 3ae6fa2a0ec2f..482d3395516e7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -37,7 +37,8 @@ import org.apache.spark.util.Utils /** * :: Experimental :: - * A class which implements a random forest learning algorithm for classification and regression. + * A class that implements a [[http://en.wikipedia.org/wiki/Random_forest Random Forest]] + * learning algorithm for classification and regression. * It supports both continuous and categorical features. * * The settings for featureSubsetStrategy are based on the following references: @@ -70,6 +71,47 @@ private class RandomForest ( private val seed: Int) extends Serializable with Logging { + /* + ALGORITHM + This is a sketch of the algorithm to help new developers. + + The algorithm partitions data by instances (rows). + On each iteration, the algorithm splits a set of nodes. In order to choose the best split + for a given node, sufficient statistics are collected from the distributed data. + For each node, the statistics are collected to some worker node, and that worker selects + the best split. + + This setup requires discretization of continuous features. This binning is done in the + findSplitsBins() method during initialization, after which each continuous feature becomes + an ordered discretized feature with at most maxBins possible values. + + The main loop in the algorithm operates on a queue of nodes (nodeQueue). These nodes + lie at the periphery of the tree being trained. If multiple trees are being trained at once, + then this queue contains nodes from all of them. Each iteration works roughly as follows: + On the master node: + - Some number of nodes are pulled off of the queue (based on the amount of memory + required for their sufficient statistics). + - For random forests, if featureSubsetStrategy is not "all," then a subset of candidate + features are chosen for each node. See method selectNodesToSplit(). + On worker nodes, via method findBestSplits(): + - The worker makes one pass over its subset of instances. + - For each (tree, node, feature, split) tuple, the worker collects statistics about + splitting. Note that the set of (tree, node) pairs is limited to the nodes selected + from the queue for this iteration. The set of features considered can also be limited + based on featureSubsetStrategy. + - For each node, the statistics for that node are aggregated to a particular worker + via reduceByKey(). The designated worker chooses the best (feature, split) pair, + or chooses to stop splitting if the stopping criteria are met. + On the master node: + - The master collects all decisions about splitting nodes and updates the model. + - The updated model is passed to the workers on the next iteration. + This process continues until the node queue is empty. + + Most of the methods in this implementation support the statistics aggregation, which is + the heaviest part of the computation. In general, this implementation is bound by either + the cost of statistics computation on workers or by communicating the sufficient statistics. + */ + strategy.assertValid() require(numTrees > 0, s"RandomForest requires numTrees > 0, but was given numTrees = $numTrees.") require(RandomForest.supportedFeatureSubsetStrategies.contains(featureSubsetStrategy), diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala index e8288668094df..d1bde15e6b150 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/AbsoluteError.scala @@ -17,7 +17,6 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.SparkContext._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel @@ -25,11 +24,11 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * Class for least absolute error loss calculation. - * The features x and the corresponding label y is predicted using the function F. - * For each instance: - * Loss: |y - F| - * Negative gradient: sign(y - F) + * Class for absolute error loss calculation (for regression). + * + * The absolute (L1) error is defined as: + * |y - F(x)| + * where y is the label and F(x) is the model prediction for features x. */ @DeveloperApi object AbsoluteError extends Loss { @@ -37,7 +36,8 @@ object AbsoluteError extends Loss { /** * Method to calculate the gradients for the gradient boosting calculation for least * absolute error calculation. - * @param model Model of the weak learner + * The gradient with respect to F(x) is: sign(F(x) - y) + * @param model Ensemble model * @param point Instance of the training dataset * @return Loss gradient */ @@ -48,19 +48,17 @@ object AbsoluteError extends Loss { } /** - * Method to calculate error of the base learner for the gradient boosting calculation. + * Method to calculate loss of the base learner for the gradient boosting calculation. * Note: This method is not used by the gradient boosting algorithm but is useful for debugging * purposes. - * @param model Model of the weak learner. + * @param model Ensemble model * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @return + * @return Mean absolute error of model on data */ override def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { - val sumOfAbsolutes = data.map { y => + data.map { y => val err = model.predict(y.features) - y.label math.abs(err) - }.sum() - sumOfAbsolutes / data.count() + }.mean() } - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala index 8b8adb44aea92..7ce9fa6f86c42 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/LogLoss.scala @@ -24,12 +24,12 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * Class for least squares error loss calculation. + * Class for log loss calculation (for classification). + * This uses twice the binomial negative log likelihood, called "deviance" in Friedman (1999). * - * The features x and the corresponding label y is predicted using the function F. - * For each instance: - * Loss: log(1 + exp(-2yF)), y in {-1, 1} - * Negative gradient: 2y / ( 1 + exp(2yF)) + * The log loss is defined as: + * 2 log(1 + exp(-2 y F(x))) + * where y is a label in {-1, 1} and F(x) is the model prediction for features x. */ @DeveloperApi object LogLoss extends Loss { @@ -37,7 +37,8 @@ object LogLoss extends Loss { /** * Method to calculate the loss gradients for the gradient boosting calculation for binary * classification - * @param model Model of the weak learner + * The gradient with respect to F(x) is: - 4 y / (1 + exp(2 y F(x))) + * @param model Ensemble model * @param point Instance of the training dataset * @return Loss gradient */ @@ -45,19 +46,28 @@ object LogLoss extends Loss { model: TreeEnsembleModel, point: LabeledPoint): Double = { val prediction = model.predict(point.features) - 1.0 / (1.0 + math.exp(-prediction)) - point.label + - 4.0 * point.label / (1.0 + math.exp(2.0 * point.label * prediction)) } /** - * Method to calculate error of the base learner for the gradient boosting calculation. + * Method to calculate loss of the base learner for the gradient boosting calculation. * Note: This method is not used by the gradient boosting algorithm but is useful for debugging * purposes. - * @param model Model of the weak learner. + * @param model Ensemble model * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @return + * @return Mean log loss of model on data */ override def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { - val wrongPredictions = data.filter(lp => model.predict(lp.features) != lp.label).count() - wrongPredictions / data.count + data.map { case point => + val prediction = model.predict(point.features) + val margin = 2.0 * point.label * prediction + // The following are equivalent to 2.0 * log(1 + exp(-margin)) but are more numerically + // stable. + if (margin >= 0) { + 2.0 * math.log1p(math.exp(-margin)) + } else { + 2.0 * (-margin + math.log1p(math.exp(margin))) + } + }.mean() } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala index cfe395b1d0491..50ecaa2f86f35 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/SquaredError.scala @@ -17,7 +17,6 @@ package org.apache.spark.mllib.tree.loss -import org.apache.spark.SparkContext._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel @@ -25,12 +24,11 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * Class for least squares error loss calculation. + * Class for squared error loss calculation. * - * The features x and the corresponding label y is predicted using the function F. - * For each instance: - * Loss: (y - F)**2/2 - * Negative gradient: y - F + * The squared (L2) error is defined as: + * (y - F(x))**2 + * where y is the label and F(x) is the model prediction for features x. */ @DeveloperApi object SquaredError extends Loss { @@ -38,23 +36,24 @@ object SquaredError extends Loss { /** * Method to calculate the gradients for the gradient boosting calculation for least * squares error calculation. - * @param model Model of the weak learner + * The gradient with respect to F(x) is: - 2 (y - F(x)) + * @param model Ensemble model * @param point Instance of the training dataset * @return Loss gradient */ override def gradient( model: TreeEnsembleModel, point: LabeledPoint): Double = { - model.predict(point.features) - point.label + 2.0 * (model.predict(point.features) - point.label) } /** - * Method to calculate error of the base learner for the gradient boosting calculation. + * Method to calculate loss of the base learner for the gradient boosting calculation. * Note: This method is not used by the gradient boosting algorithm but is useful for debugging * purposes. - * @param model Model of the weak learner. + * @param model Ensemble model * @param data Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. - * @return + * @return Mean squared error of model on data */ override def computeError(model: TreeEnsembleModel, data: RDD[LabeledPoint]): Double = { data.map { y => @@ -62,5 +61,4 @@ object SquaredError extends Loss { err * err }.mean() } - } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala index f3f8eff2db300..d4d54cf4c9e2a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala @@ -35,32 +35,39 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { test("Regression with continuous features: SquaredError") { GradientBoostedTreesSuite.testCombinations.foreach { case (numIterations, learningRate, subsamplingRate) => - val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100) - val rdd = sc.parallelize(arr, 2) - - val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, - categoricalFeaturesInfo = Map.empty, subsamplingRate = subsamplingRate) - val boostingStrategy = - new BoostingStrategy(treeStrategy, SquaredError, numIterations, learningRate) - - val gbt = GradientBoostedTrees.train(rdd, boostingStrategy) - - assert(gbt.trees.size === numIterations) - EnsembleTestHelper.validateRegressor(gbt, arr, 0.03) - - val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) - val dt = DecisionTree.train(remappedInput, treeStrategy) - - // Make sure trees are the same. - assert(gbt.trees.head.toString == dt.toString) + GradientBoostedTreesSuite.randomSeeds.foreach { randomSeed => + val rdd = sc.parallelize(GradientBoostedTreesSuite.data, 2) + + val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, + categoricalFeaturesInfo = Map.empty, subsamplingRate = subsamplingRate) + val boostingStrategy = + new BoostingStrategy(treeStrategy, SquaredError, numIterations, learningRate) + + val gbt = GradientBoostedTrees.train(rdd, boostingStrategy) + + assert(gbt.trees.size === numIterations) + try { + EnsembleTestHelper.validateRegressor(gbt, GradientBoostedTreesSuite.data, 0.06) + } catch { + case e: java.lang.AssertionError => + println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," + + s" subsamplingRate=$subsamplingRate") + throw e + } + + val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) + val dt = DecisionTree.train(remappedInput, treeStrategy) + + // Make sure trees are the same. + assert(gbt.trees.head.toString == dt.toString) + } } } test("Regression with continuous features: Absolute Error") { GradientBoostedTreesSuite.testCombinations.foreach { case (numIterations, learningRate, subsamplingRate) => - val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100) - val rdd = sc.parallelize(arr, 2) + val rdd = sc.parallelize(GradientBoostedTreesSuite.data, 2) val treeStrategy = new Strategy(algo = Regression, impurity = Variance, maxDepth = 2, categoricalFeaturesInfo = Map.empty, subsamplingRate = subsamplingRate) @@ -70,7 +77,14 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { val gbt = GradientBoostedTrees.train(rdd, boostingStrategy) assert(gbt.trees.size === numIterations) - EnsembleTestHelper.validateRegressor(gbt, arr, 0.85, "mae") + try { + EnsembleTestHelper.validateRegressor(gbt, GradientBoostedTreesSuite.data, 0.85, "mae") + } catch { + case e: java.lang.AssertionError => + println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," + + s" subsamplingRate=$subsamplingRate") + throw e + } val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) val dt = DecisionTree.train(remappedInput, treeStrategy) @@ -83,8 +97,7 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { test("Binary classification with continuous features: Log Loss") { GradientBoostedTreesSuite.testCombinations.foreach { case (numIterations, learningRate, subsamplingRate) => - val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100) - val rdd = sc.parallelize(arr, 2) + val rdd = sc.parallelize(GradientBoostedTreesSuite.data, 2) val treeStrategy = new Strategy(algo = Classification, impurity = Variance, maxDepth = 2, numClassesForClassification = 2, categoricalFeaturesInfo = Map.empty, @@ -95,7 +108,14 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { val gbt = GradientBoostedTrees.train(rdd, boostingStrategy) assert(gbt.trees.size === numIterations) - EnsembleTestHelper.validateClassifier(gbt, arr, 0.9) + try { + EnsembleTestHelper.validateClassifier(gbt, GradientBoostedTreesSuite.data, 0.9) + } catch { + case e: java.lang.AssertionError => + println(s"FAILED for numIterations=$numIterations, learningRate=$learningRate," + + s" subsamplingRate=$subsamplingRate") + throw e + } val remappedInput = rdd.map(x => new LabeledPoint((x.label * 2) - 1, x.features)) val ensembleStrategy = treeStrategy.copy @@ -113,5 +133,9 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { object GradientBoostedTreesSuite { // Combinations for estimators, learning rates and subsamplingRate - val testCombinations = Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 1.0, 0.75), (10, 0.1, 0.75)) + val testCombinations = Array((10, 1.0, 1.0), (10, 0.1, 1.0), (10, 0.5, 0.75), (10, 0.1, 0.75)) + + val randomSeeds = Array(681283, 4398) + + val data = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures = 10, 100) } From 4d95526a75ad1630554683fe7a7e583da44ba6e4 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 25 Nov 2014 23:10:19 -0500 Subject: [PATCH 036/305] [HOTFIX]: Adding back without-hive dist --- dev/create-release/create-release.sh | 1 + 1 file changed, 1 insertion(+) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 8a0b0348db8c0..e0aca467ac949 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -197,6 +197,7 @@ make_binary_release "hadoop2.3" "-Phadoop-2.3 -Phive -Phive-thriftserver -Pyarn" make_binary_release "hadoop2.4" "-Phadoop-2.4 -Phive -Phive-thriftserver -Pyarn" & make_binary_release "mapr3" "-Pmapr3 -Phive -Phive-thriftserver" & make_binary_release "mapr4" "-Pmapr4 -Pyarn -Phive -Phive-thriftserver" & +make_binary_release "hadoop2.4-without-hive" "-Phadoop-2.4 -Pyarn" & wait # Copy data From b5fb1410c5eed1156decb4f9fcc22436a658ce4d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 25 Nov 2014 20:11:40 -0800 Subject: [PATCH 037/305] [SPARK-4604][MLLIB] make MatrixFactorizationModel public User could construct an MF model directly. I added a note about the performance. Author: Xiangrui Meng Closes #3459 from mengxr/SPARK-4604 and squashes the following commits: f64bcd3 [Xiangrui Meng] organize imports ed08214 [Xiangrui Meng] check preconditions and unit tests a624c12 [Xiangrui Meng] make MatrixFactorizationModel public --- .../MatrixFactorizationModel.scala | 28 +++++++++- .../MatrixFactorizationModelSuite.scala | 56 +++++++++++++++++++ 2 files changed, 81 insertions(+), 3 deletions(-) create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 969e23be21623..ed2f8b41bcae5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -21,23 +21,45 @@ import java.lang.{Integer => JavaInteger} import org.jblas.DoubleMatrix -import org.apache.spark.SparkContext._ +import org.apache.spark.Logging import org.apache.spark.api.java.{JavaPairRDD, JavaRDD} import org.apache.spark.rdd.RDD +import org.apache.spark.storage.StorageLevel /** * Model representing the result of matrix factorization. * + * Note: If you create the model directly using constructor, please be aware that fast prediction + * requires cached user/product features and their associated partitioners. + * * @param rank Rank for the features in this model. * @param userFeatures RDD of tuples where each tuple represents the userId and * the features computed for this user. * @param productFeatures RDD of tuples where each tuple represents the productId * and the features computed for this product. */ -class MatrixFactorizationModel private[mllib] ( +class MatrixFactorizationModel( val rank: Int, val userFeatures: RDD[(Int, Array[Double])], - val productFeatures: RDD[(Int, Array[Double])]) extends Serializable { + val productFeatures: RDD[(Int, Array[Double])]) extends Serializable with Logging { + + require(rank > 0) + validateFeatures("User", userFeatures) + validateFeatures("Product", productFeatures) + + /** Validates factors and warns users if there are performance concerns. */ + private def validateFeatures(name: String, features: RDD[(Int, Array[Double])]): Unit = { + require(features.first()._2.size == rank, + s"$name feature dimension does not match the rank $rank.") + if (features.partitioner.isEmpty) { + logWarning(s"$name factor does not have a partitioner. " + + "Prediction on individual records could be slow.") + } + if (features.getStorageLevel == StorageLevel.NONE) { + logWarning(s"$name factor is not cached. Prediction could be slow.") + } + } + /** Predict the rating of one user for one product. */ def predict(user: Int, product: Int): Double = { val userVector = new DoubleMatrix(userFeatures.lookup(user).head) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala new file mode 100644 index 0000000000000..b9caecc904a23 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModelSuite.scala @@ -0,0 +1,56 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.recommendation + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.rdd.RDD + +class MatrixFactorizationModelSuite extends FunSuite with MLlibTestSparkContext { + + val rank = 2 + var userFeatures: RDD[(Int, Array[Double])] = _ + var prodFeatures: RDD[(Int, Array[Double])] = _ + + override def beforeAll(): Unit = { + super.beforeAll() + userFeatures = sc.parallelize(Seq((0, Array(1.0, 2.0)), (1, Array(3.0, 4.0)))) + prodFeatures = sc.parallelize(Seq((2, Array(5.0, 6.0)))) + } + + test("constructor") { + val model = new MatrixFactorizationModel(rank, userFeatures, prodFeatures) + assert(model.predict(0, 2) ~== 17.0 relTol 1e-14) + + intercept[IllegalArgumentException] { + new MatrixFactorizationModel(1, userFeatures, prodFeatures) + } + + val userFeatures1 = sc.parallelize(Seq((0, Array(1.0)), (1, Array(3.0)))) + intercept[IllegalArgumentException] { + new MatrixFactorizationModel(rank, userFeatures1, prodFeatures) + } + + val prodFeatures1 = sc.parallelize(Seq((2, Array(5.0)))) + intercept[IllegalArgumentException] { + new MatrixFactorizationModel(rank, userFeatures, prodFeatures1) + } + } +} From f5f2d27385c243959f03a9d78a149d5f405b2f50 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 25 Nov 2014 23:57:04 -0500 Subject: [PATCH 038/305] [SPARK-4516] Cap default number of Netty threads at 8 In practice, only 2-4 cores should be required to transfer roughly 10 Gb/s, and each core that we use will have an initial overhead of roughly 32 MB of off-heap memory, which comes at a premium. Thus, this value should still retain maximum throughput and reduce wasted off-heap memory allocation. It can be overridden by setting the number of serverThreads and clientThreads manually in Spark's configuration. Author: Aaron Davidson Closes #3469 from aarondav/fewer-pools2 and squashes the following commits: 087c59f [Aaron Davidson] [SPARK-4516] Cap default number of Netty threads at 8 --- .../network/netty/SparkTransportConf.scala | 44 ++++++++++++++++--- 1 file changed, 37 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala index ce4225cae6d88..cef203006d685 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala @@ -20,7 +20,24 @@ package org.apache.spark.network.netty import org.apache.spark.SparkConf import org.apache.spark.network.util.{TransportConf, ConfigProvider} +/** + * Provides a utility for transforming from a SparkConf inside a Spark JVM (e.g., Executor, + * Driver, or a standalone shuffle service) into a TransportConf with details on our environment + * like the number of cores that are allocated to this JVM. + */ object SparkTransportConf { + /** + * Specifies an upper bound on the number of Netty threads that Spark requires by default. + * In practice, only 2-4 cores should be required to transfer roughly 10 Gb/s, and each core + * that we use will have an initial overhead of roughly 32 MB of off-heap memory, which comes + * at a premium. + * + * Thus, this value should still retain maximum throughput and reduce wasted off-heap memory + * allocation. It can be overridden by setting the number of serverThreads and clientThreads + * manually in Spark's configuration. + */ + private val MAX_DEFAULT_NETTY_THREADS = 8 + /** * Utility for creating a [[TransportConf]] from a [[SparkConf]]. * @param numUsableCores if nonzero, this will restrict the server and client threads to only @@ -29,15 +46,28 @@ object SparkTransportConf { */ def fromSparkConf(_conf: SparkConf, numUsableCores: Int = 0): TransportConf = { val conf = _conf.clone - if (numUsableCores > 0) { - // Only set if serverThreads/clientThreads not already set. - conf.set("spark.shuffle.io.serverThreads", - conf.get("spark.shuffle.io.serverThreads", numUsableCores.toString)) - conf.set("spark.shuffle.io.clientThreads", - conf.get("spark.shuffle.io.clientThreads", numUsableCores.toString)) - } + + // Specify thread configuration based on our JVM's allocation of cores (rather than necessarily + // assuming we have all the machine's cores). + // NB: Only set if serverThreads/clientThreads not already set. + val numThreads = defaultNumThreads(numUsableCores) + conf.set("spark.shuffle.io.serverThreads", + conf.get("spark.shuffle.io.serverThreads", numThreads.toString)) + conf.set("spark.shuffle.io.clientThreads", + conf.get("spark.shuffle.io.clientThreads", numThreads.toString)) + new TransportConf(new ConfigProvider { override def get(name: String): String = conf.get(name) }) } + + /** + * Returns the default number of threads for both the Netty client and server thread pools. + * If numUsableCores is 0, we will use Runtime get an approximate number of available cores. + */ + private def defaultNumThreads(numUsableCores: Int): Int = { + val availableCores = + if (numUsableCores > 0) numUsableCores else Runtime.getRuntime.availableProcessors() + math.min(availableCores, MAX_DEFAULT_NETTY_THREADS) + } } From 346bc17a2ec8fc9e6eaff90733aa1e8b6b46883e Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 26 Nov 2014 00:32:45 -0500 Subject: [PATCH 039/305] [SPARK-4516] Avoid allocating Netty PooledByteBufAllocators unnecessarily Turns out we are allocating an allocator pool for every TransportClient (which means that the number increases with the number of nodes in the cluster), when really we should just reuse one for all clients. This patch, as expected, greatly decreases off-heap memory allocation, and appears to make allocation only proportional to the number of cores. Author: Aaron Davidson Closes #3465 from aarondav/fewer-pools and squashes the following commits: 36c49da [Aaron Davidson] [SPARK-4516] Avoid allocating unnecessarily Netty PooledByteBufAllocators --- .../spark/network/client/TransportClientFactory.java | 12 +++++------- .../org/apache/spark/network/util/NettyUtils.java | 6 +++--- 2 files changed, 8 insertions(+), 10 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 76bce8592816a..9afd5decd5e6b 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -19,7 +19,6 @@ import java.io.Closeable; import java.io.IOException; -import java.lang.reflect.Field; import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.List; @@ -37,7 +36,6 @@ import io.netty.channel.ChannelOption; import io.netty.channel.EventLoopGroup; import io.netty.channel.socket.SocketChannel; -import io.netty.util.internal.PlatformDependent; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -67,6 +65,7 @@ public class TransportClientFactory implements Closeable { private final Class socketChannelClass; private EventLoopGroup workerGroup; + private PooledByteBufAllocator pooledAllocator; public TransportClientFactory( TransportContext context, @@ -80,6 +79,8 @@ public TransportClientFactory( this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode); // TODO: Make thread pool name configurable. this.workerGroup = NettyUtils.createEventLoop(ioMode, conf.clientThreads(), "shuffle-client"); + this.pooledAllocator = NettyUtils.createPooledByteBufAllocator( + conf.preferDirectBufs(), false /* allowCache */, conf.clientThreads()); } /** @@ -115,11 +116,8 @@ public TransportClient createClient(String remoteHost, int remotePort) throws IO // Disable Nagle's Algorithm since we don't want packets to wait .option(ChannelOption.TCP_NODELAY, true) .option(ChannelOption.SO_KEEPALIVE, true) - .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()); - - // Use pooled buffers to reduce temporary buffer allocation - bootstrap.option(ChannelOption.ALLOCATOR, NettyUtils.createPooledByteBufAllocator( - conf.preferDirectBufs(), false /* allowCache */, conf.clientThreads())); + .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()) + .option(ChannelOption.ALLOCATOR, pooledAllocator); final AtomicReference clientRef = new AtomicReference(); diff --git a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java index 5c654a6fd6ebe..b3991a6577cfe 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -109,9 +109,9 @@ public static String getRemoteAddress(Channel channel) { /** * Create a pooled ByteBuf allocator but disables the thread-local cache. Thread-local caches - * are disabled because the ByteBufs are allocated by the event loop thread, but released by the - * executor thread rather than the event loop thread. Those thread-local caches actually delay - * the recycling of buffers, leading to larger memory usage. + * are disabled for TransportClients because the ByteBufs are allocated by the event loop thread, + * but released by the executor thread rather than the event loop thread. Those thread-local + * caches actually delay the recycling of buffers, leading to larger memory usage. */ public static PooledByteBufAllocator createPooledByteBufAllocator( boolean allowDirectBufs, From e7f4d2534bb3361ec4b7af0d42bc798a7a425226 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 25 Nov 2014 23:15:58 -0800 Subject: [PATCH 040/305] [SPARK-4612] Reduce task latency and increase scheduling throughput by making configuration initialization lazy https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/executor/Executor.scala#L337 creates a configuration object for every task that is launched, even if there is no new dependent file/JAR to update. This is a heavy-weight creation that should be avoided if there is no new file/JAR to update. This PR makes that creation lazy. Quick local test in spark-perf scheduling throughput tests gives the following numbers in a local standalone scheduler mode. 1 job with 10000 tasks: before 7.8395 seconds, after 2.6415 seconds = 3x increase in task scheduling throughput pwendell JoshRosen Author: Tathagata Das Closes #3463 from tdas/lazy-config and squashes the following commits: c791c1e [Tathagata Das] Reduce task latency by making configuration initialization lazy --- core/src/main/scala/org/apache/spark/executor/Executor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 5fa584591d935..835157fc520aa 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -334,7 +334,7 @@ private[spark] class Executor( * SparkContext. Also adds any new JARs we fetched to the class loader. */ private def updateDependencies(newFiles: HashMap[String, Long], newJars: HashMap[String, Long]) { - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) + lazy val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) synchronized { // Fetch missing dependencies for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { From 288ce583b05004a8c71dcd836fab23caff5d4ba7 Mon Sep 17 00:00:00 2001 From: "Joseph E. Gonzalez" Date: Wed, 26 Nov 2014 00:55:28 -0800 Subject: [PATCH 041/305] Removing confusing TripletFields After additional discussion with rxin, I think having all the possible `TripletField` options is confusing. This pull request reduces the triplet fields to: ```java /** * None of the triplet fields are exposed. */ public static final TripletFields None = new TripletFields(false, false, false); /** * Expose only the edge field and not the source or destination field. */ public static final TripletFields EdgeOnly = new TripletFields(false, false, true); /** * Expose the source and edge fields but not the destination field. (Same as Src) */ public static final TripletFields Src = new TripletFields(true, false, true); /** * Expose the destination and edge fields but not the source field. (Same as Dst) */ public static final TripletFields Dst = new TripletFields(false, true, true); /** * Expose all the fields (source, edge, and destination). */ public static final TripletFields All = new TripletFields(true, true, true); ``` Author: Joseph E. Gonzalez Closes #3472 from jegonzal/SimplifyTripletFields and squashes the following commits: 91796b5 [Joseph E. Gonzalez] removing confusing triplet fields --- .../org/apache/spark/graphx/GraphOps.scala | 6 ++-- .../apache/spark/graphx/TripletFields.java | 29 ++----------------- .../apache/spark/graphx/lib/PageRank.scala | 4 +-- .../org/apache/spark/graphx/GraphSuite.scala | 2 +- 4 files changed, 8 insertions(+), 33 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index d5150382d599b..116d1ea700175 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -129,15 +129,15 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali ctx.sendToSrc(Array((ctx.dstId, ctx.dstAttr))) ctx.sendToDst(Array((ctx.srcId, ctx.srcAttr))) }, - (a, b) => a ++ b, TripletFields.SrcDstOnly) + (a, b) => a ++ b, TripletFields.All) case EdgeDirection.In => graph.aggregateMessages[Array[(VertexId,VD)]]( ctx => ctx.sendToDst(Array((ctx.srcId, ctx.srcAttr))), - (a, b) => a ++ b, TripletFields.SrcOnly) + (a, b) => a ++ b, TripletFields.Src) case EdgeDirection.Out => graph.aggregateMessages[Array[(VertexId,VD)]]( ctx => ctx.sendToSrc(Array((ctx.dstId, ctx.dstAttr))), - (a, b) => a ++ b, TripletFields.DstOnly) + (a, b) => a ++ b, TripletFields.Dst) case EdgeDirection.Both => throw new SparkException("collectEdges does not support EdgeDirection.Both. Use" + "EdgeDirection.Either instead.") diff --git a/graphx/src/main/scala/org/apache/spark/graphx/TripletFields.java b/graphx/src/main/scala/org/apache/spark/graphx/TripletFields.java index 8dfccfe2e23bd..7eb4ae0f44602 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/TripletFields.java +++ b/graphx/src/main/scala/org/apache/spark/graphx/TripletFields.java @@ -55,40 +55,15 @@ public TripletFields(boolean useSrc, boolean useDst, boolean useEdge) { */ public static final TripletFields EdgeOnly = new TripletFields(false, false, true); - /** - * Expose only the source field and not the edge or destination field. - */ - public static final TripletFields SrcOnly = new TripletFields(true, false, false); - - /** - * Expose only the destination field and not the edge or source field. - */ - public static final TripletFields DstOnly = new TripletFields(false, true, false); - - /** - * Expose the source and destination fields but not the edge field. - */ - public static final TripletFields SrcDstOnly = new TripletFields(true, true, false); - /** * Expose the source and edge fields but not the destination field. (Same as Src) */ - public static final TripletFields SrcAndEdge = new TripletFields(true, false, true); - - /** - * Expose the source and edge fields but not the destination field. (Same as SrcAndEdge) - */ - public static final TripletFields Src = SrcAndEdge; + public static final TripletFields Src = new TripletFields(true, false, true); /** * Expose the destination and edge fields but not the source field. (Same as Dst) */ - public static final TripletFields DstAndEdge = new TripletFields(false, true, true); - - /** - * Expose the destination and edge fields but not the source field. (Same as DstAndEdge) - */ - public static final TripletFields Dst = DstAndEdge; + public static final TripletFields Dst = new TripletFields(false, true, true); /** * Expose all the fields (source, edge, and destination). diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index e40ae0d615466..e139959c3f5c1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -85,7 +85,7 @@ object PageRank extends Logging { // Associate the degree with each vertex .outerJoinVertices(graph.outDegrees) { (vid, vdata, deg) => deg.getOrElse(0) } // Set the weight on the edges based on the degree - .mapTriplets( e => 1.0 / e.srcAttr, TripletFields.SrcOnly ) + .mapTriplets( e => 1.0 / e.srcAttr, TripletFields.Src ) // Set the vertex attributes to the initial pagerank values .mapVertices( (id, attr) => resetProb ) @@ -97,7 +97,7 @@ object PageRank extends Logging { // Compute the outgoing rank contributions of each vertex, perform local preaggregation, and // do the final aggregation at the receiving vertices. Requires a shuffle for aggregation. val rankUpdates = rankGraph.aggregateMessages[Double]( - ctx => ctx.sendToDst(ctx.srcAttr * ctx.attr), _ + _, TripletFields.SrcAndEdge) + ctx => ctx.sendToDst(ctx.srcAttr * ctx.attr), _ + _, TripletFields.Src) // Apply the final rank updates to get the new ranks, using join to preserve ranks of vertices // that didn't receive a message. Requires a shuffle for broadcasting updated ranks to the diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index df773db6e4326..a05d1ddb21295 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -328,7 +328,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { "expected ctx.dstAttr to be null due to TripletFields, but it was " + ctx.dstAttr) } ctx.sendToDst(ctx.srcAttr) - }, _ + _, TripletFields.SrcOnly) + }, _ + _, TripletFields.Src) assert(agg.collect().toSet === (1 to n).map(x => (x: VertexId, "v")).toSet) } } From 561d31d2f13cc7b1112ba9f9aa8f08bcd032aebb Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Nov 2014 08:22:50 -0800 Subject: [PATCH 042/305] [SPARK-4614][MLLIB] Slight API changes in Matrix and Matrices Before we have a full picture of the operators we want to add, it might be safer to hide `Matrix.transposeMultiply` in 1.2.0. Another update we want to change is `Matrix.randn` and `Matrix.rand`, both of which should take a `Random` implementation. Otherwise, it is very likely to produce inconsistent RDDs. I also added some unit tests for matrix factory methods. All APIs are new in 1.2, so there is no incompatible changes. brkyvz Author: Xiangrui Meng Closes #3468 from mengxr/SPARK-4614 and squashes the following commits: 3b0e4e2 [Xiangrui Meng] add mima excludes 6bfd8a4 [Xiangrui Meng] hide transposeMultiply; add rng to rand and randn; add unit tests --- .../apache/spark/mllib/linalg/Matrices.scala | 20 ++++---- .../spark/mllib/linalg/MatricesSuite.scala | 50 +++++++++++++++++++ project/MimaExcludes.scala | 6 +++ 3 files changed, 65 insertions(+), 11 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index 2cc52e94282ba..327366a1a3a82 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -17,12 +17,10 @@ package org.apache.spark.mllib.linalg -import java.util.Arrays +import java.util.{Random, Arrays} import breeze.linalg.{Matrix => BM, DenseMatrix => BDM, CSCMatrix => BSM} -import org.apache.spark.util.random.XORShiftRandom - /** * Trait for a local matrix. */ @@ -67,14 +65,14 @@ sealed trait Matrix extends Serializable { } /** Convenience method for `Matrix`^T^-`DenseMatrix` multiplication. */ - def transposeMultiply(y: DenseMatrix): DenseMatrix = { + private[mllib] def transposeMultiply(y: DenseMatrix): DenseMatrix = { val C: DenseMatrix = Matrices.zeros(numCols, y.numCols).asInstanceOf[DenseMatrix] BLAS.gemm(true, false, 1.0, this, y, 0.0, C) C } /** Convenience method for `Matrix`^T^-`DenseVector` multiplication. */ - def transposeMultiply(y: DenseVector): DenseVector = { + private[mllib] def transposeMultiply(y: DenseVector): DenseVector = { val output = new DenseVector(new Array[Double](numCols)) BLAS.gemv(true, 1.0, this, y, 0.0, output) output @@ -291,22 +289,22 @@ object Matrices { * Generate a `DenseMatrix` consisting of i.i.d. uniform random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix + * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in U(0, 1) */ - def rand(numRows: Int, numCols: Int): Matrix = { - val rand = new XORShiftRandom - new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rand.nextDouble())) + def rand(numRows: Int, numCols: Int, rng: Random): Matrix = { + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble())) } /** * Generate a `DenseMatrix` consisting of i.i.d. gaussian random numbers. * @param numRows number of rows of the matrix * @param numCols number of columns of the matrix + * @param rng a random number generator * @return `DenseMatrix` with size `numRows` x `numCols` and values in N(0, 1) */ - def randn(numRows: Int, numCols: Int): Matrix = { - val rand = new XORShiftRandom - new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rand.nextGaussian())) + def randn(numRows: Int, numCols: Int, rng: Random): Matrix = { + new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian())) } /** diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index 5f8b8c4b72697..322a0e9242918 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -17,7 +17,11 @@ package org.apache.spark.mllib.linalg +import java.util.Random + +import org.mockito.Mockito.when import org.scalatest.FunSuite +import org.scalatest.mock.MockitoSugar._ class MatricesSuite extends FunSuite { test("dense matrix construction") { @@ -112,4 +116,50 @@ class MatricesSuite extends FunSuite { assert(sparseMat(0, 1) === 10.0) assert(sparseMat.values(2) === 10.0) } + + test("zeros") { + val mat = Matrices.zeros(2, 3).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 3) + assert(mat.values.forall(_ == 0.0)) + } + + test("ones") { + val mat = Matrices.ones(2, 3).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 3) + assert(mat.values.forall(_ == 1.0)) + } + + test("eye") { + val mat = Matrices.eye(2).asInstanceOf[DenseMatrix] + assert(mat.numCols === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 0.0, 0.0, 1.0)) + } + + test("rand") { + val rng = mock[Random] + when(rng.nextDouble()).thenReturn(1.0, 2.0, 3.0, 4.0) + val mat = Matrices.rand(2, 2, rng).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + } + + test("randn") { + val rng = mock[Random] + when(rng.nextGaussian()).thenReturn(1.0, 2.0, 3.0, 4.0) + val mat = Matrices.randn(2, 2, rng).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 2.0, 3.0, 4.0)) + } + + test("diag") { + val mat = Matrices.diag(Vectors.dense(1.0, 2.0)).asInstanceOf[DenseMatrix] + assert(mat.numRows === 2) + assert(mat.numCols === 2) + assert(mat.values.toSeq === Seq(1.0, 0.0, 0.0, 2.0)) + } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 94de14ddbd2bb..230239aa40500 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -47,6 +47,12 @@ object MimaExcludes { "org.apache.spark.SparkStageInfoImpl.this"), ProblemFilters.exclude[MissingMethodProblem]( "org.apache.spark.SparkStageInfo.submissionTime") + ) ++ Seq( + // SPARK-4614 + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Matrices.randn"), + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Matrices.rand") ) case v if v.startsWith("1.2") => From 5af53ada65f62e6b5987eada288fb48e9211ef9d Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 26 Nov 2014 16:52:04 -0800 Subject: [PATCH 043/305] [SPARK-732][SPARK-3628][CORE][RESUBMIT] eliminate duplicate update on accmulator https://issues.apache.org/jira/browse/SPARK-3628 In current implementation, the accumulator will be updated for every successfully finished task, even the task is from a resubmitted stage, which makes the accumulator counter-intuitive In this patch, I changed the way for the DAGScheduler to update the accumulator, DAGScheduler maintains a HashTable, mapping the stage id to the received pairs. Only when the stage becomes independent, (no job needs it any more), we accumulate the values of the pairs, when a task finished, we check if the HashTable has contained such stageId, it saves the accumulator_id, value only when the task is the first finished task of a new stage or the stage is running for the first attempt... Author: CodingCat Closes #2524 from CodingCat/SPARK-732-1 and squashes the following commits: 701a1e8 [CodingCat] roll back change on Accumulator.scala 1433e6f [CodingCat] make MIMA happy b233737 [CodingCat] address Matei's comments 02261b8 [CodingCat] rollback some changes 6b0aff9 [CodingCat] update document 2b2e8cf [CodingCat] updateAccumulator 83b75f8 [CodingCat] style fix 84570d2 [CodingCat] re-enable the bad accumulator guard 1e9e14d [CodingCat] add NPE guard 21b6840 [CodingCat] simplify the patch 88d1f03 [CodingCat] fix rebase error f74266b [CodingCat] add test case for resubmitted result stage 5cf586f [CodingCat] de-duplicate on task level 138f9b3 [CodingCat] make MIMA happy 67593d2 [CodingCat] make if allowing duplicate update as an option of accumulator --- .../scala/org/apache/spark/Accumulators.scala | 4 +- .../apache/spark/scheduler/DAGScheduler.scala | 53 +++++++++++-------- .../spark/scheduler/DAGSchedulerSuite.scala | 34 +++++++++--- docs/programming-guide.md | 6 +++ 4 files changed, 67 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index dc1e8f6c21b62..000bbd6b532ad 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -18,6 +18,7 @@ package org.apache.spark import java.io.{ObjectInputStream, Serializable} +import java.util.concurrent.atomic.AtomicLong import scala.collection.generic.Growable import scala.collection.mutable.Map @@ -228,6 +229,7 @@ GrowableAccumulableParam[R <% Growable[T] with TraversableOnce[T] with Serializa */ class Accumulator[T](@transient initialValue: T, param: AccumulatorParam[T], name: Option[String]) extends Accumulable[T,T](initialValue, param, name) { + def this(initialValue: T, param: AccumulatorParam[T]) = this(initialValue, param, None) } @@ -282,7 +284,7 @@ private object Accumulators { val localAccums = Map[Thread, Map[Long, Accumulable[_, _]]]() var lastId: Long = 0 - def newId: Long = synchronized { + def newId(): Long = synchronized { lastId += 1 lastId } 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 b1222af662e9b..cb8ccfbdbdcbb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -449,7 +449,6 @@ class DAGScheduler( } // data structures based on StageId stageIdToStage -= stageId - logDebug("After removal of stage %d, remaining stages = %d" .format(stageId, stageIdToStage.size)) } @@ -902,6 +901,34 @@ class DAGScheduler( } } + /** Merge updates from a task to our local accumulator values */ + private def updateAccumulators(event: CompletionEvent): Unit = { + val task = event.task + val stage = stageIdToStage(task.stageId) + if (event.accumUpdates != null) { + try { + Accumulators.add(event.accumUpdates) + event.accumUpdates.foreach { case (id, partialValue) => + val acc = Accumulators.originals(id).asInstanceOf[Accumulable[Any, Any]] + // To avoid UI cruft, ignore cases where value wasn't updated + if (acc.name.isDefined && partialValue != acc.zero) { + val name = acc.name.get + val stringPartialValue = Accumulators.stringifyPartialValue(partialValue) + val stringValue = Accumulators.stringifyValue(acc.value) + stage.latestInfo.accumulables(id) = AccumulableInfo(id, name, stringValue) + event.taskInfo.accumulables += + AccumulableInfo(id, name, Some(stringPartialValue), stringValue) + } + } + } catch { + // If we see an exception during accumulator update, just log the + // error and move on. + case e: Exception => + logError(s"Failed to update accumulators for $task", e) + } + } + } + /** * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. @@ -942,27 +969,6 @@ class DAGScheduler( } event.reason match { case Success => - if (event.accumUpdates != null) { - try { - Accumulators.add(event.accumUpdates) - event.accumUpdates.foreach { case (id, partialValue) => - val acc = Accumulators.originals(id).asInstanceOf[Accumulable[Any, Any]] - // To avoid UI cruft, ignore cases where value wasn't updated - if (acc.name.isDefined && partialValue != acc.zero) { - val name = acc.name.get - val stringPartialValue = Accumulators.stringifyPartialValue(partialValue) - val stringValue = Accumulators.stringifyValue(acc.value) - stage.latestInfo.accumulables(id) = AccumulableInfo(id, name, stringValue) - event.taskInfo.accumulables += - AccumulableInfo(id, name, Some(stringPartialValue), stringValue) - } - } - } catch { - // If we see an exception during accumulator update, just log the error and move on. - case e: Exception => - logError(s"Failed to update accumulators for $task", e) - } - } listenerBus.post(SparkListenerTaskEnd(stageId, stage.latestInfo.attemptId, taskType, event.reason, event.taskInfo, event.taskMetrics)) stage.pendingTasks -= task @@ -971,6 +977,7 @@ class DAGScheduler( stage.resultOfJob match { case Some(job) => if (!job.finished(rt.outputId)) { + updateAccumulators(event) job.finished(rt.outputId) = true job.numFinished += 1 // If the whole job has finished, remove it @@ -995,6 +1002,7 @@ class DAGScheduler( } case smt: ShuffleMapTask => + updateAccumulators(event) val status = event.result.asInstanceOf[MapStatus] val execId = status.location.executorId logDebug("ShuffleMapTask finished on " + execId) @@ -1083,7 +1091,6 @@ class DAGScheduler( } failedStages += failedStage failedStages += mapStage - // Mark the map whose fetch failed as broken in the map stage if (mapId != -1) { mapStage.removeOutputLoc(mapId, bmAddress) 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 819f95634bcdc..bdd721dc7eaf7 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -207,7 +207,18 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assert(taskSet.tasks.size >= results.size) for ((result, i) <- results.zipWithIndex) { if (i < taskSet.tasks.size) { - runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, Map[Long, Any](), null, null)) + runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, null, null, null)) + } + } + } + + private def completeWithAccumulator(accumId: Long, taskSet: TaskSet, + results: Seq[(TaskEndReason, Any)]) { + assert(taskSet.tasks.size >= results.size) + for ((result, i) <- results.zipWithIndex) { + if (i < taskSet.tasks.size) { + runEvent(CompletionEvent(taskSet.tasks(i), result._1, result._2, + Map[Long, Any]((accumId, 1)), null, null)) } } } @@ -493,17 +504,16 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F runEvent(ExecutorLost("exec-hostA")) val newEpoch = mapOutputTracker.getEpoch assert(newEpoch > oldEpoch) - val noAccum = Map[Long, Any]() val taskSet = taskSets(0) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, null, null)) // should work because it's a non-failed host - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), noAccum, null, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, null, null)) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), noAccum, null, null)) + runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, null, null)) // should work because it's a new epoch taskSet.tasks(1).epoch = newEpoch - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), noAccum, null, null)) + runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, null, null)) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) complete(taskSets(1), Seq((Success, 42), (Success, 43))) @@ -728,6 +738,18 @@ class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with F assert(scheduler.sc.dagScheduler === null) } + test("accumulator not calculated for resubmitted result stage") { + //just for register + val accum = new Accumulator[Int](0, SparkContext.IntAccumulatorParam) + val finalRdd = new MyRDD(sc, 1, Nil) + submit(finalRdd, Array(0)) + completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) + completeWithAccumulator(accum.id, taskSets(0), Seq((Success, 42))) + assert(results === Map(0 -> 42)) + assert(Accumulators.originals(accum.id).value === 1) + assertDataStructuresEmpty + } + /** * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. * Note that this checks only the host and not the executor ID. diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 49f319ba775e5..c60de6e970531 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -1306,6 +1306,12 @@ vecAccum = sc.accumulator(Vector(...), VectorAccumulatorParam()) +For accumulator updates performed inside actions only, Spark guarantees that each task's update to the accumulator +will only be applied once, i.e. restarted tasks will not update the value. In transformations, users should be aware +of that each task's update may be applied more than once if tasks or job stages are re-executed. + + + # Deploying to a Cluster The [application submission guide](submitting-applications.html) describes how to submit applications to a cluster. From c86e9bc4fdd103111280a37144a518479bb9cf0e Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 26 Nov 2014 23:16:23 -0800 Subject: [PATCH 044/305] [Release] Automate generation of contributors list This commit provides a script that computes the contributors list by linking the github commits with JIRA issues. Automatically translating github usernames remains a TODO at this point. --- dev/create-release/generate-contributors.py | 206 ++++++++++++++++++++ dev/create-release/releaseutils.py | 124 ++++++++++++ 2 files changed, 330 insertions(+) create mode 100755 dev/create-release/generate-contributors.py create mode 100755 dev/create-release/releaseutils.py diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py new file mode 100755 index 0000000000000..f4bf734081583 --- /dev/null +++ b/dev/create-release/generate-contributors.py @@ -0,0 +1,206 @@ +#!/usr/bin/env python + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# This script automates the process of creating release notes. + +import os +import re +import sys + +from releaseutils import * + +# You must set the following before use! +JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") +START_COMMIT = os.environ.get("START_COMMIT", "37b100") +END_COMMIT = os.environ.get("END_COMMIT", "3693ae") + +try: + from jira.client import JIRA +except ImportError: + print "This tool requires the jira-python library" + print "Install using 'sudo pip install jira-python'" + sys.exit(-1) + +try: + import unidecode +except ImportError: + print "This tool requires the unidecode library to decode obscure github usernames" + print "Install using 'sudo pip install unidecode'" + sys.exit(-1) + +# If commit range is not specified, prompt the user to provide it +if not START_COMMIT or not END_COMMIT: + print "A commit range is required to proceed." + if not START_COMMIT: + START_COMMIT = raw_input("Please specify starting commit hash (inclusive): ") + if not END_COMMIT: + END_COMMIT = raw_input("Please specify ending commit hash (non-inclusive): ") + +# Verify provided arguments +start_commit_line = get_one_line(START_COMMIT) +end_commit_line = get_one_line(END_COMMIT) +num_commits = num_commits_in_range(START_COMMIT, END_COMMIT) +if not start_commit_line: sys.exit("Start commit %s not found!" % START_COMMIT) +if not end_commit_line: sys.exit("End commit %s not found!" % END_COMMIT) +if num_commits == 0: + sys.exit("There are no commits in the provided range [%s, %s)" % (START_COMMIT, END_COMMIT)) +print "\n==================================================================================" +print "JIRA server: %s" % JIRA_API_BASE +print "Start commit (inclusive): %s" % start_commit_line +print "End commit (non-inclusive): %s" % end_commit_line +print "Number of commits in this range: %s" % num_commits +print +response = raw_input("Is this correct? [Y/n] ") +if response.lower() != "y" and response: + sys.exit("Ok, exiting") +print "==================================================================================\n" + +# Find all commits within this range +print "Gathering commits within range [%s..%s)" % (START_COMMIT, END_COMMIT) +commits = get_one_line_commits(START_COMMIT, END_COMMIT) +if not commits: sys.exit("Error: No commits found within this range!") +commits = commits.split("\n") + +# Filter out special commits +releases = [] +reverts = [] +nojiras = [] +filtered_commits = [] +def is_release(commit): + return re.findall("\[release\]", commit.lower()) or\ + "maven-release-plugin" in commit or "CHANGES.txt" in commit +def has_no_jira(commit): + return not re.findall("SPARK-[0-9]+", commit.upper()) +def is_revert(commit): + return "revert" in commit.lower() +def is_docs(commit): + return re.findall("docs*", commit.lower()) or "programming guide" in commit.lower() +for c in commits: + if not c: continue + elif is_release(c): releases.append(c) + elif is_revert(c): reverts.append(c) + elif is_docs(c): filtered_commits.append(c) # docs may not have JIRA numbers + elif has_no_jira(c): nojiras.append(c) + else: filtered_commits.append(c) + +# Warn against ignored commits +def print_indented(_list): + for x in _list: print " %s" % x +if releases or reverts or nojiras: + print "\n==================================================================================" + if releases: print "Releases (%d)" % len(releases); print_indented(releases) + if reverts: print "Reverts (%d)" % len(reverts); print_indented(reverts) + if nojiras: print "No JIRA (%d)" % len(nojiras); print_indented(nojiras) + print "==================== Warning: the above commits will be ignored ==================\n" +response = raw_input("%d commits left to process. Ok to proceed? [y/N] " % len(filtered_commits)) +if response.lower() != "y": + sys.exit("Ok, exiting.") + +# Keep track of warnings to tell the user at the end +warnings = [] + +# Populate a map that groups issues and components by author +# It takes the form: Author name -> { Contribution type -> Spark components } +# For instance, +# { +# 'Andrew Or': { +# 'bug fixes': ['windows', 'core', 'web ui'], +# 'improvements': ['core'] +# }, +# 'Tathagata Das' : { +# 'bug fixes': ['streaming'] +# 'new feature': ['streaming'] +# } +# } +# +author_info = {} +jira_options = { "server": JIRA_API_BASE } +jira = JIRA(jira_options) +print "\n=========================== Compiling contributor list ===========================" +for commit in filtered_commits: + commit_hash = re.findall("^[a-z0-9]+", commit)[0] + issues = re.findall("SPARK-[0-9]+", commit.upper()) + author = get_author(commit_hash) + author = unidecode.unidecode(unicode(author, "UTF-8")) # guard against special characters + date = get_date(commit_hash) + # Parse components from the commit message, if any + commit_components = find_components(commit, commit_hash) + # Populate or merge an issue into author_info[author] + def populate(issue_type, components): + components = components or [CORE_COMPONENT] # assume core if no components provided + if author not in author_info: + author_info[author] = {} + if issue_type not in author_info[author]: + author_info[author][issue_type] = set() + for component in all_components: + author_info[author][issue_type].add(component) + # Find issues and components associated with this commit + for issue in issues: + jira_issue = jira.issue(issue) + jira_type = jira_issue.fields.issuetype.name + jira_type = translate_issue_type(jira_type, issue, warnings) + jira_components = [translate_component(c.name, commit_hash, warnings)\ + for c in jira_issue.fields.components] + all_components = set(jira_components + commit_components) + populate(jira_type, all_components) + # For docs without an associated JIRA, manually add it ourselves + if is_docs(commit) and not issues: + populate("documentation", commit_components) + print " Processed commit %s authored by %s on %s" % (commit_hash, author, date) +print "==================================================================================\n" + +# Write to contributors file ordered by author names +# Each line takes the format "Author name - semi-colon delimited contributions" +# e.g. Andrew Or - Bug fixes in Windows, Core, and Web UI; improvements in Core +# e.g. Tathagata Das - Bug fixes and new features in Streaming +contributors_file_name = "contributors.txt" +contributors_file = open(contributors_file_name, "w") +authors = author_info.keys() +authors.sort() +for author in authors: + contribution = "" + components = set() + issue_types = set() + for issue_type, comps in author_info[author].items(): + components.update(comps) + issue_types.add(issue_type) + # If there is only one component, mention it only once + # e.g. Bug fixes, improvements in MLlib + if len(components) == 1: + contribution = "%s in %s" % (nice_join(issue_types), next(iter(components))) + # Otherwise, group contributions by issue types instead of modules + # e.g. Bug fixes in MLlib, Core, and Streaming; documentation in YARN + else: + contributions = ["%s in %s" % (issue_type, nice_join(comps)) \ + for issue_type, comps in author_info[author].items()] + contribution = "; ".join(contributions) + # Do not use python's capitalize() on the whole string to preserve case + assert contribution + contribution = contribution[0].capitalize() + contribution[1:] + line = "%s - %s" % (author, contribution) + contributors_file.write(line + "\n") +contributors_file.close() +print "Contributors list is successfully written to %s!" % contributors_file_name + +# Log any warnings encountered in the process +if warnings: + print "\n============ Warnings encountered while creating the contributor list ============" + for w in warnings: print w + print "Please correct these in the final contributors list at %s." % contributors_file_name + print "==================================================================================\n" + diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py new file mode 100755 index 0000000000000..e56d7fa58fa2c --- /dev/null +++ b/dev/create-release/releaseutils.py @@ -0,0 +1,124 @@ +#!/usr/bin/env python + +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +# This file contains helper methods used in creating a release. + +import re +from subprocess import Popen, PIPE + +# Utility functions run git commands (written with Git 1.8.5) +def run_cmd(cmd): return Popen(cmd, stdout=PIPE).communicate()[0] +def get_author(commit_hash): + return run_cmd(["git", "show", "--quiet", "--pretty=format:%an", commit_hash]) +def get_date(commit_hash): + return run_cmd(["git", "show", "--quiet", "--pretty=format:%cd", commit_hash]) +def get_one_line(commit_hash): + return run_cmd(["git", "show", "--quiet", "--pretty=format:\"%h %cd %s\"", commit_hash]) +def get_one_line_commits(start_hash, end_hash): + return run_cmd(["git", "log", "--oneline", "%s..%s" % (start_hash, end_hash)]) +def num_commits_in_range(start_hash, end_hash): + output = run_cmd(["git", "log", "--oneline", "%s..%s" % (start_hash, end_hash)]) + lines = [line for line in output.split("\n") if line] # filter out empty lines + return len(lines) + +# Maintain a mapping for translating issue types to contributions in the release notes +# This serves an additional function of warning the user against unknown issue types +# Note: This list is partially derived from this link: +# https://issues.apache.org/jira/plugins/servlet/project-config/SPARK/issuetypes +# Keep these in lower case +known_issue_types = { + "bug": "bug fixes", + "build": "build fixes", + "improvement": "improvements", + "new feature": "new features", + "documentation": "documentation" +} + +# Maintain a mapping for translating component names when creating the release notes +# This serves an additional function of warning the user against unknown components +# Note: This list is largely derived from this link: +# https://issues.apache.org/jira/plugins/servlet/project-config/SPARK/components +CORE_COMPONENT = "Core" +known_components = { + "block manager": CORE_COMPONENT, + "build": CORE_COMPONENT, + "deploy": CORE_COMPONENT, + "documentation": CORE_COMPONENT, + "ec2": "EC2", + "examples": CORE_COMPONENT, + "graphx": "GraphX", + "input/output": CORE_COMPONENT, + "java api": "Java API", + "mesos": "Mesos", + "ml": "MLlib", + "mllib": "MLlib", + "project infra": "Project Infra", + "pyspark": "PySpark", + "shuffle": "Shuffle", + "spark core": CORE_COMPONENT, + "spark shell": CORE_COMPONENT, + "sql": "SQL", + "streaming": "Streaming", + "web ui": "Web UI", + "windows": "Windows", + "yarn": "YARN" +} + +# Translate issue types using a format appropriate for writing contributions +# If an unknown issue type is encountered, warn the user +def translate_issue_type(issue_type, issue_id, warnings): + issue_type = issue_type.lower() + if issue_type in known_issue_types: + return known_issue_types[issue_type] + else: + warnings.append("Unknown issue type \"%s\" (see %s)" % (issue_type, issue_id)) + return issue_type + +# Translate component names using a format appropriate for writing contributions +# If an unknown component is encountered, warn the user +def translate_component(component, commit_hash, warnings): + component = component.lower() + if component in known_components: + return known_components[component] + else: + warnings.append("Unknown component \"%s\" (see %s)" % (component, commit_hash)) + return component + +# Parse components in the commit message +# The returned components are already filtered and translated +def find_components(commit, commit_hash): + components = re.findall("\[\w*\]", commit.lower()) + components = [translate_component(c, commit_hash)\ + for c in components if c in known_components] + return components + +# Join a list of strings in a human-readable manner +# e.g. ["Juice"] -> "Juice" +# e.g. ["Juice", "baby"] -> "Juice and baby" +# e.g. ["Juice", "baby", "moon"] -> "Juice, baby, and moon" +def nice_join(str_list): + str_list = list(str_list) # sometimes it's a set + if not str_list: + return "" + elif len(str_list) == 1: + return next(iter(str_list)) + elif len(str_list) == 2: + return " and ".join(str_list) + else: + return ", ".join(str_list[:-1]) + ", and " + str_list[-1] + From 5d7fe178b303918faa0893cd36963158b420309f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 27 Nov 2014 09:03:17 -0800 Subject: [PATCH 045/305] SPARK-4170 [CORE] Closure problems when running Scala app that "extends App" Warn against subclassing scala.App, and remove one instance of this in examples Author: Sean Owen Closes #3497 from srowen/SPARK-4170 and squashes the following commits: 4a6131f [Sean Owen] Restore multiline string formatting a8ca895 [Sean Owen] Warn against subclassing scala.App, and remove one instance of this in examples --- .../org/apache/spark/deploy/SparkSubmit.scala | 5 ++ docs/quick-start.md | 3 + .../examples/mllib/LinearRegression.scala | 70 ++++++++++--------- 3 files changed, 44 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 8a62519bd2315..00f291823e984 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -345,6 +345,11 @@ object SparkSubmit { System.exit(CLASS_NOT_FOUND_EXIT_STATUS) } + // SPARK-4170 + if (classOf[scala.App].isAssignableFrom(mainClass)) { + printWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.") + } + val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) if (!Modifier.isStatic(mainMethod.getModifiers)) { throw new IllegalStateException("The main method in the given main class must be static") diff --git a/docs/quick-start.md b/docs/quick-start.md index 6236de0e1f2c4..bf643bb70e153 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -244,6 +244,9 @@ object SimpleApp { } {% endhighlight %} +Note that applications should define a `main()` method instead of extending `scala.App`. +Subclasses of `scala.App` may not work correctly. + This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala index 6815b1c052208..6a456ba7ec07b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -33,7 +33,7 @@ import org.apache.spark.mllib.optimization.{SimpleUpdater, SquaredL2Updater, L1U * A synthetic dataset can be found at `data/mllib/sample_linear_regression_data.txt`. * If you use it as a template to create your own app, please use `spark-submit` to submit your app. */ -object LinearRegression extends App { +object LinearRegression { object RegType extends Enumeration { type RegType = Value @@ -49,40 +49,42 @@ object LinearRegression extends App { regType: RegType = L2, regParam: Double = 0.01) extends AbstractParams[Params] - val defaultParams = Params() - - val parser = new OptionParser[Params]("LinearRegression") { - head("LinearRegression: an example app for linear regression.") - opt[Int]("numIterations") - .text("number of iterations") - .action((x, c) => c.copy(numIterations = x)) - opt[Double]("stepSize") - .text(s"initial step size, default: ${defaultParams.stepSize}") - .action((x, c) => c.copy(stepSize = x)) - opt[String]("regType") - .text(s"regularization type (${RegType.values.mkString(",")}), " + - s"default: ${defaultParams.regType}") - .action((x, c) => c.copy(regType = RegType.withName(x))) - opt[Double]("regParam") - .text(s"regularization parameter, default: ${defaultParams.regParam}") - arg[String]("") - .required() - .text("input paths to labeled examples in LIBSVM format") - .action((x, c) => c.copy(input = x)) - note( - """ - |For example, the following command runs this app on a synthetic dataset: - | - | bin/spark-submit --class org.apache.spark.examples.mllib.LinearRegression \ - | examples/target/scala-*/spark-examples-*.jar \ - | data/mllib/sample_linear_regression_data.txt - """.stripMargin) - } + def main(args: Array[String]) { + val defaultParams = Params() + + val parser = new OptionParser[Params]("LinearRegression") { + head("LinearRegression: an example app for linear regression.") + opt[Int]("numIterations") + .text("number of iterations") + .action((x, c) => c.copy(numIterations = x)) + opt[Double]("stepSize") + .text(s"initial step size, default: ${defaultParams.stepSize}") + .action((x, c) => c.copy(stepSize = x)) + opt[String]("regType") + .text(s"regularization type (${RegType.values.mkString(",")}), " + + s"default: ${defaultParams.regType}") + .action((x, c) => c.copy(regType = RegType.withName(x))) + opt[Double]("regParam") + .text(s"regularization parameter, default: ${defaultParams.regParam}") + arg[String]("") + .required() + .text("input paths to labeled examples in LIBSVM format") + .action((x, c) => c.copy(input = x)) + note( + """ + |For example, the following command runs this app on a synthetic dataset: + | + | bin/spark-submit --class org.apache.spark.examples.mllib.LinearRegression \ + | examples/target/scala-*/spark-examples-*.jar \ + | data/mllib/sample_linear_regression_data.txt + """.stripMargin) + } - parser.parse(args, defaultParams).map { params => - run(params) - } getOrElse { - sys.exit(1) + parser.parse(args, defaultParams).map { params => + run(params) + } getOrElse { + sys.exit(1) + } } def run(params: Params) { From 84376d31392858f7df215ddb3f05419181152e68 Mon Sep 17 00:00:00 2001 From: roxchkplusony Date: Thu, 27 Nov 2014 15:54:40 -0800 Subject: [PATCH 046/305] [SPARK-4626] Kill a task only if the executorId is (still) registered with the scheduler Author: roxchkplusony Closes #3483 from roxchkplusony/bugfix/4626 and squashes the following commits: aba9184 [roxchkplusony] replace warning message per review 5e7fdea [roxchkplusony] [SPARK-4626] Kill a task only if the executorId is (still) registered with the scheduler --- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 047fae104b485..88b196ac64368 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -127,7 +127,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste makeOffers() case KillTask(taskId, executorId, interruptThread) => - executorDataMap(executorId).executorActor ! KillTask(taskId, executorId, interruptThread) + executorDataMap.get(executorId) match { + case Some(executorInfo) => + executorInfo.executorActor ! KillTask(taskId, executorId, interruptThread) + case None => + // Ignoring the task kill since the executor is not registered. + logWarning(s"Attempted to kill task $taskId for unknown executor $executorId.") + } case StopDriver => sender ! true From 120a350240f58196eafcb038ca3a353636d89239 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 27 Nov 2014 18:01:14 -0800 Subject: [PATCH 047/305] [SPARK-4613][Core] Java API for JdbcRDD This PR introduces a set of Java APIs for using `JdbcRDD`: 1. Trait (interface) `JdbcRDD.ConnectionFactory`: equivalent to the `getConnection: () => Connection` parameter in `JdbcRDD` constructor. 2. Two overloaded versions of `Jdbc.create`: used to create `JavaRDD` that wraps a `JdbcRDD`. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3478) Author: Cheng Lian Closes #3478 from liancheng/japi-jdbc-rdd and squashes the following commits: 9a54625 [Cheng Lian] Only shutdowns a single DB rather than the whole Derby driver d4cedc5 [Cheng Lian] Moves Java JdbcRDD test case to a separate test suite ffcdf2e [Cheng Lian] Java API for JdbcRDD --- .../scala/org/apache/spark/rdd/JdbcRDD.scala | 84 ++++++++++++- .../org/apache/spark/JavaJdbcRDDSuite.java | 118 ++++++++++++++++++ .../org/apache/spark/rdd/JdbcRDDSuite.scala | 7 +- 3 files changed, 204 insertions(+), 5 deletions(-) create mode 100644 core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 0e38f224ac81d..642a12c1edf6c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -21,8 +21,11 @@ import java.sql.{Connection, ResultSet} import scala.reflect.ClassTag -import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} +import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.api.java.function.{Function => JFunction} +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.util.NextIterator +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext} private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { override def index = idx @@ -125,5 +128,82 @@ object JdbcRDD { def resultSetToObjectArray(rs: ResultSet): Array[Object] = { Array.tabulate[Object](rs.getMetaData.getColumnCount)(i => rs.getObject(i + 1)) } -} + trait ConnectionFactory extends Serializable { + @throws[Exception] + def getConnection: Connection + } + + /** + * Create an RDD that executes an SQL query on a JDBC connection and reads results. + * For usage example, see test case JavaAPISuite.testJavaJdbcRDD. + * + * @param connectionFactory a factory that returns an open Connection. + * The RDD takes care of closing the connection. + * @param sql the text of the query. + * The query must contain two ? placeholders for parameters used to partition the results. + * E.g. "select title, author from books where ? <= id and id <= ?" + * @param lowerBound the minimum value of the first placeholder + * @param upperBound the maximum value of the second placeholder + * The lower and upper bounds are inclusive. + * @param numPartitions the number of partitions. + * Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, + * the query would be executed twice, once with (1, 10) and once with (11, 20) + * @param mapRow a function from a ResultSet to a single row of the desired result type(s). + * This should only call getInt, getString, etc; the RDD takes care of calling next. + * The default maps a ResultSet to an array of Object. + */ + def create[T]( + sc: JavaSparkContext, + connectionFactory: ConnectionFactory, + sql: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int, + mapRow: JFunction[ResultSet, T]): JavaRDD[T] = { + + val jdbcRDD = new JdbcRDD[T]( + sc.sc, + () => connectionFactory.getConnection, + sql, + lowerBound, + upperBound, + numPartitions, + (resultSet: ResultSet) => mapRow.call(resultSet))(fakeClassTag) + + new JavaRDD[T](jdbcRDD)(fakeClassTag) + } + + /** + * Create an RDD that executes an SQL query on a JDBC connection and reads results. Each row is + * converted into a `Object` array. For usage example, see test case JavaAPISuite.testJavaJdbcRDD. + * + * @param connectionFactory a factory that returns an open Connection. + * The RDD takes care of closing the connection. + * @param sql the text of the query. + * The query must contain two ? placeholders for parameters used to partition the results. + * E.g. "select title, author from books where ? <= id and id <= ?" + * @param lowerBound the minimum value of the first placeholder + * @param upperBound the maximum value of the second placeholder + * The lower and upper bounds are inclusive. + * @param numPartitions the number of partitions. + * Given a lowerBound of 1, an upperBound of 20, and a numPartitions of 2, + * the query would be executed twice, once with (1, 10) and once with (11, 20) + */ + def create( + sc: JavaSparkContext, + connectionFactory: ConnectionFactory, + sql: String, + lowerBound: Long, + upperBound: Long, + numPartitions: Int): JavaRDD[Array[Object]] = { + + val mapRow = new JFunction[ResultSet, Array[Object]] { + override def call(resultSet: ResultSet): Array[Object] = { + resultSetToObjectArray(resultSet) + } + } + + create(sc, connectionFactory, sql, lowerBound, upperBound, numPartitions, mapRow) + } +} diff --git a/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java b/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java new file mode 100644 index 0000000000000..7fe452a48d89b --- /dev/null +++ b/core/src/test/java/org/apache/spark/JavaJdbcRDDSuite.java @@ -0,0 +1,118 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark; + +import java.io.Serializable; +import java.sql.Connection; +import java.sql.DriverManager; +import java.sql.PreparedStatement; +import java.sql.ResultSet; +import java.sql.SQLException; +import java.sql.Statement; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.rdd.JdbcRDD; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class JavaJdbcRDDSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() throws ClassNotFoundException, SQLException { + sc = new JavaSparkContext("local", "JavaAPISuite"); + + Class.forName("org.apache.derby.jdbc.EmbeddedDriver"); + Connection connection = + DriverManager.getConnection("jdbc:derby:target/JavaJdbcRDDSuiteDb;create=true"); + + try { + Statement create = connection.createStatement(); + create.execute( + "CREATE TABLE FOO(" + + "ID INTEGER NOT NULL GENERATED ALWAYS AS IDENTITY (START WITH 1, INCREMENT BY 1)," + + "DATA INTEGER)"); + create.close(); + + PreparedStatement insert = connection.prepareStatement("INSERT INTO FOO(DATA) VALUES(?)"); + for (int i = 1; i <= 100; i++) { + insert.setInt(1, i * 2); + insert.executeUpdate(); + } + insert.close(); + } catch (SQLException e) { + // If table doesn't exist... + if (e.getSQLState().compareTo("X0Y32") != 0) { + throw e; + } + } finally { + connection.close(); + } + } + + @After + public void tearDown() throws SQLException { + try { + DriverManager.getConnection("jdbc:derby:target/JavaJdbcRDDSuiteDb;shutdown=true"); + } catch(SQLException e) { + // Throw if not normal single database shutdown + // https://db.apache.org/derby/docs/10.2/ref/rrefexcept71493.html + if (e.getSQLState().compareTo("08006") != 0) { + throw e; + } + } + + sc.stop(); + sc = null; + } + + @Test + public void testJavaJdbcRDD() throws Exception { + JavaRDD rdd = JdbcRDD.create( + sc, + new JdbcRDD.ConnectionFactory() { + @Override + public Connection getConnection() throws SQLException { + return DriverManager.getConnection("jdbc:derby:target/JavaJdbcRDDSuiteDb"); + } + }, + "SELECT DATA FROM FOO WHERE ? <= ID AND ID <= ?", + 1, 100, 1, + new Function() { + @Override + public Integer call(ResultSet r) throws Exception { + return r.getInt(1); + } + } + ).cache(); + + Assert.assertEquals(100, rdd.count()); + Assert.assertEquals( + Integer.valueOf(10100), + rdd.reduce(new Function2() { + @Override + public Integer call(Integer i1, Integer i2) { + return i1 + i2; + } + })); + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala index 76e317d754ba3..6138d0bbd57f6 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -65,10 +65,11 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { after { try { - DriverManager.getConnection("jdbc:derby:;shutdown=true") + DriverManager.getConnection("jdbc:derby:target/JdbcRDDSuiteDb;shutdown=true") } catch { - case se: SQLException if se.getSQLState == "XJ015" => - // normal shutdown + case se: SQLException if se.getSQLState == "08006" => + // Normal single database shutdown + // https://db.apache.org/derby/docs/10.2/ref/rrefexcept71493.html } } } From ceb628197099e6c598cde1564ed9c1c3681ea955 Mon Sep 17 00:00:00 2001 From: maji2014 Date: Fri, 28 Nov 2014 00:36:22 -0800 Subject: [PATCH 048/305] [SPARK-4619][Storage]delete redundant time suffix Time suffix exists in Utils.getUsedTimeMs(startTime), no need to append again, delete that Author: maji2014 Closes #3475 from maji2014/SPARK-4619 and squashes the following commits: df0da4e [maji2014] delete redundant time suffix --- .../org/apache/spark/storage/ShuffleBlockFetcherIterator.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 6b1f57a069431..83170f7c5a4ab 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -265,7 +265,7 @@ final class ShuffleBlockFetcherIterator( // Get Local Blocks fetchLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") + logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime)) } override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch From 5b99bf243e2956fe933ab2dccd069266e82cad8d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 28 Nov 2014 11:42:40 -0500 Subject: [PATCH 049/305] [SPARK-4645][SQL] Disables asynchronous execution in Hive 0.13.1 HiveThriftServer2 This PR disables HiveThriftServer2 asynchronous execution by setting `runInBackground` argument in `ExecuteStatementOperation` to `false`, and reverting `SparkExecuteStatementOperation.run` in Hive 13 shim to Hive 12 version. This change makes Simba ODBC driver v1.0.0.1000 work. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3506) Author: Cheng Lian Closes #3506 from liancheng/disable-async-exec and squashes the following commits: 593804d [Cheng Lian] Disables asynchronous execution in Hive 0.13.1 HiveThriftServer2 --- .../spark/sql/hive/thriftserver/Shim13.scala | 139 +++++------------- 1 file changed, 39 insertions(+), 100 deletions(-) diff --git a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala index 99c1987158581..17f1ad3e4690e 100644 --- a/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala +++ b/sql/hive-thriftserver/v0.13.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/Shim13.scala @@ -17,30 +17,25 @@ package org.apache.spark.sql.hive.thriftserver -import java.security.PrivilegedExceptionAction import java.sql.{Date, Timestamp} -import java.util.concurrent.Future import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, Map => SMap} import scala.math._ -import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.FieldSchema -import org.apache.hadoop.hive.ql.metadata.Hive -import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.shims.ShimLoader import org.apache.hadoop.security.UserGroupInformation import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.ExecuteStatementOperation import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.plans.logical.SetCommand import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} -import org.apache.spark.sql.{SchemaRDD, Row => SparkRow} +import org.apache.spark.sql.{Row => SparkRow, SQLConf, SchemaRDD} /** * A compatibility layer for interacting with Hive version 0.13.1. @@ -48,7 +43,9 @@ import org.apache.spark.sql.{SchemaRDD, Row => SparkRow} private[thriftserver] object HiveThriftServerShim { val version = "0.13.1" - def setServerUserName(sparkServiceUGI: UserGroupInformation, sparkCliService:SparkSQLCLIService) = { + def setServerUserName( + sparkServiceUGI: UserGroupInformation, + sparkCliService:SparkSQLCLIService) = { setSuperField(sparkCliService, "serviceUGI", sparkServiceUGI) } } @@ -72,39 +69,14 @@ private[hive] class SparkExecuteStatementOperation( confOverlay: JMap[String, String], runInBackground: Boolean = true)( hiveContext: HiveContext, - sessionToActivePool: SMap[HiveSession, String]) extends ExecuteStatementOperation( - parentSession, statement, confOverlay, runInBackground) with Logging { + sessionToActivePool: SMap[HiveSession, String]) + // NOTE: `runInBackground` is set to `false` intentionally to disable asynchronous execution + extends ExecuteStatementOperation(parentSession, statement, confOverlay, false) with Logging { private var result: SchemaRDD = _ private var iter: Iterator[SparkRow] = _ private var dataTypes: Array[DataType] = _ - private def runInternal(cmd: String) = { - try { - result = hiveContext.sql(cmd) - logDebug(result.queryExecution.toString()) - val groupId = round(random * 1000000).toString - hiveContext.sparkContext.setJobGroup(groupId, statement) - iter = { - val useIncrementalCollect = - hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean - if (useIncrementalCollect) { - result.toLocalIterator - } else { - result.collect().iterator - } - } - dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray - } catch { - // Actually do need to catch Throwable as some failures don't inherit from Exception and - // HiveServer will silently swallow them. - case e: Throwable => - setState(OperationState.ERROR) - logError("Error executing query:",e) - throw new HiveSQLException(e.toString) - } - } - def close(): Unit = { // RDDs will be cleaned automatically upon garbage collection. logDebug("CLOSING") @@ -182,76 +154,43 @@ private[hive] class SparkExecuteStatementOperation( } } - private def getConfigForOperation: HiveConf = { - var sqlOperationConf: HiveConf = getParentSession.getHiveConf - if (!getConfOverlay.isEmpty || shouldRunAsync) { - sqlOperationConf = new HiveConf(sqlOperationConf) - import scala.collection.JavaConversions._ - for (confEntry <- getConfOverlay.entrySet) { - try { - sqlOperationConf.verifyAndSet(confEntry.getKey, confEntry.getValue) - } - catch { case e: IllegalArgumentException => - throw new HiveSQLException("Error applying statement specific settings", e) - } - } - } - sqlOperationConf - } - def run(): Unit = { logInfo(s"Running query '$statement'") - val opConfig: HiveConf = getConfigForOperation setState(OperationState.RUNNING) - setHasResultSet(true) - - if (!shouldRunAsync) { - runInternal(statement) - setState(OperationState.FINISHED) - } else { - val parentSessionState = SessionState.get - val sessionHive: Hive = Hive.get - val currentUGI: UserGroupInformation = ShimLoader.getHadoopShims.getUGIForConf(opConfig) - - val backgroundOperation: Runnable = new Runnable { - def run() { - val doAsAction: PrivilegedExceptionAction[AnyRef] = - new PrivilegedExceptionAction[AnyRef] { - def run: AnyRef = { - Hive.set(sessionHive) - SessionState.setCurrentSessionState(parentSessionState) - try { - runInternal(statement) - } - catch { case e: HiveSQLException => - setOperationException(e) - logError("Error running hive query: ", e) - } - null - } - } - try { - ShimLoader.getHadoopShims.doAs(currentUGI, doAsAction) - } - catch { case e: Exception => - setOperationException(new HiveSQLException(e)) - logError("Error running hive query as user : " + currentUGI.getShortUserName, e) - } - setState(OperationState.FINISHED) - } + try { + result = hiveContext.sql(statement) + logDebug(result.queryExecution.toString()) + result.queryExecution.logical match { + case SetCommand(Some((SQLConf.THRIFTSERVER_POOL, Some(value)))) => + sessionToActivePool(parentSession) = value + logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") + case _ => } - try { - val backgroundHandle: Future[_] = getParentSession.getSessionManager. - submitBackgroundOperation(backgroundOperation) - setBackgroundHandle(backgroundHandle) - } catch { - // Actually do need to catch Throwable as some failures don't inherit from Exception and - // HiveServer will silently swallow them. - case e: Throwable => - logError("Error executing query:",e) - throw new HiveSQLException(e.toString) + val groupId = round(random * 1000000).toString + hiveContext.sparkContext.setJobGroup(groupId, statement) + sessionToActivePool.get(parentSession).foreach { pool => + hiveContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) + } + iter = { + val useIncrementalCollect = + hiveContext.getConf("spark.sql.thriftServer.incrementalCollect", "false").toBoolean + if (useIncrementalCollect) { + result.toLocalIterator + } else { + result.collect().iterator + } } + dataTypes = result.queryExecution.analyzed.output.map(_.dataType).toArray + setHasResultSet(true) + } catch { + // Actually do need to catch Throwable as some failures don't inherit from Exception and + // HiveServer will silently swallow them. + case e: Throwable => + setState(OperationState.ERROR) + logError("Error executing query:", e) + throw new HiveSQLException(e.toString) } + setState(OperationState.FINISHED) } } From 052e65815f23ee92a2093a8059d7fd192b1526e5 Mon Sep 17 00:00:00 2001 From: KaiXinXiaoLei Date: Fri, 28 Nov 2014 12:34:07 -0500 Subject: [PATCH 050/305] Delete unnecessary function MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit when building spark by sbt, the function “runAlternateBoot" in sbt/sbt-launch-lib.bash is not used. And this function is not used by spark code. So I think this function is not necessary. And the option of "sbt.boot.properties" can be configured in the command line when building spark, eg: sbt/sbt assembly -Dsbt.boot.properties=$bootpropsfile. The file from https://github.com/sbt/sbt-launcher-package is changed. And the function “runAlternateBoot" is deleted in upstream project. I think spark project should delete this function in file sbt/sbt-launch-lib.bash. Thanks. Author: KaiXinXiaoLei Closes #3224 from KaiXinXiaoLei/deleteFunction and squashes the following commits: e8eac49 [KaiXinXiaoLei] Delete blank lines. efe36d4 [KaiXinXiaoLei] Delete unnecessary function --- sbt/sbt-launch-lib.bash | 7 ------- 1 file changed, 7 deletions(-) diff --git a/sbt/sbt-launch-lib.bash b/sbt/sbt-launch-lib.bash index 055e206662654..84a6f7a207186 100755 --- a/sbt/sbt-launch-lib.bash +++ b/sbt/sbt-launch-lib.bash @@ -186,10 +186,3 @@ run() { "${sbt_commands[@]}" \ "${residual_args[@]}" } - -runAlternateBoot() { - local bootpropsfile="$1" - shift - addJava "-Dsbt.boot.properties=$bootpropsfile" - run $@ -} From 53ed7f1c7f8534d0d7856d83e3b46e36d09659d2 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Fri, 28 Nov 2014 12:41:38 -0500 Subject: [PATCH 051/305] [SPARK-4643] [Build] Remove unneeded staging repositories from build The old location will return a 404. Author: Daoyuan Wang Closes #3504 from adrian-wang/repo and squashes the following commits: f604e05 [Daoyuan Wang] already in maven central, remove at all f494fac [Daoyuan Wang] spark staging repo outdated --- pom.xml | 24 ------------------------ 1 file changed, 24 deletions(-) diff --git a/pom.xml b/pom.xml index 6c1c1214a7d3e..4220c1ace0570 100644 --- a/pom.xml +++ b/pom.xml @@ -230,30 +230,6 @@ false - - - spark-staging - Spring Staging Repository - https://oss.sonatype.org/content/repositories/orgspark-project-1085 - - true - - - false - - - - - spark-staging-hive13 - Spring Staging Repository Hive 13 - https://oss.sonatype.org/content/repositories/orgspark-project-1089/ - - true - - - false - - spark-staging-1038 From e464f0ac2d7210a4bf715478885fe7a8d397fe89 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Fri, 28 Nov 2014 13:00:15 -0500 Subject: [PATCH 052/305] [SPARK-4193][BUILD] Disable doclint in Java 8 to prevent from build error. Author: Takuya UESHIN Closes #3058 from ueshin/issues/SPARK-4193 and squashes the following commits: e096bb1 [Takuya UESHIN] Add a plugin declaration to pluginManagement. 6762ec2 [Takuya UESHIN] Fix usage of -Xdoclint javadoc option. fdb280a [Takuya UESHIN] Fix Javadoc errors. 4745f3c [Takuya UESHIN] Merge branch 'master' into issues/SPARK-4193 923e2f0 [Takuya UESHIN] Use doclint option `-missing` instead of `none`. 30d6718 [Takuya UESHIN] Fix Javadoc errors. b548017 [Takuya UESHIN] Disable doclint in Java 8 to prevent from build error. --- .../spark/network/client/TransportClient.java | 2 +- .../server/OneForOneStreamManager.java | 2 +- .../network/util/LimitedInputStream.java | 2 +- .../apache/spark/network/util/NettyUtils.java | 2 +- .../spark/network/util/TransportConf.java | 2 +- pom.xml | 24 +++++++++++++++++++ project/SparkBuild.scala | 7 +++++- 7 files changed, 35 insertions(+), 6 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java index 4e944114e8176..37f2e34ceb24d 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -49,7 +49,7 @@ * to perform this setup. * * For example, a typical workflow might be: - * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100 + * client.sendRPC(new OpenFile("/foo")) --> returns StreamId = 100 * client.fetchChunk(streamId = 100, chunkIndex = 0, callback) * client.fetchChunk(streamId = 100, chunkIndex = 1, callback) * ... diff --git a/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java index 731d48d4d9c6c..a6d390e13f396 100644 --- a/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java +++ b/network/common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java @@ -29,7 +29,7 @@ import org.apache.spark.network.buffer.ManagedBuffer; /** - * StreamManager which allows registration of an Iterator, which are individually + * StreamManager which allows registration of an Iterator<ManagedBuffer>, which are individually * fetched as chunks by the client. Each registered buffer is one chunk. */ public class OneForOneStreamManager extends StreamManager { diff --git a/network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java b/network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java index 63ca43c046525..57113ed12d414 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java +++ b/network/common/src/main/java/org/apache/spark/network/util/LimitedInputStream.java @@ -27,7 +27,7 @@ * Wraps a {@link InputStream}, limiting the number of bytes which can be read. * * This code is from Guava's 14.0 source code, because there is no compatible way to - * use this functionality in both a Guava 11 environment and a Guava >14 environment. + * use this functionality in both a Guava 11 environment and a Guava >14 environment. */ public final class LimitedInputStream extends FilterInputStream { private long left; diff --git a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java index b3991a6577cfe..2a4b88b64cdc9 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -99,7 +99,7 @@ public static ByteToMessageDecoder createFrameDecoder() { return new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 8, -8, 8); } - /** Returns the remote address on the channel or "" if none exists. */ + /** Returns the remote address on the channel or "<remote address>" if none exists. */ public static String getRemoteAddress(Channel channel) { if (channel != null && channel.remoteAddress() != null) { return channel.remoteAddress().toString(); diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 621427d8cba5e..1af40acf8b4af 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -72,7 +72,7 @@ public int connectionTimeoutMs() { /** * Time (in milliseconds) that we will wait in order to perform a retry after an IOException. - * Only relevant if maxIORetries > 0. + * Only relevant if maxIORetries > 0. */ public int ioRetryWaitTime() { return conf.getInt("spark.shuffle.io.retryWaitMs", 5000); } diff --git a/pom.xml b/pom.xml index 4220c1ace0570..b7df53d3e5eb1 100644 --- a/pom.xml +++ b/pom.xml @@ -1025,6 +1025,11 @@ + + org.apache.maven.plugins + maven-javadoc-plugin + 2.10.1 + @@ -1218,6 +1223,25 @@ + + doclint-java8-disable + + [1.8,) + + + + + + org.apache.maven.plugins + maven-javadoc-plugin + + -Xdoclint:all -Xdoclint:-missing + + + + + + [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3498) Author: Cheng Lian Closes #3498 from liancheng/fix-sql-doc-typo and squashes the following commits: 865ecd7 [Cheng Lian] Fixes formatting typo in SQL programming guide --- docs/sql-programming-guide.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 5500da83b2b66..24a68bb083334 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -900,7 +900,6 @@ export HIVE_SERVER2_THRIFT_BIND_HOST= ./sbin/start-thriftserver.sh \ --master \ ... -``` {% endhighlight %} or system properties: @@ -911,7 +910,6 @@ or system properties: --hiveconf hive.server2.thrift.bind.host= \ --master ... -``` {% endhighlight %} Now you can use beeline to test the Thrift JDBC/ODBC server: From 06dc1b15e425d4dbb0e516e5788b1a4bb39a2a60 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 30 Nov 2014 20:51:13 -0800 Subject: [PATCH 067/305] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #2915 (close requested by 'JoshRosen') Closes #3140 (close requested by 'JoshRosen') Closes #3366 (close requested by 'JoshRosen') From 5e7a6dcb8faded33eb0feb302ae3fa3ed4f900fd Mon Sep 17 00:00:00 2001 From: Prabeesh K Date: Sun, 30 Nov 2014 20:51:53 -0800 Subject: [PATCH 068/305] [SPARK-4632] version update Author: Prabeesh K Closes #3495 from prabeesh/master and squashes the following commits: ab03d50 [Prabeesh K] Update pom.xml 8c6437e [Prabeesh K] Revert e10b40a [Prabeesh K] version update dbac9eb [Prabeesh K] Revert ec0b1c3 [Prabeesh K] [SPARK-4632] version update a835505 [Prabeesh K] [SPARK-4632] version update 831391b [Prabeesh K] [SPARK-4632] version update --- external/mqtt/pom.xml | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 703806735b3ff..9025915f4447e 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -43,8 +43,8 @@ org.eclipse.paho - mqtt-client - 0.4.0 + org.eclipse.paho.client.mqttv3 + 1.0.1 org.scalatest From 97eb6d7f511d56fca734ff73fcbd9694403943e0 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Mon, 1 Dec 2014 00:29:28 -0800 Subject: [PATCH 069/305] Fix wrong file name pattern in .gitignore In .gitignore, there is an entry for spark-*-bin.tar.gz but considering make-distribution.sh, the name pattern should be spark-*-bin-*.tgz. This change is really small so I don't open issue in JIRA. If it's needed, please let me know. Author: Kousuke Saruta Closes #3529 from sarutak/fix-wrong-tgz-pattern and squashes the following commits: de3c70a [Kousuke Saruta] Fixed wrong file name pattern in .gitignore --- .gitignore | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/.gitignore b/.gitignore index 34939e3a97aaa..c67cffa1c4375 100644 --- a/.gitignore +++ b/.gitignore @@ -49,7 +49,7 @@ dependency-reduced-pom.xml checkpoint derby.log dist/ -spark-*-bin.tar.gz +spark-*-bin-*.tgz unit-tests.log /lib/ rat-results.txt From 6384f42ab2e5c2b3e767ab4a428cda20a8ddcbe1 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 1 Dec 2014 16:31:04 +0800 Subject: [PATCH 070/305] SPARK-2192 [BUILD] Examples Data Not in Binary Distribution Simply, add data/ to distributions. This adds about 291KB (compressed) to the tarball, FYI. Author: Sean Owen Closes #3480 from srowen/SPARK-2192 and squashes the following commits: 47688f1 [Sean Owen] Add data/ to distributions --- make-distribution.sh | 3 +++ 1 file changed, 3 insertions(+) diff --git a/make-distribution.sh b/make-distribution.sh index 7c0fb8992a155..45c99e42e5a5b 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -201,6 +201,9 @@ if [ -e "$FWDIR"/CHANGES.txt ]; then cp "$FWDIR/CHANGES.txt" "$DISTDIR" fi +# Copy data files +cp -r "$FWDIR/data" "$DISTDIR" + # Copy other things mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf From 1d238f221c3e13c525b3af0c78eda95059ce9fc6 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 1 Dec 2014 00:32:54 -0800 Subject: [PATCH 071/305] [SPARK-4664][Core] Throw an exception when spark.akka.frameSize > 2047 If `spark.akka.frameSize` > 2047, it will overflow and become negative. Should have some assertion in `maxFrameSizeBytes` to warn people. Author: zsxwing Closes #3527 from zsxwing/SPARK-4664 and squashes the following commits: 0089c7a [zsxwing] Throw an exception when spark.akka.frameSize > 2047 --- .../src/main/scala/org/apache/spark/util/AkkaUtils.scala | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index 10010bdfa1a51..8c2457f56bffe 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -134,9 +134,16 @@ private[spark] object AkkaUtils extends Logging { Duration.create(conf.getLong("spark.akka.lookupTimeout", 30), "seconds") } + private val AKKA_MAX_FRAME_SIZE_IN_MB = Int.MaxValue / 1024 / 1024 + /** Returns the configured max frame size for Akka messages in bytes. */ def maxFrameSizeBytes(conf: SparkConf): Int = { - conf.getInt("spark.akka.frameSize", 10) * 1024 * 1024 + val frameSizeInMB = conf.getInt("spark.akka.frameSize", 10) + if (frameSizeInMB > AKKA_MAX_FRAME_SIZE_IN_MB) { + throw new IllegalArgumentException("spark.akka.frameSize should not be greater than " + + AKKA_MAX_FRAME_SIZE_IN_MB + "MB") + } + frameSizeInMB * 1024 * 1024 } /** Space reserved for extra data in an Akka message besides serialized task or task result. */ From 30a86acdefd5428af6d6264f59a037e0eefd74b4 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 1 Dec 2014 00:35:01 -0800 Subject: [PATCH 072/305] [SPARK-4661][Core] Minor code and docs cleanup Author: zsxwing Closes #3521 from zsxwing/SPARK-4661 and squashes the following commits: 03cbe3f [zsxwing] Minor code and docs cleanup --- core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala | 2 +- core/src/test/scala/org/apache/spark/ShuffleSuite.scala | 1 - .../org/apache/spark/sql/hive/execution/HiveTableScan.scala | 2 +- 3 files changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 56ac7a69be0d3..ed79032893d33 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -63,7 +63,7 @@ private[spark] class PipedRDD[T: ClassTag]( /** * A FilenameFilter that accepts anything that isn't equal to the name passed in. - * @param name of file or directory to leave out + * @param filterName of file or directory to leave out */ class NotEqualsFileNameFilter(filterName: String) extends FilenameFilter { def accept(dir: File, name: String): Boolean = { diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 85e5f9ab444b3..5d20b4dc1561a 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -270,7 +270,6 @@ object ShuffleSuite { def mergeCombineException(x: Int, y: Int): Int = { throw new SparkException("Exception for map-side combine.") - x + y } class NonJavaSerializableClass(val value: Int) extends Comparable[NonJavaSerializableClass] { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala index d39413a44a6cb..8bbcd6fec1f3b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScan.scala @@ -38,7 +38,7 @@ import org.apache.spark.sql.hive._ * :: DeveloperApi :: * The Hive table scan operator. Column and partition pruning are both handled. * - * @param attributes Attributes to be fetched from the Hive table. + * @param requestedAttributes Attributes to be fetched from the Hive table. * @param relation The Hive table be be scanned. * @param partitionPruningPred An optional partition pruning predicate for partitioned table. */ From 2b233f5fc4beb2c6ed4bc142e923e96f8bad3ec4 Mon Sep 17 00:00:00 2001 From: Madhu Siddalingaiah Date: Mon, 1 Dec 2014 08:45:34 -0800 Subject: [PATCH 073/305] Documentation: add description for repartitionAndSortWithinPartitions Author: Madhu Siddalingaiah Closes #3390 from msiddalingaiah/master and squashes the following commits: cbccbfe [Madhu Siddalingaiah] Documentation: replace with (again) 332f7a2 [Madhu Siddalingaiah] Documentation: replace with cd2b05a [Madhu Siddalingaiah] Merge remote-tracking branch 'upstream/master' 0fc12d7 [Madhu Siddalingaiah] Documentation: add description for repartitionAndSortWithinPartitions --- docs/programming-guide.md | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/docs/programming-guide.md b/docs/programming-guide.md index 7a16ee8742dc0..5e0d5c15d7069 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -934,6 +934,12 @@ for details. + + + +
Executor ID Address
spark.history.fs.logDirectory(none)file:/tmp/spark-events Directory that contains application event logs to be loaded by the history server Reshuffle the data in the RDD randomly to create either more or fewer partitions and balance it across them. This always shuffles all data over the network.
repartitionAndSortWithinPartitions(partitioner) Repartition the RDD according to the given partitioner and, within each resulting partition, + sort records by their keys. This is more efficient than calling repartition and then sorting within + each partition because it can push the sorting down into the shuffle machinery.
### Actions From 5db8dcaf494e0dffed4fc22f19b0334d95ab6bfb Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 1 Dec 2014 13:09:51 -0800 Subject: [PATCH 074/305] [SPARK-4258][SQL][DOC] Documents spark.sql.parquet.filterPushdown Documents `spark.sql.parquet.filterPushdown`, explains why it's turned off by default and when it's safe to be turned on. [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3440) Author: Cheng Lian Closes #3440 from liancheng/parquet-filter-pushdown-doc and squashes the following commits: 2104311 [Cheng Lian] Documents spark.sql.parquet.filterPushdown --- docs/sql-programming-guide.md | 22 ++++++++++++++++------ 1 file changed, 16 insertions(+), 6 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 24a68bb083334..96a3209c52eb1 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -146,7 +146,7 @@ describes the various methods for loading data into a SchemaRDD. Spark SQL supports two different methods for converting existing RDDs into SchemaRDDs. The first method uses reflection to infer the schema of an RDD that contains specific types of objects. This -reflection based approach leads to more concise code and works well when you already know the schema +reflection based approach leads to more concise code and works well when you already know the schema while writing your Spark application. The second method for creating SchemaRDDs is through a programmatic interface that allows you to @@ -566,7 +566,7 @@ for teenName in teenNames.collect(): ### Configuration -Configuration of Parquet can be done using the `setConf` method on SQLContext or by running +Configuration of Parquet can be done using the `setConf` method on SQLContext or by running `SET key=value` commands using SQL. @@ -575,8 +575,8 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or @@ -591,10 +591,20 @@ Configuration of Parquet can be done using the `setConf` method on SQLContext or + + + + + @@ -945,7 +955,7 @@ options. ## Migration Guide for Shark User -### Scheduling +### Scheduling To set a [Fair Scheduler](job-scheduling.html#fair-scheduler-pools) pool for a JDBC client session, users can set the `spark.sql.thriftserver.scheduler.pool` variable: From bafee67ebad01f7aea2cd393a70b57eb8345eeb0 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Mon, 1 Dec 2014 13:12:30 -0800 Subject: [PATCH 075/305] [SQL] add @group tab in limit() and count() group tab is missing for scaladoc Author: Jacky Li Closes #3458 from jackylk/patch-7 and squashes the following commits: 0121a70 [Jacky Li] add @group tab in limit() and count() --- sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 452baab8eb889..c6d4dabf83bc4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -225,6 +225,8 @@ class SchemaRDD( * {{{ * schemaRDD.limit(10) * }}} + * + * @group Query */ def limit(limitNum: Int): SchemaRDD = new SchemaRDD(sqlContext, Limit(Literal(limitNum), logicalPlan)) @@ -355,6 +357,8 @@ class SchemaRDD( * Return the number of elements in the RDD. Unlike the base RDD implementation of count, this * implementation leverages the query optimizer to compute the count on the SchemaRDD, which * supports features such as filter pushdown. + * + * @group Query */ @Experimental override def count(): Long = aggregate(Count(Literal(1))).collect().head.getLong(0) From b57365a1ec89e31470f424ff37d5ebc7c90a39d8 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 1 Dec 2014 13:17:56 -0800 Subject: [PATCH 076/305] [SPARK-4358][SQL] Let BigDecimal do checking type compatibility Remove hardcoding max and min values for types. Let BigDecimal do checking type compatibility. Author: Liang-Chi Hsieh Closes #3208 from viirya/more_numericLit and squashes the following commits: e9834b4 [Liang-Chi Hsieh] Remove byte and short types for number literal. 1bd1825 [Liang-Chi Hsieh] Fix Indentation and make the modification clearer. cf1a997 [Liang-Chi Hsieh] Modified for comment to add a rule of analysis that adds a cast. 91fe489 [Liang-Chi Hsieh] add Byte and Short. 1bdc69d [Liang-Chi Hsieh] Let BigDecimal do checking type compatibility. --- .../org/apache/spark/sql/catalyst/SqlParser.scala | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index dc1d349f10f1b..a9ff10f2d5533 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -340,18 +340,13 @@ class SqlParser extends AbstractSparkSQLParser { | floatLit ^^ { f => Literal(f.toDouble) } ) - private val longMax = BigDecimal(s"${Long.MaxValue}") - private val longMin = BigDecimal(s"${Long.MinValue}") - private val intMax = BigDecimal(s"${Int.MaxValue}") - private val intMin = BigDecimal(s"${Int.MinValue}") - private def toNarrowestIntegerType(value: String) = { val bigIntValue = BigDecimal(value) bigIntValue match { - case v if v < longMin || v > longMax => v - case v if v < intMin || v > intMax => v.toLong - case v => v.toInt + case v if bigIntValue.isValidInt => v.toIntExact + case v if bigIntValue.isValidLong => v.toLongExact + case v => v } } From 6a9ff19dc06745144d5b311d4f87073c81d53a8f Mon Sep 17 00:00:00 2001 From: ravipesala Date: Mon, 1 Dec 2014 13:26:44 -0800 Subject: [PATCH 077/305] [SPARK-4650][SQL] Supporting multi column support in countDistinct function like count(distinct c1,c2..) in Spark SQL Supporting multi column support in countDistinct function like count(distinct c1,c2..) in Spark SQL Author: ravipesala Author: Michael Armbrust Closes #3511 from ravipesala/countdistinct and squashes the following commits: cc4dbb1 [ravipesala] style 070e12a [ravipesala] Supporting multi column support in count(distinct c1,c2..) in Spark SQL --- .../scala/org/apache/spark/sql/catalyst/SqlParser.scala | 3 ++- .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 7 +++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala index a9ff10f2d5533..a2bcd73b6074f 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -277,7 +277,8 @@ class SqlParser extends AbstractSparkSQLParser { | SUM ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => SumDistinct(exp) } | COUNT ~ "(" ~> "*" <~ ")" ^^ { case _ => Count(Literal(1)) } | COUNT ~ "(" ~> expression <~ ")" ^^ { case exp => Count(exp) } - | COUNT ~> "(" ~> DISTINCT ~> expression <~ ")" ^^ { case exp => CountDistinct(exp :: Nil) } + | COUNT ~> "(" ~> DISTINCT ~> repsep(expression, ",") <~ ")" ^^ + { case exps => CountDistinct(exps) } | APPROXIMATE ~ COUNT ~ "(" ~ DISTINCT ~> expression <~ ")" ^^ { case exp => ApproxCountDistinct(exp) } | APPROXIMATE ~> "(" ~> floatLit ~ ")" ~ COUNT ~ "(" ~ DISTINCT ~ expression <~ ")" ^^ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 84ee3051eb682..f83e647014193 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -992,4 +992,11 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { "nulldata2 on nulldata1.value <=> nulldata2.value"), (1 to 2).map(i => Seq(i))) } + + test("Multi-column COUNT(DISTINCT ...)") { + val data = TestData(1,"val_1") :: TestData(2,"val_2") :: Nil + val rdd = sparkContext.parallelize((0 to 1).map(i => data(i))) + rdd.registerTempTable("distinctData") + checkAnswer(sql("SELECT COUNT(DISTINCT key,value) FROM distinctData"), 2) + } } From bc353819cc86c3b0ad75caf81b47744bfc2aeeb3 Mon Sep 17 00:00:00 2001 From: ravipesala Date: Mon, 1 Dec 2014 13:31:27 -0800 Subject: [PATCH 078/305] [SPARK-4658][SQL] Code documentation issue in DDL of datasource API Author: ravipesala Closes #3516 from ravipesala/ddl_doc and squashes the following commits: d101fdf [ravipesala] Style issues fixed d2238cd [ravipesala] Corrected documentation --- .../main/scala/org/apache/spark/sql/parquet/newParquet.scala | 4 ++-- .../src/main/scala/org/apache/spark/sql/sources/ddl.scala | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index bea12e6dd674e..6404fec435178 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -39,8 +39,8 @@ import scala.collection.JavaConversions._ /** * Allows creation of parquet based tables using the syntax - * `CREATE TABLE ... USING org.apache.spark.sql.parquet`. Currently the only option required - * is `path`, which should be the location of a collection of, optionally partitioned, + * `CREATE TEMPORARY TABLE ... USING org.apache.spark.sql.parquet`. Currently the only option + * required is `path`, which should be the location of a collection of, optionally partitioned, * parquet files. */ class DefaultSource extends RelationProvider { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala index 9168ca2fc6fec..ca510cb0b07e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/ddl.scala @@ -67,7 +67,7 @@ private[sql] class DDLParser extends StandardTokenParsers with PackratParsers wi protected lazy val ddl: Parser[LogicalPlan] = createTable /** - * CREATE FOREIGN TEMPORARY TABLE avroTable + * CREATE TEMPORARY TABLE avroTable * USING org.apache.spark.sql.avro * OPTIONS (path "../hive/src/test/resources/data/files/episodes.avro") */ From 7b79957879db4dfcc7c3601cb40ac4fd576259a5 Mon Sep 17 00:00:00 2001 From: wangfei Date: Mon, 1 Dec 2014 14:02:02 -0800 Subject: [PATCH 079/305] [SQL] Minor fix for doc and comment Author: wangfei Closes #3533 from scwf/sql-doc1 and squashes the following commits: 962910b [wangfei] doc and comment fix --- docs/sql-programming-guide.md | 3 ++- .../org/apache/spark/examples/sql/hive/HiveFromSpark.scala | 7 ++++--- .../scala/org/apache/spark/sql/parquet/newParquet.scala | 2 +- 3 files changed, 7 insertions(+), 5 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 96a3209c52eb1..c38ca556530e6 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1002,7 +1002,7 @@ Several caching related features are not supported yet: ## Compatibility with Apache Hive Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. Currently Spark -SQL is based on Hive 0.12.0. +SQL is based on Hive 0.12.0 and 0.13.1. #### Deploying in Existing Hive Warehouses @@ -1041,6 +1041,7 @@ Spark SQL supports the vast majority of Hive features, such as: * Sampling * Explain * Partitioned tables +* View * All Hive DDL Functions, including: * `CREATE TABLE` * `CREATE TABLE AS SELECT` diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 227acc117502d..138923c4d7f2f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -29,9 +29,10 @@ object HiveFromSpark { val sc = new SparkContext(sparkConf) val path = s"${System.getenv("SPARK_HOME")}/examples/src/main/resources/kv1.txt" - // A local hive context creates an instance of the Hive Metastore in process, storing - // the warehouse data in the current directory. This location can be overridden by - // specifying a second parameter to the constructor. + // A hive context adds support for finding tables in the MetaStore and writing queries + // using HiveQL. Users who do not have an existing Hive deployment can still create a + // HiveContext. When not configured by the hive-site.xml, the context automatically + // creates metastore_db and warehouse in the current directory. val hiveContext = new HiveContext(sc) import hiveContext._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 6404fec435178..9b89c3bfb3307 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -49,7 +49,7 @@ class DefaultSource extends RelationProvider { sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { val path = - parameters.getOrElse("path", sys.error("'path' must be specifed for parquet tables.")) + parameters.getOrElse("path", sys.error("'path' must be specified for parquet tables.")) ParquetRelation2(path)(sqlContext) } From 5edbcbfb61703398a24ce5162a74aba04e365b0c Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 1 Dec 2014 14:03:57 -0800 Subject: [PATCH 080/305] [SQL][DOC] Date type in SQL programming guide Author: Daoyuan Wang Closes #3535 from adrian-wang/datedoc and squashes the following commits: 18ff1ed [Daoyuan Wang] [DOC] Date type --- docs/sql-programming-guide.md | 23 +++++++++++++++++++++++ 1 file changed, 23 insertions(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index c38ca556530e6..85d446b9da0e7 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1057,6 +1057,7 @@ Spark SQL supports the vast majority of Hive features, such as: * `STRING` * `BINARY` * `TIMESTAMP` + * `DATE` * `ARRAY<>` * `MAP<>` * `STRUCT<>` @@ -1157,6 +1158,7 @@ evaluated by the SQL execution engine. A full list of the functions supported c * Datetime type - `TimestampType`: Represents values comprising values of fields year, month, day, hour, minute, and second. + - `DateType`: Represents values comprising values of fields year, month, day. * Complex types - `ArrayType(elementType, containsNull)`: Represents values comprising a sequence of elements with the type of `elementType`. `containsNull` is used to indicate if @@ -1264,6 +1266,13 @@ import org.apache.spark.sql._ TimestampType + + + + + @@ -1390,6 +1399,13 @@ please use factory methods provided in DataType.TimestampType + + + + + @@ -1537,6 +1553,13 @@ from pyspark.sql import * TimestampType() + + + + + From 4df60a8cbc58f2877787245c2a83b2de85579c82 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 1 Dec 2014 16:08:51 -0800 Subject: [PATCH 081/305] [SPARK-4529] [SQL] support view with column alias Support view definition like CREATE VIEW view3(valoo) TBLPROPERTIES ("fear" = "factor") AS SELECT upper(value) FROM src WHERE key=86; [valoo as the alias of upper(value)]. This is missing part of SPARK-4239, for a fully view support. Author: Daoyuan Wang Closes #3396 from adrian-wang/viewcolumn and squashes the following commits: 4d001d0 [Daoyuan Wang] support view with column alias --- .../src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 2 +- .../src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala | 4 ++-- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index b9283f668a9b5..f4c42bbc5b03d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -379,7 +379,7 @@ private[hive] object HiveQl { protected def nameExpressions(exprs: Seq[Expression]): Seq[NamedExpression] = { exprs.zipWithIndex.map { case (ne: NamedExpression, _) => ne - case (e, i) => Alias(e, s"c_$i")() + case (e, i) => Alias(e, s"_c$i")() } } 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 b255a2ebb9778..fecf8faaf4cda 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 @@ -279,7 +279,7 @@ private[hive] case class HiveGenericUdtf( } override protected def makeOutput() = { - // Use column names when given, otherwise c_1, c_2, ... c_n. + // Use column names when given, otherwise _c1, _c2, ... _cn. if (aliasNames.size == outputDataTypes.size) { aliasNames.zip(outputDataTypes).map { case (attrName, attrDataType) => @@ -288,7 +288,7 @@ private[hive] case class HiveGenericUdtf( } else { outputDataTypes.zipWithIndex.map { case (attrDataType, i) => - AttributeReference(s"c_$i", attrDataType, nullable = true)() + AttributeReference(s"_c$i", attrDataType, nullable = true)() } } } From d3e02dddf06c82e4baa8708050be291d87d4f367 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 1 Dec 2014 16:39:54 -0800 Subject: [PATCH 082/305] [SPARK-4268][SQL] Use #::: to get benefit from Stream in SqlLexical.allCaseVersions In addition, using `s.isEmpty` to eliminate the string comparison. Author: zsxwing Closes #3132 from zsxwing/SPARK-4268 and squashes the following commits: 358e235 [zsxwing] Improvement of allCaseVersions --- .../scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala index b198ed9936d95..f1a1ca6616a21 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SparkSQLParser.scala @@ -97,10 +97,10 @@ class SqlLexical(val keywords: Seq[String]) extends StdLexical { /** Generate all variations of upper and lower case of a given string */ def allCaseVersions(s: String, prefix: String = ""): Stream[String] = { - if (s == "") { + if (s.isEmpty) { Stream(prefix) } else { - allCaseVersions(s.tail, prefix + s.head.toLower) ++ + allCaseVersions(s.tail, prefix + s.head.toLower) #::: allCaseVersions(s.tail, prefix + s.head.toUpper) } } From b0a46d899541ec17db090aac6f9ea1b287ee9331 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 1 Dec 2014 17:27:14 -0800 Subject: [PATCH 083/305] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #1612 (close requested by 'marmbrus') Closes #2723 (close requested by 'marmbrus') Closes #1737 (close requested by 'marmbrus') Closes #2252 (close requested by 'marmbrus') Closes #2029 (close requested by 'marmbrus') Closes #2386 (close requested by 'marmbrus') Closes #2997 (close requested by 'marmbrus') From 64f3175bf976f5a28e691cedc7a4b333709e0c58 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Tue, 2 Dec 2014 11:40:43 +0800 Subject: [PATCH 084/305] [SPARK-4611][MLlib] Implement the efficient vector norm The vector norm in breeze is implemented by `activeIterator` which is known to be very slow. In this PR, an efficient vector norm is implemented, and with this API, `Normalizer` and `k-means` have big performance improvement. Here is the benchmark against mnist8m dataset. a) `Normalizer` Before DenseVector: 68.25secs SparseVector: 17.01secs With this PR DenseVector: 12.71secs SparseVector: 2.73secs b) `k-means` Before DenseVector: 83.46secs SparseVector: 61.60secs With this PR DenseVector: 70.04secs SparseVector: 59.05secs Author: DB Tsai Closes #3462 from dbtsai/norm and squashes the following commits: 63c7165 [DB Tsai] typo 0c3637f [DB Tsai] add import org.apache.spark.SparkContext._ back 6fa616c [DB Tsai] address feedback 9b7cb56 [DB Tsai] move norm to static method 0b632e6 [DB Tsai] kmeans dbed124 [DB Tsai] style c1a877c [DB Tsai] first commit --- .../spark/mllib/clustering/KMeans.scala | 6 +-- .../spark/mllib/feature/Normalizer.scala | 4 +- .../apache/spark/mllib/linalg/Vectors.scala | 51 +++++++++++++++++++ .../spark/mllib/linalg/VectorsSuite.scala | 24 +++++++++ 4 files changed, 79 insertions(+), 6 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 34ea0de706f08..0f8dee58d8464 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} +import breeze.linalg.{DenseVector => BDV, Vector => BV} import org.apache.spark.annotation.Experimental import org.apache.spark.Logging @@ -125,7 +125,7 @@ class KMeans private ( } // Compute squared norms and cache them. - val norms = data.map(v => breezeNorm(v.toBreeze, 2.0)) + val norms = data.map(Vectors.norm(_, 2.0)) norms.persist() val breezeData = data.map(_.toBreeze).zip(norms).map { case (v, norm) => new BreezeVectorWithNorm(v, norm) @@ -425,7 +425,7 @@ object KMeans { private[clustering] class BreezeVectorWithNorm(val vector: BV[Double], val norm: Double) extends Serializable { - def this(vector: BV[Double]) = this(vector, breezeNorm(vector, 2.0)) + def this(vector: BV[Double]) = this(vector, Vectors.norm(Vectors.fromBreeze(vector), 2.0)) def this(array: Array[Double]) = this(new BDV[Double](array)) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala index a9c2e23717896..1ced26a9b70a2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Normalizer.scala @@ -17,8 +17,6 @@ package org.apache.spark.mllib.feature -import breeze.linalg.{norm => brzNorm} - import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} @@ -47,7 +45,7 @@ class Normalizer(p: Double) extends VectorTransformer { * @return normalized vector. If the norm of the input is zero, it will return the input vector. */ override def transform(vector: Vector): Vector = { - val norm = brzNorm(vector.toBreeze, p) + val norm = Vectors.norm(vector, p) if (norm != 0.0) { // For dense vector, we've to allocate new memory for new output vector. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index c6d5fe5bc678c..47d1a76fa361d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -261,6 +261,57 @@ object Vectors { sys.error("Unsupported Breeze vector type: " + v.getClass.getName) } } + + /** + * Returns the p-norm of this vector. + * @param vector input vector. + * @param p norm. + * @return norm in L^p^ space. + */ + private[spark] def norm(vector: Vector, p: Double): Double = { + require(p >= 1.0) + val values = vector match { + case dv: DenseVector => dv.values + case sv: SparseVector => sv.values + case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + val size = values.size + + if (p == 1) { + var sum = 0.0 + var i = 0 + while (i < size) { + sum += math.abs(values(i)) + i += 1 + } + sum + } else if (p == 2) { + var sum = 0.0 + var i = 0 + while (i < size) { + sum += values(i) * values(i) + i += 1 + } + math.sqrt(sum) + } else if (p == Double.PositiveInfinity) { + var max = 0.0 + var i = 0 + while (i < size) { + val value = math.abs(values(i)) + if (value > max) max = value + i += 1 + } + max + } else { + var sum = 0.0 + var i = 0 + while (i < size) { + sum += math.pow(math.abs(values(i)), p) + i += 1 + } + math.pow(sum, 1.0 / p) + } + } } /** diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 9492f604af4d5..f99f01450992a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -21,6 +21,7 @@ import breeze.linalg.{DenseMatrix => BDM} import org.scalatest.FunSuite import org.apache.spark.SparkException +import org.apache.spark.mllib.util.TestingUtils._ class VectorsSuite extends FunSuite { @@ -197,4 +198,27 @@ class VectorsSuite extends FunSuite { assert(svMap.get(2) === Some(3.1)) assert(svMap.get(3) === Some(0.0)) } + + test("vector p-norm") { + val dv = Vectors.dense(0.0, -1.2, 3.1, 0.0, -4.5, 1.9) + val sv = Vectors.sparse(6, Seq((1, -1.2), (2, 3.1), (3, 0.0), (4, -4.5), (5, 1.9))) + + assert(Vectors.norm(dv, 1.0) ~== dv.toArray.foldLeft(0.0)((a, v) => + a + math.abs(v)) relTol 1E-8) + assert(Vectors.norm(sv, 1.0) ~== sv.toArray.foldLeft(0.0)((a, v) => + a + math.abs(v)) relTol 1E-8) + + assert(Vectors.norm(dv, 2.0) ~== math.sqrt(dv.toArray.foldLeft(0.0)((a, v) => + a + v * v)) relTol 1E-8) + assert(Vectors.norm(sv, 2.0) ~== math.sqrt(sv.toArray.foldLeft(0.0)((a, v) => + a + v * v)) relTol 1E-8) + + assert(Vectors.norm(dv, Double.PositiveInfinity) ~== dv.toArray.map(math.abs).max relTol 1E-8) + assert(Vectors.norm(sv, Double.PositiveInfinity) ~== sv.toArray.map(math.abs).max relTol 1E-8) + + assert(Vectors.norm(dv, 3.7) ~== math.pow(dv.toArray.foldLeft(0.0)((a, v) => + a + math.pow(math.abs(v), 3.7)), 1.0 / 3.7) relTol 1E-8) + assert(Vectors.norm(sv, 3.7) ~== math.pow(sv.toArray.foldLeft(0.0)((a, v) => + a + math.pow(math.abs(v), 3.7)), 1.0 / 3.7) relTol 1E-8) + } } From 6dfe38a03a619282815b4032243a20414eea712e Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 2 Dec 2014 00:18:41 -0800 Subject: [PATCH 085/305] [SPARK-4397][Core] Cleanup 'import SparkContext._' in core This PR cleans up `import SparkContext._` in core for SPARK-4397(#3262) to prove it really works well. Author: zsxwing Closes #3530 from zsxwing/SPARK-4397-cleanup and squashes the following commits: 04e2273 [zsxwing] Cleanup 'import SparkContext._' in core --- .../main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 1 - .../main/scala/org/apache/spark/api/python/PythonRDD.scala | 1 - core/src/main/scala/org/apache/spark/package.scala | 4 ++-- .../main/scala/org/apache/spark/rdd/AsyncRDDActions.scala | 1 - .../scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 1 - .../scala/org/apache/spark/rdd/OrderedRDDFunctions.scala | 7 +++---- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 -- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 6 +++--- .../org/apache/spark/rdd/SequenceFileRDDFunctions.scala | 1 - .../scala/org/apache/spark/ui/UIWorkloadGenerator.scala | 1 - .../apache/spark/util/random/StratifiedSamplingUtils.scala | 1 - .../src/test/scala/org/apache/spark/AccumulatorSuite.scala | 1 - core/src/test/scala/org/apache/spark/CheckpointSuite.scala | 1 - .../test/scala/org/apache/spark/ContextCleanerSuite.scala | 1 - .../src/test/scala/org/apache/spark/DistributedSuite.scala | 1 - .../org/apache/spark/ExternalShuffleServiceSuite.scala | 1 - core/src/test/scala/org/apache/spark/FailureSuite.scala | 1 - core/src/test/scala/org/apache/spark/FileServerSuite.scala | 1 - .../test/scala/org/apache/spark/FutureActionSuite.scala | 1 - .../scala/org/apache/spark/ImplicitOrderingSuite.scala | 1 - .../test/scala/org/apache/spark/JobCancellationSuite.scala | 1 - .../test/scala/org/apache/spark/PartitioningSuite.scala | 1 - core/src/test/scala/org/apache/spark/ShuffleSuite.scala | 1 - .../test/scala/org/apache/spark/StatusTrackerSuite.scala | 1 - .../scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala | 1 - .../test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala | 1 - .../scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala | 1 - core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 2 -- .../src/test/scala/org/apache/spark/rdd/SortingSuite.scala | 1 - .../org/apache/spark/scheduler/DAGSchedulerSuite.scala | 1 - .../org/apache/spark/scheduler/ReplayListenerSuite.scala | 1 - .../org/apache/spark/scheduler/SparkListenerSuite.scala | 1 - .../spark/serializer/KryoSerializerDistributedSuite.scala | 1 - .../test/scala/org/apache/spark/ui/UISeleniumSuite.scala | 1 - .../spark/util/collection/ExternalAppendOnlyMapSuite.scala | 1 - .../apache/spark/util/collection/ExternalSorterSuite.scala | 1 - 36 files changed, 8 insertions(+), 44 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 5a8e5bb1f721a..ac42294d56def 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -28,7 +28,6 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark._ -import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index e0bc00e1eb249..bad40e6529f74 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -34,7 +34,6 @@ import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{InputFormat, OutputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, OutputFormat => NewOutputFormat} import org.apache.spark._ -import org.apache.spark.SparkContext._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.RDD diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 436dbed1730bc..5ad73c3d27f47 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -27,8 +27,8 @@ package org.apache * contains operations available only on RDDs of Doubles; and * [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that can * be saved as SequenceFiles. These operations are automatically available on any RDD of the right - * type (e.g. RDD[(Int, Int)] through implicit conversions when you - * `import org.apache.spark.SparkContext._`. + * type (e.g. RDD[(Int, Int)] through implicit conversions except `saveAsSequenceFile`. You need to + * `import org.apache.spark.SparkContext._` to make `saveAsSequenceFile` work. * * Java programmers should reference the [[org.apache.spark.api.java]] package * for Spark programming APIs in Java. diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index 9f9f10b7ebc3a..646df283ac069 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -27,7 +27,6 @@ import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} /** * A set of asynchronous RDD actions available through an implicit conversion. - * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging { diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index e0494ee39657c..e66f83bb34e30 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -27,7 +27,6 @@ import org.apache.spark.util.StatCounter /** * Extra functions available on RDDs of Doubles through an implicit conversion. - * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** Add up the elements in this RDD. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala index d0dbfef35d03c..144f679a59460 100644 --- a/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/OrderedRDDFunctions.scala @@ -24,10 +24,9 @@ import org.apache.spark.annotation.DeveloperApi /** * Extra functions available on RDDs of (key, value) pairs where the key is sortable through - * an implicit conversion. Import `org.apache.spark.SparkContext._` at the top of your program to - * use these functions. They will work with any key type `K` that has an implicit `Ordering[K]` in - * scope. Ordering objects already exist for all of the standard primitive types. Users can also - * define their own orderings for custom types, or to override the default ordering. The implicit + * an implicit conversion. They will work with any key type `K` that has an implicit `Ordering[K]` + * in scope. Ordering objects already exist for all of the standard primitive types. Users can also + * define their own orderings for custom types, or to override the default ordering. The implicit * ordering that is in the closest scope will be used. * * {{{ diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 8c2c959e73bb6..e78e57678852f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -37,7 +37,6 @@ RecordWriter => NewRecordWriter} import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner -import org.apache.spark.SparkContext._ import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{DataWriteMethod, OutputMetrics} @@ -50,7 +49,6 @@ import org.apache.spark.util.random.StratifiedSamplingUtils /** * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. - * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ class PairRDDFunctions[K, V](self: RDD[(K, V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3add4a76192ca..8dfd952298f30 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -34,7 +34,6 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ import org.apache.spark.Partitioner._ -import org.apache.spark.SparkContext._ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.broadcast.Broadcast @@ -58,8 +57,9 @@ import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, Bernoulli * Doubles; and * [[org.apache.spark.rdd.SequenceFileRDDFunctions]] contains operations available on RDDs that * can be saved as SequenceFiles. - * These operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] - * through implicit conversions when you `import org.apache.spark.SparkContext._`. + * All operations are automatically available on any RDD of the right type (e.g. RDD[(Int, Int)] + * through implicit conversions except `saveAsSequenceFile`. You need to + * `import org.apache.spark.SparkContext._` to make `saveAsSequenceFile` work. * * Internally, each RDD is characterized by five main properties: * diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 9a1efc83cbe6a..2b48916951430 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -24,7 +24,6 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.SequenceFileOutputFormat import org.apache.spark.Logging -import org.apache.spark.SparkContext._ /** * Extra functions available on RDDs of (key, value) pairs to create a Hadoop SequenceFile, diff --git a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala index 18d2b5075aa08..b4677447c8872 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIWorkloadGenerator.scala @@ -20,7 +20,6 @@ package org.apache.spark.ui import scala.util.Random import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.SparkContext._ import org.apache.spark.scheduler.SchedulingMode /** diff --git a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala index 4fa357edd6f07..2ae308dacf1ae 100644 --- a/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/random/StratifiedSamplingUtils.scala @@ -25,7 +25,6 @@ import scala.reflect.ClassTag import org.apache.commons.math3.distribution.PoissonDistribution import org.apache.spark.Logging -import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD /** diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 52d1d5277658e..f087fc550dde3 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -22,7 +22,6 @@ import scala.collection.mutable import org.scalatest.FunSuite import org.scalatest.Matchers -import org.apache.spark.SparkContext._ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index a41914a1a9d0c..3b10b3a042317 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -23,7 +23,6 @@ import scala.reflect.ClassTag import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ import org.apache.spark.rdd._ import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} import org.apache.spark.util.Utils diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 2e3fc5ef0e336..ae2ae7ed0d3aa 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -28,7 +28,6 @@ import org.scalatest.concurrent.{PatienceConfiguration, Eventually} import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.storage._ import org.apache.spark.shuffle.hash.HashShuffleManager diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 429199f2075c6..998f3008ec0ea 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -23,7 +23,6 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.Matchers import org.scalatest.time.{Millis, Span} -import org.apache.spark.SparkContext._ import org.apache.spark.storage.{RDDBlockId, StorageLevel} class NotSerializableClass diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 55799f55146cb..cc3592ee43a35 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -21,7 +21,6 @@ import java.util.concurrent.atomic.AtomicInteger import org.scalatest.BeforeAndAfterAll -import org.apache.spark.SparkContext._ import org.apache.spark.network.TransportContext import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.server.TransportServer diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 2229e6acc425d..1212d0b43207d 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ import org.apache.spark.util.NonSerializable import java.io.NotSerializableException diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 379c2a6ea4b55..49426545c767e 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -23,7 +23,6 @@ import java.util.jar.{JarEntry, JarOutputStream} import com.google.common.io.ByteStreams import org.scalatest.FunSuite -import org.apache.spark.SparkContext._ import org.apache.spark.util.Utils class FileServerSuite extends FunSuite with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala index db9c25fc457a4..f5cdb01ec9504 100644 --- a/core/src/test/scala/org/apache/spark/FutureActionSuite.scala +++ b/core/src/test/scala/org/apache/spark/FutureActionSuite.scala @@ -22,7 +22,6 @@ import scala.concurrent.duration.Duration import org.scalatest.{BeforeAndAfter, FunSuite, Matchers} -import org.apache.spark.SparkContext._ class FutureActionSuite extends FunSuite with BeforeAndAfter with Matchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala index 8e4a9e2c9f56c..d895230ecf330 100644 --- a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark import org.scalatest.FunSuite import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ class ImplicitOrderingSuite extends FunSuite with LocalSparkContext { // Tests that PairRDDFunctions grabs an implicit Ordering in various cases where it should. diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index a57430e829ced..41ed2bce55ce1 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -27,7 +27,6 @@ import scala.concurrent.future import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.Matchers -import org.apache.spark.SparkContext._ import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} /** diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 646ede30ae6ff..b7532314ada01 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -22,7 +22,6 @@ import scala.math.abs import org.scalatest.{FunSuite, PrivateMethodTester} -import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.util.StatCounter diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 5d20b4dc1561a..5a133c0490444 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark import org.scalatest.FunSuite import org.scalatest.Matchers -import org.apache.spark.SparkContext._ import org.apache.spark.ShuffleSuite.NonJavaSerializableClass import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} import org.apache.spark.serializer.KryoSerializer diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala index 8577e4ac7e33e..41d6ea29d5b06 100644 --- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala @@ -25,7 +25,6 @@ import org.scalatest.{Matchers, FunSuite} import org.scalatest.concurrent.Eventually._ import org.apache.spark.JobExecutionStatus._ -import org.apache.spark.SparkContext._ class StatusTrackerSuite extends FunSuite with Matchers with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 3b833f2e41867..f2b0ea1063a72 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -27,7 +27,6 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkContext._ import org.apache.spark.{SparkContext, SparkException, LocalSparkContext} class AsyncRDDActionsSuite extends FunSuite with BeforeAndAfterAll with Timeouts { diff --git a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala index f89bdb6e07dea..de306533752c1 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.rdd import org.scalatest.FunSuite import org.apache.spark._ -import org.apache.spark.SparkContext._ class DoubleRDDSuite extends FunSuite with SharedSparkContext { // Verify tests on the histogram functionality. We test with both evenly diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 3620e251cc139..108f70af43f37 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -29,7 +29,6 @@ import org.apache.hadoop.mapreduce.{JobContext => NewJobContext, OutputCommitter OutputFormat => NewOutputFormat, RecordWriter => NewRecordWriter, TaskAttemptContext => NewTaskAttempContext} import org.apache.spark.{Partitioner, SharedSparkContext} -import org.apache.spark.SparkContext._ import org.apache.spark.util.Utils import org.scalatest.FunSuite diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index e079ca3b1e896..6d9be796144b6 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -24,7 +24,6 @@ import scala.reflect.ClassTag import org.scalatest.FunSuite import org.apache.spark._ -import org.apache.spark.SparkContext._ import org.apache.spark.util.Utils import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} @@ -97,7 +96,6 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("partitioner aware union") { - import SparkContext._ def makeRDDWithPartitioner(seq: Seq[Int]) = { sc.makeRDD(seq, 1) .map(x => (x, null)) diff --git a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala index 656917628f7a8..a40f2ffeffdf9 100644 --- a/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/SortingSuite.scala @@ -21,7 +21,6 @@ import org.scalatest.FunSuite import org.scalatest.Matchers import org.apache.spark.{Logging, SharedSparkContext} -import org.apache.spark.SparkContext._ class SortingSuite extends FunSuite with SharedSparkContext with Matchers with Logging { 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 bdd721dc7eaf7..436eea4f1fdcf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -27,7 +27,6 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index e05f373392d4a..90bdfe07f61c9 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -22,7 +22,6 @@ import java.io.{File, PrintWriter} import org.json4s.jackson.JsonMethods._ import org.scalatest.{BeforeAndAfter, FunSuite} -import org.apache.spark.SparkContext._ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.io.CompressionCodec diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index abe0dc35b07e2..b276343cb412c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -25,7 +25,6 @@ import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.Matchers import org.apache.spark.{LocalSparkContext, SparkContext} -import org.apache.spark.SparkContext._ import org.apache.spark.executor.TaskMetrics class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index 11e8c9c4cb37f..855f1b6276089 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -23,7 +23,6 @@ import com.esotericsoftware.kryo.Kryo import org.scalatest.FunSuite import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkEnv, TestUtils} -import org.apache.spark.SparkContext._ import org.apache.spark.serializer.KryoDistributedTest._ class KryoSerializerDistributedSuite extends FunSuite { diff --git a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala index d2857b8b55664..787f4c2b5a8b2 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -27,7 +27,6 @@ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ import org.apache.spark._ -import org.apache.spark.SparkContext._ import org.apache.spark.LocalSparkContext._ import org.apache.spark.api.java.StorageLevels import org.apache.spark.shuffle.FetchFailedException diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index 511d76c9144cc..48f79ea651018 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -22,7 +22,6 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.FunSuite import org.apache.spark._ -import org.apache.spark.SparkContext._ import org.apache.spark.io.CompressionCodec class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala index 3cb42d416de4f..72d96798b1141 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalSorterSuite.scala @@ -22,7 +22,6 @@ import scala.collection.mutable.ArrayBuffer import org.scalatest.{PrivateMethodTester, FunSuite} import org.apache.spark._ -import org.apache.spark.SparkContext._ import scala.util.Random From d9a148ba6a67a01e4bf77c35c41dd4cbc8918c82 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 2 Dec 2014 09:06:02 -0800 Subject: [PATCH 086/305] [SPARK-4686] Link to allowed master URLs is broken The link points to the old scala programming guide; it should point to the submitting applications page. This should be backported to 1.1.2 (it's been broken as of 1.0). Author: Kay Ousterhout Closes #3542 from kayousterhout/SPARK-4686 and squashes the following commits: a8fc43b [Kay Ousterhout] [SPARK-4686] Link to allowed master URLs is broken --- docs/configuration.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration.md b/docs/configuration.md index 0b77f5ab645c9..4b4bbea564d3a 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -98,7 +98,7 @@ of the most common options to set are: From b1f8fe316a6904841f0159ec02159b1af0ad730e Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 2 Dec 2014 11:59:15 -0800 Subject: [PATCH 087/305] Indent license header properly for interfaces.scala. A very small nit update. Author: Reynold Xin Closes #3552 from rxin/license-header and squashes the following commits: df8d1a4 [Reynold Xin] Indent license header properly for interfaces.scala. --- .../apache/spark/sql/sources/interfaces.scala | 32 +++++++++---------- 1 file changed, 15 insertions(+), 17 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 2b8fc05fc0102..939b4e15163a6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -1,19 +1,19 @@ /* -* 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. -*/ + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ package org.apache.spark.sql.sources import org.apache.spark.annotation.{Experimental, DeveloperApi} @@ -111,5 +111,3 @@ abstract class PrunedFilteredScan extends BaseRelation { abstract class CatalystScan extends BaseRelation { def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] } - - From e75e04f980281389b881df76f59ba1adc6338629 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 2 Dec 2014 12:07:52 -0800 Subject: [PATCH 088/305] [SPARK-4536][SQL] Add sqrt and abs to Spark SQL DSL Spark SQL has embeded sqrt and abs but DSL doesn't support those functions. Author: Kousuke Saruta Closes #3401 from sarutak/dsl-missing-operator and squashes the following commits: 07700cf [Kousuke Saruta] Modified Literal(null, NullType) to Literal(null) in DslQuerySuite 8f366f8 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into dsl-missing-operator 1b88e2e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into dsl-missing-operator 0396f89 [Kousuke Saruta] Added sqrt and abs to Spark SQL DSL --- .../spark/sql/catalyst/dsl/package.scala | 2 + .../sql/catalyst/expressions/arithmetic.scala | 1 - .../org/apache/spark/sql/DslQuerySuite.scala | 68 +++++++++++++++++++ .../scala/org/apache/spark/sql/TestData.scala | 4 ++ 4 files changed, 74 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 31dc5a58e68e5..70dabc4e6c2e9 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -147,6 +147,8 @@ package object dsl { def max(e: Expression) = Max(e) def upper(e: Expression) = Upper(e) def lower(e: Expression) = Lower(e) + def sqrt(e: Expression) = Sqrt(e) + def abs(e: Expression) = Abs(e) implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } // TODO more implicit class for literal? 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 d17c9553ac24e..900b7586adcda 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 @@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.types._ -import scala.math.pow case class UnaryMinus(child: Expression) extends UnaryExpression { type EvaluatedType = Any diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 94bd97758fe94..1a330a2bb6d46 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -282,4 +282,72 @@ class DslQuerySuite extends QueryTest { (1, "1", "11") :: (2, "2", "22") :: (3, "3", "33") :: Nil ) } + + test("sqrt") { + checkAnswer( + testData.select(sqrt('key)).orderBy('key asc), + (1 to 100).map(n => Seq(math.sqrt(n))) + ) + + checkAnswer( + testData.select(sqrt('value), 'key).orderBy('key asc, 'value asc), + (1 to 100).map(n => Seq(math.sqrt(n), n)) + ) + + checkAnswer( + testData.select(sqrt(Literal(null))), + (1 to 100).map(_ => Seq(null)) + ) + } + + test("abs") { + checkAnswer( + testData.select(abs('key)).orderBy('key asc), + (1 to 100).map(n => Seq(n)) + ) + + checkAnswer( + negativeData.select(abs('key)).orderBy('key desc), + (1 to 100).map(n => Seq(n)) + ) + + checkAnswer( + testData.select(abs(Literal(null))), + (1 to 100).map(_ => Seq(null)) + ) + } + + test("upper") { + checkAnswer( + lowerCaseData.select(upper('l)), + ('a' to 'd').map(c => Seq(c.toString.toUpperCase())) + ) + + checkAnswer( + testData.select(upper('value), 'key), + (1 to 100).map(n => Seq(n.toString, n)) + ) + + checkAnswer( + testData.select(upper(Literal(null))), + (1 to 100).map(n => Seq(null)) + ) + } + + test("lower") { + checkAnswer( + upperCaseData.select(lower('L)), + ('A' to 'F').map(c => Seq(c.toString.toLowerCase())) + ) + + checkAnswer( + testData.select(lower('value), 'key), + (1 to 100).map(n => Seq(n.toString, n)) + ) + + checkAnswer( + testData.select(lower(Literal(null))), + (1 to 100).map(n => Seq(null)) + ) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 933e027436e75..bb553a0a1e50c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -32,6 +32,10 @@ object TestData { (1 to 100).map(i => TestData(i, i.toString))).toSchemaRDD testData.registerTempTable("testData") + val negativeData = TestSQLContext.sparkContext.parallelize( + (1 to 100).map(i => TestData(-i, (-i).toString))).toSchemaRDD + negativeData.registerTempTable("negativeData") + case class LargeAndSmallInts(a: Int, b: Int) val largeAndSmallInts = TestSQLContext.sparkContext.parallelize( From 69b6fed206565ecb0173d3757bcb5110422887c3 Mon Sep 17 00:00:00 2001 From: baishuo Date: Tue, 2 Dec 2014 12:12:03 -0800 Subject: [PATCH 089/305] [SPARK-4663][sql]add finally to avoid resource leak Author: baishuo Closes #3526 from baishuo/master-trycatch and squashes the following commits: d446e14 [baishuo] correct the code style b36bf96 [baishuo] correct the code style ae0e447 [baishuo] add finally to avoid resource leak --- .../spark/sql/parquet/ParquetTableOperations.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala index 0e36852ddd9b0..232ef90b017a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -302,11 +302,14 @@ case class InsertIntoParquetTable( val committer = format.getOutputCommitter(hadoopContext) committer.setupTask(hadoopContext) val writer = format.getRecordWriter(hadoopContext) - while (iter.hasNext) { - val row = iter.next() - writer.write(null, row) + try { + while (iter.hasNext) { + val row = iter.next() + writer.write(null, row) + } + } finally { + writer.close(hadoopContext) } - writer.close(hadoopContext) committer.commitTask(hadoopContext) 1 } From 10664276007beca3843638e558f504cad44b1fb3 Mon Sep 17 00:00:00 2001 From: YanTangZhai Date: Tue, 2 Dec 2014 14:12:48 -0800 Subject: [PATCH 090/305] [SPARK-4676][SQL] JavaSchemaRDD.schema may throw NullType MatchError if sql has null val jsc = new org.apache.spark.api.java.JavaSparkContext(sc) val jhc = new org.apache.spark.sql.hive.api.java.JavaHiveContext(jsc) val nrdd = jhc.hql("select null from spark_test.for_test") println(nrdd.schema) Then the error is thrown as follows: scala.MatchError: NullType (of class org.apache.spark.sql.catalyst.types.NullType$) at org.apache.spark.sql.types.util.DataTypeConversions$.asJavaDataType(DataTypeConversions.scala:43) Author: YanTangZhai Author: yantangzhai Author: Michael Armbrust Closes #3538 from YanTangZhai/MatchNullType and squashes the following commits: e052dff [yantangzhai] [SPARK-4676] [SQL] JavaSchemaRDD.schema may throw NullType MatchError if sql has null 4b4bb34 [yantangzhai] [SPARK-4676] [SQL] JavaSchemaRDD.schema may throw NullType MatchError if sql has null 896c7b7 [yantangzhai] fix NullType MatchError in JavaSchemaRDD when sql has null 6e643f8 [YanTangZhai] Merge pull request #11 from apache/master e249846 [YanTangZhai] Merge pull request #10 from apache/master d26d982 [YanTangZhai] Merge pull request #9 from apache/master 76d4027 [YanTangZhai] Merge pull request #8 from apache/master 03b62b0 [YanTangZhai] Merge pull request #7 from apache/master 8a00106 [YanTangZhai] Merge pull request #6 from apache/master cbcba66 [YanTangZhai] Merge pull request #3 from apache/master cdef539 [YanTangZhai] Merge pull request #1 from apache/master --- .../apache/spark/sql/api/java/DataType.java | 5 ++++ .../apache/spark/sql/api/java/NullType.java | 27 +++++++++++++++++++ .../scala/org/apache/spark/sql/package.scala | 10 +++++++ .../sql/types/util/DataTypeConversions.scala | 1 + .../spark/sql/api/java/JavaSQLSuite.scala | 16 +++++++++++ 5 files changed, 59 insertions(+) create mode 100644 sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java 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 c38354039d686..c69bbd5736a5b 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 @@ -82,6 +82,11 @@ public abstract class DataType { */ public static final ShortType ShortType = new ShortType(); + /** + * Gets the NullType object. + */ + public static final NullType NullType = new NullType(); + /** * Creates an ArrayType by specifying the data type of elements ({@code elementType}). * The field of {@code containsNull} is set to {@code true}. diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java new file mode 100644 index 0000000000000..6d5ecdf46e551 --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/NullType.java @@ -0,0 +1,27 @@ +/* + * 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.api.java; + +/** + * The data type representing null and NULL values. + * + * {@code NullType} is represented by the singleton object {@link DataType#NullType}. + */ +public class NullType extends DataType { + protected NullType() {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/package.scala index 51dad54f1a3f3..1fd8e6220f83b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/package.scala @@ -263,6 +263,16 @@ package object sql { @DeveloperApi val ShortType = catalyst.types.ShortType + /** + * :: DeveloperApi :: + * + * The data type representing `NULL` values. + * + * @group dataType + */ + @DeveloperApi + val NullType = catalyst.types.NullType + /** * :: DeveloperApi :: * diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index d4258156f18f6..4160a80621c77 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -62,6 +62,7 @@ protected[sql] object DataTypeConversions { case IntegerType => JDataType.IntegerType case LongType => JDataType.LongType case ShortType => JDataType.ShortType + case NullType => JDataType.NullType case arrayType: ArrayType => JDataType.createArrayType( asJavaDataType(arrayType.elementType), arrayType.containsNull) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala index c9012c9e47cff..8afc3a9fb2187 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -68,6 +68,22 @@ class JavaSQLSuite extends FunSuite { javaSqlCtx.sql("SELECT * FROM people").collect() } + test("schema with null from JavaBeans") { + val person = new PersonBean + person.setName("Michael") + person.setAge(29) + + val rdd = javaCtx.parallelize(person :: Nil) + val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[PersonBean]) + + schemaRDD.registerTempTable("people") + val nullRDD = javaSqlCtx.sql("SELECT null FROM people") + val structFields = nullRDD.schema.getFields() + assert(structFields.size == 1) + assert(structFields(0).getDataType().isInstanceOf[NullType]) + assert(nullRDD.collect.head.row === Seq(null)) + } + test("all types in JavaBeans") { val bean = new AllTypesBean bean.setStringField("") From f6df609dcc4f4a18c0f1c74b1ae0800cf09fa7ae Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 2 Dec 2014 14:21:12 -0800 Subject: [PATCH 091/305] [SPARK-4593][SQL] Return null when denominator is 0 SELECT max(1/0) FROM src would return a very large number, which is obviously not right. For hive-0.12, hive would return `Infinity` for 1/0, while for hive-0.13.1, it is `NULL` for 1/0. I think it is better to keep our behavior with newer Hive version. This PR ensures that when the divider is 0, the result of expression should be NULL, same with hive-0.13.1 Author: Daoyuan Wang Closes #3443 from adrian-wang/div and squashes the following commits: 2e98677 [Daoyuan Wang] fix code gen for divide 0 85c28ba [Daoyuan Wang] temp 36236a5 [Daoyuan Wang] add test cases 6f5716f [Daoyuan Wang] fix comments cee92bd [Daoyuan Wang] avoid evaluation 2 times 22ecd9a [Daoyuan Wang] fix style cf28c58 [Daoyuan Wang] divide fix 2dfe50f [Daoyuan Wang] return null when divider is 0 of Double type --- .../sql/catalyst/expressions/Expression.scala | 41 +++++++++++++++++++ .../sql/catalyst/expressions/arithmetic.scala | 13 ++++-- .../expressions/codegen/CodeGenerator.scala | 19 ++++++++- .../ExpressionEvaluationSuite.scala | 15 +++++++ 4 files changed, 83 insertions(+), 5 deletions(-) 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 39b120e8de485..bc45881e42748 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 @@ -153,6 +153,25 @@ abstract class Expression extends TreeNode[Expression] { } } + /** + * Evaluation helper function for 1 Fractional children expression. + * if the expression result is null, the evaluation result should be null. + */ + @inline + protected final def f1(i: Row, e1: Expression, f: ((Fractional[Any], Any) => Any)): Any = { + val evalE1 = e1.eval(i: Row) + if(evalE1 == null) { + null + } else { + e1.dataType match { + case ft: FractionalType => + f.asInstanceOf[(Fractional[ft.JvmType], ft.JvmType) => ft.JvmType]( + ft.fractional, evalE1.asInstanceOf[ft.JvmType]) + case other => sys.error(s"Type $other does not support fractional operations") + } + } + } + /** * Evaluation helper function for 2 Integral children expressions. Those expressions are * supposed to be in the same data type, and also the return type. @@ -189,6 +208,28 @@ abstract class Expression extends TreeNode[Expression] { } } + /** + * Evaluation helper function for 1 Integral children expression. + * if the expression result is null, the evaluation result should be null. + */ + @inline + protected final def i1(i: Row, e1: Expression, f: ((Integral[Any], Any) => Any)): Any = { + val evalE1 = e1.eval(i) + if(evalE1 == null) { + null + } else { + e1.dataType match { + case i: IntegralType => + f.asInstanceOf[(Integral[i.JvmType], i.JvmType) => i.JvmType]( + i.integral, evalE1.asInstanceOf[i.JvmType]) + case i: FractionalType => + f.asInstanceOf[(Integral[i.JvmType], i.JvmType) => i.JvmType]( + i.asIntegral, evalE1.asInstanceOf[i.JvmType]) + case other => sys.error(s"Type $other does not support numeric operations") + } + } + } + /** * Evaluation helper function for 2 Comparable children expressions. Those expressions are * supposed to be in the same data type, and the return type should be Integer: 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 900b7586adcda..7ec18b8419e20 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 @@ -105,11 +105,16 @@ case class Multiply(left: Expression, right: Expression) extends BinaryArithmeti case class Divide(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "/" - override def nullable = left.nullable || right.nullable || dataType.isInstanceOf[DecimalType] + override def nullable = true - override def eval(input: Row): Any = dataType match { - case _: FractionalType => f2(input, left, right, _.div(_, _)) - case _: IntegralType => i2(input, left , right, _.quot(_, _)) + override def eval(input: Row): Any = { + val evalE2 = right.eval(input) + dataType match { + case _ if evalE2 == null => null + case _ if evalE2 == 0 => null + case ft: FractionalType => f1(input, left, _.div(_, evalE2.asInstanceOf[ft.JvmType])) + case it: IntegralType => i1(input, left, _.quot(_, evalE2.asInstanceOf[it.JvmType])) + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala index 67f8d411b6bb4..ab71e15e1f573 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala @@ -359,7 +359,24 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin case Add(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 + $eval2" } case Subtract(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 - $eval2" } case Multiply(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 * $eval2" } - case Divide(e1, e2) => (e1, e2) evaluate { case (eval1, eval2) => q"$eval1 / $eval2" } + case Divide(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(e1.dataType)} = 0 + + if (${eval1.nullTerm} || ${eval2.nullTerm} ) { + $nullTerm = true + } else if (${eval2.primitiveTerm} == 0) + $nullTerm = true + else { + $nullTerm = false + $primitiveTerm = ${eval1.primitiveTerm} / ${eval2.primitiveTerm} + } + """.children case IsNotNull(e) => val eval = expressionEvaluator(e) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 3f5b9f698f827..25f56424888aa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -149,6 +149,21 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), true) } + test("Divide") { + checkEvaluation(Divide(Literal(2), Literal(1)), 2) + checkEvaluation(Divide(Literal(1.0), Literal(2.0)), 0.5) + checkEvaluation(Divide(Literal(1), Literal(2)), 0) + checkEvaluation(Divide(Literal(1), Literal(0)), null) + checkEvaluation(Divide(Literal(1.0), Literal(0.0)), null) + checkEvaluation(Divide(Literal(0.0), Literal(0.0)), null) + checkEvaluation(Divide(Literal(0), Literal(null, IntegerType)), null) + checkEvaluation(Divide(Literal(1), Literal(null, IntegerType)), null) + checkEvaluation(Divide(Literal(null, IntegerType), Literal(0)), null) + checkEvaluation(Divide(Literal(null, DoubleType), Literal(0.0)), null) + checkEvaluation(Divide(Literal(null, IntegerType), Literal(1)), null) + checkEvaluation(Divide(Literal(null, IntegerType), Literal(null, IntegerType)), null) + } + test("INSET") { val hS = HashSet[Any]() + 1 + 2 val nS = HashSet[Any]() + 1 + 2 + null From 1f5ddf17e831ad9717f0f4b60a727a3381fad4f9 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Tue, 2 Dec 2014 14:25:12 -0800 Subject: [PATCH 092/305] [SPARK-4670] [SQL] wrong symbol for bitwise not We should use `~` instead of `-` for bitwise NOT. Author: Daoyuan Wang Closes #3528 from adrian-wang/symbol and squashes the following commits: affd4ad [Daoyuan Wang] fix code gen test case 56efb79 [Daoyuan Wang] ensure bitwise NOT over byte and short persist data type f55fbae [Daoyuan Wang] wrong symbol for bitwise not --- .../sql/catalyst/expressions/arithmetic.scala | 20 +++++++++---------- .../ExpressionEvaluationSuite.scala | 15 ++++++++++++++ 2 files changed, 25 insertions(+), 10 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala index 7ec18b8419e20..61c26c50a6662 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 @@ -42,7 +42,7 @@ case class Sqrt(child: Expression) extends UnaryExpression { override def toString = s"SQRT($child)" override def eval(input: Row): Any = { - n1(child, input, ((na,a) => math.sqrt(na.toDouble(a)))) + n1(child, input, (na,a) => math.sqrt(na.toDouble(a))) } } @@ -138,7 +138,7 @@ case class BitwiseAnd(left: Expression, right: Expression) extends BinaryArithme case ShortType => (evalE1.asInstanceOf[Short] & evalE2.asInstanceOf[Short]).toShort case IntegerType => evalE1.asInstanceOf[Int] & evalE2.asInstanceOf[Int] case LongType => evalE1.asInstanceOf[Long] & evalE2.asInstanceOf[Long] - case other => sys.error(s"Unsupported bitwise & operation on ${other}") + case other => sys.error(s"Unsupported bitwise & operation on $other") } } @@ -153,7 +153,7 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet case ShortType => (evalE1.asInstanceOf[Short] | evalE2.asInstanceOf[Short]).toShort case IntegerType => evalE1.asInstanceOf[Int] | evalE2.asInstanceOf[Int] case LongType => evalE1.asInstanceOf[Long] | evalE2.asInstanceOf[Long] - case other => sys.error(s"Unsupported bitwise | operation on ${other}") + case other => sys.error(s"Unsupported bitwise | operation on $other") } } @@ -168,7 +168,7 @@ case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithme case ShortType => (evalE1.asInstanceOf[Short] ^ evalE2.asInstanceOf[Short]).toShort case IntegerType => evalE1.asInstanceOf[Int] ^ evalE2.asInstanceOf[Int] case LongType => evalE1.asInstanceOf[Long] ^ evalE2.asInstanceOf[Long] - case other => sys.error(s"Unsupported bitwise ^ operation on ${other}") + case other => sys.error(s"Unsupported bitwise ^ operation on $other") } } @@ -181,7 +181,7 @@ case class BitwiseNot(child: Expression) extends UnaryExpression { def dataType = child.dataType override def foldable = child.foldable def nullable = child.nullable - override def toString = s"-$child" + override def toString = s"~$child" override def eval(input: Row): Any = { val evalE = child.eval(input) @@ -189,11 +189,11 @@ case class BitwiseNot(child: Expression) extends UnaryExpression { null } else { dataType match { - case ByteType => (~(evalE.asInstanceOf[Byte])).toByte - case ShortType => (~(evalE.asInstanceOf[Short])).toShort - case IntegerType => ~(evalE.asInstanceOf[Int]) - case LongType => ~(evalE.asInstanceOf[Long]) - case other => sys.error(s"Unsupported bitwise ~ operation on ${other}") + case ByteType => (~evalE.asInstanceOf[Byte]).toByte + case ShortType => (~evalE.asInstanceOf[Short]).toShort + case IntegerType => ~evalE.asInstanceOf[Int] + case LongType => ~evalE.asInstanceOf[Long] + case other => sys.error(s"Unsupported bitwise ~ operation on $other") } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 25f56424888aa..cd2f67f448b0b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -42,6 +42,21 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(1) + Literal(1), 2) } + test("unary BitwiseNOT") { + checkEvaluation(BitwiseNot(1), -2) + assert(BitwiseNot(1).dataType === IntegerType) + assert(BitwiseNot(1).eval(EmptyRow).isInstanceOf[Int]) + checkEvaluation(BitwiseNot(1.toLong), -2.toLong) + assert(BitwiseNot(1.toLong).dataType === LongType) + assert(BitwiseNot(1.toLong).eval(EmptyRow).isInstanceOf[Long]) + checkEvaluation(BitwiseNot(1.toShort), -2.toShort) + assert(BitwiseNot(1.toShort).dataType === ShortType) + assert(BitwiseNot(1.toShort).eval(EmptyRow).isInstanceOf[Short]) + checkEvaluation(BitwiseNot(1.toByte), -2.toByte) + assert(BitwiseNot(1.toByte).dataType === ByteType) + assert(BitwiseNot(1.toByte).eval(EmptyRow).isInstanceOf[Byte]) + } + /** * Checks for three-valued-logic. Based on: * http://en.wikipedia.org/wiki/Null_(SQL)#Comparisons_with_NULL_and_the_three-valued_logic_.283VL.29 From 3ae0cda83c5106136e90d59c20e61db345a5085f Mon Sep 17 00:00:00 2001 From: wangfei Date: Tue, 2 Dec 2014 14:30:44 -0800 Subject: [PATCH 093/305] [SPARK-4695][SQL] Get result using executeCollect Using ```executeCollect``` to collect the result, because executeCollect is a custom implementation of collect in spark sql which better than rdd's collect Author: wangfei Closes #3547 from scwf/executeCollect and squashes the following commits: a5ab68e [wangfei] Revert "adding debug info" a60d680 [wangfei] fix test failure 0db7ce8 [wangfei] adding debug info 184c594 [wangfei] using executeCollect instead collect --- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala index 304b9a73ee91d..34fc21e61f60f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -377,7 +377,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { command.executeCollect().map(_.head.toString) case other => - val result: Seq[Seq[Any]] = toRdd.map(_.copy()).collect().toSeq + val result: Seq[Seq[Any]] = other.executeCollect().toSeq // We need the types so we can output struct field names val types = analyzed.output.map(_.dataType) // Reformat to match hive tab delimited output. @@ -416,6 +416,8 @@ object HiveContext { case (bin: Array[Byte], BinaryType) => new String(bin, "UTF-8") case (decimal: Decimal, DecimalType()) => // Hive strips trailing zeros so use its toString HiveShim.createDecimal(decimal.toBigDecimal.underlying()).toString + case (decimal: BigDecimal, DecimalType()) => + HiveShim.createDecimal(decimal.underlying()).toString case (other, tpe) if primitiveTypes contains tpe => other.toString } From 2d4f6e70f7de50489c2b5f0d6a4756c3b1aace7d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 2 Dec 2014 14:40:26 -0800 Subject: [PATCH 094/305] Minor nit style cleanup in GraphX. --- graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala index 1db3df03c8052..09ae3f9f6c09b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/VertexRDD.scala @@ -68,7 +68,7 @@ abstract class VertexRDD[VD]( * Provides the `RDD[(VertexId, VD)]` equivalent output. */ override def compute(part: Partition, context: TaskContext): Iterator[(VertexId, VD)] = { - firstParent[ShippableVertexPartition[VD]].iterator(part, context).next.iterator + firstParent[ShippableVertexPartition[VD]].iterator(part, context).next().iterator } /** From 5da21f07d862212067719ddaa2fef6e09db21c10 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 2 Dec 2014 16:36:12 -0800 Subject: [PATCH 095/305] [Release] Translate unknown author names automatically --- dev/create-release/generate-contributors.py | 36 ++++---- dev/create-release/releaseutils.py | 93 +++++++++++++++++++++ 2 files changed, 111 insertions(+), 18 deletions(-) diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py index f4bf734081583..99c29ef9ff8b6 100755 --- a/dev/create-release/generate-contributors.py +++ b/dev/create-release/generate-contributors.py @@ -26,23 +26,11 @@ # You must set the following before use! JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", None) +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", None) START_COMMIT = os.environ.get("START_COMMIT", "37b100") END_COMMIT = os.environ.get("END_COMMIT", "3693ae") -try: - from jira.client import JIRA -except ImportError: - print "This tool requires the jira-python library" - print "Install using 'sudo pip install jira-python'" - sys.exit(-1) - -try: - import unidecode -except ImportError: - print "This tool requires the unidecode library to decode obscure github usernames" - print "Install using 'sudo pip install unidecode'" - sys.exit(-1) - # If commit range is not specified, prompt the user to provide it if not START_COMMIT or not END_COMMIT: print "A commit range is required to proceed." @@ -52,6 +40,8 @@ END_COMMIT = raw_input("Please specify ending commit hash (non-inclusive): ") # Verify provided arguments +if not JIRA_USERNAME: sys.exit("JIRA_USERNAME must be provided") +if not JIRA_PASSWORD: sys.exit("JIRA_PASSWORD must be provided") start_commit_line = get_one_line(START_COMMIT) end_commit_line = get_one_line(END_COMMIT) num_commits = num_commits_in_range(START_COMMIT, END_COMMIT) @@ -70,6 +60,14 @@ sys.exit("Ok, exiting") print "==================================================================================\n" +# Setup JIRA and github clients. We use two JIRA clients, one with authentication +# and one without, because authentication is slow and required only when we query +# JIRA user details but not Spark issues +jira_options = { "server": JIRA_API_BASE } +jira_client = JIRA(options = jira_options) +jira_client_auth = JIRA(options = jira_options, basic_auth = (JIRA_USERNAME, JIRA_PASSWORD)) +github_client = Github() + # Find all commits within this range print "Gathering commits within range [%s..%s)" % (START_COMMIT, END_COMMIT) commits = get_one_line_commits(START_COMMIT, END_COMMIT) @@ -129,14 +127,16 @@ def print_indented(_list): # } # author_info = {} -jira_options = { "server": JIRA_API_BASE } -jira = JIRA(jira_options) print "\n=========================== Compiling contributor list ===========================" for commit in filtered_commits: commit_hash = re.findall("^[a-z0-9]+", commit)[0] issues = re.findall("SPARK-[0-9]+", commit.upper()) + # Translate the author in case the github username is not an actual name + # Also guard against any special characters used in the name + # Note the JIRA client we use here must have authentication enabled author = get_author(commit_hash) - author = unidecode.unidecode(unicode(author, "UTF-8")) # guard against special characters + author = unidecode.unidecode(unicode(author, "UTF-8")) + author = translate_author(author, github_client, jira_client_auth, warnings) date = get_date(commit_hash) # Parse components from the commit message, if any commit_components = find_components(commit, commit_hash) @@ -151,7 +151,7 @@ def populate(issue_type, components): author_info[author][issue_type].add(component) # Find issues and components associated with this commit for issue in issues: - jira_issue = jira.issue(issue) + jira_issue = jira_client.issue(issue) jira_type = jira_issue.fields.issuetype.name jira_type = translate_issue_type(jira_type, issue, warnings) jira_components = [translate_component(c.name, commit_hash, warnings)\ diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index e56d7fa58fa2c..0d6830b11dc73 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -21,6 +21,29 @@ import re from subprocess import Popen, PIPE +try: + from jira.client import JIRA + from jira.exceptions import JIRAError +except ImportError: + print "This tool requires the jira-python library" + print "Install using 'sudo pip install jira-python'" + sys.exit(-1) + +try: + from github import Github + from github import GithubException +except ImportError: + print "This tool requires the PyGithub library" + print "Install using 'sudo pip install PyGithub'" + sys.exit(-1) + +try: + import unidecode +except ImportError: + print "This tool requires the unidecode library to decode obscure github usernames" + print "Install using 'sudo pip install unidecode'" + sys.exit(-1) + # Utility functions run git commands (written with Git 1.8.5) def run_cmd(cmd): return Popen(cmd, stdout=PIPE).communicate()[0] def get_author(commit_hash): @@ -122,3 +145,73 @@ def nice_join(str_list): else: return ", ".join(str_list[:-1]) + ", and " + str_list[-1] +# Return the full name of the specified user on Github +# If the user doesn't exist, return None +def get_github_name(author, github_client): + if github_client: + try: + return github_client.get_user(author).name + except GithubException as e: + # If this is not a "not found" exception + if e.status != 404: + raise e + return None + +# Return the full name of the specified user on JIRA +# If the user doesn't exist, return None +def get_jira_name(author, jira_client): + if jira_client: + try: + return jira_client.user(author).displayName + except JIRAError as e: + # If this is not a "not found" exception + if e.status_code != 404: + raise e + return None + +# Return whether the given name is in the form +def is_valid_author(author): + if not author: return False + author_words = len(author.split(" ")) + return author_words == 2 or author_words == 3 + +# Capitalize the first letter of each word in the given author name +def capitalize_author(author): + if not author: return None + words = author.split(" ") + words = [w[0].capitalize() + w[1:] for w in words if w] + return " ".join(words) + +# Maintain a mapping of translated author names as a cache +translated_authors = {} + +# Format the given author in a format appropriate for the contributors list. +# If the author is not an actual name, search github and JIRA for potential +# replacements and log all candidates as a warning. +def translate_author(github_author, github_client, jira_client, warnings): + if is_valid_author(github_author): + return capitalize_author(github_author) + # If the translated author is already cached, just return it + if github_author in translated_authors: + return translated_authors[github_author] + # Otherwise, author name is not found, so we need to search for an alternative name + candidates = set() + github_name = get_github_name(github_author, github_client) + jira_name = get_jira_name(github_author, jira_client) + if is_valid_author(github_name): github_name = capitalize_author(github_name) + if is_valid_author(jira_name): jira_name = capitalize_author(jira_name) + if github_name: candidates.add(github_name) + if jira_name: candidates.add(jira_name) + # Only use the github name as a replacement automatically + # The JIRA name may not make sense because it can belong to someone else + if is_valid_author(github_name): + candidates_message = " (another candidate is %s)" % jira_name if jira_name else "" + warnings.append("Replacing github user %s with %s%s" % (github_author, github_name, candidates_message)) + translated_authors[github_name] = github_name + return translated_authors[github_name] + # No direct replacement, so return the original author and list any candidates found + candidates_message = " (candidates: %s)" % nice_join(candidates) if candidates else "" + warnings.append("Unable to find a replacement for github user %s%s" % (github_author, candidates_message)) + translated_authors[github_author] = github_author + return translated_authors[github_author] + From fc0a1475ef7c8b33363d88adfe8e8f28def5afc7 Mon Sep 17 00:00:00 2001 From: JerryLead Date: Tue, 2 Dec 2014 17:08:02 -0800 Subject: [PATCH 096/305] [SPARK-4672][GraphX]Perform checkpoint() on PartitionsRDD to shorten the lineage The related JIRA is https://issues.apache.org/jira/browse/SPARK-4672 Iterative GraphX applications always have long lineage, while checkpoint() on EdgeRDD and VertexRDD themselves cannot shorten the lineage. In contrast, if we perform checkpoint() on their ParitionsRDD, the long lineage can be cut off. Moreover, the existing operations such as cache() in this code is performed on the PartitionsRDD, so checkpoint() should do the same way. More details and explanation can be found in the JIRA. Author: JerryLead Author: Lijie Xu Closes #3549 from JerryLead/my_graphX_checkpoint and squashes the following commits: d1aa8d8 [JerryLead] Perform checkpoint() on PartitionsRDD not VertexRDD and EdgeRDD themselves ff08ed4 [JerryLead] Merge branch 'master' of https://github.com/apache/spark c0169da [JerryLead] Merge branch 'master' of https://github.com/apache/spark 52799e3 [Lijie Xu] Merge pull request #1 from apache/master --- .../main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala | 4 ++++ .../scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala index a8169613b4fd2..504559da977d8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala @@ -70,6 +70,10 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( this } + override def checkpoint() = { + partitionsRDD.checkpoint() + } + /** The number of edges in the RDD. */ override def count(): Long = { partitionsRDD.map(_._2.size.toLong).reduce(_ + _) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index d92a55a189298..c8898b1369565 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -71,6 +71,10 @@ class VertexRDDImpl[VD] private[graphx] ( this } + override def checkpoint() = { + partitionsRDD.checkpoint() + } + /** The number of vertices in the RDD. */ override def count(): Long = { partitionsRDD.map(_.size).reduce(_ + _) From 17c162f6682520e6e2790626e37da3a074471793 Mon Sep 17 00:00:00 2001 From: JerryLead Date: Tue, 2 Dec 2014 17:14:11 -0800 Subject: [PATCH 097/305] [SPARK-4672][GraphX]Non-transient PartitionsRDDs will lead to StackOverflow error The related JIRA is https://issues.apache.org/jira/browse/SPARK-4672 In a nutshell, if `val partitionsRDD` in EdgeRDDImpl and VertexRDDImpl are non-transient, the serialization chain can become very long in iterative algorithms and finally lead to the StackOverflow error. More details and explanation can be found in the JIRA. Author: JerryLead Author: Lijie Xu Closes #3544 from JerryLead/my_graphX and squashes the following commits: 628f33c [JerryLead] set PartitionsRDD to be transient in EdgeRDDImpl and VertexRDDImpl c0169da [JerryLead] Merge branch 'master' of https://github.com/apache/spark 52799e3 [Lijie Xu] Merge pull request #1 from apache/master --- .../main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala | 2 +- .../main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala index 504559da977d8..897c7ee12a436 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala @@ -26,7 +26,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx._ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( - override val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])], + @transient override val partitionsRDD: RDD[(PartitionID, EdgePartition[ED, VD])], val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) extends EdgeRDD[ED](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index c8898b1369565..9732c5b00c6d9 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -27,7 +27,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.graphx._ class VertexRDDImpl[VD] private[graphx] ( - val partitionsRDD: RDD[ShippableVertexPartition[VD]], + @transient val partitionsRDD: RDD[ShippableVertexPartition[VD]], val targetStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) (implicit override protected val vdTag: ClassTag[VD]) extends VertexRDD[VD](partitionsRDD.context, List(new OneToOneDependency(partitionsRDD))) { From 77be8b986fd21b7bbe28aa8db1042cb22bc74fe7 Mon Sep 17 00:00:00 2001 From: JerryLead Date: Tue, 2 Dec 2014 23:53:29 -0800 Subject: [PATCH 098/305] [SPARK-4672][Core]Checkpoint() should clear f to shorten the serialization chain The related JIRA is https://issues.apache.org/jira/browse/SPARK-4672 The f closure of `PartitionsRDD(ZippedPartitionsRDD2)` contains a `$outer` that references EdgeRDD/VertexRDD, which causes task's serialization chain become very long in iterative GraphX applications. As a result, StackOverflow error will occur. If we set "f = null" in `clearDependencies()`, checkpoint() can cut off the long serialization chain. More details and explanation can be found in the JIRA. Author: JerryLead Author: Lijie Xu Closes #3545 from JerryLead/my_core and squashes the following commits: f7faea5 [JerryLead] checkpoint() should clear the f to avoid StackOverflow error c0169da [JerryLead] Merge branch 'master' of https://github.com/apache/spark 52799e3 [Lijie Xu] Merge pull request #1 from apache/master --- .../scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index 996f2cd3f34a3..95b2dd954e9f4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -77,7 +77,7 @@ private[spark] abstract class ZippedPartitionsBaseRDD[V: ClassTag]( private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( sc: SparkContext, - f: (Iterator[A], Iterator[B]) => Iterator[V], + var f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], var rdd2: RDD[B], preservesPartitioning: Boolean = false) @@ -92,13 +92,14 @@ private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag] super.clearDependencies() rdd1 = null rdd2 = null + f = null } } private[spark] class ZippedPartitionsRDD3 [A: ClassTag, B: ClassTag, C: ClassTag, V: ClassTag]( sc: SparkContext, - f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], + var f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], var rdd1: RDD[A], var rdd2: RDD[B], var rdd3: RDD[C], @@ -117,13 +118,14 @@ private[spark] class ZippedPartitionsRDD3 rdd1 = null rdd2 = null rdd3 = null + f = null } } private[spark] class ZippedPartitionsRDD4 [A: ClassTag, B: ClassTag, C: ClassTag, D:ClassTag, V: ClassTag]( sc: SparkContext, - f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], + var f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], var rdd1: RDD[A], var rdd2: RDD[B], var rdd3: RDD[C], @@ -145,5 +147,6 @@ private[spark] class ZippedPartitionsRDD4 rdd2 = null rdd3 = null rdd4 = null + f = null } } From 8af551f71d4c192753daa52f005bde831eb92429 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 3 Dec 2014 02:05:17 -0800 Subject: [PATCH 099/305] [SPARK-4397][Core] Change the 'since' value of '@deprecated' to '1.3.0' As #3262 wasn't merged to branch 1.2, the `since` value of `deprecated` should be '1.3.0'. Author: zsxwing Closes #3573 from zsxwing/SPARK-4397-version and squashes the following commits: 1daa03c [zsxwing] Change the 'since' value to '1.3.0' --- .../scala/org/apache/spark/SparkContext.scala | 36 +++++++++---------- 1 file changed, 18 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 9b0d5be7a7ab2..532f292952f05 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1630,28 +1630,28 @@ object SparkContext extends Logging { // following ones. @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") object DoubleAccumulatorParam extends AccumulatorParam[Double] { def addInPlace(t1: Double, t2: Double): Double = t1 + t2 def zero(initialValue: Double) = 0.0 } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") object IntAccumulatorParam extends AccumulatorParam[Int] { def addInPlace(t1: Int, t2: Int): Int = t1 + t2 def zero(initialValue: Int) = 0 } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") object LongAccumulatorParam extends AccumulatorParam[Long] { def addInPlace(t1: Long, t2: Long) = t1 + t2 def zero(initialValue: Long) = 0L } @deprecated("Replaced by implicit objects in AccumulatorParam. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") object FloatAccumulatorParam extends AccumulatorParam[Float] { def addInPlace(t1: Float, t2: Float) = t1 + t2 def zero(initialValue: Float) = 0f @@ -1662,34 +1662,34 @@ object SparkContext extends Logging { // and just call the corresponding functions in `object RDD`. @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "kept here only for backward compatibility.", "1.3.0") def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { RDD.rddToPairRDDFunctions(rdd) } @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "kept here only for backward compatibility.", "1.3.0") def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = RDD.rddToAsyncRDDActions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "kept here only for backward compatibility.", "1.3.0") def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( rdd: RDD[(K, V)]) = RDD.rddToSequenceFileRDDFunctions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "kept here only for backward compatibility.", "1.3.0") def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( rdd: RDD[(K, V)]) = RDD.rddToOrderedRDDFunctions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "kept here only for backward compatibility.", "1.3.0") def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = RDD.doubleRDDToDoubleRDDFunctions(rdd) @deprecated("Replaced by implicit functions in the RDD companion object. This is " + - "kept here only for backward compatibility.", "1.2.0") + "kept here only for backward compatibility.", "1.3.0") def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = RDD.numericRDDToDoubleRDDFunctions(rdd) @@ -1722,42 +1722,42 @@ object SparkContext extends Logging { // and just call the corresponding functions in `object WritableConverter`. @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def intWritableConverter(): WritableConverter[Int] = WritableConverter.intWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def longWritableConverter(): WritableConverter[Long] = WritableConverter.longWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def doubleWritableConverter(): WritableConverter[Double] = WritableConverter.doubleWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def floatWritableConverter(): WritableConverter[Float] = WritableConverter.floatWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def booleanWritableConverter(): WritableConverter[Boolean] = WritableConverter.booleanWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def bytesWritableConverter(): WritableConverter[Array[Byte]] = WritableConverter.bytesWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def stringWritableConverter(): WritableConverter[String] = WritableConverter.stringWritableConverter() @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + - "backward compatibility.", "1.2.0") + "backward compatibility.", "1.3.0") def writableWritableConverter[T <: Writable]() = WritableConverter.writableWritableConverter() From 4ac21511547dc6227d05bf61821cd2d9ab5ede74 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 3 Dec 2014 18:50:03 +0800 Subject: [PATCH 100/305] [SPARK-4710] [mllib] Eliminate MLlib compilation warnings Renamed StreamingKMeans to StreamingKMeansExample to avoid warning about name conflict with StreamingKMeans class. Added import to DecisionTreeRunner to eliminate warning. CC: mengxr Author: Joseph K. Bradley Closes #3568 from jkbradley/ml-compilation-warnings and squashes the following commits: 64d6bc4 [Joseph K. Bradley] Updated DecisionTreeRunner.scala and StreamingKMeans.scala to eliminate compilation warnings, including renaming StreamingKMeans to StreamingKMeansExample. --- .../examples/mllib/DecisionTreeRunner.scala | 2 ++ ...KMeans.scala => StreamingKMeansExample.scala} | 16 ++++++++-------- 2 files changed, 10 insertions(+), 8 deletions(-) rename examples/src/main/scala/org/apache/spark/examples/mllib/{StreamingKMeans.scala => StreamingKMeansExample.scala} (90%) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 98f9d1689c8e7..54953adb5f3df 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -17,6 +17,8 @@ package org.apache.spark.examples.mllib +import scala.language.reflectiveCalls + import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala similarity index 90% rename from examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeans.scala rename to examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala index 33e5760aed997..8bb12d2ee9ed2 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/StreamingKMeansExample.scala @@ -17,10 +17,10 @@ package org.apache.spark.examples.mllib +import org.apache.spark.SparkConf +import org.apache.spark.mllib.clustering.StreamingKMeans import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.clustering.StreamingKMeans -import org.apache.spark.SparkConf import org.apache.spark.streaming.{Seconds, StreamingContext} /** @@ -36,28 +36,28 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} * `(y,[x1,x2,x3,...,xn])` * Where y is some identifier. n must be the same for train and test. * - * Usage: StreamingKmeans + * Usage: + * StreamingKMeansExample * * To run on your local machine using the two directories `trainingDir` and `testDir`, * with updates every 5 seconds, 2 dimensions per data point, and 3 clusters, call: - * $ bin/run-example \ - * org.apache.spark.examples.mllib.StreamingKMeans trainingDir testDir 5 3 2 + * $ bin/run-example mllib.StreamingKMeansExample trainingDir testDir 5 3 2 * * As you add text files to `trainingDir` the clusters will continuously update. * Anytime you add text files to `testDir`, you'll see predicted labels using the current model. * */ -object StreamingKMeans { +object StreamingKMeansExample { def main(args: Array[String]) { if (args.length != 5) { System.err.println( - "Usage: StreamingKMeans " + + "Usage: StreamingKMeansExample " + " ") System.exit(1) } - val conf = new SparkConf().setMaster("local").setAppName("StreamingLinearRegression") + val conf = new SparkConf().setMaster("local").setAppName("StreamingKMeansExample") val ssc = new StreamingContext(conf, Seconds(args(2).toLong)) val trainingData = ssc.textFileStream(args(0)).map(Vectors.parse) From 7fc49ed91168999d24ae7b4cc46fbb4ec87febc1 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Wed, 3 Dec 2014 19:01:56 +0800 Subject: [PATCH 101/305] [SPARK-4708][MLLib] Make k-mean runs two/three times faster with dense/sparse sample Note that the usage of `breezeSquaredDistance` in `org.apache.spark.mllib.util.MLUtils.fastSquaredDistance` is in the critical path, and `breezeSquaredDistance` is slow. We should replace it with our own implementation. Here is the benchmark against mnist8m dataset. Before DenseVector: 70.04secs SparseVector: 59.05secs With this PR DenseVector: 30.58secs SparseVector: 21.14secs Author: DB Tsai Closes #3565 from dbtsai/kmean and squashes the following commits: 08bc068 [DB Tsai] restyle de24662 [DB Tsai] address feedback b185a77 [DB Tsai] cleanup 4554ddd [DB Tsai] first commit --- .../spark/mllib/clustering/KMeans.scala | 67 +++++++++---------- .../spark/mllib/clustering/KMeansModel.scala | 10 +-- .../spark/mllib/clustering/LocalKMeans.scala | 22 +++--- .../org/apache/spark/mllib/util/MLUtils.scala | 26 ++++--- .../spark/mllib/util/MLUtilsSuite.scala | 13 ++-- 5 files changed, 70 insertions(+), 68 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 0f8dee58d8464..54c301d3e9e14 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -19,12 +19,11 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{DenseVector => BDV, Vector => BV} - import org.apache.spark.annotation.Experimental import org.apache.spark.Logging import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.BLAS.{axpy, scal} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -127,10 +126,10 @@ class KMeans private ( // Compute squared norms and cache them. val norms = data.map(Vectors.norm(_, 2.0)) norms.persist() - val breezeData = data.map(_.toBreeze).zip(norms).map { case (v, norm) => - new BreezeVectorWithNorm(v, norm) + val zippedData = data.zip(norms).map { case (v, norm) => + new VectorWithNorm(v, norm) } - val model = runBreeze(breezeData) + val model = runAlgorithm(zippedData) norms.unpersist() // Warn at the end of the run as well, for increased visibility. @@ -142,9 +141,9 @@ class KMeans private ( } /** - * Implementation of K-Means using breeze. + * Implementation of K-Means algorithm. */ - private def runBreeze(data: RDD[BreezeVectorWithNorm]): KMeansModel = { + private def runAlgorithm(data: RDD[VectorWithNorm]): KMeansModel = { val sc = data.sparkContext @@ -170,9 +169,10 @@ class KMeans private ( // Execute iterations of Lloyd's algorithm until all runs have converged while (iteration < maxIterations && !activeRuns.isEmpty) { - type WeightedPoint = (BV[Double], Long) - def mergeContribs(p1: WeightedPoint, p2: WeightedPoint): WeightedPoint = { - (p1._1 += p2._1, p1._2 + p2._2) + type WeightedPoint = (Vector, Long) + def mergeContribs(x: WeightedPoint, y: WeightedPoint): WeightedPoint = { + axpy(1.0, x._1, y._1) + (y._1, x._2 + y._2) } val activeCenters = activeRuns.map(r => centers(r)).toArray @@ -185,16 +185,17 @@ class KMeans private ( val thisActiveCenters = bcActiveCenters.value val runs = thisActiveCenters.length val k = thisActiveCenters(0).length - val dims = thisActiveCenters(0)(0).vector.length + val dims = thisActiveCenters(0)(0).vector.size - val sums = Array.fill(runs, k)(BDV.zeros[Double](dims).asInstanceOf[BV[Double]]) + val sums = Array.fill(runs, k)(Vectors.zeros(dims)) val counts = Array.fill(runs, k)(0L) points.foreach { point => (0 until runs).foreach { i => val (bestCenter, cost) = KMeans.findClosest(thisActiveCenters(i), point) costAccums(i) += cost - sums(i)(bestCenter) += point.vector + val sum = sums(i)(bestCenter) + axpy(1.0, point.vector, sum) counts(i)(bestCenter) += 1 } } @@ -212,8 +213,8 @@ class KMeans private ( while (j < k) { val (sum, count) = totalContribs((i, j)) if (count != 0) { - sum /= count.toDouble - val newCenter = new BreezeVectorWithNorm(sum) + scal(1.0 / count, sum) + val newCenter = new VectorWithNorm(sum) if (KMeans.fastSquaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { changed = true } @@ -245,18 +246,18 @@ class KMeans private ( logInfo(s"The cost for the best run is $minCost.") - new KMeansModel(centers(bestRun).map(c => Vectors.fromBreeze(c.vector))) + new KMeansModel(centers(bestRun).map(_.vector)) } /** * Initialize `runs` sets of cluster centers at random. */ - private def initRandom(data: RDD[BreezeVectorWithNorm]) - : Array[Array[BreezeVectorWithNorm]] = { + private def initRandom(data: RDD[VectorWithNorm]) + : Array[Array[VectorWithNorm]] = { // Sample all the cluster centers in one pass to avoid repeated scans val sample = data.takeSample(true, runs * k, new XORShiftRandom().nextInt()).toSeq Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).map { v => - new BreezeVectorWithNorm(v.vector.toDenseVector, v.norm) + new VectorWithNorm(Vectors.dense(v.vector.toArray), v.norm) }.toArray) } @@ -269,8 +270,8 @@ class KMeans private ( * * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. */ - private def initKMeansParallel(data: RDD[BreezeVectorWithNorm]) - : Array[Array[BreezeVectorWithNorm]] = { + private def initKMeansParallel(data: RDD[VectorWithNorm]) + : Array[Array[VectorWithNorm]] = { // Initialize each run's center to a random point val seed = new XORShiftRandom().nextInt() val sample = data.takeSample(true, runs, seed).toSeq @@ -376,8 +377,8 @@ object KMeans { * Returns the index of the closest center to the given point, as well as the squared distance. */ private[mllib] def findClosest( - centers: TraversableOnce[BreezeVectorWithNorm], - point: BreezeVectorWithNorm): (Int, Double) = { + centers: TraversableOnce[VectorWithNorm], + point: VectorWithNorm): (Int, Double) = { var bestDistance = Double.PositiveInfinity var bestIndex = 0 var i = 0 @@ -402,8 +403,8 @@ object KMeans { * Returns the K-means cost of a given point against the given cluster centers. */ private[mllib] def pointCost( - centers: TraversableOnce[BreezeVectorWithNorm], - point: BreezeVectorWithNorm): Double = + centers: TraversableOnce[VectorWithNorm], + point: VectorWithNorm): Double = findClosest(centers, point)._2 /** @@ -411,26 +412,24 @@ object KMeans { * [[org.apache.spark.mllib.util.MLUtils#fastSquaredDistance]]. */ private[clustering] def fastSquaredDistance( - v1: BreezeVectorWithNorm, - v2: BreezeVectorWithNorm): Double = { + v1: VectorWithNorm, + v2: VectorWithNorm): Double = { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } } /** - * A breeze vector with its norm for fast distance computation. + * A vector with its norm for fast distance computation. * * @see [[org.apache.spark.mllib.clustering.KMeans#fastSquaredDistance]] */ private[clustering] -class BreezeVectorWithNorm(val vector: BV[Double], val norm: Double) extends Serializable { - - def this(vector: BV[Double]) = this(vector, Vectors.norm(Vectors.fromBreeze(vector), 2.0)) +class VectorWithNorm(val vector: Vector, val norm: Double) extends Serializable { - def this(array: Array[Double]) = this(new BDV[Double](array)) + def this(vector: Vector) = this(vector, Vectors.norm(vector, 2.0)) - def this(v: Vector) = this(v.toBreeze) + def this(array: Array[Double]) = this(Vectors.dense(array)) /** Converts the vector to a dense vector. */ - def toDense = new BreezeVectorWithNorm(vector.toDenseVector, norm) + def toDense = new VectorWithNorm(Vectors.dense(vector.toArray), norm) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 12a3d91cd31a6..3b95a9e6936e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -32,14 +32,14 @@ class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable { /** Returns the cluster index that a given point belongs to. */ def predict(point: Vector): Int = { - KMeans.findClosest(clusterCentersWithNorm, new BreezeVectorWithNorm(point))._1 + KMeans.findClosest(clusterCentersWithNorm, new VectorWithNorm(point))._1 } /** Maps given points to their cluster indices. */ def predict(points: RDD[Vector]): RDD[Int] = { val centersWithNorm = clusterCentersWithNorm val bcCentersWithNorm = points.context.broadcast(centersWithNorm) - points.map(p => KMeans.findClosest(bcCentersWithNorm.value, new BreezeVectorWithNorm(p))._1) + points.map(p => KMeans.findClosest(bcCentersWithNorm.value, new VectorWithNorm(p))._1) } /** Maps given points to their cluster indices. */ @@ -53,9 +53,9 @@ class KMeansModel (val clusterCenters: Array[Vector]) extends Serializable { def computeCost(data: RDD[Vector]): Double = { val centersWithNorm = clusterCentersWithNorm val bcCentersWithNorm = data.context.broadcast(centersWithNorm) - data.map(p => KMeans.pointCost(bcCentersWithNorm.value, new BreezeVectorWithNorm(p))).sum() + data.map(p => KMeans.pointCost(bcCentersWithNorm.value, new VectorWithNorm(p))).sum() } - private def clusterCentersWithNorm: Iterable[BreezeVectorWithNorm] = - clusterCenters.map(new BreezeVectorWithNorm(_)) + private def clusterCentersWithNorm: Iterable[VectorWithNorm] = + clusterCenters.map(new VectorWithNorm(_)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index f0722d7c14a46..b2f140e1b1352 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -19,9 +19,9 @@ package org.apache.spark.mllib.clustering import scala.util.Random -import breeze.linalg.{Vector => BV, DenseVector => BDV, norm => breezeNorm} - import org.apache.spark.Logging +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.linalg.BLAS.{axpy, scal} /** * An utility object to run K-means locally. This is private to the ML package because it's used @@ -35,14 +35,14 @@ private[mllib] object LocalKMeans extends Logging { */ def kMeansPlusPlus( seed: Int, - points: Array[BreezeVectorWithNorm], + points: Array[VectorWithNorm], weights: Array[Double], k: Int, maxIterations: Int - ): Array[BreezeVectorWithNorm] = { + ): Array[VectorWithNorm] = { val rand = new Random(seed) - val dimensions = points(0).vector.length - val centers = new Array[BreezeVectorWithNorm](k) + val dimensions = points(0).vector.size + val centers = new Array[VectorWithNorm](k) // Initialize centers by sampling using the k-means++ procedure. centers(0) = pickWeighted(rand, points, weights).toDense @@ -75,14 +75,12 @@ private[mllib] object LocalKMeans extends Logging { while (moved && iteration < maxIterations) { moved = false val counts = Array.fill(k)(0.0) - val sums = Array.fill(k)( - BDV.zeros[Double](dimensions).asInstanceOf[BV[Double]] - ) + val sums = Array.fill(k)(Vectors.zeros(dimensions)) var i = 0 while (i < points.length) { val p = points(i) val index = KMeans.findClosest(centers, p)._1 - breeze.linalg.axpy(weights(i), p.vector, sums(index)) + axpy(weights(i), p.vector, sums(index)) counts(index) += weights(i) if (index != oldClosest(i)) { moved = true @@ -97,8 +95,8 @@ private[mllib] object LocalKMeans extends Logging { // Assign center to a random point centers(j) = points(rand.nextInt(points.length)).toDense } else { - sums(j) /= counts(j) - centers(j) = new BreezeVectorWithNorm(sums(j)) + scal(1.0 / counts(j), sums(j)) + centers(j) = new VectorWithNorm(sums(j)) } j += 1 } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 9353351af72a0..b0d05ae33e1b5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.util import scala.reflect.ClassTag -import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV, +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} import org.apache.spark.annotation.Experimental @@ -28,7 +28,8 @@ import org.apache.spark.rdd.RDD import org.apache.spark.rdd.PartitionwiseSampledRDD import org.apache.spark.util.random.BernoulliCellSampler import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.linalg.BLAS.dot import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream.DStream @@ -281,9 +282,9 @@ object MLUtils { * @return squared distance between v1 and v2 within the specified precision */ private[mllib] def fastSquaredDistance( - v1: BV[Double], + v1: Vector, norm1: Double, - v2: BV[Double], + v2: Vector, norm2: Double, precision: Double = 1e-6): Double = { val n = v1.size @@ -306,16 +307,19 @@ object MLUtils { */ val precisionBound1 = 2.0 * EPSILON * sumSquaredNorm / (normDiff * normDiff + EPSILON) if (precisionBound1 < precision) { - sqDist = sumSquaredNorm - 2.0 * v1.dot(v2) - } else if (v1.isInstanceOf[BSV[Double]] || v2.isInstanceOf[BSV[Double]]) { - val dot = v1.dot(v2) - sqDist = math.max(sumSquaredNorm - 2.0 * dot, 0.0) - val precisionBound2 = EPSILON * (sumSquaredNorm + 2.0 * math.abs(dot)) / (sqDist + EPSILON) + sqDist = sumSquaredNorm - 2.0 * dot(v1, v2) + } else if (v1.isInstanceOf[SparseVector] || v2.isInstanceOf[SparseVector]) { + val dotValue = dot(v1, v2) + sqDist = math.max(sumSquaredNorm - 2.0 * dotValue, 0.0) + val precisionBound2 = EPSILON * (sumSquaredNorm + 2.0 * math.abs(dotValue)) / + (sqDist + EPSILON) if (precisionBound2 > precision) { - sqDist = breezeSquaredDistance(v1, v2) + // TODO: breezeSquaredDistance is slow, + // so we should replace it with our own implementation. + sqDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze) } } else { - sqDist = breezeSquaredDistance(v1, v2) + sqDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze) } sqDist } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 88bc49cc61f94..df07987093fbf 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -44,18 +44,19 @@ class MLUtilsSuite extends FunSuite with MLlibTestSparkContext { test("fast squared distance") { val a = (30 to 0 by -1).map(math.pow(2.0, _)).toArray val n = a.length - val v1 = new BDV[Double](a) - val norm1 = breezeNorm(v1, 2.0) + val v1 = Vectors.dense(a) + val norm1 = Vectors.norm(v1, 2.0) val precision = 1e-6 for (m <- 0 until n) { val indices = (0 to m).toArray val values = indices.map(i => a(i)) - val v2 = new BSV[Double](indices, values, n) - val norm2 = breezeNorm(v2, 2.0) - val squaredDist = breezeSquaredDistance(v1, v2) + val v2 = Vectors.sparse(n, indices, values) + val norm2 = Vectors.norm(v2, 2.0) + val squaredDist = breezeSquaredDistance(v1.toBreeze, v2.toBreeze) val fastSquaredDist1 = fastSquaredDistance(v1, norm1, v2, norm2, precision) assert((fastSquaredDist1 - squaredDist) <= precision * squaredDist, s"failed with m = $m") - val fastSquaredDist2 = fastSquaredDistance(v1, norm1, v2.toDenseVector, norm2, precision) + val fastSquaredDist2 = + fastSquaredDistance(v1, norm1, Vectors.dense(v2.toArray), norm2, precision) assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m") } } From d00542987ed80635782dcc826fc0bdbf434fff10 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Wed, 3 Dec 2014 22:31:39 +0800 Subject: [PATCH 102/305] [SPARK-4717][MLlib] Optimize BLAS library to avoid de-reference multiple times in loop Have a local reference to `values` and `indices` array in the `Vector` object so JVM can locate the value with one operation call. See `SPARK-4581` for similar optimization, and the bytecode analysis. Author: DB Tsai Closes #3577 from dbtsai/blasopt and squashes the following commits: 62d38c4 [DB Tsai] formating 0316cef [DB Tsai] first commit --- .../org/apache/spark/mllib/linalg/BLAS.scala | 99 +++++++++++-------- 1 file changed, 60 insertions(+), 39 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index 89539e600f48c..8c4c9c6cf6ae2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -72,17 +72,21 @@ private[spark] object BLAS extends Serializable with Logging { * y += a * x */ private def axpy(a: Double, x: SparseVector, y: DenseVector): Unit = { - val nnz = x.indices.size + val xValues = x.values + val xIndices = x.indices + val yValues = y.values + val nnz = xIndices.size + if (a == 1.0) { var k = 0 while (k < nnz) { - y.values(x.indices(k)) += x.values(k) + yValues(xIndices(k)) += xValues(k) k += 1 } } else { var k = 0 while (k < nnz) { - y.values(x.indices(k)) += a * x.values(k) + yValues(xIndices(k)) += a * xValues(k) k += 1 } } @@ -119,11 +123,15 @@ private[spark] object BLAS extends Serializable with Logging { * dot(x, y) */ private def dot(x: SparseVector, y: DenseVector): Double = { - val nnz = x.indices.size + val xValues = x.values + val xIndices = x.indices + val yValues = y.values + val nnz = xIndices.size + var sum = 0.0 var k = 0 while (k < nnz) { - sum += x.values(k) * y.values(x.indices(k)) + sum += xValues(k) * yValues(xIndices(k)) k += 1 } sum @@ -133,19 +141,24 @@ private[spark] object BLAS extends Serializable with Logging { * dot(x, y) */ private def dot(x: SparseVector, y: SparseVector): Double = { + val xValues = x.values + val xIndices = x.indices + val yValues = y.values + val yIndices = y.indices + val nnzx = xIndices.size + val nnzy = yIndices.size + var kx = 0 - val nnzx = x.indices.size var ky = 0 - val nnzy = y.indices.size var sum = 0.0 // y catching x while (kx < nnzx && ky < nnzy) { - val ix = x.indices(kx) - while (ky < nnzy && y.indices(ky) < ix) { + val ix = xIndices(kx) + while (ky < nnzy && yIndices(ky) < ix) { ky += 1 } - if (ky < nnzy && y.indices(ky) == ix) { - sum += x.values(kx) * y.values(ky) + if (ky < nnzy && yIndices(ky) == ix) { + sum += xValues(kx) * yValues(ky) ky += 1 } kx += 1 @@ -163,21 +176,25 @@ private[spark] object BLAS extends Serializable with Logging { case dy: DenseVector => x match { case sx: SparseVector => + val sxIndices = sx.indices + val sxValues = sx.values + val dyValues = dy.values + val nnz = sxIndices.size + var i = 0 var k = 0 - val nnz = sx.indices.size while (k < nnz) { - val j = sx.indices(k) + val j = sxIndices(k) while (i < j) { - dy.values(i) = 0.0 + dyValues(i) = 0.0 i += 1 } - dy.values(i) = sx.values(k) + dyValues(i) = sxValues(k) i += 1 k += 1 } while (i < n) { - dy.values(i) = 0.0 + dyValues(i) = 0.0 i += 1 } case dx: DenseVector => @@ -311,6 +328,8 @@ private[spark] object BLAS extends Serializable with Logging { s"The columns of C don't match the columns of B. C: ${C.numCols}, A: $nB") val Avals = A.values + val Bvals = B.values + val Cvals = C.values val Arows = if (!transA) A.rowIndices else A.colPtrs val Acols = if (!transA) A.colPtrs else A.rowIndices @@ -327,11 +346,11 @@ private[spark] object BLAS extends Serializable with Logging { val indEnd = Arows(rowCounterForA + 1) var sum = 0.0 while (i < indEnd) { - sum += Avals(i) * B.values(Bstart + Acols(i)) + sum += Avals(i) * Bvals(Bstart + Acols(i)) i += 1 } val Cindex = Cstart + rowCounterForA - C.values(Cindex) = beta * C.values(Cindex) + sum * alpha + Cvals(Cindex) = beta * Cvals(Cindex) + sum * alpha rowCounterForA += 1 } colCounterForB += 1 @@ -349,7 +368,7 @@ private[spark] object BLAS extends Serializable with Logging { i += 1 } val Cindex = Cstart + rowCounter - C.values(Cindex) = beta * C.values(Cindex) + sum * alpha + Cvals(Cindex) = beta * Cvals(Cindex) + sum * alpha rowCounter += 1 } colCounterForB += 1 @@ -357,7 +376,7 @@ private[spark] object BLAS extends Serializable with Logging { } } else { // Scale matrix first if `beta` is not equal to 0.0 - if (beta != 0.0){ + if (beta != 0.0) { f2jBLAS.dscal(C.values.length, beta, C.values, 1) } // Perform matrix multiplication and add to C. The rows of A are multiplied by the columns of @@ -371,9 +390,9 @@ private[spark] object BLAS extends Serializable with Logging { while (colCounterForA < kA) { var i = Acols(colCounterForA) val indEnd = Acols(colCounterForA + 1) - val Bval = B.values(Bstart + colCounterForA) * alpha - while (i < indEnd){ - C.values(Cstart + Arows(i)) += Avals(i) * Bval + val Bval = Bvals(Bstart + colCounterForA) * alpha + while (i < indEnd) { + Cvals(Cstart + Arows(i)) += Avals(i) * Bval i += 1 } colCounterForA += 1 @@ -384,12 +403,12 @@ private[spark] object BLAS extends Serializable with Logging { while (colCounterForB < nB) { var colCounterForA = 0 // The column of A to multiply with the row of B val Cstart = colCounterForB * mA - while (colCounterForA < kA){ + while (colCounterForA < kA) { var i = Acols(colCounterForA) val indEnd = Acols(colCounterForA + 1) val Bval = B(colCounterForB, colCounterForA) * alpha - while (i < indEnd){ - C.values(Cstart + Arows(i)) += Avals(i) * Bval + while (i < indEnd) { + Cvals(Cstart + Arows(i)) += Avals(i) * Bval i += 1 } colCounterForA += 1 @@ -484,41 +503,43 @@ private[spark] object BLAS extends Serializable with Logging { beta: Double, y: DenseVector): Unit = { - val mA: Int = if(!trans) A.numRows else A.numCols - val nA: Int = if(!trans) A.numCols else A.numRows + val xValues = x.values + val yValues = y.values + + val mA: Int = if (!trans) A.numRows else A.numCols + val nA: Int = if (!trans) A.numCols else A.numRows val Avals = A.values val Arows = if (!trans) A.rowIndices else A.colPtrs val Acols = if (!trans) A.colPtrs else A.rowIndices - // Slicing is easy in this case. This is the optimal multiplication setting for sparse matrices - if (trans){ + if (trans) { var rowCounter = 0 - while (rowCounter < mA){ + while (rowCounter < mA) { var i = Arows(rowCounter) val indEnd = Arows(rowCounter + 1) var sum = 0.0 - while(i < indEnd){ - sum += Avals(i) * x.values(Acols(i)) + while (i < indEnd) { + sum += Avals(i) * xValues(Acols(i)) i += 1 } - y.values(rowCounter) = beta * y.values(rowCounter) + sum * alpha + yValues(rowCounter) = beta * yValues(rowCounter) + sum * alpha rowCounter += 1 } } else { // Scale vector first if `beta` is not equal to 0.0 - if (beta != 0.0){ + if (beta != 0.0) { scal(beta, y) } // Perform matrix-vector multiplication and add to y var colCounterForA = 0 - while (colCounterForA < nA){ + while (colCounterForA < nA) { var i = Acols(colCounterForA) val indEnd = Acols(colCounterForA + 1) - val xVal = x.values(colCounterForA) * alpha - while (i < indEnd){ + val xVal = xValues(colCounterForA) * alpha + while (i < indEnd) { val rowIndex = Arows(i) - y.values(rowIndex) += Avals(i) * xVal + yValues(rowIndex) += Avals(i) * xVal i += 1 } colCounterForA += 1 From a975dc32799bb8a14f9e1c76defaaa7cfbaf8b53 Mon Sep 17 00:00:00 2001 From: Jim Lim Date: Wed, 3 Dec 2014 11:16:02 -0800 Subject: [PATCH 103/305] SPARK-2624 add datanucleus jars to the container in yarn-cluster If `spark-submit` finds the datanucleus jars, it adds them to the driver's classpath, but does not add it to the container. This patch modifies the yarn deployment class to copy all `datanucleus-*` jars found in `[spark-home]/libs` to the container. Author: Jim Lim Closes #3238 from jimjh/SPARK-2624 and squashes the following commits: 3633071 [Jim Lim] SPARK-2624 update documentation and comments fe95125 [Jim Lim] SPARK-2624 keep java imports together 6c31fe0 [Jim Lim] SPARK-2624 update documentation 6690fbf [Jim Lim] SPARK-2624 add tests d28d8e9 [Jim Lim] SPARK-2624 add spark.yarn.datanucleus.dir option 84e6cba [Jim Lim] SPARK-2624 add datanucleus jars to the container in yarn-cluster --- docs/running-on-yarn.md | 15 ++++ .../apache/spark/deploy/yarn/ClientBase.scala | 66 ++++++++++++++++ .../spark/deploy/yarn/ClientBaseSuite.scala | 76 +++++++++++++++++++ 3 files changed, 157 insertions(+) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index dfe2db4b3fce8..45e219e0c136c 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -132,6 +132,21 @@ Most of the configs are the same for Spark on YARN as for other deployment modes The maximum number of threads to use in the application master for launching executor containers. + + + + +
spark.sql.parquet.binaryAsString false - Some other Parquet-producing systems, in particular Impala and older versions of Spark SQL, do - not differentiate between binary data and strings when writing out the Parquet schema. This + Some other Parquet-producing systems, in particular Impala and older versions of Spark SQL, do + not differentiate between binary data and strings when writing out the Parquet schema. This flag tells Spark SQL to interpret binary data as a string to provide compatibility with these systems.
spark.sql.parquet.compression.codec gzip - Sets the compression codec use when writing Parquet files. Acceptable values include: + Sets the compression codec use when writing Parquet files. Acceptable values include: uncompressed, snappy, gzip, lzo.
spark.sql.parquet.filterPushdownfalse + Turn on Parquet filter pushdown optimization. This feature is turned off by default because of a known + bug in Paruet 1.6.0rc3 (PARQUET-136). + However, if your table doesn't contain any nullable string or binary columns, it's still safe to turn + this feature on. +
spark.sql.hive.convertMetastoreParquet true
DateType java.sql.Date + DateType +
ArrayType scala.collection.Seq
DateType java.sql.Date + DataType.DateType +
ArrayType java.util.List
DateType datetime.date + DateType() +
ArrayType list, tuple, or array (none) The cluster manager to connect to. See the list of - allowed master URL's. + allowed master URL's.
spark.yarn.datanucleus.dir$SPARK_HOME/lib + The location of the DataNucleus jars, in case overriding the default location is desired. + By default, Spark on YARN will use the DataNucleus jars installed at + $SPARK_HOME/lib, but the jars can also be in a world-readable location on HDFS. + This allows YARN to cache it on nodes so that it doesn't need to be distributed each time an + application runs. To point to a directory on HDFS, for example, set this configuration to + "hdfs:///some/path". + + This is required because the datanucleus jars cannot be packaged into the + assembly jar due to metadata conflicts (involving plugin.xml.) +
# Launching Spark on YARN diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index f95d72379171c..8e4360ea4476b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -18,6 +18,7 @@ package org.apache.spark.deploy.yarn import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} +import java.io.{File, FilenameFilter} import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, ListBuffer, Map} @@ -223,10 +224,48 @@ private[spark] trait ClientBase extends Logging { } } } + if (cachedSecondaryJarLinks.nonEmpty) { sparkConf.set(CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) } + /** + * Do the same for datanucleus jars, if they exist in spark home. Find all datanucleus-* jars, + * copy them to the remote fs, and add them to the class path. + * + * This is necessary because the datanucleus jars cannot be included in the assembly jar due + * to metadata conflicts involving plugin.xml. At the time of writing, these are the only + * jars that cannot be distributed with the uber jar and have to be treated differently. + * + * For more details, see SPARK-2624, and https://github.com/apache/spark/pull/3238 + */ + for (libsDir <- dataNucleusJarsDir(sparkConf)) { + val libsURI = new URI(libsDir) + val jarLinks = ListBuffer.empty[String] + if (libsURI.getScheme != LOCAL_SCHEME) { + val localURI = getQualifiedLocalPath(libsURI).toUri() + val jars = FileSystem.get(localURI, hadoopConf).listFiles(new Path(localURI.getPath), false) + while (jars.hasNext) { + val jar = jars.next() + val name = jar.getPath.getName + if (name.startsWith("datanucleus-")) { + // copy to remote and add to classpath + val src = jar.getPath + val destPath = copyFileToRemote(dst, src, replication) + distCacheMgr.addResource(fs, hadoopConf, destPath, + localResources, LocalResourceType.FILE, name, statCache) + jarLinks += name + } + } + } else { + jarLinks += libsURI.toString + Path.SEPARATOR + "*" + } + + if (jarLinks.nonEmpty) { + sparkConf.set(CONF_SPARK_DATANUCLEUS_JARS, jarLinks.mkString(",")) + } + } + localResources } @@ -551,6 +590,13 @@ private[spark] object ClientBase extends Logging { // Internal config to propagate the location of the user's jar to the driver/executors val CONF_SPARK_USER_JAR = "spark.yarn.user.jar" + // Location of the datanucleus jars + val CONF_SPARK_DATANUCLEUS_DIR = "spark.yarn.datanucleus.dir" + + // Internal config to propagate the locations of datanucleus jars found to add to the + // classpath of the executors. Value should be a comma-separated list of paths to each jar. + val CONF_SPARK_DATANUCLEUS_JARS = "spark.yarn.datanucleus.jars" + // Internal config to propagate the locations of any extra jars to add to the classpath // of the executors val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" @@ -583,6 +629,19 @@ private[spark] object ClientBase extends Logging { } } + /** + * Find the user-defined provided jars directory if configured, or return SPARK_HOME/lib if not. + * + * This method first looks for $CONF_SPARK_DATANUCLEUS_DIR inside the SparkConf, then looks for + * Spark home inside the the SparkConf and the user environment. + */ + private def dataNucleusJarsDir(conf: SparkConf): Option[String] = { + conf.getOption(CONF_SPARK_DATANUCLEUS_DIR).orElse { + val sparkHome = conf.getOption("spark.home").orElse(sys.env.get("SPARK_HOME")) + sparkHome.map(path => path + Path.SEPARATOR + "lib") + } + } + /** * Return the path to the given application's staging directory. */ @@ -684,6 +743,13 @@ private[spark] object ClientBase extends Logging { addUserClasspath(args, sparkConf, env) } + // Add datanucleus jars to classpath + for (entries <- sparkConf.getOption(CONF_SPARK_DATANUCLEUS_JARS)) { + entries.split(",").filter(_.nonEmpty).foreach { entry => + addFileToClasspath(entry, null, env) + } + } + // Append all jar files under the working directory to the classpath. addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env) } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index 17b79ae1d82c4..b055e9b72dc61 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -21,6 +21,7 @@ import java.io.File import java.net.URI import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.yarn.api.ApplicationConstants.Environment @@ -104,6 +105,81 @@ class ClientBaseSuite extends FunSuite with Matchers { cp should not contain (ClientBase.APP_JAR) } + test("DataNucleus in classpath") { + val dnJars = "local:/dn/core.jar,/dn/api.jar" + val conf = new Configuration() + val sparkConf = new SparkConf() + .set(ClientBase.CONF_SPARK_JAR, SPARK) + .set(ClientBase.CONF_SPARK_DATANUCLEUS_JARS, dnJars) + val env = new MutableHashMap[String, String]() + val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) + + ClientBase.populateClasspath(args, conf, sparkConf, env) + + val cp = env("CLASSPATH").split(File.pathSeparator) + s"$dnJars".split(",").foreach({ entry => + val uri = new URI(entry) + if (ClientBase.LOCAL_SCHEME.equals(uri.getScheme())) { + cp should contain (uri.getPath()) + } else { + cp should not contain (uri.getPath()) + } + }) + } + + test("DataNucleus using local:") { + val dnDir = "local:/datanucleus" + val conf = new Configuration() + val sparkConf = new SparkConf() + .set(ClientBase.CONF_SPARK_JAR, SPARK) + .set(ClientBase.CONF_SPARK_DATANUCLEUS_DIR, dnDir) + val yarnConf = new YarnConfiguration() + val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) + + val client = spy(new DummyClient(args, conf, sparkConf, yarnConf)) + doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), + any(classOf[Path]), anyShort(), anyBoolean()) + + val tempDir = Utils.createTempDir() + try { + client.prepareLocalResources(tempDir.getAbsolutePath()) + val jars = sparkConf.get(ClientBase.CONF_SPARK_DATANUCLEUS_JARS).split(",") + val uri = new URI(dnDir) + jars should contain (uri.toString + Path.SEPARATOR + "*") + } finally { + Utils.deleteRecursively(tempDir) + } + } + + test("DataNucleus using file:") { + val dnDir = Utils.createTempDir() + val tempDir = Utils.createTempDir() + + try { + // create mock datanucleus jar + val tempJar = File.createTempFile("datanucleus-", null, dnDir) + + val conf = new Configuration() + val sparkConf = new SparkConf() + .set(ClientBase.CONF_SPARK_JAR, SPARK) + .set(ClientBase.CONF_SPARK_DATANUCLEUS_DIR, dnDir.toURI.toString) + val yarnConf = new YarnConfiguration() + val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) + + val client = spy(new DummyClient(args, conf, sparkConf, yarnConf)) + doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), + any(classOf[Path]), anyShort(), anyBoolean()) + + client.prepareLocalResources(tempDir.getAbsolutePath()) + + val jars = sparkConf.get(ClientBase.CONF_SPARK_DATANUCLEUS_JARS).split(",") + jars should contain (tempJar.getName) + } finally { + Utils.deleteRecursively(dnDir) + Utils.deleteRecursively(tempDir) + } + } + test("Jar path propagation through SparkConf") { val conf = new Configuration() val sparkConf = new SparkConf().set(ClientBase.CONF_SPARK_JAR, SPARK) From 96786e3ee53a13a57463b74bec0e77b172f719a3 Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Wed, 3 Dec 2014 12:08:00 -0800 Subject: [PATCH 104/305] [SPARK-4701] Typo in sbt/sbt Modified typo. Author: Masayoshi TSUZUKI Closes #3560 from tsudukim/feature/SPARK-4701 and squashes the following commits: ed2a3f1 [Masayoshi TSUZUKI] Another whitespace position error. 1af3a35 [Masayoshi TSUZUKI] [SPARK-4701] Typo in sbt/sbt --- sbt/sbt | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/sbt/sbt b/sbt/sbt index c172fa74bc771..0a251d97db95c 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -71,8 +71,8 @@ Usage: $script_name [options] -Dkey=val pass -Dkey=val directly to the java runtime -J-X pass option -X directly to the java runtime (-J is stripped) - -S-X add -X to sbt's scalacOptions (-J is stripped) - -PmavenProfiles Enable a maven profile for the build. + -S-X add -X to sbt's scalacOptions (-S is stripped) + -PmavenProfiles Enable a maven profile for the build. In the case of duplicated or conflicting options, the order above shows precedence: JAVA_OPTS lowest, command line options highest. From edd3cd477c9d6016bd977c2fa692fdeff5a6e198 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 3 Dec 2014 12:19:40 -0800 Subject: [PATCH 105/305] [SPARK-4715][Core] Make sure tryToAcquire won't return a negative value ShuffleMemoryManager.tryToAcquire may return a negative value. The unit test demonstrates this bug. It will output `0 did not equal -200 granted is negative`. Author: zsxwing Closes #3575 from zsxwing/SPARK-4715 and squashes the following commits: a193ae6 [zsxwing] Make sure tryToAcquire won't return a negative value --- .../spark/shuffle/ShuffleMemoryManager.scala | 5 +++-- .../shuffle/ShuffleMemoryManagerSuite.scala | 17 ++++++++++++++++- 2 files changed, 19 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala index ee91a368b76ea..3bcc7178a3d8b 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleMemoryManager.scala @@ -66,8 +66,9 @@ private[spark] class ShuffleMemoryManager(maxMemory: Long) extends Logging { val curMem = threadMemory(threadId) val freeMemory = maxMemory - threadMemory.values.sum - // How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads - val maxToGrant = math.min(numBytes, (maxMemory / numActiveThreads) - curMem) + // How much we can grant this thread; don't let it grow to more than 1 / numActiveThreads; + // don't let it be negative + val maxToGrant = math.min(numBytes, math.max(0, (maxMemory / numActiveThreads) - curMem)) if (curMem < maxMemory / (2 * numActiveThreads)) { // We want to let each thread get at least 1 / (2 * numActiveThreads) before blocking; diff --git a/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala index d31bc22ee74f7..e0e646f0a3652 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/ShuffleMemoryManagerSuite.scala @@ -159,7 +159,7 @@ class ShuffleMemoryManagerSuite extends FunSuite with Timeouts { test("threads can block to get at least 1 / 2N memory") { // t1 grabs 1000 bytes and then waits until t2 is ready to make a request. It sleeps - // for a bit and releases 250 bytes, which should then be greanted to t2. Further requests + // for a bit and releases 250 bytes, which should then be granted to t2. Further requests // by t2 will return false right away because it now has 1 / 2N of the memory. val manager = new ShuffleMemoryManager(1000L) @@ -291,4 +291,19 @@ class ShuffleMemoryManagerSuite extends FunSuite with Timeouts { assert(state.t2WaitTime > 200, s"t2 waited less than 200 ms (${state.t2WaitTime})") } } + + test("threads should not be granted a negative size") { + val manager = new ShuffleMemoryManager(1000L) + manager.tryToAcquire(700L) + + val latch = new CountDownLatch(1) + startThread("t1") { + manager.tryToAcquire(300L) + latch.countDown() + } + latch.await() // Wait until `t1` calls `tryToAcquire` + + val granted = manager.tryToAcquire(300L) + assert(0 === granted, "granted is negative") + } } From 692f49378f7d384d5c9c5ab7451a1c1e66f91c50 Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Wed, 3 Dec 2014 13:16:24 -0800 Subject: [PATCH 106/305] [SPARK-4642] Add description about spark.yarn.queue to running-on-YARN document. Added descriptions about these parameters. - spark.yarn.queue Modified description about the defalut value of this parameter. - spark.yarn.submit.file.replication Author: Masayoshi TSUZUKI Closes #3500 from tsudukim/feature/SPARK-4642 and squashes the following commits: ce99655 [Masayoshi TSUZUKI] better gramatically. 21cf624 [Masayoshi TSUZUKI] Removed intentionally undocumented properties. 88cac9b [Masayoshi TSUZUKI] [SPARK-4642] Documents about running-on-YARN needs update --- docs/running-on-yarn.md | 9 ++++++++- 1 file changed, 8 insertions(+), 1 deletion(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 45e219e0c136c..e97ac9f0c4a00 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -30,7 +30,7 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.submit.file.replication - 3 + The default HDFS replication (usually 3) HDFS replication level for the files uploaded into HDFS for the application. These include things like the Spark jar, the app jar, and any distributed cache files/archives. @@ -91,6 +91,13 @@ Most of the configs are the same for Spark on YARN as for other deployment modes The amount of off heap memory (in megabytes) to be allocated per driver. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the container size (typically 6-10%). + + spark.yarn.queue + default + + The name of the YARN queue to which the application is submitted. + + spark.yarn.jar (none) From 90ec643e9af4c8bbb9000edca08c07afb17939c7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 3 Dec 2014 13:56:23 -0800 Subject: [PATCH 107/305] [HOT FIX] [YARN] Check whether `/lib` exists before listing its files This is caused by a975dc32799bb8a14f9e1c76defaaa7cfbaf8b53 Author: Andrew Or Closes #3589 from andrewor14/yarn-hot-fix and squashes the following commits: a4fad5f [Andrew Or] Check whether lib directory exists before listing its files --- .../apache/spark/deploy/yarn/ClientBase.scala | 27 ++++++++++--------- 1 file changed, 15 insertions(+), 12 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 8e4360ea4476b..290d9943a5077 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -243,18 +243,21 @@ private[spark] trait ClientBase extends Logging { val libsURI = new URI(libsDir) val jarLinks = ListBuffer.empty[String] if (libsURI.getScheme != LOCAL_SCHEME) { - val localURI = getQualifiedLocalPath(libsURI).toUri() - val jars = FileSystem.get(localURI, hadoopConf).listFiles(new Path(localURI.getPath), false) - while (jars.hasNext) { - val jar = jars.next() - val name = jar.getPath.getName - if (name.startsWith("datanucleus-")) { - // copy to remote and add to classpath - val src = jar.getPath - val destPath = copyFileToRemote(dst, src, replication) - distCacheMgr.addResource(fs, hadoopConf, destPath, - localResources, LocalResourceType.FILE, name, statCache) - jarLinks += name + val localPath = getQualifiedLocalPath(libsURI) + val localFs = FileSystem.get(localPath.toUri, hadoopConf) + if (localFs.exists(localPath)) { + val jars = localFs.listFiles(localPath, /* recursive */ false) + while (jars.hasNext) { + val jar = jars.next() + val name = jar.getPath.getName + if (name.startsWith("datanucleus-")) { + // copy to remote and add to classpath + val src = jar.getPath + val destPath = copyFileToRemote(dst, src, replication) + distCacheMgr.addResource(localFs, hadoopConf, destPath, + localResources, LocalResourceType.FILE, name, statCache) + jarLinks += name + } } } } else { From 513ef82e85661552e596d0b483b645ac24e86d4d Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 3 Dec 2014 14:13:35 -0800 Subject: [PATCH 108/305] [SPARK-4552][SQL] Avoid exception when reading empty parquet data through Hive This is a very small fix that catches one specific exception and returns an empty table. #3441 will address this in a more principled way. Author: Michael Armbrust Closes #3586 from marmbrus/fixEmptyParquet and squashes the following commits: 2781d9f [Michael Armbrust] Handle empty lists for newParquet 04dd376 [Michael Armbrust] Avoid exception when reading empty parquet data through Hive --- .../apache/spark/sql/parquet/newParquet.scala | 5 +- .../spark/sql/hive/HiveStrategies.scala | 96 ++++++++++--------- .../spark/sql/parquet/parquetSuites.scala | 6 ++ 3 files changed, 62 insertions(+), 45 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 9b89c3bfb3307..14f8659f15b3f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -191,7 +191,10 @@ case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext) val selectedPartitions = partitions.filter(p => partitionFilters.forall(_(p))) val fs = FileSystem.get(new java.net.URI(path), sparkContext.hadoopConfiguration) val selectedFiles = selectedPartitions.flatMap(_.files).map(f => fs.makeQualified(f.getPath)) - org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(job, selectedFiles:_*) + // FileInputFormat cannot handle empty lists. + if (selectedFiles.nonEmpty) { + org.apache.hadoop.mapreduce.lib.input.FileInputFormat.setInputPaths(job, selectedFiles: _*) + } // Push down filters when possible predicates 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 56fc85239e1c0..edf291f917f07 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 @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.types.StringType -import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan} +import org.apache.spark.sql.execution.{DescribeCommand, OutputFaker, SparkPlan, PhysicalRDD} import org.apache.spark.sql.hive import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation @@ -104,53 +104,61 @@ private[hive] trait HiveStrategies { case a: AttributeReference => UnresolvedAttribute(a.name) }) - if (relation.hiveQlTable.isPartitioned) { - val rawPredicate = pruningPredicates.reduceOption(And).getOrElse(Literal(true)) - // Translate the predicate so that it automatically casts the input values to the correct - // data types during evaluation - val castedPredicate = rawPredicate transform { - case a: AttributeReference => - val idx = relation.partitionKeys.indexWhere(a.exprId == _.exprId) - val key = relation.partitionKeys(idx) - Cast(BoundReference(idx, StringType, nullable = true), key.dataType) - } - - val inputData = new GenericMutableRow(relation.partitionKeys.size) - val pruningCondition = - if(codegenEnabled) { - GeneratePredicate(castedPredicate) - } else { - InterpretedPredicate(castedPredicate) + try { + if (relation.hiveQlTable.isPartitioned) { + val rawPredicate = pruningPredicates.reduceOption(And).getOrElse(Literal(true)) + // Translate the predicate so that it automatically casts the input values to the + // correct data types during evaluation. + val castedPredicate = rawPredicate transform { + case a: AttributeReference => + val idx = relation.partitionKeys.indexWhere(a.exprId == _.exprId) + val key = relation.partitionKeys(idx) + Cast(BoundReference(idx, StringType, nullable = true), key.dataType) } - val partitions = relation.hiveQlPartitions.filter { part => - val partitionValues = part.getValues - var i = 0 - while (i < partitionValues.size()) { - inputData(i) = partitionValues(i) - i += 1 + val inputData = new GenericMutableRow(relation.partitionKeys.size) + val pruningCondition = + if (codegenEnabled) { + GeneratePredicate(castedPredicate) + } else { + InterpretedPredicate(castedPredicate) + } + + val partitions = relation.hiveQlPartitions.filter { part => + val partitionValues = part.getValues + var i = 0 + while (i < partitionValues.size()) { + inputData(i) = partitionValues(i) + i += 1 + } + pruningCondition(inputData) } - pruningCondition(inputData) - } - hiveContext - .parquetFile(partitions.map(_.getLocation).mkString(",")) - .addPartitioningAttributes(relation.partitionKeys) - .lowerCase - .where(unresolvedOtherPredicates) - .select(unresolvedProjection:_*) - .queryExecution - .executedPlan - .fakeOutput(projectList.map(_.toAttribute)):: Nil - } else { - hiveContext - .parquetFile(relation.hiveQlTable.getDataLocation.toString) - .lowerCase - .where(unresolvedOtherPredicates) - .select(unresolvedProjection:_*) - .queryExecution - .executedPlan - .fakeOutput(projectList.map(_.toAttribute)) :: Nil + hiveContext + .parquetFile(partitions.map(_.getLocation).mkString(",")) + .addPartitioningAttributes(relation.partitionKeys) + .lowerCase + .where(unresolvedOtherPredicates) + .select(unresolvedProjection: _*) + .queryExecution + .executedPlan + .fakeOutput(projectList.map(_.toAttribute)) :: Nil + } else { + hiveContext + .parquetFile(relation.hiveQlTable.getDataLocation.toString) + .lowerCase + .where(unresolvedOtherPredicates) + .select(unresolvedProjection: _*) + .queryExecution + .executedPlan + .fakeOutput(projectList.map(_.toAttribute)) :: Nil + } + } catch { + // parquetFile will throw an exception when there is no data. + // TODO: Remove this hack for Spark 1.3. + case iae: java.lang.IllegalArgumentException + if iae.getMessage.contains("Can not create a Path from an empty string") => + PhysicalRDD(plan.output, sparkContext.emptyRDD[Row]) :: Nil } case _ => Nil } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala index 7159ebd0353ad..488ebba043794 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/parquetSuites.scala @@ -218,6 +218,12 @@ abstract class ParquetTest extends QueryTest with BeforeAndAfterAll { 10) } + test(s"non-existant partition $table") { + checkAnswer( + sql(s"SELECT COUNT(*) FROM $table WHERE p = 1000"), + 0) + } + test(s"multi-partition pruned count $table") { checkAnswer( sql(s"SELECT COUNT(*) FROM $table WHERE p IN (1,2,3)"), From 96b27855c5f9789d1f15316564a8e0fa2cd5a51b Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Wed, 3 Dec 2014 15:08:01 -0800 Subject: [PATCH 109/305] [SPARK-4498][core] Don't transition ExecutorInfo to RUNNING until Driver adds Executor The ExecutorInfo only reaches the RUNNING state if the Driver is alive to send the ExecutorStateChanged message to master. Else, appInfo.resetRetryCount() is never called and failing Executors will eventually exceed ApplicationState.MAX_NUM_RETRY, resulting in the application being removed from the master's accounting. JoshRosen Author: Mark Hamstra Closes #3550 from markhamstra/SPARK-4498 and squashes the following commits: 8f543b1 [Mark Hamstra] Don't transition ExecutorInfo to RUNNING until Executor is added by Driver --- .../main/scala/org/apache/spark/deploy/client/AppClient.scala | 1 + .../scala/org/apache/spark/deploy/worker/ExecutorRunner.scala | 2 -- 2 files changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala index 98a93d1fcb2a3..4efebcaa350fe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/AppClient.scala @@ -134,6 +134,7 @@ private[spark] class AppClient( val fullId = appId + "/" + id logInfo("Executor added: %s on %s (%s) with %d cores".format(fullId, workerId, hostPort, cores)) + master ! ExecutorStateChanged(appId, id, ExecutorState.RUNNING, None, None) listener.executorAdded(fullId, workerId, hostPort, cores, memory) case ExecutorUpdated(id, state, message, exitStatus) => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 8ba6a01bbcb97..f4fedc6327ab9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -144,8 +144,6 @@ private[spark] class ExecutorRunner( Files.write(header, stderr, UTF_8) stderrAppender = FileAppender(process.getErrorStream, stderr, conf) - state = ExecutorState.RUNNING - worker ! ExecutorStateChanged(appId, execId, state, None, None) // Wait for it to exit; executor may exit with code 0 (when driver instructs it to shutdown) // or with nonzero exit code val exitCode = process.waitFor() From 1826372d0a1bc80db9015106dd5d2d155ada33f5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 3 Dec 2014 16:28:24 -0800 Subject: [PATCH 110/305] [SPARK-4085] Propagate FetchFailedException when Spark fails to read local shuffle file. cc aarondav kayousterhout pwendell This should go into 1.2? Author: Reynold Xin Closes #3579 from rxin/SPARK-4085 and squashes the following commits: 255b4fd [Reynold Xin] Updated test. f9814d9 [Reynold Xin] Code review feedback. 2afaf35 [Reynold Xin] [SPARK-4085] Propagate FetchFailedException when Spark fails to read local shuffle file. --- .../storage/ShuffleBlockFetcherIterator.scala | 28 +++++++++++-------- .../spark/ExternalShuffleServiceSuite.scala | 2 -- .../scala/org/apache/spark/ShuffleSuite.scala | 23 +++++++++++++++ 3 files changed, 40 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index 83170f7c5a4ab..2499c11a65b0e 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -17,6 +17,7 @@ package org.apache.spark.storage +import java.io.{InputStream, IOException} import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.{ArrayBuffer, HashSet, Queue} @@ -289,17 +290,22 @@ final class ShuffleBlockFetcherIterator( } val iteratorTry: Try[Iterator[Any]] = result match { - case FailureFetchResult(_, e) => Failure(e) - case SuccessFetchResult(blockId, _, buf) => { - val is = blockManager.wrapForCompression(blockId, buf.createInputStream()) - val iter = serializer.newInstance().deserializeStream(is).asIterator - Success(CompletionIterator[Any, Iterator[Any]](iter, { - // Once the iterator is exhausted, release the buffer and set currentResult to null - // so we don't release it again in cleanup. - currentResult = null - buf.release() - })) - } + case FailureFetchResult(_, e) => + Failure(e) + case SuccessFetchResult(blockId, _, buf) => + // There is a chance that createInputStream can fail (e.g. fetching a local file that does + // not exist, SPARK-4085). In that case, we should propagate the right exception so + // the scheduler gets a FetchFailedException. + Try(buf.createInputStream()).map { is0 => + val is = blockManager.wrapForCompression(blockId, is0) + val iter = serializer.newInstance().deserializeStream(is).asIterator + CompletionIterator[Any, Iterator[Any]](iter, { + // Once the iterator is exhausted, release the buffer and set currentResult to null + // so we don't release it again in cleanup. + currentResult = null + buf.release() + }) + } } (result.blockId, iteratorTry) diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index cc3592ee43a35..bac6fdbcdc976 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark -import java.util.concurrent.atomic.AtomicInteger - import org.scalatest.BeforeAndAfterAll import org.apache.spark.network.TransportContext diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 5a133c0490444..58a96245a9b53 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -23,6 +23,7 @@ import org.scalatest.Matchers import org.apache.spark.ShuffleSuite.NonJavaSerializableClass import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.storage.{ShuffleDataBlockId, ShuffleBlockId} import org.apache.spark.util.MutablePair abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContext { @@ -263,6 +264,28 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex } } } + + test("[SPARK-4085] rerun map stage if reduce stage cannot find its local shuffle file") { + val myConf = conf.clone().set("spark.test.noStageRetry", "false") + sc = new SparkContext("local", "test", myConf) + val rdd = sc.parallelize(1 to 10, 2).map((_, 1)).reduceByKey(_ + _) + rdd.count() + + // Delete one of the local shuffle blocks. + val hashFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleBlockId(0, 0, 0)) + val sortFile = sc.env.blockManager.diskBlockManager.getFile(new ShuffleDataBlockId(0, 0, 0)) + assert(hashFile.exists() || sortFile.exists()) + + if (hashFile.exists()) { + hashFile.delete() + } + if (sortFile.exists()) { + sortFile.delete() + } + + // This count should retry the execution of the previous stage and rerun shuffle. + rdd.count() + } } object ShuffleSuite { From 27ab0b8a03b711e8d86b6167df833f012205ccc7 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 4 Dec 2014 08:58:03 +0800 Subject: [PATCH 111/305] [SPARK-4711] [mllib] [docs] Programming guide advice on choosing optimizer I have heard requests for the docs to include advice about choosing an optimization method. The programming guide could include a brief statement about this (so the user does not have to read the whole optimization section). CC: mengxr Author: Joseph K. Bradley Closes #3569 from jkbradley/lr-doc and squashes the following commits: 654aeb5 [Joseph K. Bradley] updated section header for mllib-optimization 5035ad0 [Joseph K. Bradley] updated based on review 94f6dec [Joseph K. Bradley] Updated linear methods and optimization docs with quick advice on choosing an optimization method --- docs/mllib-linear-methods.md | 10 +++++++--- docs/mllib-optimization.md | 17 +++++++++++------ 2 files changed, 18 insertions(+), 9 deletions(-) diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index bc914a1899801..44b7f67c57734 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -110,12 +110,16 @@ However, L1 regularization can help promote sparsity in weights leading to small It is not recommended to train models without any regularization, especially when the number of training examples is small. +### Optimization + +Under the hood, linear methods use convex optimization methods to optimize the objective functions. MLlib uses two methods, SGD and L-BFGS, described in the [optimization section](mllib-optimization.html). Currently, most algorithm APIs support Stochastic Gradient Descent (SGD), and a few support L-BFGS. Refer to [this optimization section](mllib-optimization.html#Choosing-an-Optimization-Method) for guidelines on choosing between optimization methods. + ## Binary classification [Binary classification](http://en.wikipedia.org/wiki/Binary_classification) aims to divide items into two categories: positive and negative. MLlib -supports two linear methods for binary classification: linear support vector -machines (SVMs) and logistic regression. For both methods, MLlib supports +supports two linear methods for binary classification: linear Support Vector +Machines (SVMs) and logistic regression. For both methods, MLlib supports L1 and L2 regularized variants. The training data set is represented by an RDD of [LabeledPoint](mllib-data-types.html) in MLlib. Note that, in the mathematical formulation in this guide, a training label $y$ is denoted as @@ -123,7 +127,7 @@ either $+1$ (positive) or $-1$ (negative), which is convenient for the formulation. *However*, the negative label is represented by $0$ in MLlib instead of $-1$, to be consistent with multiclass labeling. -### Linear support vector machines (SVMs) +### Linear Support Vector Machines (SVMs) The [linear SVM](http://en.wikipedia.org/wiki/Support_vector_machine#Linear_SVM) is a standard method for large-scale classification tasks. It is a linear method as described above in equation `$\eqref{eq:regPrimal}$`, with the loss function in the formulation given by the hinge loss: diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 45141c235be90..4d101afca2c97 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -138,6 +138,12 @@ vertical scalability issue (the number of training features) when computing the explicitly in Newton's method. As a result, L-BFGS often achieves rapider convergence compared with other first-order optimization. +### Choosing an Optimization Method + +[Linear methods](mllib-linear-methods.html) use optimization internally, and some linear methods in MLlib support both SGD and L-BFGS. +Different optimization methods can have different convergence guarantees depending on the properties of the objective function, and we cannot cover the literature here. +In general, when L-BFGS is available, we recommend using it instead of SGD since L-BFGS tends to converge faster (in fewer iterations). + ## Implementation in MLlib ### Gradient descent and stochastic gradient descent @@ -168,10 +174,7 @@ descent. All updaters in MLlib use a step size at the t-th step equal to * `regParam` is the regularization parameter when using L1 or L2 regularization. * `miniBatchFraction` is the fraction of the total data that is sampled in each iteration, to compute the gradient direction. - -Available algorithms for gradient descent: - -* [GradientDescent](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) + * Sampling still requires a pass over the entire RDD, so decreasing `miniBatchFraction` may not speed up optimization much. Users will see the greatest speedup when the gradient is expensive to compute, for only the chosen samples are used for computing the gradient. ### L-BFGS L-BFGS is currently only a low-level optimization primitive in `MLlib`. If you want to use L-BFGS in various @@ -359,13 +362,15 @@ public class LBFGSExample { {% endhighlight %} -#### Developer's note + +## Developer's notes + Since the Hessian is constructed approximately from previous gradient evaluations, the objective function can not be changed during the optimization process. As a result, Stochastic L-BFGS will not work naively by just using miniBatch; therefore, we don't provide this until we have better understanding. -* `Updater` is a class originally designed for gradient decent which computes +`Updater` is a class originally designed for gradient decent which computes the actual gradient descent step. However, we're able to take the gradient and loss of objective function of regularization for L-BFGS by ignoring the part of logic only for gradient decent such as adaptive step size stuff. We will refactorize From 657a88835d8bf22488b53d50f75281d7dc32442e Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 4 Dec 2014 09:57:50 +0800 Subject: [PATCH 112/305] [SPARK-4580] [SPARK-4610] [mllib] [docs] Documentation for tree ensembles + DecisionTree API fix Major changes: * Added programming guide sections for tree ensembles * Added examples for tree ensembles * Updated DecisionTree programming guide with more info on parameters * **API change**: Standardized the tree parameter for the number of classes (for classification) Minor changes: * Updated decision tree documentation * Updated existing tree and tree ensemble examples * Use train/test split, and compute test error instead of training error. * Fixed decision_tree_runner.py to actually use the number of classes it computes from data. (small bug fix) Note: I know this is a lot of lines, but most is covered by: * Programming guide sections for gradient boosting and random forests. (The changes are probably best viewed by generating the docs locally.) * New examples (which were copied from the programming guide) * The "numClasses" renaming I have run all examples and relevant unit tests. CC: mengxr manishamde codedeft Author: Joseph K. Bradley Author: Joseph K. Bradley Closes #3461 from jkbradley/ensemble-docs and squashes the following commits: 70a75f3 [Joseph K. Bradley] updated forest vs boosting comparison d1de753 [Joseph K. Bradley] Added note about toString and toDebugString for DecisionTree to migration guide 8e87f8f [Joseph K. Bradley] Combined GBT and RandomForest guides into one ensembles guide 6fab846 [Joseph K. Bradley] small fixes based on review b9f8576 [Joseph K. Bradley] updated decision tree doc 375204c [Joseph K. Bradley] fixed python style 2b60b6e [Joseph K. Bradley] merged Java RandomForest examples into 1 file. added header. Fixed small bug in same example in the programming guide. 706d332 [Joseph K. Bradley] updated python DT runner to print full model if it is small c76c823 [Joseph K. Bradley] added migration guide for mllib abe5ed7 [Joseph K. Bradley] added examples for random forest in Java and Python to examples folder 07fc11d [Joseph K. Bradley] Renamed numClassesForClassification to numClasses everywhere in trees and ensembles. This is a breaking API change, but it was necessary to correct an API inconsistency in Spark 1.1 (where Python DecisionTree used numClasses but Scala used numClassesForClassification). cdfdfbc [Joseph K. Bradley] added examples for GBT 6372a2b [Joseph K. Bradley] updated decision tree examples to use random split. tested all of them. ad3e695 [Joseph K. Bradley] added gbt and random forest to programming guide. still need to update their examples --- docs/mllib-decision-tree.md | 241 ++++--- docs/mllib-ensembles.md | 653 ++++++++++++++++++ docs/mllib-guide.md | 29 +- .../mllib/JavaGradientBoostedTreesRunner.java | 2 +- .../mllib/JavaRandomForestExample.java | 139 ++++ .../main/python/mllib/decision_tree_runner.py | 17 +- .../python/mllib/random_forest_example.py | 89 +++ .../examples/mllib/DecisionTreeRunner.scala | 2 +- .../mllib/GradientBoostedTreesRunner.scala | 2 +- .../mllib/api/python/PythonMLLibAPI.scala | 4 +- .../spark/mllib/tree/DecisionTree.scala | 22 +- .../spark/mllib/tree/RandomForest.scala | 20 +- .../tree/configuration/BoostingStrategy.scala | 6 +- .../mllib/tree/configuration/Strategy.scala | 26 +- .../tree/impl/DecisionTreeMetadata.scala | 2 +- .../spark/mllib/tree/DecisionTreeSuite.scala | 46 +- .../tree/GradientBoostedTreesSuite.scala | 2 +- .../spark/mllib/tree/RandomForestSuite.scala | 14 +- python/pyspark/mllib/tree.py | 6 +- 19 files changed, 1140 insertions(+), 182 deletions(-) create mode 100644 docs/mllib-ensembles.md create mode 100644 examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestExample.java create mode 100755 examples/src/main/python/mllib/random_forest_example.py diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index 12a6afbeea829..fc8e732251a30 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -11,7 +11,7 @@ displayTitle: MLlib - Decision Tree and their ensembles are popular methods for the machine learning tasks of classification and regression. Decision trees are widely used since they are easy to interpret, handle categorical features, extend to the multiclass classification setting, do not require -feature scaling and are able to capture nonlinearities and feature interactions. Tree ensemble +feature scaling, and are able to capture non-linearities and feature interactions. Tree ensemble algorithms such as random forests and boosting are among the top performers for classification and regression tasks. @@ -19,6 +19,8 @@ MLlib supports decision trees for binary and multiclass classification and for r using both continuous and categorical features. The implementation partitions data by rows, allowing distributed training with millions of instances. +Ensembles of trees (Random Forests and Gradient-Boosted Trees) are described in the [Ensembles guide](mllib-ensembles.html). + ## Basic algorithm The decision tree is a greedy algorithm that performs a recursive binary partitioning of the feature @@ -42,18 +44,18 @@ impurity measure for regression (variance). Gini impurity Classification - $\sum_{i=1}^{M} f_i(1-f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels. + $\sum_{i=1}^{C} f_i(1-f_i)$$f_i$ is the frequency of label $i$ at a node and $C$ is the number of unique labels. Entropy Classification - $\sum_{i=1}^{M} -f_ilog(f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels. + $\sum_{i=1}^{C} -f_ilog(f_i)$$f_i$ is the frequency of label $i$ at a node and $C$ is the number of unique labels. Variance Regression - $\frac{1}{n} \sum_{i=1}^{N} (x_i - \mu)^2$$y_i$ is label for an instance, - $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^n x_i$. + $\frac{1}{N} \sum_{i=1}^{N} (x_i - \mu)^2$$y_i$ is label for an instance, + $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^N x_i$. @@ -103,36 +105,73 @@ and the resulting `$M-1$` split candidates are considered. ### Stopping rule -The recursive tree construction is stopped at a node when one of the two conditions is met: +The recursive tree construction is stopped at a node when one of the following conditions is met: 1. The node depth is equal to the `maxDepth` training parameter. -2. No split candidate leads to an information gain at the node. +2. No split candidate leads to an information gain greater than `minInfoGain`. +3. No split candidate produces child nodes which each have at least `minInstancesPerNode` training instances. + +## Usage tips + +We include a few guidelines for using decision trees by discussing the various parameters. +The parameters are listed below roughly in order of descending importance. New users should mainly consider the "Problem specification parameters" section and the `maxDepth` parameter. + +### Problem specification parameters + +These parameters describe the problem you want to solve and your dataset. +They should be specified and do not require tuning. + +* **`algo`**: `Classification` or `Regression` + +* **`numClasses`**: Number of classes (for `Classification` only) + +* **`categoricalFeaturesInfo`**: Specifies which features are categorical and how many categorical values each of those features can take. This is given as a map from feature indices to feature arity (number of categories). Any features not in this map are treated as continuous. + * E.g., `Map(0 -> 2, 4 -> 10)` specifies that feature `0` is binary (taking values `0` or `1`) and that feature `4` has 10 categories (values `{0, 1, ..., 9}`). Note that feature indices are 0-based: features `0` and `4` are the 1st and 5th elements of an instance's feature vector. + * Note that you do not have to specify `categoricalFeaturesInfo`. The algorithm will still run and may get reasonable results. However, performance should be better if categorical features are properly designated. + +### Stopping criteria + +These parameters determine when the tree stops building (adding new nodes). +When tuning these parameters, be careful to validate on held-out test data to avoid overfitting. + +* **`maxDepth`**: Maximum depth of a tree. Deeper trees are more expressive (potentially allowing higher accuracy), but they are also more costly to train and are more likely to overfit. + +* **`minInstancesPerNode`**: For a node to be split further, each of its children must receive at least this number of training instances. This is commonly used with [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) since those are often trained deeper than individual trees. + +* **`minInfoGain`**: For a node to be split further, the split must improve at least this much (in terms of information gain). + +### Tunable parameters -## Implementation details +These parameters may be tuned. Be careful to validate on held-out test data when tuning in order to avoid overfitting. -### Max memory requirements +* **`maxBins`**: Number of bins used when discretizing continuous features. + * Increasing `maxBins` allows the algorithm to consider more split candidates and make fine-grained split decisions. However, it also increases computation and communication. + * Note that the `maxBins` parameter must be at least the maximum number of categories `$M$` for any categorical feature. -For faster processing, the decision tree algorithm performs simultaneous histogram computations for -all nodes at each level of the tree. This could lead to high memory requirements at deeper levels -of the tree, potentially leading to memory overflow errors. To alleviate this problem, a `maxMemoryInMB` -training parameter specifies the maximum amount of memory at the workers (twice as much at the -master) to be allocated to the histogram computation. The default value is conservatively chosen to -be 256 MB to allow the decision algorithm to work in most scenarios. Once the memory requirements -for a level-wise computation cross the `maxMemoryInMB` threshold, the node training tasks at each -subsequent level are split into smaller tasks. +* **`maxMemoryInMB`**: Amount of memory to be used for collecting sufficient statistics. + * The default value is conservatively chosen to be 256 MB to allow the decision algorithm to work in most scenarios. Increasing `maxMemoryInMB` can lead to faster training (if the memory is available) by allowing fewer passes over the data. However, there may be decreasing returns as `maxMemoryInMB` grows since the amount of communication on each iteration can be proportional to `maxMemoryInMB`. + * *Implementation details*: For faster processing, the decision tree algorithm collects statistics about groups of nodes to split (rather than 1 node at a time). The number of nodes which can be handled in one group is determined by the memory requirements (which vary per features). The `maxMemoryInMB` parameter specifies the memory limit in terms of megabytes which each worker can use for these statistics. -Note that, if you have a large amount of memory, increasing `maxMemoryInMB` can lead to faster -training by requiring fewer passes over the data. +* **`subsamplingRate`**: Fraction of the training data used for learning the decision tree. This parameter is most relevant for training ensembles of trees (using [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) and [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees)), where it can be useful to subsample the original data. For training a single decision tree, this parameter is less useful since the number of training instances is generally not the main constraint. -### Binning feature values +* **`impurity`**: Impurity measure (discussed above) used to choose between candidate splits. This measure must match the `algo` parameter. -Increasing `maxBins` allows the algorithm to consider more split candidates and make fine-grained -split decisions. However, it also increases computation and communication. +### Caching and checkpointing -Note that the `maxBins` parameter must be at least the maximum number of categories `$M$` for -any categorical feature. +MLlib 1.2 adds several features for scaling up to larger (deeper) trees and tree ensembles. When `maxDepth` is set to be large, it can be useful to turn on node ID caching and checkpointing. These parameters are also useful for [RandomForest](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest) when `numTrees` is set to be large. -### Scaling +* **`useNodeIdCache`**: If this is set to true, the algorithm will avoid passing the current model (tree or trees) to executors on each iteration. + * This can be useful with deep trees (speeding up computation on workers) and for large Random Forests (reducing communication on each iteration). + * *Implementation details*: By default, the algorithm communicates the current model to executors so that executors can match training instances with tree nodes. When this setting is turned on, then the algorithm will instead cache this information. + +Node ID caching generates a sequence of RDDs (1 per iteration). This long lineage can cause performance problems, but checkpointing intermediate RDDs can alleviate those problems. +Note that checkpointing is only applicable when `useNodeIdCache` is set to true. + +* **`checkpointDir`**: Directory for checkpointing node ID cache RDDs. + +* **`checkpointInterval`**: Frequency for checkpointing node ID cache RDDs. Setting this too low will cause extra overhead from writing to HDFS; setting this too high can cause problems if executors fail and the RDD needs to be recomputed. + +## Scaling Computation scales approximately linearly in the number of training instances, in the number of features, and in the `maxBins` parameter. @@ -148,7 +187,7 @@ The example below demonstrates how to load a [LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), parse it as an RDD of `LabeledPoint` and then perform classification using a decision tree with Gini impurity as an impurity measure and a -maximum tree depth of 5. The training error is calculated to measure the algorithm accuracy. +maximum tree depth of 5. The test error is calculated to measure the algorithm accuracy.
@@ -158,8 +197,10 @@ import org.apache.spark.mllib.tree.DecisionTree import org.apache.spark.mllib.util.MLUtils // Load and parse the data file. -// Cache the data since we will use it again to compute training error. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache() +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) // Train a DecisionTree model. // Empty categoricalFeaturesInfo indicates all features are continuous. @@ -169,17 +210,17 @@ val impurity = "gini" val maxDepth = 5 val maxBins = 32 -val model = DecisionTree.trainClassifier(data, numClasses, categoricalFeaturesInfo, impurity, - maxDepth, maxBins) +val model = DecisionTree.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, + impurity, maxDepth, maxBins) -// Evaluate model on training instances and compute training error -val labelAndPreds = data.map { point => +// Evaluate model on test instances and compute test error +val labelAndPreds = testData.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / data.count -println("Training Error = " + trainErr) -println("Learned classification tree model:\n" + model) +val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() +println("Test Error = " + testErr) +println("Learned classification tree model:\n" + model.toDebugString) {% endhighlight %}
@@ -187,7 +228,6 @@ println("Learned classification tree model:\n" + model) {% highlight java %} import java.util.HashMap; import scala.Tuple2; -import org.apache.spark.api.java.function.Function2; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -203,37 +243,42 @@ SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); JavaSparkContext sc = new JavaSparkContext(sparkConf); // Load and parse the data file. -// Cache the data since we will use it again to compute training error. String datapath = "data/mllib/sample_libsvm_data.txt"; -JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; // Set parameters. // Empty categoricalFeaturesInfo indicates all features are continuous. Integer numClasses = 2; -HashMap categoricalFeaturesInfo = new HashMap(); +Map categoricalFeaturesInfo = new HashMap(); String impurity = "gini"; Integer maxDepth = 5; Integer maxBins = 32; // Train a DecisionTree model for classification. -final DecisionTreeModel model = DecisionTree.trainClassifier(data, numClasses, +final DecisionTreeModel model = DecisionTree.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, impurity, maxDepth, maxBins); -// Evaluate model on training instances and compute training error +// Evaluate model on test instances and compute test error JavaPairRDD predictionAndLabel = - data.mapToPair(new PairFunction() { - @Override public Tuple2 call(LabeledPoint p) { + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { return new Tuple2(model.predict(p.features()), p.label()); } }); -Double trainErr = +Double testErr = 1.0 * predictionAndLabel.filter(new Function, Boolean>() { - @Override public Boolean call(Tuple2 pl) { + @Override + public Boolean call(Tuple2 pl) { return !pl._1().equals(pl._2()); } - }).count() / data.count(); -System.out.println("Training error: " + trainErr); -System.out.println("Learned classification tree model:\n" + model); + }).count() / testData.count(); +System.out.println("Test Error: " + testErr); +System.out.println("Learned classification tree model:\n" + model.toDebugString()); {% endhighlight %} @@ -244,26 +289,23 @@ from pyspark.mllib.tree import DecisionTree from pyspark.mllib.util import MLUtils # Load and parse the data file into an RDD of LabeledPoint. -# Cache the data since we will use it again to compute training error. -data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) # Train a DecisionTree model. # Empty categoricalFeaturesInfo indicates all features are continuous. -model = DecisionTree.trainClassifier(data, numClasses=2, categoricalFeaturesInfo={}, +model = DecisionTree.trainClassifier(trainingData, numClasses=2, categoricalFeaturesInfo={}, impurity='gini', maxDepth=5, maxBins=32) -# Evaluate model on training instances and compute training error -predictions = model.predict(data.map(lambda x: x.features)) -labelsAndPredictions = data.map(lambda lp: lp.label).zip(predictions) -trainErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(data.count()) -print('Training Error = ' + str(trainErr)) +# Evaluate model on test instances and compute test error +predictions = model.predict(testData.map(lambda x: x.features)) +labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) +testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(testData.count()) +print('Test Error = ' + str(testErr)) print('Learned classification tree model:') -print(model) +print(model.toDebugString()) {% endhighlight %} - -Note: When making predictions for a dataset, it is more efficient to do batch prediction rather -than separately calling `predict` on each data point. This is because the Python code makes calls -to an underlying `DecisionTree` model in Scala. @@ -285,8 +327,10 @@ import org.apache.spark.mllib.tree.DecisionTree import org.apache.spark.mllib.util.MLUtils // Load and parse the data file. -// Cache the data since we will use it again to compute training error. -val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt").cache() +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) // Train a DecisionTree model. // Empty categoricalFeaturesInfo indicates all features are continuous. @@ -295,17 +339,17 @@ val impurity = "variance" val maxDepth = 5 val maxBins = 32 -val model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo, impurity, +val model = DecisionTree.trainRegressor(trainingData, categoricalFeaturesInfo, impurity, maxDepth, maxBins) -// Evaluate model on training instances and compute training error -val labelsAndPredictions = data.map { point => +// Evaluate model on test instances and compute test error +val labelsAndPredictions = testData.map { point => val prediction = model.predict(point.features) (point.label, prediction) } -val trainMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() -println("Training Mean Squared Error = " + trainMSE) -println("Learned regression tree model:\n" + model) +val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() +println("Test Mean Squared Error = " + testMSE) +println("Learned regression tree model:\n" + model.toDebugString) {% endhighlight %} @@ -325,45 +369,51 @@ import org.apache.spark.mllib.tree.model.DecisionTreeModel; import org.apache.spark.mllib.util.MLUtils; import org.apache.spark.SparkConf; -// Load and parse the data file. -// Cache the data since we will use it again to compute training error. -String datapath = "data/mllib/sample_libsvm_data.txt"; -JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD().cache(); - SparkConf sparkConf = new SparkConf().setAppName("JavaDecisionTree"); JavaSparkContext sc = new JavaSparkContext(sparkConf); +// Load and parse the data file. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; + // Set parameters. // Empty categoricalFeaturesInfo indicates all features are continuous. -HashMap categoricalFeaturesInfo = new HashMap(); +Map categoricalFeaturesInfo = new HashMap(); String impurity = "variance"; Integer maxDepth = 5; Integer maxBins = 32; // Train a DecisionTree model. -final DecisionTreeModel model = DecisionTree.trainRegressor(data, +final DecisionTreeModel model = DecisionTree.trainRegressor(trainingData, categoricalFeaturesInfo, impurity, maxDepth, maxBins); -// Evaluate model on training instances and compute training error +// Evaluate model on test instances and compute test error JavaPairRDD predictionAndLabel = - data.mapToPair(new PairFunction() { - @Override public Tuple2 call(LabeledPoint p) { + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { return new Tuple2(model.predict(p.features()), p.label()); } }); -Double trainMSE = +Double testMSE = predictionAndLabel.map(new Function, Double>() { - @Override public Double call(Tuple2 pl) { + @Override + public Double call(Tuple2 pl) { Double diff = pl._1() - pl._2(); return diff * diff; } }).reduce(new Function2() { - @Override public Double call(Double a, Double b) { + @Override + public Double call(Double a, Double b) { return a + b; } }) / data.count(); -System.out.println("Training Mean Squared Error: " + trainMSE); -System.out.println("Learned regression tree model:\n" + model); +System.out.println("Test Mean Squared Error: " + testMSE); +System.out.println("Learned regression tree model:\n" + model.toDebugString()); {% endhighlight %} @@ -374,26 +424,23 @@ from pyspark.mllib.tree import DecisionTree from pyspark.mllib.util import MLUtils # Load and parse the data file into an RDD of LabeledPoint. -# Cache the data since we will use it again to compute training error. -data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt').cache() +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) # Train a DecisionTree model. # Empty categoricalFeaturesInfo indicates all features are continuous. -model = DecisionTree.trainRegressor(data, categoricalFeaturesInfo={}, +model = DecisionTree.trainRegressor(trainingData, categoricalFeaturesInfo={}, impurity='variance', maxDepth=5, maxBins=32) -# Evaluate model on training instances and compute training error -predictions = model.predict(data.map(lambda x: x.features)) -labelsAndPredictions = data.map(lambda lp: lp.label).zip(predictions) -trainMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(data.count()) -print('Training Mean Squared Error = ' + str(trainMSE)) +# Evaluate model on test instances and compute test error +predictions = model.predict(testData.map(lambda x: x.features)) +labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) +testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(testData.count()) +print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression tree model:') -print(model) +print(model.toDebugString()) {% endhighlight %} - -Note: When making predictions for a dataset, it is more efficient to do batch prediction rather -than separately calling `predict` on each data point. This is because the Python code makes calls -to an underlying `DecisionTree` model in Scala. diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md new file mode 100644 index 0000000000000..23ede04b62d5b --- /dev/null +++ b/docs/mllib-ensembles.md @@ -0,0 +1,653 @@ +--- +layout: global +title: Ensembles - MLlib +displayTitle: MLlib - Ensembles +--- + +* Table of contents +{:toc} + +An [ensemble method](http://en.wikipedia.org/wiki/Ensemble_learning) +is a learning algorithm which creates a model composed of a set of other base models. +MLlib supports two major ensemble algorithms: [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBosotedTrees) and [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest). +Both use [decision trees](mllib-decision-tree.html) as their base models. + +## Gradient-Boosted Trees vs. Random Forests + +Both [Gradient-Boosted Trees (GBTs)](mllib-ensembles.html#Gradient-Boosted-Trees-(GBTS)) and [Random Forests](mllib-ensembles.html#Random-Forests) are algorithms for learning ensembles of trees, but the training processes are different. There are several practical trade-offs: + + * GBTs train one tree at a time, so they can take longer to train than random forests. Random Forests can train multiple trees in parallel. + * On the other hand, it is often reasonable to use smaller (shallower) trees with GBTs than with Random Forests, and training smaller trees takes less time. + * Random Forests can be less prone to overfitting. Training more trees in a Random Forest reduces the likelihood of overfitting, but training more trees with GBTs increases the likelihood of overfitting. (In statistical language, Random Forests reduce variance by using more trees, whereas GBTs reduce bias by using more trees.) + * Random Forests can be easier to tune since performance improves monotonically with the number of trees (whereas performance can start to decrease for GBTs if the number of trees grows too large). + +In short, both algorithms can be effective, and the choice should be based on the particular dataset. + +## Random Forests + +[Random forests](http://en.wikipedia.org/wiki/Random_forest) +are ensembles of [decision trees](mllib-decision-tree.html). +Random forests are one of the most successful machine learning models for classification and +regression. They combine many decision trees in order to reduce the risk of overfitting. +Like decision trees, random forests handle categorical features, +extend to the multiclass classification setting, do not require +feature scaling, and are able to capture non-linearities and feature interactions. + +MLlib supports random forests for binary and multiclass classification and for regression, +using both continuous and categorical features. +MLlib implements random forests using the existing [decision tree](mllib-decision-tree.html) +implementation. Please see the decision tree guide for more information on trees. + +### Basic algorithm + +Random forests train a set of decision trees separately, so the training can be done in parallel. +The algorithm injects randomness into the training process so that each decision tree is a bit +different. Combining the predictions from each tree reduces the variance of the predictions, +improving the performance on test data. + +#### Training + +The randomness injected into the training process includes: + +* Subsampling the original dataset on each iteration to get a different training set (a.k.a. bootstrapping). +* Considering different random subsets of features to split on at each tree node. + +Apart from these randomizations, decision tree training is done in the same way as for individual decision trees. + +#### Prediction + +To make a prediction on a new instance, a random forest must aggregate the predictions from its set of decision trees. This aggregation is done differently for classification and regression. + +*Classification*: Majority vote. Each tree's prediction is counted as a vote for one class. The label is predicted to be the class which receives the most votes. + +*Regression*: Averaging. Each tree predicts a real value. The label is predicted to be the average of the tree predictions. + +### Usage tips + +We include a few guidelines for using random forests by discussing the various parameters. +We omit some decision tree parameters since those are covered in the [decision tree guide](mllib-decision-tree.html). + +The first two parameters we mention are the most important, and tuning them can often improve performance: + +* **`numTrees`**: Number of trees in the forest. + * Increasing the number of trees will decrease the variance in predictions, improving the model's test-time accuracy. + * Training time increases roughly linearly in the number of trees. + +* **`maxDepth`**: Maximum depth of each tree in the forest. + * Increasing the depth makes the model more expressive and powerful. However, deep trees take longer to train and are also more prone to overfitting. + * In general, it is acceptable to train deeper trees when using random forests than when using a single decision tree. One tree is more likely to overfit than a random forest (because of the variance reduction from averaging multiple trees in the forest). + +The next two parameters generally do not require tuning. However, they can be tuned to speed up training. + +* **`subsamplingRate`**: This parameter specifies the size of the dataset used for training each tree in the forest, as a fraction of the size of the original dataset. The default (1.0) is recommended, but decreasing this fraction can speed up training. + +* **`featureSubsetStrategy`**: Number of features to use as candidates for splitting at each tree node. The number is specified as a fraction or function of the total number of features. Decreasing this number will speed up training, but can sometimes impact performance if too low. + +### Examples + +#### Classification + +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform classification using a Random Forest. +The test error is calculated to measure the algorithm accuracy. + +
+ +
+{% highlight scala %} +import org.apache.spark.mllib.tree.RandomForest +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) + +// Train a RandomForest model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +val numClasses = 2 +val categoricalFeaturesInfo = Map[Int, Int]() +val numTrees = 3 // Use more in practice. +val featureSubsetStrategy = "auto" // Let the algorithm choose. +val impurity = "gini" +val maxDepth = 4 +val maxBins = 32 + +val model = RandomForest.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, + numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins) + +// Evaluate model on test instances and compute test error +val labelAndPreds = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() +println("Test Error = " + testErr) +println("Learned classification forest model:\n" + model.toDebugString) +{% endhighlight %} +
+ +
+{% highlight java %} +import scala.Tuple2; +import java.util.HashMap; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.RandomForest; +import org.apache.spark.mllib.tree.model.RandomForestModel; +import org.apache.spark.mllib.util.MLUtils; + +SparkConf sparkConf = new SparkConf().setAppName("JavaRandomForestClassification"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Load and parse the data file. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; + +// Train a RandomForest model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +Integer numClasses = 2; +HashMap categoricalFeaturesInfo = new HashMap(); +Integer numTrees = 3; // Use more in practice. +String featureSubsetStrategy = "auto"; // Let the algorithm choose. +String impurity = "gini"; +Integer maxDepth = 5; +Integer maxBins = 32; +Integer seed = 12345; + +final RandomForestModel model = RandomForest.trainClassifier(trainingData, numClasses, + categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, + seed); + +// Evaluate model on test instances and compute test error +JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double testErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override + public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / testData.count(); +System.out.println("Test Error: " + testErr); +System.out.println("Learned classification forest model:\n" + model.toDebugString()); +{% endhighlight %} +
+ +
+{% highlight python %} +from pyspark.mllib.tree import RandomForest +from pyspark.mllib.util import MLUtils + +# Load and parse the data file into an RDD of LabeledPoint. +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a RandomForest model. +# Empty categoricalFeaturesInfo indicates all features are continuous. +# Note: Use larger numTrees in practice. +# Setting featureSubsetStrategy="auto" lets the algorithm choose. +model = RandomForest.trainClassifier(trainingData, numClasses=2, categoricalFeaturesInfo={}, + numTrees=3, featureSubsetStrategy="auto", + impurity='gini', maxDepth=4, maxBins=32) + +# Evaluate model on test instances and compute test error +predictions = model.predict(testData.map(lambda x: x.features)) +labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) +testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count() / float(testData.count()) +print('Test Error = ' + str(testErr)) +print('Learned classification forest model:') +print(model.toDebugString()) +{% endhighlight %} +
+ +
+ +#### Regression + +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform regression using a Random Forest. +The Mean Squared Error (MSE) is computed at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +
+ +
+{% highlight scala %} +import org.apache.spark.mllib.tree.RandomForest +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) + +// Train a RandomForest model. +// Empty categoricalFeaturesInfo indicates all features are continuous. +val numClasses = 2 +val categoricalFeaturesInfo = Map[Int, Int]() +val numTrees = 3 // Use more in practice. +val featureSubsetStrategy = "auto" // Let the algorithm choose. +val impurity = "variance" +val maxDepth = 4 +val maxBins = 32 + +val model = RandomForest.trainRegressor(trainingData, categoricalFeaturesInfo, + numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins) + +// Evaluate model on test instances and compute test error +val labelsAndPredictions = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() +println("Test Mean Squared Error = " + testMSE) +println("Learned regression forest model:\n" + model.toDebugString) +{% endhighlight %} +
+ +
+{% highlight java %} +import java.util.HashMap; +import scala.Tuple2; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.RandomForest; +import org.apache.spark.mllib.tree.model.RandomForestModel; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.SparkConf; + +SparkConf sparkConf = new SparkConf().setAppName("JavaRandomForest"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Load and parse the data file. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; + +// Set parameters. +// Empty categoricalFeaturesInfo indicates all features are continuous. +Map categoricalFeaturesInfo = new HashMap(); +String impurity = "variance"; +Integer maxDepth = 4; +Integer maxBins = 32; + +// Train a RandomForest model. +final RandomForestModel model = RandomForest.trainRegressor(trainingData, + categoricalFeaturesInfo, impurity, maxDepth, maxBins); + +// Evaluate model on test instances and compute test error +JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double testMSE = + predictionAndLabel.map(new Function, Double>() { + @Override + public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override + public Double call(Double a, Double b) { + return a + b; + } + }) / testData.count(); +System.out.println("Test Mean Squared Error: " + testMSE); +System.out.println("Learned regression forest model:\n" + model.toDebugString()); +{% endhighlight %} +
+ +
+{% highlight python %} +from pyspark.mllib.tree import RandomForest +from pyspark.mllib.util import MLUtils + +# Load and parse the data file into an RDD of LabeledPoint. +data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') +# Split the data into training and test sets (30% held out for testing) +(trainingData, testData) = data.randomSplit([0.7, 0.3]) + +# Train a RandomForest model. +# Empty categoricalFeaturesInfo indicates all features are continuous. +# Note: Use larger numTrees in practice. +# Setting featureSubsetStrategy="auto" lets the algorithm choose. +model = RandomForest.trainRegressor(trainingData, categoricalFeaturesInfo={}, + numTrees=3, featureSubsetStrategy="auto", + impurity='variance', maxDepth=4, maxBins=32) + +# Evaluate model on test instances and compute test error +predictions = model.predict(testData.map(lambda x: x.features)) +labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) +testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum() / float(testData.count()) +print('Test Mean Squared Error = ' + str(testMSE)) +print('Learned regression forest model:') +print(model.toDebugString()) +{% endhighlight %} +
+ +
+ +## Gradient-Boosted Trees (GBTs) + +[Gradient-Boosted Trees (GBTs)](http://en.wikipedia.org/wiki/Gradient_boosting) +are ensembles of [decision trees](mllib-decision-tree.html). +GBTs iteratively train decision trees in order to minimize a loss function. +Like decision trees, GBTs handle categorical features, +extend to the multiclass classification setting, do not require +feature scaling, and are able to capture non-linearities and feature interactions. + +MLlib supports GBTs for binary classification and for regression, +using both continuous and categorical features. +MLlib implements GBTs using the existing [decision tree](mllib-decision-tree.html) implementation. Please see the decision tree guide for more information on trees. + +*Note*: GBTs do not yet support multiclass classification. For multiclass problems, please use +[decision trees](mllib-decision-tree.html) or [Random Forests](mllib-ensembles.html#Random-Forest). + +### Basic algorithm + +Gradient boosting iteratively trains a sequence of decision trees. +On each iteration, the algorithm uses the current ensemble to predict the label of each training instance and then compares the prediction with the true label. The dataset is re-labeled to put more emphasis on training instances with poor predictions. Thus, in the next iteration, the decision tree will help correct for previous mistakes. + +The specific mechanism for re-labeling instances is defined by a loss function (discussed below). With each iteration, GBTs further reduce this loss function on the training data. + +#### Losses + +The table below lists the losses currently supported by GBTs in MLlib. +Note that each loss is applicable to one of classification or regression, not both. + +Notation: $N$ = number of instances. $y_i$ = label of instance $i$. $x_i$ = features of instance $i$. $F(x_i)$ = model's predicted label for instance $i$. + + + + + + + + + + + + + + + + + + + + + + +
LossTaskFormulaDescription
Log LossClassification$2 \sum_{i=1}^{N} \log(1+\exp(-2 y_i F(x_i)))$Twice binomial negative log likelihood.
Squared ErrorRegression$\sum_{i=1}^{N} (y_i - F(x_i))^2$Also called L2 loss. Default loss for regression tasks.
Absolute ErrorRegression$\sum_{i=1}^{N} |y_i - F(x_i)|$Also called L1 loss. Can be more robust to outliers than Squared Error.
+ +### Usage tips + +We include a few guidelines for using GBTs by discussing the various parameters. +We omit some decision tree parameters since those are covered in the [decision tree guide](mllib-decision-tree.html). + +* **`loss`**: See the section above for information on losses and their applicability to tasks (classification vs. regression). Different losses can give significantly different results, depending on the dataset. + +* **`numIterations`**: This sets the number of trees in the ensemble. Each iteration produces one tree. Increasing this number makes the model more expressive, improving training data accuracy. However, test-time accuracy may suffer if this is too large. + +* **`learningRate`**: This parameter should not need to be tuned. If the algorithm behavior seems unstable, decreasing this value may improve stability. + +* **`algo`**: The algorithm or task (classification vs. regression) is set using the tree [Strategy] parameter. + + +### Examples + +GBTs currently have APIs in Scala and Java. Examples in both languages are shown below. + +#### Classification + +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform classification using Gradient-Boosted Trees with log loss. +The test error is calculated to measure the algorithm accuracy. + +
+ +
+{% highlight scala %} +import org.apache.spark.mllib.tree.GradientBoostedTrees +import org.apache.spark.mllib.tree.configuration.BoostingStrategy +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) + +// Train a GradientBoostedTrees model. +// The defaultParams for Classification use LogLoss by default. +val boostingStrategy = BoostingStrategy.defaultParams("Classification") +boostingStrategy.numIterations = 3 // Note: Use more iterations in practice. +boostingStrategy.treeStrategy.numClassesForClassification = 2 +boostingStrategy.treeStrategy.maxDepth = 5 +// Empty categoricalFeaturesInfo indicates all features are continuous. +boostingStrategy.treeStrategy.categoricalFeaturesInfo = Map[Int, Int]() + +val model = GradientBoostedTrees.train(trainingData, boostingStrategy) + +// Evaluate model on test instances and compute test error +val labelAndPreds = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val testErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / testData.count() +println("Test Error = " + testErr) +println("Learned classification GBT model:\n" + model.toDebugString) +{% endhighlight %} +
+ +
+{% highlight java %} +import scala.Tuple2; +import java.util.HashMap; +import java.util.Map; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.GradientBoostedTrees; +import org.apache.spark.mllib.tree.configuration.BoostingStrategy; +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel; +import org.apache.spark.mllib.util.MLUtils; + +SparkConf sparkConf = new SparkConf().setAppName("JavaGradientBoostedTrees"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Load and parse the data file. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; + +// Train a GradientBoostedTrees model. +// The defaultParams for Classification use LogLoss by default. +BoostingStrategy boostingStrategy = BoostingStrategy.defaultParams("Classification"); +boostingStrategy.setNumIterations(3); // Note: Use more iterations in practice. +boostingStrategy.getTreeStrategy().setNumClassesForClassification(2); +boostingStrategy.getTreeStrategy().setMaxDepth(5); +// Empty categoricalFeaturesInfo indicates all features are continuous. +Map categoricalFeaturesInfo = new HashMap(); +boostingStrategy.treeStrategy().setCategoricalFeaturesInfo(categoricalFeaturesInfo); + +final GradientBoostedTreesModel model = + GradientBoostedTrees.train(trainingData, boostingStrategy); + +// Evaluate model on test instances and compute test error +JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double testErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override + public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / testData.count(); +System.out.println("Test Error: " + testErr); +System.out.println("Learned classification GBT model:\n" + model.toDebugString()); +{% endhighlight %} +
+ +
+ +#### Regression + +The example below demonstrates how to load a +[LIBSVM data file](http://www.csie.ntu.edu.tw/~cjlin/libsvmtools/datasets/), +parse it as an RDD of `LabeledPoint` and then +perform regression using Gradient-Boosted Trees with Squared Error as the loss. +The Mean Squared Error (MSE) is computed at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +
+ +
+{% highlight scala %} +import org.apache.spark.mllib.tree.GradientBoostedTrees +import org.apache.spark.mllib.tree.configuration.BoostingStrategy +import org.apache.spark.mllib.util.MLUtils + +// Load and parse the data file. +val data = MLUtils.loadLibSVMFile(sc, "data/mllib/sample_libsvm_data.txt") +// Split the data into training and test sets (30% held out for testing) +val splits = data.randomSplit(Array(0.7, 0.3)) +val (trainingData, testData) = (splits(0), splits(1)) + +// Train a GradientBoostedTrees model. +// The defaultParams for Regression use SquaredError by default. +val boostingStrategy = BoostingStrategy.defaultParams("Regression") +boostingStrategy.numIterations = 3 // Note: Use more iterations in practice. +boostingStrategy.treeStrategy.maxDepth = 5 +// Empty categoricalFeaturesInfo indicates all features are continuous. +boostingStrategy.treeStrategy.categoricalFeaturesInfo = Map[Int, Int]() + +val model = GradientBoostedTrees.train(trainingData, boostingStrategy) + +// Evaluate model on test instances and compute test error +val labelsAndPredictions = testData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val testMSE = labelsAndPredictions.map{ case(v, p) => math.pow((v - p), 2)}.mean() +println("Test Mean Squared Error = " + testMSE) +println("Learned regression GBT model:\n" + model.toDebugString) +{% endhighlight %} +
+ +
+{% highlight java %} +import scala.Tuple2; +import java.util.HashMap; +import java.util.Map; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.GradientBoostedTrees; +import org.apache.spark.mllib.tree.configuration.BoostingStrategy; +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel; +import org.apache.spark.mllib.util.MLUtils; + +SparkConf sparkConf = new SparkConf().setAppName("JavaGradientBoostedTrees"); +JavaSparkContext sc = new JavaSparkContext(sparkConf); + +// Load and parse the data file. +String datapath = "data/mllib/sample_libsvm_data.txt"; +JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); +// Split the data into training and test sets (30% held out for testing) +JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); +JavaRDD trainingData = splits[0]; +JavaRDD testData = splits[1]; + +// Train a GradientBoostedTrees model. +// The defaultParams for Regression use SquaredError by default. +BoostingStrategy boostingStrategy = BoostingStrategy.defaultParams("Regression"); +boostingStrategy.setNumIterations(3); // Note: Use more iterations in practice. +boostingStrategy.getTreeStrategy().setMaxDepth(5); +// Empty categoricalFeaturesInfo indicates all features are continuous. +Map categoricalFeaturesInfo = new HashMap(); +boostingStrategy.treeStrategy().setCategoricalFeaturesInfo(categoricalFeaturesInfo); + +final GradientBoostedTreesModel model = + GradientBoostedTrees.train(trainingData, boostingStrategy); + +// Evaluate model on test instances and compute test error +JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); +Double testMSE = + predictionAndLabel.map(new Function, Double>() { + @Override + public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override + public Double call(Double a, Double b) { + return a + b; + } + }) / data.count(); +System.out.println("Test Mean Squared Error: " + testMSE); +System.out.println("Learned regression GBT model:\n" + model.toDebugString()); +{% endhighlight %} +
+ +
diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 94fc98ce4fabe..dcb6819f46cba 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -16,8 +16,9 @@ filtering, dimensionality reduction, as well as underlying optimization primitiv * 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) * [naive Bayes](mllib-naive-bayes.html) + * [decision trees](mllib-decision-tree.html) + * [ensembles of trees](mllib-ensembles.html) (Random Forests and Gradient-Boosted Trees) * [Collaborative filtering](mllib-collaborative-filtering.html) * alternating least squares (ALS) * [Clustering](mllib-clustering.html) @@ -60,6 +61,32 @@ To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 # Migration Guide +## From 1.1 to 1.2 + +The only API changes in MLlib v1.2 are in +[`DecisionTree`](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree), +which continues to be an experimental API in MLlib 1.2: + +1. *(Breaking change)* The Scala API for classification takes a named argument specifying the number +of classes. In MLlib v1.1, this argument was called `numClasses` in Python and +`numClassesForClassification` in Scala. In MLlib v1.2, the names are both set to `numClasses`. +This `numClasses` parameter is specified either via +[`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. *(Breaking change)* The API for +[`Node`](api/scala/index.html#org.apache.spark.mllib.tree.model.Node) has changed. +This should generally not affect user code, unless the user manually constructs decision trees +(instead of using the `trainClassifier` or `trainRegressor` methods). +The tree `Node` now includes more information, including the probability of the predicted label +(for classification). + +3. Printing methods' output has changed. The `toString` (Scala/Java) and `__repr__` (Python) methods used to print the full model; they now print a summary. For the full model, use `toDebugString`. + +Examples in the Spark distribution and examples in the +[Decision Trees Guide](mllib-decision-tree.html#examples) have been updated accordingly. + ## From 1.0 to 1.1 The only API changes in MLlib v1.1 are in diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTreesRunner.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTreesRunner.java index 4a5ac404ea5ea..a1844d5d07ad4 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTreesRunner.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaGradientBoostedTreesRunner.java @@ -73,7 +73,7 @@ public static void main(String[] args) { return p.label(); } }).countByValue().size(); - boostingStrategy.treeStrategy().setNumClassesForClassification(numClasses); + boostingStrategy.treeStrategy().setNumClasses(numClasses); // Train a GradientBoosting model for classification. final GradientBoostedTreesModel model = GradientBoostedTrees.train(data, boostingStrategy); diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestExample.java new file mode 100644 index 0000000000000..89a4e092a5af7 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestExample.java @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.mllib; + +import scala.Tuple2; + +import java.util.HashMap; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.Function2; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.tree.RandomForest; +import org.apache.spark.mllib.tree.model.RandomForestModel; +import org.apache.spark.mllib.util.MLUtils; + +public final class JavaRandomForestExample { + + /** + * Note: This example illustrates binary classification. + * For information on multiclass classification, please refer to the JavaDecisionTree.java + * example. + */ + private static void testClassification(JavaRDD trainingData, + JavaRDD testData) { + // Train a RandomForest model. + // Empty categoricalFeaturesInfo indicates all features are continuous. + Integer numClasses = 2; + HashMap categoricalFeaturesInfo = new HashMap(); + Integer numTrees = 3; // Use more in practice. + String featureSubsetStrategy = "auto"; // Let the algorithm choose. + String impurity = "gini"; + Integer maxDepth = 4; + Integer maxBins = 32; + Integer seed = 12345; + + final RandomForestModel model = RandomForest.trainClassifier(trainingData, numClasses, + categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, + seed); + + // Evaluate model on test instances and compute test error + JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); + Double testErr = + 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + @Override + public Boolean call(Tuple2 pl) { + return !pl._1().equals(pl._2()); + } + }).count() / testData.count(); + System.out.println("Test Error: " + testErr); + System.out.println("Learned classification forest model:\n" + model.toDebugString()); + } + + private static void testRegression(JavaRDD trainingData, + JavaRDD testData) { + // Train a RandomForest model. + // Empty categoricalFeaturesInfo indicates all features are continuous. + HashMap categoricalFeaturesInfo = new HashMap(); + Integer numTrees = 3; // Use more in practice. + String featureSubsetStrategy = "auto"; // Let the algorithm choose. + String impurity = "variance"; + Integer maxDepth = 4; + Integer maxBins = 32; + Integer seed = 12345; + + final RandomForestModel model = RandomForest.trainRegressor(trainingData, + categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, + seed); + + // Evaluate model on test instances and compute test error + JavaPairRDD predictionAndLabel = + testData.mapToPair(new PairFunction() { + @Override + public Tuple2 call(LabeledPoint p) { + return new Tuple2(model.predict(p.features()), p.label()); + } + }); + Double testMSE = + predictionAndLabel.map(new Function, Double>() { + @Override + public Double call(Tuple2 pl) { + Double diff = pl._1() - pl._2(); + return diff * diff; + } + }).reduce(new Function2() { + @Override + public Double call(Double a, Double b) { + return a + b; + } + }) / testData.count(); + System.out.println("Test Mean Squared Error: " + testMSE); + System.out.println("Learned regression forest model:\n" + model.toDebugString()); + } + + public static void main(String[] args) { + SparkConf sparkConf = new SparkConf().setAppName("JavaRandomForestExample"); + JavaSparkContext sc = new JavaSparkContext(sparkConf); + + // Load and parse the data file. + String datapath = "data/mllib/sample_libsvm_data.txt"; + JavaRDD data = MLUtils.loadLibSVMFile(sc.sc(), datapath).toJavaRDD(); + // Split the data into training and test sets (30% held out for testing) + JavaRDD[] splits = data.randomSplit(new double[]{0.7, 0.3}); + JavaRDD trainingData = splits[0]; + JavaRDD testData = splits[1]; + + System.out.println("\nRunning example of classification using RandomForest\n"); + testClassification(trainingData, testData); + + System.out.println("\nRunning example of regression using RandomForest\n"); + testRegression(trainingData, testData); + sc.stop(); + } +} diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index 61ea4e06ecf3a..fccabd841b139 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -106,8 +106,7 @@ def reindexClassLabels(data): def usage(): print >> sys.stderr, \ - "Usage: decision_tree_runner [libsvm format data filepath]\n" + \ - " Note: This only supports binary classification." + "Usage: decision_tree_runner [libsvm format data filepath]" exit(1) @@ -127,16 +126,20 @@ def usage(): # Re-index class labels if needed. (reindexedData, origToNewLabels) = reindexClassLabels(points) + numClasses = len(origToNewLabels) # Train a classifier. categoricalFeaturesInfo = {} # no categorical features - model = DecisionTree.trainClassifier(reindexedData, numClasses=2, + model = DecisionTree.trainClassifier(reindexedData, numClasses=numClasses, categoricalFeaturesInfo=categoricalFeaturesInfo) # Print learned tree and stats. print "Trained DecisionTree for classification:" - print " Model numNodes: %d\n" % model.numNodes() - print " Model depth: %d\n" % model.depth() - print " Training accuracy: %g\n" % getAccuracy(model, reindexedData) - print model + print " Model numNodes: %d" % model.numNodes() + print " Model depth: %d" % model.depth() + print " Training accuracy: %g" % getAccuracy(model, reindexedData) + if model.numNodes() < 20: + print model.toDebugString() + else: + print model sc.stop() diff --git a/examples/src/main/python/mllib/random_forest_example.py b/examples/src/main/python/mllib/random_forest_example.py new file mode 100755 index 0000000000000..d3c24f7664329 --- /dev/null +++ b/examples/src/main/python/mllib/random_forest_example.py @@ -0,0 +1,89 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +""" +Random Forest classification and regression using MLlib. + +Note: This example illustrates binary classification. + For information on multiclass classification, please refer to the decision_tree_runner.py + example. +""" + +import sys + +from pyspark.context import SparkContext +from pyspark.mllib.tree import RandomForest +from pyspark.mllib.util import MLUtils + + +def testClassification(trainingData, testData): + # Train a RandomForest model. + # Empty categoricalFeaturesInfo indicates all features are continuous. + # Note: Use larger numTrees in practice. + # Setting featureSubsetStrategy="auto" lets the algorithm choose. + model = RandomForest.trainClassifier(trainingData, numClasses=2, + categoricalFeaturesInfo={}, + numTrees=3, featureSubsetStrategy="auto", + impurity='gini', maxDepth=4, maxBins=32) + + # Evaluate model on test instances and compute test error + predictions = model.predict(testData.map(lambda x: x.features)) + labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) + testErr = labelsAndPredictions.filter(lambda (v, p): v != p).count()\ + / float(testData.count()) + print('Test Error = ' + str(testErr)) + print('Learned classification forest model:') + print(model.toDebugString()) + + +def testRegression(trainingData, testData): + # Train a RandomForest model. + # Empty categoricalFeaturesInfo indicates all features are continuous. + # Note: Use larger numTrees in practice. + # Setting featureSubsetStrategy="auto" lets the algorithm choose. + model = RandomForest.trainRegressor(trainingData, categoricalFeaturesInfo={}, + numTrees=3, featureSubsetStrategy="auto", + impurity='variance', maxDepth=4, maxBins=32) + + # Evaluate model on test instances and compute test error + predictions = model.predict(testData.map(lambda x: x.features)) + labelsAndPredictions = testData.map(lambda lp: lp.label).zip(predictions) + testMSE = labelsAndPredictions.map(lambda (v, p): (v - p) * (v - p)).sum()\ + / float(testData.count()) + print('Test Mean Squared Error = ' + str(testMSE)) + print('Learned regression forest model:') + print(model.toDebugString()) + + +if __name__ == "__main__": + if len(sys.argv) > 1: + print >> sys.stderr, "Usage: random_forest_example" + exit(1) + sc = SparkContext(appName="PythonRandomForestExample") + + # Load and parse the data file into an RDD of LabeledPoint. + data = MLUtils.loadLibSVMFile(sc, 'data/mllib/sample_libsvm_data.txt') + # Split the data into training and test sets (30% held out for testing) + (trainingData, testData) = data.randomSplit([0.7, 0.3]) + + print('\nRunning example of classification using RandomForest\n') + testClassification(trainingData, testData) + + print('\nRunning example of regression using RandomForest\n') + testRegression(trainingData, testData) + + sc.stop() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index 54953adb5f3df..205d80dd02682 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -278,7 +278,7 @@ object DecisionTreeRunner { impurity = impurityCalculator, maxDepth = params.maxDepth, maxBins = params.maxBins, - numClassesForClassification = numClasses, + numClasses = numClasses, minInstancesPerNode = params.minInstancesPerNode, minInfoGain = params.minInfoGain, useNodeIdCache = params.useNodeIdCache, diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala index 1def8b45a230c..431ead8c0c165 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala @@ -103,7 +103,7 @@ object GradientBoostedTreesRunner { params.dataFormat, params.testInput, Algo.withName(params.algo), params.fracTest) val boostingStrategy = BoostingStrategy.defaultParams(params.algo) - boostingStrategy.treeStrategy.numClassesForClassification = numClasses + boostingStrategy.treeStrategy.numClasses = numClasses boostingStrategy.numIterations = params.numIterations boostingStrategy.treeStrategy.maxDepth = params.maxDepth diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 9f20cd5d00dcd..c4e5fd8e461fc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -477,7 +477,7 @@ class PythonMLLibAPI extends Serializable { algo = algo, impurity = impurity, maxDepth = maxDepth, - numClassesForClassification = numClasses, + numClasses = numClasses, maxBins = maxBins, categoricalFeaturesInfo = categoricalFeaturesInfo.asScala.toMap, minInstancesPerNode = minInstancesPerNode, @@ -513,7 +513,7 @@ class PythonMLLibAPI extends Serializable { algo = algo, impurity = impurity, maxDepth = maxDepth, - numClassesForClassification = numClasses, + numClasses = numClasses, maxBins = maxBins, categoricalFeaturesInfo = categoricalFeaturesInfo.asScala.toMap) val cached = data.rdd.persist(StorageLevel.MEMORY_AND_DISK) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 3d91867c896d9..73e7e32c6db31 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -136,7 +136,7 @@ object DecisionTree extends Serializable with Logging { * @param impurity impurity criterion used for information gain calculation * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * @param numClassesForClassification number of classes for classification. Default value of 2. + * @param numClasses number of classes for classification. Default value of 2. * @return DecisionTreeModel that can be used for prediction */ def train( @@ -144,8 +144,8 @@ object DecisionTree extends Serializable with Logging { algo: Algo, impurity: Impurity, maxDepth: Int, - numClassesForClassification: Int): DecisionTreeModel = { - val strategy = new Strategy(algo, impurity, maxDepth, numClassesForClassification) + numClasses: Int): DecisionTreeModel = { + val strategy = new Strategy(algo, impurity, maxDepth, numClasses) new DecisionTree(strategy).run(input) } @@ -164,7 +164,7 @@ object DecisionTree extends Serializable with Logging { * @param impurity criterion used for information gain calculation * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * @param numClassesForClassification number of classes for classification. Default value of 2. + * @param numClasses number of classes for classification. Default value of 2. * @param maxBins maximum number of bins used for splitting features * @param quantileCalculationStrategy algorithm for calculating quantiles * @param categoricalFeaturesInfo Map storing arity of categorical features. @@ -177,11 +177,11 @@ object DecisionTree extends Serializable with Logging { algo: Algo, impurity: Impurity, maxDepth: Int, - numClassesForClassification: Int, + numClasses: Int, maxBins: Int, quantileCalculationStrategy: QuantileStrategy, categoricalFeaturesInfo: Map[Int,Int]): DecisionTreeModel = { - val strategy = new Strategy(algo, impurity, maxDepth, numClassesForClassification, maxBins, + val strategy = new Strategy(algo, impurity, maxDepth, numClasses, maxBins, quantileCalculationStrategy, categoricalFeaturesInfo) new DecisionTree(strategy).run(input) } @@ -191,7 +191,7 @@ object DecisionTree extends Serializable with Logging { * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * Labels should take values {0, 1, ..., numClasses-1}. - * @param numClassesForClassification number of classes for classification. + * @param numClasses number of classes for classification. * @param categoricalFeaturesInfo Map storing arity of categorical features. * E.g., an entry (n -> k) indicates that feature n is categorical * with k categories indexed from 0: {0, 1, ..., k-1}. @@ -206,13 +206,13 @@ object DecisionTree extends Serializable with Logging { */ def trainClassifier( input: RDD[LabeledPoint], - numClassesForClassification: Int, + numClasses: Int, categoricalFeaturesInfo: Map[Int, Int], impurity: String, maxDepth: Int, maxBins: Int): DecisionTreeModel = { val impurityType = Impurities.fromString(impurity) - train(input, Classification, impurityType, maxDepth, numClassesForClassification, maxBins, Sort, + train(input, Classification, impurityType, maxDepth, numClasses, maxBins, Sort, categoricalFeaturesInfo) } @@ -221,12 +221,12 @@ object DecisionTree extends Serializable with Logging { */ def trainClassifier( input: JavaRDD[LabeledPoint], - numClassesForClassification: Int, + numClasses: Int, categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], impurity: String, maxDepth: Int, maxBins: Int): DecisionTreeModel = { - trainClassifier(input.rdd, numClassesForClassification, + trainClassifier(input.rdd, numClasses, categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, impurity, maxDepth, maxBins) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index 482d3395516e7..e9304b5e5c650 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -56,7 +56,7 @@ import org.apache.spark.util.Utils * etc. * @param numTrees If 1, then no bootstrapping is used. If > 1, then bootstrapping is done. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * Supported: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; * if numTrees > 1 (forest) set to "sqrt" for classification and @@ -269,7 +269,7 @@ object RandomForest extends Serializable with Logging { * @param strategy Parameters for training each tree in the forest. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * Supported: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; * if numTrees > 1 (forest) set to "sqrt". @@ -293,13 +293,13 @@ object RandomForest extends Serializable with Logging { * * @param input Training dataset: RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. * Labels should take values {0, 1, ..., numClasses-1}. - * @param numClassesForClassification number of classes for classification. + * @param numClasses number of classes for classification. * @param categoricalFeaturesInfo Map storing arity of categorical features. * E.g., an entry (n -> k) indicates that feature n is categorical * with k categories indexed from 0: {0, 1, ..., k-1}. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * Supported: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; * if numTrees > 1 (forest) set to "sqrt". @@ -315,7 +315,7 @@ object RandomForest extends Serializable with Logging { */ def trainClassifier( input: RDD[LabeledPoint], - numClassesForClassification: Int, + numClasses: Int, categoricalFeaturesInfo: Map[Int, Int], numTrees: Int, featureSubsetStrategy: String, @@ -325,7 +325,7 @@ object RandomForest extends Serializable with Logging { seed: Int = Utils.random.nextInt()): RandomForestModel = { val impurityType = Impurities.fromString(impurity) val strategy = new Strategy(Classification, impurityType, maxDepth, - numClassesForClassification, maxBins, Sort, categoricalFeaturesInfo) + numClasses, maxBins, Sort, categoricalFeaturesInfo) trainClassifier(input, strategy, numTrees, featureSubsetStrategy, seed) } @@ -334,7 +334,7 @@ object RandomForest extends Serializable with Logging { */ def trainClassifier( input: JavaRDD[LabeledPoint], - numClassesForClassification: Int, + numClasses: Int, categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer], numTrees: Int, featureSubsetStrategy: String, @@ -342,7 +342,7 @@ object RandomForest extends Serializable with Logging { maxDepth: Int, maxBins: Int, seed: Int): RandomForestModel = { - trainClassifier(input.rdd, numClassesForClassification, + trainClassifier(input.rdd, numClasses, categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed) } @@ -355,7 +355,7 @@ object RandomForest extends Serializable with Logging { * @param strategy Parameters for training each tree in the forest. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * Supported: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; * if numTrees > 1 (forest) set to "onethird". @@ -384,7 +384,7 @@ object RandomForest extends Serializable with Logging { * with k categories indexed from 0: {0, 1, ..., k-1}. * @param numTrees Number of trees in the random forest. * @param featureSubsetStrategy Number of features to consider for splits at each node. - * Supported: "auto" (default), "all", "sqrt", "log2", "onethird". + * Supported: "auto", "all", "sqrt", "log2", "onethird". * If "auto" is set, this parameter is set based on numTrees: * if numTrees == 1, set to "all"; * if numTrees > 1 (forest) set to "onethird". diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala index e703adbdbfbb3..cf51d041c65a9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/BoostingStrategy.scala @@ -51,7 +51,7 @@ case class BoostingStrategy( private[tree] def assertValid(): Unit = { treeStrategy.algo match { case Classification => - require(treeStrategy.numClassesForClassification == 2, + require(treeStrategy.numClasses == 2, "Only binary classification is supported for boosting.") case Regression => // nothing @@ -80,12 +80,12 @@ object BoostingStrategy { treeStrategy.maxDepth = 3 algo match { case "Classification" => - treeStrategy.numClassesForClassification = 2 + treeStrategy.numClasses = 2 new BoostingStrategy(treeStrategy, LogLoss) case "Regression" => new BoostingStrategy(treeStrategy, SquaredError) case _ => - throw new IllegalArgumentException(s"$algo is not supported by the boosting.") + throw new IllegalArgumentException(s"$algo is not supported by boosting.") } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index d75f38433c081..d5cd89ab94e81 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -37,7 +37,7 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * Supported for Regression: [[org.apache.spark.mllib.tree.impurity.Variance]]. * @param maxDepth Maximum depth of the tree. * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. - * @param numClassesForClassification Number of classes for classification. + * @param numClasses Number of classes for classification. * (Ignored for regression.) * Default value is 2 (binary classification). * @param maxBins Maximum number of bins used for discretizing continuous features and @@ -73,7 +73,7 @@ class Strategy ( @BeanProperty var algo: Algo, @BeanProperty var impurity: Impurity, @BeanProperty var maxDepth: Int, - @BeanProperty var numClassesForClassification: Int = 2, + @BeanProperty var numClasses: Int = 2, @BeanProperty var maxBins: Int = 32, @BeanProperty var quantileCalculationStrategy: QuantileStrategy = Sort, @BeanProperty var categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int](), @@ -86,7 +86,7 @@ class Strategy ( @BeanProperty var checkpointInterval: Int = 10) extends Serializable { def isMulticlassClassification = - algo == Classification && numClassesForClassification > 2 + algo == Classification && numClasses > 2 def isMulticlassWithCategoricalFeatures = isMulticlassClassification && (categoricalFeaturesInfo.size > 0) @@ -97,10 +97,10 @@ class Strategy ( algo: Algo, impurity: Impurity, maxDepth: Int, - numClassesForClassification: Int, + numClasses: Int, maxBins: Int, categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer]) { - this(algo, impurity, maxDepth, numClassesForClassification, maxBins, Sort, + this(algo, impurity, maxDepth, numClasses, maxBins, Sort, categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap) } @@ -117,8 +117,8 @@ class Strategy ( */ def setCategoricalFeaturesInfo( categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer]): Unit = { - setCategoricalFeaturesInfo( - categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap) + this.categoricalFeaturesInfo = + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap } /** @@ -128,9 +128,9 @@ class Strategy ( private[tree] def assertValid(): Unit = { algo match { case Classification => - require(numClassesForClassification >= 2, - s"DecisionTree Strategy for Classification must have numClassesForClassification >= 2," + - s" but numClassesForClassification = $numClassesForClassification.") + require(numClasses >= 2, + s"DecisionTree Strategy for Classification must have numClasses >= 2," + + s" but numClasses = $numClasses.") require(Set(Gini, Entropy).contains(impurity), s"DecisionTree Strategy given invalid impurity for Classification: $impurity." + s" Valid settings: Gini, Entropy") @@ -160,7 +160,7 @@ class Strategy ( /** Returns a shallow copy of this instance. */ def copy: Strategy = { - new Strategy(algo, impurity, maxDepth, numClassesForClassification, maxBins, + new Strategy(algo, impurity, maxDepth, numClasses, maxBins, quantileCalculationStrategy, categoricalFeaturesInfo, minInstancesPerNode, minInfoGain, maxMemoryInMB, subsamplingRate, useNodeIdCache, checkpointDir, checkpointInterval) } @@ -176,9 +176,9 @@ object Strategy { def defaultStrategy(algo: String): Strategy = algo match { case "Classification" => new Strategy(algo = Classification, impurity = Gini, maxDepth = 10, - numClassesForClassification = 2) + numClasses = 2) case "Regression" => new Strategy(algo = Regression, impurity = Variance, maxDepth = 10, - numClassesForClassification = 0) + numClasses = 0) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala index 5bc0f2635c6b1..951733fada6be 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impl/DecisionTreeMetadata.scala @@ -110,7 +110,7 @@ private[tree] object DecisionTreeMetadata extends Logging { val numFeatures = input.take(1)(0).features.size val numExamples = input.count() val numClasses = strategy.algo match { - case Classification => strategy.numClassesForClassification + case Classification => strategy.numClasses case Regression => 0 } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 972c905ec9ffa..9347eaf9221a8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -57,7 +57,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { Classification, Gini, maxDepth = 2, - numClassesForClassification = 2, + numClasses = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) @@ -81,7 +81,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { Classification, Gini, maxDepth = 2, - numClassesForClassification = 2, + numClasses = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) @@ -177,7 +177,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { Classification, Gini, maxDepth = 2, - numClassesForClassification = 100, + numClasses = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) @@ -271,7 +271,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { Classification, Gini, maxDepth = 2, - numClassesForClassification = 100, + numClasses = 100, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1-> 10)) // 2^(10-1) - 1 > 100, so categorical features will be ordered @@ -295,7 +295,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val strategy = new Strategy( Classification, Gini, - numClassesForClassification = 2, + numClasses = 2, maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) @@ -377,7 +377,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, - numClassesForClassification = 2, maxBins = 100) + numClasses = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -401,7 +401,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, - numClassesForClassification = 2, maxBins = 100) + numClasses = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -426,7 +426,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, - numClassesForClassification = 2, maxBins = 100) + numClasses = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -451,7 +451,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, - numClassesForClassification = 2, maxBins = 100) + numClasses = 2, maxBins = 100) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -485,7 +485,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { // Train a 1-node model val strategyOneNode = new Strategy(Classification, Entropy, maxDepth = 1, - numClassesForClassification = 2, maxBins = 100) + numClasses = 2, maxBins = 100) val modelOneNode = DecisionTree.train(rdd, strategyOneNode) val rootNode1 = modelOneNode.topNode.deepCopy() val rootNode2 = modelOneNode.topNode.deepCopy() @@ -545,7 +545,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) + numClasses = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(strategy.isMulticlassClassification) assert(metadata.isUnordered(featureIndex = 0)) @@ -568,7 +568,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { arr(3) = new LabeledPoint(1.0, Vectors.dense(3.0)) val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 2) + numClasses = 2) val model = DecisionTree.train(rdd, strategy) DecisionTreeSuite.validateClassifier(model, arr, 1.0) @@ -585,7 +585,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 2) + numClasses = 2) val model = DecisionTree.train(rdd, strategy) DecisionTreeSuite.validateClassifier(model, arr, 1.0) @@ -600,7 +600,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, maxBins = maxBins, + numClasses = 3, maxBins = maxBins, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) @@ -629,7 +629,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, maxBins = 100) + numClasses = 3, maxBins = 100) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) @@ -650,7 +650,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateContinuousDataPointsForMulticlass() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) + numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) assert(metadata.isUnordered(featureIndex = 0)) @@ -671,7 +671,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, maxBins = 100, + numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) val metadata = DecisionTreeMetadata.buildMetadata(rdd, strategy) @@ -692,7 +692,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val arr = DecisionTreeSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, - numClassesForClassification = 3, maxBins = 10, + numClasses = 3, maxBins = 10, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) @@ -708,7 +708,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, - maxDepth = 2, numClassesForClassification = 2, minInstancesPerNode = 2) + maxDepth = 2, numClasses = 2, minInstancesPerNode = 2) val model = DecisionTree.train(rdd, strategy) assert(model.topNode.isLeaf) @@ -737,7 +737,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxBins = 2, maxDepth = 2, categoricalFeaturesInfo = Map(0 -> 2, 1-> 2), - numClassesForClassification = 2, minInstancesPerNode = 2) + numClasses = 2, minInstancesPerNode = 2) val rootNode = DecisionTree.train(rdd, strategy).topNode @@ -755,7 +755,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val input = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, minInfoGain = 1.0) + numClasses = 2, minInfoGain = 1.0) val model = DecisionTree.train(input, strategy) assert(model.topNode.isLeaf) @@ -781,7 +781,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val input = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 1, - numClassesForClassification = 2, categoricalFeaturesInfo = Map(0 -> 3)) + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) @@ -824,7 +824,7 @@ class DecisionTreeSuite extends FunSuite with MLlibTestSparkContext { val input = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, - numClassesForClassification = 2, categoricalFeaturesInfo = Map(0 -> 3)) + numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) val metadata = DecisionTreeMetadata.buildMetadata(input, strategy) val (splits, bins) = DecisionTree.findSplitsBins(input, metadata) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala index d4d54cf4c9e2a..3aa97e544680b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/GradientBoostedTreesSuite.scala @@ -100,7 +100,7 @@ class GradientBoostedTreesSuite extends FunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(GradientBoostedTreesSuite.data, 2) val treeStrategy = new Strategy(algo = Classification, impurity = Variance, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = Map.empty, + numClasses = 2, categoricalFeaturesInfo = Map.empty, subsamplingRate = subsamplingRate) val boostingStrategy = new BoostingStrategy(treeStrategy, LogLoss, numIterations, learningRate) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala index 90a8c2dfdab80..f7f0f20c6c125 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/RandomForestSuite.scala @@ -57,7 +57,7 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) binaryClassificationTestWithContinuousFeatures(strategy) } @@ -65,7 +65,7 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, + numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, useNodeIdCache = true) binaryClassificationTestWithContinuousFeatures(strategy) } @@ -93,7 +93,7 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Regression, impurity = Variance, - maxDepth = 2, maxBins = 10, numClassesForClassification = 2, + maxDepth = 2, maxBins = 10, numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) regressionTestWithContinuousFeatures(strategy) } @@ -102,7 +102,7 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { " comparing DecisionTree vs. RandomForest(numTrees = 1)") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Regression, impurity = Variance, - maxDepth = 2, maxBins = 10, numClassesForClassification = 2, + maxDepth = 2, maxBins = 10, numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, useNodeIdCache = true) regressionTestWithContinuousFeatures(strategy) } @@ -169,14 +169,14 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { test("Binary classification with continuous features: subsampling features") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) + numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo) binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy) } test("Binary classification with continuous features and node Id cache: subsampling features") { val categoricalFeaturesInfo = Map.empty[Int, Int] val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 2, - numClassesForClassification = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, + numClasses = 2, categoricalFeaturesInfo = categoricalFeaturesInfo, useNodeIdCache = true) binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy) } @@ -191,7 +191,7 @@ class RandomForestSuite extends FunSuite with MLlibTestSparkContext { val input = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 5, - numClassesForClassification = 3, categoricalFeaturesInfo = categoricalFeaturesInfo) + numClasses = 3, categoricalFeaturesInfo = categoricalFeaturesInfo) val model = RandomForest.trainClassifier(input, strategy, numTrees = 2, featureSubsetStrategy = "sqrt", seed = 12345) EnsembleTestHelper.validateClassifier(model, arr, 1.0) diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index 46e253991aa56..66702478474dc 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -250,7 +250,7 @@ def _train(cls, data, algo, numClasses, categoricalFeaturesInfo, numTrees, return RandomForestModel(model) @classmethod - def trainClassifier(cls, data, numClassesForClassification, categoricalFeaturesInfo, numTrees, + def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, featureSubsetStrategy="auto", impurity="gini", maxDepth=4, maxBins=32, seed=None): """ @@ -259,7 +259,7 @@ def trainClassifier(cls, data, numClassesForClassification, categoricalFeaturesI :param data: Training dataset: RDD of LabeledPoint. Labels should take values {0, 1, ..., numClasses-1}. - :param numClassesForClassification: number of classes for classification. + :param numClasses: number of classes for classification. :param categoricalFeaturesInfo: Map storing arity of categorical features. E.g., an entry (n -> k) indicates that feature n is categorical with k categories indexed from 0: {0, 1, ..., k-1}. @@ -320,7 +320,7 @@ def trainClassifier(cls, data, numClassesForClassification, categoricalFeaturesI >>> model.predict(rdd).collect() [1.0, 0.0] """ - return cls._train(data, "classification", numClassesForClassification, + return cls._train(data, "classification", numClasses, categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, seed) From a4dfb4efef89f686cbf146db42c2d891fef42500 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 3 Dec 2014 19:08:29 -0800 Subject: [PATCH 113/305] [Release] Correctly translate contributors name in release notes This commit involves three main changes: (1) It separates the translation of contributor names from the generation of the contributors list. This is largely motivated by the Github API limit; even if we exceed this limit, we should at least be able to proceed manually as before. This is why the translation logic is abstracted into its own script translate-contributors.py. (2) When we look for candidate replacements for invalid author names, we should look for the assignees of the associated JIRAs too. As a result, the intermediate file must keep track of these. (3) This provides an interactive mode with which the user can sit at the terminal and manually pick the candidate replacement that he/she thinks makes the most sense. As before, there is a non-interactive mode that picks the first candidate that the script considers "valid." TODO: We should have a known_contributors file that stores known mappings so we don't have to go through all of this translation every time. This is also valuable because some contributors simply cannot be automatically translated. --- .gitignore | 3 + dev/create-release/generate-contributors.py | 52 +++-- dev/create-release/releaseutils.py | 39 +--- dev/create-release/translate-contributors.py | 190 +++++++++++++++++++ 4 files changed, 229 insertions(+), 55 deletions(-) create mode 100755 dev/create-release/translate-contributors.py diff --git a/.gitignore b/.gitignore index c67cffa1c4375..3b9086c7187dc 100644 --- a/.gitignore +++ b/.gitignore @@ -5,6 +5,7 @@ *.ipr *.iml *.iws +*.pyc .idea/ .idea_modules/ sbt/*.jar @@ -49,6 +50,8 @@ dependency-reduced-pom.xml checkpoint derby.log dist/ +dev/create-release/*txt +dev/create-release/*new spark-*-bin-*.tgz unit-tests.log /lib/ diff --git a/dev/create-release/generate-contributors.py b/dev/create-release/generate-contributors.py index 99c29ef9ff8b6..a3b78a3eac6d0 100755 --- a/dev/create-release/generate-contributors.py +++ b/dev/create-release/generate-contributors.py @@ -26,8 +26,6 @@ # You must set the following before use! JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") -JIRA_USERNAME = os.environ.get("JIRA_USERNAME", None) -JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", None) START_COMMIT = os.environ.get("START_COMMIT", "37b100") END_COMMIT = os.environ.get("END_COMMIT", "3693ae") @@ -40,8 +38,6 @@ END_COMMIT = raw_input("Please specify ending commit hash (non-inclusive): ") # Verify provided arguments -if not JIRA_USERNAME: sys.exit("JIRA_USERNAME must be provided") -if not JIRA_PASSWORD: sys.exit("JIRA_PASSWORD must be provided") start_commit_line = get_one_line(START_COMMIT) end_commit_line = get_one_line(END_COMMIT) num_commits = num_commits_in_range(START_COMMIT, END_COMMIT) @@ -60,14 +56,6 @@ sys.exit("Ok, exiting") print "==================================================================================\n" -# Setup JIRA and github clients. We use two JIRA clients, one with authentication -# and one without, because authentication is slow and required only when we query -# JIRA user details but not Spark issues -jira_options = { "server": JIRA_API_BASE } -jira_client = JIRA(options = jira_options) -jira_client_auth = JIRA(options = jira_options, basic_auth = (JIRA_USERNAME, JIRA_PASSWORD)) -github_client = Github() - # Find all commits within this range print "Gathering commits within range [%s..%s)" % (START_COMMIT, END_COMMIT) commits = get_one_line_commits(START_COMMIT, END_COMMIT) @@ -105,13 +93,17 @@ def print_indented(_list): if reverts: print "Reverts (%d)" % len(reverts); print_indented(reverts) if nojiras: print "No JIRA (%d)" % len(nojiras); print_indented(nojiras) print "==================== Warning: the above commits will be ignored ==================\n" -response = raw_input("%d commits left to process. Ok to proceed? [y/N] " % len(filtered_commits)) -if response.lower() != "y": +response = raw_input("%d commits left to process. Ok to proceed? [Y/n] " % len(filtered_commits)) +if response.lower() != "y" and response: sys.exit("Ok, exiting.") # Keep track of warnings to tell the user at the end warnings = [] +# Mapping from the invalid author name to its associated JIRA issues +# E.g. andrewor14 -> set("SPARK-2413", "SPARK-3551", "SPARK-3471") +invalid_authors = {} + # Populate a map that groups issues and components by author # It takes the form: Author name -> { Contribution type -> Spark components } # For instance, @@ -127,16 +119,23 @@ def print_indented(_list): # } # author_info = {} +jira_options = { "server": JIRA_API_BASE } +jira_client = JIRA(options = jira_options) print "\n=========================== Compiling contributor list ===========================" for commit in filtered_commits: commit_hash = re.findall("^[a-z0-9]+", commit)[0] issues = re.findall("SPARK-[0-9]+", commit.upper()) - # Translate the author in case the github username is not an actual name - # Also guard against any special characters used in the name - # Note the JIRA client we use here must have authentication enabled author = get_author(commit_hash) - author = unidecode.unidecode(unicode(author, "UTF-8")) - author = translate_author(author, github_client, jira_client_auth, warnings) + author = unidecode.unidecode(unicode(author, "UTF-8")).strip() # guard against special characters + # If the author name is invalid, keep track of it along + # with all associated issues so we can translate it later + if is_valid_author(author): + author = capitalize_author(author) + else: + if author not in invalid_authors: + invalid_authors[author] = set() + for issue in issues: + invalid_authors[author].add(issue) date = get_date(commit_hash) # Parse components from the commit message, if any commit_components = find_components(commit, commit_hash) @@ -147,7 +146,7 @@ def populate(issue_type, components): author_info[author] = {} if issue_type not in author_info[author]: author_info[author][issue_type] = set() - for component in all_components: + for component in components: author_info[author][issue_type].add(component) # Find issues and components associated with this commit for issue in issues: @@ -168,7 +167,6 @@ def populate(issue_type, components): # Each line takes the format "Author name - semi-colon delimited contributions" # e.g. Andrew Or - Bug fixes in Windows, Core, and Web UI; improvements in Core # e.g. Tathagata Das - Bug fixes and new features in Streaming -contributors_file_name = "contributors.txt" contributors_file = open(contributors_file_name, "w") authors = author_info.keys() authors.sort() @@ -192,11 +190,23 @@ def populate(issue_type, components): # Do not use python's capitalize() on the whole string to preserve case assert contribution contribution = contribution[0].capitalize() + contribution[1:] + # If the author name is invalid, use an intermediate format that + # can be translated through translate-contributors.py later + # E.g. andrewor14/SPARK-3425/SPARK-1157/SPARK-6672 + if author in invalid_authors and invalid_authors[author]: + author = author + "/" + "/".join(invalid_authors[author]) line = "%s - %s" % (author, contribution) contributors_file.write(line + "\n") contributors_file.close() print "Contributors list is successfully written to %s!" % contributors_file_name +# Prompt the user to translate author names if necessary +if invalid_authors: + warnings.append("Found the following invalid authors:") + for a in invalid_authors: + warnings.append("\t%s" % a) + warnings.append("Please run './translate-contributors.py' to translate them.") + # Log any warnings encountered in the process if warnings: print "\n============ Warnings encountered while creating the contributor list ============" diff --git a/dev/create-release/releaseutils.py b/dev/create-release/releaseutils.py index 0d6830b11dc73..76a10c32886d4 100755 --- a/dev/create-release/releaseutils.py +++ b/dev/create-release/releaseutils.py @@ -44,6 +44,9 @@ print "Install using 'sudo pip install unidecode'" sys.exit(-1) +# Contributors list file name +contributors_file_name = "contributors.txt" + # Utility functions run git commands (written with Git 1.8.5) def run_cmd(cmd): return Popen(cmd, stdout=PIPE).communicate()[0] def get_author(commit_hash): @@ -69,7 +72,8 @@ def num_commits_in_range(start_hash, end_hash): "build": "build fixes", "improvement": "improvements", "new feature": "new features", - "documentation": "documentation" + "documentation": "documentation", + "test": "test" } # Maintain a mapping for translating component names when creating the release notes @@ -182,36 +186,3 @@ def capitalize_author(author): words = [w[0].capitalize() + w[1:] for w in words if w] return " ".join(words) -# Maintain a mapping of translated author names as a cache -translated_authors = {} - -# Format the given author in a format appropriate for the contributors list. -# If the author is not an actual name, search github and JIRA for potential -# replacements and log all candidates as a warning. -def translate_author(github_author, github_client, jira_client, warnings): - if is_valid_author(github_author): - return capitalize_author(github_author) - # If the translated author is already cached, just return it - if github_author in translated_authors: - return translated_authors[github_author] - # Otherwise, author name is not found, so we need to search for an alternative name - candidates = set() - github_name = get_github_name(github_author, github_client) - jira_name = get_jira_name(github_author, jira_client) - if is_valid_author(github_name): github_name = capitalize_author(github_name) - if is_valid_author(jira_name): jira_name = capitalize_author(jira_name) - if github_name: candidates.add(github_name) - if jira_name: candidates.add(jira_name) - # Only use the github name as a replacement automatically - # The JIRA name may not make sense because it can belong to someone else - if is_valid_author(github_name): - candidates_message = " (another candidate is %s)" % jira_name if jira_name else "" - warnings.append("Replacing github user %s with %s%s" % (github_author, github_name, candidates_message)) - translated_authors[github_name] = github_name - return translated_authors[github_name] - # No direct replacement, so return the original author and list any candidates found - candidates_message = " (candidates: %s)" % nice_join(candidates) if candidates else "" - warnings.append("Unable to find a replacement for github user %s%s" % (github_author, candidates_message)) - translated_authors[github_author] = github_author - return translated_authors[github_author] - diff --git a/dev/create-release/translate-contributors.py b/dev/create-release/translate-contributors.py new file mode 100755 index 0000000000000..ef4625b003cb6 --- /dev/null +++ b/dev/create-release/translate-contributors.py @@ -0,0 +1,190 @@ +#!/usr/bin/env python +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +# This script translates invalid authors in the contributors list generated +# by generate-contributors.py. When the script encounters an author name that +# is considered invalid, it searches Github and JIRA in an attempt to search +# for replacements. This tool runs in two modes: +# +# (1) Interactive mode: For each invalid author name, this script presents +# all candidate replacements to the user and awaits user response. In this +# mode, the user may also input a custom name. This is the default. +# +# (2) Non-interactive mode: For each invalid author name, this script replaces +# the name with the first valid candidate it can find. If there is none, it +# uses the original name. This can be enabled through the --non-interactive flag. + +import os +import sys + +from releaseutils import * + +# You must set the following before use! +JIRA_API_BASE = os.environ.get("JIRA_API_BASE", "https://issues.apache.org/jira") +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", None) +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", None) +if not JIRA_USERNAME or not JIRA_PASSWORD: + sys.exit("Both JIRA_USERNAME and JIRA_PASSWORD must be set") + +# Write new contributors list to .new +if not os.path.isfile(contributors_file_name): + print "Contributors file %s does not exist!" % contributors_file_name + print "Have you run ./generate-contributors.py yet?" + sys.exit(1) +contributors_file = open(contributors_file_name, "r") +new_contributors_file_name = contributors_file_name + ".new" +new_contributors_file = open(new_contributors_file_name, "w") +warnings = [] + +# In non-interactive mode, this script will choose the first replacement that is valid +INTERACTIVE_MODE = True +if len(sys.argv) > 1: + options = set(sys.argv[1:]) + if "--non-interactive" in options: + INTERACTIVE_MODE = False +if INTERACTIVE_MODE: + print "Running in interactive mode. To disable this, provide the --non-interactive flag." + +# Setup Github and JIRA clients +jira_options = { "server": JIRA_API_BASE } +jira_client = JIRA(options = jira_options, basic_auth = (JIRA_USERNAME, JIRA_PASSWORD)) +github_client = Github() + +# Generate candidates for the given author. This should only be called if the given author +# name does not represent a full name as this operation is somewhat expensive. Under the +# hood, it makes several calls to the Github and JIRA API servers to find the candidates. +# +# This returns a list of (candidate name, source) 2-tuples. E.g. +# [ +# (NOT_FOUND, "No full name found for Github user andrewor14"), +# ("Andrew Or", "Full name of JIRA user andrewor14"), +# ("Andrew Orso", "Full name of SPARK-1444 assignee andrewor14"), +# ("Andrew Ordall", "Full name of SPARK-1663 assignee andrewor14"), +# (NOT_FOUND, "No assignee found for SPARK-1763") +# ] +NOT_FOUND = "Not found" +def generate_candidates(author, issues): + candidates = [] + # First check for full name of Github user + github_name = get_github_name(new_author, github_client) + if github_name: + candidates.append((github_name, "Full name of Github user %s" % new_author)) + else: + candidates.append((NOT_FOUND, "No full name found for Github user %s" % new_author)) + # Then do the same for JIRA user + jira_name = get_jira_name(new_author, jira_client) + if jira_name: + candidates.append((jira_name, "Full name of JIRA user %s" % new_author)) + else: + candidates.append((NOT_FOUND, "No full name found for JIRA user %s" % new_author)) + # Then do the same for the assignee of each of the associated JIRAs + # Note that a given issue may not have an assignee, or the assignee may not have a full name + for issue in issues: + jira_issue = jira_client.issue(issue) + jira_assignee = jira_issue.fields.assignee + if jira_assignee: + user_name = jira_assignee.name + display_name = jira_assignee.displayName + if display_name: + candidates.append((display_name, "Full name of %s assignee %s" % (issue, user_name))) + else: + candidates.append((NOT_FOUND, "No full name found for %s assignee %" % (issue, user_name))) + else: + candidates.append((NOT_FOUND, "No assignee found for %s" % issue)) + # Guard against special characters in candidate names + # Note that the candidate name may already be in unicode (JIRA returns this) + for i, (candidate, source) in enumerate(candidates): + try: + candidate = unicode(candidate, "UTF-8") + except TypeError: + # already in unicode + pass + candidate = unidecode.unidecode(candidate).strip() + candidates[i] = (candidate, source) + return candidates + +# Translate each invalid author by searching for possible candidates from Github and JIRA +# In interactive mode, this script presents the user with a list of choices and have the user +# select from this list. Additionally, the user may also choose to enter a custom name. +# In non-interactive mode, this script picks the first valid author name from the candidates +# If no such name exists, the original name is used (without the JIRA numbers). +print "\n========================== Translating contributor list ==========================" +for line in contributors_file: + author = line.split(" - ")[0] + print "Processing author %s" % author + if not author: + print " ERROR: Expected the following format - " + print " ERROR: Actual = %s" % line + if not is_valid_author(author): + new_author = author.split("/")[0] + issues = author.split("/")[1:] + candidates = generate_candidates(new_author, issues) + # Print out potential replacement candidates along with the sources, e.g. + # [X] No full name found for Github user andrewor14 + # [0] Andrew Or - Full name of JIRA user andrewor14 + # [1] Andrew Orso - Full name of SPARK-1444 assignee andrewor14 + # [2] Andrew Ordall - Full name of SPARK-1663 assignee andrewor14 + # [X] No assignee found for SPARK-1763 + # [3] Custom + candidate_names = [] + for candidate, source in candidates: + if candidate == NOT_FOUND: + print " [X] %s" % source + else: + index = len(candidate_names) + candidate_names.append(candidate) + print " [%d] %s - %s" % (index, candidate, source) + custom_index = len(candidate_names) + # In interactive mode, additionally provide "custom" option and await user response + if INTERACTIVE_MODE: + print " [%d] Custom" % custom_index + response = raw_input(" Your choice: ") + while not response.isdigit() or int(response) > custom_index: + response = raw_input(" Please enter an integer between 0 and %d: " % custom_index) + response = int(response) + if response == custom_index: + new_author = raw_input(" Please type a custom name for this author: ") + else: + new_author = candidate_names[response] + # In non-interactive mode, just pick the first candidate + else: + valid_candidate_names = [name for name, _ in candidates\ + if is_valid_author(name) and name != NOT_FOUND] + if valid_candidate_names: + new_author = valid_candidate_names[0] + # Finally, capitalize the author and replace the original one with it + # If the final replacement is still invalid, log a warning + if is_valid_author(new_author): + new_author = capitalize_author(new_author) + else: + warnings.append("Unable to find a valid name %s for author %s" % (new_author, author)) + print " * Replacing %s with %s" % (author, new_author) + line = line.replace(author, new_author) + new_contributors_file.write(line) +print "==================================================================================\n" +contributors_file.close() +new_contributors_file.close() + +print "Translated contributors list successfully written to %s!" % new_contributors_file_name + +# Log any warnings encountered in the process +if warnings: + print "\n========== Warnings encountered while translating the contributor list ===========" + for w in warnings: print w + print "Please manually correct these in the final contributors list at %s." % new_contributors_file_name + print "==================================================================================\n" + From 3cdae038f1c233647e074858a6b7b710edf15d78 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 3 Dec 2014 22:15:46 -0800 Subject: [PATCH 114/305] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #1875 (close requested by 'marmbrus') Closes #3566 (close requested by 'andrewor14') Closes #3487 (close requested by 'pwendell') From ed88db4cb21d029ca14ebc428fae122adf5128f0 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Thu, 4 Dec 2014 00:43:55 -0800 Subject: [PATCH 115/305] [SQL] remove unnecessary import Author: Jacky Li Closes #3585 from jackylk/remove and squashes the following commits: 045423d [Jacky Li] remove unnecessary import --- sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index f83e647014193..bcebce7603f13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -22,7 +22,6 @@ import java.util.TimeZone import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan /* Implicits */ From c3ad48603632a039a51be3d33e917105149fdd7a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 4 Dec 2014 00:45:57 -0800 Subject: [PATCH 116/305] [SPARK-4719][API] Consolidate various narrow dep RDD classes with MapPartitionsRDD MappedRDD, MappedValuesRDD, FlatMappedValuesRDD, FilteredRDD, GlommedRDD, FlatMappedRDD are not necessary. They can be implemented trivially using MapPartitionsRDD. Author: Reynold Xin Closes #3578 from rxin/SPARK-4719 and squashes the following commits: eed9853 [Reynold Xin] Preserve partitioning for filter. eb1a89b [Reynold Xin] [SPARK-4719][API] Consolidate various narrow dep RDD classes with MapPartitionsRDD. --- .../org/apache/spark/rdd/BinaryFileRDD.scala | 12 ++--- .../org/apache/spark/rdd/FilteredRDD.scala | 35 ------------- .../org/apache/spark/rdd/FlatMappedRDD.scala | 34 ------------- .../spark/rdd/FlatMappedValuesRDD.scala | 35 ------------- .../org/apache/spark/rdd/GlommedRDD.scala | 31 ------------ .../org/apache/spark/rdd/MappedRDD.scala | 32 ------------ .../apache/spark/rdd/MappedValuesRDD.scala | 33 ------------ .../apache/spark/rdd/PairRDDFunctions.scala | 10 +++- .../main/scala/org/apache/spark/rdd/RDD.scala | 28 ++++++++--- .../scala/org/apache/spark/rdd/RDDSuite.scala | 50 ++++++++----------- 10 files changed, 55 insertions(+), 245 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala delete mode 100644 core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala delete mode 100644 core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala delete mode 100644 core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala delete mode 100644 core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala delete mode 100644 core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index 6e66ddbdef788..1f755db485812 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -24,12 +24,12 @@ import org.apache.spark.input.StreamFileInputFormat import org.apache.spark.{ Partition, SparkContext } private[spark] class BinaryFileRDD[T]( - sc: SparkContext, - inputFormatClass: Class[_ <: StreamFileInputFormat[T]], - keyClass: Class[String], - valueClass: Class[T], - @transient conf: Configuration, - minPartitions: Int) + sc: SparkContext, + inputFormatClass: Class[_ <: StreamFileInputFormat[T]], + keyClass: Class[String], + valueClass: Class[T], + @transient conf: Configuration, + minPartitions: Int) extends NewHadoopRDD[String, T](sc, inputFormatClass, keyClass, valueClass, conf) { override def getPartitions: Array[Partition] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala deleted file mode 100644 index 9e41b3d1e2d4f..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/FilteredRDD.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import scala.reflect.ClassTag - -import org.apache.spark.{Partition, TaskContext} - -private[spark] class FilteredRDD[T: ClassTag]( - prev: RDD[T], - f: T => Boolean) - extends RDD[T](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override val partitioner = prev.partitioner // Since filter cannot change a partition's keys - - override def compute(split: Partition, context: TaskContext) = - firstParent[T].iterator(split, context).filter(f) -} diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala deleted file mode 100644 index d8f87d4e3690e..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedRDD.scala +++ /dev/null @@ -1,34 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import scala.reflect.ClassTag - -import org.apache.spark.{Partition, TaskContext} - -private[spark] -class FlatMappedRDD[U: ClassTag, T: ClassTag]( - prev: RDD[T], - f: T => TraversableOnce[U]) - extends RDD[U](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override def compute(split: Partition, context: TaskContext) = - firstParent[T].iterator(split, context).flatMap(f) -} diff --git a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala deleted file mode 100644 index 7c9023f62d3b6..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/FlatMappedValuesRDD.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import org.apache.spark.{Partition, TaskContext} - -private[spark] -class FlatMappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => TraversableOnce[U]) - extends RDD[(K, U)](prev) { - - override def getPartitions = firstParent[Product2[K, V]].partitions - - override val partitioner = firstParent[Product2[K, V]].partitioner - - override def compute(split: Partition, context: TaskContext) = { - firstParent[Product2[K, V]].iterator(split, context).flatMap { case Product2(k, v) => - f(v).map(x => (k, x)) - } - } -} diff --git a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala deleted file mode 100644 index f6463fa715a71..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/GlommedRDD.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import scala.reflect.ClassTag - -import org.apache.spark.{Partition, TaskContext} - -private[spark] class GlommedRDD[T: ClassTag](prev: RDD[T]) - extends RDD[Array[T]](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override def compute(split: Partition, context: TaskContext) = - Array(firstParent[T].iterator(split, context).toArray).iterator -} diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala deleted file mode 100644 index 8d7c288593665..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/MappedRDD.scala +++ /dev/null @@ -1,32 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import scala.reflect.ClassTag - -import org.apache.spark.{Partition, TaskContext} - -private[spark] -class MappedRDD[U: ClassTag, T: ClassTag](prev: RDD[T], f: T => U) - extends RDD[U](prev) { - - override def getPartitions: Array[Partition] = firstParent[T].partitions - - override def compute(split: Partition, context: TaskContext) = - firstParent[T].iterator(split, context).map(f) -} diff --git a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala b/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala deleted file mode 100644 index a60952eee5901..0000000000000 --- a/core/src/main/scala/org/apache/spark/rdd/MappedValuesRDD.scala +++ /dev/null @@ -1,33 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.rdd - -import org.apache.spark.{Partition, TaskContext} - -private[spark] -class MappedValuesRDD[K, V, U](prev: RDD[_ <: Product2[K, V]], f: V => U) - extends RDD[(K, U)](prev) { - - override def getPartitions = firstParent[Product2[K, U]].partitions - - override val partitioner = firstParent[Product2[K, U]].partitioner - - override def compute(split: Partition, context: TaskContext): Iterator[(K, U)] = { - firstParent[Product2[K, V]].iterator(split, context).map { pair => (pair._1, f(pair._2)) } - } -} diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index e78e57678852f..c43e1f2fe135e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -660,7 +660,9 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def mapValues[U](f: V => U): RDD[(K, U)] = { val cleanF = self.context.clean(f) - new MappedValuesRDD(self, cleanF) + new MapPartitionsRDD[(K, U), (K, V)](self, + (context, pid, iter) => iter.map { case (k, v) => (k, cleanF(v)) }, + preservesPartitioning = true) } /** @@ -669,7 +671,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) */ def flatMapValues[U](f: V => TraversableOnce[U]): RDD[(K, U)] = { val cleanF = self.context.clean(f) - new FlatMappedValuesRDD(self, cleanF) + new MapPartitionsRDD[(K, U), (K, V)](self, + (context, pid, iter) => iter.flatMap { case (k, v) => + cleanF(v).map(x => (k, x)) + }, + preservesPartitioning = true) } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 8dfd952298f30..0bd616ec24fcb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -17,7 +17,7 @@ package org.apache.spark.rdd -import java.util.{Properties, Random} +import java.util.Random import scala.collection.{mutable, Map} import scala.collection.mutable.ArrayBuffer @@ -36,13 +36,12 @@ import org.apache.spark._ import org.apache.spark.Partitioner._ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD -import org.apache.spark.broadcast.Broadcast import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{BoundedPriorityQueue, Utils, CallSite} +import org.apache.spark.util.{BoundedPriorityQueue, Utils} import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler, BernoulliCellSampler, SamplingUtils} @@ -270,19 +269,30 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD by applying a function to all elements of this RDD. */ - def map[U: ClassTag](f: T => U): RDD[U] = new MappedRDD(this, sc.clean(f)) + def map[U: ClassTag](f: T => U): RDD[U] = { + val cleanF = sc.clean(f) + new MapPartitionsRDD[U, T](this, (context, pid, iter) => iter.map(cleanF)) + } /** * Return a new RDD by first applying a function to all elements of this * RDD, and then flattening the results. */ - def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] = - new FlatMappedRDD(this, sc.clean(f)) + def flatMap[U: ClassTag](f: T => TraversableOnce[U]): RDD[U] = { + val cleanF = sc.clean(f) + new MapPartitionsRDD[U, T](this, (context, pid, iter) => iter.flatMap(cleanF)) + } /** * Return a new RDD containing only the elements that satisfy a predicate. */ - def filter(f: T => Boolean): RDD[T] = new FilteredRDD(this, sc.clean(f)) + def filter(f: T => Boolean): RDD[T] = { + val cleanF = sc.clean(f) + new MapPartitionsRDD[T, T]( + this, + (context, pid, iter) => iter.filter(cleanF), + preservesPartitioning = true) + } /** * Return a new RDD containing the distinct elements in this RDD. @@ -503,7 +513,9 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD created by coalescing all elements within each partition into an array. */ - def glom(): RDD[Array[T]] = new GlommedRDD(this) + def glom(): RDD[Array[T]] = { + new MapPartitionsRDD[Array[T], T](this, (context, pid, iter) => Iterator(iter.toArray)) + } /** * Return the Cartesian product of this RDD and another one, that is, the RDD of all pairs of diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 6d9be796144b6..46fcb80fa1845 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -24,10 +24,9 @@ import scala.reflect.ClassTag import org.scalatest.FunSuite import org.apache.spark._ -import org.apache.spark.util.Utils - import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.rdd.RDDSuiteUtils._ +import org.apache.spark.util.Utils class RDDSuite extends FunSuite with SharedSparkContext { @@ -37,8 +36,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(nums.toLocalIterator.toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) assert(dups.distinct().count() === 4) - assert(dups.distinct.count === 4) // Can distinct and count be called without parentheses? - assert(dups.distinct.collect === dups.distinct().collect) + assert(dups.distinct().count === 4) // Can distinct and count be called without parentheses? + assert(dups.distinct().collect === dups.distinct().collect) assert(dups.distinct(2).collect === dups.distinct().collect) assert(nums.reduce(_ + _) === 10) assert(nums.fold(0)(_ + _) === 10) @@ -617,9 +616,9 @@ class RDDSuite extends FunSuite with SharedSparkContext { for(seed <- 1 to 5) { val splits = data.randomSplit(Array(1.0, 2.0, 3.0), seed) assert(splits.size == 3, "wrong number of splits") - assert(splits.flatMap(_.collect).sorted.toList == data.collect.toList, + assert(splits.flatMap(_.collect()).sorted.toList == data.collect().toList, "incomplete or wrong split") - val s = splits.map(_.count) + val s = splits.map(_.count()) assert(math.abs(s(0) - 100) < 50) // std = 9.13 assert(math.abs(s(1) - 200) < 50) // std = 11.55 assert(math.abs(s(2) - 300) < 50) // std = 12.25 @@ -762,8 +761,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val rdd3 = rdd2.map(_ + 1) val rdd4 = new UnionRDD(sc, List(rdd1, rdd2, rdd3)) assert(rdd4.parent(0).isInstanceOf[ParallelCollectionRDD[_]]) - assert(rdd4.parent(1).isInstanceOf[FilteredRDD[_]]) - assert(rdd4.parent(2).isInstanceOf[MappedRDD[_, _]]) + assert(rdd4.parent[Int](1) === rdd2) + assert(rdd4.parent[Int](2) === rdd3) } test("getNarrowAncestors") { @@ -781,20 +780,18 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Simple dependency tree with a single branch assert(ancestors1.size === 0) assert(ancestors2.size === 2) - assert(ancestors2.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) - assert(ancestors2.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors2.count(_ === rdd1) === 1) + assert(ancestors2.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 1) assert(ancestors3.size === 5) - assert(ancestors3.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) - assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) - assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + assert(ancestors3.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 4) // Any ancestors before the shuffle are not considered assert(ancestors4.size === 0) assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 0) assert(ancestors5.size === 3) assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) - assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 0) - assert(ancestors5.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 2) + assert(ancestors5.count(_ === rdd3) === 0) + assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 2) } test("getNarrowAncestors with multiple parents") { @@ -815,16 +812,16 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Simple dependency tree with multiple branches assert(ancestors6.size === 3) assert(ancestors6.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 2) - assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors6.count(_ === rdd2) === 1) assert(ancestors7.size === 5) assert(ancestors7.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) - assert(ancestors7.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) - assert(ancestors7.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors7.count(_ === rdd2) === 1) + assert(ancestors7.count(_ === rdd3) === 1) // Dependency tree with duplicate nodes (e.g. rdd1 should not be reported twice) assert(ancestors8.size === 7) - assert(ancestors8.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) - assert(ancestors8.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors8.count(_ === rdd2) === 1) + assert(ancestors8.count(_ === rdd3) === 1) assert(ancestors8.count(_.isInstanceOf[UnionRDD[_]]) === 2) assert(ancestors8.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) assert(ancestors8.count(_ == rdd1) === 1) @@ -834,7 +831,6 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Any ancestors before the shuffle are not considered assert(ancestors9.size === 2) assert(ancestors9.count(_.isInstanceOf[CoGroupedRDD[_]]) === 1) - assert(ancestors9.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 1) } /** @@ -868,12 +864,10 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ancestors3 = rdd3.getNarrowAncestors val ancestors4 = rdd4.getNarrowAncestors assert(ancestors3.size === 4) - assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) - assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors3.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 4) assert(ancestors3.count(_ == rdd3) === 0) assert(ancestors4.size === 4) - assert(ancestors4.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) - assert(ancestors4.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors4.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 3) assert(ancestors4.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) assert(ancestors4.count(_ == rdd3) === 1) assert(ancestors4.count(_ == rdd4) === 0) @@ -881,8 +875,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { // Cycles that do not involve the root val ancestors5 = rdd5.getNarrowAncestors assert(ancestors5.size === 6) - assert(ancestors5.count(_.isInstanceOf[MappedRDD[_, _]]) === 3) - assert(ancestors5.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 5) assert(ancestors5.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) assert(ancestors4.count(_ == rdd3) === 1) @@ -890,8 +883,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ancestors6 = rdd6.getNarrowAncestors assert(ancestors6.size === 12) assert(ancestors6.count(_.isInstanceOf[UnionRDD[_]]) === 2) - assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 4) - assert(ancestors6.count(_.isInstanceOf[FilteredRDD[_]]) === 3) + assert(ancestors6.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 7) assert(ancestors6.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 3) } From 20bfea4ab7c0923e8d3f039d0c5098669db4d5b0 Mon Sep 17 00:00:00 2001 From: lewuathe Date: Thu, 4 Dec 2014 16:51:41 +0800 Subject: [PATCH 117/305] [SPARK-4685] Include all spark.ml and spark.mllib packages in JavaDoc's MLlib group This is #3554 from Lewuathe except that I put both `spark.ml` and `spark.mllib` in the group 'MLlib`. Closes #3554 jkbradley Author: lewuathe Author: Xiangrui Meng Closes #3598 from mengxr/Lewuathe-modify-javadoc-setting and squashes the following commits: 184609a [Xiangrui Meng] merge spark.ml and spark.mllib into the same group in javadoc f7535e6 [lewuathe] [SPARK-4685] Update JavaDoc settings to include spark.ml and all spark.mllib subpackages in the right sections --- project/SparkBuild.scala | 5 ++++- 1 file changed, 4 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b16ed66aeb3c3..6ff08723772aa 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -365,7 +365,10 @@ object Unidoc { "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg", "mllib.linalg.distributed", "mllib.optimization", "mllib.rdd", "mllib.recommendation", "mllib.regression", "mllib.stat", "mllib.tree", "mllib.tree.configuration", - "mllib.tree.impurity", "mllib.tree.model", "mllib.util" + "mllib.tree.impurity", "mllib.tree.model", "mllib.util", + "mllib.evaluation", "mllib.feature", "mllib.random", "mllib.stat.correlation", + "mllib.stat.test", "mllib.tree.impl", "mllib.tree.loss", + "ml", "ml.classification", "ml.evaluation", "ml.feature", "ml.param", "ml.tuning" ), "-group", "Spark SQL", packageList("sql.api.java", "sql.api.java.types", "sql.hive.api.java"), "-noqualifier", "java.lang" From c6c7165e7ecf1690027d6bd4e0620012cd0d2310 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 4 Dec 2014 00:58:42 -0800 Subject: [PATCH 118/305] [SQL] Minor: Avoid calling Seq#size in a loop Just found this instance while doing some jstack-based profiling of a Spark SQL job. It is very unlikely that this is causing much of a perf issue anywhere, but it is unnecessarily suboptimal. Author: Aaron Davidson Closes #3593 from aarondav/seq-opt and squashes the following commits: 962cdfc [Aaron Davidson] [SQL] Minor: Avoid calling Seq#size in a loop --- .../spark/sql/catalyst/expressions/nullFunctions.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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 84a3567895175..08b982bc671e7 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 @@ -45,9 +45,9 @@ case class Coalesce(children: Seq[Expression]) extends Expression { override def eval(input: Row): Any = { var i = 0 var result: Any = null - while(i < children.size && result == null) { - result = children(i).eval(input) - i += 1 + val childIterator = children.iterator + while (childIterator.hasNext && result == null) { + result = childIterator.next().eval(input) } result } From 529439bd506949f272a2b6f099ea549b097428f3 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 4 Dec 2014 00:59:32 -0800 Subject: [PATCH 119/305] [docs] Fix outdated comment in tuning guide When you use the SPARK_JAVA_OPTS env variable, Spark complains: ``` SPARK_JAVA_OPTS was detected (set to ' -verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps '). This is deprecated in Spark 1.0+. Please instead use: - ./spark-submit with conf/spark-defaults.conf to set defaults for an application - ./spark-submit with --driver-java-options to set -X options for a driver - spark.executor.extraJavaOptions to set -X options for executors - SPARK_DAEMON_JAVA_OPTS to set java options for standalone daemons (master or worker) ``` This updates the docs to redirect the user to the relevant part of the configuration docs. CC: mengxr but please CC someone else as needed Author: Joseph K. Bradley Closes #3592 from jkbradley/tuning-doc and squashes the following commits: 0760ce1 [Joseph K. Bradley] fixed outdated comment in tuning guide --- docs/tuning.md | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/docs/tuning.md b/docs/tuning.md index 9b5c9adac6a4f..0e2447dd46394 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -143,8 +143,7 @@ the space allocated to the RDD cache to mitigate this. **Measuring the Impact of GC** The first step in GC tuning is to collect statistics on how frequently garbage collection occurs and the amount of -time spent GC. This can be done by adding `-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps` to your -`SPARK_JAVA_OPTS` environment variable. Next time your Spark job is run, you will see messages printed in the worker's logs +time spent GC. This can be done by adding `-verbose:gc -XX:+PrintGCDetails -XX:+PrintGCTimeStamps` to the Java options. (See the [configuration guide](configuration.html#Dynamically-Loading-Spark-Properties) for info on passing Java options to Spark jobs.) Next time your Spark job is run, you will see messages printed in the worker's logs each time a garbage collection occurs. Note these logs will be on your cluster's worker nodes (in the `stdout` files in their work directories), *not* on your driver program. From 469a6e5f3bdd5593b3254bc916be8236e7c6cb74 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Thu, 4 Dec 2014 17:00:06 +0800 Subject: [PATCH 120/305] [SPARK-4575] [mllib] [docs] spark.ml pipelines doc + bug fixes Documentation: * Added ml-guide.md, linked from mllib-guide.md * Updated mllib-guide.md with small section pointing to ml-guide.md Examples: * CrossValidatorExample * SimpleParamsExample * (I copied these + the SimpleTextClassificationPipeline example into the ml-guide.md) Bug fixes: * PipelineModel: did not use ParamMaps correctly * UnaryTransformer: issues with TypeTag serialization (Thanks to mengxr for that fix!) CC: mengxr shivaram etrain Documentation for Pipelines: I know the docs are not complete, but the goal is to have enough to let interested people get started using spark.ml and to add more docs once the package is more established/complete. Author: Joseph K. Bradley Author: jkbradley Author: Xiangrui Meng Closes #3588 from jkbradley/ml-package-docs and squashes the following commits: d393b5c [Joseph K. Bradley] fixed bug in Pipeline (typo from last commit). updated examples for CV and Params for spark.ml c38469c [Joseph K. Bradley] Updated ml-guide with CV examples 99f88c2 [Joseph K. Bradley] Fixed bug in PipelineModel.transform* with usage of params. Updated CrossValidatorExample to use more training examples so it is less likely to get a 0-size fold. ea34dc6 [jkbradley] Merge pull request #4 from mengxr/ml-package-docs 3b83ec0 [Xiangrui Meng] replace TypeTag with explicit datatype 41ad9b1 [Joseph K. Bradley] Added examples for spark.ml: SimpleParamsExample + Java version, CrossValidatorExample + Java version. CrossValidatorExample not working yet. Added programming guide for spark.ml, but need to add CrossValidatorExample to it once CrossValidatorExample works. --- docs/img/ml-Pipeline.png | Bin 0 -> 74030 bytes docs/img/ml-PipelineModel.png | Bin 0 -> 76019 bytes docs/img/ml-Pipelines.pptx | Bin 0 -> 56777 bytes docs/ml-guide.md | 702 ++++++++++++++++++ docs/mllib-guide.md | 13 +- .../ml/JavaCrossValidatorExample.java | 127 ++++ .../examples/ml/JavaSimpleParamsExample.java | 111 +++ .../JavaSimpleTextClassificationPipeline.java | 6 +- .../examples/ml/CrossValidatorExample.scala | 110 +++ .../examples/ml/SimpleParamsExample.scala | 101 +++ .../ml/SimpleTextClassificationPipeline.scala | 7 +- .../scala/org/apache/spark/ml/Pipeline.scala | 10 +- .../org/apache/spark/ml/Transformer.scala | 18 +- .../apache/spark/ml/feature/HashingTF.scala | 5 +- .../apache/spark/ml/feature/Tokenizer.scala | 4 +- .../org/apache/spark/ml/param/params.scala | 11 +- .../org/apache/spark/mllib/linalg/BLAS.scala | 4 +- 17 files changed, 1205 insertions(+), 24 deletions(-) create mode 100644 docs/img/ml-Pipeline.png create mode 100644 docs/img/ml-PipelineModel.png create mode 100644 docs/img/ml-Pipelines.pptx create mode 100644 docs/ml-guide.md create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java create mode 100644 examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala diff --git a/docs/img/ml-Pipeline.png b/docs/img/ml-Pipeline.png new file mode 100644 index 0000000000000000000000000000000000000000..607928906bedd224cbac4e2ff4069ee3cf2da261 GIT binary patch literal 74030 zcmZ^LV|Zm8jqpOp%8LFY-*fcFQz}Z7NGBQ@#u&=-WNB=a@l;MgoYGq@dFX zmtdQuGJ9qZ=nw+{))^|A5VYxep7@-W$Q?EG9b$+9%p0uo5KhvMJ9Byn)RW?)|so;J4zTimGBcLrYz=$L_-pp%o4#t zKLV)1!6ykrGo={82f+xSpgyW#nAt?2pg{eg!u?!82J00a&dF4Ac&Nxkg6x>c@KiF>T>Jj=O9Qh3LW->AUtv`D6L3HW9HWt#O$|LD}n;m z99?p|^B&NgLsHKi$?>=r@+$Nc^&2VroynSP5S#|9kcS{8UgsIn&Ap?BHbI zh{dN)Tn*L?WF2uS7YYm!n2NdR-U|j2(QQy1OphFfV0KD1S{MkI3FSl@8PEUR>p!az z5y517e=pj)N9X71$T07P8p0qYr=$oXl2r*nQ+g(c$!j+>&X!bvCI@vMc{wm6F8eo& z7#5=-yThK>fAsy#;Dfv)9~5OxI41@Rk|N@SVEytzTV6+$DkH9A zseaVZf|lV--f-K&jVa)vs=Jts(((oV_mhA_l8E|1n7p4vU~XftiB$5xkzF9n3aswy z??C&*NCI0vvso2sh%H}QbKfgiS%YVga@hpcYGE&+eZ?jQK2MnGR`Wy#ygz8^dVQA4 zSPj6h;sqEaB(8X7+*c;Tz()WT9U&68_HPUm1jSLL@aQIB@Q96(=NraXJMQ9J|M0PZ zSJ2?Pt{E#n8lVyPw?pvrywSW!-#Xr>+{r^K$W8ah`6Liikquc90Dmw@%w)zHSBda% zR0sUxc5a~1D^EuJN@6Kwy*F(c+ zBmH;l5s^E#ej0Fg@qRg+eJv9c98(?Yx#bAmEKkl*LIg@}Pkj5763g-OKa+ALKHqMKZh!OtN|$?xwv zf`MqBeZ#RP0jzs}BFtU$O=Rr(p`oC`e^V(Jz)&H>7$@Y`BSD*QA%}XkgWwnd1~zbn zGBLlDD&{nSMfC^H#XcYwo^`vycUc3-E~GR_wsC^aR`Rl$!iQ&!Of{R3rDZ=4MWUa9+lc(TIrrIjn3t1#CZvWOvT-9`rHF3ZaB7FfKp7U*9|_~TVn3q9E_Kr0$3e;q|8&?RH0Eas zusUIFtNG)fheB$g*f%^H#EJk7eu_hP5_sm$touM zrVBxb@-YA0m!!>CIFI(Urz@P z%XSTWB4TOf<^EdC1uRh@DAxjDDrz{3L;MS(5$AKFTK$TpGDodu3$NYOy%!ARxDz=Dk+Px_wt$~MTvWF11SI-}|e-VAY zn3Et=IKspS-zZ66pUjE zLil*y(6g{A%>a$5Ss-B0qF=x#&_@0Z!_Fd*?b8XPZ{3Xmlj0XVYryxy*Uv{>|CS3} zh*7S;p??h4cOKf8;|jOU9*Uh;(8YE?bcAx$cf4@rCN+|g>QIFOXf_`bh7++=Vr5eW zv;n;sw7_?cLV@;udi^&YmF<`{7W5(Rl|r3$gq>9=UpOkFfy*L;8lZW;Rm>1Hwv&S* zeaiy7*9b=D$;nmdHCMQOO9)JY^m$qLF@Gx>i46&0nZ$2?QGkpXQS1K5IYSOW^P_m% z0rYxb;{yB-heBr1Wu-yV8DIzAbb^ujz>&cS=7b<9a6yTgHu_$ImRQZ-?n`l|zd6H& zMaU2LweYt*CAjS;rlH}!;@@GgyJ_vfWKXcuSA+nC7f6JR&aI}T0MzvG1kiJIk*}u} zG(;G=eO`*t$JjYJ@qdw!B4c2HdgS3~BktZ1zGHrSKi81R#Wm|{Cqeejeo6SW_ZR}y zR)mj5L08*3%TPrC&l*6f_&mWDGbmMk1kVolQ1wRLQ2_ao$r6Syugvet?*tp#$Y%sO zx}l8u_AqyqeY0nvJIE7&HH&-oibYG?!fjd+K7VAw`-yvsH%&#S|B{z1L+>!eGC+kSSncC{P8F8}j+#&VTjJEUFP6){` zwCq3{a+t-w@9~gC-Qkq0;4!oz#x4b(+25kQxL}xBt>Dp8idg>>nm@!0hx#nL6c#+U zBQau^imFXXRhtKHj7#+8e>i>_y@=PexuNF>ojiNiiwFJQZik1?bg&SUM0L*wU2EqL zor6vSL>`2oi@shc;s)q|sd!NTz8w5UUNmV1dmwmWg%*Ey&!EQXO7AZBb%k%n7*l?0 zLCj0|5|qcND>6%I{QP4I#J|X@egkzu{qYZa2ytBQINT5_gT(`kPjsEwZ1Clxa7s(Za1_cudv0 zQi}mW<9B}jumZ&689mSkFJL`S#ThoEU}qS#!eAt(SrC?8F1+dX@~ zYZcX)zh~PjL|D%)n2u}<3gA9q$aq>c?kUD_GSXo%KDrfkcH&Ii>0xsw!m(#fekSCr zFF0$Lmu*btA&`SFvUifFiU4CEA)03%BxSOA?5BWjz{)ySGU-wRBVHAIJVbCv$O`|2 zbH&-2155#Vr-~!KS{}MypZ8cSt;SIK=KBE=dx%%4fW|P%Ni2*kHrQ6T1>jPL;v*i* zA6m7K!TqN^cF1^QAi3iy2NE%{SeRH$y5V@Sm}1#fh`EPJ|C-6q9&(oMd5#ObRN- zj2GO?lk?*bF2pF97%`>~fKrhF^-*?1M|`V03m*9nRid?#Dg~*3bS`+Yew$M-_bD$Q z3)10Qy32{GSqcc=zV>-_Ks-8o{-Rl&9&ppZE!S0AOUaX%(9K~9(n#}LbQGBps z@YD05`@qP|^4N7st_LowYJxu{vC4TS5Ktul=RPT!EF9(k8^QoqM}|{6{mjQYmvVXfokpTq$7KS_DLyi#uu@MeSWFb}@!}56E0D?7Jpr zIgn#7Xl9wQ#}_BK$)KU3vV&6{oI|3V%VL%R@|fK+oxMEkxvOPYmjhuj@978lCRQ1d z2zkI5z5NIgE6yq}g^6FO+)qLqV=?vALtILPf(;F}rODtv0-(t?t?}vg#PjnXIdz74 z{p0_qv7>z*E4VHQ8#WIG6bBegEX>7`Kb&>n1gm;}#g=;F4x07@lf|b%!LhI86_jZB z6R%XhKa+}J6BzWOxb;)3;r--t_H{^`xQUbfO6{hLNz?q$+nl|DRj%PVY^>#XAq9(O z$-Uv~9MPlV6sw~1Eck#>XG(GW-?`z5*N&mh8b3uC2l~^rQz&EzigA$;Uv<+Vt0 zAFl>Z3z4Uyb0W~w05pewl94guic9ABXPBoQ{IV~Y-G5ve=7k5iaA-;!%Fvog29w%e zq!!vHTN|a>cF_gcG^stYe$BPo)m=o#%OJ0}zF1KW3#4dsv2>NrALsrPt$sQoHqiqx z8{hYjfjMD6Ul30(*=KEw2PZz>CAnf;5~`(r+TF*R9_dRKw{lt~9|{aMm_e5ce|X-9 z70ud3TDW>LTe|gWZhAV~-jWR8hGfhrTg+H0l7wX^{ zH)1Pn_>Ov0BdQFv@w22!4p@Is&;V%=lZ11=*GH(jeB5?Q?t-{ApVLxHu7KTKM$WbD z?~3ie8dnV1KF!Ap*xZWpYg{@X$=tu`NAucGYDWmBSFcBvIaw%fd4S-^FIWq76Wpm| zMrKPXxUQknnbhBg^Y53-Krr_xsB>voF2WrsRQ_HYSsxa^+uWO~Gn=`X$23A&hbx}! z#+Zmf3q)BzV2V-(<}DB~K?*PX9C{q1;|?=mL-@)mFtOA@Ic6T*bTC!^vTB)s^71ay zNhR|sXG`#b?zj(;?2s0)RiN@DJFcVrw=^0gMGl{eXfI0dc{_!Pk+_TpKR?jWPGSKr zPKAse#HxrF=}C>^fUmwU_ZXiN`I1`!A@${L-tW59WrpQl^Jr!si;Vzjo9F%#GHX)WzLdyl8K_S5=4f!>4yzX+m6Tf~_7Q*kOQB{Z~Dhl!DYOx$E zLtAHk&P^UkX#UTMQYHpkOecrfq!~m6=|o!`@voxx3Zx7u$SC5-Ts}CZ;&15We6ZZ@ zt;Nspn7NFA`N*PP?44PwpEVIOu8N^1j5aa*!MD1e-SMC%XefQDpx$>~)>kRVg<4 zk1l6S<2E$d7c>k_gJG=0?M*Sn*>99W4e%CQ?VJ*;@(@3{DCSs-xPG4c$JRH?)k2zZ z+q9D!Wtp_u=mlY3d}a|l7oXxiO5!)tq!RiObY>w^UYFD7+!$Xv+g3KXxkD8+;5C!Y zd{zm(CRw)CD_=8sv~ji5;;n@%`|{~EOopQQ_y>~X(VipQY}%X6thl~*CSIrMkrcTK z1M@LKWG33*E|5utq(pTD)F*@xr3F}Jv#9}=+{~>^PCp`SF`5S2DP>wPzP)R5|`1Y2gbx1b{ZWuPfGh2=+oZL`diAy z!EH65HOA1_yV;VqKW`V=ERO|1o_(p&XwbLgdqr2`Qyb9}#0E>vbcQh1np>Y>hi} z0!>I<&~PQkX*}4Ytb)eB|Gwg3Pe*2Ghu5W=gEZ@v$p;mpW^Z3ACW}O(_B%R3h);v{ z7z(Xw1~hBCh2eRSY+?Os-cV56q09RxeNfYu4{*cWd-`#pCUcuAFks64;=u+N5eOm zOm3S-Zr&f@619aTn0OlHL}%k6fJV!eZ!SkhreYPICy^NYr`KA2_fn%~JF3o_6ywpi zvXy=>`jx*qjl(D~yk-l>QLD{CK6yz|S(;K9z*llvR_A<-kp}kCJ6wkVan;dFB$v?8 zeA!{NXQ3=WPQp>zaAii*>befk9zJNUFhYM;zh}eJs&=@JQiD{xRlVgm;VBDmb9{jC zffjPDN}L)hsdJ%^3EVDMMO~{fgsN6IhdX<@lm5aC_-shPSgOi9V2NxzuhKT2-a7tH zA+l7cyVTo)I4OtCW>HMrf*DmzuhE&(J*rIvIaF*YlKrags>AZa_&8u|l^j)u@}H6p zhkEg4A&{J>r!qcdSpzl#5kituI4O?bW%uTQsC`Me}?!O_hcyXCuX1&!+NZ^U%v z_oxj`_5G8nb;;Fz#xREPG;+7BOzuaMrR2E$N$dBDVwH6lP*ork21*@d#A7o2qbo2k z8$DD)oHYBw2bGT&kvc{bxf*rcv5H3g4ljoYaCjtzOsCubnw$n>b}t}NJ=S=_WOuFd zp)^kpAD2-p%;Ep|^(vZvD5cvZ^V<>gANSp%A6Pev+F&b)(D9^{cyOK;HKGn_DcgE= z;J)J*G;YQjjU}MnJl<@5M|F%B&YH4+lkER-;(s3YBm?Q!Kj)Mw<25my0WnJ2+As*~ zGm0la4>5K#Ho&Kj-Gcvu}MR%=m|mJ1bS6J7|ME0M{7%Cmx@! zg85YpqH-dv+-qx!1}bp_w(Bp;{+b)z#F-~vSkBM(F7TD!*56XJV=eso*Oi69Sd}$v zWT1gDrk9LqYBu$&XNFIjvvljzbGlk&vihP&anz)YOw7?tcyck323^91PjU)z%DyEB zX;PUTX|0i^B3e2rzC;rJx^fo;o_c|}X@I~BqrPaKB@#pgXC+`xy-mCuA*5HW)FVB4 z^U^kDt*mP3F>hsY*(qHErjdh_zgFY~D8ZqbxAHaTyBwRrUg`?P9_`JUEJP zEASF2hkkOEv`~k9BkuW2!{CDrx{Tlgg8;g>I;=u`I^BT{^k0Xw$?V}g>!t1(pV$0ttlw^()JKti$T;G)3w&d|P`=D_1~O7W z(D`3YnOORuLy^8#g=XhJxpCn=xui?V$lSPzt8#p1kIp@J!)1Nsav5s?$K_s?K8X1yoH7E^td%|LJwg>2rG?aBis z*pUFKE%6p>e)XxSIDYqIb$u>_6)^H80^^(XA}nB zN=-{k%ZcgP^o=5$qd6b**$Ol7WnnhSaTc_Kc-Cj6&cAyUv37-KjpZt3yOOPp>Tj(9 zDUtsVYJ~-YR&1s#9Y^Vo z*c%Zw>g=<$VsNiB2+jM<3puTDZoFkUBx>7Q_E}NuyYs#UF`oZE!Mj+tD&H0eFRttz zt_qU}w5V-xutDdMhZc^Q?=#wTws1_jBDmWVK)9xfaCE~wq($srqUpKnpln z6mbLjcAqPf0*U%6b3yUk1{L zg266zYRNJ?(LVWKsx@8i*;LG!nu@eHHpH3V=2cbiMi}@aFbTc}uK?IA-ktL)9AvV`N)_ zDfT#m5Rs@OY*!gQp!kW1ncywxbXW`~88I}2$t-Hl$n(bK$wBLyly}Y4`cD^JI<2fY zTr74=8{ge(kMP?S1)`^(41LH8?07DUHsv1jL9^^oyG##s*Ynl=DVhM(QoB8dzUElV zl;r;f2tiRIf{tSVc{t}W`CJ?6bl`VEj_1yZTe|99h`R8aFpu0RJU%z(Rpwn8|S%s+AoD-l!`=Dc%rW?CP6w% z{!~21n9oH++1Vj%{&Gp{k(73mz7uq?BCxt#yPdWQP%oPMB zQxk$g@4G1CYLuJQI4;nHu|ao?(6~$~mkWev`*L~cp`p_zP?_y4&1;Je)KYIh;5m)~ zHEt_yBujGi8pB3oo}PG9;Ch_edQfpOQB8R5y(8}JZY~WNy}kUg1zi`5%X-PYBupo; zo)@tg!5%lPYHTM}B0f#tv-8bFn!bf}OA&Pt)Hpdx>zCK1xLtTRe|qkQCSkv(J^51z zm~nu`sNW{^w`hJh!dk-^u`VgXs<3|C8;@{yQY;F7^qvIwl?w5oh6HIwO@qFQM*B8# zou{{HAY?7{UpWH=Drm`^B+wea8o0VP_G=|5RQZ0Fae!!?#Agh<#1h8$`4-L`n zK60vc&ES7LsqOoVX)zJHK^ZYvjXa}%J<9B^7$vn8w??CanAaX9x?@E*5K4lU@Gw|c zORUt_rNlD;+wRMu7}QSDw1Ua`W4xf+J$^AOod0Je8?p^5Q2 zZxZvux5;h2-WC~SioE|$7|`PTTWyvF5m#Y7w8n}4>ITLHkINMNeAdu95*i?}E#pyj zaPMv}VOP(KH-gFAr}QHr(VdhI&1ALbo+nwW@pS&E!D3c_fj!HJUB{pm`oY-d5|?Z= z(p-qzl{2^2aHyhwAiobE<&}c2_;Bs9&5a?;u5-Klr<@czW`iA{>F7+JXR}T^95&C* zuKA&h*Gt1IP8VRBItFN!M-W428+psk8NP3iN`TE)(;`b5P??BiK`nQ3+bC{&G81ar z@sXr>tVA5Mnte~Cc)g(N3H(ZSO;W5i>1uaOA)_(#VVcTAY4+3fDj$IMn~b?0!*-ejd{;-#fYAK;%MO)oqFc*5gvbfgWQ^i;Kj3 ze16HAUoy?DZTu55%=5wO#_s!^4*-XB-;rpQO_&0n!E1vp57Q2OS8pkm3{)axHP7C^ zH$I}f^A!g_$#2ngu{`Q);uD61bP%1?)fu8R$saWLqN1>y(3H#PMMwTr43#g4Uu{Nn zx}%4UOh|fyAY(~Yi1Vc8z&|@{R(gL6*!h?;or^X`gA*K8aBiOHh?E)7Wtx#8Eug9y zj*r$ppXiK%e6b(kA=DK*td{L$6w{)n>-K;oV_nsp=B|zPfzjSz?cg9s{6PWk8n6~K z%I2khYwNt)zl1^Zo)=kR`!`u2>-U@DvoaF(;W_$Ht;hDiX81+!JK*NEl}vVNOGOQv zs^(h?_k6?6`!Oc1DmiMhL_meWT*liNbx}3I(zHWF zbA#ICwT&c6)_bpY#zhc=BSlRcql&bn2n*}1E2cIk&r0Uecb_-J##|39xA8RRLzrOl z4fHUU7{3|AvfauM_V4D97+MNM#fkBl(HF`~1P^5t?L*Z~=!94E>bPK7duKNUZxzj{ zmtKEKy8!_x1)^$cVXV)p|5VwD>6xxb`^TV18korj+YIjNyKQ?A5DkjEBXv_N(?;p{p{NhkzpuL{rCd zA$JH%Ig7eWna)uI(N5HG7nx~yJH;TPJoNS88;$5)lwHF+djWDA?yCP0r9zR$ld&kw zt_oH&bUEo$em=6|7>ou`0cX)OqEvWcjR_-JQP&a7J9b17=MbGkGREI@;E15M>ggnY z6vst}pR8DGQQB*&8^n=zHFsXXl_cH`@b6FS)KE=omB(|%XIOtmX4gSIo6=jvnLfL% zX!XBirloJfMx|9z> z&kV&`Gx2=eBm8t+16CAcO=g*Sb_5-sC#XIhvtK3@a)Ls=r0hqD(eUnol-wwJ+ix~e z&rAUqhQr}e&Va@KQ?>sanR zfpp{2%Rvf^+|{0G^G0SrYJhyuOmT{`A@k&&W7iaNqsd}^eeTaxg5d!35otik&HYKtMP4a(Q|W%w6@TpWZd{~Fui+$B5lU^1FAGTK8+eIZ?l*t^ohScPf8{_ z3pz%_KlAfM8Zk+dk0~V)Ib{FL5rm5s;?0VW7FlZA4aYUn<}^O;m^zgw%#_oN-H_Ra z3bXc2u^ewEa~VU5)oj`VsW#QV1+ONWR@W2dT!Dbi`l)s<)2q&-FK^GY0r_8j#`0gL zMHKDO>yy+n*b3cMzTce<;wKA4$|qe&X)gUtkUNo%k;;p~jX^bt_Y2~hiJkFKos9k)BJU}WNsXK0Dd z`8HG60%D2e{vl7$aszjGaed8;J^6!XhvcDziU~gvL(nYbU2+vC7-3+Nvfpozb1{J< zo0+_KYjO|$3n)5cQxM&W7IA>z#Jr)X$I2)%90vt-5^G2j9nZlp; zgb=kpJ7}1vlGaRtlLst6n5)J%+ORr@84KHM;4EeXf@ifF7d#`qjl6s%;NWv-#w(P1iXP;lCv->hj?y zg}UiHHZ0c{{?mxUx;g3Nw_u(Wz6s03W3q6^H!l7yP!uJpAJ!vher~M8rf;aj1|U_G z9Q%5RoFAsyHeCE3IX(n>M-)2(3Xd4xz)dF~9OH*rqO4)KzKKD#wh>cTclWff5@R~m zVFYD9WBlE#nRmGkf*dHo>(IbvCm>Lj=u}i-$mt^eQ&gn>_w=7FgmAf&z>r={CLH*j8Kq9-bhxG0ju{ur*rsV5-=CrQM|ijx1H?Ed?K5i zbdig*6T!oDE2oVkunW;otkK@JL*$eW5iblxb4?ZryF0Pvzj=4lhipa#NEA|JAH{aNiuu41gM^nAHNV-@fcZv{gF?T5vp?vYa&ijx_7i3=wANXBh)$#P`+=i6y%du*$2Ys1^GvL18L}2T|(3WcL z-hHdD)u#RDGFlLug>W-m`=K}lCN>r1Dlt#iXDmK-eaL|@a&KzIPofI0wQ|iD`jy{g zGJ-q+K4gI9(5e|lMjjtFtC8=3pvle6O;SODPzSQI-H8G@JnUqiKrKu9JD^r?clFE9 z1G8+YE^gyo#xHIP5ES^JeQqOAqot%CaDGDa9etcEe+PwNC5U3bWke?==V0V<%H&7c zl5OE8U^_TShE*O-S7<@pHFIt~W~td*QXNi72@B8D^T(LlvmDR;S{B%+LKGrfkD;9y z*4GhEyui^$?JFW*;o%kSMmGwv<3$vG+)%Qu&pp z4ez_tdHpE8dqlW6DHkc)i4-6}7x1E(?f{!8veeES7q0Kx;vajN4`&O17Sb?m1Z+i% z&gM;yL_bR!7X-9NFd?@6t4Y)j&sk*K9X0ZMNQe1ziHu#nGmWU)aKU4DKPrX)i-qQb zy+MLq?vgmWcuQMb$yx4~fn!cTZ*=qFA>&KAxm!MxkU^B;2Oj*CE2?rCQ(qF}EC$yn z4p8T@m&bb?yNl$$SC{?f{y3h9IVJyA#LlWOv5SL=a(}Tc(--1x35VyI0a8o$Zv_af zZt+PG$xbNR7%|De9pR-l2&Q zlI`aU1}+H?i0g}I#0ag+f7w%V<6P&|OPG)@^I8}{Kh(nhh*98=Mj+P~>#WXo#(z^| z2oFAVc7ykHk|Se}%#Lz%@NxoIW~Vw&8<@Z;n7B)gQ?qK#tD zoi0we%4?V#_1gT%q=-0|tdB;iKYQGW`%TtE2mG9+-X6p)w#~7F9c%jJUev&b@ z0gx3S)$z$3{;T(sffNMNc9GGD_v(RZXqR;B$&~CAe$jNi*Jw8;?g!_NArRf2_q&jq z-C!qn*#648zgqKPouI&1;-*JmJ$+20s3icID55q%H^OO-gbs)2Wxc`kNTFeK9GhxU zCQUDvK)?%L>yM*mbodOA0l@|ZJusjoMV6HNPdel&IIyZ2(*`wNXy3qbX3U!}u!iz*wI66X5r*ArE!<=)&=*p_{P>{PYSPkCAvap5 zb$1vntDsdp7_ca*S&0l?nmn%B2*sAl$RIx&s2XH59FmY@fS_&R`$Y^$jktSfjnfNW*pC#o=)=hkMlSDlgBI@; z{ZXt-ZaOr1$A2r6x%vjUSCZrg`!jvt*C;(po_7@fmkxr@bN+~Yg$REsxx8qtW_#lr z%;cnQM^NJ)J@@Tm8yesM6mT{hCXsbXJb3e8(gTiS9w&J1M7mY#X%znuaXED|R0${j zWmk6I_F3_e<?(Yt)Jrh8NIgElK27_`=V-nl4h zZn?3Rwr3-Z8VI8(HW!;jw%gJQF8IsA$|PX>i5NSXu9QEFlqBV68@eg-ZcP7<!s3F+KfC^f?JM$FPOa78G&9)Jk;dgReR0LWyzvsxuIM1f9F zv0Lh_F(aQ4yYXED6pHfz>#TuuIt=MY0x=y8tn9PLMm?tkiSdc8BAtBt2_>#Z3013O zQ;TOKPeYRBuh{Jjx&wJ#3gzTe`!sR+7hv8o7l36D%K=L8TwF9FEA|}ldB6{=o9Cl~ zz0RY2ILUWDHSL^s7%MF;Ng0`xzR>WHq#4FQ5gwV?0FYzWiGmNp`Oa%_O+MG-gKSG~y`c*=^>H{^9!!~Idok+w|} zHLAoo1Z&kVR)P`|c-9klX&9&cWgro5x=f+SlFn!le|mZ)p)cH9p-hUJ#I{e0@ll@R^D^gZh}5>vAFEcQZlrgqj(_Z+h)t^r{qQdK z$I{F%8tHEzs)2+L=SE6uOtTSj{b8xNz66v_B7YDpL4Ywoj%o5#Zj&r3 z-H6dxwW3V#G5W=;Mub8oKPm1dU%Pfv(iqMr1SKssY~$YS)|yU&|4P0QHv=a&G7{zz zBRs#PPbfdwqe}5LmdbK56=1 zF)0l{0c~c8auJpi-`TijJS$*Bt0{chVoWtktlIc@4&4Wy>`V5WOyTD|r(9LLXMUJa znh%0k$r%s78&t&YXj0&Iq~AsBGBPeMcf7XZ^1rN^X%&cq?oE15O&Nbl=~!zNTC@0%A|qjqBb-JU_w-gW zO!1H}-h7WN*eLvuF-_m0+LPMCwAvVnL{PJ6NiOUURvVkXbWbsTEfyTd`Bo*mLo-^d z$&_5;z*ES;A8c2X-2FAUIsp>Yc?dDZ*%904sR%v)2Sm#FseG$K@WJ=Wc8FOtmH{V| zV&Bu3nfn;zx$y|U)0J5NsJxeE`egi1-n8e%ca>?WG7K(b z%Ebc;lYd~|Urg`>tJr{Aa^hcR)4xw*sHlnA#23REh*J(QWvAU?t>F_-(yR*^-NSpS1+i_hP5JcOJPswO+y$ zgS#&#Z*J3p2<;^Tmp@J2;AR?7Frkk5v_5y$y7zmFE>rU*k<{H-PoW7oj0j4CYzg7} zAv3(N9?bFaJF=Hn1351>s{U_@I$I>zC0kmL3Myhjinw^<2w5blw3PsV^I}#;kqW%J z1DtwRj9wye?@3OXx0e0eC{iyof?7vgS~odzux_-jX;M*?Vstp@s+-M497tuXsxpG@ zNcm=3m#Bd4PCp}7gVD^kaZ-ahu3pP-UkDqv|A7JOAMz zmoi^j0i5K)#KsENJUG?P;sI=bJb2=p%Eb7+Li4e_SF)i12N6iOCvYrrpJa$lUHHFPM3n=MB)Ac*$M4ybZfw zu@$t?Qmbzp7~j;M^q=?}4s;+&xT)2QPAhAqRs5@19Npx5L=Vs*8B z!=0>Di_U&xV5PHS1FQ^I-p@xUedku?ym7(;cDYKOo`fnpeaP3_m%~W6JTxWeVPJsP z0dZaV4=))A^AsDX5KxW(dO(UOL8sFAl6EzL-$((x2cRXYm4GP3qrzl9JSyTeNuLlJ z+kmR`(QG3jn3XT1OurtD&X+;M-oC_9zN+O|7u`MQWp_WFl$4NE)u z?*$}Zp~54B;8&4VA#TpfeO#%buNkZ+E{>`mg#NYjE8-|!*?3rKhR@sVtJ800J_r&d zUA0i3W%*$;^K$S6YPD5~IGTy4_9NLXbH};M>Acm)L~D=2i0HtwD>aj+iB(AAD5gTZQUGhEW5h?cDnso%IZcQUhK;3>QZYN+err2id|y7 zOEMweaB+t-8pK>`T8vSOeVlB$-YApN9ak;H-@ck5Xzm%m<-)YR4nx9#kOQys#l)13 zo^U@O(RFoy@o^d6n}|KMEtKYl;+i9-!^}EkZ(J@pw)F*HXSdvM7HO4mT)S>a35<~6 zf3a%=t$(-dv5Wz8j|j@mfYrwqkB(;owjW7g zvJJc?1*F0QVLHgRZ`BvgWR!}&eBOy*8ut5}me{J4zfYwf`#m!1^B9MrEXXj*u@=V+k72+`|`>em!edj30pjZ%Y1hTHAeO7M}{jOr4_ zg*ax!?|Wiflf%X+{34lC~aLUaB+iLxNUn8HbWglN-e>%kW<)7c1q6&az+ zzPiVCzDg3S;WH{me!9RF)OFR1R}k>gc?T#_yd9@3DD3Q9{bu_28=@HhLNUWw*r%bv zRiAT6+kJn!gY#EdWsEPf1%7=(w)xU#S*ypl>7w^ky{E=)@tTFNtR}CMne9VPtlGL; z5#Ml>Vu_!Cih;qI~=AVdOuc+Rq~&#OHPIxqYR ziKBpOHN=KEWDE#wP3AONeQr8XDDEvs#t~77j_s*vI73O>Ro$apy^Z&^g!53aBotP#CTB>oa-{k5>?q~7!5Kw!Hd z3Tb;)PRrgcnR;h1c7)H1rJ|XLziXR6)YYSS<97SkoROy~zHx+k7UcH}#F`O-`v`$I zqp37b-zO&ZI^B~d1E7e8_Ckc|H|^(1^IoS{{XKaw8@E+wkVy3>5Jq2KY*ht54LSr^ zwOZZ-qqtThC?cFm7B&9xwy@Fm;~ zKodHBuTErNUPNkRs`V*?#`$zhf(%0VF%?QCyKpnf!+<~FBo}Tn17;7U_e+3Yh^=@; zjq)$Y7nFn=Dcwg{HpQh)bkA}u-9T0cLDLEq#W_eZD@_Z3Ut2ZQUb?>Dp-4k=zon$o zfk!JtcAHt!x;>n>^6FS!<|aGR$J}#k>b;m&J6wy3(R6nuV!eyM{p-%sp;7`;piARj z44v8H{3%W$Z)H}1?Z8v@n?Wf)j=oxWZ@C<19{9TZjHWaVDv?AjNji#|{$&Qotj;Z4 zZoMxqtpn7imzmFgB@f7d{4W4Xz>{6W{-PEl943V)shgT+UrRLk!Wg3;NQ^^tV9`4n zJLW}dyK)7okdbg`w=u}RC>cr2YnH{Y4q%t@+$ryytfz(*;O6a;#B@X+9l6^r^&uRD z%XkAgvE1Ga_fD|`8-I&}j~|<8l@u8nDJYtWqzgTzmgWdAp)5B+9#7l&jc6U3gCT*n(b85J7r4wcUhDG`RXyK7O+o~1O>gA~$ zv1iJ`vc;dzJW%z=e+y%BuBECLW_PJn=VGC+szzAVtg-K}_B3*JD$77r>XG?mytAwe^k$PL<)&kVL*;Uwxh)PGgI(dvQZoIEN@9cBRw2VCFi*i37 z-Al?2(B<>2g?xM{y%EGYj5P%z{gE`=LG^z};?c0<#xdcRDWJ!lOPTQenyxlz-+azD z1g0x%^zOOBaLt267zaVPFPri$nlGrPu9K!zXO?KZ!vBZ7w`z!UTed)JpHz3)9|uXTRGedB>|O4S%OYtE8pqcgYh zXv+tGb8oI)m}dr87UX1);NBxCUUPN&9)G`1BOe>r4#kF4CrNh$5w$fHT4w0~3gnHg z{Yt>M*m=J|GOXJCpoxr3H%r5US121*2v`zCA-!$$IC;?Q7(F@2rY{;L$8VX7#5t4G zPjM58ftqp~Nr1`wc~Prh0GoB61VLXGn9l_{9gQ@*KFUzzi?cCvp)qNkTmzMXpeC=G z>_tvmW2{CwSsN+An^rH?1uZvQ{kBB@^WQSM8tfZx+MmvuqH(ugs2=Sdmsb+F!@Nwm7y&Iq%rfu=WDjw{c_l%9%`l8-`W(m>vy zQ0Oinb}Xn^k9mn*%RDhS@nS%flkiLxX8ZY5i(O!#X+}MlJfKv z@Ch}VV%$;~i5ProOZDGMP38dpHlJf>6T-G?@sZ|IwC(B>&CsHGEt6-=n@tY z_LpLeMn^sY<;I21i1689!f4onKYHxvB(n0n955X>LnGaHU7fRhx;R8jnG+~VIW{`6 z@6V>Ec>58tj@LdrADrZcd4eXd8>^3?{K@reQ{x`t0IF{4;Nv@y4k5G|#iDLyBWyB* zFYZJE88P2I^kF8K#}je?iqV@YQ+{4nEGbwsuj*&5bPwM;8ZzHEsg@1D?6qanqWYnl z-+YPa4S&ZaTRvN>-Z(LC`-3hrmG0p?amEd7eW)`i(ps~m=xw3r_Z5iVCnskf_`*3QfUawr^z(4C9%CRkby3fA z%HG~2F>bNr!78{98EFhZ**WJYVy*j0KaYeV@tRn?mx6ysbtgd)msCeotAy^RM3lbN zS*0^#;Ud>N{vF@8t-Qmogw0)azJ%vtabY?XhQ+h;p61wG5A*k)oW9{)joZn)>{r6h z!s@qq>MMbjaeVbP-5(|@WNXyJGn)Ouceiqwjv|x-q;ePW&A#_QY53SjFW$ny}h1Bh#r8G z(G7kY({SlcL6E8P3$J_(nhDtt3i5Xe2oU$7Angv%Yv$^WJ$B~{mw zUkr?WpG_7510&_lfJ)bn2zi71*_1QqQN{hUtZbs(#_>?YaKp&n{+Q$LrtNq5&F>l| z^7)&0+9i>+~8)q$5#S>jOTLO?HJ?9kEa`5hOTRWR`_sfW-&L! z0zTna9j9OqJh}M|@&KlxXS=@X1gf@zowJIWM7HJl)h+Z8QB#4M_~~;@E4=Dybv}(cfQ38Fj+DN|VZnwG~NSUR8~Fds#wLq{Z?xx+BcRvmEEoF`|5gDyzK5XKK;r z3{Daz1vC=X^p=tf5kgmqfzVA1{N{scFsE)0UBrC7xRl^&nE2K&oh`RMjgsh#{x~iZ z;yX>=t4TJOEw$Ak1-a}C%<7Fgs#pn!ETwMU@^$0f>dE>#rv?bOBg#L;K}|&5pecO# z1e0NwV~;o3n8$tCb`Bm5h-i@6eiHs3Tv>n}8~W+J2N%*$e&YC~Ei6Q-{l1ltV+O8} zOYmk6Y}jec>1!GO^H;s*SDQ%(y~3V0ZOTOG4)P8d+qITm+1bra%D27wSU9wuXnYM6 zn6j;!DS4-{tnX_z#@4=U&%SHFem2lE@XnzzanlyM#i;fJ^}=5a?zerf5&r5jJ*@?g z*G3Ru)21lFr{{#gzy{MXFQV%FZyNzy)P6ajp9jXead5`H4n+MDoI&Z8^eCeX6&~FqTQJT z=(3J8+~Mcinw2}Ps6_eVWp!Nm-%>76HDb1y#Rs*_F}6EQ={W@g(k?r#@=^t`YR!iD z%&(RH{hn=9ivxLr`V2-vWOWGO)KiReKzgfA61T?SM@h$8h^`%{?{7 z`&#d?>5v#3JMvSF$~>RqeoYs-?9;;(d<<7`apT_P_f$5O^ zeXFq;Q>%cRnqv#;9>fQSLoP}rAc?4WZHN5qWCM6rwpeXs+)U1`heR?oA7dFF&`yrl z^Yjk3_xr0@$Xd7(y)uFw{I4?nYj~jp@L;?$0DRov_y@Aj1XWpD!S(*PnGRU|i^be2 zd+*O=%yJAHTH4=vi`#-`(+_R1dCj5rl(1RzXx4kT zh`DcSJRqtJiDBX_lt54EWeWWq-)5Agnz0X}p^N*bOWR5?;_o+>nuCO$tZ5y`R8nP| z(9c(>{8^2LlhJB{3I7U?NWjw32C@5JK$;uq*Yoxiv_C`JxL$e$pdV~a>18AvR!G^G zUV=;)Sby+*MP$*bN3`A0=5taewdDS4i-@b=S4qGsTVGZIY1lMfOA(!?FeRC2+dNBZ$dgH% z$L9N)!adCIp86+f{?Q5<_c8G>`3^N*x~D{T7QdMV-$8I^ye=jEud+;N8x*D!uSp!8 zG2SunOG;fZv-sz74+qnp!JTT!R+AK)(mzpsACK=TWBG5eQc+>U_F{P5c_K2Mm&PE~ zm$uoE7ou@ULpF_yXVn^B@B^InlJjkU42x12x(yEBrek#Wm`fIqd%eIna9yd#n%II2 z433!4ClG_h<2uWes0TDiVX&U3ZFs3?ns~obJydWhg9~v=Nh`IKY%v51g=eJ)dPybZSHU%%qeKx<4slf4wd$>Hu60CgC8_pVK`I zy}rJLn*1)bxq;_CzlHsh+GwYO3Jl@-!q&?7g*d?yZ{Gq0O@?;miJcCc=AuO9+(}`* zZN96xx-#)KeYQsV_WjVYxByZ<*UN+ib#H_xgf{?}AlZ-ebJzi~B zd`2S-3EG=}iZ7nreLu5X7+;T9uHu|CR+;iwSt!nf+?U3)e;0=K>hp%R0&c6tDy!XaGUVl%4`~t{oe1tljVO1ZG<9r6Kk4p@){sVV+L3O%_-#*l*lQtGNo1 zk1|!CUB_sKU2D6dD`rB0>rJ#vZOBY1H$KyPJvzDJY{oT|Y3Yufyb)prRH)+oq_>( zg2wp&)qMPM4ZyJfXOhOg@4u5Ye3a>EytV-St*fpx;0ao^$nJX_%(mm0N2T^xmS`E( zo;8w{r11^JhsP_sqja#%kxw}#{NCzn5yk#jpVY|z}1 zm%R-tSKws=iXgbnqViDqlQZ*Q9?OI$nIx8|zwgsWq8_j%)-^94-RW42$~Z?t;Z4Nd z5*A*6`3arjIMfEoC7ALm3*VHYGiVM#84l-kek#eGmpulo$iJ$3zsOJi&nUm~wB1MRb^3iZL5*PWnB zsvIZ!b*6v9RR$m1T<{0Od;ZJ;C;M&{o%JdT2hK2S+TPSUc$R9^K4|lW4gS@Cjo|ocDSSN_nSG(1a^J2%d*Q zrf1Lkc*_5cPUTi6)k1{AZT+RLT=mkVVQVCF;hTRE45C&ENUH#8N=#Dx(A{g)F&pYd zDLzQ+C)(VJv3Htw)YNIF)nG-kqh=qInn`O?FOwEayy72EA4FI`2{I`LFGF%3Y;0AU zOc#tmf#^SbH88bUAqfwc7ml@ak>0m!JKJZApCBm%<*Pz<^nc1>wdcH!2hf*BwMgAQ z1H!oOyH}P5<~&b2TPy?G;OT2{trUwuCq~fl#|3pJb0rJ*QiETs1jHDTh?wWMA}xrW z^2oF~k}zTlD!9i69{#G#^~Ea3<1st*^upP;ODt)`{qFu0qOIBe#Pg-}Kb5%zDEpJ7 z<#Q#K0nm3A{ieAu415#1beBHb>6SL<@l*?wVK`E9)ErbusHPbJ7$ttkkfUl$1}ATTsU;D#I+CcTOuX7QJ+l5|5S_<)=Ub^`%g;|e1Y(#|>mL$NVja&!Ulrs+ zlL{=A5abdjYS)Iudp_6s>@|q)^)}jJ3fA!sH_r{o5U68io@-<(9kGn3@Ce?Ki?e^@ zd`^IHmQZZT`5L(TgV3nCGF*9SeNY9by$5QmS$xiv`!^kUZFa>@C#BK+yTbU zSU%cPDY5HYuQ>i1pB?Fz(P%tWgoiI43egxk!iaxEFwE7SRBWxNbN=?v8fxaOj8)!* z#%$<6NgYc!CMxFB)hRChI?|z3HVU-b9Ii{>{r2qk`P@A|=XVz4A9E!TZWu|^ni+fe zrJ2xNwA&6aL3)|dp#~WXm_ALr3>tSGi6z?xXvs4A6#ZO(^Z1qBfW^c1@@B1&CV{Kw zG?0i4O=njD;Y%w(#OGt{25RDPuzJ?+^HA*`I6NqciC7QGr$*O)QN2zz?S|(p^bYNj{JCQK7^#e+O8oc zKBprI{F%4Z<{@tmLeIV8IZ!d5R0w4?s>oAh^>nADI2`jjXPZP#(UT?#tV}Q>DvFh723n^%a^;)r6dgm(@9<&D9$6cH}0cD4}%;&xlA6tKy zd0t5NC#9o9yZtl~3PXMdQK_*nIUo(K)j z?7o%Es=2ij{a=28F1HRel_=lJS-sS8ZBag=ukWD|%$SL};8zannLa6KGp0A_L_Mj= z1n}C9y3(f$H2j>!l}dNw?H?%;?_e6epDbmMoc#R-QIQX@pq$vobq!9p#T_=F<`4no z+-5kPrZ=Bn9}gV3^0mVD;+@UG!w-`PxBilQl%~q9=kiI<(6GfBaIfuaf>pQOx6zd) zalPSXnEW<%^raFuy0ZbL19>fq4#be;li0A?Y1_C`em{-RBIT)r@9d&Q!@vdid|K3C zF7_>&CMpun(W3HO-_vYdNz8}oeFfGnT;q|b;D%HJvf$J3rJ|Zu&VOmeUM;BuaEN5OfHsvuYSI!am=Xb&b^(I3;$W>`T;3f(>y<*+jS>I710Z#+|& zq9;m{>ODNHvA^*W7AQwGYwBA_uTV-$kKPD^XRInENw@RxioPyczlYSqo;We1ttU55 zr)j|RT=lg5^4OuH-^r*sPRdwnn=M4HxCbw_k=&P*3Oj6erH${32Eq0zK*7ATb)>ET zNKpSaLqh)$r6#H@baC{Sv1cCOP7&GjwrMf+ZAH5}WB=)P9c+0ZTM$#dZbX!ZHc9#S zWClq|RCc5{ZxU1F%(<>IUFZ|X$2$@g6XdwqyH=)L&oX_^S`K{KTh}>NSZBsBb{q;G;jo zw@n$N)WHxbf_i$uxhf!$5wjCOz*Y+O$q`Y1p|QJ1uG#BEj+weAo&7n>X|+sRzWjiH z%Qlbr<{L}jU>~s*%%eA^!-HPukZkcrh;N0F{wE}46#Xs)cYIC9L#G0d3*Ihn3-PL! zdMRl#PKO;N(hZgAQ^~7?^_bIQy{^qWvkiNU;xtKSiQ{ogJA22<*7g8E1x1J+mt!&j z_ndnA!01H333#PipV>F!cUf7R`aV_Kv7xoKHCn4;teK`^F`b6o>V0MuD}vGwZ&RHz zWs|*j(k7#m^4~b>H%=5#?T8KmACHGkD`g%=5}OaPPk6nQ)(bfYD-L$`)ABoD0TT!3 zabp3@wfn0_H!8kUBW=GrJkSWmmu|$vXgyEp+l1lA8|SkJ&XSi>TT4C;8mtjajI=>* z=%e0;uN1c>w;TomGMf*{zjk;YL6OUkyMmJ2JlV;oZczsSIWKd5fOC>1pXZKJY`pX|>Lr;qJt6JO1sye0z%xx3gZ?tdIKVk4hElg^NVyaqdw zMU;)T$Q*{vGKq}>w7|(J=fzT!pBE7w^l7SleRHG36tD@@#jtA zHV`7X2>EUrPvFhz3f$k-mq*{wpY3{JAI4I1c6;WpvBfC@u74TN)cXts4$#XE@J$&9 z_m`&}WM@>3%P$$^-|`E3X;}zQ&)qkj2}dcLKN~+`r~G1JX>Ga^Uu7d{mJh1Np8oCH zyb~=%Dr60^^oS)a+B`pY_cd)48lH)dWX45wfv#JMOEElsZi^?~o+E8Dv#0&rOE&V$^_CUuLj<@UbGirCp-= zrp58V-M;Su!T7Cr6g}--mM5;lN|)5wFNWAxIa7xb>A`|*c)1Ur50RnvWR<(sKG6h_ zy3`#R1}pGn8f|B|`NTxe%bmZJ;_k`b#ZQCk$0{;%1fbGD44lHiF@iwAC2zU@n@)Wn z{{TqhCx#+M8U4)BSSR_MqyH~tp(-)$FpZ_K2_)Po`yoCOAnph`IJ zb#Bd=hb`h>VXxBJW2{wr&}I6BX^6rCn`!yeef@TVUb7YN<<-?((Bi5cq|ubD3s+5~ z%PRdt0)i{N$)5Kk+GLQ@`2C^&LhhmK>S4spsBPSU>GqHD-|O_CQtU*J76U_v436GU zQA0kg>%u3O6at)*^Omt3Uao^7+bUQ<{eiyM${_6ric&IJ^zW1kyEfwYr)j1l9;#wb zXVkUyihcW1HtKsxcm&2j>D(y!TxtX;3d_D0&7jYiE!$mIR-$SkwBHM5Gr7v))d@|i zd$hZabVF<>Fi>;)Mr+0Pgp5QC47)gMfOxWKDm$lfM+QnQ?W=wR&$X)Yc$e6d+ZSVX zw%_2kO@HWj7IgVq0jE$bagNmPJRCQwaCY-L#D0iLaqxP#n;in7OIvr$>R#!E?%w)X zw&?C`G&giwQ`r!nc8XzvV5e=0EPtE0KGT(XMF8B>msMZQ5&KeETmTjm`K~G3c z&?7?xdGdV}cO(cmH_!nfiZM8d+Ec-1X4ta4dpY9gw#cfshU5E?pQa(U^?@9bE%gca5gTG)xcHsLBF@dm?F+oUD0QcI1LOL98j%%1QH%gHd zZ^DlTq{@w@PBGDre5P#X1+kZ5S}Vb#LP6k80*h9K)_J4?3*TM#Z*oaXi&x9>4M^=F z+!1(sgtW!RebaS9MI=*{m_TTBY(O7Jgl07E($~XjS2?2&nYUObw<+3!nakyyDd@t* zGoVd?LMKT3lR5?}ne5p!IKOLcGlr8n(ubl$mm6xM1z~{!Qw&gNam3NPGZg7U5LhfD z@eKuzAs3&as(NRkO9SjD!jmJ3+Mo$89tVdlzbt^Q`sRq?KR+4f$RfTAdW09wp1F5; z_A^uiv%;A^X&*pgy`Shc)iE53p5Fl;RFEZ;3bVGEc9<5jP+*j=h96DZl4Th2ZJk8+ zEuL$Wldm|}yOg|*hSL&qVYH!rP>X+=HeIYXMrpL&h&yF(4_s^WFx?1(5pdteowWH- zw5<|WMe#C^j#avzRUBe}DYfWVzDnzOS`yh%GWgglcWJ6?lb`0`#~uc8+h`JOHpr;F+49DU)->D?O>xu>fm!4>%$ zafc`e*<%H^GNL&Rt&fHqF?3Ere1mi*+Lwq0Z@iEkt;xo$dk03De(^`Y#B#nT6(eRu z%Eppn9^W^wX~LSBF&M>nc0{6)jApQSCaX-gaVAIs$Fu>$^w3%v%Q!Ul4I;%Ki@S~OPJF1 zsct$alU)%s{qRm28QL>kBccjpGU71J1sf0R&bD=}g+(c^0-py)x+@Q;1@wZ1QN%-A z0!gzs4gGEVNI=a2$t1LcUjuf!XpC?>CR$MAx-B-^@xB3-%h1Ja=QQ7l-A6F_RbwmJ z1t6x8caXX8MT2e?<~B~tfOg&IX%D5P5RT%=YK>beR}F@Z^u=#1HI2{)&^f0BkR=WR zVUO5$Y=1l9wxZUBmrs-kw$}UkC+pabC=J$Pr>yyDPmD8m*Xe4RzAHWU@We5|tcjL6 ztwE9H6lAE_s3J$s-64Fn4)-XG)43cY?)<&qQR=OE?}o6^Ov z>-W(Qb}(o5(M>WdnGyF;KKJ;2-hMXc`!-$$RMn$qA$$s^5II=WtgKJG9v8BCRK$!w zthVREdr^_zf553?i{^-cvf;15a*M?8(J;ghK|Fl?ra{X8J!7Ypbo%Sp;A>PMnN-P2B}%;;4UAk_XS|H%PhKy1*WK@QXzVJQHmv#?JA zA+RgpaJ`p)eworj_thyIQoPWW+j4xa){NAC$1i-TS^xbp-Dk|~`Q`^6YJY1P>MA`E zRDkmj@@R4wu;g!c5RAg5-Qs0ZHqZcL_4j5xKwBct?d4wWNTR&_5(juh6oIzsLQt-u z(6S@O5>Ci8oexd>5GwF69CO=kj!eE(gZ7}7;<=kZG4`=uj$3{un}5_S=M9U$Kl`^e z-!286yE3zZV~)CeNJ0`OYv2kGt~mj>~Z8&HApj08_>9;yJ6D@?mSpW zapX82HB~d>fL#Z7&y<1h0#w1QYXhuE8D+_@>YeGcow{#bxPNJ6y%Ba_d!l1*b+8*Oha{922q$JuU6qT zvOS}j6As%$+?i{GkF{0#kZwCGGn01fR6I?)+wb>mhUf3u&LA+AxrM-#qEn4B3D$dt z8=uf`&E ztx7;pexO#GHiOFXB0sZ{AQwZ z&`zQI^8>qkz1Q;xf#c_iHaG3t0K$GpZ`A!KHRfvpzoHNAt2R{O?4Cw)B(m=|p(!ssa_A=R}LJ41jjy#X6-%FF=$f?J%VTYMKv z^{RaIn!Z7ga$&8(V(aJhjZUZ0CNQ4hSk%^Sf7hbBRQXIx+hw+pDI<% z0$0xfMOCE`tiE@jqhMOhi7|joZF3tB-=|I`c<@ikq52E7kF|U}zPePF>%0i;D?7oP zn4v+`vbu5SUOYLBb8xKm?7jof@=5EB$GiboUG;uVj**-_pDIrm|Gocs_Ali&t`|rk zdJdTYruF@T!Rm`4204{0oHwU=c{fMSTLVS&AS?W0j%oF>)2d!=HSpo41*%skmAP(Cu=B#ef3 zVJfRys@V)bx-iveE+KF#uGw~{v_izh+G-6w#zjO6;=7AQg@+}T4 z0D;{;wmVhj*@+r+^0^S?OSC`Buc1NI`WY|5zy4jaSP9bUm1jiSyM8?y69%yf$_yK{ zy=Nv(?Ojr~cUWJNrKa3kfT?$TCbVpZol(cJ74nkt@lJ%)-hdcFvA5&=dFXm6UFVo| zY|a?SZ0jmvrx?o46%nQ{&tivzUg#EILv~CQwHGE$;=J@o-EM{0V_IEPlCqon%q(iG zWhxw-oll96D*5y9;tA88Vcx}Ix-2$-c;4%GraEmIZ}?%@xKloX|^ zP#NWZEdrR(&^Xflv&RP?)(8@Y=`0CW12?4pw~EUe-4eDQma9q-=8Sn9)&zv311oL& zh-UI*Aa2qo8zy6?tk?w&=eyr9Cm;6NxbJgR!+uwL_`v!%&xSS00L?1~=l~iA;x6s= zjoh(-Ez0xImB3RHYD%zqGi;bdKblDbQ|y5AdVX8iyKHD>>zpAeGVB{o9Ko(A--p)l z%sX0rFE8d9(Y2UF*;SaD(cAdtB08*9b#@w#{*JoJ2%<4BJ>QDDR_P_R28+{kmcGC@ zoZwjB8{gkAouc+l_jg|t=q6J zBu0%%gv0{wB=_kD;zJImZrJWpr{@D6n;BL2x7=BgXMlKO8d}_aFzl03QvUqsusQ3{ zAC1KH7!I;=gO2dStuXly8>k-Y{zo-|O)@gEk)x_H-)W$SLv}Vc}SI8IVW5z!B^nr`imJEd1`U^^Y>tdb3}FhhMfm}z7s)9 zS-%54&Yqu8^GN`I>Xu<-gc+F!&>LAz)PGb!8lf`AcjG|ZI41dB- z(G@Ytp$h>g#WPOvxNWkrmU#Dri3R7zKVp z08u<6j>ck>ab;-akCd>m>FKBU=jDXqm9yw-G}EJ7=5II4Ymu@6-dM#EIHI;;8&Hab zRedE`sCu5na4QFz(Ww!q0jHzWaC_TGy;E#UGnAFa>vZNQjkh(OxQ=*VO{vCgv`Z0N z%_EA#ik0V#_j$5Br85_}w2CEk9oh`?VV#W#On-Q?Na2e|?wX0;z2B4NhjtMpk4}r2 zB$8Tfbbl~@sN}NM%jImeP4RJ}fm?-BL~!4>?(tZ~SrL!*z7WXcF1E5arH1pS?l!Vj zu1`$?;O0WXF-So61CjopKc_>x2SN?sju~!f?oDG5h1CQIVfd9FZjTSN1^#AncX;Gy z(W~I}FCR2WYk{=)+8~#47xL-1KIiy>0B~<3BLF=Cbi{oP8&Wdcj|UUv4+U&WM#GRq z2_S+yFkcP%+kCl(gyiE_!6|4|nRCT`#!R=NbeqWn_%+ z&pA`SW(UQr*vnXBBSg-@el`KZwI1PR+-D7Ouz=IZQ%1-ba%sDb%D*B1{GU7e*XBAf zt9gIz{QShS3i466K5y@myieA=7v_?--gT9Bv@cB4`;`f;u$L5 zfbLLj&jJ9~iZ$&~mz4oIhW}qV{rk^z6$y%*h<-Sr^+98O-cildjAlO8`OJFA58PY{ zag;@!@sh_2aT^4aftsDOPGHcYzZk-nrijmA*b0$_&wme?YY1msB&2u$cwp4#E^{Ri zk5g>48}N*UB^7REZ$S;mz`+5Sl(4-zMrgpWjOk6?R^ur8Fugvhj(q(#P}I~XXi)#} zJ70&rMv~&N4ZXXd)m6^vr{$c{vKc^Fk1`U1XnP9aY#BTTJ}5gUj=&qDczmHJj5s~G z!i*c>CM1co$g(T?-|&zK^2dt+4+8rAXcmq+Sq{Yd2N3LK*WwI&>>Io%hR`6lZ!jD& zIzFRme`BWan9Y_-P8W{UmH)PzF#pHgCID$sZjpqh6c71Ui^Gx9iR7aeqDx+Z@!0<+=CO(&OK^*?VnQ)rA_tBz z=9HTX<-d3Y!G|Zo$H5{!5rl8R(RPb%eB_Tt({P`Gi}9i1L=j}y+*r$ao2fP5{rs1N z{u@< z5-$oXAFsx7s`-DpWyR|#&3|H*|fj z|GkXdZYadXQ}=p*ZTPFme9MQGwrbx?s&Xx{kd7HxAB8KZUxDGd5{Srz`ME#^i7$k8 zO9HPh5u^7adBtWlM~y3jq>_l)XGYf8AvHa&joMm&8~@+SrnH2+meh68%>0`u|3i2Q zOe79M5^4Hqjl#ce_1~5=SSzq+aZ}WzT=0L{)BlVB`3zGUfI0|2Gh(9g-@N~~?Ue}H zgd8w@o>OEtN`JF7|Hc<=JVa@duC>}4|L1NzA&5jhzR3wd_yRubamwO2 z(2I>dKQMsuK@yk|vC#JNI8`KWsXQv-!d)eO{YZgI3*wSGS^6wN(~k2&|ESCEE`WI* zS%7C>99Zm4aCJaU{@nJ)>Mz##E(Z(77us%ea zAO)=x29{c55a@LD9L6}dXHYE{1CTEZ0AJdPcKQL)IcNMptdJonl;A!7ktIQIKKf0> zSv0PkUzWLR3%uG;W)aWR4#Opc1H3SP*BnZc#e}URm=VRGT?QbQi~SRgs+Ud}yqDsN z1588Y6o>c5xSrt!HH*+|4-DUCnsU*=SP)!>#(4$nC5d?~G0?H^7w;i+MJ}`cLGfG=ei&b?8%`Uoon)(&9YGalN0pyM9j~ zh2{7i3eKDUe9(y`d&yjC$?EhaoEDnhHmci>sPiR@ld$Y`SpA=@3T8E6>a2Q8i&~DJZGymJpZkE|4`dOsF|d5E8RUi;bAD6pmB8)_h|iz}u(NY7e`!DiKji5r z+gVaqgB}nIfLG*S%{Vj%24Vkk!=3=M;?q=cz_L+I>jEnk`8}{$kR_-MSzIMI01i3Y zR+~P!HhZkOrn?7B5Ogna8psmBc7eNN@sEQU0dCOTMOsxM>x;z#z7rpluno*&)1*g* z#XmT@_!*!Pt~sR-rTO}ntiUayV7R@A=rh0l)y)k&F9kzubA810ntI(%mJ#^=Jp=zoZq(b;esVa#r6`% zuA7f3h?XyCc%$OWc-)R@2l>!DJ^QlG_% zk=d=bJS}nph^uoBmu*lXdxUuB{YwwQ|AFLYgAHuGmUSQbPgV*xrklBH`QpOs?g$l& zyAk|^gZjI<>>LLUUfJjnlWZQp1WXs8Wjhoqf45$9q!*&)K%Vwb<9y83&#jn<+jvfC zwOEUyIE;tH__cYqAVItyocl{M?_u)eN(EgKNXG2b21iIuZ! zJYsY*hlrvM3W?q;1xaAPC&}+?Z(9S4>9Tf1ry$)!j_U&M!Pfs=!TyKKAOTJF(QPiS zY26KDFz$U2)3g)|wpkY>g)bZF z`5`@<>&I*-MvF$su+825NJ5I4n+794EduKf$iyo-2#@Xil>dog5V!U9x$CUM#4PkI zZH!|wH-}I&5Fm!Q`ow#)akO$Ox;Cfl%kcHn-xu&_v`5))^SYgWpR=AtmA;^yLzt1j?=zGm+WbMd)7{^{3B77$J2SA9pK^a5G zj#9NgcJgC{F>VA~5CZ)DYfLD>WDTu;8J8Kv0jwm~E0)Mb@qC@oG=w=KIoafZ3-aIB z7XOD2d*wHGu8#JLfTkRpe_~Crl@gBvZfm<6d?7W~1V#TalSGgtPP5~TU%U3F_zZi} z`lIu1h~;62wQp5OX!+370;8 z)`#2w4r6!53j(BW3IHc@K+}Xdyy=Va7}r1PFK`3))#Qmx`P8mc!KdA?(7CdFVu=&~rfry}(}l@&=l6}! zp$rcr^;?8|N#Oeju6LAls_l8GqYC?5Zy~mdvK)4NPT=N@QC;_QS?v1^SK(PTo>nz< z%BYq{mUU4#@;D~6fr6D{qpxK=ffW>~Qk;as9Fro*koz$d;1VUU;e#(UCO??Z>x$j~ z!?!NVF_M72eZ$qwv5bKALyYc`O)7{*AV(6Y{A6OT;niWnktgdu4hdM?8%nGz%>8Q0 zt-=4Y3-#|_0HuvY3r7uSh&WIHfV)ZfnOHiqeezc=Xui7 zOH&DQ68B-}bBHj)%Wc;wt*<&kc-gDmZQc72<rL>Ncm0FtJBr&hvNfL=e;Qgxv>HzXXS@`4g8sAP^vm^;P_{Ve@R@l?K|qVrMByGG+O}4dYvj5ETn7pr#I1eq9vQdlG8b zOEW3h<$j=?`=PYBcE^2rh+azc&YDwU;twtvc7F~nXC_b{>`b&~=k>O%8%+8UjZrCo zuL(bcl|1Ju*aEY|T9$6eL{UD|b6{sM{$jMP-M|?l`nS59zZ@vu?kAWuy)(CC8<;6n zTb&L59rEJ@O@mT%zUxrgLYl*UHkW5_oZ`-s)Pw9*uZGx)C=5r>^gS&%b*_8QQ+ZA3+k})Kd)0eqtYZ_IXLMTplt7t`W`oF-78&4R7-CJaXJl@ujNaI1u_>8bCX)C! zm0EcSOA*FgMpQ6(P7wq>eG~`XGYA{RB*y)m|6UIT;K+L6S)y{6{n57*YHO&sE`)pP! zC!@v{<{(1@_gBWS<)eo>!|Ra#j@cE+3b{ z;X2=yHG+U2tn%&xPU4RyQlRn=gQyI-HkS>fQGXacHPPfCnA{VCh}5r3oiQ- z>$2sL;`%aYy$7$)r{MG#2?KElf}pTD2Cd=5WZ-CC2ury;fE44I(bx;_7WVHZo)qGD)8{+wubrbA z!eh3!#i7NsjlmjGw2)KyJH2DYWQ?lQmEofbFWUkSGg?y60-+yw7|8$u)C>%bPCK;> zlxa0S5eF7Ve`Llh-`t0;@xSD1P*dN z(lg8nTFEmV9aLTa7ypA*?dIS!PIr)J!kqr!`MZFIN^2$?7|d4YK^eU30Tks|#f<!?|^&SVChB(Q{mV-~j0LJw5SIzdYt=8Eu_%k7m>*;~5 zkFf2JaYW&90i}97`$}zCePJ9Cm@x-wZbcF$DAL6n(fl5`o5dsx-wPBQ`EXinE^97r ztX^$eXCh{@3uA%U8$JywF$e*1cA$)EH#JUDjnUq84P{gFD0ly&Tcz%d0FeBS!5^B5 zz)nW2SIH`CG68!>K5zwc_FBGuE1h2}Mkt92WO{s7xgCv-GF(xkV;TM3j{w$DUSoF@`)Vz7g@8D@^{7!;$*P^U*J5{Q?#KL&_Ghb!|$F2EIr2iy|8+iA>5Kv>Jc zgRJnGewuDD*iQr2c+W5O8CqmOJLGfUjpWPLNiCCs4yAc+5rG@nr{?jlLc7?Aooe8$he&e=SE0 zsiE@HSzg=-m zj7qKaocqR*#u2T;#4|+a7}M8%B=|t)+$%E`@!+-G)hzQ<07b?g1otq^MtF%C_QK~L zoc*d~_8h-Vq<7~eiLTs?bl1Fih*Ceqmi9V6@#<-}l_4sJE-a<3lmwW<+Tc^nW8^4R`~2IxWyqMuwLhzHJ@ z-~oFy7$p~9Nsk+vZm#eAxu!Y$r*qrKP0E)PD*9v`4@@($OjG?1a_l8xhmvmi1;_FO z4Pljh9pTs`DS*`dr-#}Xdw!PEe$&^1rznA>_~z2PytLX*9jsUcWnnvMDeFTpOZ?2H ztha@>X9^IcRCa?l1gl_89Aa8ao{dDB8;IKT$0#O--sx**4|;=Hk_>w5y8a)!-ZCn# zXUiK7?g=iz9fG?<2PXsz?hXm=?(Psgc(5eUxVzK12G`*3uJ0lLJNKEH_s&}Vq3G3| zv#V;?_TMgEvy*q-lz<%+$YLNI>#aS#_Vmzcvlky|9qQxX%tx?%K9y`dm_}+aM^lHP z*|M!(Ym!@EiGBXbly@WuZ;RGQsbZh`r2r|G-lnTx%YLt&qi?w16_t23ZOPR8bc=Gf znN#~Cwq=h$#5k_1-!yOELPofkdaJ3>sU@okWmo_&%wa0>b`9x)-Iq-bk3R72PmZv} zMJ~^ZRs-Qg8p|!jHt`W|ET8QRrb18~a$i4rEuQZyU2P5;ao;FH@#9K6#BPsV$w(yc zUMb&@@vkYTGZDpZx5Em8RVN%kg!mP-1NZ5YZxJY?^u12iNW^maAB-^Pf)pU@?=pBL zi?Bf^9J}wYNCN{fps! zdhR*i?X<2u`?p#Q8ATVPdD%xe{qtX=RGfd{B_TYmWk@udBi3?U(5+6#+Bau

qaP{fO1$hq`(*Y7&Uset(>B?GRp_!>8BDql>?eZd3ZW>G|~uA zTg0b^s9*;cg`83>un^}%-v8+YtWiXHe_UBYEk>Sck96#lz|~pvn_{3z7cCw=)>m2U&oL3Z62O#P_L7+dK)5; ziA1VV;2Ja!-S-wer{1y+U5ou3Jm&5R%FSLprcOE0Y1Kjfwd(&Z%m$Kq`N=?<#uL6y ztn#ZBHeiU^L37(DEW9%-=c?(UAP>;r+7$A&UO?Z3l_q>KJq?ihL~0&kL=%IwWj(F3Pb@JA!ljSdBq#eJ2EgjbWwg*$T<0W}C`vLr&=xOY?OtvIAsy z=EN=}FQ>XW`X_qFSUO(goB77|f)T+YXih8(`@;NgV_Do7yt1C^D+5a{5B|boO+SdQ z4<>E*>}=h;-SO16JOVE54OqQn3aJZ37Y1rep%EMwiAo5p9z0uv#&Ae-{a06@c-5bS&h1-Vf(&^$f@QXR#t#+^qPe7A5p^YKBkX_HDUB$1XyPoH1?c1J{3%=rO5 z0#Vi8W6Axc8XMc4a>Y>6m8e%K#Ke$#KDb<7xm5W$@!sQQ9RoB#7w}1u!$!SR%Fofo zz9N1hy)<6$e8_~C1fi0cD7)YERFLDP)`YvHgY9p?ruh>iqvGg|9V(%FmEJ!wgr=?3 zVwR3?QaYZ0fa;M_?e>^Pvr0i`9TXpT!*m+csK})R&C2wKie7z|%BnPrm8t8V{EGK* z3ZsZB8GY@xs88fH+4+mrY}@d9JmDvo>+jC%B#&cF0ja9XWT^!XOK)ZxK2Q9nLkrMd z;+NFK!tldnjtt@QeKFol>)Iq}7%Brx%lA{Bp6Dt31ogYJKWzy^XAwE#DN2pQQm1Cn z1F$~yK~~eO*m7rNC`uy@kxcBmbR1vLjJCcO8&d2pTx18`38Df;HpPDD7!T8Tvh`Qj zE5GHc=E|2TR()a>6O5wj+ z+1W_}C9(>=Rg63FgUPPjhn;5&a0@pl!u7@YUYQ>+I)Vnz8w~`0?}> zc$`6w&9ph+E*N#6yjz3VVi?rEzu>B@qZbL{(P0#`Try|&d;u#VszW8nES&{A(*D9) zP>zR~|ExILAm(im(KvXSQg`{h<$Ru483+=*`azlhp&GRodj7D4g~-?FZE1XJ7< zy#AOWw;|JF+JuBAkugQeI7>IglWnB-o%1!CfxZ&Zaa%?`{EWSugupeui$!~&holj$+0vNkmM z3h@-Kz0eDa)!PLVcha;#f*pVGMXgvtX@Z*8{Eb<4e)G}{I8wl@LtL&!=J9FfqR8XR%XYP7p`LAAFG8+3t$vYF& zaC@qiyGtDtW(F6+yBuC$Tq74#!_y0KY}_KxHRwS$fvW8mv4lbrSka`0H9I9gLRYZ_T8X_sxT} znG}gB2eI=s9#iu@O&WMYHl_&L?yK>}tSn`DW1dCDz${eR*nlYYKWzPuBX$ zY55_NECjW6faO&stz_}=?2~GzKT}(Gbr@t>Ywumpx^0q?SC8xfULYAi4)`4$9T{wf zaj%78-z!t)sV|$4yci|Z;6LOaW7$y zu>9tg@*dfj=_NL$!V7~NlM~#mM&APIc9wtSu3KODTnKD7Q( zIyoHeaH3^BT6&E-S^^lm+wSJY^sGD3)-+K1kGA)OSkq!cmH{Rflx@e)(2G%?1-(CIzy3B}*AyANAn zu;dZNQ{{3$=e&oZ7mVB50jb_U?IplpbDO(fZl^Cp}Z7#B+St7unOv+ncfPN{sXy#nsBsjD_EvvfXPcjOhaN3 zsTi{kg9f20l)G_`5V45Lc*b!u(Bf%2I@&$AtbimVuE{ZeBkWO7^W$*fGUAkxu0Hh7 zl)EU!G}79fVt5&N%nojt9Gc>_#eTpL7~WB>0q1hWH^$tSH3yQ8rp~M@*-udiOCbkV z{ISoP`+yAF>WAo}@lic7#oW|+O;Ut{U9Wj9F~#AW#r9zn@73uectXoW$0ci9D_}3EIF8qKHQev?Uu}4%WGo(`MM_+W4cfL*S#I@2=6D8s!iQa@{$ z2CBt@=*^u3NGE&4JKmw}Hw%Kn_m{7{u*1%#zd)RyfvXrs7<1~oxs*Oe)Idq8kGyw@ z@MIs_baRpUDwTV~>yDimQ?cdf2FX*mn!fu{;oXD%^~(>BHXj)_GV1|BE8j=N7~0V@ zpriRA{=)SA_ntVEWgeF*YNR?dq)|+TQMsGR_4Ay)S60HYc*x0yzGs*ZHd7g;o(C?D z8;bKoKzT`~Tx~Ek`8xx=7HaSrb+5yVcce%b2WPUXqF3zJg1k}H-S}ULKy+m2o>0B+ zBSWJwoM?IhEp2T!hrJm#^H0b&07UF_snm(0R~DBibqwjd_G%T#`sLQ1^6c5;M{JAL zgTAJP@%>4o4xHG-7jT-lMFa1&oA{e>Q56+zObW^#WtCHWHX}is<}XAOpFGYH+X$M@ zjml5lfX=1&R7aU#|KJW)PycpUvmA6IT7NB>Ix?o@zI`7eP+IKd|WL~A)W^}wS#0bZl_YJ<~; zcRS8~LjxF;ojw%&qaJ3TSs2X`jvR$2vsp>z-V$5OMsWY?Y_U=8{}%$wLZKJF93&`+ zoQPxi=i{6<56a+-h~EN)s@>^Imr=8&{}Mgs5=@W3ZI4*OYBye~NvE~7D1;SSvkNeI_s^3ET5c zcf_lmuO$u4%D{DCr{|t6$(z<<-w=sLiG0uWUbRPCLh_ zU|=A^o7bxt?sP9?!%&_qI*ziD39$Zt$s?|s`%96y!~RgnzH{PZcOO9#7L5ny8rCA<;Ml5G4I` z4~41XgN~orVOnb+=nskcKXH6zt894&OB72g!hKRU1B2#DH;Jj8>Hy8B9Xmgv>)z3t z*)eM%5(pOg%PX)2engPH*83x45I5Z@2?MaWPPFZQR<5D|!~q#Lw|#Rkcb9#%CulLCGJbjTL~-0(Pu zd-|My0Wdf3Mpbh}Vb2w^nY+?MT?{JJf+LsPzr}n8185sJv$wIcG4B1j6$a*TE+v|a zrVAVZT-Ejq^%YCzcRp-Ca%+-31w6``bHx$04WBpqA01&cuzG!<_(m?Su7{Ai<^dPV zHnrM-(qiEC_jbz;Kkz~+55oyWbuP`>k%i4mxFPhq>&KE6{XeGzs5@XZf__C1=Y6}G zayo45<2bh8^_Ry^Qrf|qRb()g4(?ZT=8vk3gMbT3H+BrIs;c?{bagqS59toax_@*N z1hByL6mfU0LZfvyAFL6ZYOvYHFDP-a(-`)4yJ5#PtuPTA-IdYIU8;tkN*X_1T^r6>NmkwH8F0SE zAU?!+kduZ4s4n#E^ZnKXcPClSD7v;N4Qb+rf2Zd$*a#b!mN}b+tcRT$i}ng0c&)@R zO>Nt6ItZnSr;Xz?1ax$LP5Zsfpl0^*oByK=3IIseV3hm{A=}#!a!Td=@_PCxBXCN) zSV_~JKz% zvGDSf!|f8RZQe5*W+rrD2VL%(um7#FGbo3f25Sa>U4xGi!$rONwt5PD?Tfr~>A|Sm zh+Snqss>$|@p~}4kuegeDO2#z&!-Z+{H;t3r5QuMkUig_?qULN90|S^>1f5;{K@sr zutv+UsHC^QYvF4iTLx6+QTq-FEZ>xr@dr9Ypi&pv{Hyg}A?u%mgU|0o_w=~(oC!6T zBNPbc1x~jd=qkz-G@btXC3^tEO=UHNhN<-T3V<)M8LzjF*bXboWj773_q`LRqV1&< zL!+X31Nq$`tI|c#XJNaL(G?DcKYV~2@drv3GaNh8Uc_DwC>0sVB%=0(?Dm^i(W9?H z1H&66vvWH1&X?gy|Do60z~IKC-{PTKb;b^q zd0oa2I*tncXcoCYs2UJ>eksrgk)iOyR2Hk1tmW(Zd*#Ac+92&}lv~g~-JUZ2yji96 zx;WL;X}lpQ+S4Mc35)1F`D4sVE(AkRw-oW4>NTc7+z(p(m>wYEmO(xJ_2qf^$jvuv zk%*f}Sj%qfXv0+leqb6DEP8Zy=%l2ixUvPCCMJ{ljd^r;w1i01Ru5)}Pb}~zLaI04 zpgX8fE+!T7T3dzd-z^2sX9;<#aqz%YrZF03e-4uu?FH_@V>n<#9HRupyk~dM&|d}1 zstE0Qbdii>cEeLBkpytpA}gq+B6WfIQaoE!P9ROO>BG(f8er+@>UUJkc+`{ z%jsuqL2V4n8Oj3D`19sTdWSOMbT#gSZ;Msh3jR943GdMzVyocggQ} z@?tH_O#mkk$eUr=!J7~g5+=#Ch`MJ5zRELqrbEXBiqu28znD<|wNS_s;qc?oHme3} zmB^Z3hR5fl)5>J2FT<@WNTafW9Z+pftm8|P32ZU&s?`L5UST#ipv?<<&Kv1Ht&`2h+rM+eH2`t@ zDZDP0^nZm76xF{{Kra60Pylf7c7ix!HX~Tt%RS_X1>i9`SiDX<=mUd;Y>rzWy^cLs zP#W*5IsulwL3RV!qvi!N2!djQ3RI)`H?#~VMQ62kbZ7~(bkgJw4KbDS&z4N4>by_XczYSR1@zcYC zf?%n{Ibb(&-i>i<44!HXiCdlhoLP=A8&WU_O$Qc0NN-cdjM@Lo9K{!eA_&uMaG(ZW z=8EiQ0@RL;LIY4ig2bMe#3?g+mcK@h*Yz7Vd&BRe%is7SPXiZyJ49AE&}5<{Zx6Wf z3FJ&|e?kLy@k#jdQwCHdz~_A+cm5bdK=|oLLiCIB1{_^n12XQ!QRW3nAbS7|1xE$6 zJej+Hmnv?otq6d~^K4Yj*zZjicXKYxXSUlYsYqXE8xFT9clAQ&X} zY=FRgF{2IaVDZCXfidhPB4_{BmT3kNO*=v^$7OhTH6A<<0mZE zUYgH%B>l|8^Q=O$rmVH*dP6S;?2AQf2mCKWMg~GYz?PbR0EP?R>=a;*HWXI$1@$!} z!MnjcZ>FI8>jaR~U5lHe6jUX9CPJV1QJSvCZ4IY6^vXS1dz=gDNj<|qgypos0Vpe5 z!(;`2f4rZ?#a)pst-L1)B<++w_X{gm9-Qz-EfaP?2?shAu*e=FN_^?E6~4RJQt-tD zh4UXC3>Vtgo~C+=G6$v)+xr8oKAuh>Dc8VY!7GxzxU_sJwW+k5G8PFp z7JD=nNH+7HEAa^n-qXL;BGz6{u3xA-^6(*yU2o(;&vgq^)`xIOWL zYWVF<0kIm^$baR=12Dq>Ut(P9lX=pV^UiRqQTATZTTXh~{;+P0t{yfOts_W0gzphb z-q!&t#WKHIgMyOH!GyoJU?nJh^3(6-7@dJ<&l+6q;VJV|n_bU{x*^7A;>D|;g|A=V zdW}EWVB?@X0TK4U%%uoxVPO%MlEOm=A^912G*^Yd#>Q6ErQ7I)q#%4t1>C-5cXML6 z=scraB>3?~vn#A7sq6XoOf0F7u3Dy4^`13ermsBo-i{w;`fKpK_Q?T_Atxp$hg>|) z%~lv!IPa?sp99HSn#ecd@E+jD&7B6S$;UCwlUmq zC+DPJ#so>`5^1lL(@k1$MTrYv8~8LNn?%lm<#S(S!e~F|ZGXxq=snm}Z@KnJ=eWRy zXF4pv)=y2hm+hkQuT=OoZ`MLbh1)rXm0sE*M$XY;D@`-22u9>0Kj(H%wNIOig4wzI z!9peO^)_Ck`iDz;sf;RpH$L;X)1IS-_g<8G8n-ZvTH9*hV&XaQY`gro&*E^!F-FLW zEJ8Wd|IsKw**l9~RGH6ZG5Nv-;Dfz;uT}y-KvIF)ZPCvf4c)~G5`-wH>A zeoP-xYp}agye<(5@Mfq~6=E*gezSgL@AwB(YU-UhWeJe}Rs$jRLMw$IX(Bt~{;8*% z5<+J<56fz;e1G^%f1w}%=Dqv9lv>kxuZGQh&-Y}i#9ip@gnc;J3X1fsN>b3;9t862 zTe0bIhl;<<$$Qo^aMlX~jIhfxLt~0+&6PE6bt4OZAS>nf9?{iTAc@M86`cvQVRl@ig)oOG^zF{BCcPK^7aU{y|MXxlrM5k9O`2Icf;d zmQbSAhna6((xC6tb#A;>b*5JxF;}I5sffWzVRCR54B;hpJ69I4P_VkBN8kS`ovW>| z4zybtoJ7tp9#KE6Ab~X^u1S2>uVhAe6D`Qxi)?8LnxXrqiM80mG{cucpxbp_=Uq&&@y$ zji2K+_P3ZOjn2{4Mk@+tUoK3ByYE_Tb8{;(f5aY3lu9N|4?nVC_#EmK#p00bJNGdj z@P3ML+Ymit4CDc+f;PLDP{{8zFUnyI&{}E>!7p&%6J%C&!shLm7latm%UmDy zMNcbrOh}1i#ee5LC1580gw7pdud>@XowE@`(O`K(S_^lIrN;7PkT>x0xUYKtbzV`` zfLg)!&2F^utuLc~OehD@``;+XEr&|4%hxX~jhgZ@{wByZZT0H=nO0&__|NOD?QRvf zCJGn1nof3RAZrKOp(o5{MwN7vzK1`1zG063L8`QCcC&^k!hYG^P31O+B6+@*$>2p< zoW~G58f;~bJ~W#0IOPI6h#v_0oWNbV znWSKnAEhZqB-Zms!S~yPbPbPlM(5bMHL;t%#2Lf&nz2{2?O7F2c9@%xJS0FHl@|}8y(1%QVkaxmFO%2EQ z$eMsPRJT!bR~&ZbQ$XNKN6t$1gLX46o(O%9!O&%J1FxPlHcDXuiat#IL{-J* zjoGk8!}Z|w5GHy!fa~o3VnF?mfYSlAJK7A#sEaaLHC(OjO3XJ7^EKceJ~15~ozFbU zB>)gsRQ|<`5G)wH&aJ%lW=u`out#TjH|o3g72ZTaA$U$Da$YMXr9UdQ;yGu)(qcSK zuSiH+lS8m>T2C?d4fx$}S<&eqp zGTw7D+TTidX)-(pt9}&YcYZg(D0J)YAoiptH*c68LpM`3n7gFQ6(8rgbN_gr`i;Tx zOgF7dwf>l}J{!QKEIM|j{f)(f@Dag!EpGwybHGH;b;(&BzdBV8Fortc1GBK>9kil^ z6IEW_n<{qO`}Kp>YzV*k>24oiz{Tn_w^O+mHFQ`Cmq0f6`U3*<=yTnSt?~$PX9x`? z-`r*re&9Gqit`+_Y-GRjJwM}u9X4@=tTLZZ1MdoDY)17!%DA9oxs7|Rg}zjP$bw;e z6atAN{gX&WM``1A0_&-w=_4uzA4hXZ{f)G?Lbqg5>osd6BOv@=b|xlBH>r0e&6ZI0GD zzS7$-Epm;Q`wbLP+pXB!qoP9_RvVj3FuH5vVC5d46yw_aD5E2|dau1bdpF-&$}jVu zuvxCH!eu48EexL?>-pxw&_a81Dwt*Uubk<7_lwj7^46=Nf5cE2VM={TINVW`X=`|F6b&-!XSWjcYJDShT4`xy9>&T4 zNU~uGw?FVfQRAmivtF{sOW)J`3AX9MKaa@=6R$CxyPKnF?YyMk*Y#|Vwh$~RG`Mg8 z8?rul{-2mW;CJe*IF2`cz*)Foq~If&e8sV&;wA(}tx+ z6CFbEQ8a8s>?+J;*Pcu@(EUh?D}%HqV5Im>vY~42)h++^mG+3Pz)BT6D?ch_0??P? zJo^>U?(#3KFf{F;13#s6&|h|X`L3N7YB~G1`}g`s4u_QDUfZN#M$aq7?I5jO9@DME zJ$tz6O}sBLEDNU8LqNY-sj^w{rGoMORVMFUE3r${y0g=>Ja{1e7;xndaQ5n4+^qAV zz9?@Q-tS9s>OfpL;4J}eFHwXR_BBto3^4YdgeRJMi$1FpW1fQj>s)~(nDqc31>Sh- z89VhQVe}8*{EOJXG_Jn!|IH|vT&$!x?(uNiTw~ld`(=)JpP0e)*3Dp3SjE+Be-^e1 z$y|Y`Hw1EaGMk*Y(?>zm*D~=nXyWI5N#me^&9N-HP-oGYlufY0Y z*Q<Snq9^{`Xt5~g^0BuMO(K-zg^yvgiuDJ22s;v#ABHNE5Fq(=wjZC%JR`#n^2 z6m=?)n3C5yxZqUJ?hHYo%~hnlj%jYf2q8A^svLg%BU|R*OZz`2^MoheK7&_=5OF3H z!Q54tvfR;(!;yHA_+_Q{ZpgIn+U=s%Wa$yVl^LsHuq{SrT?*Pc#lH4}7dcWlM7Zw` z0j!sXW;zxablQiT$*s#X2$P6y{V7^^U+eo&2DD(4GnZNZm31FTe^0J-$`x!b*TGwG=RK#c3s(Cuk?7fTtCh>IPp2Kan>Rzw z!GN%S8qWKMg!Z?l#>nf%lyM$QbES?ek{GBR0|6;8%Rf%i|Cky!qLzQ6z)_dOMi+;d zM!bGlLhR6I(sfR_*n`=ShhMUs^O_;Ka(V$CACBl!yyMGnZd){5&L5e-ER`kWlywcQ zsoVdo#CH62kI8(lE+DwC6CjsmeC%PO`CMh|@lCjx<=%rRf8AL95Swiy-Y7b&X(Y4C z&cw1brIG(U=tv-x9&yQWF!{fdi~lhCKbzq1gSq2{-Le3KKXE!{c9~k9+r;?)Zv#j2 zXj9r$II^bv-lRMmOwQJ;*QfU^S@lM@3Ab*wu+Ty!o6 z$0@*B!u8v4(WRtS00^|#h+f-NX^2-FT#DUzjOnJ0H=*h^vsyHk+)-1n7LOcJ&P^AX zL?2Bi9os8*%N3BS24*?&h^^@U03eWA+iGZ@agR78z*jTZ=(yZ4DBL{G#K3B%!HTcM!oVd);3}TvijF zy^*&Y70?==AWdCj%95*T^uED_v{AheO(<*DSy$4l&FHC*LM(3!>dq#6th=95fQF?H z?z=Rrm+g~NOu6Ya#BdYQ2DaG|NwoV*eoNQ`&HXsh5lX?MAy zxK8olCiw~-vxOn51?3g+-Qm=cog$_*Ze$B?z=Vs+q#Kbp% zt?ce8zW(i;s~)k_w>^m*npoURwqJyeTmf@no3upkzl@&h?I}Dh{0t|la3||^H{#;t zJ2#%bm!dLf_+_pS@MF4|`cs`Pm#Mk9Ux&cL!$L`X#0mJU;E*#vBc6}g!|n2(1Mb`0 z?Nh7;rPe^;qd<))xb0%f=nqT6FjKeOI`>=3>XAlK4*B;M+2XTtm&-AfDl8Y7VHXEF z&$NkH_pU4ZS00ON$K)L94|;2pGbKFMSA|yk_m0AQr$W)^?~}M?R@rF`n&=7X`@{Px zeNe=dOBvJPK)y{IPI9DaFNmo+1iGM@lG6d@yWc=sJoyQ10-XE-FhWrnq;7ymXMctj-ZL)SN8OfTq%> z@l9SZkJ=Pb2PKnB!{_p+s{HxE06PwIZ&=AOv* zK3_WUlReN%-?|cF+eZD;Cyv(CA8o3J?;DS6rxsl;M9N#eTjL}ugQBM_?db2iUPHpkSvu*ut1a#7fMvw$ z?Lq9#*(~WQ-zk)rIZRpX>%Nie3S}|uH!VuH^=EtG_}ml-GGbVEkVH1R%tWTOp4W$Y zs%kHbr33qk)jnPqxSHj#whGs=aN6RW;=mdmC0YgpnIh4M8|fnRm2vM?Cb4DwD->3d2RO{qW3GcV?01EwKj5cPZGp?kxV9y!biV~F zQO2L*6|A%RpsrZpX8vfVcsIzG@zjh%q8{n?>Jnqb z%YHE%uML+aj@tf^i_w`gUcwDW)}!>9ydIPA-Q$2^ea+3=#j~kmw&v|u$#_4jtU_Qt zqzuNoqYjtKnLmC}z{1r=R+3d>c+D^8tORpGZ;?PIDXY+13zvGO0a{TUeOhTU9qW!pdHr(W|?rU7z-}|JA zX*w}CR?Yrk2r!20;RoJqo6fLsK%4haUABeJkC#6i6S;;{J9(RrJ6HA`zb{0@9eCiH z<=N!g#Qp=94pb#4N9jCg6m*9&WqL6^<+GxQ^}IV+1SAEsF7&|su#}d7ea406&Y%6Rc82$Z#?wFF^9m~aL&7|*NBjN>H zaJx$*^-TiWn-auUYrCis2?#BD*jC=D=(^Xw?*vT+>@Kie4ZacmN&2pHk-hsM?iCY` z@~klK{Q*`H?-s_MFrHJ#EjFC+)t~ptjGgFI`bjX<3Uj5>1QAB|r)6#(Tif~ef71PL zU@8@;#!#S1Yn|7wkDbHZi(&LVpc!(q1kQuqdg{1X&5>BxO-Ih$6|Ot?f|=4-Ip;*o z(V9L9;(ut8WE`U20voE|349*{O7KU!H_A-b?hdL^7T9WEw2uLz*PSL_OxW!a%;^`t zE!;P`jbs<|CtCsx%ed&d;F(E!66pr}tLmGKTTDPVg&!-6f`yp2-Yc|`l+vQgbjEd? zP*D{a{H!m1OA#RST2;9Jm-I(@tO}dSI8H%b!DABKqm;A7oxQJJUtOJxd|wZa5|ERG z(1ylxyoXMKhKi^-y~SxE!Ha3W7hZnOA;_-wj&eE0P4tzOmsFds!OzzB3kc^S(`MWo zh=yI0^E0w)4xb=FiUVaMWt_R4E|ZF2Wf5fOX?h9s%@xdJih|YMvYmb1F_%k6AHF;X z`|hmi>)fE!s4N`$V-rOrYzqFh86J zhn*(vmv+{CQ&;pu>-fWiq$3K8%~BaF(AxbS7cUK`Lv%>fym^nvNj?nIyj75Vi-BA% zES5seO#wkX(s5~Eu`oGWBpLP;)JL&Z9A`k@+T&H=b}$)m682xc0N!9_8H;s$;m(Ip z(*$jUI~ScsG|5>XXrM61(08t~x@c)zyV>lmT^5~@4@EPCP8$$t`Grn0!q3WeblPrL zTrh0~GqFfl=0J3?rmxY40+Iap^U1|n#e;AaMFo)HxKD^pPrLv0uSHW<>`ec}pzr%o zXHb2klgYNo(oOTaWe89A_k+V?12&RkR13DRYRs+!CAv)1CbVA4f<81v78=IS53@;D zlT+%JZef?heEe=VMN#yvm%~|N{Y@J(L_P}HGw(|yBl8{(3)*H)LzQ{s0|~*l_(!cx zcu>R1^$Ow*%R`Tx|Fd>;aIwxt@wGJ(DX_etBw`t5kKFr!^8RSR;F_Sx$=DvC;2n^f8wqzW&<(>Tb@lp# z_a!!*Z-2Jmse-+%UzGRh_QSVg>^GHTUewKf;zaZ9Uev9lNM+guL_vK3*AM6fuVzqh zQJI#WjcTqLj-&tCEpm%RBc_4X_`{XVsv=0n{w;Ju;wN^S2Kg4_=KFDA?xzR;*J-UtuoLvmk%OG8(qe$s(V5l`HEd_(LKUrtFLl zL1IkE^D09`Fp1q9{s;FnQNCOCYaBfFE^&bYTA3inN>nUTduv`nKBeX_i9ywWWTbJj zIWex2B5ihk;f@ks0U8)d2N{q%ddO`Q5D;`=2B@+gQC%-#s_mU;<2|EJ<_`(Ms>vH2 zu*(rH-t&N5JD#q8?4V_-Cxj5hE+6n{nLr+^+Iz=W;gsO8AA=;)flHLWwN(AVj z?Sxbq^_Myo`TBs#pYX3y)eIp@PSMhd2-2%(lsSlL16>50=QYsI4|;3}nb_wlZX=kY z!Ig2}?+0zX{$v+JpC?I$;Vzj6U(BO~YDzRKZk3MdaOs5Ms7J)Is~1G@Sq7#4nCRJW z7|19X(hWEASqEec2>O&JN2km*BJ}+?^>BpQ7m@wX=k6nLptHv-D3IdXZ)xHfo2f-} zrEKRvHM5Q&6^PxEPO5)WpCUo#wxT!KO%08~1iTPuv z6)58}ym`#4AgopF@l9-qgHnlxUnT0xO8)zUiV2NvjL`q4#|F^g-SQv|wh%;&K*Et=#Q3wo1@Ss{DF0FkN%>VCU|9*Iee;b(MkOCj3?EXn`*A;ZYQXXr49{~t8`cY6LIDWYOKlJO9j%D*Z2-vd8!KO-<~ zpv?bG5ELe_K0tOBOJ%j}|4k4G8q;5#emFnG504}kPx_;H#Es5TI^?iZ#x;dBkeUl- z_ZK}LYIt29QrqpJPlli)Ji+ax-F3BBUt3s{sfgZZyEW91?_(}39~@Y8R+#Z&xyAqU z%V7Nd)IXc8i4B8oMc{>uhwGhx=P{Ba;YPwG=Tlh;?r<|WYi`sWu0#w>z-mme$0a1J zKbz!hDmJ?@2;v3!)e#IU|^d^xCnf;`V&oXwUc=jg8n2 zIL-fIhx}zTK!n_C$rR+|&btFvNa`>kP_n*eI#s);wG(}H8aq6P`?@Zj%n<9ug zs~o+qu69j}&e41K^;o7`g zy|$Cil$DQn3Fzjq2p%Dul~Gn19Iq9MObi`E*2jGGTa1NQv-9Cr~S_B8^Uus?bN%Ylm8|^#eNA=p2z(&6joplTnwpz;8Ccv zWCbBiWQ+3%yQ?pdWE`)Z+ZZ~M+3${`vw(E-Og87EE2DPbvQ0FTswJw6L`mU2<3z9z zht>UAIj+i^CMa6+BaRge$c?{H?DczmIyn^jv@&2gJoyawX1PKJCNk}U+m;KtLXFT$ z^a2|L%J0-&(Wl@0`H3~TkM`+GDhOX2bIJvbAT&H`aEEzGRadece2)_4Yx&FA7V{Cg zzoKS#rSdi0ZLdQ`?V!VdJV}9pAjlC{Us@}bk-54RUyx}P3I)jw>E~bifz1S859TV0 zy%`Ek-)?fYWTI5EWFTBdBu$itD;O>|CRiEmdexdwEar@|2ho3CsH21@S0kvyaxTVg zPc!XMzSosR7tLJ3!Zm^kEh9B$OD44Av*FcGcrdb7W>qWON1LzcL0$;l_T`mQb_-rd z_d?iH@@)?txpK+S;gKbBT|Z1wsZjaFI>j%k%u>Go{=C)0YNeHUL4Q8FHZ@nc{^S{G zR8y%M&63?@p&b;w|6bY-Js87bVE5$PH69>1kjXcv2)+`vYM(mHf za>GQyp9cKvgu`ThBc_`Kz+S+j8XZV=wrEt*FUQrkLoB9V4N*Rt!vXCNOCMN`LAz}PF73Dvap`hIBFZA};JR8PIw_?o30G||ma`+zRtp(Ln6%Z7q= z@SQlP@Q>e@Fti$2<^Q;UDbW7;CLAk@hkaRIf8w|)JN0C{s)PjS_BKip3sdEwF_3m1 zih_TvlgL0-m4f#DP2)DX%B0+4o?OvrMm3oUoDB|9*@_({4|zUmRhf%eV-u1~w4OZ)d-wptA<4{SnH!^Js=tFDmy#Njy>6%NG{=*~ z(AAH=|IZT>go3OujHzknkJm*7y8J1i9iACxOE!HCc$j|E9Mg{2XM}gDT4AV}huw)P zAJbY;rHH1|C`(9PKVKzfCSF)!V`GQiWCn_d;dr}vZ(u{=|1NRs{Om0C0j;+}W+^u9 z;l^4sh8dE0s?Df%QO_UH<)tT09q3PE61kH5UOf^VB zvtf;WR$HFFR-{#BYP)5?Xwf{6+@#BF4NCjNi$za<0?ktC1b&z8HTxUYD= z)y}tXnAfs@5Z{I)RX^BR{Jco+DTVi4N^1INudbVV41*;DDCR+#aIf~eP5(RkI^@{J zOol^9+(Y@&F#(~3XiNuUmEjsZkU9K};%$fW0j(1( zEayglXY28>-Z*+g3V>Z}>^`O)U$vWPA^FS@F@4) z_ei9sB4;osLhdm}EUrO!=Od7&(~VlP{_t%leP$}iTum*Px1VCJJh$AAsB(glj#)e6 zZcAE0OpuqZ2|k@^nfeaj+iDz=^floJvHIMH)Dqu9Qq$+&g1W*LSL5QF7EDacZd$Z+ zY9`zf>b|pW21u?KXPZ3&_XtYTa67bnbXf?TG;ln+Z0Li)p3f*66vK{JXe;l1R8G}I za4U|jx~0DQ7$vL?QT!*(Td#tsQQ7%}UVdL?sCu;H;r1LRyTOsou~!9q%aok{J1U{R#a>4(-hH-aIBX=kehFh3aYq&IkP z^-6*$dwnkMC4VKO@kB=-M90lMzR&6>2<{>_XS)l@cSG{RwQ#qnbv z>KTJ{>s+>@&#}e{5bBM+7@={!*~X>w2dv)TX1Zp9wxFU8frCS^`rN zRg0tsq1|jAmv8KwIjAZ<$4@TSKiL!FE^AXALb6N*4`($isQ$yL)!~QD=u7jWJxaj6 zdmUARyRgCNGuiVcswo=TBt zh3Bx<$E+s7%90S(RAG1c#Odmh_$}2%Iql#llb&3yIpYzqRbZQ~spXz(CIpOse)kcC zY3v;7YWS`+@@J37$%5RxNZ3Stg$1osq(KsHy7kvWHZB>D}xO zAmBh-1v3y8+X>g_s+)!D@tB_);A}bn zwpvrd@1=bGjpt%E87<7Yqw1xPNu#~2s744+g&ilt$IhzkYH$;FRZMTlBB&t)(ftN1 zsi&HU$Ox&?$uVbbDME|NP}u3*9Q|ciHR*6JqB4(@Nk+R+IwgCM6LRg$KTo_-#b+>1 z-F*W*F+hdU(tRT*rMdKI6OX~T#tT9;(LtV}9o?;3GtTTEgXFG@%aD#r3?C>Vjn^s9kQT7@WC` zNLGr+MSHZD=!%lUTDo;^AS`Xdq#~t@el-|KBzlu&m%55b3mHi*SFB-5c6xihaHHsh zvyh%PP#C>1vy2&Z%zrZ*Pes8}Ar!ytklCwn7uXanL5&IDYpxHRctRQ=v9bhm*WO;}SzMMphu;nfx7cpQnygCLN$^0-5mswg>__XAD-Y|CEq2)X$ zS;0z+aproKotvx}X8ip#rtA*lAC`P{cY#2*Mmt@anr{${4JqffrQhOOUBQl;U$nV* zGfYiq@tM^dY+{NKX?0cKOeM(j^YV8t%5=~s+~eNK1uP$`;WnVpqn8i8$IL=>W{VSl z_+{m8*II0FxaaAWR`i9yng9uNq|HW^>e=b8Urvw5=VrrGlfpknEV(ug{1>fc8R~H8 zSWgx#B3v);efU(v7w9&bPg99q)@hqHcjsmT3qKxIyXs*u7unUBHyYu1JG=PIIU0qk z;Vze&7eilBZGRui?U$ZvP8(B}7u@2XnMN!bj5TJN?Yb#idcoD*O~f~&oi7tX`UMu_ zym}?+mw%hC=p080Utl7M6v0T+LoA(_9KJo+%p)TX_7?a*EUxYz!Yux5S!AoN?*DCzMjpYaVBy0*mgYvvjt?W%YC60wB)zIno4bLu%p1J0L};h1YV=$Z3uWnh$vc{e8k z5JE;U`vuweGg7K1gt;7_@X0wg3oLL%yx%aQ5OQ#Tfw>Vb*)FMeE49)?_ zF6)D+{HSA1Wnkd>_5}kivzh7l+Zn9+n;JGf58uah<)&sxC*}$?tg(o&XYwDoMhBq$ zNJ*I;dpR?sZc^WiQsQtP%i)sreE$jQ*sdjv{)~7F&w;{;+IwjH1)bDnM203NZRo|; z7(_pDOo8E(h7Hc_E0~?!;KoCm5{h+Ft)i5&d<&W}%H?9ywQq?=ZC2riQt(w}!;;#= zE$OhrB2SM_UBgg{MmG1SdE@D?5;i?b{3h^31|8;P+bp;++dzZyV{D&a`Lvmk?HwY@ zb#5%mHLE^ZTpSWyCev0NKdP1`r{}2cNlue&goQ$SV$EIpRb${j7+Miu8Lu19dv0YL zyY4B+>D%Z0=iu{hy z*kUJ00JF}(DY24b_>AP~5Byjhsex`0b{PNmk3kj+grRg#U$6U59|62@N|2E37 z>Emde311;fyv`xTy=ao=Qhs1^DE2<|yro`44hVU0*RW8HhrLuggnTH9kZ-K&= zW$E)paHe1#5MjDt=vPoox3mf)?K)`Jf{O6pFrr|p{1O7w z(`PUkzu_a_Ld&WuhVSOC2j^Z5*?^;=KtGJ68r+jznOWkW4LTn6i|)yD#gH&8@`$P1 zj$$v_l@Pd#r}XzR6JBH6c|TVQH+L#Exu|55pl_N-52@n3Z;Ze047NqcuR!R*kDF7v zO0{Xn!>wF5oXr(@l8XHgEcD+vB&`Z?%iJYu)$Nk$PZ%~;ogbjx55y~-_)nS|Ww_}!3-Hrs2K`0J zszuI1EZYu*Pdvhon&%N8lTR7&D$R2iUQ%2AAhgT%Zz@>}vdB(%wb());g#YR-OD?k zaW$?-DOaSH0)0iXnpgi5{s7{0kA1UYe?!_p{6?NbuVS{R{VJw7)y18QtYahF;9b~H zOOM{x+9tJ&rAr-HHbVs~)v>r7I!YLK{qAyTX_~@ zU3Q*RW=FIzxe7^CF&4jLuJ|zc2n=~hP#&>B-⁣ufA~3NA5OmqX@k|?6I*(5~WdN z7-FGq&q_y=1~&Upt*03D>Y{j9qi4|*z)RX`o>%e=S6ExBht?Rd(v(l*CS8Ni)239^ z#5pl!q}^m5H1v`~yNb2_wz{ijo5h!xUlchp5$Xy=D>HYdO7gJA^5kSF@%A4O#7R(j zySH`oY-wfqahG_5$M{VuC}aJjTl92QWpnGZy~uudo#^^m9M zpdHl2aA6Sk5z^F&WJqNcl$0mBSlq**sBQ%EFy{qM;xL7nEtM($4PcdmfR@+|UHSMJ zq7XE69TRHg@f)U3Cbne-g|sC&CwtDE4Z8C;Zfn$lMS1>KgV7L`Dy5<2B}Zo%LX0MAqr z5k!S{H5!IssSe0CZb19uFb8&Pp0*MOq}+#A$3zeXXjxGr&4B z*AY?tpAW=%sT%5ORSLIv!WPWzEf=gF zH=9Oz1=GasH7Xg01e?t6z~IE{+`VVFqx&HoZ!ERc9%Cs#lEqw@KkqW!0{TR=5eipz zgTQ6Ybmb0dfu^E=^Xj$}Ge)xs8VHysBblX-VnUvgQHX1Vs<-@V`fQU*YBhNuT)3a-m&|H|7)zo<&k`zjkwz0;zyo?!?E;{= zb7T$WZ+%TnKdNjR4vWDH6k`YBJK@|nVSjh(&zi1w;agn}7u{7T+!9N~#9e;Snen5< z+mIUs`wlm|`2Vlo)Qv_E6`gjPKbpIAsA>Y0w_Wq3%2h1w&4_zLt7Y_@?NGI1W}vyR zEMYezr=<~4Gh?7uQ>Ko|_$Hv_zxhxBtmH4&u<27e&G53O=6AAJt6O8(&0*k^KatE_ zsXR{oA~E-yxjfacJYRoYqaDR!v1%waK|)SVITTa^3xLapkT5qCuRKP|)-*pPbhkJF zER9++j*-Hg#dbL4I1bbeMw0Erp;c4pe2sP^X?|xLhnE3a_F8dW2(aPUN5v0bEH|9| zxF|a(Y`UI8?{4kRdEBaj*qcSmX*UNe)Qk%^$YR>t0CC?jmA$C&+9yjcfDBp<@(L3g zsy7H55Jt$o7W9(hc&QI_1{>`+(X?)Fl^WxS?S|4CUpKY$(8#DwrbWA&^@V_wf0@m! zgeEoFW>13!{Qdq;^_baT8w-J~B^g+}|ym*6_64nx{fe!NEj{xlrjPExeQrOjowdHcfLg}!v)&<7k3M}Uy z?V4rIB!_!3G)l!OL_cejS(NU+)8SQkOV2L|mQ}erudQ=A5Fs6ZX9tj%SCM$1J@XVP zA^{0zMlmGCwv8a!s~9YJ>@K|GEQMdtSk2ur9sPu0lf-x@v|}2jZbp^kZDkn)Z_5{| z^4EpJpPsdY(>4Mrs+OQ4duH6l>^D|AZJWQo(e8a$MUCN~4NTh)rS99+irIZWMZZc7 z7q#d>-BhB5vGAUVMkxD9=pA1#nXTF(lW6^fZwV%;0;y>JwW)DSui(8-MMXc2>HQ6J3__29BL8}Xx_Tg`41-g zbr~EbNELw7W={@%%U!GdHy2)^u+wC+ZmQxc#oSDBFRjx5m0kiUl+ zL*HN3A8!##c};PS>|8_U)EbYyAFn|tbgXxEaTV~$Le@0%i#iCTrjhiFJE?4;U)^T* z7JVDreakGtdYbD-+~8NEa%#-I8-`O{&JUHZ)qg;%cjAti)m+fh;5e5-<8DS2d9oIE z1;zHcRbHl2)(yHtT`%|~iYnORPB=MfoMq_t(AIi0aASL<*I@z`F6RR>gzR9lHypXZ|C`0Sv6SZd&f|o& zA-o$_zGg2gGxl*j?m?z}=H7LT$lL?<4r?4xl>UF$6McGq@F7~>r=2-?!abn^_lZO< z_*Co9+vassyd&zl8`jAck#GHSCoJb>^vEw)22EvXf;E&W6LfAv#x6xYswYod2t-YD z%I(lN)>?^W$t{WXRg1dl+nk^Z|A0-T(y74|X=B~}u|_Nl%O== zl;a3e$uw9aH%(kNuUk+K846F)UfVD*Gc!z485w5smmH4YS9q_u3V-G{_5;)=5MUg7 z@=sE0=Z@?`TCJ1D@GVzjGw~SEt~;5HlCj`4@m)&wlpuiiNM~?^WLTp(Sz0m=>>XV9Y7jIbdr3%7|Lh@;_ne zfPocrOS}rq@>ClAn1wPT7nMDJo-25iG&6@?gi?KHupc+wH!OUb?#hmsPMTIykHi_M ziJZz-ZQU)E3?IQRT2SsyRYXE^$LZ2ny{`hWZv{Vj-o8(F>Gf512g?yLa9A+SD&(DU z18GUjxa!Xb@*A{_cY42GncO@ps9!DX@Uk;qzYP(W4ggcj1&7Ib(O>YZ9gq(zc(2co zYogrWG2g}COYVotR5jtfUNkK!q+~`)WXMa0lPBlL`WR*nk(3reA+o$K1}3qBE6EmQ ze<+8mc%#LYjtUlNwU6gVrzRwao@?I7IY`Q*?Qarjrs4Xy)JEb;HwR!?Y=)xKaW@L0 z>JQxE3~g;~w1tf4gZ~qJ@8iS2{i(lJomXEywkw*vxPpcbjmSbVp} z)iVTBw?Rmrzy|Y-wd^m2@ zavryNOz0qc3Zq@9c}))9RU3M4|2Ebc|Hac`>4B`zUj<)DI78qW>+eX&$U-i$O!52*U%^4{xR0$Y$pjRN6de$$=yS_85?4RcayC)|2B`mYRXpGl>?E& zaDt^>&y3qEk`?xRDArMezGq9TTXU&Emd`515Llvx9)i1DRENFrEx7KJXyUeYP~*}Q90ofnHF1FAR}rl6@&uhX=2 zl_VT&D_AbWNMjq~9)ojU7~sSb0|QH9?vmOVz}igjmK=vm$xX^NFJ$B=6s10{3tmTs z`>2p+s6An_;E%XYIfk)~j1w9vx^mO5%4W6$lir2kGF>A&1Rx#FBh@Qq*Bb!|%}wto=)1iToQ`h4O*=x(VztynxW9|uJezIXTqb_Fd~^@} z#t6zITMv)@Jy?MKEMN>T1|abV!bx>!t;he$s44-rOiQMQ^WN@zK8w(1qd01^q3_R9!`38{2X3Fp%8^18R>Oz5Inxx*LDD)K~otW$;gr9k19>A!Lu zA8llW($bxPYP@h&DP^^*oKYdR^FhWtGK0bmxVC4>j;pTiR)`MWIAZ!_$o207?>~3EV5@xwZH=SG$S8XV4S+Cb==vbPYejI1S}gEKGIukBb$1{FLXdsCM&)G?ZzD zh8{5Cv!+Pl$C(UYZUAi5g6udocz>kg*%Z;*`@a?wFI6ZXu?yCMX(fV?1Xxr#* zR|9!-gr8b4?K3qZfHy=>U%+*T%c~X5G^ppkZN&LCNw=u8-Bi`2+o<}$ zvU|uLu5y;2cfe=|(x3_na2f}&>1_7l_6xV`vPC-iOH$5%*>_j zwg)o7^OqchPln*Ga1HUcK_!ws)1C3OthxdmYaXL6wq_Og4S}ZD=g2hA*e3(ZT`a1` zjO6YwNTC~yWZTX$yQKsY93BrPi?qJJW&1(^p3542RXueErG9RG_G=RvnuPXGCvjM( z%*z_jsv(f5=YDX6Wg&M$+??dQ}gr`VgwK$c5h=8E$_~g z^y@?=uvLulYCtWd8;tN;t_(i%b0#xOtQWQMoc2Mp5X0Ta3+B((fdK6zDCwQ8sZhOB z(3BTSDZazAK@3W_ZP1o<{G!jhezLg?tD+A;qXBM%vt+wGFYMBNaP7zp^f~C@Sf>+d zWowAco98 zirsHj0UjP7z0&7Iau@VMz{IvZkS;TdpD2_P$hsrLz0JM+sujvN6oT5)Ut^KPYwk;4 z@v17?D~&C;nvsk`GwNaX_ag-KrYD=*aq`t-Wz}20zO9DW@y#)Y#Y9IF@L;A6IL}l= zVE*@*5yT6cxmAhk{<;bL zNBqzq*L|9m)wfco9=6p=29!cXF=$?58wDz=+l?TKg8p7(`OzYFjiX4nO(O=W+EqoQ zABWqvB!T{BR2-#=0=V@bQ;$WG=O7I<<9^;NHXAl+NP*z*I`sK=?&Ka{qMhmZvo74?b{n*wOGpJ8_whf;~G>+q|f+VleL+8@H&r<-+n&K%UShL73=L0^9>Av(l zy55z(aAg-v?|gMwb`j1RlAdAuQ}DYN;|BJOTK z8#ah5BZ>*vek~kUPe<0_vZs<(5U#)`WXHeTLkyX&1H4x>G0#-};#}I4;L*Kyi#D%e z$BE;P8=Yb9-2PapsVUXAH2@dXYLVv2`BR#*$IseQM?JK?#HqYAtX!WB+q_1%5LN6h zlb?7+)k;e2fnU`2qK``g?~pdB-(V#$Ss_!58jjallYq^yT#K93h?HqMsx!MeQQAx#A!d>?`9Iw0;EQ1B}iMs$90U+?3K6%V>OSnmI zw^0VRuTtvJPxuYs#mjaWtS zhNnps<|wAZW2Z?rD(g+`5Sd=_{32tc8D=zw*Ed*nHnY$Qjb*0dk8g{kXj0<~ z&YDwJ2Hkn7T%NdAS1DX(tEUthz@tGG_VXeuNNYpUq--;s=z`3y_5s4ldd0s~qY91{ zi#Jj)PBNNL69NX==I-P8D&jU`*Qe=4Mfiq{HewdAne$aaynl>3n0^Rqka&V#ANLJ9 zs6f7s14ID;N;aKOrEr#ukLu0BllM7x`ru>hswby%QqUOa=&DDX^l6yW3(JaMqoop$ zUEKEB`C=j>Dd9sq*t60K8bLE8RbUGgLzGVG~3GKlVcQv0vqoZy? zHwqK*Jw1x!YqAk(P~w2E=)|pP9qCy*Sd-70gk3z%_zicVMKcA?vNh>s!24u=7;dkV z=Olik!q(#4Fz=O}u>e<^lSfaJk1dJXxl z=g3s?C@n9NQVTcDrtW8ZUBgUMtELe`Y7$VJp5&`*g-4&Z8G(;snL|+3sP$dNrro@4 z_@AaRAWdtL^(bGgwHt-m+#OCJ^1y8&PTs1W59y zD>UNWURZJI4=0J_WnC?%NQLrfb)91iO7Bm!{OE*ZniR8_x_CF$R@NY;H(m?PkQ*#G zt4cD}7IlvHSq=dW=}!0#Qz*_HT)i>p1m599Td^l2+CJXT`gCkeNw*K zhSfHvOfaXzXiD=4aJoA@sUf85cC4cVZ8@mMfc>sNjPI=yb>~hI5wi@Zl|BD0TE=B6 zaLgYs^(RXu^by4a|BHFos^^)dId{CxB2@Wr-;7S!9B`o?7Ue5>A^8Pmf`fI<95dvD zG#Emms{sGFPFQPFc=FoPYnKxW2%hgAzepRbC{`2YL(N{#fEWm1+0)<%;oAn@? zP&@5IepZ$^a(arPNxfU?oOF>Qda^1ye6PC2ohhra-w`#_$|`eCo^a-{n{$d8P(3rx zQlk?PXeY-JhyW}+ed6ntqazQUtO8U6)E<-EinXVpr^x}+fPp{RgV^>-g$XmfC_lU; zUge{(!Sd{Kq=Tu-T_o0x+1zD-_bomWTReKgPFmyIf$ z@N2Nd9M`KXW-yQ6*OM!$dbN-UeD0(P9Cj~KljoRW!I_{9T{RvhZuG5^wNZj!Q{DP1 zQeSol(9=`hY?<{m{8m4p+(T$RLX#O%wY!1Mun>G{(4-loD9cnSTfHetmEs69cUW*1 zr4?>x_|)5XHJZwqcGr)rRw9N9yeK`3$r;kZcNkL zTs`c~Q`O+vaa|QIk()Jjsit^l%6E;v!=#x<(-5~Gbjn!X{78J1qeVIB5-P`7I{?uL zK&zH7)9<8|Fl*4T0snmSc4ZS?)j{D&2IEswTsjrW60^2MNZwHlT z)n3MySWS6F^HHMaf;9bt=c$5}_QU%ctZzl(7kfSQ9uyTCNGUuxHjRyBMFg|Q;2)xF zH)5u#x2#IcM`v=c(Z0N89$yf`39S-ih?s-Wq^5e0G#+)x0NcZUBB z6a&3Q72IVuaeX5KS!GC$-q*ZZbQAjZwe zcSn_+==D=HNB)mAj=;wS2|!q?JVn|bqq~{eS-Qc8PL$veY7WL=#a3yuQ3#kJw|Y7v z-6`Z8`xT8r<#sJ@04;ts`p|U9-ioeAqwJE$ycpTNUjhxxZD&1jwN}bN#YrE~$lOT? zNuca8Cs-Wv9TX1$ZUkckI6cawib^6{)B3QF4dwh^owcc%*G}Hd9?E?jZ^jQkq0!V5 z#-XBfjyZew+}qn%#5=tl*gDXeZ}v@5oiBy;RPEN#wfUZR`~GYFIK&fP0F~|`*hdRD zG^7}ZkkNSqrrbHu#zs*dp3L|MBO551J46_HhsUHMHTv(N1?m%y zjubJ;nImx!3_YP6|=R&VH%hZsXlc48}R>{ipD15T& zp>OzZ5ZFgYx<5pD5v%$ppVv&LU+zB;xh41P;-a-J@48ZP%$*jhw)&yI4N@ZS0dl86 zZ^AQfoxc27%aU?rx=XX3@|Bm|aBz384oXcl0JCRBuviZI?>uQ=Dve@1XwqDF=rXUF zx9iKbdB=vy7VT|5&C}Tyf_v#{vw==E-s<(MveH8JM27;B9SKOPAy%z?LJIL~BKaY_ zs{}S)h4d7E?~LCYiGx#oxC-}CK1hdXq3+^Tl>WvI7d4bC_L~K(?gH=uk^gF`qiW~9 zp5$ow;*&PhNG{;>)omuk9PTMGt3#t^6}*d4hQ(tCOz!z*>8vm`a^oz9*)vA{^1IkT z+MbbbL|=O_Xy<1)!Vce6-z!*>S2lDimA+d3u{oFUE1=P|ToU_B- zq)!(*JZc2SRO_%cDgIHHWh0a3)vev>))jJbJH4sMR@<^CNjiiRm* zok3F|O8GQ%D{SbBj?X*|StG;zY3wpN`|_T#;K`5Mv()>mjHjFC_ia9hHgJ-XL3TyT zd57-|=+I4$JCm;_H*{e~dD8khica&?NLggnD=%1=bh8v3+!`j3fQ8J4AS#YqSyTI> z(uD}0^XhWxJio`2fnZU!`Rf$?ve!R-Mzn#vDJn$t=aqokpyJK*`|Qf@=*?_TZw!nr za2QpfzTzPJ`(0Nia#)=Or~5Iz?UnP?uErDbwoNA^Xv@+gHG&zd0}~y@hj5I?cSdpV1O`M^%rH3F+29S&ReY$B5r{VqBYVZ z9!F?ghSqMwn)S_1kF!0U;PEhCBg*=%RLcYac%tT7;JPvpdA{ro1mrl-oV(1%?g{2q z*5I3{;-N>^cw*Z?ot{H34KRQ7E$BXcw7=_DOB>IYL;7|#tu{VOQ^|G*(99ZC?Wob6 zk56$?A2`z6^67~WsyCu5S7YQiiUN+zUC#PV)Qgs`5v};C64O~GT~>Fw&m&q+(4<0S zc&{>?GYV^s$-ux0;CD|zHcYweq0dV1>&lQFD#m8YRgwURcbc*cfAai zJ(y%O&vOR~7Z~}U!by};Zax9}$?lG{elCJOD)*xPHPOO-jt%E67bV{1F@M!COI&Qf zd$!%??g+!ARvtT2u(rwk|B{CRyKHx@-{} zCXLURcXk>~={3W3Rk*o{=1TphVJSwE!QJHA@7>|%oc|5g)mrtmfGkwo3!uC6c|#EE z-;S@u`fxDenfKu5yVNwcu%@|Hqfexll{6j6g2P?9$xTpYh1{rrFnOGxLf)Ppz`AsZ zvAPCGl!iEKZx&t#rK$ubWMjF(!f1VgqO4JnbaWW%h0TzNYSE zQQ+!V^@6kMKz4n8@@53;LgTmGOkU>F7f8e?BoOrd1Dv8~{ko2<&@>UTVq*g&haD~m z1@LGAZ)jU-pn5DH!d$v4Dmu>R%U_3%u7cL;COV{0t!leS+q=G)Tr+Rr_If_vaqFS0 zFDp->bUOYOMX~sdhcvz6*D`@});g>?V05ZnENC^M7IVr|I_yrQx^Xsspr<~5m2r`) z9Mwj$yGWYHxmf9Hzl2qkNjr%8LBpIw+A;C0D1vX!V~-q~oCu3;LE-mi^`H zzcb@9o@*sw`uD%h|8C6^n`t3}Wx?IEVzG-sO^?8Ro1wj55dGFr$As4@&GkFWM)$GS zjAPH*Qp>vfwV1XO>bl(hW=wb_^Bl71p=O)!I z`c0Sh8AouRcV6c6eufn70Bai%>}Edza1FXvsY*50L&As$mCm}^fws&13H2F}80Gn# z4aWAOW+^a&4e}0<_#0Jy`GW5#Cip|~Dp2=zz7t?hf)WB-)Fd;2E_({Vy5O~(0+l$H-31vevtUq1Bv|2WxG32fpFjMoBI$1y)2~--50`=OjK{T zuouaE6meYdw5>W`wz%Mi?sJ#iTVhJG6!*4E_xB+4a7($Z#!E!RDs}Tp_>p+vG~wtU z6W=Y}_pQe#mvk;${FCiC56?ilPgn0-IsI*8L(hNSMUoU6nf9MeD#Wxe{|syapvwp* zz%4GO4q(a?uoHaJ%XgRkhaG9x7qQ*F`R0Dw&FXL=wvsaX54mpCXg8Vo@w?nQ+j}`fEfkmW{X38-9s}&NB%t zcwj3&WO5}-o7pJ}Scx)2v;1Vr!MCH-hh}S05AZUBvCF9}XVUPlH+#zh0$h^7M*zb@ zePFy+ULfzs2+;TYttq`V37&Lr>=t$s{U2v!qawt;Y3Xn#-vPin0Yo^$XNA%@?k?|C zY*0uMrsw)A>CR&f!yh9V0-sC>;=|Qyv0(DkMyRO^hX7{~R>+)3>ZKlS_|B ztwShVZhj5a;4ggNDmzq-kyQ_z1%DN~^E^f^i|*bgp;v#lfA1p_xNbEd z$MdpE_f5uHCa2(GEx1+hVGgRsG5TN__ zGH*jmx^Mc59sG^y7DN{kb#&E1O|%*mElXmrJ|G%NU}HtqgE03_bsJU~Zv8=ENs;pj zjWO!lwyk07)!IZM@rjlbRJxVp;XP)3j|beR__8+o0I2giTAa zhyjDd@|~h~hnqn*KV5=)c!~oD!kn~`m3#fV4_`pu;Q@@7>$mB&wD5C{(RW{yW+KO_ zy^Uy}Z7KfeWZs;0Vrqn*2BUAvEn>lrWBs`BX56dssRxW?VFX`WX^gneIjUd_`p@0T z?oJZnP@ToL^^pMK?~epA@KN#oX`(@3ioHT}bR-xwiZ%i5>@n4;ETUcQD(;r0{7z)Y zhK53Z%ZEe(RZb0vbOuWXNXV19etvaqn8oza=Nv>+%jNdDohu7+4GUh>y&`kZNvu=z z8jlbJrF;JJNwyy~kZiRgRaY`brw88bmT!L%&|6|$quPmi zJ%xjEy)Fqge}vOD>E<#^A9g7;svO-Ef2PWOq!1&XC9vhjViuR)SV0OC$vQg!@b(c)J+`O z^g;%xggVyu+V4cPhXr30c9olg69T0vtjJ*D>uIWf9{{ZWNZ{1v5ldXFjLS6Auw7w%k7ueWj0OJt?HHc~z%CJR9_+`XhEVj(d zD(;6AeM=dCm-N3ptK7Wf5SxEp3dahqCkrwAzRYtVJrI@jlEL12a^ z6YPV+$Y6(*rym3RTefup%A+LyC|wwN!I=D_BV&mCpz_n&|ME>q zM#{#p2n|R9M9BZ~0-V~2Vz4(BxA+Gf{7+YR{Feb$1?utts{zb2;Az-8nf3puC;L|m z3IUcZsrq`P|F>rMCw&=Me!k6T{$Jd~|G`G2C*fmQM1f}fM*+v*FGA1>c-mkRSp^{U z{JZ1+_suT~fKOMUYO>gW?jr#E0Pr-HPA;>*pzHtCO_Bi5DdvG*$^TokgX#Xbf*pui z{Ko+S^wj^y82~tHkxoYapDkbh?kNSp;7Zld$NlG_1;F$Y0H31&pQcjC4YSBUz1o*% zjmNzQ5d6LHUkq-&6XX=b#AttaQ}a0_@pHe^iUPqGz&;Btu1l|z41d_EGmW1T@$Yfd zJOo`|R!t2AgZ%-s=*Z@(`)!uaff5Js`TT1@rx_EyLH^`{$-l6t0Hzym}I-XZ>k`}hC+`(~RE`Pz-^I+TaFFu&g=?hs&5TICZTf86AVrvk1XmDOExX+6V_ zGz;*6kaltlzZlj9FlFlVJ_N~7B+w+VW`#|@ZPa_xac~y2W)yww#o6nNyK&RjH{yvL zQ4|QxyzQ7_q%dQplu$4%!rMwAY6_$+ZTKWA2Y^RAIl|d9DhLEIZP~AhiS+^GlHH~~ zQVsbOlem93KRDzIw}C-#KR=**ZiuEop&bqabN?r0#jUgXiISB%b6ZLRJ`XYhb!)Za z({l+RCjajx^94LBHHbJ&C^!v`m}I<<#~EI+eL^)gdY{gwhFI_B(KrrDn`wbJ;E#3D z_gebOY6RsnB@d{Wq_}UZ6a%wGr#s(E%*O2vo)|4;m8t#L%W=9Y`1R=M=)f^^KW0Z0 zify1&&E1R{{_l za~&41%Wkem!q(xyc)*(Wv^7Y!=xsN7H+r(t49~AefM*CSXNtGtDn1En+9Oz`qn)eQ zQe~T*n)sU+A>f@-T+{vUe){9qWyr$C5BS`P0DpARUk$+qnD8Ewl(14mCQK2`az_@q zQiTc_4LrSm(B5}l*@24vx4TOcjt}uf@aeUsKp3e=*PeZ6u#)^?2_3L@o_Qkl6m@2J zFXo*b6?CpSP>gwZ=?i1DTsx8!-UQG^WE-T!0Kewk5v@;O2{+2BMX^>SW&oxtrQnYB zQ&7%1URRX2+h~3`_F7URp-m}n*(sqD-ejOUb+VbzYgRUn29 z*kJ$1!sI|{(&~P?ZY#*yRR+idPhV(zOo?4{3B6gU+JEm@2*k3sN1Cc6Sj;jkf*!Wz zC~Hj*kR|n~TkW3d^;bQM1DMGgdD=7N;1~a9K8P9#&9>kKINOA8e_IP=ZbH*fkVbA; z9hH-lt30XG|EV+)#+E=Wk+xb0<36H}iFK{}wyUAH#7eGCSyjv@)Uu%q?06SL(ymR! zZTh4lyS+%$n=a%y_)1vx@KbbLxf1MRwjrC%w5t2l<_bnPxAT(iw_}S$4=9$cY>-8^ z5%Y3*T@Qw+N_|k;jOEU7t_9VyX{%j0lP$HyMT(g6x#99pds1e;BU@xiR?DH4pRLCH zepXNn%`Vh5X3TMxCh{n%k)ZUN_U@mY&7bZ=8+IZxtGiEIvffvpURNDkZ`1QzZyQ%$ zRV4*rG*YnDqFux?wvx*fFaOeT5F!E2S0lu70&G{rvA^s_B3*&uPy#jEr>|ri=bG<~ zU;+$%V6t9N;Y5$n!t!tnR=!eiI2nnt{P#{nz~xiUuQ+BWe3WI&F5g<@jNxq+kDJKh zK_^j#$;mJsxM6==G!p3b#MG)0)!k}?Zys5%@AMPFsm@T`26YodRkd1of4wZ!1~hqw z{djO8bI#O?JEE-9Y#rmAKY(fB)$<}bGc(X;F@n?;RsnfE*SKPl_0wT7l`aC5^lM}v z5|W@b(n<4NXBpeZYUtwJhW@7Az4K~<-7&dH+cS$ozesPvN%BQTGYdzg-4nCsxp~k< zNWnh&GRe2R<5hn&Rg(vRj090(yrf&hqaEM*lEznlB3i^&tcK* zdt2q#w5i2evI#`}u!4ajNH_>G7Y~}yi)-eGmOhHttriX{$F&Ae#d^Brtvz!&NB>N< zXYbl73cBo54G%xWMyjYB$-hI68zfz0%XwL5PCzh&B+^Y_{fg>5g(cr#06kj{zY&EZ zkS%kz*cSLK5x z{RUkOd8LWY3Nc~seu&{2*xqlVU>Ofb5@6r`uP_gdBwFjeewcG1s}cDVLV=?@%!l;= z2@H+PY7!3aR{N9lQKCSaCejV12&_%%TLlI)>};qNzP?~ZQJ1aJmodw($imx)#dmczLd#3O$fE<)|y zsi$~M>be)vY$5KPTq^8hS((ghC72x%S`t86BcTcXA7jksQy)>BIM(u*+1)Kg!BILv ze+S8NP<0d5T9KBIR57PuFv`I>5k+}1ExhP=mapz}^L2R68#buXrOiq|XGL9`V*f7RKmbI>TM! zY_jYD_UO|;?XhiwB;MtVp~97Aa<7v(+o5O8bK-$rR11)@M++==TimQBvLEc<*kn2) z`&Z1-ECNZaXCVYKt<2lxjn1*v6^QG8>&r!2nHl5(rlc^SJYa(Pf10}Ta45Jh%ouyf zE_?QECfgWGmXApDjddat#>kR&R3wDSHg;KusmPKRdq{>bMj4VNOj4F1OSTea$@ooQ z-|wsY-2d-=?tP#4yyx6=&$*Tf429h5JBu~Ok1>>0sq5%$9A%#Bqg?vX66I#!gj^}p zh_VHxW6!mP>^5kV(m>Y>?G4&mqWue(hQ7}frU*eGv$L$FQZZZ;#y1f_5@;l8Ynza2 zd*X8b8c65l{@hw`KUPwPJ6L)8UuPuNT6%BVqE%A8T@4^n5dW?PBp5F7(>?OJ&Z23!*lmhZm! zb{F~u-+Hiw+jn?1xEfHC@MKp`-&G1}L!%sd`$J(z@gMr@EP&BFGoBDM7N$c0YcaDf zq2!G|c!O5up(1W++gWz#i;F-1PwP9e0Pw{R6OEbQbmViowilONbvFu1ZeWxc*&Fl1 zS2&tz->-6<3VnV6T}{5aoL~B^PJ?q+;7n!_L^V{-@^-sy?PAwqV>!kAh9*SxhT{C* zcm+N`zAgKP?u{mZRkze^z>5EazM2YVbmSOtGhF^rX=HQTe!=W|A9iPPlbUh5NX8+% z8>_rBDlv8~O2BY%EK(wv($5@1Iqo?Of2Yp*az{m;AW~brV&AB>;S-bti0AqRQQXgt z%=`)}^rhu8Oja|l%kT&=?zguLN&Cf|!C(6H9T_onyq(c6RTOR5$X4hs8#bB8KJf=& zb=d;;W^Lrw+??PtvWN(ThsGI@E1W#dNSKO$T+h8|LDYw2JBDk>FKUDo;%L5C?r?2L zCr32P`ZfiT9<}?FQ&}#BlLE#S(QaM70tkD_i4>TtnD3LB`Mh4KES}ptq3V%NV%}jV zGozaPFehE$4c9h>f~+4D+$1w`JzUNT0}h_e7xYC zyS+>a%}>axiuG4$rbpJy%?+Z#93%z7v!w{5KsE}^N0CK4Jxeic(u}3y4^Lc!Uwxj& zU^P2eeSVa8vXQf$XfzRCgW$f@iX=~DXferpG{D<&BFBh@bD?ir{A8`U$PkyFdF6x$ z0a<35;8ogd@a=Y)fsc1c=t1RNxy2ILAuW2SAq}o4(afbtNLtwQR@aPT?uJ%CayCCl zrRMUgX+PF6Odh}M(K~Cm@yhCg?sH-Drp=ct-fchN9_SAh>{>g2hboL$D0WBu5Vue; z1yoPrbsZ3_dcxoFk8=u|1y28h&-*t(k+k;8n1lBx*`xG;sB5qI!%I3twSyjI>jV&Y z@25PAJf2;*cun|t@%0G*RHAnbDTvT0@CR*TQ%zW4E4el)ab>D8trEj3Wee+i*u;`% zg{{&yCDs=X@^#}9q}gs(JD(4#fI5rvMVzhM zBs>Zf%TgPk^lw7dI=#WQaU0~Bs3BJ(n0ki$`#&Kpa_?&_b6;&9JD-&kbc*bz$}MG> z(jDp;pk32V%RqKH@d^ZYy>2^!OA#9u`YCErfT#|GxmNGrj;t=LN zJFl7W!rl@zi=Y}ReSJn0LouuTMr?HNYdBh{IA;jy!zda}#`XV6>pN`k+hWsK!c=JK zSOzN9H?!rf->ruVN$Ah){PQP(AWZ-W%v0q7X2y%_$0@=;9uJEz9$5bYJPrF={@OD7 zf=SN^)$}REeMFc_VyId+hdHx2{1rGsAQchjA=@>}*CDU)XzEnJ5%M>o+-G5V>f8DP?=m8P0H@$6SHP6N2n{j_YW z3ElaSE^uY_US#C(_4DA$;#<@QsE|?s0*^o>EkgfX*K7uj#kxi>8m?7hEY_{x1nT>iXvc^{H zUc*C9=zSZlql8!~_g|18!D0*R>$6Mg0S6NlJN(G3uN5YczLBP2*eyR zpsSUP4}!k9uN~C08t zu5qtI>(jP_1w}h9Y3S(bu_u~SIMaq{^dGFU;D~Lwhmb{ijWhpOw{2&oe{D>sDZK|V z_#zQXzo~vpq;kftJ+}bk=?g6*WGb2-`qQURaN=UZ3ZFiG27dYk5)29c@yZB3&mPoF5z40LswY#eOujF5G82gj%>q3xU%LP8=GbUQjbyE?}RMs-JZGm^Em zHZc%4dp-eNA6<(Gu+ThO`a2qN$sH76J;75D>BRa?#0|85poWEiLVm zeipFp=M<uMhOz?>!w0E;oF+V>CU8Oycm?#*fx3Yf?!kz8awU)Tg6NZf zU1TC~5;y-m_#J%z#c%l6=uHIjXJRlgu+IZOUUHQJH-~@6&93Tyo@m{9`Q`WWt2b^I z^`}P2vp_QjfDrmCVZNTJnOQazdL9u-7(|F|7FgHT2!#_kVNfMSQ*9yauz?|yu`WTc zXfmUypKq5pB^cNUen7G~Jy;JYE;#sG5hNq45I8u9CwP#j{imJ^IqMVB(%O#*LfMF^ z+kg7>1^Leph)LG;M}P%Bi3@*KasfGML(o!MNO^w;`Pv1E4r@*b4ww2Xp{j4MWJn~f zi=||2{xZ!p9(u{N1teY?ZAByu8z^)YA6KiqI{R%Jy+yFTte$$Pp-Y-NH3-Ed^E&;$$$o!zKl7SLkFClV8!G10 zUoJqXfMPh@M2BvVq<7ucEG9PJBZ-(FWe}jB!*>xwSKJNbZA(&5EdeR$P&GL661~PX z6Ea}R^cV_mM%)J=_Mhv8^kS%h?d1%C?d2|on&Y~;BQH4`Al8ZEdttM=huv*+pUUvE zO{)iqPyn)kQc{&HlXI$eqFf*Pm-h+^^)W2vby*CWzM?*8z=X0RVW@a{tWl@*7%X9T#*9RUDM$Ct*w`I8QSW z6kSWUrwn`?L-Oa8Q-mW$n$Ul47XTHK?$oc++mEi;oUCc@*%?jwyh`z6??p!nyFd!2 zIuNWkY)C+5URamwfefh)st{209p++cp!3lQd)))u<@B7bRm-F&5dPdv(h8;CIC$IF z{gA1PyJ*TYQDkLm&3iWUZ|hFzOtW68G+QxBcOS8@n^rjEwni8@TIr5qiDw4ojsJ~_ zJc<0^9uFs4T)ug)%vRe`!Fw*%^!Og2cW$s1k~Vyv96zM?lsn}_b4#`&__tMwZ6+*O zKJ3kp96ZT&AoqA|u53ULgryxiD`p`oig$X0da7W1(fd80KF^`Gt`+OZq_%loY>*S! zg2OdD1&CNvMb_suoELKbC8J=meV1q8{mW`|HQV~U!3W=+8k4mp4a!pCqQ!M90lB)~ zdAn>cC!J^>SD`A)qlcXp?*^c871>)Vk|sC#lcwBm&rzxI|IO9TN<^1eIP9}!hov)b zMeXcbVfTCDEhnA2BYPe)5M)R^-~I+m7XqCP)}cD_&aDD60zE9ahAv{h>NwL{T=loo zjK73hFx3Z@6Lm{|D(_11d(4&?1exG37p?C|5^%9vXhXNfz=GrI%@Gu?Uc*v~(3g-H z5e+RU>lAu`eclwVrs5TsV$suR@F;(Ci&P-z^y11G`2T0|M_x0SL%uxIS8RIgOcl4B z`r3G0Tcx7b8YT;Z3dstjF1@{b^5a;jn8P*5zBj-r&#|n+d(y*t!Z}1U+(f8@{bidX zxIm?8*-ACMF+ElCj*j0e3>i{?O6?#)#dKDp9Mmk^Of*z%=0YMVSGItG2Y^yhsX~{dgr8#Fi>`j z1o4w^fSKMRsTOHr!&MKP)`6T?@SLj)&N8634Qu&H;cp*~z?YEh!Z2li@tQtGFPS?S ze1%X%_L2mV!FR<9D)W%P3*wNQ|#e$paZH+u{mgYYAMS>P|UJY9}LKN?Ja0Yx`86HrVV}5G}3AQrOY&e|7ELQ?f!IHeb?1J+buc*~=X>Y&Y8BFO0)DJkZLZEXzUW z6LR!_O9W$3`d4L(RrMkbJKj;MDz1;e;fD$cG}9)zQTOf0obsx8MIEvv7V{BVK?;~C z{tKB9{D4e%`)0A)bSRl_ge~DItS#1w?ThW$Ucs{LMvUIPPb8?yYA*5@cq3s*2c&BJ z5LpQ@>ntU~0tVM01hvvhLX2l}g-7uf<%^LNtoi&1C?u?ySN)tAMV8LB0vfE5^>rEU z5E7)$qX^e-zoS;{*%N3E&&gU8gAXvRy#+(A`3r$|s*&3~R>N6$lvo-KTb6-{r5Hru ztGiE5FGhqX(LovLlaR?~hAr7EA~I`~QO<)~MrKCy7F%)8E{4}p5}P!hnz>+AT{A0d z?B+CCe~A;NwotYRI%cpk$M~HOs*T@l;c*Sz9l^NAJPTw3X%>atUC*}q+pb9g=$t^} z-X$N|p0OUM9#;6D!`&I&+d;y=(DzOEh3h5rf64bO5Ty;lyUt<-*DPY(_2y##1?0Ma zv35M7@72Mg_E1!{Tzi>- zoMU!YD1uyM7R9SKUOV(e4xVoQwI}q8%~Qtp!;PZu?_SE}1SgclJkleHi1gfDSPX_K z|Mq6VU@9$(--N=M*QGC+QoAN*@9Dj1&<)LQR1C zp8ECOEK3@r>(LVrm-k>Y?hHvp(OO>hZz)U!s*+#mqKi-Pnm{K0M0N`o)r zU*iT=4NX<~S~4^qu<0QpLL^RWHcsNg1Xr;BiXmYqt#1WO@Q!dGud})aw05S#@BN206g5&jK*1V@krs5h{KTZD}B7U8!MjR zCsk~lH{z|I=qQY&e0Vi`qdm9R3c~8*GVjK9KT8S_kk;^j&ZX?VoV+1s@Df!f*L~rN zZ_a4a%3`#&U3O|52Td$zYo&8=Q@l!}o+g(G9f+_nHrVo0%9E*PS|#4Tz*#9!Iz-8 z+v);o*i?pV(&QFA0f$)54JzrLdj*3`yInLB<;xRvd_lj*Fhh{@7ftZOg3O|+{I4s? z5NZsr(?#3)Yp5pOZQ5c=R%w9bm%`uKoD$vVsY*5x!>$&HVpZr#dGI2sKyS>ri<`rT z|N3bo&_U-D%}YZwPlLNu%;ibh%2J~vPA7g22!WN+ipC=G^9%Twk&H$}H{)Jw@=wJA zvm8pq8h$-Zt>|9pRVIY}uyTbpIf1_a-PV9A2<|+&?EgI>AS(VDDWi6_q)UZLq1_%g z?(G({!}zASc#{w+02TxbD3niK2>BOIi0MowoOA@WFsVSyU+k<;6+}5(a#tvj!}e-+ zLjz6Gq5d*Gv(GRl6Gwh`E?PX@EOXWqmhm=^N}WyvJ4Tj8VFBcfIFw|6)R}6!=fUcD z{nDLI!eIA-h`5}0r73?Ysl;@pkP*v4+dEeMyXJjPUO-XNc6$sbtvz-VJ<<;pGLJdr zqj(}kKM@Z{&1E+=At{qK4TFvo`y+u1(OIRIU3zbhW}`rkU3Wm2c~nsHQh2*$1mpObSe4ss|MbIzso=Bag*0IL6Oc#A@EG~9 zu)jd|#~JH1CDjGG!*kzgEvAM|y&R8;IdPe<)uYTsZ$zTG9z8a)J`7pQ8mP-O!*p?b zyARc!uTqwH!(%JpcE{N}<;Y`5?;BHP3@5(!XAG#1Kbw3RpDQ$3!jji)f{}cRtRAAI zRI9e*#dAAhkzoUYIXRS9)0__aBYz=vxZhN9$})_%6hfQV-Ta!ENeqdLDMjRq_)#SSL!4mei2ecQ zNpd;w@t~w*OHM7{CGqdyo%B>;Tf8TaK?3q6*x^q{vB zZwllts^~}X!?5*Egz|%nDnY+TbhsjEcbj4_=&0|I%(18lfmj{0FLK?A zH+@TeEoNDE(uw2g;VSDLPhaQj`=U0gplrqZa9dMQ<>J|_e`l^`$E89xVkaU@NGCIk z_KHa|6nENErHF)&5{QATO@z`L**!AMLsTeFdgC5e{ z7~6U9r*u9zsac^_^F2jwRa8ShD>906;YPKM(A)c0W{kNpn=6 z!Pa04g_zzTK5PnMXTIjZDYLT8rT`yqW~OquSxJ4nkRKu#-c}uiFxu}D7)sU9g9ahn zJ+NpL6A>Iic{!ecx$V*D4sL`vXjALsNa?%2Ct~GQp;fvdOfgNl&&12a@DkrIE(of=}gjZil?rWT`G3Hj0|vY}b=#k{KasMf`x8XwRBSN8Uk?cXxy6~On{8+6WVhq>|AKyB)srS?%D?*K@^2JfA!p5ty!lKcHT@OEz-XD&JdKJ?g2>W zpj7_~4nN5*E=jT5n{Juj?$K^BcP~j8-Qp9qB-;6#VmsA{2_DVgyTUybc%##^W+waO9kzN0s+`NE zWfN_VU6`J}iWMw3=sJt8SldTRsQdGv#1!4N>X!SWI!juQRJcJ>f>pl=>)oEk7iuf{ z>Nz({}5`XnkeGro~$@q6C@SxPb`DCU8 zRe7*1_b0&4*a8hcIsg)#o=!5nuUEavgcnw6aP*ngTkI2BUDOrkX}3?Kmzk)It#x8-2f>IT zs_0we`o!I9`-Q8j+K6vi1zAMNkpRKePm%S6YrINPe<-XUA7n^pUnjsG@k3sfbNz8l8T`z-JRKSh!8DeWj#VzGB>gbL1^X`b(0d(f0GKTTQ zZ)bxYw@_AyG@o_Bl#GegDq^9HkmZ-@a1FGGA~8F4pxZ^uxk<60KW1!3Mu2_YZs8$& zK9;mPLphV)8t!|*EMVdcqc*`+JXE5$`Zld^1I(236~_t4VrFuug4c7*i_HVjI8M@b z1uyF!PsOU#Q1KvA!7plP-ROuor_sc%9Qa;`gBLGQh`f0`DtC%`^z9$~jOpAMU%pTZ zoYzew`_07jTmr@LYQOQtZ!eIFFo z*>b)VL+{~O+z^W5oAp9~)A1G!#j;qub>X6g7!m?m^B(km*nm>Gb-etnIx*1MfZrZ5 zl2m8lV+#@ziQ@?%`}Q=+39i#PR5(_%Z>!{N8n0*88J{*>M~?1^ULIPXXxv(wYGgt4 z#CUhkA9BRZwrCmEaYPJJMQj4RfUSovBZ^6H$9mS`$iFOu8C7`LP)4=Kg>*(Je-&{0 zUD;VUV2|*Y>xN{QWNy~wTV_$YH3qg7$rPzbh5fN^tdakD@qU{T+ru4eLVft9#a4ly@Di>6|aAz>aWxmbtAC$?A z2Jq!=R%~%7;J^7g{R-@V#j2+ThLOL+InzigH}Y0GITahfNSJH)l5U~*l`;Hue2@!W z%bQzF^F%2vGr733$kJarG73?wm3Y_P0#f^CXk?arv{7`}cs;=DctpKZ-^WM)hfK3@ z|5iq@Hf|LvhlQkZC1gnHKXQR_haB~vUJJt+U@p7D0$ylZ<#4pm|3PP~+OU?DvY_!~ zLD@H#P-;zYOkDgnCf?V;5A6d^NBb*ScG~d8N!zlnKPrYyyLhJ79csXgE_HCXCo&@7 z?A4899sRA@`%!23JYoky!w!Pk4EbEIRZcKnz$lvyBU#Hxh>>f;1MFLn^q}Ii59=3w zGX+9e5tO}6S|zSmtich<=X|DB%6no&W(Z6XJqDE}T8rU(Q>h+1d2mI(%+&^z5oEAS z@yTGkAwf$z(LFme@JghnZ_Zmm7W$Jj4$x^%XR)eVi|qLZ^~KP<_O_3SXcQa2X{-d4 zpD|0NZ1CLQA~fy3T*X2F#cJ)6VYzC4kjjgzjsOQU1FvDB0x14;_ft{X8h^l;VI303 zO@(t?#QfbeMWK?BuQ9U0!uRwxTA;g@!Q)wTIeRtxy@18UnHVV`5j1Kle&MY#R@bIq1=F; z3V%A6wC~>N%4AqN)ScufX^g8VMlONp)hYeEg;2$}N6)ydeXk3sn@|GDX;_m-bW%{m z{q+I&KyY=I84ZOFgfo0t%W+hCdHT~S6;_g=fz)0%47+N70i5;(`WFK!=l4|bdNq+ zAU&9$v*{Cw_I?MP~M z|1WWFr*>jngW?wxbsMkZi162MKNah4zY-!+M&LD+Xc69oQj(f{&zb7M_v#wH9xj=D zsMh{xoMN9RP>G>9zDQ!TlGMXs-&_qO4{RYh{07O642cF>)7~gSevs&V8bQ6*-`#7( zd0@ewL&87ePUw&H*=S&L-B}L9QrQbUH%^<}X8*S|7Us7EM^m7cuz@G<(eW6^cjX|< zHmLGl?|#m3B_8A%<3i@qF2iq@brhxiiy>itmK8wuWINl!ax_sk`WZ(d#MoA>wcCV2 z1=W4aZ}SZod4|EZT*mWX^z8EQy|MPB?IHA!WhFH`urv0kcHB{+#oUx}tb4DynH+8v zlzW7bA=N?k^qo+=k%a2q3o-sQq7ew7-W*P!5v)|{XuW9YaZ;Qw)_y-Uf-VksA-c$T zc)uy%zPdVSepKQ+_a+QlV3gENA}+#cN@gEJjE7~_MF?LT!}UvX0`CQ*mu7`Wv8NYr z=?_ixVcLMCJftI|ZY#=L9z1A`%vgBCoBrJ7>mfqPXWX(fm;IOL2Q>gqe0@o|L}#r! zbIcFK{khIXn@zmYjl!F?RBhw_H#jqmISqJX%R4URW~jO~(aKzJ{gOHTuN9D1=$M$H zg%r5NRVB4(+Q70#n@b|TmrVmCcOVg8ybY`IxR{_|dV?Ju$X!Nx6uK721{~#g6T=-t zJ&w$%-bODYE%mZWHMgmSFO*lpkZrX!k=p8gQF#SmWo7%9Z^2GQ$!@nwQfYKIVV64e zO|4b6SuNL0RPUm$)}E($Eu=7$|-YZx`1IV%*l>l$d{R2Ss zD<&pg|6MWj!qKqn&_Ke!tgUx6rTMdafUk1PSf|pGqH7u z%tRxAu*am6aYzoE`0b^Hkcte1vgr3j$I0k7(444GI!TeubUEGsVAdU^RyF;cLcPVX zAr{XMVHaW0CjiO&#L;Qoj#%_cCDOs0nC9G^?f;gbgOsy%GGh7 z=Zdg>F$C^#tEOA0F*P&iJ0p7Ele;Yf(vKaxQ^4rp5o=^2PT&A#lMg(&gJbde21(6x5P(4me>dovM&5G9b>e+AFbfr~Qxnb&7uZH!VP-SnYiKYSH9Zy{2#wEwtI z4l$&=+hJ(iB991dPoY#(qgd+NMoGa^UycsJGFb{VTf!&07tdYfDo@|{n>FfL6|8KG zkFd`Id!8lU4 z4OT<6O*I2@rwZBY%vh5TAy6>Rr`;WZ zwq`DIPky;RbEc8piE-GdEZ!+Rs=;7|DN7GWIWJcTrWn5if%-}bIg<3Cd>o3!MvN!mtUhz4VU1f@+-Nb zW-H5Yc4`Kd2^agXI|$t`SP%18Qoi%LEA%4PiB6lWfTDn1T&0CZBcnWT_#-;UEZosd z{DpDd-Wr;}344DufkuL#jLYo?UfN4G{zxJ@+Y`ESnRa+UtyZLhIV+txJppOb2fN|P zz_x>=bMsiIuykCKY`7-4Rpx*dD(_;T??UKSNGd=tKUngP&0X^&{{$k`&UZoa`^>$-t8Q7wWWxM zZB#!jJvV=dws(2P*ANr=|9d_$q5*Fg!Tg&qkJsB(hnCDA!NMSuQ&$iD7>Pt}p;w#z zy8wUEpDfE7sf72iK6%$j6~%-^RN#je&OfV|bZql)*WmlA91zixLS~h=K5>&W=0><+ zeKUg6K2HwZhPbakc)~mbTUsx)NNctJ1_q~pSG%%mE+P~k6%M2BzT3iE;O(m%C?hd0 zyTy>>&2CU=Ts@p{;l2@;=+-plL|KkYlvdWU5VcXvO#J`$cim+h?zxyA%gUAU4~3F~ znoMMV6A}3syBicKE}{Qf93&ISHx4q3;|(-E*&3^xlV@h5DTP3)FZ>T`wluhEwCG~j z>^Z-9TE10=@CXD9607!wMb)EUf;mBXh6tb3Vv5PYPELQQ_Bm~JI^c+yoR~9laWRP7 z>R-h7omsyQ-dsPMrxiFke#E`I?@Z4+so8pgFD86EoRv=KSkl`a;({7O9p;W^og~}N zGRQ-TXEJ=$f zxj~#|iLZmL;c;}Ugl$e9$`jZBFP9`h;o#uZ+C4a<@cE-sm`$i;uR{Lx=OUCKHOaSj zBeV^hfnUCN)|$a4wgSNQqF5Q-5qYo0!ekrAzn@iCFG zh$ykfOW>OQC-bC+ZYJXi;*991sNT1lxXXFJnc#bUX*H63^07XTs`Yi<}GYgVs)FP9>rDP5t*`c?(1Ex zRXwIHjaue;pxZdCj#S{wwFRCzT{o~Q4}$2o!0(kbl~mFkpTx>O+uU{bEK*9+(T#PD zI_*l$qtbgZ&;Sdnyq|d?XAZM+AJhV9HT>Hq((ACFjpkfdfyI12?&yJpUd`#p4#L4k zn^Qf4QMWuYLzeYq^j`8{Qf7#cS88u?~gCRHKf}fgQ3cPh=&nWpcU;; z#h$Dn+N2T1Ryo!GoLd?Is>q7-n+AEwY|HsXR80Y@6<}}lC8DLY4vp1)bz~22K?mma zWk_Xq_{wA27TSG5*x<66HteT7T{f$GCC{N%`b~9M$KkNM0ukE?MRiM{jHK>?xWP#8^E=) zXtY#wQ&w4(GryM5H`D_~W9#I>DM~;QXhQpR5JpOWEy3`P8rJ>3nOXT5gkYpW8!@e7 zaomk25Ot=2gg5YQ_k^ql_N*{7IQNPCXrX6Ksl__cGC^c6g$rBS&eURuO|9Zzh?3$y z`+qYVfXvUTs@R~F&Z;T^RE)rfs<*|~%N=s`!A3EikVr9@K;^H4n`8F{pS@|~CmH}u ziKDjcU(sSH4dla~)9ovApz6jwFPn_|>@16ZkYYsqj?6h(Y}X8ZDUc~fIURtLweDM0 zb{)0{N4W|Qlh^uzP**4&4l|j{rciwIJ-48NDmot+=9E%p)(0)!B4uvWyEluV_}6X7fh!pOP(*&FUrfeg3wk&+ zjL|KpleGHM;CLKn^s{~PeR5^Nay+GG>Zu3JJlb$X-<_REa;=S#<57zhB(%YtW48P0 zAj=lr|A0qe6JW`PZQ2NzjxG5CJOOMVhwCCeK+}7qx_`>7Sb|NqWeugTgFNHOfM%IQ zB1J}ZChP`Ur)S1nastH(?g~MsCMX2RVp{BkJ(Zh`A`!5)O0?x}%WQW+pGWr}mSBt=-4I0P<=Y^x#Ow3pm`bGp zZmBx*qY>i|e)D-BE-TS0ExlL>yOF4znu#D{hD0FCDwAyXs^(Ku z5UFbQWd-7}=((H;l}KXI;;DtDkT9)08d0Gn!P90!wTRayC&S7n$^Wq+g8=A<%LWlA z{EtC`1q91VsM);`=TAH%AVHF*RB#KasCmoyW8xJm%4>pc_or#YdQZa2#3F6TA5lh2 z9%m`tLAKo2+AzZ>R~zt4W{|&`G7TiA+wu>iqA=>HVriC1Q^Y{@BZIF9MUuzXNb$C? zgM_1rGZOWt*ty>)n5+9i1(5&|nRI~Hv*rEm{%RR#ma2fnRof)}yK}##8AHeLHG+D{ zwFXHzctrUcMM6;v1d;sNZd*gn@L<(J6P5iO4)Qgx^4k+LbtuO8Tds!7b+2waEZ349 zqd?!fdfXOv8lm)>_`C=tD7BUn*heO_!lG{MBdEc7aGq)jPJ@Qra+EY?1?>eXVJ+zS zb1yvQM?2y+#8P-cbm9ItE$*xQ5AVqbc!QGZ)(Jw}nIUDZemGq|4w(~XjSD0)PgW!(A|Sko+}6Kr#6IL1a7i3pT$oRAWT_5rLtqHyd(dCY^(0 zXh#$qOL}?#`l$t?wVFv5;W3duO~1jhqIR$X1EaT}YX)ywaxwL=NzcnF8_5Wqt5Xpp zgW;fbI5UH~F%ZakMY3aWSwrpRUq`87Kt1V4^~mKFIiOUxj+vZe+gmbj54J~9jJ(fV zOZ{!FJQ>gu4ocp*yca&M{cLivOzIa?1UgEa{^h)o+8mzA5JiFaGcosNFZz$+TIzC7 z$e}v}>U9ofYHniV>SD`!6Yo0BEyZ~yh2Swso^Mo={p=FehF7&gw~70A)DDd3b429Y z%0=3O02RJkO)?lc2tpV%L-9(FTrX<>^>}hD%FpYHoh@us}54 z;OLM&(EVk--85*{l~LZp;>H0Zv7!@t%gQd$CXdtEjRl@PvaZWoOum4LP@>aFLJ6D@ z>@InaIwHII2^(7Oebtyk%Es%nVY83wu8II*YvY}$m?DP28ta|sL*nzNq{#(x_*2)F zi>&Bj>H5b-I(C9^_G@14GMCp~ES@GIv`Q*&7?>Rq^@Mfs94|zsbKl|GQ>v>;I!XJC z?&KzsGZeJ|QW%XE0F}{(G2s}iKLa$IK|^OM^PA}7>3AZ!s8CNiB?Bf|;p z(b?IZTEVJ~`^$svXgcSN!8)UJ_p*Otzp+*;Nq@io>*CIkBBQ%i*u-;;^6K@`23NGC zz5p&IxJG=zSoi7_h<|q@40^7pPpS!@8_Z zcDvBaD=QGxWSdvKyho&gR!DX|r!e%bK4vpYB}{AT1E=46X5Oi^NQxt%znfx6Ex8aJ zr#(ESk`AL~eILG<9Yj1}d{!Gcq?RN$QXGQ`TDQdL7$xR_1>xS?Ia{T!`Mq1lq?)$~ z5qqIEN{zAcyxj$f6?y~aH82-Jhz~1#fohH4Aw%&XBfFrXqP}Rwh^pt64y>Tmt7>KG zC5ooYr?(_Wd*CZDNd~dnLaYB0l7v2;{Kydt45w>Q0VP8v82-eQU?%TNfKc#u^ibn% zYYwcSQ_XZ z_8=1EIGV>LSJd*D0W^hwusvPkPj^Sq3t77r`apsNZ+D@oqRtC%C5GEdfD0AxFpYX% z>o6XjPXa$KDrzm>KQrNVS3_8E;=;O``P#?JU|Fnm6I|5@KbO|lO7JCU#5*#pLamMj zilY3ufA40JdGPb~Iy=`nv(Pm*T4B|QZET9}8Sih~xX9mvI7Sfr6Ir4P3JQZ}(miGO zbce@HJG}l!rA^hSX}SD{*~O2PrBY6OhQ&-*Hak!*7(3%BY_2NR_=GcZiVoWheU$B&NxyQEqTegaH2|(d^!yyh)%9ZK`G5w=?oy&%s5EqUUPWR z+Zmc2EnW;Zsy3Tz_lsd&Xs0QGQnSAX?W?-`3E?=94z78+6h&4u;y|AmghgrxMBn(kT>a~+LNfU>rA%Th#>Ie5CM^atAPH3EfovHNJrwS zjpH`JYt!_$?%{=L>jdnSwbIXix`Dpjw)Go&rhM@DPalTh+`}KrQOY&$Hm%o?i9`4i z83m7=zX3$#?u{E#N|?4HT8bZIT_M>{uQ^Tnmo?S~`C^4DZns?iNu{4u4=6E*Yi|UG z@JnPKk1$MM~&5Uqz;o$-KI?7%v)wXTV`#_{M2Vp z=okHCIi=U`bPeN->W=|un&g^RL|S^NL)x;Cl4lCc^K_H zA`u`a*g-@k?QTP~!1NuNiGzF?D-VrB9mh! zWm&{AZ(CChg$WzB^^>3FC)`n|Py)l^c5RLcU~_FwNJ?NI_||y+Mmyza;+ilc#|cmG zCjC8IaaTHIw3FJegK=o5r+x0$A0>10b}C^RA~AO*%RE)$F{UcCw@~@DN-bOKE|*`7 zvvlk0RYG2qGj|le!XQ)Lh&DSQzN&tsZXrsO{_IbshvA!?qDG?>_dX7*))R`CZwsfk z`9L|?_4sgjoNYx3*B~VAax;vdaG!$s9QnIqbx`}AW%X%bokXh_$F@17O5@}b_*Ssf zqSm0uJE-k&>H24 z$lyZ|{8r-fp$~m{JdiKtOt{~gp5u54qJ`Ma=%x&{UM!$s$#&a>xpefb`c?D4HNTVy zo!s7j(-*0}noATye}5MJNW*sy#w31#>4)cj+GUg(>zujg9@}WNLill>ehG}PB(la+ zVX}FMIp#nElLN0JE7qDBt^wrkt1zlQvO=ZPfS%IrdXREpHI$B%{FxeOCW)l)r-`;H zpAPo6$Y?&tom6CRTE6>7rBO-oo5nCz+KFl6opcMq;}-1)vGjB%QurAZWX9uEmohlw zKZeLC`n0*}z&n_=J4O>jLz;Z@e$FcxSMs+jcpUJ-xwm^Qq)i`(F%!wy~n^re0G-yLO(gAd)lZvFdb^jC?*uLud3y z%+(SsK*c&q`|m3CujwAfH6Yi@BQO5*bdDQX)(?9%mv*FD143 zwM~fBDjXByS!6%ZOK0DbA0P$E5|nC%Rypz>3e9IKe#kDa`&iKh}c}Y*yU4;&GgbS*xQ2H!)s}JKd23m!+g5W*ap8_c2mOjMmpdzaKPIMP2YXO|8?^eW7b6|&;zvLkWirE#Wm%-CtF^o4+-9s@t}hJ zfo! zIu#lpR&xDi*HHrE4ujX3$h*En5iVB#iAwXhlB?=V$AdW!+fbOXD%+ssDLj7t1qbk3_SHX-?gG)%9WBM)$RDdc2z2&JAskA&CX>t%hDeUdV<>bNUteV ziRd`%EilJd61$%r?xcTBc*foNt%0 z4G#qf^eJ1g909i!Ox~t%>7J+df&;I1oJVijg>*H#UBhBL;$hfXIl84fv8dg}Hgtn> z{Wh>-bkz(rB71a=KXoZMaro3?p7AoJnNd*l;oZ)lnnTAlO0QNvMfm=~Uluf6XG&i$ zm&&Hs9BCou$9-1{I-F2U%|A;pw8fPCZkqgO1P4!6q(tCmI<$Z%{h07>E z4*Bdo=fG>PVad!}t*c5w!;(`Wh30-c$@^>EopgO%u^WQj>^+UBR~_D%@@KBwgX5J2 zvwAd>5%2KXC$RyM3a$i<1fyyMapz?fP4#whQ3TQNjFok)_lK-A&e+Ez)jY!q?^c}m z_S6Z^UM6Ki(@yfL&aZ@N7i_uHj3dVL*xMWlP}3$OEvl#+xQ8t_2UZp_xh;Jj7hMp= zcSS4MZUcIHdiAc?drRx?=e*8BLAur>@Fn>OeC;f?_YC%-cie=D;O^Gi))~^R#S1qE zwi#|X3l_|(2}{+H!&T|m&v_3l#w*1~bGy6dH%p@l-?%mq^6-B9>fFNqa@HBFpQ~$F zhDS49Ih!?OxW{CqsQY}sK0lFNv#yeUe4JUbLkP=WKdiEbov(eM>y-oUa5~t+GWhJx zQ;rMvl7D3vV{u7SURS4loniA`Hv5TkZ3Uf9%hdh-^`4jA(#J$5t*(Re86H;0`=P{% zV8PZI7%PLHW?9E&!1cOee|taUX-UOJmZ?;ysbilYogC(Pbeo}~=WR9L0)fT5Dp|MP zwpp7q;Z)kW;hrSlXU|~N*QTa-9GiT2uc~v+m_{iuu5aYqxKhH7yZ-y0R?;Ybx{Vwo z9LSXq<@mYBiwD^tUe9`mqmV%hT8T94vG{`y>4rYBd4n=hUPEr_8}Gl6YRa$8?iLSC zJ7GnyvwgV=O=3vc@}d+-vEQ(~ZsYTP{bjjFx@AoxO7HiTB@-LjQV{o@qNU&qxuB19jz?+dHhA@I=F1LT*!R< zt@t}ln*+OG4gyM0rVqwQPE{4sND}?P>xXZfiqvK5EI{ zfWf=jexX$6TG)}OHA=a0>%?T7#6SLZhyF`DGoml(va-l;l)I9bKRCg?vTO&T?%Kwd z?NBigi|pR%rg==_Z?^3U$#9cL%U|II`WW3?ulV+6sHR z=C#KJrl$v7O$gLZ^Me5Ql}P1LEj%gpBJ{wLzBhB$S5m)%ii|d*%|%q!K(f`E``m_p zj`7fq20QPdELP1*Nqe{l^a&i2O$6TyI;27gFWh2J?FPh0iJ|0i#7KT1@Di_eNQ0 z&(+y%8C){Y(1Lq|30HO2(EHHU(l^)Bxu!H4_6YS7o;4J(;@~K#Jyt>>sowP}b*0^e z24}!y!|vSRcd6cw(Lp9F=7&}vKKe!qW4wPZ@!pqxPejBK0e0XVxX1IymURsXaziZo zwCLA8N7iTEbnP%zg-?+*{P|R`SKa@_J_bzl-2WO<>IZ+sVDWA3N*N!cFc9H@P@)Va zIyms7y^~+|kt6%Qd@7@_OVjBD=qx)VzYxG= z{rF)I0wt5vB+=&To2t8mY#$xNEJS)Y_Y< zRy4uY7jJV*iIpJtzM0>-6cyo*Tvgnk;#{4f{yhj>6q(T=#i4vMA13AMB+JW`I}#L>$V0{$Zo50J?K*PnttRlc<}63 zX84~Q38$MM-|zHE?Oe2rCOA`Ga=Ry|9MeVd7fZtsUhXAOxOCrq<7FvceKkFwIwRHh ztS|O>HL_SjGe+CvTOrhLgJrv9&__Wa{&@J+;($eg7=^Htrb2#6u$9frb(oKwX2)2k zk~u5V1QXI9Cg61*=#C%g9uRBZ;#TxCUGL)V5bx_g)-^->YCTZ%qH}~KvKLvnEw5|Z zvsV{&!~_ofH0#HiZ+JJapmTgi1(OzY;IRS?lFgW{ia`2H#tMxHJ@$UNS8#?7*-bp zw))9n*`MyE{bz%l*wAE_(8yiCC^l2s-}pB3h7py6wOI@lSiq?sZk|u_3VvFs3qHRf zj^;tGjlvu|L&~*HoHDFVkrGLpI(AlFP4|NRN zw-J3W*8a?P)Of47HwDOwi~ryM0$7huzt=cqF8SH5qUpSLxnfF1euV<#eu)b^+7*NPxWyWB`HTH0KM{RPrcbQuy$~QugZP^f->5dS_#{ST z!_$&nj(4zJcBt_ZpP1|M6(1ZaIp)oetxA#Ks#kY+*nge;5~^33<5l>cXW!9O-%a-Y zqVXtk#O4IsGXDfrGeLb|CWPPZd3gJ{RZ;$O2}l=zXf}%#Yvc+sM`xQ;G8o5rsjrq$ ztn>sus{FW=KuLO9yqu;jUorfX=cKfvdN+;dS(rK54j62H>xn=`DlU48_nK_9o3w;O zH+N*T@T<|4iWUqy)?dH$_fPjRYV++6IoyXF)L_Vzo5x>_(FP5Eb~*a+u_{n{BG~l; z{8E7)++DYXT$Ym~$s^6#*?Huxce1HsPNno&E-{bc0bXZ?mtWYm`OZI9Ca27@l^8p` zc);Iz%`s`r9Im}WrwBpxcbC}2MZQ%{#{hIrnM2>nFoF>KUU|2OR_6w92lR}G|ouc3py zhnj+}M4cl1eZlP6PY9;9j^7Qyo>wdN{7^{@8zsKt)4txa)%kzec+04`x}{q4uQtqo#5{7?(%icdCq(9dG9yI{;~h`7`toLDw$PtVnPwgRjmeE zCx54^ea|^uDJ~HI{wq(o?E!Ec5_s6|nSI%g6MmRNvn4t|O4YD1A6o3uhmVNj&B#oY z+=AZvv*N3NnDsgx8Oq9wQ-^n#dVNdq`E#DIj@9hE!aIReea2yF8HtSY z@X~u5hipGWJR)|dzsZjm<^1oI9Plrmlf|A6i05P{QFkIl$r|=$0LX?(96gwXPZ>Y5 z(H2&f;Z)Kd-0CI*NawyR2D&eh+NA*v^-TToE zJT^3noM7vI9T~*@6C7H6=I0jHQ#FTdcqu!b{HCH3kxfSenMbiGd2cqPNjK|)mRP!Q%30We#X;B|UfG|W{~}3+*OrA8 z7R(uqSnC$bA9XL`Bz6j`d=`6`cuYKY+C@wUTZ=+&x0nk_8Q@=axNNV zXjwX^%3n?GQbfo70)GE1g*h;5<4zKmynyA=eK|oiy)Kq-C6k+q6tDCdSxqo}ra-cv zl0^Fzs_kiNg=tdrv*Cf6j5sVr8ID|C#?LjdMp?|52o)r3&gnHc6$4_RzN`xUz zfl-CybBVq_QqWILBEzIbA{r4SZYAYD#lnp7`1znnw9*EGfa_T}#08Q%OxXka=rR|! zBIEjV!}2RI<|l4~=};#MU|&Y`ZBRZPzP=;%i@mUHSe(KJXjz~laeK70Cs^UXyeMf% zEcu}2bk8%(jmXUvJ9KDHTCPMUEi^DQf@XT<2BkFp?tI5DU(W>FP<20dVs)-3ZMhN6 z%=l5)-Ej0%gLYc26e1xp;2qg-6>f{6eiNZ-xwsa&6&LJ*Z=LX~zDlQS5k&(YDn(^7 zUMV+>M((-$2PI?hYlpI%M{afl9(hZ3!Z{_c%qO`h+&EHkzh|;pdvzkRiYn1Z(^9Zd zK!#i;X7E|gAf@a8myo4*XcGR}UI|GTxrF}ZZ)4DG&yMaK>HfJ-G7^jS>Z-1D(UO93 zVIpb{t6Yk3!oLz$sMax`Ew)e}hrBFH1%`>}C-B81OvP`zn*|-$7g`tNN53tZqVJOS zbOr54PrKNI~vaiv>_j2jbe$D-7?)5JVf^ z)}77-B2xXMvH{w3!Oapa5DF6PEEzZ^dS++2y`4^Go0@(1uk>hJ6|P~Nt1>3A6ptmF zPIof*G1WZnmC1zS7TdV@Z}Pk&9cDn_P|aaLE>W;P*ZAE z*!dGpQ2SfOL|w`3*5}a8vU=*RgHKil<+aebM_XB)99ZepgyWDkPYWUsaEX1&(ZA6r7mVE5^w}PBL5ZsHw)QGb)t&t?G9+F_gxJqn3BMy z+vOpU=jx^RQTh+In@4t|)!@bVi(g2{IH>nMfwU)*ChFSD!m&PEw}V~Y-X5Jkj$a8W z5>Y?LiG-oya^(Xi-G6VJr+U0ap^Aw*gGTLtA0taxNMF>{=$-wQD~Leps}(w{PAdme zyR((T)Jn8h(buV(Ys!X}OE7(t>D=PFT=wC4CcBFIH5`I0S5(|aXkA~XPpoKi9}e^S zk9`=rPMsz!X8CU8n&00V*il`vzF!aTEt2VqA0*|h6Rd~2IG>Tb3&e$_`>j;6xIpQp zL528#LIOhg5jG;mVB#sqYvwc9u7~$h6Zh>?I#fA1uZkS?Rh%G6$s^3BEC+6Q3#+pB z8n|^xy11ZE%q8U)`EFPLNr2j+!<2FOmuPyLaSEwZj-W%18dmy+M5V8i=i zjfR# zL@v2V;91ua)96UBC@x^`zMoN1gVb~Q`CU7gm1ULW6)%Clco2I((d0x1t%NT2m~q@q zn-+Q?kJ#h`=tR0_L=p!b%N=%hspSb7H=}9tfiS2<(D|j`ISGZhHjtKHljbi6kcJSd z+^70p;OFrf{jc0}7Fmd_*{I~;Ht}pI>gAECeo}w6wQ>&@SfaO@h|VPVq7g>!L8@VEwfbRQW)M z1dNN7UGtEY_YP~!MxGaI6?Tp`%0lB@jWTML{5FS%hDQFw{x6bB`u4PJS#u{oH2CzD z92ns$M=*DhG7@?DYGOEeY#J(UI*(3E&pZ!Vvhlk;{j z@WCc3p@fRsguX;#YC2;-;z%2kfE)g`VMw=E-qSX2=jQW{s? zdB^lCk8L?(2dmLe5}+=WM8W5#!#81k;SIfpMPbI<$oniunI1}kDg!y&n58rw-2#H- z|2|zMMuv=-Sk(%x)Q5Sw+;;0V8Awev5%?q_IETP{+^I@G`qKwGN0$$qrYZ9tsSf}8APtG z02w}Fa5A2?Nw8IFEN@(OkK1I@-eBMuFnW3fC)?pq){DJc%J7Rb_=Kf>%CBN|VZH0N>^zJaKSM`^2@a=X-UL7)GsdsHiyfS{YdW~QNHSq&UYskhIi^Ci~6 zDuXUGc9B~5SLbOW9ams?3w-=_m;ssz71)!|P3j>YDS!Em7xBm4{P!D}wNs=13>#Gm z?37VCINMOKVoL)!kf9ywVDxi89h&6h-o~k@L8*sJJP;B3uB)2bKA|wWX3IR`PhJZj zq-D*VKU-Hp5W&S)rt2Io_W9k!?c2A}XYU%S&G$O=UB>wiSezu;gbU^9-Xn0+MJhY&qR|K+{-NRu(OJ+Tv!GZjwbEb8A|{&?;?Dc*Q3Ae{ zfd!sVT$c!J5J~~+=zs2*8|CJQQ8R_e*J+aLW`88I{Anep3Xx(VH1P*vLzhB+=*Ipk zlMmMf6mpej5$N*w-Mh5Gn6QUsAZ&4G9iU*DN!7nqlu9HJ@Ch1$+S38zMxD<6P9Y)8 zO#oskA>JwUAa|{_c)?T3x5#Mks&S$l-;QA-0{rpzfP3UY9aSWF4BrWyWAQoE>AIdb zEy@v{+GaWf?r7t&>T;@btt#vEJI|g#e3LEv^YS>&=wwKD>fw{*ub;L>_WOjTN&b#2Ufxd-odHiyOdmd?gR@AowJ}-h-jmjTaCVN z4H_2IIxntVGmqLr1bKS$FyaCzL2wR#0I8WqYacYFJHG3k0@!htm0S?dykYvabSI&L zKb)cSvbhsMcjS{+zXRPbB;I& zbRk--sPkR1Sc2{q6ga2W;@05U*N_?=H99*C2?~Y8!O5Ru6TcM3Rt){H(r4N2i1aPT zmQWgS<39jho%#5&(VQUAE)M@>#=+A5fkGoHK?4`Q34%(Ti}o*rLM<_P->;nW&e~fEF++J z>d$h7>sMdz*V6pXsunu18S`-wl`k1%B^8pav3wB2<(fD%QXs`kkVxuNb|&vX(pn^s zkoQDQ0o3nec)#8|LEE5)k6v1r^axUU>WJr_{hI~?B$5)5SHByJ!I%RolSC28h<;RJ z3&Wh{F}*-Ux#5Om_Nrd+A+&>g;r?ozw$J=Z>U;ehFKO_xc^_JR7C1BGdeo>3c)TV` z#=`5oi)InX;1~VHrGql-KXl&lVwaGdY|;}nWo39iFCLe_g}@i>8P6v-%m$Ru6b^cnVc3}==vx}@9X51{=%0JC-e(* z$=yBKZQUa~eMiy;z5bpohFDbwY^$di5A5d*_8+#qQ7qA013}B@#X4;o+6^$KlKk=`;>|uMfTC9>N;pF5w?y9<%7+@Z z2PJ%a--ZZRr!#ub+wh5Emj7X?q50!-m7%aI%oh?&P-$-^IW+=#>+QO5{NTz+20ct` zwLPh?(R^?r$Oi5ZK7221=Ds26b3*s^DV|%)JtOU8^!Mg1YvP;)9gFJMtc4|yzEjRK zE(q1ca$tk@Df2$`BLQcZjie+T6_me739fD6RDuJuao*3+kmCty&{4<2>@I zm6>2iYTuZP%H@JRvo{<^p?FJTPNa3qTxi=Q|LmQ^WT)7qHpNOae}sx?`g=Zv z86p)3T)OAuyjv$Gb@dw`sbnFiX0-Dw)Q$pmXAJ0*fT$CQ_}vk=ELM(s-3m69 z6Ew=c4C5nAJ+2}bOihlXPA}kiS6zD!txgN$kiDQj-&fxeHi;QRs{Z3_Ac_J-&G)UI zPqHlha6zB^eR{`{Jbl8(GWp&C&T0vW+^xQkBy1`?GYT-1DY9(1h$dC?wo1bODVP-- z4ycB{mdyBuz7Y0pOF<9I>*6n=**t7=X+cLb)!l6gY6VPsBnanH0e>5l9E#03OV91% zhh`*To?gHGc>{qhqdvg$>{ML_{*eY>nl{`AS_V$*v4m;r@>XMPay5S=B#o~g?v`Wu zcn5D})}6Z=ZTlq}Sm^t8XL6ow?u6TtrXkn%AIjsu%OWa~UU?~3ltgm>Tq;yFtkBt8 z%KOgTs>tiV#-jp@p`weMF5(+;q$tk_AKjN(y84_W5&Ng`8U`t=9Xx6zY&*%as#pmUcOVN`6;e>B~Hz$RGK%T1ADgwAKXBDr=3OnU=!VIW5qY(TdL`dlE&i>dl|P1Ksr2jK)q zm>=}VQx^PzAW;{mM(+HME82pYDM2<(Svj(#5o@8lWvC@I*Z(=Qe;|baQJ^N_fpm%n z5>aRM6v?sIl=T{a&=GC52v0lZhb8p zprD9M&4l~PsRBt>wBr%qdM(IlftxZJ9q9Rg{?~ugUPZ}rOzHt*!-4K>SIvt+u6}32 z#5!yXTdu`fUyF&HJ1u|u?0QwTm8$Ut!zzyl>*wF4%(Mc?`ZmIM?~{a0wD zfD0QG8^^{jk~Rka2L_1Z2glC`to)&_{k`E@_LaPm4aY9N(pBidurl6^rH3W*m!g@7 z7bvle$kqu^tqR1b{xTE!V#EzDJuGb}Xe^rFk`DbKWuydIq*uteewRC;n!!b&DrK~a z0TM@J3@E3X+0HFwbE5q-fn`-k=fPM9@s~yCMq%{lsp{*ISFr&6Cfq+xh?@_ocEhr) zl%+{yW6kvW)w=Fq7(UMt9bMtxflvvJY46r_T@H?c7{ALPnx@MU*bYfMV! zOU7x2j+~|>0#n9C$SxT-s80K)CHpl>7xQ+v-0bk6Hm|%A3x`RjjvS3203C<%#`!g} zj_54!RQwjSMtZ$#*{6{0AB3w&9?LYVKKzJh*Ha;43)Dd1Th?_@X8G+HmJtQwrvAb` z$VP3ye>sL|XNxmj|CywbanV6>2VJBezpD&mqX?s;Gr1F|q;-5yuxl#b`Sx+ThUBF- zm&tv(A;%_76)OS`m*NKpiHeC1Nh=l1Wz?$jf@)psB3n_|Afm6ixm3v@AfszrIDOtb z>cTjsULVk4oK4xnG&|SVs8a$(sU^Bj4oQgy4o49`Cb$2T&7Yd;c|du=d~@AOJP3MS zcbt668s1y~5PTr~;DhHQE?5oReL!DcN|vd4ZB#ekn7=+!OPrqudiR7d@`{1MKf;-! z-0711mQ?)I*wU(s@){>gBLY_4^(Wq+n%fh5YS>dPhEDzZed#4ztH+h%mF1%k-U zOGhl@o{%usua0@y)x8}S3*VBKx!4JI21>o6dueDT&C9T$ZjyF_Ar5r7N_r9EgyRK1 zOxOxda@VW9px7Grj``i!-qfZfuK)+gMx$k|-|Dn}(V82CI^@S@X2Gj=3)p&GDbagZ zyA&aRNgrpE7W%Q~Ts%^FtyJTNK<9LKP$Cq>IC{_>mlm#^7So!g%19+}y^SVlJ~a6= zjoj`2S3eD`ie2LkT|ySD#s=N+L;|mr#>m5tT!}uUc4j1vi2oG0i6j&d-s^H4j3t70FqXa1SE>Ll|xsN}C%aVI;n)zcktQ zQ~rG*Qy|Lj)VzP4#pi`WjWC$=l$>h074LaIg`GX5EE)%)GGyDq zwWIjjmcHL(@tb&h^H6UjTcbD{<2@xy2(@dz>VWtk75Y3F@fPDfg!a$J2x4T6;hM(P zu0(jomCr=BX%_vv{K;R4#Q@rPG}_U_M=@O(E_Y0t#DrK{+{{=r*cB|d^5urbZKe?C zuLsGiV2dp#i|1>NK9wZd5CMV6gl;Za6-)88rf2sg1oaGb%-z#@UUCNS##1@WeCjX~ zza=uYwPQ09?DYdyeL)LeMaGkST5LD9reaF^6-@Y{@|BYP4qpm?2_YkB*o#jcCkrMf zSM;BP>H&0yN`*vTsczF>l#j+tg`c4KXjnLAf>yT%qVI{mZ|x5H?ya$`%r+Ec@W%@7zE1NC;O=Q7)`l6*o?PlBAv&8qfsll_6Kvn*$E(xL)L{rzRQd2 zZKZsJSMmedTm%QoOLV2>%}s?^grNgxhJ88PV|)+>39SAIL$bczVa`fFr;)H6M+%i| zQ6sp~sJ>AX4CcO~1N&^SFP)6nwWqbwi$HhsBrI8xMSjkf-a(?U$4WgF*f*0$dq7&u zFv6|Ab`u6Ook5UzzxcYKyHJ8XO;5-)N$&o{tbg{>G>Lie^LCk>?OJ$omIq|B<89z7 zSg?PSR*O$#eK3*`k-H18#8aXFD_I5RL3B)IkgqypY8zEeqKgGc#+zS96O*)K+ZbLc zClI`Gk_!*01QGwb?Eo!^NRyH4m)#Yz!I57&+;mBQ-B^${y*29zURHS4ur%=Yw&%Uc ze1QjXJiy5#2?j54)oRYcg-_Js#vWrRfi|K1FLPGFy9ELd-qnmPwOf94O61AS z%;weka%K|ZJ{Yb);Dsd=vV#Ou``!(6nc0tG^u`VYQfwto*JSD)fUyrvN-U6e%gq(N zwi`^hxy7UDNmsAa>#&(Ja1rp({psp^a4!&$8ZuOeh}(E3uUFq%bCSxqcs<6{bgI7C zJdN9<_coKfx$#@f;+Uj9byn9MOV3!tqXwHl`DnDB2mz<+J}K97Tp^>XAz$sdV#^H< zkyArE6#62q1BDX?XtWxD)B%N#x$(O1u(ZJ>KsSLR@JJ)0uf7PB>fBI$9AmDXNLXr} z`FcV96dSOM47q-%d(94t*EE;oWbE79QKJtD%{9eQ{<`__2Q-`5k2i+tS9;P$N|(BZ zH(Bey%5b30A7yf0^3yZE^Ff@H;Qa2_S0=HA65_q+v3G%d_x5_b@9He9VT5Ft0Q=WP zI283y@i#@rEno1iOM)Vueml9 z?)kX$*!e;`Qkyjt5X6+au9{i)yL+FLh*={g-&JbeDA&D-?z-#3v}(!^rv>%LR1HEj z?zuNcC+8~f6t116=C>eXPV_t?YBOz6S}`jgnX<2dRiv13EH*X5pLDWAKwm#1j`=zf zXm~tO5RbU*lo)a&5LH1)Qza}}wa%_UD&Cx8dfl}?96d0RYCP2cIQVwz;U@G$MYCNc_v4(ZtOz^N zDnp_ZwxJkM9D#ufF-!eEs0psS@OnYao&WlMQ(PY={@@rhjLshei|N(Kj6Ln`+-jAt zH;AAL;gvt4tS8j!+j*9Kz}!XK=4NOY`NpdU9Z}q$qGFQte2Uq{PwyyEINeTu5T;{6RGsRfkH-2Vtin5 zwpy+zvN2J2S=+89KvA=|0A;RxxmwWgUa$rR|fi4vHeWYySrwo z7DOPwk-s;3JHXet8V}oOZL!_O`RUr&cUQUf=5%;;7s5ml!F4eRp(^OFa$(I#{NwT= zJ<=KB4I)M59eLg#zmGmayTs$hR3w-`lIy%b|7OqwyP<7GN!?s;w_3CIL^Gttuzu$u z<$@S%A+DtB+||i{&M(W0U)5Z39shEeR)4vn+f(|!V%!xJqISSh)-5E;HUjaHrVU59 zlXaWjSEBek4*JZVT&095M`lP+m@N=E+okP&6o@T*;_P9bh1+L> zeb-KdWbdAO!oe{`DsN3%AG^&7;o06V%V1tVH|OF&fz0X)77D?Yj>P4D@YUZEXKDf@ z&+$M2?%)5r*u9n95=G@}Rc#?Ch)nZ~NuyUY3_j=Qw!~PhyH-x-sVtPDzXJR#Y(Xp?S9ej=MIffI9rzAaF zJvKjm#yerYNRjD}xGUw*+s%dibS3Iz?Q@vVo2k}q%*_i_Ol;aAfcH#clhEfSOh~U2y}Y9k;dP|DrK$z|#lQ>R6=xU! z1f{bAMf8415r|Gq6B!M66+9*(@=f+B)E*IZc@Ept=} z&7eUY1&2XyzHGSQgMcPhrRq22u1y3meDG-9!(#G;GDcchy&aZ!+|HG6p39dU)@qd@ zF^~A&Qo&hkRcdJMt*O<(OMXEL!|=}=V}p8y+MWv-LAfOr+e{jM<|=$ohn#S=L#;Dm zsh4M54k4C9TDR6%F}d@6^bWV%U{hL7(9ozBxV2atu|{R{-mTiB@AcEEt4I{k6Y!Qu zHV~?V)^)y_ZatdP>lL6t6tY-#!DRA46n4LSNkOX+TT5cJR%m&dyQ5HIp=#xrc1+9OhG)%qua`l!L0^)ig}VoxYr&I2X)KZ^^JUTs)MVg;j;lMSvgE=M(UIoGe^xcG4c zgR&mHQYT+Mh!&j(H7Z{)TN{(`fV3M15sJjVA}wDy7|x6E_=TRPnuHU4!$zNRCArVS zhm}b5AnRi{ntPvf0BBVEE~m^4-%;x{@=6>9cz~1!1*EU(Zef6nMQRjJH20ryVW?_^7_AHDB>ypv?+K z6utnZ@OY28T8ONe<(J#=Uc{S>+)q#~iklr5DSOJso5|@A5RGLhlZ}*qg$944`{ufR zYRxeiG=I-tBg?i92Cy=wR}h6q*>x_ZyokzZ#gWOKmz4zzHx=RteE^bur%{4tLOj?H zo=3lFx94g`#Kh!~ESd;PfYz7&xCn+C{%e}Zo$g62ucP>oZS}7!z0b&ppILT~OK9{% zSC?|9RUwi({qWv#nseh%mh5+F*NuLWzxB4SWr7{S(HR{Nzd@ZwA<#^Xf}>BD^@hGU zgAKnP;o_Y#Gn)>)!jPtiWB^ToY_O?9O4>opiimT}<(`A<-TC!PUQwg6D5>O9qvM&n z&hqSHAi6%#43*9GOI+wx?llS6kgJn~czgh|NTTCPZ;9Z$c*R?Bs@4gPDuZ_GCrZ!B zyt-S_&iNuA?+R=M(He>FvWMAx7-xS=%x}ksTy!#boE0%kBcCixoYI}@krPx~$8C!p zL@Z)EuQi68rKgTwH-x4bCndWZo{3{2sF4Y0y_s3v;CfK5e?lZjoSu^6GL4@N>_+!D z!9mfa#G2UZm#?(2o88;`?2ac367RL6a!1b|s6Jo`tN4MhkzJx7j1P@MeSWoKKdy3; zjgt!kSEh`m&}51cupD@;W-}tN!}8p5!|>^(M2VIW zyq5C&=z-da(_FS71h7{9=8x(!`$Yp{)#p5^@(aSy02%&$Xq)!EPWd$<1|+;1b^7GZ z_IPLX`aS0oVWI{Zx8`CM)5YYB4?HcnN?%=N?u#y%kh1fyM^AJ0`>94ueBdg_Er{AaYPWYNGKAf=+Lhnxg-ZT+o@xULz^$|hh0cwV7 zHKA1}3+2##Rk8f+LJU!rFPT?fIMQN5LDh(pY&nf(Cu&qCQr4 z0<6tOK}q}5{@9PkDk2vo2%@cLPh9Thpk!ZyP|lO|tI%Zqtu*=lNN{g5qeR)}!iTi0 zH>m094Egbw0_3KdbF>mqANQ1%VIc`uz&E)`Bz;~pQsRw;Bu=&kOWyeOjhDE{Lc*Gmrrxj*Y7Snna%z!Q%0`=^=j-$;ZgbuvBVUP_2z>jiR z`%K+1b4os?0w4zsKh8YFWL(GpI2WU$GT48ajjLcNW%X+CN3E_0?xWw4}E83?~ct*mmh#~51;bMV^& zkjKla2wnlN^MDY_SU$&)?fYaV)!D{vziT>PO>_gXNK{he?v+05)l{%T0kmhb(#CfU30TB= zIM<`)7sNp4(!s?*d_2DKM>gKE>UN3_sc9qbfuKQt=b|yj7rw`ln~-)N&-1Un0v4bOC<|u@&%7t% zWoMUCzahhM>(T*d!UtThGTBpR+Ws>rkfw`dAl4Lv6E2DQvy#gzc^CeOkn;~Xkzag{ zt+i;&fVokgW!G_XE+!Tx@lyLTkT89Sn~fj`q!O>qluZ^!m#yCI-B) zeE?_pV4cR+qnR4)?g7O5Ro$rOP9pi}WoB*5i9@FfAKiKnx<0$^#(i8Y>GpObsRhW) zCD^c*<*9LtwfFY}LVC*!lNRH{D2Ra(SW&}v5K9p@5{D=ngG!ZTNj`!Ugl^WX=GF7! zpVZ6F2!kq&ybeVwK1fV=LLV0iojK73{D}D5{nT-7ZcR-t7~J_LAodl)k^Dq zuZZ;!A1HfW#HMd8Vp_b`{;slj-^QPVr-a8huJ~XHgBa?yiI7iVNa@?4dIL;V_eM3` zhX|%LZf4$~NmWM?8KY!fcHYwZQ_#xj5gJ6pO4-d^sMY=B3R`?dYoQJ8_J~(eTMz!i z4+Nl&sT8}h?eIiGg-RR@{x-nReh#tR_;<9Y68<7d0cK`g{})|ZSE91u@}}@u=8)~ zUV8;ZXEwjU_eF?^6kH|IHWSf!T7bf+W@8N5Zm4zekv{!ra`;=(6*dOnL+!SXJzbqf z_@8MQZtpl7g8>a@bP|z2yjOvlpcdL9wB-bg2R4*KB0__G)`wzoBnF(8IeYs#FSn`3y=ggoaJvw<4Gf-r7ok4cw zxVEDfkcz;)^~jDwGFd%h>paJEe@^hY2K?x*C;mWpeA57h7Im{ZqD7NK{9Lyi0hj2U z7pxodzYWsAu`&=~40@&U_Ywe}c`p9`UYeV$vvhpMQ}+XU0Q#%ZEEn}20FcP zzzvhp)^w$HFo=!~-&S*^yxN>s35ck1f0(R&1lCp-oOJnHEEr!0%ylpPdToCi)kS;e z-Rc@A}2Slu6*{3{_qHnYaCn|O8?->C>FVehrF*avxQyu7%q&Nnn3yjTuZjI zd7F8j?>V&k?d*u&vfA?Ls7O17+a8_q22pV$@W@u$g@!Pv$FouUjb%Q{!_lazBl_FhqQ zk6}LJ+Cxaw$`?%^4PW1nX5=YlS?T72TKhfXwW?LjveiG?SW{hFY=k$gG;R-ByidwB z3V*Z+4oeceH<^XoDU)Kavx;(0g3yJ3TmD~w<-cAjm41#58xH7)Ahyl>;ap2fodh;fU2IIB!-u--yB3s-v6P zusUAb0xlhT8(N;3hSXelq2v@M?a7VGxfo`yx$O06U#kxTDMrOH!#u#)Pp9LQG{+!9 z>6{%aeNTUi4qX>Vqb$Lzc0Lr;ZuCuvKT{q%rgC)6ZWPk?0T*0M3hRM)98#Ha!ehSV z0Fg>pXNR5n>&=fxpDlTE?<4IvBu1M#g+@EAXe@dSqh1&94%7JU@e)1X^efgNKhdEt zFx(B9MRNTEYc^gl_D^a|m0sMoixKFCdg@srM~y%Z21) zovj?@z+jO4nILgA>043lpSo|(6aUB7|KP-@@EkgI4lCT?)rOk3zT{FQD&Pf^0%(4B z(JSz=s}z}8jTU9@2se%vvNS*w)um;!)pjexE{OvfN|M&Rx6-5PJ_vMGgfdxWj&lXE zD3U{umS9SHbCbI)_%(E&0ufvIly5|SC(dTVAKLsVh;`BV_jy|=_Ule>`5!G`?}&-+ z9_6@wU+M+c^lXUZS$t)&T&fV8%nAW&ZRP~Xh7A3#=4Lxl+{ju{x)LwKcFg6jBq;YD za}JRfbbSf6Ew^!0o80OzweH%r7QtN)J>uw6LWnQykr1Ctt9e!JF4Lz+Ky6cyqn#%% z4vMUij?R5Z+I3{ow}bT=o+j~oJPO$5U)P=wf53ht5EI|oK{NEuW>BoPo#*ceD7c@< zY%kvwJwbJQpC~*|U}vniLiE#-5dRpF{FBGIsQ*r9cf;%kb6u_2aU1R&!nf@HksR&s z`Zn?^_g{epWO7!*&bMCYATv8pT8*$a5M`E$xhEl2#k7qiZZ$P6jm~z|~ zf~;-|Z{DbKn6XmPDY`i!tRT5GadVeL(LcAePC2jZ<9s#IgJWrNvl6j0V7i(25Y`X& z-5RRVH#p2qqkvkHC2MhOYrg4e6`c)Dkf{@!Jy77;4ju!5qcTQ@!i{&{Zncm8|DG}Z zC!C9z!nL^$1@oUf2jO>q6p&!^O=M=Q?u z{M2!;%9T*{LS^o`gVLdYY$5_ASM27q%e|1DEbif?`RJ9AlZupFYxdUC6+mJuP>L;u zeX(){nOEt!nei78W8DqD$s#)bK}&FvO>xL+_za{Jj7UYY0`i;d=L=V4K1c{ zX_%UVU2GD51J=H~FIiK|ydSc$yq?|hglDH9j`Covga6oVAUp6dyy=fy+Y32IUmo%C z@TQFTZkOi-41*b-R zG@j+88?bnY@aCo=F?_KRQe1mh%WZJO^x#d?XVdAcg|Jv=MysuQWobpp1?>yF+?eXc z_d31r_qtcDXOFm8#<)wn5912y(SAQ*b;n2z-&f%W3D{xvQ1M5}0dh$R(|1%qm3l%d zv^~;S%nL#(3tS+Own86f39=>rgk4k+h~aG+RD&me!Poneofh4+cB1O1kGSn5WFXp>YYZi2eU%W`7MxwxZ1rzxk5A#jELWfc!VM zoTiWZ!;X~|PNi0h))tc5bSOtEw+>`CMPf9Hb0r`Qb%WDs({~{jOMZZioe1ADxU8@# z%tw)#6$G6SqMW_hA0Dkj&)!?7HTt=T(^Oeq`DjBG5^SjY9DvF>s zRn3zB%hB{PSO%f&HdYCnqGXrJV*^$?2o~#{Uv^rR&q}%+Wu4b3t%tF;tCr4)O4iyx z`bd_fj&Yv1=h8gnV;SLDy72khr7?6+#-XLBT4}W2+YGP8V&HiKU)Kh#(7HaaF=AYp zgSjSBM)xLrDeEpSg14uqo_ufNaElvtIUP*0hiYEI@^WT6bMVJ+^D3?~B*pWx-K(hh zyg~2!pQ!auDU?$U_1pk?ar@M?Y(zLF^+6=u$P~-I^RtTya>Ui>Z5e4w-?;^Y7Btaq z&1KcJ@mG=TO(ek+3iiMw5MU@)lhrb(M`u$vv>z+f&hrfqCkz@ZL4S7y(XskX-UsLi z4Sm2|BC*7Nv}{SCy_;S16Bd1Y2d;IM?=`J1()iLsn0D1&rGNx+uBX3jK%Mj>{?Ih= z=U}Mdi)J>o^y`m{IBjD<7TRyvf`LD?^k@H%7Qp{~r%BSFo+)9H&PV3D%=D%e{OiM; z8gXgZZ-bP6Havsb>BQ+YUq-U&X)P3&$(B_ICgi{^orylHcEM7>z|DK2yLtLjVk^SV zPgR`xm3K)0mD)1zv7Qmwa9Z<*X=|Gg5Ch- z8J1Kx4o(Fm@MN$a4hUe~S3G*e$m$NrD>tMY>?O^@7JQpn@#kkIjI=d0rP+MB4wN*@ zsjCHxP9;20B&`|L+1U>lH6H98+%-83`k6eJ!gjEXe)-W#GzLrs{0iQCIQ>J$P#&BL z0yO?t<8fQ%=i$ZYS>ljQO&`U0AGPtLiqXGf5{J}CfbAnXX)XMf1We&$F6xFhg{OV0 zNopLR6*_&~)n7x@U7duR_S&LJ*93hY0;CjDMk2T@^WE%tW@H#+v?jNtKaz)& zvpskwVij&8nV`VeO`3|VTd>gjOos6w@!3#~a% z-!URYUu!=C_8Z=xd)_ASiq537Tkx*)DI=vhF+jJs$Iwr7<+7T5KlZ=VKFF?9 z^xzowu&d+iZ-~p#W%aJo=pEk|(lj&-LYNiLSAU;a#h-VXJc-6J8Ae(ymC~eeqTHBc zb)}`oU{Q+B(=Ufq$K2f&q{sQvCqZc=?Gdmzs!>(UJMB&ykbV0$iD`R~b>SCP>PyVw zYi4utwW#ouE3;v+b2`_zHS+Qrw{?b;ic(xo0I~+Cnn!9C@?$cIDpWYcW+*={wDMlRJH1v99wz zQtRETQ=!#@Nv~1eKaXxmv)JyOM~&9uszkE5aWWnc@lnN}!k~Ep0s5wwPnoR+|BrVD z<#$D6$kjaK8(M3xs_q9IQ^&`5itw)lQ0?tZpz2e2y-b9^hXcnNna>iWBbA@b54!-D zeUVo>d%h+b8b z9)h{OC#O=_=Q;#|00>|O97JUxE7+P zIxkl>;#^iEW$Ejeiluw}veQYau6bs*i#1*JeW4Z~DJmnLO_>of=&=MpLN1-DpzCb;9;KtdCnu z`u_UlV{dF?zQ3MTS(;5FX2;k!{@>;o-l@84#n){5NaPO1tOJ zu318^Mf}qmU54o|ytFyaCAD9X@y29B7F5Q?7_X#c;ITL70Dp4pb%F7KGJ1N4{h7({ z4W-h$i`9nE&`87qr&0A>*TpN1&xa*q{Z})O_0Q;PGH=l4MsDXcrUi%BYjKiNPdh6N zoib|Iu2tySGVe_K&Iin-xx~hDP*Wm6N++BjZ zyF+k?Ai>?;-5r7lcXyZI?)Gl3{O6f_XFkC@Yn_$F$~k*?b;+-*tE#)|dO79a3TXbD zUb%{9HZAzmdQAO@`P7TUdE+&F*Q-&#$XIL>4!S{5Y4j6$DV?!?=&6M^x$?fM@vI-; zenJ-n*Yb;GFWf)C6HtQ*%1pJXRVNvp#`P*M>)stTeOWTsPbv)1g5e!eTQpCO#_Mal z&oE%G;e-VW!{p`>?f+6FfIL`GAunU!10+5k%~o*ECm8((sE5=0BtXeLd`Mbgc-(uS z3Z@!Q)r}MBJgHiQ=!ykrvHj|!6t#z`{|V@_nSBtOTT*xDJnsWwr@al@VHoD+B8-E+ z0K@n~b2e9D02=NY7m3fR694|?8FOw^qYvqOJ29=xPfMcYDkuHunyOZt@^!%N1LQXnpM? zt3^N3mtfQvZ&In{^Z%PL-vlC9+tC3+^0YLziFnHT&cg$&a@~_u%Poc4im=%?={^X> z`NC=Bt{?*((qanoT?(#l2g_$lX3_kGNtO@LXn#~#{r7jSBj1g!jNGm9z{8Fw=aJ7( zx{Wk?WkDG+s^8rA-2DJup&i$;-`H>%x$7SC95FXk7W>gr^NJgYMS6Bx`MC2)A_U-B zZ<-rWWA_h*5sd9X9%I=mVE8RpX?hUWG4?_X@6rT|4sp4`0R2r-XU&bKRPC^5Ah;t| zLE>)hX)=^ndJpW{ha;-xJKs`&*_Ykaf2uRtcnRpz>v-r417eaFPcj8^yz_dxbz)`V zd}t^i?COo#FB@YyN}KF1f2r4rP&92QAU6kH_K@&$JvNd^4))$ImRa<+lq*{+BCy`9C z=yGr&2}Oq|B4Oy5ZpLg6WA>p`)?~YF0$tAv@ zwMy--p9*7?-AJ!*FT3#)<(?dnB@YmS<)@(mNU2jC^8L5QxN7lUH{D-)nfI1hL!8>z z?Adp4$>KjiOzY@hwwrbFP}A$%aQM|tXm7pqtFMwBqWV|-S2i;c0v^v^_tHA0O5|bP z!t7upjo#&LM`fm3@(}dW278vk{JnD#y~wdF5q7cnt7T2;zZr^)T3BYku`{PJFTdN> za@vAu)=`Z+3K>(t+XD?N31QVX@O{aed1DrXgouPr1*|&SAI}9y~&9VS}CBec6VLF5FL1-`8xWR$k z8x}9JwV-V|{KCWuVLU!d*a@T|{{_N-*@Z5?4!j>H9n8O`@?YePf9}Yn_JaQ3f6*lh zxMXz2utxs>GL8TElz#P3aQ`b2C?5#loT5U8#83Y_rLc<^tp9o4Kh0VeKNK9j7A|K{ zN9i91|Njh1XEL<*FF^Ru_daw;I^!&%6n?5eIm7?*k~c^x^(!Vqv#gJQDsO{2miCt#ZIECP&q6VP|FELbzs#n6`uX z)F?*GEh(2=p>+W^L`du0h^V)ZfSewF;KzC8U`YluE>v|W@}Cf#NeBvTg#|+nfvP2x z-^3sea^qxZ?;Tr^4?cHpD4>0~h%N!Ywwlx=>PeEa^Fbs<12eiE>I(7IDkeDz07V&x zDts598LsKr0?wiqu6U409HJY&75h{G!MeFAsdmN>!NT_DO#GJeS$rT-(6r#4$%E7= ztjo%;|A=;oX?rhDcyMrR2O9&Xgdp_;X42#8 zCeZbDf>9O7|EGXsQ?piKKVYfYm6ImqpQNZ9xXWlAG%KX-6`3nZ1Chu9D95qqq3OC- zTl2T&PL9+juQMDb!LqSXlW2bXbCy_(s4UvY1HG)a37RqvBH1PX5Xft5{ja_)4kA1y zLM^-xMvQY)zt!!}CX}4p=HTuKi5g%9f zeC9$_)9Uo*rs;_RBwG48W}*+ zDigGdbaZBO>r&aK{()jtLDNSnT=9W?bQ(5fAl%tSRZ`l;sekAPTffB5p<5#+iFH1; znPMkV8-8N^%YzjU;+xj z7p>OiLYdf0*6V4XW_rU$mg5BP+F0=8Is(L> z`^cA79`jB*V0CJPH`qjov?d@{1G zJ@Ej03aw36xYL&1l#PHi=0)}UGv_(wxL?m-@w*_VW0Gf>;n)aSD{C)XVGcM?i%-~! zXfq|O^q7b4)Ak+|JRAp;QVx*vcZB2j1}E*oB$hNrFSiB9>@5#2()V1-rDl#R*XQ!g zE?bNxo}Gz^Se+EKnXf?X;6-+(V&Zk0$s(%1zxHS!^AYPnZ$zx0sX^UN)2DZG#5x;j(h~LZfzV}TGG$3R2hi82WHvHkL1v_^;=H!R)HYRY zP;6I6wk2hpj@xc6sTDI{$_lOPH-l^NFo_aN!d;$;GU@l5Vck8SWHktV;QkP;8ggMQ zq+sCA;+3EwwRCz5RhgkjA74mwl)v!tZrmfsjE812{+=sE?c&r=%)$c1Y38EG#c7Z9 z0?m{`mvoN3%hWwF<&DLrv}RLgrE;muZK&^!Y-YN|n5E zzgcpb&8YImM8f#C^Z0Eq`AN2nDm*Pte>Tc2HluN!%cv+t>>zPQ#urP8-KZ$uWW#;< zE9pbAkeZ)kYqWkmcEz+C0z29<%vj!Kb!y)Kwi8>0Ud zD-~8UrA2P5-2H$)D~If$|8sN!F9C;qvBm6{<4PSEos(<%Y#hJH)Yb#=>=ksOmCg&r zp3D1iMTrEZlY%@2ZQqj@+bYfMeVW8R!*aAUo1ejbi6dE z4}CKd=1hTWs8OtxaDjDRL^DGCWsHg~Tc%OmKk~JBz#CSXBp>j;mT|wWG*S4O&E_t9 z)_dmVg}=g@p4Z8JDug4>N>6S!!SjY0{^CJ*_!t zZ%BGQ8rF4Q1{Y0QxsR%;6J?azv}JH778NrAr>OC_WDM#1=tz1xD5nuj|A}tMZ_K7v zGC(<9Rfi%5y73mwPGQ@$H$glF)cow#oqnr_#&%|lGWEph()*mn^zg#~gr4lCIYt{S zg!SHR%dK^}xVbnaeAZsh$>7};{G-J1`nKsu+IP*iA!m|$sfLprsjVKDZC)$b0R{+Gb!bN@ zn1=nBMkg1TsVm~-o0;;M2*5J-W0V`c3@|CPDTu)HwMN+$TA7Een(?!4eC|rbgO?1>t#XAVcClCP+3MDLWt6&Gw-;?1pit zo(|<5V6t%ilsd5hB;!5$PzZwF&~7LB@=~FrFbU;y!FU5&UtC`{NyoR+gQZv0`votx zkMcA&tl zro$Nf^$q_?)CV2WS+Q5?2P=D>`IO0Neg)SIJJE(O3*ye}(=}|WnN2)dG60j=g9dVM zrL@R}ZOiztbR4;-YE5eExlqm=b?r7yQJ;YcqC*w>^qJ)%ED_aE?}F8*o5@AMkR)3y zu1BRCx);2I_$}jU})eVTzzkz$!h`SZnr^!BpD{hQ!XxIHbb%Z z(=iYx;UN1ExZ8Ir{0X`dC@w&30k!KcuVUxJs$y{b=uWH*rlVn#YYL;$`t_{y3>l(} zH&Zoi9ACbS@snAtrZlg-J!d)7Z6Ky@65v83E`Sz)oFQMbQDDv+TKi^@kXO^>)I3~h zf$ck(Z|DI+n_k^~ z2C)PUD!906!8fKJdp_AEdfX=iUOA7q0;bTr?9rMzYwi|D`b()3$%cVGjY6c~+xYjO z4#um$;$(PYedHL=qbyOlYYGP7is$bh>xbY9Wl|uwasoKn@o<6`gFQ?NU&t;f7WdP9volQfY$imxz9r3dtKDa}nsYKVV?j4;5m^24WgxGHjJZ zAvrM{;qoX^&j4eXQ(I_`l4ulBh`gF#xcMfVzMXo&*J*KvE{~POHplh3x|8Uy)Zq19 zn1CY}5k@+Il#la3G~1Rt*zM(DV(woT+Q-)9Vlazn$_1iu4)9+uCzcu?1_)rW*}Yn+WM7cPUPz( zOYCo^d~pvnufN;u#Mn)zd%phx76^nGW9}qVKs8ZJic7lu5ni#1nx@tV+vKboVW+K_ zilXM^H_HVLc}xxx6%UUpaB6C~X*EX*pYLc3W9GbDlYpS9=98kWJ@oeoUio8og`cl( zDi6k)8OwrJxD2QXdFtp6)!CqE&&19t7FOe0s5Ol446DM@3MEw!yF?leYPNMv8Mg!Z zyZMxpU3Z$)D#x+H;DDPC9CWu~(c+s4q|JJ*6yrS^v3tVu@wA0QFVsxYnt-GTVIi1- z>-q#Zj*2Z&dFh@%ANJR_NoJQ8nK}IGCzQ6R5P)5xkMsv71-v+siDjpYuhpd49~7gm%lbJ8E#^wka`jO; zAXS8`FVYWreyWS}>YcHFKVe}VRI5kd4xqw&@FV47+x>a-#&4~#KPC@Y$UTU#{p==m zqyqPB$wFXBQpo(KiNxqwC#H!qb_}b{{f2LTRmtGA9MQmF&Pd?yE7Hy!$Bx7iX*Shf z3DK~oC6_C7I&Pp$#~@2NWH|ErmjEhFPk`U|gVHYC=~|;{xfL_Q2x?tRs#W-*CH|7L z30IM|V!EyfGk3@$8cgQ9)Pc&vZ}D-^x`T(S;N^ZZrIig){o-bs)$X{#Tf}45s`*Kl zF3;NGH_V{{rGZ*5UO$%WA--{1TZjqy-s{tCQ)TE)#JE#}*(Dk_RWQD(~+mB<0EOyvhF zq`+D^E#(Na46^JTBM-A(tgoHw9v-8gqspC9>vS9+={Pmyk)Io&ckLb>yWQ(Fl&mJp zhf|{fP4S@wcgvx1vn~O8cCoIb^yItjXXMd*4yM_3Z%xoTsk#gkT4npAZ>IW%=x=4U zI7s8o1$PUVib>Fma8%5+D0w6N9V*nCfD+f-g54_gxR&4pNxCD+8NF` zi?;aeeBE=l)1=TxVrnd1rH4`=%zhYyAFA88Jr+jYPQ-&j{!hAMJO}h(0_7SZ*EMMv zl_*y$r|wGjRH{fo4l5&jsa@hwe$toZ_%lIPUR+?as4Sn>7=*9u>F|oI1M8%R4?Y@n zK*)EtG9i%|4Z3+^*(D-0*hNionbB=LF1KfhCSJDi-l4cf%EePP*Wghn=$?3Qm@wu~ zi-(g{${#aQY5GcZMI$=`G_{nGr&_6}VnaA+RJV;{IFw9%;I3(gV13}&55DE)CGR!n z`C=Y48UkDE6a9EiTx6xbG7@$lnR4uL&My~U3HtFvrZHQypV)FE zEDKy0%<8I;u&>y2^7uWy~>6=CeP1yx)$=m?zPGkI}U3x-U+8%SiO9uAu z)*r>@(ldqq9_hvjP7dLFE(3~rx4Vja>if-OmroT^Gt5OC?J+4Ti%nX-!zGuTYa>+5 zxIV|#Ot{hJQzR}66-sWGI1+e(W01FKr24jXMYCaE%Jc8*TO6vXY5e{Ybjtbx^t+}EU>F(@b<^-;fJu$-?7R||DHjPcnErMs{ONETzkgsD{H#KYvh5-4!eYG(;-#)9NMeBWm*&*h%jbKNGXn*-4v^TJC` z2Vt^P0@4Ke7MeQNep}UYsg=1ryL8f()k2y!Hju2bwxU1lNLM~*K6nEt**xO)HR9wK zG0{*-h6a(!&gV}1tjG^FwUD;jME!00`VE4Id*3L9Y*bJ^l`a=-D z8bqSn;iLJkg(5j=H?9lAmk6a^(r+px%Zu~*Xmax<_2%N%2Cw45w2!I=yJmAc(J;Cr z0NTm~1Rm<{;XO<3dVph$nA5OX{8oG|*_1Dq^euhKf>lgrz+>VJRUbtGhY_>Z4pfU5 zu}wO#i0gtRIG*&hRk|f64FF{k9)$ROUuPgfT<>KL#K33fMZfgL9LnF;6>;>LH*sGB zPEU%8kpq|ng#L^a)r?KQeL88z^mWnRqQdCyG!TZ_jpJCeA8Z6mKa|Hce>NwuN#|SQ zMW%U6UyYjsV8I0hOe9$Uh$eKYb~v?0$KU@ETLaMQEqIhI%nPB0Q(S_2y@|pCT2?C$ zpTfyN_>X|@gXH>*vbO)!{!z`dec#_-_N2e1QI+?o=>H1S_JMOh9@XGVq(Hrud;F&K zLDO;uxm~tL0Bi721X-p9vF;n6S$?>Fl?Hz)mr}qh^eI2({s7-v}45_tqsltI2o{^;x1TMKdDjfM7#Z z{WeP}0UXhKW5{ntkc6Q;2rEU%;E<3lsrj6sRH{(j4Jx;R&TJ}4$?!ZgZEhNX zTmHdQJxN{&tC_2*j2EU}@zKq%nNASswa1#)C%_4JfF3eLWKcqpaD#Og60>A~;3x#t zvfJrTH6M(|urgO;Q^2v#;nmW@MQ>zI2I^k82s%zay7I@pIjV0@>uKnWBs0sBUL8>o z^m=`u;)ckU1%?X=4z5LQz)q!trc4w9T$V+Y#@CledXI;1)HXxhR!iHP&?vua0pX`OR59yhqzC;V)#$j>J!To z>78QT#%?yatsN=Lr(Ty1y__rw95{ZSV3YAd2hP0R@0|})RtQ=aMc{H_)>^=BY$Y+LUJ+G3IDsoGYB6PLg zv0mjY`*qB9y9`z^weSG>6#w=O3Y0Tw!&HIH+(V5SJuD`~|1X(7BmRw#gKVbLd{ z0K#+xlqNtkj!pm=kQSQGW+`NTS@EAWA3*u1&)?;rzgypdHKa9c#RCs)!_e7a-8=Ln zU6ld4=y5pq^DoWP(*hnye1$>;5-rpK9SX{{Fu*lWwFJQ5SiDckpk*KEOqQVG+yAc2 z&2)NGYaHVfpqk+~7RaYYML*gT0F7SH@m?oblh(D~*mxBM4DA6vwj$gL4EIsnpX#q9 zAU+2;r_WMqDz^LM(pvB37!Mu}gg!ZdB#CIwn?FcxHhZ9`R4U)yuDV$dy8$$z1aSev zxUtW8#P+z#w5e4lCNj>GgZBrppuG4mS8`ebniwS59m~1|lh-;L}p~ ziX}=}_4TNl&5v069mP|_I;@^zF^c`4z zTHKan6-`H`W$7!Zisv7|o5_m@L#MOU_o-tAnCl7+aD=-dhJg{*a00yGw<7R_TU^SQ8~~hD0Q%_4v5lzpz&8m=3Wmx4#ymMffR1SeVi~>owSI0g!A)rm1+aAem!~}tmpu3z&7bgfM13xvV|<^zo8F0HS%8ta^Pqt78X5O z0)f6>7{n5!l8K1bM6mM#xf^IGK`K5yJ%UWk%o|~>t3mS&wehx!^^J{H$a(D>p~y`>-YJY))b zd#-_M9#{Z8R z8y_3A#{1orZmtl`!8HJiF}}I|HA6DQPC%BzHaql(D8C7Ou&f$E4nCDl{R%$WOw+L3arz|$bfNpNNa=uR_ytvhV9|1U2SMO8TB68?}tJD6b}9YxI)JbDbGC= zBrLl3oVFK^esAcHW`=5B;B6LA^T&=yi*+zCs8k(OGCW~`4$On`iaEP;vPA_5rtUN(v7ulLVFcs_06>92SA052A*JS=sUnP-*|GFN|#53dAvD+=`<7@52zhZh)CtR-nuQ_V8HparhhTp;FOQwFm~e9PhYiVD>v{s}rhtBC!hqedO|teH&+fNd&6SpB@7)mp)G zdb_j@9~&Y%CI(Q@zF^(->^Ed7EK@@b{Yzetpg=F*zyX7UXB_+g_W&^$#aWoOyWgG* z7sYT7xef{-1K|kiVRm4d*Rf~gsG?^5VF~i$sMqqM>IyAb07!a5wSwx?-`0+@?t%*x zcKy}_q}{TqsTu4-0;7;YqZ9(64nN~S%xr}P3A!gfXX*q$!y`;GWrtmTEt`M0v`?Ty zBJn3r-WdU5zCt*yn3G+M+PRq6+ov9tn(SL!i&AS$>L8-k!}B^6!=|JT zV}Pc&wS}P1FYXLBm-<90{!LY~_P3E5x3FGQYaVd#Q1(SOa4+TlLch4gh385V!rQ5S zuc|EK5$9YZBSOMFuJ*{$qq3YOHk<7koaV4n(c`Sry1j;+p$~7@y!?&ve-ZUYUr!|O zp1@aTYhA4eNr7>OMs@CB<<-FV32S~(7+Ij6duN9&!IgE(G5i>gYhFAgl2VscABJw_ zPup{bl<6<*pL2$zRB6g&9p2kG9T-^5#mRcOAB|9OvJrM$o4OfUl4;(03~Oy9=xrEH z$I}F~Yq)(qdT}s2>(n}pKvZFZ65x16n>4#wd^xUdW^e!ZX@fUPgYnk*UpCl>MIb@Mjt54)xb3)>Z;)LPrA+=d8;%Bp-EH%F@b&;cQ;KhXZ+co)& zJ@5N_dzU>J{c3Lu9lb)AJ>N_Q09ry5(!6NpLh-b1VKlRLxM65hY_79i++)pgNB)cF z7frGBrT>{@g0;K_e)YzKJWQyZ?TG&VC(52}=agg1T>~(-_$mz*lX#`N6pNv>+t-qf z!<8B3!)(PivU4W2m1n}>wJ_m5c6EuV0k6D-&cTwTqf+wJgrAIr4}!8!k)x zcHLn?-@ zSa}DMeiPTA^=E&5Y;%4+LTRBF@gqe>+svN&kock*fCpl1uwXGwqEB-9SW)pQh8*4q!zA`C zWsn34b#9B!MfFJ}ZOeA&2!svGuMRSJxY~hb4ytIbBRM?;itxPy3zQY$#R+EerY^w7 zw6fiUbK<67>a4LwJr~*6S$u)>7<>12Kcgr7xfppO)2l8vt*Yr={)I)I?kZzUY`lM! zexIHN5x52M6tnRlT-C5Iy#S{WeXEFCQ|<8Kp)zR6{iN^WoO3g_i1hd0yIrz&w~{M9 zq;$q*qz8vCPVX&dFl$_knav|xYuoyLf`Plej+r!;1n8G;UL4iNXxoo1eNqXYb@y36 zoDXo4-^`O5YrU6fg-SBDINzk-_yWlmxNNg3f+)Tev*6Z_tQ>io%81m&0#aj)3fc`2{B)||AH+#M@WL8XR z^CBOyj}JRAXkxlg=@%Kd&rz%B7wK|=bHF9q)_mE3Y}TN$$_iRTV!XA|y+?1b>8=p> zN(#i>A4mu{DwR#k;#pi#qbEglPOdbF)aH|Ev5Wm(GQG8QPN)gb@Lp$v_~yJmp&^_G zlq{6La^!QTY$DU&1vCXo{Eny?qz`%1#3s-K;qwb?l=XpE4%nC-X(kOepJpIW6P}jc7hu< z3suHmeDeeTt9Bz_a2`84`|LLTRGUKbT_Aca3wj{8(4N5s{mdXtV7}-SmC7z1->WkB z&JJnK`ZqYckq5%_WZ+gh`<(7{CCHNx0GS_Y=!mE>c%Ej%$eM}KOa(*;?I#=Pz6~}^ zc+DI15DV?CHRi49P|Kz;1I>v!H{cC4?5Ts>IWb?)>dr-@>=oWp+7?g-a0h|E?5%lU zeFT>0jJin%x)mQF!>cx;n8*<3+)uvzp0l&$@OT<8q;xJYyB^lQJ;YZ#eXMyHZwash z7ly?tTvFoZppdKYyq$M{Ni^5B;8FeYv-uROWbIeJ3u9GnPhzN3xdoV!QAXElNDCda zAmJ#o+pO+%>ZesHUk@@`)sTgY#n#u)s4^E4q8Cpmj%!N({##)03CwFW%-lQ{h$z&a z!(%NSe^G$IqcZunkxH2eFHGXWlP47&Z zG&T^w5u8@W>zmY zW=}1V>g}0gw}#nE?Dj^l-YU9tc#KO2EJLwPb?!SKr9_*S)RIV@kLYX>z)60GmdgD2 znQ45GelPd&`S{YQ%;Sa~dz~z3?b(^r{PgFG=Z#U?+Cv%e;Ymibd4EPLg~I^S{1J~z zEo1{1fkxZiL+#PGmHpB$yHpdpW=uThlLByg%9-)#=iwOc)b~JFlPi3kW*>6Zq=>p9 z>%Lc@2Xw|y7UZ9d)-j$Zo_8XU>o~syZX2vOe>%CHzhB!qET70fa*zl=y8KLoFF8lU zx`KpFfHpGLtJa63Qgz0%jD4Z)xxyU&i39{m6w2q9lP98=W;4?3!<2T`b^d$0lC!Nl zP3w|SDKDtk9SL%BeF`JHn0`mby-y*j4WO*XJDWGlk{0BuWEZPfskIWhjv<%sR4I`J zQKvGro(d(4YQYQ^*PvY5L-~@#l)+bz&uJz~8FnOmfyko^QnJeRDTBX)Pg6UK z`jiiCcS@gsTTv}wYNg{fAKPTl2AKRrX)HYvc{uM6KtSaXGtP1w$u}#v}7>agnA!+Td804 zgf|194}XG5RJeg*|RD8_RgU)))4;79J*C61j5sAU0>$IDBGXw$OOx`h$%K-bb3qg zt9!P)`@P@cQmtZy~jZM3;s}dqr5}ai<;dLnEw!u_rqVrBI z;C}1x@S63N0oQ2N@GKyt+dxrKvUutwxpuG{II1y7ItyI}9Yz`2tUNs@hQRwSAIpIk z#|MW1^p4vXTSv8>Frc6x8T;ik#5U&w)!upr^38Z>1MJ@EJ*FiMRvW`r{gvDE+4Bp| zoiLBZ`~XxiVOP8p-pS+!5S;MBia#k91=G7pD}_uNI<%8X$VCyo|`*s`pP+eJsy-f8tr=8 z$4t?QDEPMffb2-{BNhUzGEF0_&4x0RlXwbuv%Ccso>AEwieR57phdC(>FQjkU@vF=j#3OhZli|66Ll2 zF6-g;>n=C2U>NzV^VKu!?J2b16;BE8iM0ufS}RP6Y+VqQtd^s>XfsQh+X;fzFXbqkX1gcKi|@XlsaOZh8UjU8Jt3QdkM8w^!Kz{OF~D~rZD7640cFu z$~@*@NiL9yw*6u_%2ziEwmw$3ttnL#&xj%fDw!abJA^19wudGA0{3=0mfzhHR*Prq zXw_RhaL_~qnAO7qEq)~JLH(T!^dE74p*KHi>1aHJus%!>pXfQ{;23nKH9UDUY z7)JaBsji%B9+$q$T~oRK;fq|6RWNb7BNoS~j(wCO6#e^>Yz_@_Nl!8`=Af6`*^+{6 zQk%{w>k%+?K${`Y7iS49L)32a^{+o^n+xYEY^16jWaCc0+YzDK_;v*yZM|U(Cjq&hQ(N znaS4kBKQoCl8kxYdj?+$PdWE|eD~1Xzrx$3q>Q%TB;>r^fe@8Y2aR=0Nt@_^T#a%e zwYwsRPw$L_NC&asV(=sR-sKe`VAu{mE64al+D`$b(auk6o1&D%Vcx@dqV1*|8*w1M z_e~6;w?au+xI~FO#YLQx29-AU&XM>jXUnNcIM`JE5gXJEqw##-pXU8bjijs->m#c1}j_V0e$-D_xX@x9T0MQxAfF3B+_p@Ach zDf-<1D3r9(VsWF|4tJw!_g0qZ*p7a=y%3QUy_|7}dPfUv22U!##6*b7Oj;*ZE2yo1Ve>(u znL7C*@5q`#9rKzG;>}LLEyJSB$g0A)V&lPkW3#~r3UqbaBGw6CIZ5CW86GpNz8~<8}0|2&giN+x8hZvoV_po0xF?44B|^B z1}iUzgp1v;Lm=fjFWGnLBlAF=eTG;9Uv(ndB8E~<;9nX%)q2&bZVh$k1PhCDQGJ%R z#O-~*Su(3()b9r_y55U_)`>9Ok(_r4idXY1-{w{6^>lpd)vcAQzO@gFgaxS?W8~QdkGOUoyKPe`yr+A5 zyghTo)h;0Vi1zpHBWEy#;Lz^)7Ki zJIb^CHf?QaULop|W~A4KHwjJMr-K5_Uh;skFR%KcL&E`4%>@xR<-ktM28mKii6?Ib z8#5(931^uqSWSL4gq3?FmfOBuQe%Y%S!K8QhkMlJN$MqtJFnuwFQxN<+bmy zY`@jA$ptKn*eND%{Q7jN5@0R($-ihzdjI4EBphK8cDy+4j#ct9LMU&iG zN$?blwQ@-Q@N(`wgtYEbuF}IL^{xzLYF-2cL9j$at=)C=FoN8!8UjDn8^tE|2a)Z+-{3CfwkB#N(v8s1FYgDU=0dDz!UpbRyTg zPlermKz!L{>3DV#O4x(wcwTM|F)~8oU+`yJFK9DEeu`bN-e%Ip{ldt$NF%_L-%oZ4 zfkhOIqMv%|kEm-Zu?c@#Px6DXZ#tv8ZyS(&Q(A9u6i_2CT`MzP&*|=?5Xq3*&J9Xu zWfaWr>Z?%&D~<3g)zgr!Ikck+!J>^AnauX+wm}86?+Er?{YS3DFMFuA(kf3}uk>t;gOv zCK6p=Fw99ml>4;a^XCS*e(~??^GsxWaIadu`Jb1<`ibJ%t(bQ><#Cw|IWP6g>6rU+ zlT%5PR*7H`hUv|OoA05*rAFvY4J?vB;iT*ar5X=9*X(qQIbZ+yt*dC6zgERnWAW7x zOQc;zX{VPJ*Gu%08S@8$bkvxTbPwxMFzAk?@1n4#ot9b!8OU{^PmL}oB; zY_s)EX6FeXAqp@ETw%Zc#Nhha0Dox>!;#NQn_p|mk`n1b_PHT9M`z(Pc@?<=+M$ub zGuu^OS(e*w0&>Btk}m60ejC(++DchK!dBopijPbEsb52LgFtX&8 z%~eAsNwLERB8M}ZSKdV6^#oksX5qKG#Y!!AYD2JMz^tiNvGHKTx+4Uz5Wr}cQatKw z`e3$w3=L!KTN#YY`*6tOrgfiW$AxYY+$a~+M2v=ASjy#{mPd@37VHT7au61@IN4gI zS7ouO%Pwr5fU&YzKiK~8EwsePrrz3B0d5=9ok{N*I=9-=5wNi4(Wjl_osu>T)xD|; z-H5LvlMmMVX?kj0%W=-g)#bYxfhkybtF?|rj<{|Nq46=g{$-vG?Ni8dD=AJC^=m2X zKhL)kc)sKrWLopbi?O_i>4RLaEs9d*59ifLsv;2{EHtDet>#KmCd!Frz#qdvUA zb9%|rchev@z)hsgJ+1se83CHR{*RvyI0+p|@ujv=7C40guKX{)=eRkRg9fY0wRXW@ zN6UL&l)YeQM(e*l4@Pb}fbXSz$}6x4x`~mGSZFb?98zgPRWF!eC8$}lpkT@PRV-Gb zmap7k<+$pWpcj=gSU*5w5Wm-AA*bMhQTX*HO=Tx?q@SNd+Mbc29dI7}+S_DmxXX(u z``qi`f>yql@|vH1@U-WXOV#7+5my@$WZXIRKao0_q?jrmer%_2)x*A-+eB? zs&&_(RBV7N+Qxbu)?dTPTZ9GrO8$>?T&@J~k+QkyOR1zt4w?32Da^Gf*yvIeY>hi0lPzWc)Zxy`N?`Tgwa)CIsvu&=$WNw@qZP%{7Q zQu=qhQY!^eec8o9hG+R=>XMU3`{8%gDGqT|#d_tIP^TR8!aepB4~^c(tHeJPlv?hR7#v(WlnZ@BuD0=K zIG>0}N%8WBAM-5LZfA5SS);)&T1G5E!81f=U<50}L0gSo?BQum6REOo*GoDq>@wCz z`IU7gwSs=Ow0FHZ<>mjpST_ITLz$M^GJQG`(4CoHLYVYVTm09*iRs|c%5U?jDIQh2 zd}eoKqfc?5o_Y>fJH>O~q+9&SvUB%E?tYn0~oirtt z0}G=#MYy3KbBlXdT(uQ{y@i$nhGnmq;?l5G|McGBa;1Vkdbv#2R=N${ZcFRNYp+>J zR9O%3OUF}9yz1PME*zMiQ~tZNqizICym~=&E1_Tf&o0Wi^WlXeIucsWxO{k2ZS2$-`2pl-nsw!G*&?omDMt{7JV%f08~ytkZmzQ>%n6{yOblL zbTv#qO%5)#;g%}>X`8=LIKgISlXgx>(HRk;vV^5M7P{GX?MC2tE&a|F{B6$lH2u*^ zJs7FQ=id%?ou0_mL@X3F7L`T`GL>CbDZriZ6jEYud(63JDAmU4qb~`a7~FF0zm(5b zM}!XIMCY5K7W%le*=ysR_wRDt6#hT<-hwx-Zdnsf9LJ71W@ct)c49kbW@cvQn3>&X zh}~vprkI(TnVH$sdC$3bzR`Dbf52#@t*ut8du#1lwF;lA>VWCI3#PWsSDBx>ug~nP z$AN?wu}>6G`_!(U!*}jTqn%ChbMtp*1+MTz*(HMW*1?*TttIc5#b{}MeKMl*(-xpg zkjeknF8}9rTXw3CtCq|k0|ggz#nocqOU}o^#jLC&>nY=Ry*VJq1xs}(sJIJY%F|(N zgI#){*OPW13oP)J*p8srRIgGpY)>3TkI&Zv136n{)wM%C&XRo_e5OO^$632qQLOZ4 zlRA8ZS#Wdnq^lX#ODkp+d!bl=lDu0N|37WbLiZZlyeoG(+CNR+hs&Dz1V zyU8>KhQCQzI_HgvN3@>kd|(}@=y>mRW;)8d`3qE4-g*k6#45u@2B@|1hlL$ah3tNH9PLBzrDiD z>EmL-aco!2)b?$P;=9#NnB9uE0*bdz&1R4GUlv4#Tyk_M&C(3-sjcEUTcierjPl2O zp{H%tjSgIMp&h6W55<^rZzi+2$Ie zVEi3rWf#5?zF2*YOm2Y+bAY+rtW;=y8oP=3ZrK)Y9kPqgMgOV$bSILGJ> zVUvQKOM7d9*k&TH(l4(`mr-$h(-ti8bIGHvR5EwEg+N53x!sd&o}pr_3Fa#tap)~d zc;DS5F_T$hXgI=l)eR%(mwS!G)n8wNEl0`3DN7S*h~$TekIq407i=gER>2z%FDz5afS z`=!!W+K2~Y8Q$Nwm&2FkTa+QDYnvcGFeSx~)qT*I6k3|Tm6(r!7$zm9xoKx3<=cQp z4RxG~*Yb~^IETg#Y+f4#L3tvBJpO|Z^W8&k3 zS5_=9%TqUg$nO6r2C=x&0s5YxWJ;xO7)lXD-zwW+&oL|RtMR}!#_MCMomV@op^*uh zFMWkK=6z80(D+TQS zN7nfFmz}I+F}WTA7HZVT#@hF~A5rV)uRd1N{0E8oZyD+X!h zh!*}gA^XqN@)9ZCBTB31vF-G!BUEa>(safjf5J3>yQb4R6BL%O-xy3L5crktC3Zy-%tMQxPdW4NGyoCmN0v35z6TQW1w4ZO0yj}CQ%q(` z+^k3W#k=ym0P9ATSe(y)f6hL%}{@H=-!HlNwlNNoh{Gb08~Rfz}w6vG>a=j{FS)6n&1a`dYTL672i)qK^^pG6IBDt z79;Y}Zc>99crLcyuX_u-lwD_h?mNt(HIOHB>e6y@qh7w?z8z5hRcevMK!YhR<@;@+ zjCC)&p-9x@)y_&My`7q6>C&H`pzDqgOL@F=M& zqlZiWcBV-f;KAkt^U4~ov#M-I-5WLO8rjx@O|W-sK&dwdZMX`oY-%QDgf& zRp<|%{tOpXCg&>spLq-Pj>rVIS`=13t81k!Rp8JCd@?uYO)WT~iuGUXT)YLnSp=ip0znz?zv%1Ss7|L}HdsB!^Z~pf;@ug& z1NeD!+7(Tlid;<0KtxCn?#sEn;eq57f49fS0i`oFJNU4Q`EVPA-{kB}#$#xttDc6U zHGW!e=6HAauNWt%zk4N&+c5`Hco9BdJIopa14zAmvT{4Xv*I-yqM>;tP)o*6eERoO@Z)?6p z8ph)~aF{$?2Z!oP{b_3)?qzIoo(Kc^4u*!)urahu-sBSd<-5|;YHyo!HPEaPI5SMO zWa<$joJ5-M-rDd}%9dT($YoG3jZyoUJ^RXWWVaNseB3pw#<l+;xKk|g3?0T*Q9Kp%Jtkt$d9C`iK6!@VV z0p3@fgJp2H9l`fP%`!)tb_YfWA8OB{LL>dE<@%{+>BeL))wj_m8X+? zn{%>q+?DuR?5s+=?*F=c-hKUk5Kusw-r=S*i35>y|4WS$|CN4wBL8+Fk4k3Na`m~A z3gLh;v(xj|o#W)ZZ#oU8id@@0bl(oYCox+eHe<>6Wb?W(JOO`=b1 zX;>kx7LVPzX~i{vB{B(~CjbLx+K%^A#xg*z(JkuaJK@<+y9YwYjtdrO_F_J>Tds2l z+TW##A9?DTR5}R9%MY7usOQsNPilq%*E{~0gQ7-~|D_{+OC&p&-C#@5tp#QEse?^+ z1YY1oD(tnAd7)Mgz`>|DpMv80VgM%P#o-Nc42PB&vH$2Of{!1E#Gy7Mt27Ynb`y$K z`Y>j6qedGW2yVL9<^%VBeB)Ulht^o`ZbAzus{V;jf1*VVt)YZzk#FC%3$}7rNHjl< z0+)7wuB|Hve&=God#>4`O}|FyHfHS?m$$MWe4a(y9|cgpEts)b|ArP=_Gwp1r!!2g zvneU__wQ!gj0*7Ce$aG}s_7z1O`U(CG?G9ljpX2P4*3F$DBk}@Xvncg?_;Vqo>p(} zCD6{Ycs<5K@H}5SEVO%N^(`F6C3ESXa7~0%@8b#X#HLXOX9{SVi=38Qr&g<{7sPvk272dT1nihq zqv6tG_tFz8U-_yXIrmKYe_lf002|uiY}%>MLV8H$uMW3%+1=*7lIR0b>Wy*fz*oo) zg*ta;BT3^s;e!Y_!)A zYmAhb`ZmiAO{eHV^ybTQx-j;|4tu-buGnA>vMUBu+|%>8?z+kyut}5*%mHmv;`t3N zO(jYpH2zmQrOR+~POyz(&jseUpq3TAFSQc|Bc_-!$M!ZPHCWsJ zM*=*TrMjUy=8YQeLT*nHd#OyP?Iq2Mh#g`-iYbp_}s~HL_J51Vd3RQ`v$WSXEz=PBsQ#4Pfw*rT;;*O5V($-{E z!Y2=$v8^9j8Up*NPa~Yv3xDX zy~VFhL8ys2ODw#wuqH~Sc&AHq2TKWT!d^>Z`+g&mm)aYm^G%hkMTOQ{`29p`bOvj~ zuwII3@eSHuoP{VVaUs>nTC~Ac*{W>F!?c~v!EC@>4nxA@^B(s30+YemTRcgu%=XB<@VEpZ#no!c)ObZI40nP$p1`p(#=cX+ooaa{foK z#c9vB8nJ==4;@%!~WJPvV@{Esh5GgE=$ z7lo##-IwHSK!REgP2*T(Y^6>lDYL3{w@c8pV7LI7@>#U-P1&_79c|5twZS@yTAri@ zmR2S!s0d9o?wg$`GCQMcr&lCHD|f7n{1%IUr5!^A=By~>&RcxKu7P*B-mcR875QLy z?-;93sLM5H%>1+ENh=D48P@@SPFtU(fbvK6d8DYsV1qqV#{8~VJ)G5cXlx_3dN6P>l9_6K1ora0hpL z&E{E+>t+vBddzsr-CID4Cf*c4=CP))a)w85OyPMiL1_u1(v5yYQ32&wNl)xX@Ot=M z(hcd0045K=E6!)TwX!w*@Ni$#yPS0^H1^MENs!sk$3q?YJ>cT}Z*s%p+>|<=2S=in zFUg~2)%yfGp$jWcVw^A#;~kFTEq9sYNBk-+Wmyj%7h&?K!48-w8lP+5vOcAb>!?V< z;A!A0Cmbz;eSQc}mEYB`r&&;puHyfs;ENTl(YlXDH>oJmQk-@o0b-Ev0_-@}D&K!9JwkKbz z#iFbfZq?WNkN1lRCLO!nOGml6zv4TZ$f!*6m;gO8AXi+Y8`LZCy{Vr7l<%D(i_zsV zv6i%PVL4s*tMkxF-gEz{N#k^M+My3=AT-~{_OV{~woTm(DAudKbNpZ^PHokubt*es zdB5gm;yHsRoQGLe{D=+3mWk1GZ=q-*@$Uraz4NPji%T?6mOFT1xh+4+WyiYps!fYF zHOF!K=yPK0&(9@F&sYfcDAxTjr($E9@Xp)maR!5b(soc`d6)brinT2{J8ymH%2bH@ z{;#LzFDVl%2+@z2$&dZ^B~jNa5KqrH(4174r~N0HYGiw1J^=%RTQLSa^;mYlF#eM6X4>CN%ieM<(n=P!YOw#yG@zsrfH6(9lQT_4?gxJ#c z1GlHy!$Tg48vNtN5$uA`T&^@ZFlAmpPRvh66w7P*9xiquI-LUPe8t^f%Z-ALGdcXr z>)BilDcyoCDVhx2jsm-@B!BLfd*gi4!u5Tsb-LMHh}xY;>dytR<^k%_*IlJPZvO;Q z_;MPqyP4J(m8-w(9)Vvz`}m$0W9co19A$z}dZ_!9&-AVYj9hh@re<{efbt%&Vw06zYebY|c1D}fJEx?GzdmxlnE@p=6gyDamWwGH- z69ox)IWUJaZvlsbowqY;sInW6F;+@!TVVLm>-2*aL3l$Tx`!+T#&9S*T)gNR?$xJxRU`&k{hhL>LxvB*7c<+ zNw%a|C^vf<;iUs=x8yiWm<&2Byvp;@9%Z!Zpo6T)uIaO|?-R&}Y|>NakTlCXtpFwE zDuke*_xt84{(v0WWQ$%(WXKis;ogq;S$d6k?;w$2Wm_M1WdO|_>6J&(kn1z3d7(o- z)4*cI`3>M>m9*S644#mRLEhAxZsp+s-bw=q&VPp|Y?F?E)u633JD+E}uw%0~H(N@T z<11y6)R$18gCR03Hj5(K!98c9n=hTY%%IJiXTC^%VT-d{-cX>WP+DHCYI4@7$+2W?n#p?(6J`~gf@{fM6tZAR!G;LSb?5o}p6HWin7*KPmWP=+)!CYR0 zLH4*1kiYK>g*c!U6g>NgcS-uLzT&3AKgL?r@h(2-6bbdZnfZPwMq2$TtDSrd8kaP2 zojc@gta2(B^)zGk0AD&6OBqSRwhJ2aoWY%Vm%QNDgvD20h2^QWHB8wLr0a2}e6X<} zpfshxylb=E(YVE@VgT&uFiAANO^%D3s(xKBJchXd-Se6P5kT3Ce+Gx$gP0}L_2A$) z?@b{ZsNI@M27JzHg;p~XhkW*WbW{FjY%?>ntHEwMCyx1A7&qVJy|xXL{>i&95qCiD zm{>p{$h*ru6(QQ@v3aif5ub7N$&R^x)6$ZaVa1&zPG_ke_)+*GrMo3I7I}|Ia&J#} zdUNgvZjSGaZNy&xSEv;V@aoHh?CB*#t}z{?{m>^yy+@I1iNyeMXhTvTQ&== zxp%J?GTLL2jv}p+hk+=7@{L6+vx(Vdxx{9(m3O1xwV6>fcy2hvK44g31q4Z)z9+b> zpX{d_DTz49SZ1+q70nLCNPXU(D>!$oLz)L%Xq|Cu<@M0AbItG!==U zOp8%to9O~ua5eiv{KeCwHz~Z?bE-+NuKM?)YL;(GUJ4nf^;`IR5#aUkIjpX^h`c%Q z4vS_?g|-%Q!i5lP`O++SD(JxJS(e?q)cS%5ZNB&N&a!6wr;3ds7}IsWTfG(wwsJp3 z8$F(o$eK{2^O?WK;d|+VWcV}dV~JgW5Hi^pv&^9B=Lv>uwA*uWnfhvN&4dV`#V(tCX2ZK8Yk`YTNDZY<21F}ou1 zYG0DJ3Q)AWy8ji#j+7XfbX#YD`lJOdDr8Be)EbSDmuF8hU1CE~i;&glnd|20?<)sA2P% zq5Yk`WhNQP`B!Zz4gi2kK|Pzx(nRhN=2@g%voe~DI8Ci~r66r`u*^#B(zmn7?oAg5 zq`a9u?Ml?T^U@zj&62k)=Acqz`T@p{<$JS8ArGkmwagFkDB_+9Y*Eo0d^+VoQt2`8 zdOJOiZP*$Nn0xw>T?$^P5v`_a5VXv<64ge^*=Tc&*LZ8qh4V>D^iHp>@oB{)huL@fP!P`E*{?9rHVY&>!j>1aj;>+Z21>jh2@PfUO&I zoT^^pxCZQTv6SG8;sh5^b5Gh-(8LVokI7CZ-g)&5T}4zAkHdwfSQ03%CQL+~IP)g( z1jCKJ>WemegeOvWDxZxwe3X81UbrX<5PL`IS63=EMHM3yBGM1dnUi_6g1@(Gj+i!3 zT$p%mSJ) zO_bit*(UUtvR&QQVW6U67E|TZV~0k0rSFP-a|mWMtjfhd8hb<08uYRQDkx^c%Hr2%2#6n%! zoVU}@A&L}RCy5z%_NV0XvXLXlrHIKqema}lqs=~gkD;?Yh3f+|#iT~=$UeN=f^qr% z_V}QYND=KldzWBRzuTFdT47HGo01%1%iW0SIhSsA?{6wm3j*?h5rxeDBFh(gBsb%um~-A`wfSmhKNuFa%~*!Cq=8f-2T`FzFd#0v z{Gz%;!a*JJ6a)cmZ|589w{kRP=0zh$wnP8Q3=fmUYpbr3KVVqvl)ySQZOqc(*rSPH zqUWvg+=5|{Az&!cCSvE}LYq)8&4^W9&%Q(_;YB{cm`eYRgaPi0>QfQMtM60~hFXvQ z2ov68UvUTKdryL1&x{RNb=;z58*kOsjOlJD+Tek?Sx7T-1O^T%|3|dUFW8uunC*OR zSq<@wWnPVxI@)k4vr30-L~GDSt?ROSa%<`+$H=j~6;)Ht0-jom;j^lFp2d-Sbf+q6 zj}Dbocqag;P4GGNRcSHADD0Q|Cx<%xf!dj+5%UEK_A+Iv&;++prK4_|@XMwh?6^g_ z`wnozN<_@>&iD+-K&m*#r8k$XRK^ z6O}I?3(Kmhk_>2SX*29gE67>dSuN`%Ya1Np6&O4}eXVSj`AXPWT}(C8pE6ff+(l9> zTf}9S;RaJJ-mBa70O9bt<%4@IuGiDsM}16eYBp6eH>u5g#~F9(R27DuqJ{zKygJ6Y`X>T{1$HuJ z4prsJAGlj;;Oy$ktjVv{v&B;MI%oZCOG`AFAnPY0SRf)D1rV4M-}l5iD>T8dn3b!d2APuWp-F%tPdpvEh>TQ zDh{TQ_;c50&HBV}&a1FYkQs&Lnc=wjFu6l2G+wp6>V_0KRaUI56ved0c-96`P^#Rf zAA@1TT3ae@T1dk_WFf6VmPB)kZv-}C{u7lre)^gLygk zBj#`SS<>xsq*2Z0A~x$ftH6zXpkZ=B<)!joR<_~?v(qTX|5s?QLAxpue|&wCLEEc| zGIgmA$z2KZ(n*p`qlzXSO$zwA9dYw%%hTkF3j;| zlGh@k_7B2J-&y?Bir3eQ6TVj_lZ4WuQ$@njq};#<;zwp$0}Q`PTAiMmCVwru&tvKo z*`i!Mg$idhTzOjo!(GlOV6%o^x5K`z)>r{2%SW69!bSkh`eeC=e5U`jKbC8^m(`3z zd~#wcxL*R=a?x;%zV8SiEZfGQLA+P)`n8%-f5){&<~})ItrdQ2iqyWb<5@n}21fg4 z3!OjgDGK(6Jp1h8A$mO|7Y5?#a>_0bu3V~-`Z6nll=qJYlNZz-tA=FBT7XO&`|J#Z zuzz~aSMnmwZMS~)=&P|PF>n6bd%b>dddRR4wH>A^zo!mYvan(G&gV?E`a%5oa-muG zZS!jOd^h?eurVSG&#B)|`#eAAzKc*4^(d1fbou_VMW?3$;GXJ4p>lF2%+j`wjmjcKIEs@mSxgsqKh#bpXYPvOi%Xl2hgK_O|E^D!8 zw`S0~Ct7~#TebsAM`FXXq(6vv*$w8}reoQv!XRNV`@*bxVcftQ*xG*ee|kmVl7L3-gTa^KU)?k*$f21>p~irWEB&%;Fpp;Wn z^ig~Jq0&Ad(r2~2F9A+rFX}4H!bi+qdy<*{?e${ppa?dj!NJ>={k3Rm1I8!$eA@bud&d#vP;!+Y`_gXo%O3gPu2Brj%S-A&;YVezANtVaE3>Zgb(9?CO3PccMY-zrV>s-u)x>zP#-7FP zH{Q*$ZaA5724%E<k+Cs;E<5ryOhVxHGhq3K3%=1K5-WCEQc(&>P}IU8 z5Wof~cIZ}ImuRJ(XIj3>IMhf1^+@&^SwWca(wwBXI?k$QHaryxX@Ls)J>E;xE2R!) z8R@|HioaP%aHEe}zpm2W=;EW)_dCgzZfareNL}Z#k;X;3Mct*iKvDAFrXc*ho3f*Z z>AF*R&Qz7Lnvk#&135v>!}z3ZJ+21Ju5dhsWjEu94Y|1tUljYBGN?=b;D>t-zFd(H zeX{Kh1xmG;nfnJRsf!}#rxqBPujB1j2yzxF#Jyz+C-d8CDjK2R(7s8~bf7*&Wk z8IZ1E-%}QSXFn~Uh7jFMi>k(Ad0T}aX=rCP(EU1QL9_D2wOso|qrpNS7m0__L$56x zz&a1Tqj5}?5zd8cTVDsMs$FVgzPgz#4Sz0;_%` zcxlY(W(|0uvhF^tnDB>gai&y3xdnn^&qCmutkK(-xP#bynn(5JLpLEGWvZ1h{Dcd> z{@_^fhF*U;UbS<1lV-awLY5I>GyaC`sHWn-J!8giAp|y`_!C%)wf?*wO;}13sq=Jj zB}GN9C{Ih>Yol@@Jx?7ej)H!slUkB4BYkl+TpIeUf$#Wu9ai0LMFjHd@|;i)jW?n` z6FSTaP#P_qhP?wbv@0=jBb6Rb`gp^p2vm7Z+&=Lkv$fsn1et%jI%E|Vt0zQGL%7e@ zyF;%s>c}sG%cUPu*+3}0rn zqm_CceQ?HuMHopSF`iHhTByNBRnqoMKFAKHXx+8pZ^!m%wcOAbksO{Po2?FKtdWVIxjz zjwTtVq)X;yi#rJf@bxdqbW`76y4aQJt89=NEMdp`I4?pzR|bD;K}zbn1EbTqdHT43 z`RL$m@TwA}f6yeVzdri4R1ucUUWMb6L{4|_*?WVEETywv8K?u`cjet*Zs1_U=m+ye~uI-C;h_*axSs&f{HR!L)xUDXp{Y$ z7L8@%9)2XfedDz^CKHcW{lIp8XwF+=xZiaoELie+cMHQm#r5F4S=OuSYfBfHAVDzM z$-0^M_iPRANH33$0DuG1tXD)Y6W1SnL-|6H-1fJ@N^yy;20GttD;A(c($Bc@P2$Az zocG>u$sX(Frmff{3FcAkU)b2Nb&d>$&zqq_@>m zru0A8eyi`zi%NLHo$0`(I1$a_b)2}tk;)bA_&Q)Lv13et`ef{prGAB-pjf)#7<+s} zYUB07z`7(PCWhg3xMmc^WeCZ+=3M{MPJB;z&&=FEmCL#CGGq5Z^)$J9%|@4|HUdJv zzJTo*2jTx;|=-hh0e&r zLUbEfou^%My{9K#tG@{?(!Tz1I2>EaVJ~2(my0-byP#`*3Blv!`H2dIvNcFF^NS(w z6ms1!ZZ(LP$m_p^?Tayf1dji{-(QEa9UdBss9=zy@B7#t{Mt=?xgL1aL=i)pk63mJ z+X6v3|D@A-7SfxeN$+By;Mye++D8Bd{zVm%`0%<4SPysVB$4w*c3i)u^WJZAS$MQW z5K#yjG^(1(8Ipv3OoVVeeUB=^A)~?P)#C**n^QY_lR3Xa(TL|v zq=rZ;|KU%FfRZ>yt(I$}g6kx<`OKBM{krf4&Lv$*v0BSF9)@hG35JMgqm{T#D^EroPQez$K zczsiFUC60)Qkwa-<$ztu;e%IQGs0-dH5ELrtFV?jAWL{SB)u6zxyuo_JXjnH3Kesd zq}d%Ys&Tr(MJY-bYNDzgVlgD25XsZlkXep%7>!#rBc?3=w3M6MBg?LgWMSjQzJ>{t z)zqXJ{s7|`XvHDlOd(JsoQ(yPgA@Jnl99;^Dcn2!PkKiJlhIvnkbilEKaq`sK+%uAPZQ@6i#aMq^%W2Nozw*g&r6yefL;Rb?id zkbET}h=w$i|}r3Ht9${TJzqq@Nca&22{p9zsl_P=vv-R9Zj2*pMCLMEWglhhTq&;jV(dyW*tq#jY8RHmU@>J=OC$Wd!Y-mQb z+KIxO`25w$Eg6r#sfPQ!T_yZsti_;aZW_sgJcWYJ4P`vzI=kSMrFgiktprj7_j$q$ za8SqN3bKxD>%O4S=@c2gjx{=?!5A|>l)THP*N3=CNWnz$*L^ApH##xF3d{}fn!_@Ae5<%&25fnAM|D161#Mfyz9NDV3ksW_wB9P^{;>9G)&r@+NT&RX~lNMagi`S{tR-!iG1|TPQKh(yo)W@gp_&OToBa-5;FiLI+HF|~2tfk@8{2sR4$XJV8M0|{ z!(somfHn+ zX#;t?&JxOG*yc|X4e3XJaby~R=V)STE>ef;%`!uSzZ~jfb6}X=8V+)|1)jd)7I=Ge zJ!z4Ly)AvS)~>o*)&|9I(cq{(sY#SQ@9oNi#I`C(+rq5EiSNO1?O#*--R8as`s=Ck zM$geaEnCp7IM*KfaQO&iltJM!%TNw(&Uc;=A1lmSvW-8*-K+$ zu?Jme@5R&s#F!dvo{^gKVBVUT@^m{CdTARYma|3~kR8n!He{w4>)f`TWWk|Y;-vkd zvjIgQ=TH%>C|$PcsX9*GL3a)pCIQAe-OcODD6fd1hDufhij3Yv-i#Z1sL0);@miSY z`Ghgu{SqaLuk90z)@ER|airEmb%|@Sve(oxpyMRd6gO3R6K=U`ogm6n4tFOp`>D!l z(h)>^)h{7+r%~ofaXC1}pU-^+{^uE5o zbhPu{2?y8|31iwkzjd}ra4?U*t#VA_%4}jr0Y>h> zdeg7XoJ7?)t;@FtFZVK3?CQrVq+q#Z3G4WokIvEr@jl$~zpapM>)|Z7p=m^ae>P70 zvEA%=%Mn$=CHWl$oFR%Cf&y4jQm4#jwFo3}_OlnU)k9TIYNvz99;GbW@(dizR9sq; zOS_tXF@2iz0-^R`=GGhVH15B%V=$IzE5E~)W+uxGCy+D5LK~yesg&XwDIa*Al*;Z` ziJyd>jr$G7`w+hgkjqm|uN{Enl#g3{IjVn0Zmy#~8vG84Y^=SgdT$cuRpEhOe_j-h zQhn$>>jQ#}cCZ$2I-M^@@ycb;;(1?`|KNi7L;BdCl6^clZwi3W#AYv#uej?i7ni+^ z?TCJWQ7s72EKSapXXj1b9riKbcl(O+@o31()uUf*noSc8K8^To{+F*jx%)e2;%C{z zP_>MSt#0C=1`6UJppH-*(Qt?Yy2>WG-Hl|!Xip@*|7sbByD7VJZdqqdS+i2#X}Ag{ zszp~4b+q8|;L9t=o6Ccx}xdN8c z76gyzI+3M~Q{M0b_FdiZ>Mj{-%Pw*dRBW@D;S=cCfAIxrs9bgBFhwS5f|iT&PUE@F z<-Ob`MvX;Deb*K{*M8_+D{gbUVg}G=+Nxc)7x;}}Rak>!nMJjcqE6yE_#*R^&(MkU z4I7n~3|+(@eMd+i420jV)e9QG2Y7f7|2BoFepD3>J$#Obgo?-#aQ8*7(=(DFNPB%a zOF6J5ob`vQ3GsvED{JJHJ*I1^(0Eay^O{&mSv#ZXevN`k9b7aTwg55iHbG3}Gdloc zS}n3Fom3UgrHc*yo)1Udgm$(HwPcUHl4O*oFUg{I42D~U_h)y(Qg$)#>za0}tts~0 zNtiTzN;zWt8+y^g2=GHWmGCi5SQ;%%!?vPHEdnlDUVnVrRh+E5>m|KTYWPbl%?XDi z*s4ETU}*>XIA6WSwU^uS2n{vspJ1dWF_>`liEy5>GID)6<#!QvDCd*4aPq8`@Kx^m z&>Xf;Zh`QYt4^}+Ul4*kpVseQl(HsJ3NrmQVv_2iWbn8mzR85;up7{6J||#Wl#MLJ zFDD(f*WcUN5tJDsi%(U9rdXi1JDzy5$es1el}GT7U>keZz@>>R6HxmA*wn1oRL}%c z3Xu8ad+oYV^&mi$f&OO;<(1`&G3b-qu#5o8i09_&{fRlM!$AB zp1_l-tjxs>Wby+>urz2V)nz2_SX?!ez}?lcB9DrxfYsGgg{8%8N2}zKQNFRq)wJ>Q zhp~J>VA-hY@c|QrXB8{P(J|A~j<=|2v|w?Srz`1q-Q3=!UN^5t+-3vgncnzx!0P6{!g3n+@2(9i68R&1z#+Z(F(nIFH4Cn-$t5J~cwp;f zfo9$A5bik%UvKY zsZkp$!j31YFlDY|GPv(Z z0}>Ep764U*D+zwQDt{XGNJvu$PXZ<<8WB`|K|uJV<4wlud=}Pr1p7{KWD=r|dax%52gv8L+j5Lu;nkH(VT% zPw}A%czoN2fyDVbGuL1CwanAcwrc^aH5;r*Kf-nWUU&J zsBDA><>ai=Z*BsRdp%t;%Gp8Hlyd5F7LEg1o}ylAxzQo)k0=AKod$}4rF(MLQMRjT zfiyWSWB$S2xawsv=p{BIq_!Dnd5Zy{bQb_ClTj@>H^=~1RU2e*0XkLt9iWytwI4-{ z_2roujg)CwtK}69{kOaoB#I>y%$( z8S%+CcaV_?WU_#NVVN&{4WXP#B)@HNZysU_-7aNm?#qe4xy2;6px#!mPZe@EbE}(N zF*3l)uXQ1U4$h8>7YQ8J7DJMC2X~w=_EXQofl)T-zMcloCsX ziX%!H;yk_+qUW22@b8aGBVoQq}iV%79YTiUxE&NMRyejnKTrzI5$109&R zYug@LkJ`Y9ff60;G+qwM^*1$kOtHftP@g6aR@cD9?uX&@#TrJNv-AM7 zt^Gdkf1N_meIDPpDtdpb&MxOFGNpnt4csXGXACL^%-GoQH*4#=Fx#5nsQL*1Fv#H{z zMEqZzo9ql8WYI)sakDT?oGa~$mD zR`VYFwawreJx%1Sfjq{8D5dVZd8G5caHt)|8~kiw4MRKNSo z$da3j+l*wIWlvi43j}7(%nZk9woIg`%S!QIblBIEb~cVX^w(q#w0O8cxCgfM?|j@@ z$&B{nT-?-HYEX9;>60|v-O@mJm{iLXf&=wwbucJ_RBb=a6( zT#K5UQ+})kwz$~d2>x8hS#bU4pC*?av^~_|E!leJ?#=Vx^e6cE>vqv8^gdEMp%D9~yrxN`Sq%;!{EbgSHaUxYH{lP0 zmccZh>28Wao9Bszt?X;y`O1rcwggE>SC{9G*dr--W>3+buJoTeV_I4%KDPzbI};1z zh5s*xb|FVdrFlHPbIpphixeRcrrXIycd$!nWOT4AEDo3BMaV)3)3))}HoyV|qxEex zfj<`!kxmY-)w70Y!?m70+B~=E&0bBbtxlp*!`Mg!)PuDkWC(O9NzT}B{F_j9Bp_K< zmpfVw+Uz)7kr@sCJQ~00a_jYhHvQ8P>jaWVymtxas%icG1_gU&}TYh*i zXPBIAL`Y~8lLFlfJp~l{|IPXWAp)1=4PDt291jRmcI3&UrY2Fmz#iuf8GO~kbPqdA z7rc7B2yvD@DHXNP=H?c1T5|b4fL|u~iJnKcIN{6MJZ;WUcN&fb@FtW)L*jK@X(?y? z4?J|^vP!?)eHMg39el(aQye z;LT$T!9ax~@o7-ihh!-*;0$?v#*nx(^eGW9o%75`*M|x8XNHdfO3Mc}h8l;y_q~BieYN6B^CqPn_Iy}JI)=S3oO@T*yHtLKD4AW5Et18#u7W%o5jAD%4M{ikz(OXVA z_4|h9fx88ahJ7FJ^Y&F(7qD5ksK)x~y)e2uRoA08m6@ff=jHZ#WktclO(?~aQs!nl z!#Jh`cec3~fql|^2Rn*HU!S*4RK{Y%=YNGn6cUHb|2ml6&2}n(gcEdO{DI2Hs(C7& zP8LtzAm!A5o=EwmP^#B?J_Yfe&zmp*pmeUJHa|i^&;|r+3lVEQ5iZGKD!OSPh?e7Ac zCU(YWs`c0^${M`yFO_>pwsm%k-{(?}cbt%N^2CbMJvD)wxnwJSUKZI_~WGVwTK|go`UvxpP)p!!ty1BD2QQQQB zHk(A!DUNY$-rg$F6N~PeD!KPAWc472lfjmMeat7{0$5ZoJWj2BEkUwX4=(&aNw)S0 zl2OVLftUGQO3~XPBFPnQW+#lq{zsRL9yXYX@*+RhzC|n8Mcj8Ilda`OWqacpCQVDr z=-Wx$#jhJbTJPD<@<<+SI2NMg1GU@)k};Q_#V}Mg`x;p?TqdX(6?e6(?oih+MtcPd z-cIPPVcfbYoL+lzk*YUH_PRfK1;}fuKt#3UU6-R)9dXL9L%po6( z>{98O@JIIgKYT?q3sIjwQE0OL@a%z+%UmGgsbG5>s(7mUuZGQnI!^-!&0J4+fv6R8 zl6Yx;lK5E_bxV^<-+al1P5(PLW~mwj)@Jlqu>MDb>99_;CxfVeH{?E4ixsNNpK}|g z9-(?^^peb1$`@?*^j`bgmR2PFg*_U8v3-E8fr#V>T(2(Gss5na?v^Mnt)T0M@qiG!u?h4E zu@9c@TQN0_iPra7vjj6notJrWVnN+tt2W-5fS)QB^c&e8QZnx0{2110%U}tLqoprD z(lR_ol<>UqcUvhfGYUd~tEYWV&)j!JIljhB+zb#r_bFA=iG$pyZhV_Wq#i9_@D zGq2%{Q+Ao1s=oPE#;L|%H4*RB%V~aL%l_F@Kfz}fTU@SvWjIgtstN?qr}-7v)wnoH z?uEW4FmS7}2)wQymU*U>p$oBsYM5#l=SVi(AGP2W~CB;Mjo z%^xLphJR(hn$0#J!837oolmTdYN)UO-n>+N;e&OpLY3VHGM~HX?Ol1P=iKRpx1IB~ zvJ%-|8S4Z7jgzD0X~{osAJ#%91pQpG20?Uz@0FMfSZxU&#f1etpD_*esID>U6z`EB zI!6=7!x#72I~LxmSI69qo~nLR5IMFF5^+|}`7oGh^O8(%Hj@%AKt^YR_$E|)5Xq|S zGjS`naoxY;yQh<9ZZ1c+@6$%(fc~xNBQ@v10$;%=zix#WQaHi0(&%y zWvgcP@u6_2Z;i&d;EQJvTZm&&C+{4}`LOgSsi>?`)tSg?RI`9Y+^I^s|LuJ8X)a{a1J9L`*jM}_ALKkdG8I)$2+M*JjYeEW@1xlQ*O zwp5v8YfK~eLP(H(j8}cm!}dC#IRM{7z2`9u4Jw#whbCg{%XQ@D&hqfBYDqyV&Q zdOCf07*c2P8&otAwP(fP?y2OP6uG${#Z#JNj_VxIbtyLuz}eN5Rnlfe zaia>5Wh4pM$UVQF7ge!>0?>ygio9&icTz^A_nJp-w@yE_?1$~=1En|TlAP>647+Q- zs&)ep_X4ilU8$owV|M;AFux|Mp<8JN+YQ1^iK7hUidu$~hvC{IQ|_9@yd3-ZvzP|H z$KO+kk8GvtKS_AI^^mq(jzw({rk@ui`MaR4@9GllVNhi>a~wsZ^pyqRH`?TQJQF0 z+|1S-?6wJ+$tP$%uatRJhO`(5d?oB4AN~*bjL_w@C3db8_aAG4DE-{*L7ZRf<0_U! z>QwAwPD$YsE=7||@iM3%E(0cywx5;@wr60brwa-fz?!Y&Z0=mYI2=}k?9moHX)Ft) z^6yqlbrr~iqNd$B7-cJ+@U& zC%F{{L~mHHP>`eD=MGr+H20yb7F#);O?L z%w*~x2M{wZOM`Wo0nrQdxlq3 z+^oUGz%?bTi#%kC8{V@p1XKb5mR?WI`mBSCk+Mw;WZX#GhX|3#FcD{!n_$+U#Hvng26ICzZt6iJAT|Bn3R@Wi&W?a;h z@Q3m`EVPVriPZcIwu&%ae>akW^(Ds-$*mq-warsZ3M@}muTHfBZ6N2FDE||r6JS8g z1o$$|uK0W5v8jesX6D|@(Gg*D%a6&;H<)WmR_R{y_9Swxu7dUSH1D%dX5UniOCUhC zkdZY_YxZSsG~P_j&vf!Cn0a@!r)69s?MZc55a3Em9@gf|bBEdJ!FBL*=rtzx zwg)%m6ci|uET?yeMj}G-K`F!@b7E!*fL^XUd=aMBSFT?`U`7oSnaTt)ho9K!Um|=u Mns-4}>UJ^z2ga@rr2qf` literal 0 HcmV?d00001 diff --git a/docs/img/ml-Pipelines.pptx b/docs/img/ml-Pipelines.pptx new file mode 100644 index 0000000000000000000000000000000000000000..1f773376abc7a8015572f3a796ec028c8be5ba2d GIT binary patch literal 56777 zcmeFZV{~TQwl4g}w(V4$R8*WvzYhX={Ib-FEl=yU&?F zCdX`V8-0%c^r!2Ul>h-n1%Lw}0RR9IAa>9kcpnG==zs$NPymp?8Uof<4u)0^I*P6~ zhW1)?E|wOAIiSE4SpeYA@&B{^HD;+815{SvmdM?b%|*&2Jp$3+0|L5MrNAi(Lq-I z<)RtAu#XgB<1YHle*HlE-xm!$KoqKFF+FsqHprj?Yd4!)=@Y|}tCWWX34O0U5-uX# z>ljv`;??LhB~PPVqusvbmDJB3N%Mg)r4NJ3$I&oSk<`I+zM-2W=Va$yui_ELhQFi9 zLJWUXsLu~15<-{vqATZFQ@sk#$d01~ZS3B%j)~D*Ro_)=xU^J2pt#LHXF!o(=yDMX z=m_}Y%vlK=WErPq*AZ?iFasxo;wSe-nwjYFl?e37-0kG&on@&t%7SZE{ADGIpXD|? za9$sZi!Il?5Y6-~RTu=A^Ka$J=8MPJ=ExkA`AmIF^xsE4b~de0bA3_h`(cyWl54ld zY8Rgehf`Zgo#pc#^o+Fye8%*<=pzk;DD@%JU~P}-qt30Yrmqt? z=bUc3rq00dWUWnm>r~TFIZd16O~BVXvd~BPmfE--f8 z{i*#hpYWh#XJ}zhPxr_Ee_-LiF&6w~>SeKA(mf2YK^LN}0=q9MR)hhQ(^`nrR*=6S z7(!RTmr2Dr>)N>Vf`5tUrhpDCUS(cPT-+J)+-}F|*Q`^jqLV?|E#sW1gH_(mY~u^+ zJAk`ih<5oRTa=uX-oe}rx>7i@lEx@$%1?d`hMor;0P+>vAbwdyJs@d_c;!@R7E?nXtpDUdQ+fbL zr~up*|M1?9aD!QSCU>F57dP{Y#&lvAHrbUFRxi@zDN|5k|H_+R7mGOGVmIjpckW3<^JS7$rxq1LZu7Btd}wQmsB;+ztx@}nk02ypQ8Hq`IPYUXk+6*Z*O60 zU}#VO4@LU(;_Dwq_D_`xi0U%yVL%kT2z>QlbSLEqmJc)3;U|n%u;tc`nQsW8g(^or z?x>7p1$9fq+aHsjd84a!k<-U$4N_Mo98A?sCtz898M%T1HNHhZ*c9s#ht+a_H1RU& z*W~AH8qAU9G*7aN+z_u$HsMPkilrY%oc3!PWG)HW_a!M-Q1UFPGvsag5mq`EUC{tO zRawpaF#+GR37hWkO1FEH3f)&*>(Il(Y2T%fO=%u&=YCLLWavtuxKgyXdXMmT;PLNG zpXon!`YaEuhWwvj-UAE(AOJuD{qgnx@!4(c4DCNTmY##Dwbg&k{u57uK55qH3;(^h z)_5r^e+INPDS(Wl>sFbAK01)K8REr75DXfi^LWqd*lM!=p)0`_4 znf*6Ov|c5Qi~d5zO2gZq45LW$Kpu_qMYUsfx5U)F( z8Iuhoka7hn`OEm%6+SYCS@3v`B{K~uyL{T;NK1tjK40XR_Zlv`_P=CwG0G%s?W3TV zA@r}kHa4zX8G=u+zm)J!B~!B{FhUqpSOCS>H23DF1VqMP-5@r8fPTgD|9u0cSEdN+ z1SWW1TzxX6>x*#AwkqmaETzy9;A5?viqasAZ9dZ`6VeuN0VGHr!S{RiF;(5p^jOML z>-rfd>MN(4Az~`BvIB+I`^u~Amq_1UZocEuc4&v572n%l^6>$@+&8h-A8vP}`Pmg9OA6C&~q;xfF9QS0vuXl?;sam-8~qM08QDI54W zL-1KrYbAcc3n=5fzbutXExR%^6I8H_`=OPfmZ&Hmpn_roE(kU=b9^v;AE~Y5`MCEx_#(((rFmNO>aq_TDmW=DjFy|nm({mJO(Ly(9Cs~-qbaL^MRl`9)S4obD5lRq?+0lB9B9 zuxTmNCApvG?wYDN6YJKev(wZluOr~9V2<-EgE&5RJ#+dd83Tdklr#hR#=Dpttx;FL z3u~CsJMvtd8j-07Bd`*-G>Qb&Nk2(xK9m* z{X!39NRbP#7y?BYBlF}TVQXS&7(_fgApdIiO)dhSm1vZx*vR#Ex9jaAqu_HYg$<=;9}1}KLv-n<@0zMBWL>FU&dtjEI?Oh@)sDlCz~RIV&z% zOM88s{Z6M+RwW#piyuAxHGej1tJt0cCnu7VkK6u>pH;A@v@DH3R%3FRVZeE9(NuKo zti07DvXx?lz0jDr!<1qn9a#)(O1DXUuKc*Zeqw4~yo5I*X8r3CDCY>hnr}x*TWtI` zicUw|nV~z~*a5e3dF9Qb9?P{sX za9tfeAB2Jh4!Py6IHLno9(PwUO;5*XB(>>YKvlY`sBOobZimu~E%roKE_^U2e*0Aq zXt~M^E+P5|t0#_>C^=qhU0)|)at9}Vy0;U6nd;s~NV|D$H|yzY9GWG8YMbRSxbPN~ zD?a0)MFb)+Y2*Z&WJFXGpFD{FC>n5O8||wx8WoJQJ#jL6F^w0j5}oPZr_{#MBU9Dd z6hxJ)Y1U>iEr;??_vzR}=7Q*ZDCol3F^U>SmRzieh&e+RJmLU1^{=Hffl3BmIdn89 z;y$bb4B3nFTetWnN9dDbGC_jNz*rqbp~H2*&vef|0+5XiyY2{AXEg^0P^-FjYf`Oa zJx&}s=2Dv|&SlqFUV#S()fegRq!)N>XPiSy~dYVLzEqBtRvLY8i_VUvsD zCNhi@;uy2;#mOW#I9dW!XXSA`R7<8{grOjt)8};7R2wI%)*=FxY`%Khk-ZnRzDtr> zwX;&{w##a#p@`9CRDu6mop}{CI{T_SyRlarYDpl^YX`CMMi}&ZKAsm$2>*50rqxr` z8zBT_0WKCak~4(WhfR8|Y^D1>kKJbx7nUm?p78yN@ZBH4vN<<*jR9iYO3*tM&Wl@M z8qIgdLU^yg0K%J^>kP0>$`kVuLe|9>u}w$;BdT?;u`B9*k$B8TPNn*2{`J^aWR`z| zduReyhgV6}fteNuH11eGJzpHZJs+k6YCzGcm*A&g*3Y+JP%-sQ-i#;0*Vviuo{L_3 z?4fcKQ;I18>$)yj{z5)Qi?#F{SKM0ioZMbIk-ZeI>1D28K>~63G32qY0S@sah}gX- zr?UyGBE%xJ?tUj%hi?%7hOUIbMZU*%PCP=-Vgy4z0;TtFmGMJbvl>9_Jl%zq5QBngv%;9H2}9TWuhAvnk1L)M=R;#5P!euWj?Ya{s$0hhZTArca?)s)y-zfs(o+>$er z$PL-|0i zT%6X8;c{zI zAf$F4EwgpSP4n03<6zNJFl$2LxpmniRIU+jRHmnn;!4ni>*7{V>SKhvKv|v-H+P~m zkDx2NbVeGDdoGsnE1$x1AW(LcD!8c@weW-P45J6IZC{nm-(UF z0gF497OxEj)|U6;!rMV+0Gvhc+D^8#+)Qa>PIECHf4lTWk<50$5qE3=;snAscWgOv zeP+5A2@1GZpCjOg89n4FRf$WYh`wq{-RBOG{)}uBGcI^8rf}uk*^I*cxJ9IBGH3Dy zMeBIqB)P^J>{=e-ZCbo12HV;?hy1|2 z#)}l?L}s$xg957SC1!f)J7T=pOpNX(sqoO1brhxg-)isjW4DbIm#yRbmue}ARN`aC z6gjKv;&Nt|%$*1J8TIMJ(}s0X%x4F{8F7)}*Rc~@G*t#`b-(gc#cE-_iX>s-il=Lq zai3Sh+NYm%`zl(?N?2-s9VJq)V3+KbSXk~EYAff+@aidLoLmo2v}iBv+`pHlcqVLf zZ}Cj>*Y&vuLiwoW8#-wLUDqg^oY1D+`{UB>-B*c9vYF-e0f$(?T-iCrgxw-{pcaD~ zYiD@8PGUh*bqz@6y37W7cDB;mX>~zQl(+{|paAaQ+^ENvV!CJ3xaOA^Jej3)czXt* zC+4}B4Tw0mtLkBr5>tG+_O+F#=ksyO$|imW;JtIi=&PB$d&WVZS4aino)&o7&+v;L zh`V*~A|Rw-5J`lUtJxYgnV^|)jMZ?_yMz^=dX#asNnWI(_t0szD&gk`oF;GmDjT1B z*Jhp}zm*~+G- zWRZ-oKk|X>Gy?G33-mhH58E1;_|m;{oq#}bCRjZ1GZAN_Al@8~t22P;%*IjMD@991 zwPoQ3NGrnS`}8Ik!@cE}mRxZq(1khoGQ`CR{&8^V1+_ypezSnl`^V;6qxfhF57Y`m z*&#q#N|pYm&+()cDW};!#7~vuEGm$713?W}5IUw9!v*OGv&q+Clf@+`$U$5blUm$c zw;|t3oJFOA&^_T6&aX-6g`3t1%i@utL_NM{OrV>Ls~?l^qh8)+h6SSJ#jf}qLk!`k zYKw@7TYNTm`YovKE*KK#FJdnI$Fcc>1ea==57Bq{|0Lb4@HK)<;Sa+Wk>om5^fys=ZDorY1id-$VeBv zdFAb=?}+t$EpYI+b%$EK0#RVtTg3wu60=UWXp@+1hzZ!jaQ1O|L#LB1GWm8oKGE~=u_kBqP^SSY|X8(m3Ol;nF$tuV` zm&G}g%^iv64j;u++?(@GvbzR(f8Tqo8JQm1hg4s-lA*2Cal`n5fPtWy)lm0c-FNP) z0eefNkcL5G)xGu6lo|ia+|QvmC{~8ADO0_I1@^O&43pkLA1Nvz{BaMbm) zPi3SX8iX%Q7Bz-=VmWTDBDzL0yWg;G{28W)ka|gYGwCwN0Pqn4Jm6uQ ze1i9Wya>egT-H!N+>}#8|9jGqXzF0m;=IHOk9VDIt9g{^t0gqp9)Yg*b9b7F9Vd>4F-2XsMZ%q#B8e;Z2O3&>3v{;$%2RZP1JCcMDkSj^v z$t~HBzwHU^lZ|uCN?c#4N)t#ttL+t4oHb{aD$bpdD_aOlDsqJ6wu8MkDUX32v$5}i zjyvShqxOf5ltXl;pWn){%~u62n|K#-GR*`f@~CqUOCZ;6!tZWB{eJ&Juf6be0vJ-yU@jKfLLOfs&VquhgkoMX{bf7FCTS000{Zy z!^=fOPx3|uXyjDsI$P4*ZLUtA&KOxLu;}ABQ}1YMvA&s+A+sm z9Q@@Hj|Qu0yz73(j+VXLXgwd?xFXl;TtAu()riA&t6?iex3)@#)nJt`8rJ#MBKHig zbHzwk)jSHYqj-;O`!SJo8!lA3TxGjV>`ho#ZrGC~>vc$ygeNSb+be#^ev8v#V(tl^ z5`Pe4l4ovDzu6nAHThYO)rf+U>iy+iq|jv4ikd_jVdvh|x=!Q@%>ie}u#tsPS}o>7AzGM~`G|Qx)gC&_&2ixOS@Mj)t?FooOHZ?h z1rrWm!=_&5Uhy@bMMWz=ZC++N7qvG1r)fk2a&in{=p)H$rK!7<8f1>O29&5;kCeRP z)lp_9EAj$?w$2{X5#Wo8QS=)7hWl_D6>VyB&7%E8_h)+cv2aA)8L1$+BL?`_MVD7*e*RyYFm_x!@Cc&e)RYt z4zqbU;OtR6b8nqh16n|NzGW>adt;cae5H!I0eAiE?2eoD^w}5hx0hv4_63+foIo)I z4nB_>&N~Y*!yr1wjP+=cp23R6BFA8m=R~alOf)=xoV@fGba$OPg?(mMvs z%ejVPJOpL7N1To+WSh%A5tVXsmadBIkwxSj#-$t;Gg0OF`ytvnJl{d13hA*~jw>}L zQZ5=Z2}GjFVi)a>6wKBX&xlz0?t~a4|6YpRi6JL8=^jJ?fv5}f@d5=6mng>0-ivJb znVs8T!5o`=b)puJwHPZfEq(m#RfN8d(vm%Z2ptO3NSI8r(T&~bpuV4gRFa&_HVhAC z3v5-OGV<+7(pJz{ML^+6)y6)S-7aMSQE-5RQkfnVzVpr&GS5@elcR6jM9>+CmUGM+ z&cs7y01k0{$QF&hg6gu2<*fvh$CUI1#XXgh%Oam}dps8>x4YpY3G}c!USToB0K0=+ z^NBzD=Ha^woTVAWMox-9E@!s(A+b8-0}U>xcDYL}ZB8L81J7&LrG2(U=$AD!W~;ZB zlV;}Di4JRg{+3}Q95dvW^*~|0E^OAX@NpClU)EY~8{K=FyV?*AKp-*^Jk%bf0kQF3 z<_~nHjO9BJzaJtatn4ndV92~gLj+xy`4>7z9}d>NC!We)#oImI1w}B|qO16k3|`l7 zIs+CXaRLbSHmmVu%V!gJ>gn*_vDxIBZU=}=xoKy=8(?%jxA4vR4w|~`7kVl;N6~vg z-gd&x?Pv2w@uyD{;DfqxW~yf7GlWZl4AiUCsITNCpmei=9WSNLZ}XDCqSDTeAgt6m ztq?AkpF?{9hS)nro{aEN^9m35@8E2xz9kC6b{Ye?W#AqUpEJQ9i%11e9^X3)&GS=z zWF~|^GDFYeh33$VDf+`p(>@?DMHLbc@TN2BKnrY4GEeq6N2*2Be*uFmajSi|#b-bi zcMEc>rq`fvKmc$4@_o`nWP$;Hhsy(dGX$Od8!3bU5QI2)hY-FdD#(2(a-;XWJ)}0T zewHj39L+@Ht6S>#9h#(O{8RaPN-jEtjBNOK0J_HTCAOjH^v z#(5Hv=pLWK@{bkw!%Xd*Zt%e8?;lOy*jYU!>QBO25#_}E6idEycZ%#OG`cH0pAmvl zkr4??HJw)S(sjv&QbpU!!lI~CLr}6pX{teeVLps0ZV;Iw?3Xi&7;Vx(Ys(QExG}iH zH1tMu(;P#pY^Y#RYbn_rR4I=`FP6!SNrI@5@UWEvg!y(;a(qTO@D65ZW9w+Q-FAJM z4z?Qj4^nz;H2Bs74wv&UH`|(h>dh%MhINocW{Zm|Vl}TeQqP`F*3%Z%i<;sXf-eHE zK<@!}L^N74KQw6GsH}6-E!<5KkeZvef=C=uWy9mF<<#u^khVU!xrAblu(-I`>(`EA zUL-Ja+mfEwr)Ehi+?~XA(qa-nk`8~g>?HYj@Y#-*J565@Y_%I zYoxofi{d7@Pv;ktW!A25nmU|MpJCR24Z_Y%#a^|45|BUAvowDV!bdrpe=a0DRc{xV6XzdLTZqExRLq;tZV0#!P+V1%^bTg z4!LEMWwr*GZyof+qjJlMtD=FBEw_;kTn2l+x5_#(e4n3WA9=e=2t9CHttZ-km9oz$ z3ra0Ww4F^$6lI@Sp^`X-<34X*sb0=>+~+==Qd(JneF?kw4o~97+v$+q_z-dQ2XUsi zA_abWK9gO6VpWpcFm_(rhAW+!+B(D>>#@1j%jZo0RSsMsJZvWr0Khce9~EJLNmEPd z**h58+5aPF{f~=(P%m{YtDQk4FNA20&)@8fd6a=Bp}%EXhTTkOnO_H~8%*d+B2G3M zF~89FZ4$7G)*y9V$sHAIC}|dxYhaGi`(A|(4jXz(hu(7XQqo*f_OP*9Znu*V&Q2Yk zZQ5~?3Qix(I2Lx2{j@VT8YY>WP$sITjqYKxezW|#e>&mqrUQ4(`4%}B3j^YSdglW- zmLETtwNM&Irc?c_rFI3pkDjmk6{>N&IT89EZKBgN24A4)={75nffDtsX~5GPZ%z?RyP5EoBT~>sXma3}6R8@fnIm z=+kg$6aMfiOCWvmtNERU2y)in8X;&Eke_mc2I_Nv4G3QJ*$^X<^B*!|R`co*U{&*4 zBw91q256Y;gtf?XxV7RHE{!XZ+goYw(rk?|%J@*z(ic1P)NKLK+xAYt?PD{VK8k9QI^);Nw+!U|Rmn{zBL`z=G) z=Mq3QB_3-bMCBTz&CuoLlBJd!VllG_8m%C>Ec2x?AH}X|gd76MjV^VPx4`M_AbJ@5 zp0I1MHDbZCdiVy92Yz*7f-0{sE(8?cc(<`vborerG;p3odrqSlI2OrrPUJ>B3{z(F znoMSxn8nU(Bg<2{B2W{*)kUWIN|()ONx#sEO*ry+T6BEr*%A&t8 z1K;;4EbYnJY1rL58|P9@fc2s11hr0nz-$O*bPOsCr+C?(QKT)~Zlp;57)TM)w&+9o z(jYv=975t^jw#g9oh>dV04*7}6gE+FQop%TIXC4wxMj@2RcP#e~O&8tFzhERYI^P}1bcs&bG08b=|2k#iO;><;gyncl#XYoV8T{zC7jeCQQ z?u&m=T;k@%>SLYu7y&~Z|GIrLR6c3Zx{x&Sdt8pw=F9oQ(31$lN(sD+=d4AK(*|Bd zL9UINC+67meBDLUJX=a6QP>KKJ8P|o^k#!zi|{^LC;ZRf!Gb6BnICV7pF8^i6RCVL zg(d-UXd!%MoTRi0Ay#H@tf-yNWWgWi7Vb^KXa#g-!7?XT5H<) zh}rJ2xC@#$fc&G}P)li$47KQXTkMSc&){ z&S_#Azrq|Ih{tUClN4j4df0~?4EGcn8{S45VeA&XVt)ZXl5dXgVw0-PmpXxT5(@~* zbLwYjE4HcOsuAcgGYi@v!XC&{FAickm=u!LD8d3L4dc=Qg!&BmzjM66w6V@ z$HS`u`#=?o{4}?;vieYOh=&T=W#ORMEX`{wWIrrU^hT(fL>=rTH2ika`fpD?-k{v_ z<29W980wsPFW@j1jk%bkz{!f5Q!QusDf6{>RV+>39q*i}6CvqH^G>@!jm?OQ2_#8b zq)$hh8q1;IBoCuG8Lz@ZZ?ccvq<`X!K!;50Qk7zvh=(LL@FdCKY2xcLP-nk(j_M?v zk}@Vx{}^3}BSR4eCJ?w8Iq)u?f>)&Sc7ZAi^OQxK$1j3DDV0Uq!J}D1#U<+WB`Oc> z3H6}pAUr+}JD=;hnh>#Jo04!HN7~|rbpY6XvthQ}g~yi`{53VSd9}!uh7oQc(hI^j z>EeRN=@5P&aJ9HafC{wQ+mkrpuFV%3bm$ae^_AO(~Ekk6a8L1yYTOFpU z8#$S8@PEe?{@)qX_y0)7^jA61{*^KPADQD<84C-`)}2R_~Ge_9X!+r|D%J@{W7?LVu+ zd*{~7dI&MWH*dPm`Py&#k{XT>Sh%qKyCa32fsi|1s|5X*t5@Ih7({jLQGqicu{}Jm z@ei+V=bU#@A}s5-J7ZBHOXNqC=Wy5Q>18+w=b^}i3eoH^r@D(>JlNXxzNBp!q&ijg z7E+94+gh>7nvH!EOEUF%R;x^2DA&N?9O1sBB_nWiW(jrr&&h>gekf0yX@(d_Rc zc=P^UQPR+BtflR)c%H~qbfz9!sXwc7@FTEz%2qV?s5xC4Q{oi?@Ya(+Yii%loK|c_ zeajkzC&74LK`}d-1GYvz!e65_I<8E##9m;57i+0M%u1w&uaFRIu#!|`DY|cc?}Bkc zN4hc)eCo_1UrZudxhR8KnfQ&%(M2SO9?d?=175<#qos(_jC@ zo;sB^tIs@AJ3*B*VV$|qM-6E&odq>Ec%GPSOPetnJGyu%u273pFtBI&ne8(vN1kAz zE$66O4bD>R_>;@{L%O}FR{IhQs%7Y**?c7JMRz@-sf6;}+1sL92hC|Oe|oQFAf@@x z;&R-$8`INMp+tsbU30&CLkuLtGF{IPb6TzT3hbnkA^)Jd;$8U;q|(vzFae8A$88u)gE8?%i!ed5WWt66)k5UdUkMUDT= zIg(n=r!-XR_=tU%BAOo@m{i2qq@B}M`fZLG>q1(=a+?Pcb-)X|nM>-O;-~-aSCw|0 zh+FqQ)p$YeT@g1Vf6(yTNzS+ytkV0iqo_+5zmgm*4bakEch%*gz*U8uJHWgRIAs^u zPfFzoVfB|cVC?c3dvONl?X2{)QBy51{#YXUIpns!bW&hb=qy^GYvkS-SsV~j(^3wa z%N7!YKB{qD!;qEUQU%%3_u9xV1w18|H95^#mRz)e?NZomq@TJUV)8<(F*%dyvb6>H z<0`B_g&9;G-ZGI{o!Xe?EveyyUrv3fsQPro)9nbNh*20U?0>jnhYwvz8t4GIn zCFK}M9AaIn9~*blFVwRUa2;ywS+qkpAlZ33U2fTLoy>_jBo$UJ>t~x*<4Rl?)iM*& zpOuMsD?R~h5EW3HA5wB@+{P=$DSbTIY94#B3UtG=a5ZH_R-;*Xn|(Z!T*k_s=T~q| zC~VNhGStsBWiHQ#+i5@$1ezW}_gW`DfQ6aO{g*-6h@HSCug1EY!7s;VANyrmi9$WvnI zZX0fif}71_YS8gE25PXWfwhg2twlJ5Ox$0o&8THRs9oi1`i~}7;f*sxH_PS+tS0cr z&oy9E(+Tpd5Zz*o4g4tXOukO>Lm!uwML{dJ_Psj6)Iu0v{sP?bEJU$+VDN@g&p@F3 zYIzz@di`5qq3Su88V3OY#$o;qVEL2mmnr{O`L6FIH+8dWgCUWVp2n2EFEf-`;{}aS zpCF~2#B{})knj8a`N6G(lW3@ET(i&2qfkRpau1VnP*0ObedYIMx#A>~0&NE*w-8pr z__C%=Z<}xD3vA;gyN4!L#Gg=;i#)RaKeBD1m(^IT(yU?duBM zEyomF65_FgjUEauRL;`t*1xrIgsPfOx}nk^^4uET*8?#SFyg}r3{CrltPoS0mp&td_B#ndG-` zJs%$2&=HZT!n$7Q1}7iu)fTK1_;e)E>lx{5Q#8nz-P{y@XIzwKq*hR>tM9!m@~)zy z;K5m?P6)@yVf}{t-@{T_fhkaNri0eLz7c7KTs-E4#S-rnQuv zpzuW_10H87Qz1-8`_OK=&n7cV82bT%?RjC55P5l zJ_M-pWudjKUrDUx&)PilbSe3|?ul}_zk2LoR0s!t|HgxR{H@d#>De_2_Eji_J}W5KP!V;C6CVygOgVWFN7vb`-0SW7 zfo=G*WsLa~H! zhjv^|Pw;KacmqLs4)}Ma`yZ4v?+oJeKSi?z^XWs1{H@60)URTOl^Hk^g!On<&KWeF z*zgKS6u>(%RGW21&=W=_O-)efbaw8YAle{|8H5Dh=+ER~?ClTU(nCYn#)A!E2I<_O_f5xEM$)6b?2` zS~LT~fQ)7<5vm@md`1U-tKKqjB<3V4_6HW!-iKSrF%cNIV|k^Gd}~R`j)W9*t*vT> zS>Vqba^*`M2)ujet|fsH3rq!OT8|E_|1#}X`+Ao+!kSR_yWhQC36cRDYbQ{Na;`s; z+?wMgAPW}Px36hr@utEkxqf5YpR3=9^@R9C4nEzyNX5A^INHu`z7J}3|e#a09` z=?}sccfJr-%jONC7SoG>tCpj4(44@2tXEp&M~0;21@O_$sGIILTfy$D3e>x6(E;4dt8Tdml?}ExanHemFwELaFRk%gm{$qsJfYTzKmW%59Y3 z3+5T#(M4O)h0+}|^c_dS4C%E&*3`31mCUvNS|3xFUza5|XQT6-%(YJ)w-GMQ(QQZb zmW#MZf)F8b1_1B~S($oki+R->)>+l&zZm?ouOO~x8TFVyd{9O5vY^}g(B_#*l755#R4o-_r(lN$n2F9R_Tqsm7rpyLlp%)p-?u zj%NDGQ2_Mu+;h$agB(||{q~rG+|l8FU1Byeo3`c@;l$dr<^FUCzyq>Q@&bGH6zaP= z^sU&To@8oAD`97UysW>h;|zfpx0_#wA6S>JCZ51vK?JU0FX{T24(IQ(eFsg6QZw}( zJU03kygV3$o)~wV3p%tg{N}tVR6Cp6;uJgX$r0Yl3E)D|{<#F_H@oF|Y&2yo`Uo+8;l7qf+93gC+W1YBP=>($7NuVuVNb<=$MbHy6 zx_1(kc?t}jYcZXjHpiRIQQa#E5^bST`|YZp=-H>z^KFj%4W^DcVDSD%+05v!{oj@T zPuF7fvw>%a715J^(wlJBSV-k0h!jRACx@36YJLepmy{hn0a+>Rewh}1al?$qxWmAh zV#=w8(hOC23nMirsE5hkXb?-QePuSSEcujbN>gn;&>U$VHK#1)fcj+Tht(JKBUDTA zVWFOiGTM_e=LMb*hb1FCdg~+QGE^Wusl+L2^J2x+@6swXYEmsQQ#Ii0v;k$sej-n& z@v&auYb?$l3lV84$`LGOi34RA;X+DP$rQ>g=8BbqjM8YE7YRx$Se9pBm(_>ZWphAT*CGaax1Ksu!Y&skbeMeRTKr>!HckC$l2cTa1`c^r$9bV@UKe5?Gefn#XG zRAQZ&_u!VDH_sY!e^DavTB}jq90}~dq4Sw;YN#2 zC@aHc1fjZz#bX}o6eN-Ja|;E2B?9(QTT8+ z2BNM8QhyRjd5m#E;>s(ymNFHZi4S^l<-KAp`VIQifP% z(ExTMsVCrbGS8O>3?}7@Wp}nDa}j;d9GD`{vFm-j zw#pK8S)Cu%ug05XSECYI?(7nt95UUr(uH1TQE_VV;O)0$;FU0h`M+%6cR{VsxO&-oa%7g}R17d*KCp4`{!t{Pp&ApqVaT zrbg4|Nn)--tg;5zO1#|mFtoY^1`L-2KW1fl3!zIk93m?G9j$GfvmE`eTv#KG@5k7j z&ysP#?@5!T2!q;+6A#uGqx`fcofg?yea;tE-|p`!b=p1OUO=l$%#CL%_%B>m_&}zQ z>Fmg;2iu28BVJ7ztG{R`t!&XmoY~?>oO+-JY&pdKEbJP?0Ia*|DT@}Y9BvF}rw0!a zs9!cyiJ!`L<_8F6F0R3XByUF12K$$eGClL6Bl4oi_j46A(K9B{5QiUb5 z2$pzZ>fR9)muhr*2eeHU_ZSZlnGsVvugBL7ftWeVAO&cwfGlz50w?>)Mayi_)0HC@qy5Lty6h z`Z(ze=vrVE*8tWs*oZ`}27Q#84lTOtaG_E&l}oWYSu9L}PlwvRWNZ2}=j7~P-)aLexlct#F5IPPmS_xYy@rupdKS}?A(s>0^^qhVg|SRI zlFmhD_U=$>j;^<%Oj|boFnPr5j^H*KO-DXR2kLE6mw_B!I;6v5#jbn?byVc{GS*cg)|Dh)X(8$)-JaN+m+ zqFs@%Uq& z9xi(}EU8wJAFMzno@$Uq!FxzZVrV_nGW$TU+e7Ps3f;Fk!x6PlVFjWEASx$fg99+x zB5?w>43vCWCXoA{Abr(03Q7}sh-#?BQ| zW!kIAxc!8V=fEFUO zk?8bDYkgNYdLp#)o~mtvDaoiZX94;lF z3?;mMz0oiNTknkSImvvY-ZJH!?pZqjV#Yz~%)I(N`>mOGK)UTAlUw|CHUiI_F@nn+ z9jqEnrgafpXhsh=J0VE!cIk9^61JeEb+h)>i|px6Jh`{#NB5RngK2Sx6f(?bH{A}cEo=2~x zHDO}df4Hh`WU6IaD{$0ANB)lit6CRHUbp&U_~ zA|b0I=~o29bc#S8OBy6%t}~C4SOs>KGI!NU3lD9ctp^Zwh9i7|u|i5Nwn~-FOXp0;W0EKMZWBRw5{h$*(m2jNMVNJ0^Cw1wE9RH(Cmd?$trDZQ`(k%nE%i~wn zFP)(38ETtVzn)=IZTJ(A4bSxNf2WL9IP2Cf^}*NufCs?M2m4i~&gUHb;Tx>T%loy~ zE779%%u6JAA{80w7BY6Q(6^{#>J1x_wI^psI+2z1)YG#NfYbXZNF``Ss1kdq|F)!og z)@$*Ha)LRLvK69;9lwIxznV(a)F6DZVWhBR4j4%2P7FQ8B5pC^|AuO8&~E}S{#co- zkqsvgad&l~VmA}|J{rNU+ICDjRLMepYbQ(Oh}?8O7`zU(U; zOP=LeyJb+gFuUZy$Cb{?HYApPsPz0I=T2eM0^}p&4Z*tDiw)3q=cqa-p9cVNI0xKh z92fwd_YZn-fT}I-G9Z9c6@odyC*uB(8&}*GFtVD?M|rrpwNOsP(N)r^Dn^N6gz1{w zVovo~b)I?ft0c&7;*81%A+ccD5Fra{P63ZAYC{N#6)4+~N0h1d-#JvDvc?Ae*J2X- zq>pSqo9?t=yp$K1zu07^jBE~`=$BTQejJtZD`CG?xf-9AnCl_7VX*#2T8GiN)?S>C zm~_|U$kIDW?Pl$YJ8U& z=M44ai11sC9PqM0$Q8pB@m#*f_YIN0oZW19v4PF(!gxmN4@$+v~O$#NCwA&tgYmH}L82E4nBglEi&R$3B}p27Hp z?|dpZ$5!Eqm$WXvdVr91v9Sz|Crq<>zLWc)o*^5CYWkpJbD7mF$q)U}F8D8H%ql&L}9V9;|ys!#|n2pj~6DhfN2_ zqZgCB^YJWeejkL1D(;PN)C9NMsZAb zuQiWu{1^Nm&3Yk0k31BBMVyK(~g;!H7(EnI24tYW3X+ zIH%?QL*wgQcz5R`exJ zB6W9X%}^_D3d=g}iNH6}?Jg5efuJdo0rV5kx{Ohs6*jT;>C*}Qtl=x>Es^&4q-^1% zpY$fN=~DVK zj73bsikPZCQA=9N z`MIa;gjg*S_!l!M&D79wXYSXTa^!e-u^)_hcP+dPz^8Rt=(#TxTkEu_SBxxlX9^W4 zn-8ZRx^oDH?aWH9FP@Fv?|JC(;>W0b;Zp2&oehG{;!v)gBageq=0{7bPNlo|ZQ!>8xSkYq{=oe;!=@n;W z(6>Z5m5OCA@^?vYF0^tJ6^(|zV~?u2_I^(0wGG9?u1x->Ha1GyinW0Fwvg8VA$nnU zuh)Eo-E2FJo9b&@a-Y0WSvIB zs;;4paEes*N5SP`|68xDn_HDQbz^b;9jv;Z?sj1>WhwZ1%@7WEUl?8s2>lj1p0yc5 zJ=0=U1HRe%F)}NG@9wiOHgxT3Bh-r?p^rU@S@1JYB&zn7p$Pk^z-rEm(h>8fKc4Yf z*s+*=xf8N>Pm=QuOHnU6GWhI{WDDltzNPo0-uGQPHKGYb62C0FlY(ce9`+~_!ylBa zCM*EF54)0DOC*>5VKjq#kj_t4At_Bddm=9Gy~Ll}4I;A=X`0)u9~(_Vc4t*$Ql_ge zJly0-BU`%pDX42L2*>@8LqofmK|3QjG+g`Nuxfv=^R4~csy%lAlw943RiLy%ojbg@ zcS$AbFHzRq7g}g+GkXB9Imq7rM># z40M2->+U2q*%6<%3Wn$s!@lY{^=(N%*m&l8Kg)StM(p4r6zT|nZB{0gmxU2_j&ZrE zFj5Je?KoOYQ`9LE*P&`jSYN-XNuM)$;Q4frhQlhu4n{?JyXV5lT1b`)@u)2lk}{Ae zy?32S+w+#llsWXoU)0 zW%p!O?&&C_3%A$l)%pQVsh(iu>NfSpPaEzPW^T`SpP|cj8z%URcW%`%lVfx39U}<z# zQb;7W*pu{+wg;~{Zf2QFbYAlz9E(w-3YSJJuaJMaOhHcV`n=|j$Dm`AS~jE)9?vO; zawLCt9Ixb(Ic+E2%GS>Ysh7UM?kW=&X)w$~>5|znP}E=qXWRZ`-{(td6HI_cZ|%lL?5 zs@Rez+qf@3n~bY59R7(&$5=yv#`2b=xvoyMd})zk1aFO$82xkR)A(YT<3ZU_#Xvd_I3PT6djkY;{|XN>hOJ z(ktnxh(=!GDQSKbMfet|{;7QPwktu`4W0%9!Qiae_=yY0q>s8qc!Of!gD^?h&D%;h z1TS%4(V}GK0zwJ#AD;6J`|56ID_IU)qA^R0>qkC_Jr(uku_3g*fDw0Xb-8}qAbt1p zgu2H|f7h3Gu;~k_av6^a>q@qmPM8-jKY1HDGr?K%2d?7T9XW*=mF0oUWrn45j~4tA}rSE_HH|l|{`G!UNEu&(o-v7k3E~%2XKUQ^c>; zeJt|%L*~mP!yx{A{}(xr?Mj50Y7w`+xbJ4hn=0SV<&Qu0&y2r{ueyGBwD3~LA2wgg z+8@0kTNc1QytAsO>B5%b>ca8rEmew-UEH8xPN=P9gD8}&dwb8N;8E3;(RtDE$eocT ztxdD9712?1FFfgk_5<~SpGS+Al$kQj%~7|*=D5GG{(N<2dpVdPtH&f0LGCtu+MDyM z@ZG@il0MWShcirvRA83=C8lLLaJ`a+TK zGV9=HLj7u%>rOwu*_wQnUNe3+TtQ>kvM=CH(Y#GN8+0rG-j_#Wff+xqmy6{*qG9oC z9>!`|EY69{0Flo zKNPq+asTd$%9^!O>uKDcW)nhhGk`zOf#L+{WA3$D^m9~sUQjSU>&ER|RHo6S_NXCA z({G$Z5;a-z1D+cA%hu|Pi&{uQUx>geI( z32Vpwsu&S|`QsqVW6pqQRd6TAl8y3m^<2}og9@dAO+#W9dLKwtUc|B^G;i!xbgEgCsEZIgy{`Y2o4C*?#w|`Y9Sz3$GdD{g?k8Q9V{#dB zg5RAbF_7$zB=)e3T7Q34@17=w8EcaPKo~mQe}PJq@2h@JN3Ne1Ux(${aE<2S+v%A% z(}Wu4!c`U&2sgZo?Y1+yWN-s@E7XmjzJ8;#z5A+^8BtyRMW4^=5q6=8Y&jTB0|8^0a1N&o6X8A(iv>R0?Qx=W%6ACah4~ zb^9!7+Gs{&*71b7pjTcYHS>L>%JYxxN*gUX@jBFA{J!18Qe@G0aAeOiRcZfFDF3ly zTZP*h2A5{vlY~3kKPHLj8)*^w^v}Hm>1Pqj+7RAe zS{>=fkaP?E)XXIwPnB!$#i!U3)3de+k+&&YMntVt>=f8`YXW131{YU;UT5`%QgZpF ze?PJhcp~NNO3}nvF@MI|-_WhIMeLJ!k^)vqB%mp1zH*J^w znyH0AdT*{BvkzYalodTYzp1OWCrtxyJ2V=W#X%Ew=Th&m%i{2Jf=7pXwz5D& zuTU)s&>BQhL-2^|cfh!i2cbfiY6AlSLe6|S$FS)|b!PQXzv}lKPnFTdX-1F3)YWh<@w%@$zin3Bs9a~^&vW%O+Aj4IX-@vnG+>{8?lE9t+*Km2{hV2f)h z!MnbYYwBH=L|>sYQy^-wCWSPx`76SUhSZ9@lzYlzltr;v^ss>F^O{oSkXX@LG|#d- zmfEd#eJ7;avp`}+#mf8FRdeUK-K{YF6Iq+ppVk{`cn$cY4>Oa|?2_P4;%BdiXP>&< zCR(bwUH!1Tg}xl6cByZ)nP){VmHEm$j5Ag0N7ZVI%K1QQ361Z>8{RB&U394){tAhl z{+yvt*r*EL?Z%bzP=qgji$SuK791Mt8#@qRa!%9M-Os*doFmRF`A~GIZ`ps5h`?bW zWaS&JOHndghV+pa9POwPR%5yzDev{mUsECf(?c?t9SLMedf)v?Ve7lAV+M)evIchF ziuZ=i5x-=6bX@7G8!+a7_Z5}J;@(e2HpgwF%&26?%z zdsEUUYj5R;x_3elDs$W&@-{co6k)!7RUL%C#60)y9$*H}G@tP0PkV@q)0>!$swEpY z3Ji;6RlQ;LXk$?nxnM8<@X8Q%_?W30EzP#7`~8<5(9w^Zq2|RgVGC@msCz^ zrVWU*)mAmg!(;C(9e$lW{AVY~E0kw34cKA3fXnm$r{!F~KisJ`_VmiTNqc5f1v)|G zk&|*fi*z~q###(Dy3`Gi2h`0Uw!Lwm^aO>1eo!-W*8+J;Tvspo)!FD-u-Y8 zutIAe7gNht@a$WHJ$nvwZS7X5leNuP6Bl-a%NAGq1Ph;$ijGeHIteo5(l#h8C0rCH z^wVc6efIiZ;!vwt5|!5<%C?+_SQDvn9^!gC&+H4~QJ<+A!8crsRMl1ULrHD@ud1sl z^Be6JA5?c-y}(fRJmPiLYtAN2o5R`#?K`EVcyY~33eUSP$1mr`K4egmY+vX6gGI!E zol*9|ofZxE%u}rgpCRK*8oj14q2~PoFQMcP5Bn3=$#2jDuj8kY9LjTJeY2|kjQb+v zA1VY%D3Z@U5S3H!Ne~sY?mxX3Fo=3@;UR8=5tDPS4_-tN==ZJ_i&ayPYbz`R0;&`o z+L08Uwq^Z&KHLNk?~yg-ze1U?XA*qLN}zW2ZRGggieAjX4sKRTFv%gp;+)y^)|`IVW7(VolH}?CBbf!y+WA&AdaB-_G-NiJ zv)=U*RI|w;-P~F8o7po4tO>i5&_Tl+pC6xfIveqG%}r{$#T{)&+laF<8k(Oysh?^( z9OD|vq_-_^{qVn0*IE+0nEhgW?tXOD(+(9cVsn|sbeM9}_b1#$Niuhtar2iLKQ8fz z6U{^&8_r8ylVmb(znGX$}14wh}Ti`QFe3QpYW8PsD|<5grhGLcHDAbPUZh`-uRv-y;}ur zO&#DCUjI{T`n{t@t%#KO=4CSTal-$qq_A$awpq!0d14ZsT`7)<`Q2%@3M&1GnOAEctZtWXJ!u* z=X*;qRg2_3r#y)Krn<3ieZ9R`6@B%zbu9YTl9uc%`+Ncx7x}~Ij$vUjJo)1VDrK_i zq$1`vovYe20usTbXBYH)&q|n$lQysCx5rI>r(G&_vyzG~r1);mD-Kk|tn~42EJB_Q zcubDShTev4%ti#j4#2H?0v(<$-7*TJJSF+|F<$z`=)^8tVD!y-g{Ut+1jlTx9WOrj zSVz9($dFD{`FSbCDeF}(OkJJFVw$AD*0b`KE^z}T&sV7i`{&oKw1#W6Yta<~&%_}4 zy~FweHkXxK8F6{kTNWawb7jif-S`dF|%I;CnQb`qh9CKFEJ^wFsEd<*yO zj68cVeHKnzzIG)yI>P#{UC-T&1nB#<1gKup_|;3XNCln!&X?m%H)R7=M~)wsZVn6v z*LxZ>lKRr}(Oi0b!DH7pEdAhyP)vhG@aLz!mphuSReO$PQ<*f}T6q?c&3W@I&e35` zy2&9>ZT#1Z*CBn{eq`1UI&N;WRQbL^zAd35{ShL?&4PR1dGYMZT@u_CG3Mt;!L;FP zxkIJBT`nzZAB#I?-41`I741jhv4KEchkaR*gwI@&n?h=t(vf_1>{6TAU3K3=@l&AD{p7YUF(xMWrK1Zt;ajZA- zucbAy8ZM>Wug!MPphLU=ynWt(uH^ZzkInwQIQc(5Hv2Q^_dCgdA3^f}^N;Q*z&`2x zx(s|BvkdXp1%UH^a04K~!P{Qn&)dgeBETik)6UD*)kECP$I%%-hhGI~wbV7$0RjR7 z;12i);8CRQnyRWc`UZOHns?Q}2mm0hwDs}vxWEGdo?Zcd1{$|`%*-u#Naw+m(b&Ml zk=OxBTYG;W6@C4?=h>g*o|yWY7XU6x{Q`z71O@njFe&&p2VnFA83bX;bJ+P$`1Uz$_a|)j8^_2%6~sB` zmD=9L)&YcJAS`P4AK}jb0sqMZhyl{kaP)HYbG7F=*9E9qCsz;0-w}V_{)g!Q!2G`g zJv{-4(;@V8v(?fE zVRjIX_wzP5&kN+4u*A_r<2)~rKH;EGfd0RDAl&rw&^^x!l$VIw(f_xMAWg!}02d>$ zo}Q;Ak_+%N`d3;a3ny1~&41y;d^~>FMUXe5GQU8BbKXGuM4xQ^)HFaCq(k)G(aZQ; zZy-#}>R@~OT({yN3;|3CYyn5W8?XcK_J9{~2Jip|fCg|2eEI->V2l&s3V47aj$o`K z7~)Dm1-OG@|BR>q8|Tk7e`7fRPQ7`a8qDRNasJF<>t8tDL@Y#-MAG2D6470v8$=33 z*8v_PDA7$K2+?gY^g5B!UmWQF2cJBEGq446`X?P?+!vw)Tk*Np(+U4v6{T}{F4;*+$ z0l?o$`r7Z`1Q&0CebX`?{{!r=NYeq}>z`~|BM2!b8>dH4kYh;s#q{)KbiSD#+e{`J_`F}*)Hs=zt4!1cY=1_jogLSkYfP!R;-U_3xXM@)ZNN`-{so-OGW zU&d>XpA?hv-fHY+G8ji+mA3PXB&T3zVP#|I;};MV5|)vblUKN|sCrvXUE_|XmZ6cc zi78mT_709t&MvNQ{sDmxgMvdsqoQMC6zKN`2{3u?aSBojm@oX%=aG$hetoLzmCu4I+yd`*FVYy zDuLhv5fLE~>A7447lO_Orz0Z1EJZ@Ea*x#3m*L8_$7GDRo)kCslJiO%pqcFa#wnPu z${_hL=c4_V?0-+N$p2fC{UzAnaxDNC3BgK9M@R=i0US;ccz)p#1!w~NfB!)f$$!=f zgH~XF4B}X)aje*hF~l+lCICl+9!lbEowiRxaaEPW%=I2Y_*&4PW?|Qncz|-mAGx2R zIQtkTRb^F?;?*hg!o^yy5hD?QYKq)C=(?}c3T$}cfz@(6K+HET2D5)2?2n?_c%=IT zCBa~c9qZ?!!$`z5pJ^bvDKJHmr<7~2VNAB-F;#9j$=ObB6b*}xeH}JJKII5>MS7EB>w9sc zdgSg<>O(xhbpP}6raty$G+itLajU7jTSudBg352=yBT2#5YBg#7(8Z7pEMFT31{k5 zMVqcxBt`H9Yp9QdNm=gO1WWWSGfZ}BgoN26SFlZLTkBLsN@im|2;bEty+=Mud=Vb~B(K-XF>j%kWtaOdWEZm$ih1wL&YX{L9H8O->mB~stv`kz zZjB0Jxpl>DKCF?cy=ft%N{TIvEtbL z4j!Og9OOP@Ie^#V0gbU^>1BR6>c=}gaC|2YOpQytn~ewPV0w7qb}=5H*g4w31EsZR zg*_OmPFp-M4%avyJAi>LLrm==~BQ_){U)~Sk0i(OEEgDfJ(E`OAu#EPJx(aPrK!5aM?cp#~z zp3K@hm;(ik^-$J9j?}XTH!Tkhba-{~KdoQhoMjJA8o~I-DCouce&)#ckw5N@L)y?P z(1@&Az7j;`d)ee}%oB&K|N2R}5dx7`9B$gpCMft++|MbIXLqf3B1<#FxZh!}wnO1V zYph}+-&h3K2kDEBq9atfXS%_srE>LIrv0lzCJ3aIuNjOVN|jI%+iyBx-1pJ!ZUh%= zBK?4!{Y4$cj|Hw1nPR(*r@B~w`Tm-fD5>BdeKZ(|xq{hL5+G8mJn9Okz8+=?9 z(SuD`=RRX4J>{`!t2n#FvfGFU;&m{pAok0&!?evkJfNO2uzzX;l;{lXF4*cMOPoWGZZ7&VY zVvIFL)yJ*@Z49`k& zGMg4p$KpCeP~H?)?%gaiFg|UzB-gn+aJm3HBT}f7$E(e3* zjnT@frx%@gsf6c3+g*g)y*-fe669MpYkHdtbyD>fj+|1xc+lL*;al-%iqhDT%LC>rN!=!5g#{^AdO>|da; z=2ttC;P;u|HtMOCm+a+>Cd_pf3Pv%aCd*c$;CAtn3u4kJ-P2K2DZ3(Tjc&3-8?CdV-pWNi95ZE z2PQahY24gkomWl417F4jCnFT#y&Nq)tV7649B+$gT5hh;%7QJ(yME!61e+YdjI9AJ z5xQU+ggh5{)6`WoyGqUbz#5o3dO9ULA>f$=>~dhWwOaB0ok{ms8WmDt2Zcba9P>8O zoXU^m0UkV%P;{&cA75L;1E+UarS;QLZYw&+b{e9Skj{fTES4C@<LM=h2mKkuDkg{bLROOZr<_(VYL0cJ6X; z9fUZT5 z{UNhsP!*3cOnhZQTSY!I;{VxE`6vI-WO8Oj^yoF%LaMab2ej-&07gOdA#*neot8m` zNq#YZYSGbMjA=Z~^F;QP86%%0d%rHMKR4}Bv(Zf8A}XcQJ!k=@^FpWAO^;oJGdJsn z$G}mk=)D2XwpU|UeQzi8TCdPxuph=E+ub>G?1b$wElv5?`cc*MGXuuuEb3;7ABEJ7 zYfhl<7Mf~WW#xQ~1=$G)kKg%4uwrWZJ2QCR>JK+rax*&_jQDj5r}sJLSX$1=6cyt7L)6#1t$PV>Gf0S~kc_13I#o@%3=2jGvb zNilkTcp!>-p{+mfnYU~Yxiz+9eL^C&PzigvYGnXjWE0tkunJ2{S)o)mM%CT=w$V>n zHc|}l<3&9~6%9$vyIv*OISGVOPM+GX711o=hOLEjw-@`D6_fO*JH;v@#w_ZxrcXD}UiWPxJ~m0us61{)rj_Xp)e8}z-g z`Im@+wuOq=2+H7>lV+)i-DO#f_B@Op-4ib=BVr}77TXhE=0>N3;eyh?C#$u3o}5Zmb}6X~pJB(`-`UsNM0H);J7`p*#x3Fn86unM%%Wg^E9Z9x+}- zAzKh+IbHwFTEt4}&UiqQ$daMSR6SqpnC@hqxVP7O>;k!i%~C{4yN9mQz;3ogukFs7 z)URwRY<8C8oT!*^PO>%h+J{c5HBM@qN4>}@^}t56&k^!;s^n39pQ3kAzQ$GhrR399 zWIa#UEdD4*v?oqGvj@1?e-}Qio=T5VG!k^42N#c4f7DTDtTKq+Hmc$;B~v%dxTllZ z<6iX0h65tzwsNu49ZiaQx{vVYJe8k}kg|0BZhfkQtwQz?Y&-dw(g%lAeGo_ z@cC>UrG{-K*iYD?EB2z$+IOd6K^3)k+P6Z21D4%V^eY?%4HC}2ywlUNvB$USonXzHriU>KwTZ&9W8M35 z#h>`rxCIrm=urGHaD`A|od+c6T(T^@6Y6E&D0y>ei`C-z*`2!Nlh!5<&J3mqztAiw z9m(b4DD+N*;k~9%2MNkkJH4pyrw(UL%4{zX7<(P`X`x~gViro#Npr%4o$Oci`2>z0 zj98TxAqj6B=MEPzjY{j!@+?`ga)WwC9(fAysJ%X$9JIUvlb}B17PNDjMt}xGJWF)H zqAXI_*!W^lVubJ-+PL6)xvVzZ;{n^ocJr1I8}fwWgt86wmg^&5gWY_pQr5_C-OGp1 zRR6J_WRP}4G>)g-MBdbLYP&C>T4b5&vVzt`a{4Ja!Hx6*2gn-G7-89E0jIjGShW+q zw8dplIFh^1h6WFKFS|t$qsvz!AtI-07{{+iRTXYUB(n7s*j#u*1XU|m=q1)JxM&a} znm52=EjC>`)#6QwweN$_PG@LhtI8vo!|$CjT4DVMx#^U_xf8JiRu5e8BoD12zpC4b^}JHPjsn=U+#O+vu_FRvcR7oU$b3t0WITD31Sn zWH5k~eD=~bu^*Ya#@!hZN&QWY6Fz+Bzpr`RW$7+)Li0-pPb%Q z)fxMeTaQBMRFCdZMJYMaf7xGow7GH>A}ncY};zsea*vQSfF#s*o)TK3H5Hy zmX2)mj?{G#9eK3<4s{rgycZ7;+;oSDROz6PZ zxY$#$2GMRao3Joj@F!HcWiz)ng(I;lzZ$vsDnQ{rot48hkvH@Pdz#AQ`r!#hY=>iy zxXDsBl6stJVpnLrl{M;McP8p#8pXjXI6gOC1JLqLIez&yeXu8>HNIE&Y9e5f>e_^-s`2#KmdQPyS8dtGv$FR`!z`xC zVwb^98P06(Pe~l6=^0Kp%-{VsqsBD;cAhAU&wPJ;ks#~ zjjLY>(~YW>kE?{kjh;+A5`?6cy{}o^`RK96ct1Bs<1H=)-xD0M^yPfRRZgDN0 zHX*%!aS?E?`_dr)m8pl=R73k(KEa(wSB29$<+T`BWAOZc6LZG+DX#@}a%7-GCiyFMo z*s^y)7fq}L5*%-(1O8u3ckJG4O^UWbcvEh1kzs?7+!upsdLy{o(q~oBX;A@}BoJI| z-Gi`*Lb^l0vauz7tG7cWnpnBhxb52^5(7~yUE%J!4i`Lki(nm>;J<14Q# zWB$TV>T)*B*X@%>Us;Bm8OE?04?3|P&m~*)`;_Vy2x_&s4$D;aw#Wlk;b*oz_)R|06 z#|ee?XG^Q{j?a3cX~zO&!LL!-fFriHG*Vf04Z=9=+rF|VYHj=nhRZ$6F(-LwPL=Ac z*pwh( z=s`&59X!zAb{!9_zZ$~>2EnShgj&#@O@)QyZbFeO(1w*WcAH&1=-`We(+5ATE5-4f zZV`fh@Vx%CQ*HkuoIA~-Ks0nS_dOiF5(m0B2yx)i;#6QYE60HZvF@D$cwl$|`hUu< z;eX9-FM>N_2ppa2``}Dw!8^!`-5s?3Tb%5KU00kY#yhtAQt*8(3|9tMxvN}J<;$ma z_g$+hUE}l{+S?nt-8tgfzYU_olw#8tTgsN^P?1`Wn0;d5X=8}+pSVf$+y zvk5<7PNpe^%*|2?rpa%o2S%upj$8Q)lyxuJ2~8RKGH%VjC^ezW-RboiqIPW0v6p1y zGL9aYG&I7frewJ3wz4fC^$t!k=`InIwjsV`xh$G?66@;Q+qDY* z%5O;X9bd;9<`@{G*m zpme!W%iIHGPj6OFdRIIX$Bx>M$;TapwF_DV&g zq^qWTRLt4-$S-GAbuiUpR|eN>TY4DERr$zHE0lT$qU|WRC=Q&CnYYZslq$AyS0)0f zpt@7Vv#V%awqIZvN4T{@@f58tL@}G%x0N|x5(}rjWWmVGZsg`Ugfwa3 z)8MMBoiArqm=>l_KhQB`m~$cn-iYedKOl!3KGe14q7af7o~dRQp86H$6Vdc&C>v;jDe+@)b*p*^nS$r2*#2*rbjkYU+2O!k&!;QCT$CC47IC0JuJWwn>UUBv$;)<2c+;$t3 z5kmUeE!TmItz|M$2oKoz#7^359W7JX;i%dgJ2M8vd|gSS3!a@yU!>RIiu`b2>%M_z z=?P65bhEP;j=B_s2R7cq>mfw&@z{@WjL$kAI3q8*j$MjihOuJgidJcuBfx20Q%{Vt z?dtv8?Tu*9xZOPT@<~7BLeuO))$Z!kW*X|?R6L-Mwq!qw3kmde^AZavn~_P%%c;Xn zI*Ru8=plEdJ#z|xRM4!@$9Gw(3#2zh9m;~Z=$Zm-N| zvt+jxS!0E)wkUu*%VnS&mnF{F{XXAC2d|J|rhEg|$c+&aZ2qC`iBgrXtA<8qKgN9sg~tXa`NN-tg_kTX zrG#lr9#(vw=&iB|4l+$us>^1WTVk3EpFd;3WwwALDZJU$u!ikcppP=N4-0ePy*gD9 zTOU7$X%?K-Y||~bM(9*o zg>IqqZ6DxuQ`!>K0` z5OVZU2C{&fG7rjXy_BM)@hWk6S$k(Eh4`h!{>GiolarI!u%yNJ&K(8It_5i`j@)WJ ztbWS%67Ro*wz#xgFy|}MxVK1qL+;G!s$HL{H7)uYlaP!8i^+Q%H71@P>om2>7Blmp zd?@hUYXhh6fU@eKvT7mHmu-b%gDo~B*SI*UE*8=0WS>x#Sa(sJIqTBBKBSOVMN=Qr zWl+XvF4&-0OR7Rw=pi?Hr3h5?8O!;5l+~I6`04X{4ICe0aOGJ3R|SsgEBEPlNYGgT zGVPcHbc^+x)#LOvY4#3JI5lOYEF(Te0OMu<|mW*RTaMXALuiMOoKo4H!Z zJ}Gp*V=JGH7JYT+$z|6|Mgw}K*{M=7nv>|+1gWM9-9T$@k>a>f%w&vU$5TfQiD3-u zhljlLlgZUaMv{k|m)UeR)x}=4kHzF^nJ-hPXJ&?A=nL7?{T?Ph^|&oG5wqCjpZmtu zdXL*W_9CIs#acp<3uF(}Rfv34I3pE!b5D(5l6xc;$w^DBPIbuWWww_i_W8 zOW0qSSD;(-)lgp%Bg_i(^hFIEw2vv_4s}Z}60>yRG95E*6)tkonpMkN3*jxbETQ@G ziZM$|yV{<%R394oriXi-eRmzM%Yv_DrAxDHcaZ4G8a*a!V~jz*%Vc>}-^`#%W>>zj zQD%WKLyCerO<`>@n}W)487zNazVyl0=~u!B?x@RJFWA#dW0UzT$}O$=Q<>APCjyYH z)lW->GcsqzWQEjp-6XTur9t&S2q5&k%T2sjHW^gVMt^EGd3oxhIr3wEA!Xr`qhl{E z9&n>;-}~YFgZm|n@SgUQNz^Z$I^l^`mi~CbwaV z88~w%)Q~JSr4l-6cAu)#H0YnKVjZJNDmRERvy=S39#B>{CQ_t^_;pCl(`Pzx4}p># zqe`xOT&G7{TxTI!Zn}WnzGV;t6SD~1nku%IGv*cf4j#Dv>&MdXSqIHD^Tr}~VrrSyI(fAk!!FpOoJ9`nzKXk66skU1*|SuHk*6@Do| zLs1>@G*DM6b89SX%W@m;oQ4&$c??lJH9+_Dp40=|-OL#8jz_un-P-ZjGOGt z#VE@4X>lS5*L2-wcO5zRVCUv>*{dv*^N66MJI?Mo4IkP$7@~$7f3+yiEf_Uz2Z)&T zIfuwe^A4Gm+OVjSRIhFs6^f@`k*TqB>G(XCXxaRAl;yewjRz?k5))ELDpJ77XYnk0 z$DlOv{SO$we`KjeEPJ=u#K2UvPfO)+x#$gVhZ(L4nKjq%>fde3pQ4=?VgQY+jkk)| z(-oeSCGt!tmOOKQ$`dL1;pR0RnI{N(=YaHgPo6tF5?#;&zH_J9u)&m0Ah0W#wm~Sf z*>-4l+$l}Pg3Y#EsKTq00I&rbH;mWCXXtk{y=rSt{9({mbg#2;RV=mO($Z=w^7;Lc z@Z>mwsv@VkyVY4Dl#y9NGkybW)rw{oj1Ab(x$0>Vqk9=zGA>hJJG2bHCYO`7Df}{D zcGtT)B4u817b3vId95sqgOu2m!MZ&3pkbJ@-^Z6AyfX2qgN-NG0L{r{Ew&x=&?C z@AaJuxi+X8)L~^Iug|8X9#NoyqKR%!TA~+Y=>NG)JH7i&mI`Xtq}UY`n^*Z+vvh|& z?8>%9s9JTe$B)l*TS30t%s*!)%!E~#28KRhxM_@#yXw^WjYf?Djaa@^#8jky4F%l1 z-_}`!NlRA5ur#6KK9#XixgK47vZ>g%aQ*@Ez?>vZ??!QA6{oDFm?g8=>Q0~M$NY_5 zK|5PhDii&3gI2C`Q8jsrr-J&KMY2`xejnlbvmXo5`o!8NvCWNLZ`7!dTC?RT>I!N< z^?i@=@Q)RVO7#g|Z$%gwV;BYBH`kbyYqoS43#*IOELjNzMwETDY{+i|%V)=79@BHm zfanXQ0nL1SiTVBh4yOWdnuLgJFmbfT&{$&k+@%!fwUBis^*VeNn)))$}Mu?4_USBWqzfn@ zy+|h@Oe@AU zx~FW$CAIIBTo?);c& z%!RqHx-~eC{i=`QfIS8*bLD0U!MfSGp#WZJW}B2#ey|I$1*cL#@_l|0D$IuC!q9ha z(DkSULAY!E^>3o_fnUwfQAY2wpMd%SmMl!j3e*1@bsfoA-KV7s+Gr*RjL2>LS(q($ z!V|5mfoAVuCGPNM;~HzchSVF)Y^ys0q2s5pc8mHOtZ0Q!GwIkxMa-!zWRY-KA{)

Kr86+j3E^FP_e>1*#WvVHm zHaFK{UoY25Fe5w9!PoPnyOH_o*S&ZRM?V`+^4bZ#;T}TFJDTYHe4>~xn-9Ggt&&g} zN%Hm+5m_;B7R&Fh0nQTvTe@OLWCiq=21#2pJ2jB2QZLIa8mSj!pp<*9%S}VkNOygt zK&eJjFWZPZS1$qhSgLwWu`5^Vaf(|S58a}qs^ojYJivuQxkpzO+&)h%qmSNF2iU0f z$h?|zgC3oY+)L%!F%b?6V8QY%@_%A*(&Z}-ei#ok zJOUMu0}DxI{9-WhJ>bbX0<9|~vM-0cu1%Pk*5ZEP6~CtB>bQI@*V)YO{a1ajgId{j z{vAW}rkaDV;D#0~2X!_uSiqt+`g-=w{MaS#yZ3H6d(4x*r;;b(?YhSKnE&3>C+y6I zs{5MfqZ64!rZxCn#~2l>ho8c0cEVpPHcg1yFc+2DzjJ=JGfAdJ-BBPq1C1<<)7+3q zu(_hT=X@o)c(*!!GA|$tKa#@UhgmWTXk}Dw1e&LFsxcA0 zYDS0cF2vHCm1AvCvYL)fJmxi#?Kd&S`xkS41p7luuCq2(BgErhb}{w4q{xNiy#W_R z3Vfl8{R{13F&JaPPTcZsY>;@G9j)XNY+jSp=0Xm$%S7Q37-nSI8({`$P8& z52KEg$4=+NOviWOG}x%U7;M#Hvu)&ZE1Wz)7&>oKH_<4Io;Is7)ibA^#&)9w!$Rd~ zSDM9Z(4KFJH#$M^v@jg4;3RzEd1j3QCoLyxIZDpP@-%qL8?=4K|2j_5UzqxuGGn6X zjQM_DyHOtCtA8RyRyg`?lU2>Yq*D}-sDWgdQfbRax9ClOV2?`3&fT(*115{2Qbof% zPpWdo!I5O6-?+9RA0>R?40T>dps_-YI^c*cE`1WBO%&o+}`6N0t1XDS($Q}&4dY}zG*we!hN66@- z{N{t*0jn)wb_q=%*l6R4GUaLA*AK-Dt)?V}?J_rAn38T`UX^+(zF$1v?ELfub_}zQ zCjri^hy*d_kG=?AMd`PfP4O?5K66=YA#pTgLVJgT7fRV##8Ug1Q9>avS&fks9exQI z;otx^sJm)4)?$&_l(RdyW2!6ywnaK`FK)((-U~*x&_Pm&L&aPv*If3zf#^ANVjKxp z5veuB-aW^LEx-)4XHqr_zJTdrpCHXCRgtE|-E0yg9;uERejVddWPKMVStQxZA_;3dJ|@v+)kNo#H0pCOpr#?ukBjdAM5( z+T{d0y3P0b+qc-hMQtS?#;T#qS`)&yKB^IvaxB;$&H%e! zJ0gI@)J4fq9(dx;qw9ij8mMHX;)LfG3Dnj+SW2Lo6+K>3K$f~esSvL+{BgMY-9|B6 z+t9*gAk-lfeQV$$ybjSmKP9{ugvsv|6kcR+pAQCxU?@(*1~b*Y7mY1*#3UYp7AH1H ztXz;A%x~b-0VjdAjL=h=^Vv{-WL?-I+78hOO#z4C&ST`p8u_yA=g2Uj`3qmM*;W&Q zN1%XK_y;p)fami4b?Cw)(8TK_(5k>8AyyR=X+F7#C!t37>;;Cko1>rgp|9de+7||* z??Ax^hF{pRYD-v~gDa3`F(9&Hx**rF>;({&*&Hs~vSB?W7Bi#PBD8xKoK8l$IG~fA>a!=^oo&6F^8Yzta*TEAjsD?ns@j(%7a3Pe`Vqr^H9VKUb&@*JulN%dlz zRmcxD^YkmePAkOzMetTVu*EEKcgtBWpq~@YLIJEe!{BzK(AO3VT0w5m!e!KEm?P%E z*o6wB&yhL?c0mmk#4w z9JL4oCh!y(mlzq!g{^Qil1g+~ln0#DbTyMoP=*~+w8q&1CktSKS)jj$i*};*V`SX( z#KVfA5~!rmyoX{GT3Qn&Vhp>|6#5DyA8R5?ez7$bhl={d$;=c^!29 z2*jC53IF^SdkrvCY?`nZ(j!oV9s8cH8xOw3jrhY|Y{V`x1xBG;4@}m~1e{SnE0{8I zU~*|MBY@U*8-;`vIn_5+pkp7tga&O6Pe!WOqCQc`saROCBtIGb46m05%TnTKmfP^e ziC>km8|dJ!jFXsvs63qF7)~8IPqLOG9JT2oNRRAC- zFp9~vMm${W8*%#JYuKG9~9*^?a$0t zUND5Y5Ne^lmKRIoi`MG>zas9-2)g88%aF@qIL|_;5wkFz`MU4ZIVykARkwAi+~%ay zr|P2M3oinwlNRn^2hc^x`4|~9A(-(?NQ!cpn2!u)*80>ykcY*d zqxbMEnXO=ofb)<5`dSRdau`gp0J?ZRRVnj&C@*)S{F!c~@b=ol755puJ~+*jHNR%*lx?3DPHb&tJOB z>DhQ`IBoUrqcYO!d?`;(Ud=nO4~=Z}>|q6ux(0J2+V3vyJ$1aUN#&a6pp%d>lCKS% zojCx%&J~a-xkV?nRst&(F8D&)$Xsm}&*!QIuLj7iPuGjEv-Ev}4Xz(iR*%-5X`tFle!P1H67 zCcLxT_e&FqmOk!C54d@Il26b}P^rsos|r*~`xCM8_S+*@EbNR)V%m|AOAHP^rFWk$ zlTqBb@-$oNQI!eZ6%YH?eodMJwX0t9L!!E(! zvg%HBk$$XkbX!}EA#bNy3dFMNyqk1Fi`*zRQSytx5{!Ph3}A< ztiE2AdMe~|sb)I7Y)z`BEBULqzPFb?6=d63RV;mMEGK&WE49)$6c!=gWl^}0Z%i51 zrvGuji(aUwaweZLTz5^WL@Ycv0eRWAjLFl2v^>T}va9&~9d5}2{dX4L1j=t;%a zQ7?B6STrpVy;s$~u&St~tKKepW?A8$#W#|++OM0&eaSst{2MnHzOsyb@5Ya?Jsz_` zQ83NQR8kmBJqJ>ce^@-FVNqmX-nH|3 z273l!l6kg88YO{ zQl`)La}2{JqN&$h%*Y%Kg-hecMIC#-3Euy_k<;7&+5A{nUGwmheF;)`TjX0JShR#i z$ttQWx`fm%ThsH?^5>Ur+E01!AX=58xv#w!g(%T1^7VD`y_+9xwZRJ1K7D!baPTtUY!1+MAQABQ9?U;#6yr zKxcB|O`*j1y(sFeE1+IL*h|^q*t0P_t@-{}@F92RWqd0XT;Bcpo^S9055B*7As8Lp zI-lVPtQ+)kIrOo>-Z#4&<&CeM1lD!HVudRVuu78>Pk_4(Risd19L)f*6uNdGjk|_b zbU%>C4;WEykf5zllwD*lWH8n83DG8S$2__e#+MJ}=?dU)!Dd=5m@2oK@rHG4A@hbf z5e9>Fi|r5Ek+RAiDKoyEJrFn=v7Q;rt^&&h)D2M^Rx1l8F_wcx9LQpK-oWW`tVmX8 zb848_Fw6?=r-wEVh3UM&$)kDN?;w>hThWb9=)%3}cJ^uTIT_B6IF$(;8B#cOzi$H( z1*1T+pTib*XVVXOLHvwN~Eu5*ww0E}B`X%V5F?A?OP7*9ZH~60iA#V15S- zN1(U)n87`8E&Yah@H!&K?2-;5`y}Nn&X-~b-mxv)*U(?_y@+7=eyjD?w)eM8;kK5| z$?n4h7GP)=nOxta4Y=ShfBTfRuh5e-RemVm>W0(18|t@4M4c22)m7+qn}$V=OU%Bi z00A69WY0c}_VmF`>MLsEl!C?=^+#<(sMOz0$y0C?KL36b_WSD|h*+K&(+ZUEMyzAG_vW*OrO}bjT(B9;m5b> z#Py_P)0y+}$i`5O~NgYEFx^wH>ALe5i%njca6`wrZC{qrT zkrT>t3}$Ux@s&?7fz(=_Jrf!zGr!0a+GxB8egvk4u`jSfvHIO+0$7*DObV!Q$9ln< zFF(}1vwka7OhdRm1wHOzP%4(rwX)wimhoIn_9DryG=a-r(dnG}NUnR711oqUc&jQ- z+Rhtkej;Fm{mhpw3IqGOYlFCJWzfDYJSVnPL%EBc&f7i%JZ|0vgtVqxYA7RH+9=OK z5$5Q+z|BUtQmEf;Y}CR)7;sHs)@vQPnT>KeXB-$1f)lL9M%kec+MeJ{MypwMP##md z-gv3_wivGFbLAMJ0)hI(0>GU_JUg?}%sT2Rd>U$o+GrF*np&9*iOy3wnz0XiY#4Xc zE}A!vJ`0py`yNeA@Az|I1dxP@UWTR2qjY(9$-4)r(!(XG(yOghfdH&0X> z&kBun_;PO)mzzvE>~nX{_-3Sfy2xCeo-Xf$=~V5utSm@h7KGrO&s5vAiAu+-Qk#rz4O(oOm4w&&8VOt?U<1HEo4? zr1P6#=E_QO9OJkEVHo%GfWASaSxB8D(maBRG8AViHqBqc-qy^CZmP*|ZvfLk=D08} z;ep^=D}UW=#TG5reV)KDcjKinUpwZ**lBps;K<$i5c5-bMyR!sP(t?Hs+So#wjH&_ zj?-||Vl4}XsV{<0&A@J=y;|%R!fU{!GpT;!b@&fsIO&pkh#X7iEzpiS`(*oGeHAti zIBnjNlbK$BVm~p}fVFG$W|PInC4Grtx#cbnn`1bz8P!cR%bI1a4g(z3fHPZ z=iq74p35D=OfXd}1i3}oIFByddLy+(7%-4o{V4+A>mxR9jQ0j>DK=xG9f;(;J@i@Z zP+4_YM#skqy$)!alDz7{T`&a}q24mgjo}FixU`lM8n}uqDk_lU%a5S$7zDOTX6(is zl#MUHvqFvv9o7n;ir<5eaADohlwFyGICHeJ%Il|q+c&?=Sn;7MXQam^Pov7?(?p=Rq@5XfbSQ2# zvy&Gf*Ty=f92)D(B=)`swPaF$*f|27nrDK4Q#VuQ8EU}?$pFF6ZE}P_M9g9y4uVSq zLIQ%Xy5Mys;2Zm@Jer5Gm0lkKmje#+ZmKhh=YTrBSY0@qO8H`}RyO>Q5IPHp07h~I z>T3-U#cR?7i`-{`k-9_!-qc4Q-;dEIJN5KMrD{UD#2M~7WE9pIaym_vSGci zxpH@JDY=KV+a(thk&b^XF{enh6gijA=f`fRpO|CowrEt5AZ8ru8Vl)OznuGQkwKtYObmb+t4X-dmdWW zZ6=NhZ7IOEbZ5w7?#`AprKintV(Ox6`rJGA+g8V=0^*(g3f1khSL>bQU2H6zzHPC) zm0F-EX>Cn(k}a;_Dp1J4H%pU3ca-wjd62j;uAcH|!e`K%ca&<3c997mud$gYElJWx zc38S4*jt=8XN~N%azKfQ`A6pqWXCV?F7S?)m`~P^Iif14bbx%3(84hhgBj-wckV=$ zMRkssgcrR#trNW_37ii+wK%=s1an*caew;6_iNNAB2Z3LU}~sK2OD$B+GZC!pvYrch+zwm0J#QBU*~K{(V1_})N_c#(yj z|A74(rrbFNA7m2Uvl*3^S7Y^pbTF8NbBQovr>5eHxP$M-GPBIUT$sT**<@-y;$r+v{FAMnIn^4Q7%@+A?fnE+=V;dj=&`u7#sf}a@b>*=N1Nx5X9-{ zu6nsTnq3`r#Z{k>M{(C(s_>1#h@vd^aahyMe@c zuq%g&_={MAZXhT(vEV=pIId}AV;u!FLRcuUv;Ka`M|x3^%{+*W7;BZgqBjK&2jZ9! zs)I>NnC=iUt;oT3?|ScuM&*LF0KPFNu|s`*4C{EsV#$4i4YmI((i{BWl7`{`(fR+? zR2fH;x77&%@r! zqX_kHjwcmw=itd2pG}%>bu4!AAr%H$jJ7^+HTi6PmHT8>W+tu4AjNezuZlXsKI>fC^oaag12UwH0M?Y)p3nO>^h(1Q8ikBPpVx^C z#3mjmo+U`0N^1xPQAjF#2J};&9fH5>OdChMZ zH?2RA`2zS!lYx0{qr>8r5t?iPE}qwdBwVEmNt2N!;xlW4lRt>yN}3oqdmxw38d+?NAR=5k{W1VWS&1Gna;fF z_f!%o*T6iJ%h{a+30je`LPjikwx#1XG(McA9&vbHCcPn*wK_f5+q3QYA z=hcXReV^jZqeCZOV#J8-$pn!Gy{3&}mj(}X=GO4e+BcNnPBp7S{YJ3+IsU0 zOoeg1fP}LyXY!G{!J`*Kmb)7f)japyC zmzcau2y$_`VrU9nnE{d`ra&V5@8|ZUSh~v-zgM5QS8h%1C2@N1=9_J%w4>coS9ELi z%@l9u>aK!pN^;7*|EDG zM@FO*cOC5^R#9zaETE{|chAbxM@;^Fe@g4w*hci#cO3TGTf`#Ftxb1cKI)sg>NrHR zI=V;!*Be99S!uRP+N6;bc$G&xk@pqJ=n9NoH9SbtXO@~2>N=ry)vz}*!Bgc4yqENJ z4YSb|mblw@gH0KSW+uBEskbXGyr5TBwtv%Q$*zEXsI#GUlY7P_IcS{HwJ&c{yliGJ z#XDtfDY13Zt+$e4c~;wegof4*nV0!gJUKtoXCZaMYjNhA=@2daPO0t)7XB&H?92k4 z1^4i|)5t{qdpZ2MjILRhXH0pcARlPUH&f?MOrF%QK{LG#Yp~sE;RxG@8@ua;@i|J8 z7y3)x2x@!q!t(lCwrcKpe>>iyWp%{ecMlXVW?%aBzBfgdBka+xC#A`Vtfah1PW|dd zALh%iM!qe}Ez^oJtjem@%%y4S38mMb!f(bz3EH}@jhfw9c4>t}EMM1a>4W)QI-83tj12Vq##05~3DQh+b8WPsNHYVzwPDSZ=zFTQLZHy~;R4bpDXC7_R%M zH{jJEnM3B5f{P-WwQW0$qeytQ^b}(bbIzeypg)4O`d!XlLx-Jogl&mIu`o2-sNXO0 zzSHCdj2zG8(#7}b{RN#-T+((QkW;aJxl9-Q>Tg=u8>A`Ry4Oozwx-M{2L(Nxd$@ii zQk7X?eM&3uOr@+vrQ@4yReZc^!@^YYh#f1twPdWkei_@r^wl)e1#%d%-;F z5*iZ>E0akpOPS(^gCVJKrl5&h^0O?Pq_$7mefc1{r%N%$>>D7L)Z$$udi_=!^&@f{ zGKQ`EUcYU_r&s)L!EPY&m+_j>k86nnutPiT8hc;QCDMx?V%uhbu{C zJ7G~3G%N{%x=R8}4-^Ep@|NwWGfT+*n+Q*UdY-*sx}ucHO|s5C7Ac(06E)QqVx;&e z>sI9RW{`>gyUQBNswL40hcN6wqp?7Ao?kxLv9f}IM7QAdcB5!Uojo3GFL#z8FCDK4 zA$6dy^MHHd& z((VH|XVB;E!XYJu4@34(1X_ua;bz$?kEdzQuECvE~h9w*p__6%}K;x6XLm z){@NaXJmPzdd2QO4;f8hBh95uJ}c{W;)L(D2H~hzeq_CEH?bRSbu_?ma2J0pcc;d&iPBmp#M`jL6-QlO*XG|qCZj7sIfpkC3Kn7 zF_q3P@Pp(Dvi!W7OX}x$03rQdgeGaqG@5`2BY;$V{}Q1qkYVuOBCLMg=KzTC(N;)v z@Q~>fYiY0hmc-ItS%V5^HV94|YZ8uA#%=4U%{R5VsVfM-{!-g>rE0#{*ZrK8t7X>% zZZdf~2Etojh071j1M4PuUY&gCsk`T)8%Mj4ZdlI8&ORO+Y*@q8jl z$$a-m_r320O@$2K+Mj+N{k*@-7U8QffPctvC)_QFaES^)N0?C%Y#C(NXQM&-)a_jn z9c-^^=biW2>gKOGKUQOpGo6X@+@WEi5PqYadU0Ajia&Y5p3I6Wth|4U=^{mhb~H(8 z+ZSpmMZ`|VbZiRVa8TCKZc&?MWkJKl3NR%nG7bz9G-EVH>1L}zdhy4GTZUN9&ZlS?=8>( za2SIq2EM5UeyIOz{&-ke-nY^5@VRf}{(T0K_o|)=y38PtY>M!< z$NLK^+~J~WTW=!XThR)0_OBrErJ}fr+nZmz(bR*z1DBO9BD%mEb>i zZhHJ0aKjh`Y6G$wo&}vg@$W3qKnCXTJ>z&mc!Kx&k3^3^`~UyGs&zFGBbU!Mabnnu z(-sEnH%>jeG{HYQR!FSIDb|n{BMT8=k*?J6?kTgQa@lV!mFIXelKL#EV!1DRa@Wf0 zEgNjGU#M~w8h5UIn9s>O>T_u$tN&Y-@U}}9X(5VUfsajhCdv$uretysrwMgFl)qhL zn7)>KeS9$da#wk$$`Xw8LP~(L>0Jnkgl2%t8zrgKTXQYhwyn4z{G97SXT)57rog@8+!DiyH?C7GTawklQ>7DuUJW^J-u>6 zgY+w?+0|aew_ou($Gy9UnRWg{Fe~Njz99A58b1z~?QdhoO6DAuH=q?eFG>h9n-t!^ zvo_AAB@F0Xp|;F4z1jU}i133NKhYjvYG5$z^{V8{nJVA6ogB zxnt$>z~=u=o}bDhI<8f#or9rs>mr_#;%nltpiXM7`sn9-CY2qGF%l=|#VyGQU1#xr zS+frGvd~9gv6yD!PTb(=5L+~@T4FdCU!?q%&XAeJ70;${mpj(132EbMHzH#14K>y` zUrO$OcBTA^TCm&6`yW+@Kix{QUry^6sl883Q4`^aX zfj2)E1&ZF4zsbX8N1zRVR!`s>niYB~rz)9?@CeG9+=cgfuA|hi^eTJrYC0-*GHn;N zAZH-LFkP!LrAptv%!Bp+h8vDyB4ez#3@i+)njz6wsL>WoPM{iA?%BMZu3S1DZ<4Mt9ix)xT+)4CL$??v&3$E_I9-SWNX)eX^8uP8T0IpxB*V z@s&I(H06!8@x3)fmzfuzHGKcPs4GhphiD3d#!2c~AMNvpgPlNS&;YC6Z{dR@BRb4G zrcUXlSS3n`kY{+0xDLm75P?%LqsFU?&zsxN6f@=Z@5j{dtBCt^oTG9#8L@Su5Pp|j z&rhD2lv&`2&k=7Qr@-iTd~L5lL-&1b*AS0tJd^GznY!hb`jWpD&Ru6)F)i0uM7qTI z3aQ3F8GGZ2u~nzpfj{Jyb`oECX`J-Bw{RC#sq%4j#HL&!^0vw14X3aMCV@ykTdCSN z2}M?+$?|tN0`X+@#O9!-TbJc?(Y%T;-<}S+$=T}K>;4|0!Wg7D&a0vVU2pJtY{yFO zb2MJPwdxdzT`kWsb(m6F^a~s1(1$8^$nm-3HP|L4vX+Cqu4GT#eZu_ueav&shXaAP zVrgLN21>U`tOze5obI0Q?{-X=x7w%{ZSK2KA!CM!6Q5xKiWQ7ayck%W>8!Q;cFpgT zKDhaWS_w|}W}HApGGFKAu9+NllLs4iZ#fF4y9A3s&FqNE^kKhyGR3k=KkrSC}L|>)V>V zl&My^5$n@At&SYwMyX^kmhAf5>D+0A6N=EL4tEW#J*z#ny&jscz4p-=^Tu?|gVPT0 zHx4AWEC$45N6AhPbZmWz7t@+?Z&~LUiZyHZ@#gZOb=j3H^3k={?EJ=+ZRvY%S@&|` z1^C5b7nV02pS-hAQm399o8V_SLrA62*w3O?axat<#_5pzk?)oL)l`4N_@D$t$8)h> zWv;g(o7_B0sl&FW7QDH;@JcqjwwOT6QC!5hw(tT zba!DRf0I%Tn$!U5MddVZTBwX_?`^K$euZJqp$!Uu#M zH2Q;dZdukkEUaFBvm=Y~&SjGBpBPlWnQGtH7m%mYb}M>YGSvH}LPKD(D{A;k{{DPr zfPqoty(RvJ{;JL`AmR0YEp$(u6a+l0|MIuSy>&eK39x7_6n`ysf2#2B?~D~(0R7?Y zVXo`rYU6JD-NM~g{~aLvbI?T!000G;M6%ynvjYpvzbdQzsHAedp_K_|bt}+70%*+f zTSHU8|NC8gx!;EX)is2Fsm}7l{rkT}GXWgSe@46dy+40M6Z%FN>KpfNb)#D0?XeaehB_J#PPa5Kl_3MY~a7C_{SwV99l0 z6b=KWKhFPz1>O+?iT3}EtYP5}SXcMI2I^lA|4TxJfIskWxPK({M=|L?68b9(1b-t` z9N60a_sAlb|8b!G3Jp|D`Q-%$3h4az&_Bwx{_ZeS$;&SyKZUy%Pbiht3wzl`NS z@|j%gZXzR~LP`OZbK#ef0Rjz*|APFl0sTV@{PiI6#|d(&eN&?!SiANE3h{S5K0w3# zO>V#9{TRT6{r@#O9b$KpT81-Q{+G4{ZUsrmgx0l0^L0Yc;bGC6HxfiaDVLW zuM+=#(vqtD0qz&M{K~^W3g%bn-$hOhR6+V%$NvT`_K)g7)S|BKGr$9*zwv-s?GMPm zi0F?NWd53p2n>LOP#tHflGcBQ`%#5jxxcLip{h<#_Ysp8cQA!#n7UJsy8es) z-#yjG0e<)Y{%j5`;Q@f--r(b{fA@g>+M4bCFRlOO4?EucxRc^1Kz;AOI{)9hEN-h4 To%-JL67a7RFhQGx-=F>ud3GEi literal 0 HcmV?d00001 diff --git a/docs/ml-guide.md b/docs/ml-guide.md new file mode 100644 index 0000000000000..012fbd91e698b --- /dev/null +++ b/docs/ml-guide.md @@ -0,0 +1,702 @@ +--- +layout: global +title: Spark ML Programming Guide +--- + +Spark ML is Spark's new machine learning package. It is currently an alpha component but is potentially a successor to [MLlib](mllib-guide.html). The `spark.ml` package aims to replace the old APIs with a cleaner, more uniform set of APIs which will help users create full machine learning pipelines. + +MLlib vs. Spark ML: + +* Users can use algorithms from either of the two packages, but APIs may differ. Currently, `spark.ml` offers a subset of the algorithms from `spark.mllib`. Since Spark ML is an alpha component, its API may change in future releases. +* Developers should contribute new algorithms to `spark.mllib` and can optionally contribute to `spark.ml`. See below for more details. +* Spark ML only has Scala and Java APIs, whereas MLlib also has a Python API. + +**Table of Contents** + +* This will become a table of contents (this text will be scraped). +{:toc} + +# Main Concepts + +Spark ML standardizes APIs for machine learning algorithms to make it easier to combine multiple algorithms into a single pipeline, or workflow. This section covers the key concepts introduced by the Spark ML API. + +* **[ML Dataset](ml-guide.html#ml-dataset)**: Spark ML uses the [`SchemaRDD`](api/scala/index.html#org.apache.spark.sql.SchemaRDD) from Spark SQL as a dataset which can hold a variety of data types. +E.g., a dataset could have different columns storing text, feature vectors, true labels, and predictions. + +* **[`Transformer`](ml-guide.html#transformers)**: A `Transformer` is an algorithm which can transform one `SchemaRDD` into another `SchemaRDD`. +E.g., an ML model is a `Transformer` which transforms an RDD with features into an RDD with predictions. + +* **[`Estimator`](ml-guide.html#estimators)**: An `Estimator` is an algorithm which can be fit on a `SchemaRDD` to produce a `Transformer`. +E.g., a learning algorithm is an `Estimator` which trains on a dataset and produces a model. + +* **[`Pipeline`](ml-guide.html#pipeline)**: A `Pipeline` chains multiple `Transformer`s and `Estimator`s together to specify an ML workflow. + +* **[`Param`](ml-guide.html#param)**: All `Transformer`s and `Estimator`s now share a common API for specifying parameters. + +## ML Dataset + +Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data. +Spark ML adopts the [`SchemaRDD`](api/scala/index.html#org.apache.spark.sql.SchemaRDD) from Spark SQL in order to support a variety of data types under a unified Dataset concept. + +`SchemaRDD` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#spark-sql-datatype-reference) for a list of supported types. +In addition to the types listed in the Spark SQL guide, `SchemaRDD` can use ML [`Vector`](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) types. + +A `SchemaRDD` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. + +Columns in a `SchemaRDD` are named. The code examples below use names such as "text," "features," and "label." + +## ML Algorithms + +### Transformers + +A [`Transformer`](api/scala/index.html#org.apache.spark.ml.Transformer) is an abstraction which includes feature transformers and learned models. Technically, a `Transformer` implements a method `transform()` which converts one `SchemaRDD` into another, generally by appending one or more columns. +For example: + +* A feature transformer might take a dataset, read a column (e.g., text), convert it into a new column (e.g., feature vectors), append the new column to the dataset, and output the updated dataset. +* A learning model might take a dataset, read the column containing feature vectors, predict the label for each feature vector, append the labels as a new column, and output the updated dataset. + +### Estimators + +An [`Estimator`](api/scala/index.html#org.apache.spark.ml.Estimator) abstracts the concept of a learning algorithm or any algorithm which fits or trains on data. Technically, an `Estimator` implements a method `fit()` which accepts a `SchemaRDD` and produces a `Transformer`. +For example, a learning algorithm such as `LogisticRegression` is an `Estimator`, and calling `fit()` trains a `LogisticRegressionModel`, which is a `Transformer`. + +### Properties of ML Algorithms + +`Transformer`s and `Estimator`s are both stateless. In the future, stateful algorithms may be supported via alternative concepts. + +Each instance of a `Transformer` or `Estimator` has a unique ID, which is useful in specifying parameters (discussed below). + +## Pipeline + +In machine learning, it is common to run a sequence of algorithms to process and learn from data. +E.g., a simple text document processing workflow might include several stages: + +* Split each document's text into words. +* Convert each document's words into a numerical feature vector. +* Learn a prediction model using the feature vectors and labels. + +Spark ML represents such a workflow as a [`Pipeline`](api/scala/index.html#org.apache.spark.ml.Pipeline), +which consists of a sequence of [`PipelineStage`s](api/scala/index.html#org.apache.spark.ml.PipelineStage) (`Transformer`s and `Estimator`s) to be run in a specific order. We will use this simple workflow as a running example in this section. + +### How It Works + +A `Pipeline` is specified as a sequence of stages, and each stage is either a `Transformer` or an `Estimator`. +These stages are run in order, and the input dataset is modified as it passes through each stage. +For `Transformer` stages, the `transform()` method is called on the dataset. +For `Estimator` stages, the `fit()` method is called to produce a `Transformer` (which becomes part of the `PipelineModel`, or fitted `Pipeline`), and that `Transformer`'s `transform()` method is called on the dataset. + +We illustrate this for the simple text document workflow. The figure below is for the *training time* usage of a `Pipeline`. + +

+ Spark ML Pipeline Example +

+ +Above, the top row represents a `Pipeline` with three stages. +The first two (`Tokenizer` and `HashingTF`) are `Transformer`s (blue), and the third (`LogisticRegression`) is an `Estimator` (red). +The bottom row represents data flowing through the pipeline, where cylinders indicate `SchemaRDD`s. +The `Pipeline.fit()` method is called on the original dataset which has raw text documents and labels. +The `Tokenizer.transform()` method splits the raw text documents into words, adding a new column with words into the dataset. +The `HashingTF.transform()` method converts the words column into feature vectors, adding a new column with those vectors to the dataset. +Now, since `LogisticRegression` is an `Estimator`, the `Pipeline` first calls `LogisticRegression.fit()` to produce a `LogisticRegressionModel`. +If the `Pipeline` had more stages, it would call the `LogisticRegressionModel`'s `transform()` method on the dataset before passing the dataset to the next stage. + +A `Pipeline` is an `Estimator`. +Thus, after a `Pipeline`'s `fit()` method runs, it produces a `PipelineModel` which is a `Transformer`. This `PipelineModel` is used at *test time*; the figure below illustrates this usage. + +

+ Spark ML PipelineModel Example +

+ +In the figure above, the `PipelineModel` has the same number of stages as the original `Pipeline`, but all `Estimator`s in the original `Pipeline` have become `Transformer`s. +When the `PipelineModel`'s `transform()` method is called on a test dataset, the data are passed through the `Pipeline` in order. +Each stage's `transform()` method updates the dataset and passes it to the next stage. + +`Pipeline`s and `PipelineModel`s help to ensure that training and test data go through identical feature processing steps. + +### Details + +*DAG `Pipeline`s*: A `Pipeline`'s stages are specified as an ordered array. The examples given here are all for linear `Pipeline`s, i.e., `Pipeline`s in which each stage uses data produced by the previous stage. It is possible to create non-linear `Pipeline`s as long as the data flow graph forms a Directed Acyclic Graph (DAG). This graph is currently specified implicitly based on the input and output column names of each stage (generally specified as parameters). If the `Pipeline` forms a DAG, then the stages must be specified in topological order. + +*Runtime checking*: Since `Pipeline`s can operate on datasets with varied types, they cannot use compile-time type checking. `Pipeline`s and `PipelineModel`s instead do runtime checking before actually running the `Pipeline`. This type checking is done using the dataset *schema*, a description of the data types of columns in the `SchemaRDD`. + +## Parameters + +Spark ML `Estimator`s and `Transformer`s use a uniform API for specifying parameters. + +A [`Param`](api/scala/index.html#org.apache.spark.ml.param.Param) is a named parameter with self-contained documentation. +A [`ParamMap`](api/scala/index.html#org.apache.spark.ml.param.ParamMap)] is a set of (parameter, value) pairs. + +There are two main ways to pass parameters to an algorithm: + +1. Set parameters for an instance. E.g., if `lr` is an instance of `LogisticRegression`, one could call `lr.setMaxIter(10)` to make `lr.fit()` use at most 10 iterations. This API resembles the API used in MLlib. +2. Pass a `ParamMap` to `fit()` or `transform()`. Any parameters in the `ParamMap` will override parameters previously specified via setter methods. + +Parameters belong to specific instances of `Estimator`s and `Transformer`s. +For example, if we have two `LogisticRegression` instances `lr1` and `lr2`, then we can build a `ParamMap` with both `maxIter` parameters specified: `ParamMap(lr1.maxIter -> 10, lr2.maxIter -> 20)`. +This is useful if there are two algorithms with the `maxIter` parameter in a `Pipeline`. + +# Code Examples + +This section gives code examples illustrating the functionality discussed above. +There is not yet documentation for specific algorithms in Spark ML. For more info, please refer to the [API Documentation](api/scala/index.html). Spark ML algorithms are currently wrappers for MLlib algorithms, and the [MLlib programming guide](mllib-guide.html) has details on specific algorithms. + +## Example: Estimator, Transformer, and Param + +This example covers the concepts of `Estimator`, `Transformer`, and `Param`. + +
+ +
+{% highlight scala %} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.sql.{Row, SQLContext} + +val conf = new SparkConf().setAppName("SimpleParamsExample") +val sc = new SparkContext(conf) +val sqlContext = new SQLContext(sc) +import sqlContext._ + +// Prepare training data. +// We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes +// into SchemaRDDs, where it uses the case class metadata to infer the schema. +val training = sparkContext.parallelize(Seq( + LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), + LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)))) + +// Create a LogisticRegression instance. This instance is an Estimator. +val lr = new LogisticRegression() +// Print out the parameters, documentation, and any default values. +println("LogisticRegression parameters:\n" + lr.explainParams() + "\n") + +// We may set parameters using setter methods. +lr.setMaxIter(10) + .setRegParam(0.01) + +// Learn a LogisticRegression model. This uses the parameters stored in lr. +val model1 = lr.fit(training) +// Since model1 is a Model (i.e., a Transformer produced by an Estimator), +// we can view the parameters it used during fit(). +// This prints the parameter (name: value) pairs, where names are unique IDs for this +// LogisticRegression instance. +println("Model 1 was fit using parameters: " + model1.fittingParamMap) + +// We may alternatively specify parameters using a ParamMap, +// which supports several methods for specifying parameters. +val paramMap = ParamMap(lr.maxIter -> 20) +paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. +paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.5) // Specify multiple Params. + +// One can also combine ParamMaps. +val paramMap2 = ParamMap(lr.scoreCol -> "probability") // Changes output column name. +val paramMapCombined = paramMap ++ paramMap2 + +// Now learn a new model using the paramMapCombined parameters. +// paramMapCombined overrides all parameters set earlier via lr.set* methods. +val model2 = lr.fit(training, paramMapCombined) +println("Model 2 was fit using parameters: " + model2.fittingParamMap) + +// Prepare test documents. +val test = sparkContext.parallelize(Seq( + LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) + +// Make predictions on test documents using the Transformer.transform() method. +// LogisticRegression.transform will only use the 'features' column. +// Note that model2.transform() outputs a 'probability' column instead of the usual 'score' +// column since we renamed the lr.scoreCol parameter previously. +model2.transform(test) + .select('features, 'label, 'probability, 'prediction) + .collect() + .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) => + println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction) + } +{% endhighlight %} +
+ +
+{% highlight java %} +import java.util.List; +import com.google.common.collect.Lists; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionModel; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; + +SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample"); +JavaSparkContext jsc = new JavaSparkContext(conf); +JavaSQLContext jsql = new JavaSQLContext(jsc); + +// Prepare training data. +// We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of case classes +// into SchemaRDDs, where it uses the case class metadata to infer the schema. +List localTraining = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), + new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), + new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), + new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); +JavaSchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class); + +// Create a LogisticRegression instance. This instance is an Estimator. +LogisticRegression lr = new LogisticRegression(); +// Print out the parameters, documentation, and any default values. +System.out.println("LogisticRegression parameters:\n" + lr.explainParams() + "\n"); + +// We may set parameters using setter methods. +lr.setMaxIter(10) + .setRegParam(0.01); + +// Learn a LogisticRegression model. This uses the parameters stored in lr. +LogisticRegressionModel model1 = lr.fit(training); +// Since model1 is a Model (i.e., a Transformer produced by an Estimator), +// we can view the parameters it used during fit(). +// This prints the parameter (name: value) pairs, where names are unique IDs for this +// LogisticRegression instance. +System.out.println("Model 1 was fit using parameters: " + model1.fittingParamMap()); + +// We may alternatively specify parameters using a ParamMap. +ParamMap paramMap = new ParamMap(); +paramMap.put(lr.maxIter(), 20); // Specify 1 Param. +paramMap.put(lr.maxIter(), 30); // This overwrites the original maxIter. +paramMap.put(lr.regParam(), 0.1); + +// One can also combine ParamMaps. +ParamMap paramMap2 = new ParamMap(); +paramMap2.put(lr.scoreCol(), "probability"); // Changes output column name. +ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2); + +// Now learn a new model using the paramMapCombined parameters. +// paramMapCombined overrides all parameters set earlier via lr.set* methods. +LogisticRegressionModel model2 = lr.fit(training, paramMapCombined); +System.out.println("Model 2 was fit using parameters: " + model2.fittingParamMap()); + +// Prepare test documents. +List localTest = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), + new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); +JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class); + +// Make predictions on test documents using the Transformer.transform() method. +// LogisticRegression.transform will only use the 'features' column. +// Note that model2.transform() outputs a 'probability' column instead of the usual 'score' +// column since we renamed the lr.scoreCol parameter previously. +model2.transform(test).registerAsTable("results"); +JavaSchemaRDD results = + jsql.sql("SELECT features, label, probability, prediction FROM results"); +for (Row r: results.collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + + ", prediction=" + r.get(3)); +} +{% endhighlight %} +
+ +
+ +## Example: Pipeline + +This example follows the simple text document `Pipeline` illustrated in the figures above. + +
+ +
+{% highlight scala %} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.sql.{Row, SQLContext} + +// Labeled and unlabeled instance types. +// Spark SQL can infer schema from case classes. +case class LabeledDocument(id: Long, text: String, label: Double) +case class Document(id: Long, text: String) + +// Set up contexts. Import implicit conversions to SchemaRDD from sqlContext. +val conf = new SparkConf().setAppName("SimpleTextClassificationPipeline") +val sc = new SparkContext(conf) +val sqlContext = new SQLContext(sc) +import sqlContext._ + +// Prepare training documents, which are labeled. +val training = sparkContext.parallelize(Seq( + LabeledDocument(0L, "a b c d e spark", 1.0), + LabeledDocument(1L, "b d", 0.0), + LabeledDocument(2L, "spark f g h", 1.0), + LabeledDocument(3L, "hadoop mapreduce", 0.0))) + +// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. +val tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words") +val hashingTF = new HashingTF() + .setNumFeatures(1000) + .setInputCol(tokenizer.getOutputCol) + .setOutputCol("features") +val lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.01) +val pipeline = new Pipeline() + .setStages(Array(tokenizer, hashingTF, lr)) + +// Fit the pipeline to training documents. +val model = pipeline.fit(training) + +// Prepare test documents, which are unlabeled. +val test = sparkContext.parallelize(Seq( + Document(4L, "spark i j k"), + Document(5L, "l m n"), + Document(6L, "mapreduce spark"), + Document(7L, "apache hadoop"))) + +// Make predictions on test documents. +model.transform(test) + .select('id, 'text, 'score, 'prediction) + .collect() + .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => + println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) + } +{% endhighlight %} +
+ +
+{% highlight java %} +import java.io.Serializable; +import java.util.List; +import com.google.common.collect.Lists; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineModel; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.feature.HashingTF; +import org.apache.spark.ml.feature.Tokenizer; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; +import org.apache.spark.SparkConf; + +// Labeled and unlabeled instance types. +// Spark SQL can infer schema from Java Beans. +public class Document implements Serializable { + private Long id; + private String text; + + public Document(Long id, String text) { + this.id = id; + this.text = text; + } + + public Long getId() { return this.id; } + public void setId(Long id) { this.id = id; } + + public String getText() { return this.text; } + public void setText(String text) { this.text = text; } +} + +public class LabeledDocument extends Document implements Serializable { + private Double label; + + public LabeledDocument(Long id, String text, Double label) { + super(id, text); + this.label = label; + } + + public Double getLabel() { return this.label; } + public void setLabel(Double label) { this.label = label; } +} + +// Set up contexts. +SparkConf conf = new SparkConf().setAppName("JavaSimpleTextClassificationPipeline"); +JavaSparkContext jsc = new JavaSparkContext(conf); +JavaSQLContext jsql = new JavaSQLContext(jsc); + +// Prepare training documents, which are labeled. +List localTraining = Lists.newArrayList( + new LabeledDocument(0L, "a b c d e spark", 1.0), + new LabeledDocument(1L, "b d", 0.0), + new LabeledDocument(2L, "spark f g h", 1.0), + new LabeledDocument(3L, "hadoop mapreduce", 0.0)); +JavaSchemaRDD training = + jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + +// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. +Tokenizer tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words"); +HashingTF hashingTF = new HashingTF() + .setNumFeatures(1000) + .setInputCol(tokenizer.getOutputCol()) + .setOutputCol("features"); +LogisticRegression lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.01); +Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); + +// Fit the pipeline to training documents. +PipelineModel model = pipeline.fit(training); + +// Prepare test documents, which are unlabeled. +List localTest = Lists.newArrayList( + new Document(4L, "spark i j k"), + new Document(5L, "l m n"), + new Document(6L, "mapreduce spark"), + new Document(7L, "apache hadoop")); +JavaSchemaRDD test = + jsql.applySchema(jsc.parallelize(localTest), Document.class); + +// Make predictions on test documents. +model.transform(test).registerAsTable("prediction"); +JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); +for (Row r: predictions.collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + + ", prediction=" + r.get(3)); +} +{% endhighlight %} +
+ +
+ +## Example: Model Selection via Cross-Validation + +An important task in ML is *model selection*, or using data to find the best model or parameters for a given task. This is also called *tuning*. +`Pipeline`s facilitate model selection by making it easy to tune an entire `Pipeline` at once, rather than tuning each element in the `Pipeline` separately. + +Currently, `spark.ml` supports model selection using the [`CrossValidator`](api/scala/index.html#org.apache.spark.ml.tuning.CrossValidator) class, which takes an `Estimator`, a set of `ParamMap`s, and an [`Evaluator`](api/scala/index.html#org.apache.spark.ml.Evaluator). +`CrossValidator` begins by splitting the dataset into a set of *folds* which are used as separate training and test datasets; e.g., with `$k=3$` folds, `CrossValidator` will generate 3 (training, test) dataset pairs, each of which uses 2/3 of the data for training and 1/3 for testing. +`CrossValidator` iterates through the set of `ParamMap`s. For each `ParamMap`, it trains the given `Estimator` and evaluates it using the given `Evaluator`. +The `ParamMap` which produces the best evaluation metric (averaged over the `$k$` folds) is selected as the best model. +`CrossValidator` finally fits the `Estimator` using the best `ParamMap` and the entire dataset. + +The following example demonstrates using `CrossValidator` to select from a grid of parameters. +To help construct the parameter grid, we use the [`ParamGridBuilder`](api/scala/index.html#org.apache.spark.ml.tuning.ParamGridGuilder) utility. + +Note that cross-validation over a grid of parameters is expensive. +E.g., in the example below, the parameter grid has 3 values for `hashingTF.numFeatures` and 2 values for `lr.regParam`, and `CrossValidator` uses 2 folds. This multiplies out to `$(3 \times 2) \times 2 = 12$` different models being trained. +In realistic settings, it can be common to try many more parameters and use more folds (`$k=3$` and `$k=10$` are common). +In other words, using `CrossValidator` can be very expensive. +However, it is also a well-established method for choosing parameters which is more statistically sound than heuristic hand-tuning. + +
+ +
+{% highlight scala %} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator +import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator} +import org.apache.spark.sql.{Row, SQLContext} + +val conf = new SparkConf().setAppName("CrossValidatorExample") +val sc = new SparkContext(conf) +val sqlContext = new SQLContext(sc) +import sqlContext._ + +// Prepare training documents, which are labeled. +val training = sparkContext.parallelize(Seq( + LabeledDocument(0L, "a b c d e spark", 1.0), + LabeledDocument(1L, "b d", 0.0), + LabeledDocument(2L, "spark f g h", 1.0), + LabeledDocument(3L, "hadoop mapreduce", 0.0), + LabeledDocument(4L, "b spark who", 1.0), + LabeledDocument(5L, "g d a y", 0.0), + LabeledDocument(6L, "spark fly", 1.0), + LabeledDocument(7L, "was mapreduce", 0.0), + LabeledDocument(8L, "e spark program", 1.0), + LabeledDocument(9L, "a e c l", 0.0), + LabeledDocument(10L, "spark compile", 1.0), + LabeledDocument(11L, "hadoop software", 0.0))) + +// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. +val tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words") +val hashingTF = new HashingTF() + .setInputCol(tokenizer.getOutputCol) + .setOutputCol("features") +val lr = new LogisticRegression() + .setMaxIter(10) +val pipeline = new Pipeline() + .setStages(Array(tokenizer, hashingTF, lr)) + +// We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. +// This will allow us to jointly choose parameters for all Pipeline stages. +// A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. +val crossval = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator) +// We use a ParamGridBuilder to construct a grid of parameters to search over. +// With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, +// this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. +val paramGrid = new ParamGridBuilder() + .addGrid(hashingTF.numFeatures, Array(10, 100, 1000)) + .addGrid(lr.regParam, Array(0.1, 0.01)) + .build() +crossval.setEstimatorParamMaps(paramGrid) +crossval.setNumFolds(2) // Use 3+ in practice + +// Run cross-validation, and choose the best set of parameters. +val cvModel = crossval.fit(training) +// Get the best LogisticRegression model (with the best set of parameters from paramGrid). +val lrModel = cvModel.bestModel + +// Prepare test documents, which are unlabeled. +val test = sparkContext.parallelize(Seq( + Document(4L, "spark i j k"), + Document(5L, "l m n"), + Document(6L, "mapreduce spark"), + Document(7L, "apache hadoop"))) + +// Make predictions on test documents. cvModel uses the best model found (lrModel). +cvModel.transform(test) + .select('id, 'text, 'score, 'prediction) + .collect() + .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => + println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) +} +{% endhighlight %} +
+ +
+{% highlight java %} +import java.util.List; +import com.google.common.collect.Lists; +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.Model; +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; +import org.apache.spark.ml.feature.HashingTF; +import org.apache.spark.ml.feature.Tokenizer; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.tuning.CrossValidator; +import org.apache.spark.ml.tuning.CrossValidatorModel; +import org.apache.spark.ml.tuning.ParamGridBuilder; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; + +SparkConf conf = new SparkConf().setAppName("JavaCrossValidatorExample"); +JavaSparkContext jsc = new JavaSparkContext(conf); +JavaSQLContext jsql = new JavaSQLContext(jsc); + +// Prepare training documents, which are labeled. +List localTraining = Lists.newArrayList( + new LabeledDocument(0L, "a b c d e spark", 1.0), + new LabeledDocument(1L, "b d", 0.0), + new LabeledDocument(2L, "spark f g h", 1.0), + new LabeledDocument(3L, "hadoop mapreduce", 0.0), + new LabeledDocument(4L, "b spark who", 1.0), + new LabeledDocument(5L, "g d a y", 0.0), + new LabeledDocument(6L, "spark fly", 1.0), + new LabeledDocument(7L, "was mapreduce", 0.0), + new LabeledDocument(8L, "e spark program", 1.0), + new LabeledDocument(9L, "a e c l", 0.0), + new LabeledDocument(10L, "spark compile", 1.0), + new LabeledDocument(11L, "hadoop software", 0.0)); +JavaSchemaRDD training = + jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + +// Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. +Tokenizer tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words"); +HashingTF hashingTF = new HashingTF() + .setNumFeatures(1000) + .setInputCol(tokenizer.getOutputCol()) + .setOutputCol("features"); +LogisticRegression lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.01); +Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); + +// We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. +// This will allow us to jointly choose parameters for all Pipeline stages. +// A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. +CrossValidator crossval = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator()); +// We use a ParamGridBuilder to construct a grid of parameters to search over. +// With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, +// this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. +ParamMap[] paramGrid = new ParamGridBuilder() + .addGrid(hashingTF.numFeatures(), new int[]{10, 100, 1000}) + .addGrid(lr.regParam(), new double[]{0.1, 0.01}) + .build(); +crossval.setEstimatorParamMaps(paramGrid); +crossval.setNumFolds(2); // Use 3+ in practice + +// Run cross-validation, and choose the best set of parameters. +CrossValidatorModel cvModel = crossval.fit(training); +// Get the best LogisticRegression model (with the best set of parameters from paramGrid). +Model lrModel = cvModel.bestModel(); + +// Prepare test documents, which are unlabeled. +List localTest = Lists.newArrayList( + new Document(4L, "spark i j k"), + new Document(5L, "l m n"), + new Document(6L, "mapreduce spark"), + new Document(7L, "apache hadoop")); +JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class); + +// Make predictions on test documents. cvModel uses the best model found (lrModel). +cvModel.transform(test).registerAsTable("prediction"); +JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); +for (Row r: predictions.collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + + ", prediction=" + r.get(3)); +} +{% endhighlight %} +
+ +
+ +# Dependencies + +Spark ML currently depends on MLlib and has the same dependencies. +Please see the [MLlib Dependencies guide](mllib-guide.html#Dependencies) for more info. + +Spark ML also depends upon Spark SQL, but the relevant parts of Spark SQL do not bring additional dependencies. + +# Developers + +**Development plan** + +If all goes well, `spark.ml` will become the primary ML package at the time of the Spark 1.3 release. Initially, simple wrappers will be used to port algorithms to `spark.ml`, but eventually, code will be moved to `spark.ml` and `spark.mllib` will be deprecated. + +**Advice to developers** + +During the next development cycle, new algorithms should be contributed to `spark.mllib`, but we welcome patches sent to either package. If an algorithm is best expressed using the new API (e.g., feature transformers), we may ask for developers to use the new `spark.ml` API. +Wrappers for old and new algorithms can be contributed to `spark.ml`. + +Users will be able to use algorithms from either of the two packages. The main difficulty will be the differences in APIs between the two packages. + diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index dcb6819f46cba..efd7dda310712 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -1,6 +1,6 @@ --- layout: global -title: Machine Learning Library (MLlib) +title: Machine Learning Library (MLlib) Programming Guide --- MLlib is Spark's scalable machine learning library consisting of common learning algorithms and utilities, @@ -35,6 +35,17 @@ MLlib is under active development. The APIs marked `Experimental`/`DeveloperApi` may change in future releases, and the migration guide below will explain all changes between releases. +# spark.ml: The New ML Package + +Spark 1.2 includes a new machine learning package called `spark.ml`, currently an alpha component but potentially a successor to `spark.mllib`. The `spark.ml` package aims to replace the old APIs with a cleaner, more uniform set of APIs which will help users create full machine learning pipelines. + +See the **[spark.ml programming guide](ml-guide.html)** for more information on this package. + +Users can use algorithms from either of the two packages, but APIs may differ. Currently, `spark.ml` offers a subset of the algorithms from `spark.mllib`. + +Developers should contribute new algorithms to `spark.mllib` and can optionally contribute to `spark.ml`. +See the `spark.ml` programming guide linked above for more details. + # Dependencies MLlib uses the linear algebra package [Breeze](http://www.scalanlp.org/), diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java new file mode 100644 index 0000000000000..3b156fa0482fc --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import java.util.List; + +import com.google.common.collect.Lists; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.Model; +import org.apache.spark.ml.Pipeline; +import org.apache.spark.ml.PipelineStage; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator; +import org.apache.spark.ml.feature.HashingTF; +import org.apache.spark.ml.feature.Tokenizer; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.tuning.CrossValidator; +import org.apache.spark.ml.tuning.CrossValidatorModel; +import org.apache.spark.ml.tuning.ParamGridBuilder; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; + +/** + * A simple example demonstrating model selection using CrossValidator. + * This example also demonstrates how Pipelines are Estimators. + * + * This example uses the Java bean classes {@link org.apache.spark.examples.ml.LabeledDocument} and + * {@link org.apache.spark.examples.ml.Document} defined in the Scala example + * {@link org.apache.spark.examples.ml.SimpleTextClassificationPipeline}. + * + * Run with + *
+ * bin/run-example ml.JavaCrossValidatorExample
+ * 
+ */ +public class JavaCrossValidatorExample { + + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("JavaCrossValidatorExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + JavaSQLContext jsql = new JavaSQLContext(jsc); + + // Prepare training documents, which are labeled. + List localTraining = Lists.newArrayList( + new LabeledDocument(0L, "a b c d e spark", 1.0), + new LabeledDocument(1L, "b d", 0.0), + new LabeledDocument(2L, "spark f g h", 1.0), + new LabeledDocument(3L, "hadoop mapreduce", 0.0), + new LabeledDocument(4L, "b spark who", 1.0), + new LabeledDocument(5L, "g d a y", 0.0), + new LabeledDocument(6L, "spark fly", 1.0), + new LabeledDocument(7L, "was mapreduce", 0.0), + new LabeledDocument(8L, "e spark program", 1.0), + new LabeledDocument(9L, "a e c l", 0.0), + new LabeledDocument(10L, "spark compile", 1.0), + new LabeledDocument(11L, "hadoop software", 0.0)); + JavaSchemaRDD training = + jsql.applySchema(jsc.parallelize(localTraining), LabeledDocument.class); + + // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. + Tokenizer tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words"); + HashingTF hashingTF = new HashingTF() + .setNumFeatures(1000) + .setInputCol(tokenizer.getOutputCol()) + .setOutputCol("features"); + LogisticRegression lr = new LogisticRegression() + .setMaxIter(10) + .setRegParam(0.01); + Pipeline pipeline = new Pipeline() + .setStages(new PipelineStage[] {tokenizer, hashingTF, lr}); + + // We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. + // This will allow us to jointly choose parameters for all Pipeline stages. + // A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. + CrossValidator crossval = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator()); + // We use a ParamGridBuilder to construct a grid of parameters to search over. + // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, + // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. + ParamMap[] paramGrid = new ParamGridBuilder() + .addGrid(hashingTF.numFeatures(), new int[]{10, 100, 1000}) + .addGrid(lr.regParam(), new double[]{0.1, 0.01}) + .build(); + crossval.setEstimatorParamMaps(paramGrid); + crossval.setNumFolds(2); // Use 3+ in practice + + // Run cross-validation, and choose the best set of parameters. + CrossValidatorModel cvModel = crossval.fit(training); + + // Prepare test documents, which are unlabeled. + List localTest = Lists.newArrayList( + new Document(4L, "spark i j k"), + new Document(5L, "l m n"), + new Document(6L, "mapreduce spark"), + new Document(7L, "apache hadoop")); + JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class); + + // Make predictions on test documents. cvModel uses the best model found (lrModel). + cvModel.transform(test).registerAsTable("prediction"); + JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); + for (Row r: predictions.collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + + ", prediction=" + r.get(3)); + } + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java new file mode 100644 index 0000000000000..cf58f4dfaa15b --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -0,0 +1,111 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +import java.util.List; + +import com.google.common.collect.Lists; + +import org.apache.spark.SparkConf; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.classification.LogisticRegressionModel; +import org.apache.spark.ml.param.ParamMap; +import org.apache.spark.ml.classification.LogisticRegression; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; + +/** + * A simple example demonstrating ways to specify parameters for Estimators and Transformers. + * Run with + * {{{ + * bin/run-example ml.JavaSimpleParamsExample + * }}} + */ +public class JavaSimpleParamsExample { + + public static void main(String[] args) { + SparkConf conf = new SparkConf().setAppName("JavaSimpleParamsExample"); + JavaSparkContext jsc = new JavaSparkContext(conf); + JavaSQLContext jsql = new JavaSQLContext(jsc); + + // Prepare training data. + // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of Java Beans + // into SchemaRDDs, where it uses the bean metadata to infer the schema. + List localTraining = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), + new LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), + new LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), + new LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5))); + JavaSchemaRDD training = jsql.applySchema(jsc.parallelize(localTraining), LabeledPoint.class); + + // Create a LogisticRegression instance. This instance is an Estimator. + LogisticRegression lr = new LogisticRegression(); + // Print out the parameters, documentation, and any default values. + System.out.println("LogisticRegression parameters:\n" + lr.explainParams() + "\n"); + + // We may set parameters using setter methods. + lr.setMaxIter(10) + .setRegParam(0.01); + + // Learn a LogisticRegression model. This uses the parameters stored in lr. + LogisticRegressionModel model1 = lr.fit(training); + // Since model1 is a Model (i.e., a Transformer produced by an Estimator), + // we can view the parameters it used during fit(). + // This prints the parameter (name: value) pairs, where names are unique IDs for this + // LogisticRegression instance. + System.out.println("Model 1 was fit using parameters: " + model1.fittingParamMap()); + + // We may alternatively specify parameters using a ParamMap. + ParamMap paramMap = new ParamMap(); + paramMap.put(lr.maxIter().w(20)); // Specify 1 Param. + paramMap.put(lr.maxIter(), 30); // This overwrites the original maxIter. + paramMap.put(lr.regParam().w(0.1), lr.threshold().w(0.55)); // Specify multiple Params. + + // One can also combine ParamMaps. + ParamMap paramMap2 = new ParamMap(); + paramMap2.put(lr.scoreCol().w("probability")); // Change output column name + ParamMap paramMapCombined = paramMap.$plus$plus(paramMap2); + + // Now learn a new model using the paramMapCombined parameters. + // paramMapCombined overrides all parameters set earlier via lr.set* methods. + LogisticRegressionModel model2 = lr.fit(training, paramMapCombined); + System.out.println("Model 2 was fit using parameters: " + model2.fittingParamMap()); + + // Prepare test documents. + List localTest = Lists.newArrayList( + new LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + new LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), + new LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5))); + JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), LabeledPoint.class); + + // Make predictions on test documents using the Transformer.transform() method. + // LogisticRegression.transform will only use the 'features' column. + // Note that model2.transform() outputs a 'probability' column instead of the usual 'score' + // column since we renamed the lr.scoreCol parameter previously. + model2.transform(test).registerAsTable("results"); + JavaSchemaRDD results = + jsql.sql("SELECT features, label, probability, prediction FROM results"); + for (Row r: results.collect()) { + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") -> prob=" + r.get(2) + + ", prediction=" + r.get(3)); + } + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java index 22ba68d8c354c..54f18014e4b2f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleTextClassificationPipeline.java @@ -80,14 +80,14 @@ public static void main(String[] args) { new Document(5L, "l m n"), new Document(6L, "mapreduce spark"), new Document(7L, "apache hadoop")); - JavaSchemaRDD test = - jsql.applySchema(jsc.parallelize(localTest), Document.class); + JavaSchemaRDD test = jsql.applySchema(jsc.parallelize(localTest), Document.class); // Make predictions on test documents. model.transform(test).registerAsTable("prediction"); JavaSchemaRDD predictions = jsql.sql("SELECT id, text, score, prediction FROM prediction"); for (Row r: predictions.collect()) { - System.out.println(r); + System.out.println("(" + r.get(0) + ", " + r.get(1) + ") --> score=" + r.get(2) + + ", prediction=" + r.get(3)); } } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala new file mode 100644 index 0000000000000..ce6bc066bd70d --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/CrossValidatorExample.scala @@ -0,0 +1,110 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.ml.Pipeline +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.evaluation.BinaryClassificationEvaluator +import org.apache.spark.ml.feature.{HashingTF, Tokenizer} +import org.apache.spark.ml.tuning.{ParamGridBuilder, CrossValidator} +import org.apache.spark.sql.{Row, SQLContext} + +/** + * A simple example demonstrating model selection using CrossValidator. + * This example also demonstrates how Pipelines are Estimators. + * + * This example uses the [[LabeledDocument]] and [[Document]] case classes from + * [[SimpleTextClassificationPipeline]]. + * + * Run with + * {{{ + * bin/run-example ml.CrossValidatorExample + * }}} + */ +object CrossValidatorExample { + + def main(args: Array[String]) { + val conf = new SparkConf().setAppName("CrossValidatorExample") + val sc = new SparkContext(conf) + val sqlContext = new SQLContext(sc) + import sqlContext._ + + // Prepare training documents, which are labeled. + val training = sparkContext.parallelize(Seq( + LabeledDocument(0L, "a b c d e spark", 1.0), + LabeledDocument(1L, "b d", 0.0), + LabeledDocument(2L, "spark f g h", 1.0), + LabeledDocument(3L, "hadoop mapreduce", 0.0), + LabeledDocument(4L, "b spark who", 1.0), + LabeledDocument(5L, "g d a y", 0.0), + LabeledDocument(6L, "spark fly", 1.0), + LabeledDocument(7L, "was mapreduce", 0.0), + LabeledDocument(8L, "e spark program", 1.0), + LabeledDocument(9L, "a e c l", 0.0), + LabeledDocument(10L, "spark compile", 1.0), + LabeledDocument(11L, "hadoop software", 0.0))) + + // Configure an ML pipeline, which consists of three stages: tokenizer, hashingTF, and lr. + val tokenizer = new Tokenizer() + .setInputCol("text") + .setOutputCol("words") + val hashingTF = new HashingTF() + .setInputCol(tokenizer.getOutputCol) + .setOutputCol("features") + val lr = new LogisticRegression() + .setMaxIter(10) + val pipeline = new Pipeline() + .setStages(Array(tokenizer, hashingTF, lr)) + + // We now treat the Pipeline as an Estimator, wrapping it in a CrossValidator instance. + // This will allow us to jointly choose parameters for all Pipeline stages. + // A CrossValidator requires an Estimator, a set of Estimator ParamMaps, and an Evaluator. + val crossval = new CrossValidator() + .setEstimator(pipeline) + .setEvaluator(new BinaryClassificationEvaluator) + // We use a ParamGridBuilder to construct a grid of parameters to search over. + // With 3 values for hashingTF.numFeatures and 2 values for lr.regParam, + // this grid will have 3 x 2 = 6 parameter settings for CrossValidator to choose from. + val paramGrid = new ParamGridBuilder() + .addGrid(hashingTF.numFeatures, Array(10, 100, 1000)) + .addGrid(lr.regParam, Array(0.1, 0.01)) + .build() + crossval.setEstimatorParamMaps(paramGrid) + crossval.setNumFolds(2) // Use 3+ in practice + + // Run cross-validation, and choose the best set of parameters. + val cvModel = crossval.fit(training) + + // Prepare test documents, which are unlabeled. + val test = sparkContext.parallelize(Seq( + Document(4L, "spark i j k"), + Document(5L, "l m n"), + Document(6L, "mapreduce spark"), + Document(7L, "apache hadoop"))) + + // Make predictions on test documents. cvModel uses the best model found (lrModel). + cvModel.transform(test) + .select('id, 'text, 'score, 'prediction) + .collect() + .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => + println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) + } + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala new file mode 100644 index 0000000000000..44d5b084c269a --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleParamsExample.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.ml.classification.LogisticRegression +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.sql.{Row, SQLContext} + +/** + * A simple example demonstrating ways to specify parameters for Estimators and Transformers. + * Run with + * {{{ + * bin/run-example ml.SimpleParamsExample + * }}} + */ +object SimpleParamsExample { + + def main(args: Array[String]) { + val conf = new SparkConf().setAppName("SimpleParamsExample") + val sc = new SparkContext(conf) + val sqlContext = new SQLContext(sc) + import sqlContext._ + + // Prepare training data. + // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of Java Beans + // into SchemaRDDs, where it uses the bean metadata to infer the schema. + val training = sparkContext.parallelize(Seq( + LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), + LabeledPoint(0.0, Vectors.dense(2.0, 1.0, -1.0)), + LabeledPoint(0.0, Vectors.dense(2.0, 1.3, 1.0)), + LabeledPoint(1.0, Vectors.dense(0.0, 1.2, -0.5)))) + + // Create a LogisticRegression instance. This instance is an Estimator. + val lr = new LogisticRegression() + // Print out the parameters, documentation, and any default values. + println("LogisticRegression parameters:\n" + lr.explainParams() + "\n") + + // We may set parameters using setter methods. + lr.setMaxIter(10) + .setRegParam(0.01) + + // Learn a LogisticRegression model. This uses the parameters stored in lr. + val model1 = lr.fit(training) + // Since model1 is a Model (i.e., a Transformer produced by an Estimator), + // we can view the parameters it used during fit(). + // This prints the parameter (name: value) pairs, where names are unique IDs for this + // LogisticRegression instance. + println("Model 1 was fit using parameters: " + model1.fittingParamMap) + + // We may alternatively specify parameters using a ParamMap, + // which supports several methods for specifying parameters. + val paramMap = ParamMap(lr.maxIter -> 20) + paramMap.put(lr.maxIter, 30) // Specify 1 Param. This overwrites the original maxIter. + paramMap.put(lr.regParam -> 0.1, lr.threshold -> 0.55) // Specify multiple Params. + + // One can also combine ParamMaps. + val paramMap2 = ParamMap(lr.scoreCol -> "probability") // Change output column name + val paramMapCombined = paramMap ++ paramMap2 + + // Now learn a new model using the paramMapCombined parameters. + // paramMapCombined overrides all parameters set earlier via lr.set* methods. + val model2 = lr.fit(training, paramMapCombined) + println("Model 2 was fit using parameters: " + model2.fittingParamMap) + + // Prepare test documents. + val test = sparkContext.parallelize(Seq( + LabeledPoint(1.0, Vectors.dense(-1.0, 1.5, 1.3)), + LabeledPoint(0.0, Vectors.dense(3.0, 2.0, -0.1)), + LabeledPoint(1.0, Vectors.dense(0.0, 2.2, -1.5)))) + + // Make predictions on test documents using the Transformer.transform() method. + // LogisticRegression.transform will only use the 'features' column. + // Note that model2.transform() outputs a 'probability' column instead of the usual 'score' + // column since we renamed the lr.scoreCol parameter previously. + model2.transform(test) + .select('features, 'label, 'probability, 'prediction) + .collect() + .foreach { case Row(features: Vector, label: Double, prob: Double, prediction: Double) => + println("(" + features + ", " + label + ") -> prob=" + prob + ", prediction=" + prediction) + } + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala index ee7897d9062d9..92895a05e479a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/SimpleTextClassificationPipeline.scala @@ -20,10 +20,11 @@ package org.apache.spark.examples.ml import scala.beans.BeanInfo import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.SparkContext._ import org.apache.spark.ml.Pipeline import org.apache.spark.ml.classification.LogisticRegression import org.apache.spark.ml.feature.{HashingTF, Tokenizer} -import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.{Row, SQLContext} @BeanInfo case class LabeledDocument(id: Long, text: String, label: Double) @@ -81,6 +82,8 @@ object SimpleTextClassificationPipeline { model.transform(test) .select('id, 'text, 'score, 'prediction) .collect() - .foreach(println) + .foreach { case Row(id: Long, text: String, score: Double, prediction: Double) => + println("(" + id + ", " + text + ") --> score=" + score + ", prediction=" + prediction) + } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala index e545df1e37b9c..081a574beea5d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -162,11 +162,15 @@ class PipelineModel private[ml] ( } override def transform(dataset: SchemaRDD, paramMap: ParamMap): SchemaRDD = { - transformSchema(dataset.schema, paramMap, logging = true) - stages.foldLeft(dataset)((cur, transformer) => transformer.transform(cur, paramMap)) + // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap + val map = (fittingParamMap ++ this.paramMap) ++ paramMap + transformSchema(dataset.schema, map, logging = true) + stages.foldLeft(dataset)((cur, transformer) => transformer.transform(cur, map)) } private[ml] override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - stages.foldLeft(schema)((cur, transformer) => transformer.transformSchema(cur, paramMap)) + // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap + val map = (fittingParamMap ++ this.paramMap) ++ paramMap + stages.foldLeft(schema)((cur, transformer) => transformer.transformSchema(cur, map)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala index 490e6609ad311..23fbd228d01cb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -18,16 +18,14 @@ package org.apache.spark.ml import scala.annotation.varargs -import scala.reflect.runtime.universe.TypeTag import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ import org.apache.spark.sql.SchemaRDD import org.apache.spark.sql.api.java.JavaSchemaRDD -import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.Star -import org.apache.spark.sql.catalyst.dsl._ +import org.apache.spark.sql.catalyst.expressions.ScalaUdf import org.apache.spark.sql.catalyst.types._ /** @@ -86,7 +84,7 @@ abstract class Transformer extends PipelineStage with Params { * Abstract class for transformers that take one input column, apply transformation, and output the * result as a new column. */ -private[ml] abstract class UnaryTransformer[IN, OUT: TypeTag, T <: UnaryTransformer[IN, OUT, T]] +private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, OUT, T]] extends Transformer with HasInputCol with HasOutputCol with Logging { def setInputCol(value: String): T = set(inputCol, value).asInstanceOf[T] @@ -99,6 +97,11 @@ private[ml] abstract class UnaryTransformer[IN, OUT: TypeTag, T <: UnaryTransfor */ protected def createTransformFunc(paramMap: ParamMap): IN => OUT + /** + * Returns the data type of the output column. + */ + protected def outputDataType: DataType + /** * Validates the input type. Throw an exception if it is invalid. */ @@ -111,9 +114,8 @@ private[ml] abstract class UnaryTransformer[IN, OUT: TypeTag, T <: UnaryTransfor if (schema.fieldNames.contains(map(outputCol))) { throw new IllegalArgumentException(s"Output column ${map(outputCol)} already exists.") } - val output = ScalaReflection.schemaFor[OUT] val outputFields = schema.fields :+ - StructField(map(outputCol), output.dataType, output.nullable) + StructField(map(outputCol), outputDataType, !outputDataType.isPrimitive) StructType(outputFields) } @@ -121,7 +123,7 @@ private[ml] abstract class UnaryTransformer[IN, OUT: TypeTag, T <: UnaryTransfor transformSchema(dataset.schema, paramMap, logging = true) import dataset.sqlContext._ val map = this.paramMap ++ paramMap - val udf = this.createTransformFunc(map) - dataset.select(Star(None), udf.call(map(inputCol).attr) as map(outputCol)) + val udf = ScalaUdf(this.createTransformFunc(map), outputDataType, Seq(map(inputCol).attr)) + dataset.select(Star(None), udf as map(outputCol)) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index b98b1755a3584..e0bfb1e484a2e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -21,7 +21,8 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.param.{IntParam, ParamMap} import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.{VectorUDT, Vector} +import org.apache.spark.sql.catalyst.types.DataType /** * :: AlphaComponent :: @@ -39,4 +40,6 @@ class HashingTF extends UnaryTransformer[Iterable[_], Vector, HashingTF] { val hashingTF = new feature.HashingTF(paramMap(numFeatures)) hashingTF.transform } + + override protected def outputDataType: DataType = new VectorUDT() } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala index 0a6599b64c011..9352f40f372d3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Tokenizer.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.UnaryTransformer import org.apache.spark.ml.param.ParamMap -import org.apache.spark.sql.{DataType, StringType} +import org.apache.spark.sql.{DataType, StringType, ArrayType} /** * :: AlphaComponent :: @@ -36,4 +36,6 @@ class Tokenizer extends UnaryTransformer[String, Seq[String], Tokenizer] { protected override def validateInputType(inputType: DataType): Unit = { require(inputType == StringType, s"Input type must be string type but got $inputType.") } + + override protected def outputDataType: DataType = new ArrayType(StringType, false) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 8fd46aef4b99d..4b4340af543b0 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -17,13 +17,12 @@ package org.apache.spark.ml.param -import java.lang.reflect.Modifier - -import org.apache.spark.annotation.AlphaComponent - import scala.annotation.varargs import scala.collection.mutable +import java.lang.reflect.Modifier + +import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.Identifiable /** @@ -221,7 +220,9 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten /** * Puts a list of param pairs (overwrites if the input params exists). + * Not usable from Java */ + @varargs def put(paramPairs: ParamPair[_]*): this.type = { paramPairs.foreach { p => put(p.param.asInstanceOf[Param[Any]], p.value) @@ -282,6 +283,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten * where the latter overwrites this if there exists conflicts. */ def ++(other: ParamMap): ParamMap = { + // TODO: Provide a better method name for Java users. new ParamMap(this.map ++ other.map) } @@ -290,6 +292,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten * Adds all parameters from the input param map into this param map. */ def ++=(other: ParamMap): this.type = { + // TODO: Provide a better method name for Java users. this.map ++= other.map this } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala index 8c4c9c6cf6ae2..9fed513becddc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/BLAS.scala @@ -96,7 +96,9 @@ private[spark] object BLAS extends Serializable with Logging { * dot(x, y) */ def dot(x: Vector, y: Vector): Double = { - require(x.size == y.size) + require(x.size == y.size, + "BLAS.dot(x: Vector, y:Vector) was given Vectors with non-matching sizes:" + + " x.size = " + x.size + ", y.size = " + y.size) (x, y) match { case (dx: DenseVector, dy: DenseVector) => dot(dx, dy) From 7e758d709286e73d2c878d4a2d2b4606386142c7 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 4 Dec 2014 20:16:35 +0800 Subject: [PATCH 121/305] [FIX][DOC] Fix broken links in ml-guide.md and some minor changes in ScalaDoc. Author: Xiangrui Meng Closes #3601 from mengxr/SPARK-4575-fix and squashes the following commits: c559768 [Xiangrui Meng] minor code update ce94da8 [Xiangrui Meng] Java Bean -> JavaBean 0b5c182 [Xiangrui Meng] fix links in ml-guide --- docs/ml-guide.md | 8 ++++---- .../spark/examples/ml/JavaCrossValidatorExample.java | 1 - .../apache/spark/examples/ml/JavaSimpleParamsExample.java | 2 +- .../src/main/scala/org/apache/spark/ml/param/params.scala | 1 - 4 files changed, 5 insertions(+), 7 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index 012fbd91e698b..1c2e27341473b 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -31,7 +31,7 @@ E.g., a learning algorithm is an `Estimator` which trains on a dataset and produ * **[`Pipeline`](ml-guide.html#pipeline)**: A `Pipeline` chains multiple `Transformer`s and `Estimator`s together to specify an ML workflow. -* **[`Param`](ml-guide.html#param)**: All `Transformer`s and `Estimator`s now share a common API for specifying parameters. +* **[`Param`](ml-guide.html#parameters)**: All `Transformer`s and `Estimator`s now share a common API for specifying parameters. ## ML Dataset @@ -134,7 +134,7 @@ Each stage's `transform()` method updates the dataset and passes it to the next Spark ML `Estimator`s and `Transformer`s use a uniform API for specifying parameters. A [`Param`](api/scala/index.html#org.apache.spark.ml.param.Param) is a named parameter with self-contained documentation. -A [`ParamMap`](api/scala/index.html#org.apache.spark.ml.param.ParamMap)] is a set of (parameter, value) pairs. +A [`ParamMap`](api/scala/index.html#org.apache.spark.ml.param.ParamMap) is a set of (parameter, value) pairs. There are two main ways to pass parameters to an algorithm: @@ -148,7 +148,7 @@ This is useful if there are two algorithms with the `maxIter` parameter in a `Pi # Code Examples This section gives code examples illustrating the functionality discussed above. -There is not yet documentation for specific algorithms in Spark ML. For more info, please refer to the [API Documentation](api/scala/index.html). Spark ML algorithms are currently wrappers for MLlib algorithms, and the [MLlib programming guide](mllib-guide.html) has details on specific algorithms. +There is not yet documentation for specific algorithms in Spark ML. For more info, please refer to the [API Documentation](api/scala/index.html#org.apache.spark.ml.package). Spark ML algorithms are currently wrappers for MLlib algorithms, and the [MLlib programming guide](mllib-guide.html) has details on specific algorithms. ## Example: Estimator, Transformer, and Param @@ -492,7 +492,7 @@ The `ParamMap` which produces the best evaluation metric (averaged over the `$k$ `CrossValidator` finally fits the `Estimator` using the best `ParamMap` and the entire dataset. The following example demonstrates using `CrossValidator` to select from a grid of parameters. -To help construct the parameter grid, we use the [`ParamGridBuilder`](api/scala/index.html#org.apache.spark.ml.tuning.ParamGridGuilder) utility. +To help construct the parameter grid, we use the [`ParamGridBuilder`](api/scala/index.html#org.apache.spark.ml.tuning.ParamGridBuilder) utility. Note that cross-validation over a grid of parameters is expensive. E.g., in the example below, the parameter grid has 3 values for `hashingTF.numFeatures` and 2 values for `lr.regParam`, and `CrossValidator` uses 2 folds. This multiplies out to `$(3 \times 2) \times 2 = 12$` different models being trained. diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java index 3b156fa0482fc..f4b4f8d8c7b2f 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaCrossValidatorExample.java @@ -23,7 +23,6 @@ import org.apache.spark.SparkConf; import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.ml.Model; import org.apache.spark.ml.Pipeline; import org.apache.spark.ml.PipelineStage; import org.apache.spark.ml.classification.LogisticRegression; diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java index cf58f4dfaa15b..e25b271777ed4 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaSimpleParamsExample.java @@ -47,7 +47,7 @@ public static void main(String[] args) { JavaSQLContext jsql = new JavaSQLContext(jsc); // Prepare training data. - // We use LabeledPoint, which is a case class. Spark SQL can convert RDDs of Java Beans + // We use LabeledPoint, which is a JavaBean. Spark SQL can convert RDDs of JavaBeans // into SchemaRDDs, where it uses the bean metadata to infer the schema. List localTraining = Lists.newArrayList( new LabeledPoint(1.0, Vectors.dense(0.0, 1.1, 0.1)), diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala index 4b4340af543b0..04f9cfb1bfc2f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/params.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/params.scala @@ -220,7 +220,6 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten /** * Puts a list of param pairs (overwrites if the input params exists). - * Not usable from Java */ @varargs def put(paramPairs: ParamPair[_]*): this.type = { From 28c7acacef974fdabd2b9ecc20d0d6cf6c58728f Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Thu, 4 Dec 2014 10:21:03 -0800 Subject: [PATCH 122/305] [SPARK-4683][SQL] Add a beeline.cmd to run on Windows Tested locally with a Win7 VM. Connected to a Spark SQL Thrift server instance running on Mac OS X with the following command line: ``` bin\beeline.cmd -u jdbc:hive2://10.0.2.2:10000 -n lian ``` [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3599) Author: Cheng Lian Closes #3599 from liancheng/beeline.cmd and squashes the following commits: 79092e7 [Cheng Lian] Windows script for BeeLine --- bin/beeline.cmd | 21 +++++++++++++++++++++ 1 file changed, 21 insertions(+) create mode 100644 bin/beeline.cmd diff --git a/bin/beeline.cmd b/bin/beeline.cmd new file mode 100644 index 0000000000000..8293f311029dd --- /dev/null +++ b/bin/beeline.cmd @@ -0,0 +1,21 @@ +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +set SPARK_HOME=%~dp0.. +cmd /V /E /C %SPARK_HOME%\bin\spark-class.cmd org.apache.hive.beeline.BeeLine %* From 8106b1e36b2c2b9f5dc5d7252540e48cc3fc96d5 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Thu, 4 Dec 2014 11:52:47 -0800 Subject: [PATCH 123/305] [SPARK-4253] Ignore spark.driver.host in yarn-cluster and standalone-cluster modes In yarn-cluster and standalone-cluster modes, we don't know where driver will run until it is launched. If the `spark.driver.host` property is set on the submitting machine and propagated to the driver through SparkConf then this will lead to errors when the driver launches. This patch fixes this issue by dropping the `spark.driver.host` property in SparkSubmit when running in a cluster deploy mode. Author: WangTaoTheTonic Author: WangTao Closes #3112 from WangTaoTheTonic/SPARK4253 and squashes the following commits: ed1a25c [WangTaoTheTonic] revert unrelated formatting issue 02c4e49 [WangTao] add comment 32a3f3f [WangTaoTheTonic] ingore it in SparkSubmit instead of SparkContext 667cf24 [WangTaoTheTonic] document fix ff8d5f7 [WangTaoTheTonic] also ignore it in standalone cluster mode 2286e6b [WangTao] ignore spark.driver.host in yarn-cluster mode --- .../src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 5 +++++ .../spark/deploy/yarn/ApplicationMasterArguments.scala | 2 +- 2 files changed, 6 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 0c7d247519447..955cbd6dab96d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -281,6 +281,11 @@ object SparkSubmit { sysProps.getOrElseUpdate(k, v) } + // Ignore invalid spark.driver.host in cluster modes. + if (deployMode == CLUSTER) { + sysProps -= ("spark.driver.host") + } + // Resolve paths in certain spark properties val pathConfigs = Seq( "spark.jars", diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index 8b32c76d14037..d76a63276d752 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -36,7 +36,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var args = inputArgs - while (! args.isEmpty) { + while (!args.isEmpty) { // --num-workers, --worker-memory, and --worker-cores are deprecated since 1.0, // the properties with executor in their names are preferred. args match { From 8dae26f83818ee0f5ce8e5b083625170d2e901c5 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 4 Dec 2014 12:11:41 -0800 Subject: [PATCH 124/305] [HOTFIX] Fixing two issues with the release script. 1. The version replacement was still producing some false changes. 2. Uploads to the staging repo specifically. Author: Patrick Wendell Closes #3608 from pwendell/release-script and squashes the following commits: 3c63294 [Patrick Wendell] Fixing two issues with the release script: --- dev/create-release/create-release.sh | 31 ++++++++++++++++++---------- 1 file changed, 20 insertions(+), 11 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index e0aca467ac949..3b89aaba29609 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -39,7 +39,6 @@ RC_NAME=${RC_NAME:-rc2} M2_REPO=~/.m2/repository SPARK_REPO=$M2_REPO/org/apache/spark NEXUS_ROOT=https://repository.apache.org/service/local/staging -NEXUS_UPLOAD=$NEXUS_ROOT/deploy/maven2 NEXUS_PROFILE=d63f592e7eac0 # Profile for Spark staging uploads if [ -z "$JAVA_HOME" ]; then @@ -64,19 +63,28 @@ if [[ ! "$@" =~ --package-only ]]; then # NOTE: This is done "eagerly" i.e. we don't check if we can succesfully build # or before we coin the release commit. This helps avoid races where # other people add commits to this branch while we are in the middle of building. - old=" ${RELEASE_VERSION}-SNAPSHOT<\/version>" - new=" ${RELEASE_VERSION}<\/version>" - find . -name pom.xml -o -name package.scala | grep -v dev | xargs -I {} sed -i \ - -e "s/$old/$new/" {} + cur_ver="${RELEASE_VERSION}-SNAPSHOT" + rel_ver="${RELEASE_VERSION}" + next_ver="${NEXT_VERSION}-SNAPSHOT" + + old="^\( \{2,4\}\)${cur_ver}<\/version>$" + new="\1${rel_ver}<\/version>" + find . -name pom.xml | grep -v dev | xargs -I {} sed -i \ + -e "s/${old}/${new}/" {} + find . -name package.scala | grep -v dev | xargs -I {} sed -i \ + -e "s/${old}/${new}/" {} + git commit -a -m "Preparing Spark release $GIT_TAG" echo "Creating tag $GIT_TAG at the head of $GIT_BRANCH" git tag $GIT_TAG - old=" ${RELEASE_VERSION}<\/version>" - new=" ${NEXT_VERSION}-SNAPSHOT<\/version>" - find . -name pom.xml -o -name package.scala | grep -v dev | xargs -I {} sed -i \ + old="^\( \{2,4\}\)${rel_ver}<\/version>$" + new="\1${next_ver}<\/version>" + find . -name pom.xml | grep -v dev | xargs -I {} sed -i \ -e "s/$old/$new/" {} - git commit -a -m "Preparing development version ${NEXT_VERSION}-SNAPSHOT" + find . -name package.scala | grep -v dev | xargs -I {} sed -i \ + -e "s/${old}/${new}/" {} + git commit -a -m "Preparing development version $next_ver" git push origin $GIT_TAG git push origin HEAD:$GIT_BRANCH git checkout -f $GIT_TAG @@ -118,12 +126,13 @@ if [[ ! "$@" =~ --package-only ]]; then gpg --print-md SHA1 $file > $file.sha1 done - echo "Uplading files to $NEXUS_UPLOAD" + nexus_upload=$NEXUS_ROOT/deployByRepositoryId/$staged_repo_id + echo "Uplading files to $nexus_upload" for file in $(find . -type f) do # strip leading ./ file_short=$(echo $file | sed -e "s/\.\///") - dest_url="$NEXUS_UPLOAD/org/apache/spark/$file_short" + dest_url="$nexus_upload/org/apache/spark/$file_short" echo " Uploading $file_short" curl -u $ASF_USERNAME:$ASF_PASSWORD --upload-file $file_short $dest_url done From 794f3aec24acb578e258532ad0590554d07958ba Mon Sep 17 00:00:00 2001 From: alexdebrie Date: Thu, 4 Dec 2014 14:13:59 -0800 Subject: [PATCH 125/305] [SPARK-4745] Fix get_existing_cluster() function with multiple security groups The current get_existing_cluster() function would only find an instance belonged to a cluster if the instance's security groups == cluster_name + "-master" (or "-slaves"). This fix allows for multiple security groups by checking if the cluster_name + "-master" security group is in the list of groups for a particular instance. Author: alexdebrie Closes #3596 from alexdebrie/master and squashes the following commits: 9d51232 [alexdebrie] Fix get_existing_cluster() function with multiple security groups --- ec2/spark_ec2.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index b83decadc2988..5f9e484212635 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -504,9 +504,9 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): active = [i for i in res.instances if is_active(i)] for inst in active: group_names = [g.name for g in inst.groups] - if group_names == [cluster_name + "-master"]: + if (cluster_name + "-master") in group_names: master_nodes.append(inst) - elif group_names == [cluster_name + "-slaves"]: + elif (cluster_name + "-slaves") in group_names: slave_nodes.append(inst) if any((master_nodes, slave_nodes)): print "Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes)) From 743a889d2778f797aabc3b1e8146e7aa32b62a48 Mon Sep 17 00:00:00 2001 From: Saldanha Date: Thu, 4 Dec 2014 14:22:09 -0800 Subject: [PATCH 126/305] [SPARK-4459] Change groupBy type parameter from K to U Please see https://issues.apache.org/jira/browse/SPARK-4459 Author: Saldanha Closes #3327 from alokito/master and squashes the following commits: 54b1095 [Saldanha] [SPARK-4459] changed type parameter for keyBy from K to U d5f73c3 [Saldanha] [SPARK-4459] added keyBy test 316ad77 [Saldanha] SPARK-4459 changed type parameter for groupBy from K to U. 62ddd4b [Saldanha] SPARK-4459 added failing unit test --- .../apache/spark/api/java/JavaRDDLike.scala | 17 ++++---- .../java/org/apache/spark/JavaAPISuite.java | 41 +++++++++++++++++++ 2 files changed, 51 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index ac42294d56def..bd451634e53d2 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -211,8 +211,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JIterable[T]] = { - implicit val ctagK: ClassTag[K] = fakeClassTag + def groupBy[U](f: JFunction[T, U]): JavaPairRDD[U, JIterable[T]] = { + // The type parameter is U instead of K in order to work around a compiler bug; see SPARK-4459 + implicit val ctagK: ClassTag[U] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(fakeClassTag))) } @@ -221,10 +222,11 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JIterable[T]] = { - implicit val ctagK: ClassTag[K] = fakeClassTag + def groupBy[U](f: JFunction[T, U], numPartitions: Int): JavaPairRDD[U, JIterable[T]] = { + // The type parameter is U instead of K in order to work around a compiler bug; see SPARK-4459 + implicit val ctagK: ClassTag[U] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag - JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(fakeClassTag[K]))) + JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(fakeClassTag[U]))) } /** @@ -458,8 +460,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Creates tuples of the elements in this RDD by applying `f`. */ - def keyBy[K](f: JFunction[T, K]): JavaPairRDD[K, T] = { - implicit val ctag: ClassTag[K] = fakeClassTag + def keyBy[U](f: JFunction[T, U]): JavaPairRDD[U, T] = { + // The type parameter is U instead of K in order to work around a compiler bug; see SPARK-4459 + implicit val ctag: ClassTag[U] = fakeClassTag JavaPairRDD.fromRDD(rdd.keyBy(f)) } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 59c86eecac5e8..3ad4f2f193af4 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -323,6 +323,47 @@ public Boolean call(Integer x) { Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } + @Test + public void groupByOnPairRDD() { + // Regression test for SPARK-4459 + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function, Boolean> areOdd = + new Function, Boolean>() { + @Override + public Boolean call(Tuple2 x) { + return (x._1() % 2 == 0) && (x._2() % 2 == 0); + } + }; + JavaPairRDD pairRDD = rdd.zip(rdd); + JavaPairRDD>> oddsAndEvens = pairRDD.groupBy(areOdd); + Assert.assertEquals(2, oddsAndEvens.count()); + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + + oddsAndEvens = pairRDD.groupBy(areOdd, 1); + Assert.assertEquals(2, oddsAndEvens.count()); + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds + } + + @SuppressWarnings("unchecked") + @Test + public void keyByOnPairRDD() { + // Regression test for SPARK-4459 + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function, String> sumToString = + new Function, String>() { + @Override + public String call(Tuple2 x) { + return String.valueOf(x._1() + x._2()); + } + }; + JavaPairRDD pairRDD = rdd.zip(rdd); + JavaPairRDD> keyed = pairRDD.keyBy(sumToString); + Assert.assertEquals(7, keyed.count()); + Assert.assertEquals(1, (long) keyed.lookup("2").get(0)._1()); + } + @SuppressWarnings("unchecked") @Test public void cogroup() { From ab8177da2defab1ecd8bc0cd5a21f07be5b8d2c5 Mon Sep 17 00:00:00 2001 From: lewuathe Date: Thu, 4 Dec 2014 15:14:36 -0800 Subject: [PATCH 127/305] [SPARK-4652][DOCS] Add docs about spark-git-repo option There might be some cases when WIPS spark version need to be run on EC2 cluster. In order to setup this type of cluster more easily, add --spark-git-repo option description to ec2 documentation. Author: lewuathe Author: Josh Rosen Closes #3513 from Lewuathe/doc-for-development-spark-cluster and squashes the following commits: 6dae8ee [lewuathe] Wrap consistent with other descriptions cfaf9be [lewuathe] Add docs about spark-git-repo option (Editing / cleanup by Josh Rosen) --- docs/ec2-scripts.md | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/docs/ec2-scripts.md b/docs/ec2-scripts.md index 66bf5f1a855ed..ed51d0abb3a45 100644 --- a/docs/ec2-scripts.md +++ b/docs/ec2-scripts.md @@ -85,6 +85,11 @@ another. specified version of Spark. The `` can be a version number (e.g. "0.7.3") or a specific git hash. By default, a recent version will be used. +- `--spark-git-repo=` will let you run a custom version of + Spark that is built from the given git repository. By default, the + [Apache Github mirror](https://github.com/apache/spark) will be used. + When using a custom Spark version, `--spark-version` must be set to git + commit hash, such as 317e114, instead of a version number. - If one of your launches fails due to e.g. not having the right permissions on your private key file, you can run `launch` with the `--resume` option to restart the setup process on an existing cluster. From ed92b47e83c2882f0e76da78dc268577df820382 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 4 Dec 2014 16:32:20 -0800 Subject: [PATCH 128/305] [SPARK-4397] Move object RDD to the front of RDD.scala. I ran into multiple cases that SBT/Scala compiler was confused by the implicits in continuous compilation mode. Adding explicit return types fixes the problem. Author: Reynold Xin Closes #3580 from rxin/rdd-implicit and squashes the following commits: ee32fcd [Reynold Xin] Move object RDD to the end of the file. b8562c9 [Reynold Xin] Merge branch 'master' of github.com:apache/spark into rdd-implicit d4e9f85 [Reynold Xin] Code review. a836a37 [Reynold Xin] Move object RDD to the front of RDD.scala. --- .../scala/org/apache/spark/SparkContext.scala | 8 ++--- .../main/scala/org/apache/spark/rdd/RDD.scala | 29 ++++++++++++++----- 2 files changed, 26 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 532f292952f05..aded7c12e274e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1758,7 +1758,7 @@ object SparkContext extends Logging { @deprecated("Replaced by implicit functions in WritableConverter. This is kept here only for " + "backward compatibility.", "1.3.0") - def writableWritableConverter[T <: Writable]() = + def writableWritableConverter[T <: Writable](): WritableConverter[T] = WritableConverter.writableWritableConverter() /** @@ -2017,15 +2017,15 @@ object WritableConverter { simpleWritableConverter[Boolean, BooleanWritable](_.get) implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = { - simpleWritableConverter[Array[Byte], BytesWritable](bw => + simpleWritableConverter[Array[Byte], BytesWritable] { bw => // getBytes method returns array which is longer then data to be returned Arrays.copyOfRange(bw.getBytes, 0, bw.getLength) - ) + } } implicit def stringWritableConverter(): WritableConverter[String] = simpleWritableConverter[String, Text](_.toString) - implicit def writableWritableConverter[T <: Writable]() = + implicit def writableWritableConverter[T <: Writable](): WritableConverter[T] = new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T]) } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 0bd616ec24fcb..214f22bc5b603 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1398,6 +1398,13 @@ abstract class RDD[T: ClassTag]( } } + +/** + * Defines implicit functions that provide extra functionalities on RDDs of specific types. + * + * For example, [[RDD.rddToPairRDDFunctions]] converts an RDD into a [[PairRDDFunctions]] for + * key-value-pair RDDs, and enabling extra functionalities such as [[PairRDDFunctions.reduceByKey]]. + */ object RDD { // The following implicit functions were in SparkContext before 1.2 and users had to @@ -1406,22 +1413,30 @@ object RDD { // compatibility and forward to the following functions directly. implicit def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) - (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null): PairRDDFunctions[K, V] = { new PairRDDFunctions(rdd) } - implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) + implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]): AsyncRDDActions[T] = { + new AsyncRDDActions(rdd) + } implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( - rdd: RDD[(K, V)]) = + rdd: RDD[(K, V)]): SequenceFileRDDFunctions[K, V] = { new SequenceFileRDDFunctions(rdd) + } - implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag]( - rdd: RDD[(K, V)]) = + implicit def rddToOrderedRDDFunctions[K : Ordering : ClassTag, V: ClassTag](rdd: RDD[(K, V)]) + : OrderedRDDFunctions[K, V, (K, V)] = { new OrderedRDDFunctions[K, V, (K, V)](rdd) + } - implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]) = new DoubleRDDFunctions(rdd) + implicit def doubleRDDToDoubleRDDFunctions(rdd: RDD[Double]): DoubleRDDFunctions = { + new DoubleRDDFunctions(rdd) + } - implicit def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) = + implicit def numericRDDToDoubleRDDFunctions[T](rdd: RDD[T])(implicit num: Numeric[T]) + : DoubleRDDFunctions = { new DoubleRDDFunctions(rdd.map(x => num.toDouble(x))) + } } From ddfc09c36381a0880dfa6778be2ca0bc7d80febf Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Thu, 4 Dec 2014 18:14:36 -0800 Subject: [PATCH 129/305] [SPARK-4421] Wrong link in spark-standalone.html Modified the link of building Spark. Author: Masayoshi TSUZUKI Closes #3279 from tsudukim/feature/SPARK-4421 and squashes the following commits: 56e31c1 [Masayoshi TSUZUKI] Modified the link of building Spark. --- docs/spark-standalone.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index a3028aa86dc45..d503bc961e200 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -10,7 +10,7 @@ In addition to running on the Mesos or YARN cluster managers, Spark also provide # Installing Spark Standalone to a Cluster -To install Spark Standalone mode, you simply place a compiled version of Spark on each node on the cluster. You can obtain pre-built versions of Spark with each release or [build it yourself](index.html#building). +To install Spark Standalone mode, you simply place a compiled version of Spark on each node on the cluster. You can obtain pre-built versions of Spark with each release or [build it yourself](building-spark.html). # Starting a Cluster Manually From 15cf3b0125fe238dea2ce13e703034ba7cef477f Mon Sep 17 00:00:00 2001 From: Andy Konwinski Date: Thu, 4 Dec 2014 18:27:02 -0800 Subject: [PATCH 130/305] Fix typo in Spark SQL docs. Author: Andy Konwinski Closes #3611 from andyk/patch-3 and squashes the following commits: 7bab333 [Andy Konwinski] Fix typo in Spark SQL docs. --- docs/sql-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 85d446b9da0e7..be284fbe217a5 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -278,7 +278,7 @@ performed on JSON files. from pyspark.sql import SQLContext, Row sqlContext = SQLContext(sc) -# Load a text file and convert each line to a dictionary. +# Load a text file and convert each line to a Row. lines = sc.textFile("examples/src/main/resources/people.txt") parts = lines.map(lambda l: l.split(",")) people = parts.map(lambda p: Row(name=p[0], age=int(p[1]))) From ca379039f701e423fa07933db4e063cb85d0236a Mon Sep 17 00:00:00 2001 From: Masayoshi TSUZUKI Date: Thu, 4 Dec 2014 19:33:02 -0800 Subject: [PATCH 131/305] [SPARK-4464] Description about configuration options need to be modified in docs. Added description about -h and -host. Modified description about -i and -ip which are now deprecated. Added description about --properties-file. Author: Masayoshi TSUZUKI Closes #3329 from tsudukim/feature/SPARK-4464 and squashes the following commits: 6c07caf [Masayoshi TSUZUKI] [SPARK-4464] Description about configuration options need to be modified in docs. --- docs/spark-standalone.md | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index d503bc961e200..ae7b81d5bb71f 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -34,8 +34,12 @@ Finally, the following configuration options can be passed to the master and wor - - + + + + + + @@ -57,6 +61,10 @@ Finally, the following configuration options can be passed to the master and wor + + + +
ArgumentMeaning
-i IP, --ip IPIP address or DNS name to listen on-h HOST, --host HOSTHostname to listen on
-i HOST, --ip HOSTHostname to listen on (deprecated, use -h or --host)
-p PORT, --port PORT-d DIR, --work-dir DIR Directory to use for scratch space and job output logs (default: SPARK_HOME/work); only on worker
--properties-file FILEPath to a custom Spark properties file to load (default: conf/spark-defaults.conf)
From 87437df036305c6f467dc6e0bf7891d27d266a6b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 4 Dec 2014 21:53:38 -0800 Subject: [PATCH 132/305] Revert "[HOT FIX] [YARN] Check whether `/lib` exists before listing its files" This reverts commit 90ec643e9af4c8bbb9000edca08c07afb17939c7. --- .../apache/spark/deploy/yarn/ClientBase.scala | 27 +++++++++---------- 1 file changed, 12 insertions(+), 15 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 290d9943a5077..8e4360ea4476b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -243,21 +243,18 @@ private[spark] trait ClientBase extends Logging { val libsURI = new URI(libsDir) val jarLinks = ListBuffer.empty[String] if (libsURI.getScheme != LOCAL_SCHEME) { - val localPath = getQualifiedLocalPath(libsURI) - val localFs = FileSystem.get(localPath.toUri, hadoopConf) - if (localFs.exists(localPath)) { - val jars = localFs.listFiles(localPath, /* recursive */ false) - while (jars.hasNext) { - val jar = jars.next() - val name = jar.getPath.getName - if (name.startsWith("datanucleus-")) { - // copy to remote and add to classpath - val src = jar.getPath - val destPath = copyFileToRemote(dst, src, replication) - distCacheMgr.addResource(localFs, hadoopConf, destPath, - localResources, LocalResourceType.FILE, name, statCache) - jarLinks += name - } + val localURI = getQualifiedLocalPath(libsURI).toUri() + val jars = FileSystem.get(localURI, hadoopConf).listFiles(new Path(localURI.getPath), false) + while (jars.hasNext) { + val jar = jars.next() + val name = jar.getPath.getName + if (name.startsWith("datanucleus-")) { + // copy to remote and add to classpath + val src = jar.getPath + val destPath = copyFileToRemote(dst, src, replication) + distCacheMgr.addResource(fs, hadoopConf, destPath, + localResources, LocalResourceType.FILE, name, statCache) + jarLinks += name } } } else { From fd8525334c443fbdc7dd58a62cb1aecae51b460b Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 4 Dec 2014 21:53:49 -0800 Subject: [PATCH 133/305] Revert "SPARK-2624 add datanucleus jars to the container in yarn-cluster" This reverts commit a975dc32799bb8a14f9e1c76defaaa7cfbaf8b53. --- docs/running-on-yarn.md | 15 ---- .../apache/spark/deploy/yarn/ClientBase.scala | 66 ---------------- .../spark/deploy/yarn/ClientBaseSuite.scala | 76 ------------------- 3 files changed, 157 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index e97ac9f0c4a00..16897dbb65311 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -139,21 +139,6 @@ Most of the configs are the same for Spark on YARN as for other deployment modes The maximum number of threads to use in the application master for launching executor containers. - - spark.yarn.datanucleus.dir - $SPARK_HOME/lib - - The location of the DataNucleus jars, in case overriding the default location is desired. - By default, Spark on YARN will use the DataNucleus jars installed at - $SPARK_HOME/lib, but the jars can also be in a world-readable location on HDFS. - This allows YARN to cache it on nodes so that it doesn't need to be distributed each time an - application runs. To point to a directory on HDFS, for example, set this configuration to - "hdfs:///some/path". - - This is required because the datanucleus jars cannot be packaged into the - assembly jar due to metadata conflicts (involving plugin.xml.) - - # Launching Spark on YARN diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 8e4360ea4476b..f95d72379171c 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -18,7 +18,6 @@ package org.apache.spark.deploy.yarn import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} -import java.io.{File, FilenameFilter} import scala.collection.JavaConversions._ import scala.collection.mutable.{HashMap, ListBuffer, Map} @@ -224,48 +223,10 @@ private[spark] trait ClientBase extends Logging { } } } - if (cachedSecondaryJarLinks.nonEmpty) { sparkConf.set(CONF_SPARK_YARN_SECONDARY_JARS, cachedSecondaryJarLinks.mkString(",")) } - /** - * Do the same for datanucleus jars, if they exist in spark home. Find all datanucleus-* jars, - * copy them to the remote fs, and add them to the class path. - * - * This is necessary because the datanucleus jars cannot be included in the assembly jar due - * to metadata conflicts involving plugin.xml. At the time of writing, these are the only - * jars that cannot be distributed with the uber jar and have to be treated differently. - * - * For more details, see SPARK-2624, and https://github.com/apache/spark/pull/3238 - */ - for (libsDir <- dataNucleusJarsDir(sparkConf)) { - val libsURI = new URI(libsDir) - val jarLinks = ListBuffer.empty[String] - if (libsURI.getScheme != LOCAL_SCHEME) { - val localURI = getQualifiedLocalPath(libsURI).toUri() - val jars = FileSystem.get(localURI, hadoopConf).listFiles(new Path(localURI.getPath), false) - while (jars.hasNext) { - val jar = jars.next() - val name = jar.getPath.getName - if (name.startsWith("datanucleus-")) { - // copy to remote and add to classpath - val src = jar.getPath - val destPath = copyFileToRemote(dst, src, replication) - distCacheMgr.addResource(fs, hadoopConf, destPath, - localResources, LocalResourceType.FILE, name, statCache) - jarLinks += name - } - } - } else { - jarLinks += libsURI.toString + Path.SEPARATOR + "*" - } - - if (jarLinks.nonEmpty) { - sparkConf.set(CONF_SPARK_DATANUCLEUS_JARS, jarLinks.mkString(",")) - } - } - localResources } @@ -590,13 +551,6 @@ private[spark] object ClientBase extends Logging { // Internal config to propagate the location of the user's jar to the driver/executors val CONF_SPARK_USER_JAR = "spark.yarn.user.jar" - // Location of the datanucleus jars - val CONF_SPARK_DATANUCLEUS_DIR = "spark.yarn.datanucleus.dir" - - // Internal config to propagate the locations of datanucleus jars found to add to the - // classpath of the executors. Value should be a comma-separated list of paths to each jar. - val CONF_SPARK_DATANUCLEUS_JARS = "spark.yarn.datanucleus.jars" - // Internal config to propagate the locations of any extra jars to add to the classpath // of the executors val CONF_SPARK_YARN_SECONDARY_JARS = "spark.yarn.secondary.jars" @@ -629,19 +583,6 @@ private[spark] object ClientBase extends Logging { } } - /** - * Find the user-defined provided jars directory if configured, or return SPARK_HOME/lib if not. - * - * This method first looks for $CONF_SPARK_DATANUCLEUS_DIR inside the SparkConf, then looks for - * Spark home inside the the SparkConf and the user environment. - */ - private def dataNucleusJarsDir(conf: SparkConf): Option[String] = { - conf.getOption(CONF_SPARK_DATANUCLEUS_DIR).orElse { - val sparkHome = conf.getOption("spark.home").orElse(sys.env.get("SPARK_HOME")) - sparkHome.map(path => path + Path.SEPARATOR + "lib") - } - } - /** * Return the path to the given application's staging directory. */ @@ -743,13 +684,6 @@ private[spark] object ClientBase extends Logging { addUserClasspath(args, sparkConf, env) } - // Add datanucleus jars to classpath - for (entries <- sparkConf.getOption(CONF_SPARK_DATANUCLEUS_JARS)) { - entries.split(",").filter(_.nonEmpty).foreach { entry => - addFileToClasspath(entry, null, env) - } - } - // Append all jar files under the working directory to the classpath. addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + "*", env) } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala index b055e9b72dc61..17b79ae1d82c4 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala @@ -21,7 +21,6 @@ import java.io.File import java.net.URI import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.yarn.api.ApplicationConstants.Environment @@ -105,81 +104,6 @@ class ClientBaseSuite extends FunSuite with Matchers { cp should not contain (ClientBase.APP_JAR) } - test("DataNucleus in classpath") { - val dnJars = "local:/dn/core.jar,/dn/api.jar" - val conf = new Configuration() - val sparkConf = new SparkConf() - .set(ClientBase.CONF_SPARK_JAR, SPARK) - .set(ClientBase.CONF_SPARK_DATANUCLEUS_JARS, dnJars) - val env = new MutableHashMap[String, String]() - val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - - ClientBase.populateClasspath(args, conf, sparkConf, env) - - val cp = env("CLASSPATH").split(File.pathSeparator) - s"$dnJars".split(",").foreach({ entry => - val uri = new URI(entry) - if (ClientBase.LOCAL_SCHEME.equals(uri.getScheme())) { - cp should contain (uri.getPath()) - } else { - cp should not contain (uri.getPath()) - } - }) - } - - test("DataNucleus using local:") { - val dnDir = "local:/datanucleus" - val conf = new Configuration() - val sparkConf = new SparkConf() - .set(ClientBase.CONF_SPARK_JAR, SPARK) - .set(ClientBase.CONF_SPARK_DATANUCLEUS_DIR, dnDir) - val yarnConf = new YarnConfiguration() - val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - - val client = spy(new DummyClient(args, conf, sparkConf, yarnConf)) - doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), - any(classOf[Path]), anyShort(), anyBoolean()) - - val tempDir = Utils.createTempDir() - try { - client.prepareLocalResources(tempDir.getAbsolutePath()) - val jars = sparkConf.get(ClientBase.CONF_SPARK_DATANUCLEUS_JARS).split(",") - val uri = new URI(dnDir) - jars should contain (uri.toString + Path.SEPARATOR + "*") - } finally { - Utils.deleteRecursively(tempDir) - } - } - - test("DataNucleus using file:") { - val dnDir = Utils.createTempDir() - val tempDir = Utils.createTempDir() - - try { - // create mock datanucleus jar - val tempJar = File.createTempFile("datanucleus-", null, dnDir) - - val conf = new Configuration() - val sparkConf = new SparkConf() - .set(ClientBase.CONF_SPARK_JAR, SPARK) - .set(ClientBase.CONF_SPARK_DATANUCLEUS_DIR, dnDir.toURI.toString) - val yarnConf = new YarnConfiguration() - val args = new ClientArguments(Array("--jar", USER, "--addJars", ADDED), sparkConf) - - val client = spy(new DummyClient(args, conf, sparkConf, yarnConf)) - doReturn(new Path("/")).when(client).copyFileToRemote(any(classOf[Path]), - any(classOf[Path]), anyShort(), anyBoolean()) - - client.prepareLocalResources(tempDir.getAbsolutePath()) - - val jars = sparkConf.get(ClientBase.CONF_SPARK_DATANUCLEUS_JARS).split(",") - jars should contain (tempJar.getName) - } finally { - Utils.deleteRecursively(dnDir) - Utils.deleteRecursively(tempDir) - } - } - test("Jar path propagation through SparkConf") { val conf = new Configuration() val sparkConf = new SparkConf().set(ClientBase.CONF_SPARK_JAR, SPARK) From f5801e813f3c2573ebaf1af839341489ddd3ec78 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 4 Dec 2014 22:25:21 -0800 Subject: [PATCH 134/305] [SPARK-4753][SQL] Use catalyst for partition pruning in newParquet. Author: Michael Armbrust Closes #3613 from marmbrus/parquetPartitionPruning and squashes the following commits: 4f138f8 [Michael Armbrust] Use catalyst for partition pruning in newParquet. --- .../apache/spark/sql/parquet/newParquet.scala | 58 +++++++++---------- 1 file changed, 28 insertions(+), 30 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 14f8659f15b3f..2e0c6c51c00e5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce.{JobContext, InputSplit, Job} +import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate import parquet.hadoop.ParquetInputFormat import parquet.hadoop.util.ContextUtil @@ -31,8 +32,8 @@ import org.apache.spark.{Partition => SparkPartition, Logging} import org.apache.spark.rdd.{NewHadoopPartition, RDD} import org.apache.spark.sql.{SQLConf, Row, SQLContext} -import org.apache.spark.sql.catalyst.expressions.{SpecificMutableRow, And, Expression, Attribute} -import org.apache.spark.sql.catalyst.types.{IntegerType, StructField, StructType} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types.{StringType, IntegerType, StructField, StructType} import org.apache.spark.sql.sources._ import scala.collection.JavaConversions._ @@ -151,8 +152,6 @@ case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext) override def buildScan(output: Seq[Attribute], predicates: Seq[Expression]): RDD[Row] = { // This is mostly a hack so that we can use the existing parquet filter code. val requiredColumns = output.map(_.name) - // TODO: Parquet filters should be based on data sources API, not catalyst expressions. - val filters = DataSourceStrategy.selectFilters(predicates) val job = new Job(sparkContext.hadoopConfiguration) ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) @@ -160,35 +159,34 @@ case class ParquetRelation2(path: String)(@transient val sqlContext: SQLContext) val requestedSchema = StructType(requiredColumns.map(schema(_))) - // TODO: Make folder based partitioning a first class citizen of the Data Sources API. - val partitionFilters = filters.collect { - case e @ EqualTo(attr, value) if partitionKeys.contains(attr) => - logInfo(s"Parquet scan partition filter: $attr=$value") - (p: Partition) => p.partitionValues(attr) == value - - case e @ In(attr, values) if partitionKeys.contains(attr) => - logInfo(s"Parquet scan partition filter: $attr IN ${values.mkString("{", ",", "}")}") - val set = values.toSet - (p: Partition) => set.contains(p.partitionValues(attr)) - - case e @ GreaterThan(attr, value) if partitionKeys.contains(attr) => - logInfo(s"Parquet scan partition filter: $attr > $value") - (p: Partition) => p.partitionValues(attr).asInstanceOf[Int] > value.asInstanceOf[Int] - - case e @ GreaterThanOrEqual(attr, value) if partitionKeys.contains(attr) => - logInfo(s"Parquet scan partition filter: $attr >= $value") - (p: Partition) => p.partitionValues(attr).asInstanceOf[Int] >= value.asInstanceOf[Int] + val partitionKeySet = partitionKeys.toSet + val rawPredicate = + predicates + .filter(_.references.map(_.name).toSet.subsetOf(partitionKeySet)) + .reduceOption(And) + .getOrElse(Literal(true)) + + // Translate the predicate so that it reads from the information derived from the + // folder structure + val castedPredicate = rawPredicate transform { + case a: AttributeReference => + val idx = partitionKeys.indexWhere(a.name == _) + BoundReference(idx, IntegerType, nullable = true) + } - case e @ LessThan(attr, value) if partitionKeys.contains(attr) => - logInfo(s"Parquet scan partition filter: $attr < $value") - (p: Partition) => p.partitionValues(attr).asInstanceOf[Int] < value.asInstanceOf[Int] + val inputData = new GenericMutableRow(partitionKeys.size) + val pruningCondition = InterpretedPredicate(castedPredicate) - case e @ LessThanOrEqual(attr, value) if partitionKeys.contains(attr) => - logInfo(s"Parquet scan partition filter: $attr <= $value") - (p: Partition) => p.partitionValues(attr).asInstanceOf[Int] <= value.asInstanceOf[Int] - } + val selectedPartitions = + if (partitionKeys.nonEmpty && predicates.nonEmpty) { + partitions.filter { part => + inputData(0) = part.partitionValues.values.head + pruningCondition(inputData) + } + } else { + partitions + } - val selectedPartitions = partitions.filter(p => partitionFilters.forall(_(p))) val fs = FileSystem.get(new java.net.URI(path), sparkContext.hadoopConfiguration) val selectedFiles = selectedPartitions.flatMap(_.files).map(f => fs.makeQualified(f.getPath)) // FileInputFormat cannot handle empty lists. From 6f61e1f961826a6c9e98a66d10b271b7e3c7dd55 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 5 Dec 2014 10:27:40 -0800 Subject: [PATCH 135/305] [SPARK-4761][SQL] Enables Kryo by default in Spark SQL Thrift server Enables Kryo and disables reference tracking by default in Spark SQL Thrift server. Configurations explicitly defined by users in `spark-defaults.conf` are respected (the Thrift server is started by `spark-submit`, which handles configuration properties properly). [Review on Reviewable](https://reviewable.io/reviews/apache/spark/3621) Author: Cheng Lian Closes #3621 from liancheng/kryo-by-default and squashes the following commits: 70c2775 [Cheng Lian] Enables Kryo by default in Spark SQL Thrift server --- .../spark/sql/hive/thriftserver/SparkSQLEnv.scala | 14 ++++++++++++-- 1 file changed, 12 insertions(+), 2 deletions(-) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 89732c939b0ec..158c225159720 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -32,11 +32,21 @@ private[hive] object SparkSQLEnv extends Logging { def init() { if (hiveContext == null) { - val sparkConf = new SparkConf() + val sparkConf = new SparkConf(loadDefaults = true) + val maybeSerializer = sparkConf.getOption("spark.serializer") + val maybeKryoReferenceTracking = sparkConf.getOption("spark.kryo.referenceTracking") + + sparkConf .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}") .set("spark.sql.hive.version", HiveShim.version) - sparkContext = new SparkContext(sparkConf) + .set( + "spark.serializer", + maybeSerializer.getOrElse("org.apache.spark.serializer.KryoSerializer")) + .set( + "spark.kryo.referenceTracking", + maybeKryoReferenceTracking.getOrElse("false")) + sparkContext = new SparkContext(sparkConf) sparkContext.addSparkListener(new StatsReportListener()) hiveContext = new HiveContext(sparkContext) From 98a7d09978eeb775600ff41f9cc6ae8622026b71 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Fri, 5 Dec 2014 12:00:32 -0800 Subject: [PATCH 136/305] [SPARK-4005][CORE] handle message replies in receive instead of in the individual private methods In BlockManagermasterActor, when handling message type UpdateBlockInfo, the message replies is in handled in individual private methods, should handle it in receive of Akka. Author: Zhang, Liye Closes #2853 from liyezhang556520/akkaRecv and squashes the following commits: 9b06f0a [Zhang, Liye] remove the unreachable code bf518cd [Zhang, Liye] change the indent 242166b [Zhang, Liye] modified accroding to the comments d4b929b [Zhang, Liye] [SPARK-4005][CORE] handle message replies in receive instead of in the individual private methods --- .../spark/storage/BlockManagerMasterActor.scala | 17 +++++++---------- 1 file changed, 7 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 685b2e11440fb..9cbda41223a8b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -73,9 +73,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case UpdateBlockInfo( blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) => - // TODO: Ideally we want to handle all the message replies in receive instead of in the - // individual private methods. - updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) + sender ! updateBlockInfo( + blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) case GetLocations(blockId) => sender ! getLocations(blockId) @@ -355,23 +354,21 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus storageLevel: StorageLevel, memSize: Long, diskSize: Long, - tachyonSize: Long) { + tachyonSize: Long): Boolean = { if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.isDriver && !isLocal) { // We intentionally do not register the master (except in local mode), // so we should not indicate failure. - sender ! true + return true } else { - sender ! false + return false } - return } if (blockId == null) { blockManagerInfo(blockManagerId).updateLastSeenMs() - sender ! true - return + return true } blockManagerInfo(blockManagerId).updateBlockInfo( @@ -395,7 +392,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus if (locations.size == 0) { blockLocations.remove(blockId) } - sender ! true + true } private def getLocations(blockId: BlockId): Seq[BlockManagerId] = { From 6eb1b6f6204ea3c8083af3fb9cd990d9f3dac89d Mon Sep 17 00:00:00 2001 From: CrazyJvm Date: Fri, 5 Dec 2014 13:42:13 -0800 Subject: [PATCH 137/305] Streaming doc : do you mean inadvertently? Author: CrazyJvm Closes #3620 from CrazyJvm/streaming-foreachRDD and squashes the following commits: b72886b [CrazyJvm] do you mean inadvertently? --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 44a1f3ad7560b..5ebe834a32d31 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -1081,7 +1081,7 @@ Some of the common mistakes to avoid are as follows. - Often writing data to external system requires creating a connection object (e.g. TCP connection to a remote server) and using it to send data to a remote system. -For this purpose, a developer may inadvertantly try creating a connection object at +For this purpose, a developer may inadvertently try creating a connection object at the Spark driver, but try to use it in a Spark worker to save records in the RDDs. For example (in Scala), From e895e0cbecbbec1b412ff21321e57826d2d0a982 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Sat, 6 Dec 2014 00:56:51 -0800 Subject: [PATCH 138/305] [SPARK-3623][GraphX] GraphX should support the checkpoint operation Author: GuoQiang Li Closes #2631 from witgo/SPARK-3623 and squashes the following commits: a70c500 [GuoQiang Li] Remove java related 4d1e249 [GuoQiang Li] Add comments e682724 [GuoQiang Li] Graph should support the checkpoint operation --- .../scala/org/apache/spark/graphx/Graph.scala | 8 +++++++ .../apache/spark/graphx/impl/GraphImpl.scala | 5 +++++ .../org/apache/spark/graphx/GraphSuite.scala | 21 +++++++++++++++++++ 3 files changed, 34 insertions(+) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 637791543514c..23538b71562de 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -96,6 +96,14 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab */ def cache(): Graph[VD, ED] + /** + * Mark this Graph for checkpointing. It will be saved to a file inside the checkpoint + * directory set with SparkContext.setCheckpointDir() and all references to its parent + * RDDs will be removed. It is strongly recommended that this Graph is persisted in + * memory, otherwise saving it on a file will require recomputation. + */ + def checkpoint(): Unit + /** * Uncaches only the vertices of this graph, leaving the edges alone. This is useful in iterative * algorithms that modify the vertex attributes but reuse the edges. This method can be used to diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 0eae2a673874a..a617d84aea9d4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -65,6 +65,11 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( this } + override def checkpoint(): Unit = { + vertices.checkpoint() + replicatedVertexView.edges.checkpoint() + } + override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { vertices.unpersist(blocking) // TODO: unpersist the replicated vertices in `replicatedVertexView` but leave the edges alone diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index a05d1ddb21295..9da0064104fb6 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.graphx import org.scalatest.FunSuite +import com.google.common.io.Files + import org.apache.spark.SparkContext import org.apache.spark.graphx.Graph._ import org.apache.spark.graphx.PartitionStrategy._ @@ -365,4 +367,23 @@ class GraphSuite extends FunSuite with LocalSparkContext { } } + test("checkpoint") { + val checkpointDir = Files.createTempDir() + checkpointDir.deleteOnExit() + withSpark { sc => + sc.setCheckpointDir(checkpointDir.getAbsolutePath) + val ring = (0L to 100L).zip((1L to 99L) :+ 0L).map { case (a, b) => Edge(a, b, 1)} + val rdd = sc.parallelize(ring) + val graph = Graph.fromEdges(rdd, 1.0F) + graph.checkpoint() + graph.edges.map(_.attr).count() + graph.vertices.map(_._2).count() + + val edgesDependencies = graph.edges.partitionsRDD.dependencies + val verticesDependencies = graph.vertices.partitionsRDD.dependencies + assert(edgesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]])) + assert(verticesDependencies.forall(_.rdd.isInstanceOf[CheckpointRDD[_]])) + } + } + } From 2e6b736b0e6e5920d0523533c87832a53211db42 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 7 Dec 2014 19:36:08 -0800 Subject: [PATCH 139/305] [SPARK-4646] Replace Scala.util.Sorting.quickSort with Sorter(TimSort) in Spark This patch just replaces a native quick sorter with Sorter(TimSort) in Spark. It could get performance gains by ~8% in my quick experiments. Author: Takeshi Yamamuro Closes #3507 from maropu/TimSortInEdgePartitionBuilderSpike and squashes the following commits: 8d4e5d2 [Takeshi Yamamuro] Remove a wildcard import 3527e00 [Takeshi Yamamuro] Replace Scala.util.Sorting.quickSort with Sorter(TimSort) in Spark --- .../scala/org/apache/spark/graphx/Edge.scala | 30 ++++++++++++++ .../graphx/impl/EdgePartitionBuilder.scala | 39 ++++++++++++++++--- 2 files changed, 64 insertions(+), 5 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala index 7e842ec4cc82f..ecc37dcaad1fe 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Edge.scala @@ -17,6 +17,8 @@ package org.apache.spark.graphx +import org.apache.spark.util.collection.SortDataFormat + /** * A single directed edge consisting of a source id, target id, * and the data associated with the edge. @@ -65,4 +67,32 @@ object Edge { else 1 } } + + private[graphx] def edgeArraySortDataFormat[ED] = new SortDataFormat[Edge[ED], Array[Edge[ED]]] { + override def getKey(data: Array[Edge[ED]], pos: Int): Edge[ED] = { + data(pos) + } + + override def swap(data: Array[Edge[ED]], pos0: Int, pos1: Int): Unit = { + val tmp = data(pos0) + data(pos0) = data(pos1) + data(pos1) = tmp + } + + override def copyElement( + src: Array[Edge[ED]], srcPos: Int, + dst: Array[Edge[ED]], dstPos: Int) { + dst(dstPos) = src(srcPos) + } + + override def copyRange( + src: Array[Edge[ED]], srcPos: Int, + dst: Array[Edge[ED]], dstPos: Int, length: Int) { + System.arraycopy(src, srcPos, dst, dstPos, length) + } + + override def allocate(length: Int): Array[Edge[ED]] = { + new Array[Edge[ED]](length) + } + } } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index b0cb0fe47d461..409cf60977f6f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -18,12 +18,10 @@ package org.apache.spark.graphx.impl import scala.reflect.ClassTag -import scala.util.Sorting - -import org.apache.spark.util.collection.{BitSet, OpenHashSet, PrimitiveVector} import org.apache.spark.graphx._ import org.apache.spark.graphx.util.collection.GraphXPrimitiveKeyOpenHashMap +import org.apache.spark.util.collection.{SortDataFormat, Sorter, PrimitiveVector} /** Constructs an EdgePartition from scratch. */ private[graphx] @@ -38,7 +36,8 @@ class EdgePartitionBuilder[@specialized(Long, Int, Double) ED: ClassTag, VD: Cla def toEdgePartition: EdgePartition[ED, VD] = { val edgeArray = edges.trim().array - Sorting.quickSort(edgeArray)(Edge.lexicographicOrdering) + new Sorter(Edge.edgeArraySortDataFormat[ED]) + .sort(edgeArray, 0, edgeArray.length, Edge.lexicographicOrdering) val localSrcIds = new Array[Int](edgeArray.size) val localDstIds = new Array[Int](edgeArray.size) val data = new Array[ED](edgeArray.size) @@ -97,7 +96,8 @@ class ExistingEdgePartitionBuilder[ def toEdgePartition: EdgePartition[ED, VD] = { val edgeArray = edges.trim().array - Sorting.quickSort(edgeArray)(EdgeWithLocalIds.lexicographicOrdering) + new Sorter(EdgeWithLocalIds.edgeArraySortDataFormat[ED]) + .sort(edgeArray, 0, edgeArray.length, EdgeWithLocalIds.lexicographicOrdering) val localSrcIds = new Array[Int](edgeArray.size) val localDstIds = new Array[Int](edgeArray.size) val data = new Array[ED](edgeArray.size) @@ -140,4 +140,33 @@ private[impl] object EdgeWithLocalIds { } } + private[graphx] def edgeArraySortDataFormat[ED] + = new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] { + override def getKey( + data: Array[EdgeWithLocalIds[ED]], pos: Int): EdgeWithLocalIds[ED] = { + data(pos) + } + + override def swap(data: Array[EdgeWithLocalIds[ED]], pos0: Int, pos1: Int): Unit = { + val tmp = data(pos0) + data(pos0) = data(pos1) + data(pos1) = tmp + } + + override def copyElement( + src: Array[EdgeWithLocalIds[ED]], srcPos: Int, + dst: Array[EdgeWithLocalIds[ED]], dstPos: Int) { + dst(dstPos) = src(srcPos) + } + + override def copyRange( + src: Array[EdgeWithLocalIds[ED]], srcPos: Int, + dst: Array[EdgeWithLocalIds[ED]], dstPos: Int, length: Int) { + System.arraycopy(src, srcPos, dst, dstPos, length) + } + + override def allocate(length: Int): Array[EdgeWithLocalIds[ED]] = { + new Array[EdgeWithLocalIds[ED]](length) + } + } } From 8817fc7fe8785d7b11138ca744f22f7e70f1f0a0 Mon Sep 17 00:00:00 2001 From: Takeshi Yamamuro Date: Sun, 7 Dec 2014 19:42:02 -0800 Subject: [PATCH 140/305] [SPARK-4620] Add unpersist in Graph and GraphImpl Add an IF to uncache both vertices and edges of Graph/GraphImpl. This IF is useful when iterative graph operations build a new graph in each iteration, and the vertices and edges of previous iterations are no longer needed for following iterations. Author: Takeshi Yamamuro This patch had conflicts when merged, resolved by Committer: Ankur Dave Closes #3476 from maropu/UnpersistInGraphSpike and squashes the following commits: 77a006a [Takeshi Yamamuro] Add unpersist in Graph and GraphImpl --- graphx/src/main/scala/org/apache/spark/graphx/Graph.scala | 6 ++++++ .../main/scala/org/apache/spark/graphx/impl/GraphImpl.scala | 6 ++++++ 2 files changed, 12 insertions(+) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 23538b71562de..84b72b390ca35 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -104,6 +104,12 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab */ def checkpoint(): Unit + /** + * Uncaches both vertices and edges of this graph. This is useful in iterative algorithms that + * build a new graph in each iteration. + */ + def unpersist(blocking: Boolean = true): Graph[VD, ED] + /** * Uncaches only the vertices of this graph, leaving the edges alone. This is useful in iterative * algorithms that modify the vertex attributes but reuse the edges. This method can be used to diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index a617d84aea9d4..3f4a900d5b601 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -70,6 +70,12 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( replicatedVertexView.edges.checkpoint() } + override def unpersist(blocking: Boolean = true): Graph[VD, ED] = { + unpersistVertices(blocking) + replicatedVertexView.edges.unpersist(blocking) + this + } + override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { vertices.unpersist(blocking) // TODO: unpersist the replicated vertices in `replicatedVertexView` but leave the edges alone From ab2abcb5ef925f15fa0e08d34a79b94a7b6578ef Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Christophe=20Pr=C3=A9aud?= Date: Mon, 8 Dec 2014 11:44:54 -0800 Subject: [PATCH 141/305] [SPARK-4764] Ensure that files are fetched atomically MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit tempFile is created in the same directory than targetFile, so that the move from tempFile to targetFile is always atomic Author: Christophe Préaud Closes #2855 from preaudc/master and squashes the following commits: 9ba89ca [Christophe Préaud] Ensure that files are fetched atomically 54419ae [Christophe Préaud] Merge remote-tracking branch 'upstream/master' c6a5590 [Christophe Préaud] Revert commit 8ea871f8130b2490f1bad7374a819bf56f0ccbbd 7456a33 [Christophe Préaud] Merge remote-tracking branch 'upstream/master' 8ea871f [Christophe Préaud] Ensure that files are fetched atomically --- core/src/main/scala/org/apache/spark/util/Utils.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 336b0798cade9..9c04e45a58479 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -425,8 +425,7 @@ private[spark] object Utils extends Logging { conf: SparkConf, securityMgr: SecurityManager, hadoopConf: Configuration) { - val tempDir = getLocalDir(conf) - val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) + val tempFile = File.createTempFile("fetchFileTemp", null, new File(targetDir.getAbsolutePath)) val targetFile = new File(targetDir, filename) val uri = new URI(url) val fileOverwrite = conf.getBoolean("spark.files.overwrite", defaultValue = false) From d6a972b3e4dc35a2d95df47d256462b325f4bda6 Mon Sep 17 00:00:00 2001 From: Kostas Sakellis Date: Mon, 8 Dec 2014 15:44:18 -0800 Subject: [PATCH 142/305] [SPARK-4774] [SQL] Makes HiveFromSpark more portable HiveFromSpark read the kv1.txt file from SPARK_HOME/examples/src/main/resources/kv1.txt which assumed you had a source tree checked out. Now we copy the kv1.txt file to a temporary file and delete it when the jvm shuts down. This allows us to run this example outside of a spark source tree. Author: Kostas Sakellis Closes #3628 from ksakellis/kostas-spark-4774 and squashes the following commits: 6770f83 [Kostas Sakellis] [SPARK-4774] [SQL] Makes HiveFromSpark more portable --- .../spark/examples/sql/hive/HiveFromSpark.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala index 138923c4d7f2f..5725da1848114 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/HiveFromSpark.scala @@ -17,6 +17,10 @@ package org.apache.spark.examples.sql.hive +import com.google.common.io.{ByteStreams, Files} + +import java.io.File + import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql._ import org.apache.spark.sql.hive.HiveContext @@ -24,10 +28,15 @@ import org.apache.spark.sql.hive.HiveContext object HiveFromSpark { case class Record(key: Int, value: String) + // Copy kv1.txt file from classpath to temporary directory + val kv1Stream = HiveFromSpark.getClass.getResourceAsStream("/kv1.txt") + val kv1File = File.createTempFile("kv1", "txt") + kv1File.deleteOnExit() + ByteStreams.copy(kv1Stream, Files.newOutputStreamSupplier(kv1File)) + def main(args: Array[String]) { val sparkConf = new SparkConf().setAppName("HiveFromSpark") val sc = new SparkContext(sparkConf) - val path = s"${System.getenv("SPARK_HOME")}/examples/src/main/resources/kv1.txt" // A hive context adds support for finding tables in the MetaStore and writing queries // using HiveQL. Users who do not have an existing Hive deployment can still create a @@ -37,7 +46,7 @@ object HiveFromSpark { import hiveContext._ sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") - sql(s"LOAD DATA LOCAL INPATH '$path' INTO TABLE src") + sql(s"LOAD DATA LOCAL INPATH '${kv1File.getAbsolutePath}' INTO TABLE src") // Queries are expressed in HiveQL println("Result of 'SELECT *': ") From 65f929d5b3a50a73cd6397bd4b72c3e7d94c99d7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 8 Dec 2014 16:02:33 -0800 Subject: [PATCH 143/305] [SPARK-4750] Dynamic allocation - synchronize kills Simple omission on my part. Author: Andrew Or Closes #3612 from andrewor14/dynamic-allocation-synchronization and squashes the following commits: 1f03b60 [Andrew Or] Synchronize kills --- .../spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 88b196ac64368..29cd34429b881 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -334,7 +334,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val actorSyste * Request that the cluster manager kill the specified executors. * Return whether the kill request is acknowledged. */ - final def killExecutors(executorIds: Seq[String]): Boolean = { + final def killExecutors(executorIds: Seq[String]): Boolean = synchronized { logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") val filteredExecutorIds = new ArrayBuffer[String] executorIds.foreach { id => From e829bfa1ab9b68f44c489d26efb042f793fd9362 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 8 Dec 2014 16:13:03 -0800 Subject: [PATCH 144/305] SPARK-3926 [CORE] Reopened: result of JavaRDD collectAsMap() is not serializable My original 'fix' didn't fix at all. Now, there's a unit test to check whether it works. Of the two options to really fix it -- copy the `Map` to a `java.util.HashMap`, or copy and modify Scala's implementation in `Wrappers.MapWrapper`, I went with the latter. Author: Sean Owen Closes #3587 from srowen/SPARK-3926 and squashes the following commits: 8586bb9 [Sean Owen] Remove unneeded no-arg constructor, and add additional note about copied code in LICENSE 7bb0e66 [Sean Owen] Make SerializableMapWrapper actually serialize, and add unit test --- LICENSE | 3 +- .../org/apache/spark/api/java/JavaUtils.scala | 62 ++++++++++++++++++- .../java/org/apache/spark/JavaAPISuite.java | 13 ++++ 3 files changed, 75 insertions(+), 3 deletions(-) diff --git a/LICENSE b/LICENSE index 3c667bf45059a..0a42d389e4c3c 100644 --- a/LICENSE +++ b/LICENSE @@ -646,7 +646,8 @@ THE SOFTWARE. ======================================================================== For Scala Interpreter classes (all .scala files in repl/src/main/scala -except for Main.Scala, SparkHelper.scala and ExecutorClassLoader.scala): +except for Main.Scala, SparkHelper.scala and ExecutorClassLoader.scala), +and for SerializableMapWrapper in JavaUtils.scala: ======================================================================== Copyright (c) 2002-2013 EPFL diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index b52d0a5028e84..86e94931300f8 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -19,7 +19,8 @@ package org.apache.spark.api.java import com.google.common.base.Optional -import scala.collection.convert.Wrappers.MapWrapper +import java.{util => ju} +import scala.collection.mutable private[spark] object JavaUtils { def optionToOptional[T](option: Option[T]): Optional[T] = @@ -32,7 +33,64 @@ private[spark] object JavaUtils { def mapAsSerializableJavaMap[A, B](underlying: collection.Map[A, B]) = new SerializableMapWrapper(underlying) + // Implementation is copied from scala.collection.convert.Wrappers.MapWrapper, + // but implements java.io.Serializable. It can't just be subclassed to make it + // Serializable since the MapWrapper class has no no-arg constructor. This class + // doesn't need a no-arg constructor though. class SerializableMapWrapper[A, B](underlying: collection.Map[A, B]) - extends MapWrapper(underlying) with java.io.Serializable + extends ju.AbstractMap[A, B] with java.io.Serializable { self => + override def size = underlying.size + + override def get(key: AnyRef): B = try { + underlying get key.asInstanceOf[A] match { + case None => null.asInstanceOf[B] + case Some(v) => v + } + } catch { + case ex: ClassCastException => null.asInstanceOf[B] + } + + override def entrySet: ju.Set[ju.Map.Entry[A, B]] = new ju.AbstractSet[ju.Map.Entry[A, B]] { + def size = self.size + + def iterator = new ju.Iterator[ju.Map.Entry[A, B]] { + val ui = underlying.iterator + var prev : Option[A] = None + + def hasNext = ui.hasNext + + def next() = { + val (k, v) = ui.next + prev = Some(k) + new ju.Map.Entry[A, B] { + import scala.util.hashing.byteswap32 + def getKey = k + def getValue = v + def setValue(v1 : B) = self.put(k, v1) + override def hashCode = byteswap32(k.hashCode) + (byteswap32(v.hashCode) << 16) + override def equals(other: Any) = other match { + case e: ju.Map.Entry[_, _] => k == e.getKey && v == e.getValue + case _ => false + } + } + } + + def remove() { + prev match { + case Some(k) => + underlying match { + case mm: mutable.Map[a, _] => + mm remove k + prev = None + case _ => + throw new UnsupportedOperationException("remove") + } + case _ => + throw new IllegalStateException("next must be called at least once before remove") + } + } + } + } + } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 3ad4f2f193af4..e5bdad6bda2fa 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -1357,6 +1357,19 @@ public Tuple2 call(Integer x) { pairRDD.collectAsMap(); // Used to crash with ClassCastException } + @SuppressWarnings("unchecked") + @Test + public void collectAsMapAndSerialize() throws Exception { + JavaPairRDD rdd = + sc.parallelizePairs(Arrays.asList(new Tuple2("foo", 1))); + Map map = rdd.collectAsMap(); + ByteArrayOutputStream bytes = new ByteArrayOutputStream(); + new ObjectOutputStream(bytes).writeObject(map); + Map deserializedMap = (Map) + new ObjectInputStream(new ByteArrayInputStream(bytes.toByteArray())).readObject(); + Assert.assertEquals(1, deserializedMap.get("foo").intValue()); + } + @Test @SuppressWarnings("unchecked") public void sampleByKey() { From cda94d15ea2a70ed3f0651ba2766b1e2f80308c1 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 8 Dec 2014 16:28:36 -0800 Subject: [PATCH 145/305] SPARK-4770. [DOC] [YARN] spark.scheduler.minRegisteredResourcesRatio doc... ...umented default is incorrect for YARN Author: Sandy Ryza Closes #3624 from sryza/sandy-spark-4770 and squashes the following commits: bd81a3a [Sandy Ryza] SPARK-4770. [DOC] [YARN] spark.scheduler.minRegisteredResourcesRatio documented default is incorrect for YARN --- docs/configuration.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 4b4bbea564d3a..d50b04642b6a2 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -939,11 +939,11 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.minRegisteredResourcesRatio - 0 + 0.0 for Mesos and Standalone mode, 0.8 for YARN The minimum ratio of registered resources (registered resources / total expected resources) (resources are executors in yarn mode, CPU cores in standalone mode) - to wait for before scheduling begins. Specified as a double between 0 and 1. + to wait for before scheduling begins. Specified as a double between 0.0 and 1.0. Regardless of whether the minimum ratio of resources has been reached, the maximum amount of time it will wait before scheduling begins is controlled by config spark.scheduler.maxRegisteredResourcesWaitingTime. From 944384363d390a133529c08a1d0ac70aa8e778b5 Mon Sep 17 00:00:00 2001 From: Jacky Li Date: Mon, 8 Dec 2014 17:27:46 -0800 Subject: [PATCH 146/305] [SQL] remove unnecessary import in spark-sql Author: Jacky Li Closes #3630 from jackylk/remove and squashes the following commits: 150e7e0 [Jacky Li] remove unnecessary import --- sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 2 +- .../scala/org/apache/spark/sql/execution/ExistingRDD.scala | 4 +--- .../org/apache/spark/sql/execution/GeneratedAggregate.scala | 1 - .../main/scala/org/apache/spark/sql/execution/SparkPlan.scala | 3 --- .../org/apache/spark/sql/types/util/DataTypeConversions.scala | 2 +- 5 files changed, 3 insertions(+), 9 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index c6d4dabf83bc4..95d73c1711523 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql import java.util.{Map => JMap, List => JList} -import java.io.StringWriter + import scala.collection.JavaConversions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index ed6b95dc6d9d0..d2d8cb1c62d40 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -19,13 +19,11 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataType, StructType, Row, SQLContext} +import org.apache.spark.sql.{StructType, Row, SQLContext} import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.ScalaReflection.Schema import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} -import org.apache.spark.sql.catalyst.types.UserDefinedType /** * :: DeveloperApi :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala index 18afc5d74137d..7c3bf947e743d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GeneratedAggregate.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.trees._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ 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 81c60e00505c5..4cd8e7d657250 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 @@ -22,14 +22,11 @@ import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.{ScalaReflection, trees} -import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ - object SparkPlan { protected[sql] val currentContext = new ThreadLocal[SQLContext]() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala index 4160a80621c77..d4ef517981699 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/types/util/DataTypeConversions.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql._ import org.apache.spark.sql.api.java.{DataType => JDataType, StructField => JStructField, - MetadataBuilder => JMetaDataBuilder, UDTWrappers, JavaToScalaUDTWrapper} + MetadataBuilder => JMetaDataBuilder, UDTWrappers} import org.apache.spark.sql.api.java.{DecimalType => JDecimalType} import org.apache.spark.sql.catalyst.types.decimal.Decimal import org.apache.spark.sql.catalyst.ScalaReflection From 51b1fe1426ffecac6c4644523633ea1562ff9a4e Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 8 Dec 2014 17:39:12 -0800 Subject: [PATCH 147/305] [SPARK-4769] [SQL] CTAS does not work when reading from temporary tables This is the code refactor and follow ups for #2570 Author: Cheng Hao Closes #3336 from chenghao-intel/createtbl and squashes the following commits: 3563142 [Cheng Hao] remove the unused variable e215187 [Cheng Hao] eliminate the compiling warning 4f97f14 [Cheng Hao] fix bug in unittest 5d58812 [Cheng Hao] revert the API changes b85b620 [Cheng Hao] fix the regression of temp tabl not found in CTAS --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 26 +++++++++++++++++-- .../spark/sql/hive/HiveStrategies.scala | 14 +++++++--- .../hive/execution/CreateTableAsSelect.scala | 16 ++++-------- .../sql/hive/execution/SQLQuerySuite.scala | 9 +++++++ 4 files changed, 49 insertions(+), 16 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index 91a157785d5bb..60865638e1073 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -254,15 +254,37 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with * For example, because of a CREATE TABLE X AS statement. */ object CreateTables extends Rule[LogicalPlan] { + import org.apache.hadoop.hive.ql.Context + import org.apache.hadoop.hive.ql.parse.{QB, ASTNode, SemanticAnalyzer} + def apply(plan: LogicalPlan): LogicalPlan = plan transform { // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case CreateTableAsSelect(db, tableName, child, allowExisting, extra) => + case CreateTableAsSelect(db, tableName, child, allowExisting, Some(extra: ASTNode)) => val (dbName, tblName) = processDatabaseAndTableName(db, tableName) val databaseName = dbName.getOrElse(hive.sessionState.getCurrentDatabase) - CreateTableAsSelect(Some(databaseName), tableName, child, allowExisting, extra) + // Get the CreateTableDesc from Hive SemanticAnalyzer + val desc: Option[CreateTableDesc] = if (tableExists(Some(databaseName), tblName)) { + None + } else { + val sa = new SemanticAnalyzer(hive.hiveconf) { + override def analyzeInternal(ast: ASTNode) { + // A hack to intercept the SemanticAnalyzer.analyzeInternal, + // to ignore the SELECT clause of the CTAS + val method = classOf[SemanticAnalyzer].getDeclaredMethod( + "analyzeCreateTable", classOf[ASTNode], classOf[QB]) + method.setAccessible(true) + method.invoke(this, ast, this.getQB) + } + } + + sa.analyze(extra, new Context(hive.hiveconf)) + Some(sa.getQB().getTableDesc) + } + + CreateTableAsSelect(Some(databaseName), tblName, child, allowExisting, desc) } } 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 edf291f917f07..5f02e95ac3c34 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import org.apache.hadoop.hive.ql.parse.ASTNode +import org.apache.hadoop.hive.ql.plan.CreateTableDesc import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute @@ -181,13 +182,20 @@ private[hive] trait HiveStrategies { execution.InsertIntoHiveTable( table, partition, planLater(child), overwrite)(hiveContext) :: Nil case logical.CreateTableAsSelect( - Some(database), tableName, child, allowExisting, Some(extra: ASTNode)) => - CreateTableAsSelect( + Some(database), tableName, child, allowExisting, Some(desc: CreateTableDesc)) => + execution.CreateTableAsSelect( database, tableName, child, allowExisting, - extra) :: Nil + Some(desc)) :: Nil + case logical.CreateTableAsSelect(Some(database), tableName, child, allowExisting, None) => + execution.CreateTableAsSelect( + database, + tableName, + child, + allowExisting, + None) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala index 3d24d87bc3d38..b83689ceabb84 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateTableAsSelect.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.hive.execution -import org.apache.hadoop.hive.ql.Context -import org.apache.hadoop.hive.ql.parse.{SemanticAnalyzer, ASTNode} +import org.apache.hadoop.hive.ql.plan.CreateTableDesc + import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row @@ -35,8 +35,7 @@ import org.apache.spark.sql.hive.MetastoreRelation * @param query the query whose result will be insert into the new relation * @param allowExisting allow continue working if it's already exists, otherwise * raise exception - * @param extra the extra information for this Operator, it should be the - * ASTNode object for extracting the CreateTableDesc. + * @param desc the CreateTableDesc, which may contains serde, storage handler etc. */ @Experimental @@ -45,7 +44,7 @@ case class CreateTableAsSelect( tableName: String, query: LogicalPlan, allowExisting: Boolean, - extra: ASTNode) extends LeafNode with Command { + desc: Option[CreateTableDesc]) extends LeafNode with Command { def output = Seq.empty @@ -53,13 +52,8 @@ case class CreateTableAsSelect( // A lazy computing of the metastoreRelation private[this] lazy val metastoreRelation: MetastoreRelation = { - // Get the CreateTableDesc from Hive SemanticAnalyzer - val sa = new SemanticAnalyzer(sc.hiveconf) - - sa.analyze(extra, new Context(sc.hiveconf)) - val desc = sa.getQB().getTableDesc // Create Hive Table - sc.catalog.createTable(database, tableName, query.output, allowExisting, Some(desc)) + sc.catalog.createTable(database, tableName, query.output, allowExisting, desc) // Get the Metastore Relation sc.catalog.lookupRelation(Some(database), tableName, None) match { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala index e9b1943ff8db7..b341eae512417 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala @@ -119,6 +119,15 @@ class SQLQuerySuite extends QueryTest { checkAnswer( sql("SELECT f1.f2.f3 FROM nested"), 1) + checkAnswer(sql("CREATE TABLE test_ctas_1234 AS SELECT * from nested"), + Seq.empty[Row]) + checkAnswer( + sql("SELECT * FROM test_ctas_1234"), + sql("SELECT * FROM nested").collect().toSeq) + + intercept[org.apache.hadoop.hive.ql.metadata.InvalidTableException] { + sql("CREATE TABLE test_ctas_12345 AS SELECT * from notexists").collect() + } } test("test CTAS") { From bcb5cdad614d4fce43725dfec3ce88172d2f8c11 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 8 Dec 2014 23:54:15 -0800 Subject: [PATCH 148/305] [SPARK-3154][STREAMING] Replace ConcurrentHashMap with mutable.HashMap and remove @volatile from 'stopped' Since `sequenceNumberToProcessor` and `stopped` are both protected by the lock `sequenceNumberToProcessor`, `ConcurrentHashMap` and `volatile` is unnecessary. So this PR updated them accordingly. Author: zsxwing Closes #3634 from zsxwing/SPARK-3154 and squashes the following commits: 0d087ac [zsxwing] Replace ConcurrentHashMap with mutable.HashMap and remove @volatile from 'stopped' --- .../flume/sink/SparkAvroCallbackHandler.scala | 23 ++++++++++--------- 1 file changed, 12 insertions(+), 11 deletions(-) diff --git a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala index 3c656a381bd9b..4373be443e67d 100644 --- a/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala +++ b/external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala @@ -16,10 +16,10 @@ */ package org.apache.spark.streaming.flume.sink -import java.util.concurrent.{CountDownLatch, ConcurrentHashMap, Executors} +import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong -import scala.collection.JavaConversions._ +import scala.collection.mutable import org.apache.flume.Channel import org.apache.commons.lang.RandomStringUtils @@ -47,8 +47,8 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha val transactionExecutorOpt = Option(Executors.newFixedThreadPool(threads, new ThreadFactoryBuilder().setDaemon(true) .setNameFormat("Spark Sink Processor Thread - %d").build())) - private val sequenceNumberToProcessor = - new ConcurrentHashMap[CharSequence, TransactionProcessor]() + // Protected by `sequenceNumberToProcessor` + private val sequenceNumberToProcessor = mutable.HashMap[CharSequence, TransactionProcessor]() // This sink will not persist sequence numbers and reuses them if it gets restarted. // So it is possible to commit a transaction which may have been meant for the sink before the // restart. @@ -58,8 +58,8 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha private val seqBase = RandomStringUtils.randomAlphanumeric(8) private val seqCounter = new AtomicLong(0) - - @volatile private var stopped = false + // Protected by `sequenceNumberToProcessor` + private var stopped = false @volatile private var isTest = false private var testLatch: CountDownLatch = null @@ -131,7 +131,7 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha * @param success Whether the batch was successful or not. */ private def completeTransaction(sequenceNumber: CharSequence, success: Boolean) { - Option(removeAndGetProcessor(sequenceNumber)).foreach(processor => { + removeAndGetProcessor(sequenceNumber).foreach(processor => { processor.batchProcessed(success) }) } @@ -139,10 +139,11 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha /** * Helper method to remove the TxnProcessor for a Sequence Number. Can be used to avoid a leak. * @param sequenceNumber - * @return The transaction processor for the corresponding batch. Note that this instance is no - * longer tracked and the caller is responsible for that txn processor. + * @return An `Option` of the transaction processor for the corresponding batch. Note that this + * instance is no longer tracked and the caller is responsible for that txn processor. */ - private[sink] def removeAndGetProcessor(sequenceNumber: CharSequence): TransactionProcessor = { + private[sink] def removeAndGetProcessor(sequenceNumber: CharSequence): + Option[TransactionProcessor] = { sequenceNumberToProcessor.synchronized { sequenceNumberToProcessor.remove(sequenceNumber.toString) } @@ -160,7 +161,7 @@ private[flume] class SparkAvroCallbackHandler(val threads: Int, val channel: Cha logInfo("Shutting down Spark Avro Callback Handler") sequenceNumberToProcessor.synchronized { stopped = true - sequenceNumberToProcessor.values().foreach(_.shutdown()) + sequenceNumberToProcessor.values.foreach(_.shutdown()) } transactionExecutorOpt.foreach(_.shutdownNow()) } From 383c5555c9f26c080bc9e3a463aab21dd5b3797f Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Tue, 9 Dec 2014 10:28:15 -0800 Subject: [PATCH 149/305] [SPARK-4785][SQL] Initilize Hive UDFs on the driver and serialize them with a wrapper Different from Hive 0.12.0, in Hive 0.13.1 UDF/UDAF/UDTF (aka Hive function) objects should only be initialized once on the driver side and then serialized to executors. However, not all function objects are serializable (e.g. GenericUDF doesn't implement Serializable). Hive 0.13.1 solves this issue with Kryo or XML serializer. Several utility ser/de methods are provided in class o.a.h.h.q.e.Utilities for this purpose. In this PR we chose Kryo for efficiency. The Kryo serializer used here is created in Hive. Spark Kryo serializer wasn't used because there's no available SparkConf instance. Author: Cheng Hao Author: Cheng Lian Closes #3640 from chenghao-intel/udf_serde and squashes the following commits: 8e13756 [Cheng Hao] Update the comment 74466a3 [Cheng Hao] refactor as feedbacks 396c0e1 [Cheng Hao] avoid Simple UDF to be serialized e9c3212 [Cheng Hao] update the comment 19cbd46 [Cheng Hao] support udf instance ser/de after initialization --- .../org/apache/spark/sql/hive/HiveQl.scala | 5 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 93 +++++++-------- .../sql/hive/execution/HiveUdfSuite.scala | 7 ++ .../org/apache/spark/sql/hive/Shim12.scala | 11 ++ .../org/apache/spark/sql/hive/Shim13.scala | 107 ++++++++++++++++++ 5 files changed, 173 insertions(+), 50 deletions(-) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala index f4c42bbc5b03d..cd4e5a239ec66 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -1128,7 +1128,10 @@ private[hive] object HiveQl { Explode(attributes, nodeToExpr(child)) case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) => - HiveGenericUdtf(functionName, attributes, children.map(nodeToExpr)) + HiveGenericUdtf( + new HiveFunctionWrapper(functionName), + attributes, + children.map(nodeToExpr)) case a: ASTNode => throw new NotImplementedError( 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 fecf8faaf4cda..ed2e96df8ad77 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 @@ -54,46 +54,31 @@ private[hive] abstract class HiveFunctionRegistry val functionClassName = functionInfo.getFunctionClass.getName if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveSimpleUdf(functionClassName, children) + HiveSimpleUdf(new HiveFunctionWrapper(functionClassName), children) } else if (classOf[GenericUDF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdf(functionClassName, children) + HiveGenericUdf(new HiveFunctionWrapper(functionClassName), children) } else if ( classOf[AbstractGenericUDAFResolver].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdaf(functionClassName, children) + HiveGenericUdaf(new HiveFunctionWrapper(functionClassName), children) } else if (classOf[UDAF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveUdaf(functionClassName, children) + HiveUdaf(new HiveFunctionWrapper(functionClassName), children) } else if (classOf[GenericUDTF].isAssignableFrom(functionInfo.getFunctionClass)) { - HiveGenericUdtf(functionClassName, Nil, children) + HiveGenericUdtf(new HiveFunctionWrapper(functionClassName), Nil, children) } else { sys.error(s"No handler for udf ${functionInfo.getFunctionClass}") } } } -private[hive] trait HiveFunctionFactory { - val functionClassName: String - - def createFunction[UDFType]() = - getContextOrSparkClassLoader.loadClass(functionClassName).newInstance.asInstanceOf[UDFType] -} - -private[hive] abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { - self: Product => - - type UDFType +private[hive] case class HiveSimpleUdf(funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) + extends Expression with HiveInspectors with Logging { type EvaluatedType = Any + type UDFType = UDF def nullable = true - lazy val function = createFunction[UDFType]() - - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" -} - -private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[Expression]) - extends HiveUdf with HiveInspectors { - - type UDFType = UDF + @transient + lazy val function = funcWrapper.createFunction[UDFType]() @transient protected lazy val method = @@ -131,6 +116,8 @@ private[hive] case class HiveSimpleUdf(functionClassName: String, children: Seq[ .convertIfNecessary(wrap(children.map(c => c.eval(input)), arguments, cached): _*): _*), returnInspector) } + + override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } // Adapter from Catalyst ExpressionResult to Hive DeferredObject @@ -144,16 +131,23 @@ private[hive] class DeferredObjectAdapter(oi: ObjectInspector) override def get(): AnyRef = wrap(func(), oi) } -private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq[Expression]) - extends HiveUdf with HiveInspectors { +private[hive] case class HiveGenericUdf(funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) + extends Expression with HiveInspectors with Logging { type UDFType = GenericUDF + type EvaluatedType = Any + + def nullable = true + + @transient + lazy val function = funcWrapper.createFunction[UDFType]() @transient protected lazy val argumentInspectors = children.map(toInspector) @transient - protected lazy val returnInspector = + protected lazy val returnInspector = { function.initializeAndFoldConstants(argumentInspectors.toArray) + } @transient protected lazy val isUDFDeterministic = { @@ -183,18 +177,19 @@ private[hive] case class HiveGenericUdf(functionClassName: String, children: Seq } unwrap(function.evaluate(deferedObjects), returnInspector) } + + override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } private[hive] case class HiveGenericUdaf( - functionClassName: String, + funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) extends AggregateExpression - with HiveInspectors - with HiveFunctionFactory { + with HiveInspectors { type UDFType = AbstractGenericUDAFResolver @transient - protected lazy val resolver: AbstractGenericUDAFResolver = createFunction() + protected lazy val resolver: AbstractGenericUDAFResolver = funcWrapper.createFunction() @transient protected lazy val objectInspector = { @@ -209,22 +204,22 @@ private[hive] case class HiveGenericUdaf( def nullable: Boolean = true - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" + override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" - def newInstance() = new HiveUdafFunction(functionClassName, children, this) + def newInstance() = new HiveUdafFunction(funcWrapper, children, this) } /** It is used as a wrapper for the hive functions which uses UDAF interface */ private[hive] case class HiveUdaf( - functionClassName: String, + funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) extends AggregateExpression - with HiveInspectors - with HiveFunctionFactory { + with HiveInspectors { type UDFType = UDAF @transient - protected lazy val resolver: AbstractGenericUDAFResolver = new GenericUDAFBridge(createFunction()) + protected lazy val resolver: AbstractGenericUDAFResolver = + new GenericUDAFBridge(funcWrapper.createFunction()) @transient protected lazy val objectInspector = { @@ -239,10 +234,10 @@ private[hive] case class HiveUdaf( def nullable: Boolean = true - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" + override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" def newInstance() = - new HiveUdafFunction(functionClassName, children, this, true) + new HiveUdafFunction(funcWrapper, children, this, true) } /** @@ -257,13 +252,13 @@ private[hive] case class HiveUdaf( * user defined aggregations, which have clean semantics even in a partitioned execution. */ private[hive] case class HiveGenericUdtf( - functionClassName: String, + funcWrapper: HiveFunctionWrapper, aliasNames: Seq[String], children: Seq[Expression]) - extends Generator with HiveInspectors with HiveFunctionFactory { + extends Generator with HiveInspectors { @transient - protected lazy val function: GenericUDTF = createFunction() + protected lazy val function: GenericUDTF = funcWrapper.createFunction() @transient protected lazy val inputInspectors = children.map(_.dataType).map(toInspector) @@ -320,25 +315,24 @@ private[hive] case class HiveGenericUdtf( } } - override def toString = s"$nodeName#$functionClassName(${children.mkString(",")})" + override def toString = s"$nodeName#${funcWrapper.functionClassName}(${children.mkString(",")})" } private[hive] case class HiveUdafFunction( - functionClassName: String, + funcWrapper: HiveFunctionWrapper, exprs: Seq[Expression], base: AggregateExpression, isUDAFBridgeRequired: Boolean = false) extends AggregateFunction - with HiveInspectors - with HiveFunctionFactory { + with HiveInspectors { def this() = this(null, null, null) private val resolver = if (isUDAFBridgeRequired) { - new GenericUDAFBridge(createFunction[UDAF]()) + new GenericUDAFBridge(funcWrapper.createFunction[UDAF]()) } else { - createFunction[AbstractGenericUDAFResolver]() + funcWrapper.createFunction[AbstractGenericUDAFResolver]() } private val inspectors = exprs.map(_.dataType).map(toInspector).toArray @@ -361,3 +355,4 @@ private[hive] case class HiveUdafFunction( function.iterate(buffer, inputs) } } + diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala index 872f28d514efe..5fcaf671a80de 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUdfSuite.scala @@ -60,6 +60,13 @@ class HiveUdfSuite extends QueryTest { | getStruct(1).f5 FROM src LIMIT 1 """.stripMargin).first() === Row(1, 2, 3, 4, 5)) } + + test("SPARK-4785 When called with arguments referring column fields, PMOD throws NPE") { + checkAnswer( + sql("SELECT PMOD(CAST(key as INT), 10) FROM src LIMIT 1"), + 8 + ) + } test("hive struct udf") { sql( diff --git a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala index 76f09cbcdec99..754ffc422072d 100644 --- a/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala +++ b/sql/hive/v0.12.0/src/main/scala/org/apache/spark/sql/hive/Shim12.scala @@ -43,6 +43,17 @@ import scala.language.implicitConversions import org.apache.spark.sql.catalyst.types.DecimalType +class HiveFunctionWrapper(var functionClassName: String) extends java.io.Serializable { + // for Serialization + def this() = this(null) + + import org.apache.spark.util.Utils._ + def createFunction[UDFType <: AnyRef](): UDFType = { + getContextOrSparkClassLoader + .loadClass(functionClassName).newInstance.asInstanceOf[UDFType] + } +} + /** * A compatibility layer for interacting with Hive version 0.12.0. */ diff --git a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala index 91f7ceac21177..7c8cbf10c1c30 100644 --- a/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala +++ b/sql/hive/v0.13.1/src/main/scala/org/apache/spark/sql/hive/Shim13.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.hive import java.util.{ArrayList => JArrayList} import java.util.Properties + import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapred.InputFormat @@ -42,6 +43,112 @@ import org.apache.spark.sql.catalyst.types.decimal.Decimal import scala.collection.JavaConversions._ import scala.language.implicitConversions + +/** + * This class provides the UDF creation and also the UDF instance serialization and + * de-serialization cross process boundary. + * + * Detail discussion can be found at https://github.com/apache/spark/pull/3640 + * + * @param functionClassName UDF class name + */ +class HiveFunctionWrapper(var functionClassName: String) extends java.io.Externalizable { + // for Serialization + def this() = this(null) + + import java.io.{OutputStream, InputStream} + import com.esotericsoftware.kryo.Kryo + import org.apache.spark.util.Utils._ + import org.apache.hadoop.hive.ql.exec.Utilities + import org.apache.hadoop.hive.ql.exec.UDF + + @transient + private val methodDeSerialize = { + val method = classOf[Utilities].getDeclaredMethod( + "deserializeObjectByKryo", + classOf[Kryo], + classOf[InputStream], + classOf[Class[_]]) + method.setAccessible(true) + + method + } + + @transient + private val methodSerialize = { + val method = classOf[Utilities].getDeclaredMethod( + "serializeObjectByKryo", + classOf[Kryo], + classOf[Object], + classOf[OutputStream]) + method.setAccessible(true) + + method + } + + def deserializePlan[UDFType](is: java.io.InputStream, clazz: Class[_]): UDFType = { + methodDeSerialize.invoke(null, Utilities.runtimeSerializationKryo.get(), is, clazz) + .asInstanceOf[UDFType] + } + + def serializePlan(function: AnyRef, out: java.io.OutputStream): Unit = { + methodSerialize.invoke(null, Utilities.runtimeSerializationKryo.get(), function, out) + } + + private var instance: AnyRef = null + + def writeExternal(out: java.io.ObjectOutput) { + // output the function name + out.writeUTF(functionClassName) + + // Write a flag if instance is null or not + out.writeBoolean(instance != null) + if (instance != null) { + // Some of the UDF are serializable, but some others are not + // Hive Utilities can handle both cases + val baos = new java.io.ByteArrayOutputStream() + serializePlan(instance, baos) + val functionInBytes = baos.toByteArray + + // output the function bytes + out.writeInt(functionInBytes.length) + out.write(functionInBytes, 0, functionInBytes.length) + } + } + + def readExternal(in: java.io.ObjectInput) { + // read the function name + functionClassName = in.readUTF() + + if (in.readBoolean()) { + // if the instance is not null + // read the function in bytes + val functionInBytesLength = in.readInt() + val functionInBytes = new Array[Byte](functionInBytesLength) + in.read(functionInBytes, 0, functionInBytesLength) + + // deserialize the function object via Hive Utilities + instance = deserializePlan[AnyRef](new java.io.ByteArrayInputStream(functionInBytes), + getContextOrSparkClassLoader.loadClass(functionClassName)) + } + } + + def createFunction[UDFType <: AnyRef](): UDFType = { + if (instance != null) { + instance.asInstanceOf[UDFType] + } else { + val func = getContextOrSparkClassLoader + .loadClass(functionClassName).newInstance.asInstanceOf[UDFType] + if (!func.isInstanceOf[UDF]) { + // We cache the function if it's no the Simple UDF, + // as we always have to create new instance for Simple UDF + instance = func + } + func + } + } +} + /** * A compatibility layer for interacting with Hive version 0.13.1. */ From 912563aa3553afc0871d5b5858f533aa39cb99e5 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 9 Dec 2014 11:02:43 -0800 Subject: [PATCH 150/305] SPARK-4338. [YARN] Ditch yarn-alpha. Sorry if this is a little premature with 1.2 still not out the door, but it will make other work like SPARK-4136 and SPARK-2089 a lot easier. Author: Sandy Ryza Closes #3215 from sryza/sandy-spark-4338 and squashes the following commits: 1c5ac08 [Sandy Ryza] Update building Spark docs and remove unnecessary newline 9c1421c [Sandy Ryza] SPARK-4338. Ditch yarn-alpha. --- assembly/pom.xml | 10 - dev/scalastyle | 5 +- docs/building-spark.md | 25 +- docs/running-on-yarn.md | 2 +- pom.xml | 7 - project/SparkBuild.scala | 20 +- yarn/README.md | 12 - yarn/alpha/pom.xml | 35 --- .../org/apache/spark/deploy/yarn/Client.scala | 145 ----------- .../spark/deploy/yarn/ExecutorRunnable.scala | 139 ----------- .../deploy/yarn/YarnAllocationHandler.scala | 229 ------------------ .../spark/deploy/yarn/YarnRMClientImpl.scala | 118 --------- yarn/pom.xml | 129 +++++----- .../spark/deploy/yarn/ApplicationMaster.scala | 0 .../yarn/ApplicationMasterArguments.scala | 0 .../org/apache/spark/deploy/yarn/Client.scala | 0 .../spark/deploy/yarn/ClientArguments.scala | 36 +-- .../apache/spark/deploy/yarn/ClientBase.scala | 0 .../yarn/ClientDistributedCacheManager.scala | 0 .../spark/deploy/yarn/ExecutorRunnable.scala | 0 .../deploy/yarn/ExecutorRunnableUtil.scala | 7 +- .../deploy/yarn/YarnAllocationHandler.scala | 0 .../spark/deploy/yarn/YarnAllocator.scala | 0 .../spark/deploy/yarn/YarnRMClient.scala | 0 .../spark/deploy/yarn/YarnRMClientImpl.scala | 0 .../deploy/yarn/YarnSparkHadoopUtil.scala | 10 +- .../cluster/YarnClientClusterScheduler.scala | 0 .../cluster/YarnClientSchedulerBackend.scala | 0 .../cluster/YarnClusterScheduler.scala | 0 .../cluster/YarnClusterSchedulerBackend.scala | 0 .../src/test/resources/log4j.properties | 0 .../spark/deploy/yarn/ClientBaseSuite.scala | 0 .../ClientDistributedCacheManagerSuite.scala | 0 .../deploy/yarn/YarnAllocatorSuite.scala | 0 .../spark/deploy/yarn/YarnClusterSuite.scala | 0 .../yarn/YarnSparkHadoopUtilSuite.scala | 0 yarn/stable/pom.xml | 95 -------- 37 files changed, 96 insertions(+), 928 deletions(-) delete mode 100644 yarn/README.md delete mode 100644 yarn/alpha/pom.xml delete mode 100644 yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala delete mode 100644 yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala delete mode 100644 yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala delete mode 100644 yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala (100%) rename yarn/{stable => }/src/main/scala/org/apache/spark/deploy/yarn/Client.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala (82%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala (100%) rename yarn/{stable => }/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala (97%) rename yarn/{stable => }/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala (100%) rename yarn/{stable => }/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala (96%) rename yarn/{common => }/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala (100%) rename yarn/{common => }/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala (100%) rename yarn/{stable => }/src/test/resources/log4j.properties (100%) rename yarn/{common => }/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala (100%) rename yarn/{common => }/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala (100%) rename yarn/{common => }/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala (100%) rename yarn/{stable => }/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala (100%) rename yarn/{common => }/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala (100%) delete mode 100644 yarn/stable/pom.xml diff --git a/assembly/pom.xml b/assembly/pom.xml index 4e2b773e7d2f3..78fb908f9a9ef 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -169,16 +169,6 @@ - - yarn-alpha - - - org.apache.spark - spark-yarn-alpha_${scala.binary.version} - ${project.version} - - - yarn diff --git a/dev/scalastyle b/dev/scalastyle index c3c6012e74ffa..3a4df6e4bf1bc 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -18,11 +18,8 @@ # echo -e "q\n" | sbt/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt -# Check style with YARN alpha built too -echo -e "q\n" | sbt/sbt -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.9 yarn-alpha/scalastyle \ - >> scalastyle.txt # Check style with YARN built too -echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 yarn/scalastyle \ +echo -e "q\n" | sbt/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ >> scalastyle.txt ERRORS=$(cat scalastyle.txt | awk '{if($1~/error/)print}') diff --git a/docs/building-spark.md b/docs/building-spark.md index 6cca2da8e86d2..4922e877e9b78 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -60,32 +60,11 @@ mvn -Dhadoop.version=2.0.0-mr1-cdh4.2.0 -DskipTests clean package mvn -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package {% endhighlight %} -For Apache Hadoop 2.x, 0.23.x, Cloudera CDH, and other Hadoop versions with YARN, you can enable the "yarn-alpha" or "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". The additional build profile required depends on the YARN version: - - - - - - - - - -
YARN versionProfile required
0.23.x to 2.1.xyarn-alpha (Deprecated.)
2.2.x and lateryarn
- -Note: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445). +For Apache Hadoop 2.x, 0.23.x, Cloudera CDH, and other Hadoop versions with YARN, you can enable the "yarn" profile and optionally set the "yarn.version" property if it is different from "hadoop.version". As of Spark 1.3, Spark only supports YARN versions 2.2.0 and later. Examples: {% highlight bash %} -# Apache Hadoop 2.0.5-alpha -mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -DskipTests clean package - -# Cloudera CDH 4.2.0 -mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -DskipTests clean package - -# Apache Hadoop 0.23.x -mvn -Pyarn-alpha -Phadoop-0.23 -Dhadoop.version=0.23.7 -DskipTests clean package - # Apache Hadoop 2.2.X mvn -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 -DskipTests clean package @@ -99,7 +78,7 @@ Versions of Hadoop after 2.5.X may or may not work with the -Phadoop-2.4 profile released after this version of Spark). # Different versions of HDFS and YARN. -mvn -Pyarn-alpha -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=0.23.7 -DskipTests clean package +mvn -Pyarn -Phadoop-2.3 -Dhadoop.version=2.3.0 -Dyarn.version=2.2.0 -DskipTests clean package {% endhighlight %} # Building With Hive and JDBC Support diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 16897dbb65311..62b317129b72a 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -216,7 +216,7 @@ If you need a reference to the proper location to put log files in the YARN so t # Important notes -- Before Hadoop 2.2, YARN does not support cores in container resource requests. Thus, when running against an earlier version, the numbers of cores given via command line arguments cannot be passed to YARN. Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. +- Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. - The local directories used by Spark executors will be the local directories configured for YARN (Hadoop YARN config `yarn.nodemanager.local-dirs`). If the user specifies `spark.local.dir`, it will be ignored. - The `--files` and `--archives` options support specifying file names with the # similar to Hadoop. For example you can specify: `--files localtest.txt#appSees.txt` and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name `appSees.txt`, and your application should use the name as `appSees.txt` to reference it when running on YARN. - The `--jars` option allows the `SparkContext.addJar` function to work if you are using it with local files and running in `yarn-cluster` mode. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. diff --git a/pom.xml b/pom.xml index b7df53d3e5eb1..f42257265ede3 100644 --- a/pom.xml +++ b/pom.xml @@ -1293,13 +1293,6 @@
- - yarn-alpha - - yarn - - - yarn diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6ff08723772aa..39ac27f820d89 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -38,9 +38,9 @@ object BuildCommons { "streaming-flume", "streaming-kafka", "streaming-mqtt", "streaming-twitter", "streaming-zeromq").map(ProjectRef(buildLocation, _)) - val optionallyEnabledProjects@Seq(yarn, yarnStable, yarnAlpha, java8Tests, - sparkGangliaLgpl, sparkKinesisAsl) = Seq("yarn", "yarn-stable", "yarn-alpha", - "java8-tests", "ganglia-lgpl", "kinesis-asl").map(ProjectRef(buildLocation, _)) + val optionallyEnabledProjects@Seq(yarn, yarnStable, java8Tests, sparkGangliaLgpl, + sparkKinesisAsl) = Seq("yarn", "yarn-stable", "java8-tests", "ganglia-lgpl", + "kinesis-asl").map(ProjectRef(buildLocation, _)) val assemblyProjects@Seq(assembly, examples, networkYarn) = Seq("assembly", "examples", "network-yarn").map(ProjectRef(buildLocation, _)) @@ -79,14 +79,8 @@ object SparkBuild extends PomBuild { case None => } if (Properties.envOrNone("SPARK_YARN").isDefined) { - if(isAlphaYarn) { - println("NOTE: SPARK_YARN is deprecated, please use -Pyarn-alpha flag.") - profiles ++= Seq("yarn-alpha") - } - else { - println("NOTE: SPARK_YARN is deprecated, please use -Pyarn flag.") - profiles ++= Seq("yarn") - } + println("NOTE: SPARK_YARN is deprecated, please use -Pyarn flag.") + profiles ++= Seq("yarn") } profiles } @@ -335,9 +329,9 @@ object Unidoc { publish := {}, unidocProjectFilter in(ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, catalyst, streamingFlumeSink, yarn, yarnAlpha), + inAnyProject -- inProjects(OldDeps.project, repl, examples, tools, catalyst, streamingFlumeSink, yarn), unidocProjectFilter in(JavaUnidoc, unidoc) := - inAnyProject -- inProjects(OldDeps.project, repl, bagel, examples, tools, catalyst, streamingFlumeSink, yarn, yarnAlpha), + inAnyProject -- inProjects(OldDeps.project, repl, bagel, examples, tools, catalyst, streamingFlumeSink, yarn), // Skip class names containing $ and some internal packages in Javadocs unidocAllSources in (JavaUnidoc, unidoc) := { diff --git a/yarn/README.md b/yarn/README.md deleted file mode 100644 index 65ee85447e04a..0000000000000 --- a/yarn/README.md +++ /dev/null @@ -1,12 +0,0 @@ -# YARN DIRECTORY LAYOUT - -Hadoop Yarn related codes are organized in separate directories to minimize duplicated code. - - * common : Common codes that do not depending on specific version of Hadoop. - - * alpha / stable : Codes that involve specific version of Hadoop YARN API. - - alpha represents 0.23 and 2.0.x - stable represents 2.2 and later, until the API changes again. - -alpha / stable will build together with common dir into a single jar diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml deleted file mode 100644 index 40e9e99c6f855..0000000000000 --- a/yarn/alpha/pom.xml +++ /dev/null @@ -1,35 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - yarn-parent_2.10 - 1.3.0-SNAPSHOT - ../pom.xml - - - yarn-alpha - - - org.apache.spark - spark-yarn-alpha_2.10 - jar - Spark Project YARN Alpha API - - diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala deleted file mode 100644 index 73b705ba50051..0000000000000 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.nio.ByteBuffer - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.security.UserGroupInformation -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.client.YarnClientImpl -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.util.Records - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.deploy.SparkHadoopUtil - -/** - * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. - */ -@deprecated("use yarn/stable", "1.2.0") -private[spark] class Client( - val args: ClientArguments, - val hadoopConf: Configuration, - val sparkConf: SparkConf) - extends YarnClientImpl with ClientBase with Logging { - - def this(clientArgs: ClientArguments, spConf: SparkConf) = - this(clientArgs, SparkHadoopUtil.get.newConfiguration(spConf), spConf) - - def this(clientArgs: ClientArguments) = this(clientArgs, new SparkConf()) - - val yarnConf: YarnConfiguration = new YarnConfiguration(hadoopConf) - - /* ------------------------------------------------------------------------------------- * - | The following methods have much in common in the stable and alpha versions of Client, | - | but cannot be implemented in the parent trait due to subtle API differences across | - | hadoop versions. | - * ------------------------------------------------------------------------------------- */ - - /** Submit an application running our ApplicationMaster to the ResourceManager. */ - override def submitApplication(): ApplicationId = { - init(yarnConf) - start() - - logInfo("Requesting a new application from cluster with %d NodeManagers" - .format(getYarnClusterMetrics.getNumNodeManagers)) - - // Get a new application from our RM - val newAppResponse = getNewApplication() - val appId = newAppResponse.getApplicationId() - - // Verify whether the cluster has enough resources for our AM - verifyClusterResources(newAppResponse) - - // Set up the appropriate contexts to launch our AM - val containerContext = createContainerLaunchContext(newAppResponse) - val appContext = createApplicationSubmissionContext(appId, containerContext) - - // Finally, submit and monitor the application - logInfo(s"Submitting application ${appId.getId} to ResourceManager") - submitApplication(appContext) - appId - } - - /** - * Set up a context for launching our ApplicationMaster container. - * In the Yarn alpha API, the memory requirements of this container must be set in - * the ContainerLaunchContext instead of the ApplicationSubmissionContext. - */ - override def createContainerLaunchContext(newAppResponse: GetNewApplicationResponse) - : ContainerLaunchContext = { - val containerContext = super.createContainerLaunchContext(newAppResponse) - val capability = Records.newRecord(classOf[Resource]) - capability.setMemory(args.amMemory + amMemoryOverhead) - containerContext.setResource(capability) - containerContext - } - - /** Set up the context for submitting our ApplicationMaster. */ - def createApplicationSubmissionContext( - appId: ApplicationId, - containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { - val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) - appContext.setApplicationId(appId) - appContext.setApplicationName(args.appName) - appContext.setQueue(args.amQueue) - appContext.setAMContainerSpec(containerContext) - appContext.setUser(UserGroupInformation.getCurrentUser.getShortUserName) - appContext - } - - /** - * Set up security tokens for launching our ApplicationMaster container. - * ContainerLaunchContext#setContainerTokens is renamed `setTokens` in the stable API. - */ - override def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { - val dob = new DataOutputBuffer() - credentials.writeTokenStorageToStream(dob) - amContainer.setContainerTokens(ByteBuffer.wrap(dob.getData())) - } - - /** - * Return the security token used by this client to communicate with the ApplicationMaster. - * If no security is enabled, the token returned by the report is null. - * ApplicationReport#getClientToken is renamed `getClientToAMToken` in the stable API. - */ - override def getClientToken(report: ApplicationReport): String = - Option(report.getClientToken).map(_.toString).getOrElse("") -} - -object Client { - def main(argStrings: Array[String]) { - if (!sys.props.contains("SPARK_SUBMIT")) { - println("WARNING: This client is deprecated and will be removed in a " + - "future version of Spark. Use ./bin/spark-submit with \"--master yarn\"") - } - println("WARNING: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445)") - - // Set an env variable indicating we are running in YARN mode. - // Note that any env variable with the SPARK_ prefix gets propagated to all (remote) processes - System.setProperty("SPARK_YARN_MODE", "true") - val sparkConf = new SparkConf - - val args = new ClientArguments(argStrings, sparkConf) - new Client(args, sparkConf).run() - } -} diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala deleted file mode 100644 index 7023a1170654f..0000000000000 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ /dev/null @@ -1,139 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.net.URI -import java.nio.ByteBuffer -import java.security.PrivilegedExceptionAction - -import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.security.UserGroupInformation -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} - -import org.apache.spark.{SecurityManager, SparkConf, Logging} -import org.apache.spark.network.util.JavaUtils - -@deprecated("use yarn/stable", "1.2.0") -class ExecutorRunnable( - container: Container, - conf: Configuration, - spConf: SparkConf, - masterAddress: String, - slaveId: String, - hostname: String, - executorMemory: Int, - executorCores: Int, - appAttemptId: String, - securityMgr: SecurityManager) - extends Runnable with ExecutorRunnableUtil with Logging { - - var rpc: YarnRPC = YarnRPC.create(conf) - var cm: ContainerManager = _ - val sparkConf = spConf - val yarnConf: YarnConfiguration = new YarnConfiguration(conf) - - def run = { - logInfo("Starting Executor Container") - cm = connectToCM - startContainer - } - - def startContainer = { - logInfo("Setting up ContainerLaunchContext") - - val ctx = Records.newRecord(classOf[ContainerLaunchContext]) - .asInstanceOf[ContainerLaunchContext] - - ctx.setContainerId(container.getId()) - ctx.setResource(container.getResource()) - val localResources = prepareLocalResources - ctx.setLocalResources(localResources) - - val env = prepareEnvironment - ctx.setEnvironment(env) - - ctx.setUser(UserGroupInformation.getCurrentUser().getShortUserName()) - - val credentials = UserGroupInformation.getCurrentUser().getCredentials() - val dob = new DataOutputBuffer() - credentials.writeTokenStorageToStream(dob) - ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) - - val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, - appAttemptId, localResources) - logInfo("Setting up executor with commands: " + commands) - ctx.setCommands(commands) - - ctx.setApplicationACLs(YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr)) - - // If external shuffle service is enabled, register with the Yarn shuffle service already - // started on the NodeManager and, if authentication is enabled, provide it with our secret - // key for fetching shuffle files later - if (sparkConf.getBoolean("spark.shuffle.service.enabled", false)) { - val secretString = securityMgr.getSecretKey() - val secretBytes = - if (secretString != null) { - // This conversion must match how the YarnShuffleService decodes our secret - JavaUtils.stringToBytes(secretString) - } else { - // Authentication is not enabled, so just provide dummy metadata - ByteBuffer.allocate(0) - } - ctx.setServiceData(Map[String, ByteBuffer]("spark_shuffle" -> secretBytes)) - } - - // Send the start request to the ContainerManager - val startReq = Records.newRecord(classOf[StartContainerRequest]) - .asInstanceOf[StartContainerRequest] - startReq.setContainerLaunchContext(ctx) - cm.startContainer(startReq) - } - - def connectToCM: ContainerManager = { - val cmHostPortStr = container.getNodeId().getHost() + ":" + container.getNodeId().getPort() - val cmAddress = NetUtils.createSocketAddr(cmHostPortStr) - logInfo("Connecting to ContainerManager at " + cmHostPortStr) - - // Use doAs and remoteUser here so we can add the container token and not pollute the current - // users credentials with all of the individual container tokens - val user = UserGroupInformation.createRemoteUser(container.getId().toString()) - val containerToken = container.getContainerToken() - if (containerToken != null) { - user.addToken(ProtoUtils.convertFromProtoFormat(containerToken, cmAddress)) - } - - val proxy = user - .doAs(new PrivilegedExceptionAction[ContainerManager] { - def run: ContainerManager = { - rpc.getProxy(classOf[ContainerManager], cmAddress, conf).asInstanceOf[ContainerManager] - } - }) - proxy - } - -} diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala deleted file mode 100644 index abd37834ed3cc..0000000000000 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ /dev/null @@ -1,229 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import java.util.concurrent.CopyOnWriteArrayList -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, HashMap} - -import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.scheduler.SplitInfo - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.yarn.api.AMRMProtocol -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords.AllocateRequest -import org.apache.hadoop.yarn.util.Records - -/** - * Acquires resources for executors from a ResourceManager and launches executors in new containers. - */ -private[yarn] class YarnAllocationHandler( - conf: Configuration, - sparkConf: SparkConf, - resourceManager: AMRMProtocol, - appAttemptId: ApplicationAttemptId, - args: ApplicationMasterArguments, - preferredNodes: collection.Map[String, collection.Set[SplitInfo]], - securityMgr: SecurityManager) - extends YarnAllocator(conf, sparkConf, appAttemptId, args, preferredNodes, securityMgr) { - - private val lastResponseId = new AtomicInteger() - private val releaseList: CopyOnWriteArrayList[ContainerId] = new CopyOnWriteArrayList() - - override protected def allocateContainers(count: Int, pending: Int): YarnAllocateResponse = { - var resourceRequests: List[ResourceRequest] = null - - logDebug("asking for additional executors: " + count + " with already pending: " + pending) - val totalNumAsk = count + pending - if (count <= 0) { - resourceRequests = List() - } else if (preferredHostToCount.isEmpty) { - logDebug("host preferences is empty") - resourceRequests = List(createResourceRequest( - AllocationType.ANY, null, totalNumAsk, YarnSparkHadoopUtil.RM_REQUEST_PRIORITY)) - } else { - // request for all hosts in preferred nodes and for numExecutors - - // candidates.size, request by default allocation policy. - val hostContainerRequests: ArrayBuffer[ResourceRequest] = - new ArrayBuffer[ResourceRequest](preferredHostToCount.size) - for ((candidateHost, candidateCount) <- preferredHostToCount) { - val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) - - if (requiredCount > 0) { - hostContainerRequests += createResourceRequest( - AllocationType.HOST, - candidateHost, - requiredCount, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - } - } - val rackContainerRequests: List[ResourceRequest] = createRackResourceRequests( - hostContainerRequests.toList) - - val anyContainerRequests: ResourceRequest = createResourceRequest( - AllocationType.ANY, - resource = null, - totalNumAsk, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - - val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( - hostContainerRequests.size + rackContainerRequests.size + 1) - - containerRequests ++= hostContainerRequests - containerRequests ++= rackContainerRequests - containerRequests += anyContainerRequests - - resourceRequests = containerRequests.toList - } - - val req = Records.newRecord(classOf[AllocateRequest]) - req.setResponseId(lastResponseId.incrementAndGet) - req.setApplicationAttemptId(appAttemptId) - - req.addAllAsks(resourceRequests) - - val releasedContainerList = createReleasedContainerList() - req.addAllReleases(releasedContainerList) - - if (count > 0) { - logInfo("Allocating %d executor containers with %d of memory each.".format(totalNumAsk, - executorMemory + memoryOverhead)) - } else { - logDebug("Empty allocation req .. release : " + releasedContainerList) - } - - for (request <- resourceRequests) { - logInfo("ResourceRequest (host : %s, num containers: %d, priority = %s , capability : %s)". - format( - request.getHostName, - request.getNumContainers, - request.getPriority, - request.getCapability)) - } - new AlphaAllocateResponse(resourceManager.allocate(req).getAMResponse()) - } - - override protected def releaseContainer(container: Container) = { - releaseList.add(container.getId()) - } - - private def createRackResourceRequests(hostContainers: List[ResourceRequest]): - List[ResourceRequest] = { - // First generate modified racks and new set of hosts under it : then issue requests - val rackToCounts = new HashMap[String, Int]() - - // Within this lock - used to read/write to the rack related maps too. - for (container <- hostContainers) { - val candidateHost = container.getHostName - val candidateNumContainers = container.getNumContainers - assert(YarnSparkHadoopUtil.ANY_HOST != candidateHost) - - val rack = YarnSparkHadoopUtil.lookupRack(conf, candidateHost) - if (rack != null) { - var count = rackToCounts.getOrElse(rack, 0) - count += candidateNumContainers - rackToCounts.put(rack, count) - } - } - - val requestedContainers: ArrayBuffer[ResourceRequest] = - new ArrayBuffer[ResourceRequest](rackToCounts.size) - for ((rack, count) <- rackToCounts){ - requestedContainers += - createResourceRequest(AllocationType.RACK, rack, count, - YarnSparkHadoopUtil.RM_REQUEST_PRIORITY) - } - - requestedContainers.toList - } - - private def createResourceRequest( - requestType: AllocationType.AllocationType, - resource:String, - numExecutors: Int, - priority: Int): ResourceRequest = { - - // If hostname specified, we need atleast two requests - node local and rack local. - // There must be a third request - which is ANY : that will be specially handled. - requestType match { - case AllocationType.HOST => { - assert(YarnSparkHadoopUtil.ANY_HOST != resource) - val hostname = resource - val nodeLocal = createResourceRequestImpl(hostname, numExecutors, priority) - - // Add to host->rack mapping - YarnSparkHadoopUtil.populateRackInfo(conf, hostname) - - nodeLocal - } - case AllocationType.RACK => { - val rack = resource - createResourceRequestImpl(rack, numExecutors, priority) - } - case AllocationType.ANY => createResourceRequestImpl( - YarnSparkHadoopUtil.ANY_HOST, numExecutors, priority) - case _ => throw new IllegalArgumentException( - "Unexpected/unsupported request type: " + requestType) - } - } - - private def createResourceRequestImpl( - hostname:String, - numExecutors: Int, - priority: Int): ResourceRequest = { - - val rsrcRequest = Records.newRecord(classOf[ResourceRequest]) - val memCapability = Records.newRecord(classOf[Resource]) - // There probably is some overhead here, let's reserve a bit more memory. - memCapability.setMemory(executorMemory + memoryOverhead) - rsrcRequest.setCapability(memCapability) - - val pri = Records.newRecord(classOf[Priority]) - pri.setPriority(priority) - rsrcRequest.setPriority(pri) - - rsrcRequest.setHostName(hostname) - - rsrcRequest.setNumContainers(java.lang.Math.max(numExecutors, 0)) - rsrcRequest - } - - private def createReleasedContainerList(): ArrayBuffer[ContainerId] = { - val retval = new ArrayBuffer[ContainerId](1) - // Iterator on COW list ... - for (container <- releaseList.iterator()){ - retval += container - } - // Remove from the original list. - if (!retval.isEmpty) { - releaseList.removeAll(retval) - logInfo("Releasing " + retval.size + " containers.") - } - retval - } - - private class AlphaAllocateResponse(response: AMResponse) extends YarnAllocateResponse { - override def getAllocatedContainers() = response.getAllocatedContainers() - override def getAvailableResources() = response.getAvailableResources() - override def getCompletedContainersStatuses() = response.getCompletedContainersStatuses() - } - -} diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala deleted file mode 100644 index e342cc82f454e..0000000000000 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn - -import scala.collection.{Map, Set} -import java.net.URI - -import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.yarn.api._ -import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{ConverterUtils, Records} - -import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.scheduler.SplitInfo -import org.apache.spark.util.Utils - -/** - * YarnRMClient implementation for the Yarn alpha API. - */ -private class YarnRMClientImpl(args: ApplicationMasterArguments) extends YarnRMClient with Logging { - - private var rpc: YarnRPC = null - private var resourceManager: AMRMProtocol = _ - private var uiHistoryAddress: String = _ - private var registered: Boolean = false - - override def register( - conf: YarnConfiguration, - sparkConf: SparkConf, - preferredNodeLocations: Map[String, Set[SplitInfo]], - uiAddress: String, - uiHistoryAddress: String, - securityMgr: SecurityManager) = { - this.rpc = YarnRPC.create(conf) - this.uiHistoryAddress = uiHistoryAddress - - synchronized { - resourceManager = registerWithResourceManager(conf) - registerApplicationMaster(uiAddress) - registered = true - } - - new YarnAllocationHandler(conf, sparkConf, resourceManager, getAttemptId(), args, - preferredNodeLocations, securityMgr) - } - - override def getAttemptId() = { - val envs = System.getenv() - val containerIdString = envs.get(ApplicationConstants.AM_CONTAINER_ID_ENV) - val containerId = ConverterUtils.toContainerId(containerIdString) - val appAttemptId = containerId.getApplicationAttemptId() - appAttemptId - } - - override def unregister(status: FinalApplicationStatus, diagnostics: String = "") = synchronized { - if (registered) { - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(getAttemptId()) - finishReq.setFinishApplicationStatus(status) - finishReq.setDiagnostics(diagnostics) - finishReq.setTrackingUrl(uiHistoryAddress) - resourceManager.finishApplicationMaster(finishReq) - } - } - - override def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String) = { - val proxy = YarnConfiguration.getProxyHostAndPort(conf) - val parts = proxy.split(":") - val uriBase = "http://" + proxy + proxyBase - Map("PROXY_HOST" -> parts(0), "PROXY_URI_BASE" -> uriBase) - } - - override def getMaxRegAttempts(conf: YarnConfiguration) = - conf.getInt(YarnConfiguration.RM_AM_MAX_RETRIES, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) - - private def registerWithResourceManager(conf: YarnConfiguration): AMRMProtocol = { - val rmAddress = NetUtils.createSocketAddr(conf.get(YarnConfiguration.RM_SCHEDULER_ADDRESS, - YarnConfiguration.DEFAULT_RM_SCHEDULER_ADDRESS)) - logInfo("Connecting to ResourceManager at " + rmAddress) - rpc.getProxy(classOf[AMRMProtocol], rmAddress, conf).asInstanceOf[AMRMProtocol] - } - - private def registerApplicationMaster(uiAddress: String): RegisterApplicationMasterResponse = { - val appMasterRequest = Records.newRecord(classOf[RegisterApplicationMasterRequest]) - .asInstanceOf[RegisterApplicationMasterRequest] - appMasterRequest.setApplicationAttemptId(getAttemptId()) - // Setting this to master host,port - so that the ApplicationReport at client has some - // sensible info. - // Users can then monitor stderr/stdout on that node if required. - appMasterRequest.setHost(Utils.localHostName()) - appMasterRequest.setRpcPort(0) - // remove the scheme from the url if it exists since Hadoop does not expect scheme - val uri = new URI(uiAddress) - val authority = if (uri.getScheme == null) uiAddress else uri.getAuthority - appMasterRequest.setTrackingUrl(authority) - resourceManager.registerApplicationMaster(appMasterRequest) - } - -} diff --git a/yarn/pom.xml b/yarn/pom.xml index bba73648c7abe..d7579bf9622d6 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -25,9 +25,9 @@ org.apache.spark - yarn-parent_2.10 - pom - Spark Project YARN Parent POM + spark-yarn_2.10 + jar + Spark Project YARN yarn @@ -58,6 +58,12 @@ org.apache.hadoop hadoop-client + + org.apache.hadoop + hadoop-yarn-server-tests + tests + test + org.scalatest scalatest_${scala.binary.version} @@ -70,41 +76,54 @@ + - yarn-alpha - - - - maven-antrun-plugin - - - validate - - run - - - - ******************************************************************************************* - ***WARNING***: Support for YARN-alpha API's will be removed in Spark 1.3 (see SPARK-3445).* - ******************************************************************************************* - - - - - - - - - alpha - - - - - yarn - - stable - + hadoop-2.2 + + 1.9 + + + + org.mortbay.jetty + jetty + 6.1.26 + + + org.mortbay.jetty + servlet-api + + + test + + + com.sun.jersey + jersey-core + ${jersey.version} + test + + + com.sun.jersey + jersey-json + ${jersey.version} + test + + + stax + stax-api + + + + + com.sun.jersey + jersey-server + ${jersey.version} + test + + @@ -124,38 +143,6 @@ true - - org.codehaus.mojo - build-helper-maven-plugin - - - add-scala-sources - generate-sources - - add-source - - - - src/main/scala - ../common/src/main/scala - - - - - add-scala-test-sources - generate-test-sources - - add-test-source - - - - src/test/scala - ../common/src/test/scala - - - - - org.scalatest scalatest-maven-plugin @@ -169,12 +156,6 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes - - - - ../common/src/main/resources - - diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala similarity index 100% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala similarity index 82% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 4d859450efc63..c439969510fc9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -178,21 +178,25 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) private def getUsageMessage(unknownParam: List[String] = null): String = { val message = if (unknownParam != null) s"Unknown/unsupported param $unknownParam\n" else "" - message + - "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + - "Options:\n" + - " --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster mode)\n" + - " --class CLASS_NAME Name of your application's main class (required)\n" + - " --arg ARG Argument to be passed to your application's main class.\n" + - " Multiple invocations are possible, each will be passed in order.\n" + - " --num-executors NUM Number of executors to start (Default: 2)\n" + - " --executor-cores NUM Number of cores for the executors (Default: 1).\n" + - " --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb)\n" + - " --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G)\n" + - " --name NAME The name of your application (Default: Spark)\n" + - " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + - " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" + - " --files files Comma separated list of files to be distributed with the job.\n" + - " --archives archives Comma separated list of archives to be distributed with the job." + message + """ + |Usage: org.apache.spark.deploy.yarn.Client [options] + |Options: + | --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster + | mode) + | --class CLASS_NAME Name of your application's main class (required) + | --arg ARG Argument to be passed to your application's main class. + | Multiple invocations are possible, each will be passed in order. + | --num-executors NUM Number of executors to start (Default: 2) + | --executor-cores NUM Number of cores for the executors (Default: 1). + | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb) + | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G) + | --name NAME The name of your application (Default: Spark) + | --queue QUEUE The hadoop queue to use for allocation requests (Default: + | 'default') + | --addJars jars Comma separated list of local jars that want SparkContext.addJar + | to work with. + | --files files Comma separated list of files to be distributed with the job. + | --archives archives Comma separated list of archives to be distributed with the job. + """ } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala similarity index 100% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala similarity index 97% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 88dad0febd03f..22d73ecf6d010 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -93,12 +93,13 @@ trait ExecutorRunnableUtil extends Logging { /* else { // If no java_opts specified, default to using -XX:+CMSIncrementalMode - // It might be possible that other modes/config is being done in spark.executor.extraJavaOptions, - // so we dont want to mess with it. + // It might be possible that other modes/config is being done in + // spark.executor.extraJavaOptions, so we dont want to mess with it. // In our expts, using (default) throughput collector has severe perf ramnifications in // multi-tennent machines // The options are based on - // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use%20the%20Concurrent%20Low%20Pause%20Collector|outline + // http://www.oracle.com/technetwork/java/gc-tuning-5-138395.html#0.0.0.%20When%20to%20Use + // %20the%20Concurrent%20Low%20Pause%20Collector|outline javaOpts += " -XX:+UseConcMarkSweepGC " javaOpts += " -XX:+CMSIncrementalMode " javaOpts += " -XX:+CMSIncrementalPacing " diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala similarity index 100% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala similarity index 100% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClientImpl.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala similarity index 96% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala rename to yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 7d453ecb7983c..d7cf904db1c9e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -48,15 +48,17 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { dest.addCredentials(source.getCredentials()) } - // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true. + // Note that all params which start with SPARK are propagated all the way through, so if in yarn + // mode, this MUST be set to true. override def isYarnMode(): Boolean = { true } - // Return an appropriate (subclass) of Configuration. Creating config can initializes some hadoop subsystems - // Always create a new config, dont reuse yarnConf. + // Return an appropriate (subclass) of Configuration. Creating a config initializes some Hadoop + // subsystems. Always create a new config, dont reuse yarnConf. override def newConfiguration(conf: SparkConf): Configuration = new YarnConfiguration(super.newConfiguration(conf)) - // add any user credentials to the job conf which are necessary for running on a secure Hadoop cluster + // Add any user credentials to the job conf which are necessary for running on a secure Hadoop + // cluster override def addCredentials(conf: JobConf) { val jobCreds = conf.getCredentials() jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala rename to yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala rename to yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala rename to yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala similarity index 100% rename from yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala rename to yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala diff --git a/yarn/stable/src/test/resources/log4j.properties b/yarn/src/test/resources/log4j.properties similarity index 100% rename from yarn/stable/src/test/resources/log4j.properties rename to yarn/src/test/resources/log4j.properties diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala similarity index 100% rename from yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientBaseSuite.scala diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala similarity index 100% rename from yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala similarity index 100% rename from yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala diff --git a/yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala similarity index 100% rename from yarn/stable/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala similarity index 100% rename from yarn/common/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala rename to yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml deleted file mode 100644 index 8b6521ad7f859..0000000000000 --- a/yarn/stable/pom.xml +++ /dev/null @@ -1,95 +0,0 @@ - - - - 4.0.0 - - org.apache.spark - yarn-parent_2.10 - 1.3.0-SNAPSHOT - ../pom.xml - - - yarn-stable - - - org.apache.spark - spark-yarn_2.10 - jar - Spark Project YARN Stable API - - - - org.apache.hadoop - hadoop-yarn-server-tests - tests - test - - - - - - - hadoop-2.2 - - 1.9 - - - - org.mortbay.jetty - jetty - 6.1.26 - - - org.mortbay.jetty - servlet-api - - - test - - - com.sun.jersey - jersey-core - ${jersey.version} - test - - - com.sun.jersey - jersey-json - ${jersey.version} - test - - - stax - stax-api - - - - - com.sun.jersey - jersey-server - ${jersey.version} - test - - - - - - From 61f1a7022767e64ab092aa91c0c5aa1b2fdbef7c Mon Sep 17 00:00:00 2001 From: jbencook Date: Tue, 9 Dec 2014 12:16:19 -0800 Subject: [PATCH 151/305] [SPARK-874] adding a --wait flag This PR adds a --wait flag to the `./sbin/stop-all.sh` script. Author: jbencook Closes #3567 from jbencook/master and squashes the following commits: d05c5bb [jbencook] [SPARK-874] adding a --wait flag --- sbin/stop-all.sh | 17 +++++++++++++++++ 1 file changed, 17 insertions(+) diff --git a/sbin/stop-all.sh b/sbin/stop-all.sh index 298c6a9859795..971d5d49da664 100755 --- a/sbin/stop-all.sh +++ b/sbin/stop-all.sh @@ -30,3 +30,20 @@ sbin="`cd "$sbin"; pwd`" # Stop the slaves, then the master "$sbin"/stop-slaves.sh "$sbin"/stop-master.sh + +if [ "$1" == "--wait" ] +then + printf "Waiting for workers to shut down..." + while true + do + running=`$sbin/slaves.sh ps -ef | grep -v grep | grep deploy.worker.Worker` + if [ -z "$running" ] + then + printf "\nAll workers successfully shut down.\n" + break + else + printf "." + sleep 10 + fi + done +fi From b31074466a83d3d1387fc1e4337dfab9e164fc04 Mon Sep 17 00:00:00 2001 From: maji2014 Date: Tue, 9 Dec 2014 13:13:12 -0800 Subject: [PATCH 152/305] [SPARK-4691][shuffle] Restructure a few lines in shuffle code In HashShuffleReader.scala and HashShuffleWriter.scala, no need to judge "dep.aggregator.isEmpty" again as this is judged by "dep.aggregator.isDefined" In SortShuffleWriter.scala, "dep.aggregator.isEmpty" is better than "!dep.aggregator.isDefined" ? Author: maji2014 Closes #3553 from maji2014/spark-4691 and squashes the following commits: bf7b14d [maji2014] change a elegant way for SortShuffleWriter.scala 10d0cf0 [maji2014] change a elegant way d8f52dc [maji2014] code optimization for judgement --- .../org/apache/spark/shuffle/hash/HashShuffleReader.scala | 4 ++-- .../org/apache/spark/shuffle/hash/HashShuffleWriter.scala | 3 +-- .../org/apache/spark/shuffle/sort/SortShuffleWriter.scala | 4 +--- 3 files changed, 4 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala index 5baf45db45c17..de72148ccc7ac 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleReader.scala @@ -45,9 +45,9 @@ private[spark] class HashShuffleReader[K, C]( } else { new InterruptibleIterator(context, dep.aggregator.get.combineValuesByKey(iter, context)) } - } else if (dep.aggregator.isEmpty && dep.mapSideCombine) { - throw new IllegalStateException("Aggregator is empty for map-side combine") } else { + require(!dep.mapSideCombine, "Map-side combine without Aggregator specified!") + // Convert the Product2s to pairs since this is what downstream RDDs currently expect iter.asInstanceOf[Iterator[Product2[K, C]]].map(pair => (pair._1, pair._2)) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala index 183a30373b28c..755f17d6aa15a 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/hash/HashShuffleWriter.scala @@ -56,9 +56,8 @@ private[spark] class HashShuffleWriter[K, V]( } else { records } - } else if (dep.aggregator.isEmpty && dep.mapSideCombine) { - throw new IllegalStateException("Aggregator is empty for map-side combine") } else { + require(!dep.mapSideCombine, "Map-side combine without Aggregator specified!") records } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index d75f9d7311fad..27496c5a289cb 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -50,9 +50,7 @@ private[spark] class SortShuffleWriter[K, V, C]( /** Write a bunch of records to this task's output */ override def write(records: Iterator[_ <: Product2[K, V]]): Unit = { if (dep.mapSideCombine) { - if (!dep.aggregator.isDefined) { - throw new IllegalStateException("Aggregator is empty for map-side combine") - } + require(dep.aggregator.isDefined, "Map-side combine without Aggregator specified!") sorter = new ExternalSorter[K, V, C]( dep.aggregator, Some(dep.partitioner), dep.keyOrdering, dep.serializer) sorter.insertAll(records) From 1f5110630c1abb13a357b463c805a39772923b82 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 9 Dec 2014 15:10:36 -0800 Subject: [PATCH 153/305] [SPARK-4765] Make GC time always shown in UI. This commit removes the GC time for each task from the set of optional, additional metrics, and instead always shows it for each task. cc pwendell Author: Kay Ousterhout Closes #3622 from kayousterhout/gc_time and squashes the following commits: 15ac242 [Kay Ousterhout] Make TaskDetailsClassNames private[spark] e71d893 [Kay Ousterhout] [SPARK-4765] Make GC time always shown in UI. --- .../resources/org/apache/spark/ui/static/webui.css | 2 +- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 13 +++---------- .../spark/ui/jobs/TaskDetailsClassNames.scala | 3 +-- 3 files changed, 5 insertions(+), 13 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index cdf85bfbf326f..68c52ac09b55a 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -171,6 +171,6 @@ span.additional-metric-title { /* Hide all additional metrics by default. This is done here rather than using JavaScript to * avoid slow page loads for stage pages with large numbers (e.g., thousands) of tasks. */ -.scheduler_delay, .gc_time, .deserialization_time, .serialization_time, .getting_result_time { +.scheduler_delay, .deserialization_time, .serialization_time, .getting_result_time { display: none; } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index bfa54f8492068..09a936c2234c0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -132,13 +132,6 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { Task Deserialization Time
-
  • - - - GC Time - -
  • @@ -168,7 +161,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { ("Executor ID / Host", ""), ("Launch Time", ""), ("Duration", ""), ("Scheduler Delay", TaskDetailsClassNames.SCHEDULER_DELAY), ("Task Deserialization Time", TaskDetailsClassNames.TASK_DESERIALIZATION_TIME), - ("GC Time", TaskDetailsClassNames.GC_TIME), + ("GC Time", ""), ("Result Serialization Time", TaskDetailsClassNames.RESULT_SERIALIZATION_TIME), ("Getting Result Time", TaskDetailsClassNames.GETTING_RESULT_TIME)) ++ {if (hasAccumulators) Seq(("Accumulators", "")) else Nil} ++ @@ -308,7 +301,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { {deserializationQuantiles} - {gcQuantiles}, + {gcQuantiles}, {serializationQuantiles} , @@ -429,7 +422,7 @@ private[ui] class StagePage(parent: StagesTab) extends WebUIPage("stage") { class={TaskDetailsClassNames.TASK_DESERIALIZATION_TIME}> {UIUtils.formatDuration(taskDeserializationTime.toLong)} - + {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} Date: Tue, 9 Dec 2014 15:11:20 -0800 Subject: [PATCH 154/305] [SPARK-4714] BlockManager.dropFromMemory() should check whether block has been removed after synchronizing on BlockInfo instance. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit After synchronizing on the `info` lock in the `removeBlock`/`dropOldBlocks`/`dropFromMemory` methods in BlockManager, the block that `info` represented may have already removed. The three methods have the same logic to get the `info` lock: ``` info = blockInfo.get(id) if (info != null) { info.synchronized { // do something } } ``` So, there is chance that when a thread enters the `info.synchronized` block, `info` has already been removed from the `blockInfo` map by some other thread who entered `info.synchronized` first. The `removeBlock` and `dropOldBlocks` methods are idempotent, so it's safe for them to run on blocks that have already been removed. But in `dropFromMemory` it may be problematic since it may drop block data which already removed into the diskstore, and this calls data store operations that are not designed to handle missing blocks. This patch fixes this issue by adding a check to `dropFromMemory` to test whether blocks have been removed by a racing thread. Author: hushan[胡珊] Closes #3574 from suyanNone/refine-block-concurrency and squashes the following commits: edb989d [hushan[胡珊]] Refine code style and comments position 55fa4ba [hushan[胡珊]] refine code e57e270 [hushan[胡珊]] add check info is already remove or not while having gotten info.syn --- .../main/scala/org/apache/spark/storage/BlockManager.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 308c59eda594d..d7b184f8a10e9 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1014,8 +1014,10 @@ private[spark] class BlockManager( // If we get here, the block write failed. logWarning(s"Block $blockId was marked as failure. Nothing to drop") return None + } else if (blockInfo.get(blockId).isEmpty) { + logWarning(s"Block $blockId was already dropped.") + return None } - var blockIsUpdated = false val level = info.level From 5e4c06f8e54265a4024857f5978ec54c936aeea2 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 9 Dec 2014 16:26:07 -0800 Subject: [PATCH 155/305] SPARK-4567. Make SparkJobInfo and SparkStageInfo serializable Author: Sandy Ryza Closes #3426 from sryza/sandy-spark-4567 and squashes the following commits: cb4b8d2 [Sandy Ryza] SPARK-4567. Make SparkJobInfo and SparkStageInfo serializable --- core/src/main/java/org/apache/spark/SparkJobInfo.java | 4 +++- core/src/main/java/org/apache/spark/SparkStageInfo.java | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/spark/SparkJobInfo.java b/core/src/main/java/org/apache/spark/SparkJobInfo.java index 4e3c983b1170a..e31c4401632a6 100644 --- a/core/src/main/java/org/apache/spark/SparkJobInfo.java +++ b/core/src/main/java/org/apache/spark/SparkJobInfo.java @@ -17,13 +17,15 @@ package org.apache.spark; +import java.io.Serializable; + /** * Exposes information about Spark Jobs. * * This interface is not designed to be implemented outside of Spark. We may add additional methods * which may break binary compatibility with outside implementations. */ -public interface SparkJobInfo { +public interface SparkJobInfo extends Serializable { int jobId(); int[] stageIds(); JobExecutionStatus status(); diff --git a/core/src/main/java/org/apache/spark/SparkStageInfo.java b/core/src/main/java/org/apache/spark/SparkStageInfo.java index fd74321093658..b7d462abd72d6 100644 --- a/core/src/main/java/org/apache/spark/SparkStageInfo.java +++ b/core/src/main/java/org/apache/spark/SparkStageInfo.java @@ -17,13 +17,15 @@ package org.apache.spark; +import java.io.Serializable; + /** * Exposes information about Spark Stages. * * This interface is not designed to be implemented outside of Spark. We may add additional methods * which may break binary compatibility with outside implementations. */ -public interface SparkStageInfo { +public interface SparkStageInfo extends Serializable { int stageId(); int currentAttemptId(); long submissionTime(); From d8f84f26e388055ca7459810e001d05ab60af15b Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 9 Dec 2014 16:38:27 -0800 Subject: [PATCH 156/305] SPARK-4805 [CORE] BlockTransferMessage.toByteArray() trips assertion Allocate enough room for type byte as well as message, to avoid tripping assertion about capacity of the buffer Author: Sean Owen Closes #3650 from srowen/SPARK-4805 and squashes the following commits: 9e1d502 [Sean Owen] Allocate enough room for type byte as well as message, to avoid tripping assertion about capacity of the buffer --- .../spark/network/shuffle/protocol/BlockTransferMessage.java | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java index b4b13b8a6ef5d..6c1210b33268a 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java @@ -67,7 +67,8 @@ public static BlockTransferMessage fromByteArray(byte[] msg) { /** Serializes the 'type' byte followed by the message itself. */ public byte[] toByteArray() { - ByteBuf buf = Unpooled.buffer(encodedLength()); + // Allow room for encoded message, plus the type byte + ByteBuf buf = Unpooled.buffer(encodedLength() + 1); buf.writeByte(type().id); encode(buf); assert buf.writableBytes() == 0 : "Writable bytes remain: " + buf.writableBytes(); From 2b9b72682e587909a84d3ace214c22cec830eeaf Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 9 Dec 2014 17:49:59 -0800 Subject: [PATCH 157/305] [SPARK-4740] Create multiple concurrent connections between two peer nodes in Netty. It's been reported that when the number of disks is large and the number of nodes is small, Netty network throughput is low compared with NIO. We suspect the problem is that only a small number of disks are utilized to serve shuffle files at any given point, due to connection reuse. This patch adds a new config parameter to specify the number of concurrent connections between two peer nodes, default to 2. Author: Reynold Xin Closes #3625 from rxin/SPARK-4740 and squashes the following commits: ad4241a [Reynold Xin] Updated javadoc. f33c72b [Reynold Xin] Code review feedback. 0fefabb [Reynold Xin] Use double check in synchronization. 41dfcb2 [Reynold Xin] Added test case. 9076b4a [Reynold Xin] Fixed two NPEs. 3e1306c [Reynold Xin] Minor style fix. 4f21673 [Reynold Xin] [SPARK-4740] Create multiple concurrent connections between two peer nodes in Netty. --- .../client/TransportClientFactory.java | 124 ++++++++++++------ .../spark/network/util/TransportConf.java | 5 + .../network/TransportClientFactorySuite.java | 97 +++++++++++++- 3 files changed, 180 insertions(+), 46 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java index 9afd5decd5e6b..d26b9b4d6055f 100644 --- a/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java +++ b/network/common/src/main/java/org/apache/spark/network/client/TransportClientFactory.java @@ -22,6 +22,7 @@ import java.net.InetSocketAddress; import java.net.SocketAddress; import java.util.List; +import java.util.Random; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicReference; @@ -42,6 +43,7 @@ import org.apache.spark.network.TransportContext; import org.apache.spark.network.server.TransportChannelHandler; import org.apache.spark.network.util.IOMode; +import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.NettyUtils; import org.apache.spark.network.util.TransportConf; @@ -56,12 +58,31 @@ * TransportClient, all given {@link TransportClientBootstrap}s will be run. */ public class TransportClientFactory implements Closeable { + + /** A simple data structure to track the pool of clients between two peer nodes. */ + private static class ClientPool { + TransportClient[] clients; + Object[] locks; + + public ClientPool(int size) { + clients = new TransportClient[size]; + locks = new Object[size]; + for (int i = 0; i < size; i++) { + locks[i] = new Object(); + } + } + } + private final Logger logger = LoggerFactory.getLogger(TransportClientFactory.class); private final TransportContext context; private final TransportConf conf; private final List clientBootstraps; - private final ConcurrentHashMap connectionPool; + private final ConcurrentHashMap connectionPool; + + /** Random number generator for picking connections between peers. */ + private final Random rand; + private final int numConnectionsPerPeer; private final Class socketChannelClass; private EventLoopGroup workerGroup; @@ -73,7 +94,9 @@ public TransportClientFactory( this.context = Preconditions.checkNotNull(context); this.conf = context.getConf(); this.clientBootstraps = Lists.newArrayList(Preconditions.checkNotNull(clientBootstraps)); - this.connectionPool = new ConcurrentHashMap(); + this.connectionPool = new ConcurrentHashMap(); + this.numConnectionsPerPeer = conf.numConnectionsPerPeer(); + this.rand = new Random(); IOMode ioMode = IOMode.valueOf(conf.ioMode()); this.socketChannelClass = NettyUtils.getClientChannelClass(ioMode); @@ -84,10 +107,14 @@ public TransportClientFactory( } /** - * Create a new {@link TransportClient} connecting to the given remote host / port. This will - * reuse TransportClients if they are still active and are for the same remote address. Prior - * to the creation of a new TransportClient, we will execute all {@link TransportClientBootstrap}s - * that are registered with this factory. + * Create a {@link TransportClient} connecting to the given remote host / port. + * + * We maintains an array of clients (size determined by spark.shuffle.io.numConnectionsPerPeer) + * and randomly picks one to use. If no client was previously created in the randomly selected + * spot, this function creates a new client and places it there. + * + * Prior to the creation of a new TransportClient, we will execute all + * {@link TransportClientBootstrap}s that are registered with this factory. * * This blocks until a connection is successfully established and fully bootstrapped. * @@ -97,23 +124,48 @@ public TransportClient createClient(String remoteHost, int remotePort) throws IO // Get connection from the connection pool first. // If it is not found or not active, create a new one. final InetSocketAddress address = new InetSocketAddress(remoteHost, remotePort); - TransportClient cachedClient = connectionPool.get(address); - if (cachedClient != null) { - if (cachedClient.isActive()) { - logger.trace("Returning cached connection to {}: {}", address, cachedClient); - return cachedClient; - } else { - logger.info("Found inactive connection to {}, closing it.", address); - connectionPool.remove(address, cachedClient); // Remove inactive clients. + + // Create the ClientPool if we don't have it yet. + ClientPool clientPool = connectionPool.get(address); + if (clientPool == null) { + connectionPool.putIfAbsent(address, new ClientPool(numConnectionsPerPeer)); + clientPool = connectionPool.get(address); + } + + int clientIndex = rand.nextInt(numConnectionsPerPeer); + TransportClient cachedClient = clientPool.clients[clientIndex]; + + if (cachedClient != null && cachedClient.isActive()) { + logger.trace("Returning cached connection to {}: {}", address, cachedClient); + return cachedClient; + } + + // If we reach here, we don't have an existing connection open. Let's create a new one. + // Multiple threads might race here to create new connections. Keep only one of them active. + synchronized (clientPool.locks[clientIndex]) { + cachedClient = clientPool.clients[clientIndex]; + + if (cachedClient != null) { + if (cachedClient.isActive()) { + logger.trace("Returning cached connection to {}: {}", address, cachedClient); + return cachedClient; + } else { + logger.info("Found inactive connection to {}, creating a new one.", address); + } } + clientPool.clients[clientIndex] = createClient(address); + return clientPool.clients[clientIndex]; } + } + /** Create a completely new {@link TransportClient} to the remote address. */ + private TransportClient createClient(InetSocketAddress address) throws IOException { logger.debug("Creating new connection to " + address); Bootstrap bootstrap = new Bootstrap(); bootstrap.group(workerGroup) .channel(socketChannelClass) - // Disable Nagle's Algorithm since we don't want packets to wait + // Disable Nagle's Algorithm since we don't want packets to wait .option(ChannelOption.TCP_NODELAY, true) .option(ChannelOption.SO_KEEPALIVE, true) .option(ChannelOption.CONNECT_TIMEOUT_MILLIS, conf.connectionTimeoutMs()) @@ -130,7 +182,7 @@ public void initChannel(SocketChannel ch) { }); // Connect to the remote server - long preConnect = System.currentTimeMillis(); + long preConnect = System.nanoTime(); ChannelFuture cf = bootstrap.connect(address); if (!cf.awaitUninterruptibly(conf.connectionTimeoutMs())) { throw new IOException( @@ -143,43 +195,37 @@ public void initChannel(SocketChannel ch) { assert client != null : "Channel future completed successfully with null client"; // Execute any client bootstraps synchronously before marking the Client as successful. - long preBootstrap = System.currentTimeMillis(); + long preBootstrap = System.nanoTime(); logger.debug("Connection to {} successful, running bootstraps...", address); try { for (TransportClientBootstrap clientBootstrap : clientBootstraps) { clientBootstrap.doBootstrap(client); } } catch (Exception e) { // catch non-RuntimeExceptions too as bootstrap may be written in Scala - long bootstrapTime = System.currentTimeMillis() - preBootstrap; - logger.error("Exception while bootstrapping client after " + bootstrapTime + " ms", e); + long bootstrapTimeMs = (System.nanoTime() - preBootstrap) / 1000000; + logger.error("Exception while bootstrapping client after " + bootstrapTimeMs + " ms", e); client.close(); throw Throwables.propagate(e); } - long postBootstrap = System.currentTimeMillis(); - - // Successful connection & bootstrap -- in the event that two threads raced to create a client, - // use the first one that was put into the connectionPool and close the one we made here. - TransportClient oldClient = connectionPool.putIfAbsent(address, client); - if (oldClient == null) { - logger.debug("Successfully created connection to {} after {} ms ({} ms spent in bootstraps)", - address, postBootstrap - preConnect, postBootstrap - preBootstrap); - return client; - } else { - logger.debug("Two clients were created concurrently after {} ms, second will be disposed.", - postBootstrap - preConnect); - client.close(); - return oldClient; - } + long postBootstrap = System.nanoTime(); + + logger.debug("Successfully created connection to {} after {} ms ({} ms spent in bootstraps)", + address, (postBootstrap - preConnect) / 1000000, (postBootstrap - preBootstrap) / 1000000); + + return client; } /** Close all connections in the connection pool, and shutdown the worker thread pool. */ @Override public void close() { - for (TransportClient client : connectionPool.values()) { - try { - client.close(); - } catch (RuntimeException e) { - logger.warn("Ignoring exception during close", e); + // Go through all clients and close them if they are active. + for (ClientPool clientPool : connectionPool.values()) { + for (int i = 0; i < clientPool.clients.length; i++) { + TransportClient client = clientPool.clients[i]; + if (client != null) { + clientPool.clients[i] = null; + JavaUtils.closeQuietly(client); + } } } connectionPool.clear(); diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index 1af40acf8b4af..f60573998f7ae 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -40,6 +40,11 @@ public int connectionTimeoutMs() { return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000; } + /** Number of concurrent connections between two nodes for fetching data. **/ + public int numConnectionsPerPeer() { + return conf.getInt("spark.shuffle.io.numConnectionsPerPeer", 2); + } + /** Requested maximum length of the queue of incoming connections. Default -1 for no backlog. */ public int backLog() { return conf.getInt("spark.shuffle.io.backLog", -1); } diff --git a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java index 822bef1d81b2a..416dc1b969fa4 100644 --- a/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java +++ b/network/common/src/test/java/org/apache/spark/network/TransportClientFactorySuite.java @@ -18,7 +18,11 @@ package org.apache.spark.network; import java.io.IOException; -import java.util.concurrent.TimeoutException; +import java.util.Collections; +import java.util.HashSet; +import java.util.NoSuchElementException; +import java.util.Set; +import java.util.concurrent.atomic.AtomicInteger; import org.junit.After; import org.junit.Before; @@ -32,6 +36,7 @@ import org.apache.spark.network.server.NoOpRpcHandler; import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.server.TransportServer; +import org.apache.spark.network.util.ConfigProvider; import org.apache.spark.network.util.JavaUtils; import org.apache.spark.network.util.SystemPropertyConfigProvider; import org.apache.spark.network.util.TransportConf; @@ -57,16 +62,94 @@ public void tearDown() { JavaUtils.closeQuietly(server2); } + /** + * Request a bunch of clients to a single server to test + * we create up to maxConnections of clients. + * + * If concurrent is true, create multiple threads to create clients in parallel. + */ + private void testClientReuse(final int maxConnections, boolean concurrent) + throws IOException, InterruptedException { + TransportConf conf = new TransportConf(new ConfigProvider() { + @Override + public String get(String name) { + if (name.equals("spark.shuffle.io.numConnectionsPerPeer")) { + return Integer.toString(maxConnections); + } else { + throw new NoSuchElementException(); + } + } + }); + + RpcHandler rpcHandler = new NoOpRpcHandler(); + TransportContext context = new TransportContext(conf, rpcHandler); + final TransportClientFactory factory = context.createClientFactory(); + final Set clients = Collections.synchronizedSet( + new HashSet()); + + final AtomicInteger failed = new AtomicInteger(); + Thread[] attempts = new Thread[maxConnections * 10]; + + // Launch a bunch of threads to create new clients. + for (int i = 0; i < attempts.length; i++) { + attempts[i] = new Thread() { + @Override + public void run() { + try { + TransportClient client = + factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assert (client.isActive()); + clients.add(client); + } catch (IOException e) { + failed.incrementAndGet(); + } + } + }; + + if (concurrent) { + attempts[i].start(); + } else { + attempts[i].run(); + } + } + + // Wait until all the threads complete. + for (int i = 0; i < attempts.length; i++) { + attempts[i].join(); + } + + assert(failed.get() == 0); + assert(clients.size() == maxConnections); + + for (TransportClient client : clients) { + client.close(); + } + } + + @Test + public void reuseClientsUpToConfigVariable() throws Exception { + testClientReuse(1, false); + testClientReuse(2, false); + testClientReuse(3, false); + testClientReuse(4, false); + } + @Test - public void createAndReuseBlockClients() throws IOException { + public void reuseClientsUpToConfigVariableConcurrent() throws Exception { + testClientReuse(1, true); + testClientReuse(2, true); + testClientReuse(3, true); + testClientReuse(4, true); + } + + @Test + public void returnDifferentClientsForDifferentServers() throws IOException { TransportClientFactory factory = context.createClientFactory(); TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - TransportClient c3 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); + TransportClient c2 = factory.createClient(TestUtils.getLocalHost(), server2.getPort()); assertTrue(c1.isActive()); - assertTrue(c3.isActive()); - assertTrue(c1 == c2); - assertTrue(c1 != c3); + assertTrue(c2.isActive()); + assertTrue(c1 != c2); factory.close(); } From 9bd9334f588dbb44d01554f9f4ca68a153a48993 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Tue, 9 Dec 2014 19:29:09 -0800 Subject: [PATCH 158/305] Config updates for the new shuffle transport. Author: Reynold Xin Closes #3657 from rxin/conf-update and squashes the following commits: 7370eab [Reynold Xin] Config updates for the new shuffle transport. --- .../java/org/apache/spark/network/util/TransportConf.java | 8 ++++---- .../apache/spark/network/sasl/SaslClientBootstrap.java | 2 +- .../spark/network/shuffle/RetryingBlockFetcher.java | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) diff --git a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java index f60573998f7ae..13b37f96f8ce2 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java +++ b/network/common/src/main/java/org/apache/spark/network/util/TransportConf.java @@ -35,14 +35,14 @@ public boolean preferDirectBufs() { return conf.getBoolean("spark.shuffle.io.preferDirectBufs", true); } - /** Connect timeout in secs. Default 120 secs. */ + /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { return conf.getInt("spark.shuffle.io.connectionTimeout", 120) * 1000; } /** Number of concurrent connections between two nodes for fetching data. **/ public int numConnectionsPerPeer() { - return conf.getInt("spark.shuffle.io.numConnectionsPerPeer", 2); + return conf.getInt("spark.shuffle.io.numConnectionsPerPeer", 1); } /** Requested maximum length of the queue of incoming connections. Default -1 for no backlog. */ @@ -67,7 +67,7 @@ public int numConnectionsPerPeer() { public int sendBuf() { return conf.getInt("spark.shuffle.io.sendBuffer", -1); } /** Timeout for a single round trip of SASL token exchange, in milliseconds. */ - public int saslRTTimeout() { return conf.getInt("spark.shuffle.sasl.timeout", 30000); } + public int saslRTTimeoutMs() { return conf.getInt("spark.shuffle.sasl.timeout", 30) * 1000; } /** * Max number of times we will try IO exceptions (such as connection timeouts) per request. @@ -79,7 +79,7 @@ public int numConnectionsPerPeer() { * Time (in milliseconds) that we will wait in order to perform a retry after an IOException. * Only relevant if maxIORetries > 0. */ - public int ioRetryWaitTime() { return conf.getInt("spark.shuffle.io.retryWaitMs", 5000); } + public int ioRetryWaitTimeMs() { return conf.getInt("spark.shuffle.io.retryWait", 5) * 1000; } /** * Minimum size of a block that we should start using memory map rather than reading in through diff --git a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java index 7bc91e375371f..33aa1344345ff 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/sasl/SaslClientBootstrap.java @@ -59,7 +59,7 @@ public void doBootstrap(TransportClient client) { ByteBuf buf = Unpooled.buffer(msg.encodedLength()); msg.encode(buf); - byte[] response = client.sendRpcSync(buf.array(), conf.saslRTTimeout()); + byte[] response = client.sendRpcSync(buf.array(), conf.saslRTTimeoutMs()); payload = saslClient.response(response); } } finally { diff --git a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java index f8a1a266863bb..4bb0498e5d5aa 100644 --- a/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java +++ b/network/shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -106,7 +106,7 @@ public RetryingBlockFetcher( this.fetchStarter = fetchStarter; this.listener = listener; this.maxRetries = conf.maxIORetries(); - this.retryWaitTime = conf.ioRetryWaitTime(); + this.retryWaitTime = conf.ioRetryWaitTimeMs(); this.outstandingBlocksIds = Sets.newLinkedHashSet(); Collections.addAll(outstandingBlocksIds, blockIds); this.currentListener = new RetryingBlockFetchListener(); From f79c1cfc997c1a7ddee480ca3d46f5341b69d3b7 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 9 Dec 2014 23:47:05 -0800 Subject: [PATCH 159/305] [Minor] Use tag for help icon in web UI page header This small commit makes the `(?)` web UI help link into a superscript, which should address feedback that the current design makes it look like an error occurred or like information is missing. Before: ![image](https://cloud.githubusercontent.com/assets/50748/5370611/a3ed0034-7fd9-11e4-870f-05bd9faad5b9.png) After: ![image](https://cloud.githubusercontent.com/assets/50748/5370602/6c5ca8d6-7fd9-11e4-8d1a-568d71290aa7.png) Author: Josh Rosen Closes #3659 from JoshRosen/webui-help-sup and squashes the following commits: bd72899 [Josh Rosen] Use tag for help icon in web UI page header. --- core/src/main/scala/org/apache/spark/ui/UIUtils.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 315327c3c6b7c..d970fa30c1c35 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -181,7 +181,9 @@ private[spark] object UIUtils extends Logging {
  • } val helpButton: Seq[Node] = helpText.map { helpText => -
    (?) + + (?) + }.getOrElse(Seq.empty) From 94b377f94487109a1cc3e07dd230b1df7a96e28d Mon Sep 17 00:00:00 2001 From: Nathan Kronenfeld Date: Tue, 9 Dec 2014 23:53:17 -0800 Subject: [PATCH 160/305] [SPARK-4772] Clear local copies of accumulators as soon as we're done with them Accumulators keep thread-local copies of themselves. These copies were only cleared at the beginning of a task. This meant that (a) the memory they used was tied up until the next task ran on that thread, and (b) if a thread died, the memory it had used for accumulators was locked up forever on that worker. This PR clears the thread-local copies of accumulators at the end of each task, in the tasks finally block, to make sure they are cleaned up between tasks. It also stores them in a ThreadLocal object, so that if, for some reason, the thread dies, any memory they are using at the time should be freed up. Author: Nathan Kronenfeld Closes #3570 from nkronenfeld/Accumulator-Improvements and squashes the following commits: a581f3f [Nathan Kronenfeld] Change Accumulators to private[spark] instead of adding mima exclude to get around false positive in mima tests b6c2180 [Nathan Kronenfeld] Include MiMa exclude as per build error instructions - this version incompatibility should be irrelevent, as it will only surface if a master is talking to a worker running a different version of spark. 537baad [Nathan Kronenfeld] Fuller refactoring as intended, incorporating JR's suggestions for ThreadLocal localAccums, and keeping clear(), but also calling it in tasks' finally block, rather than just at the beginning of the task. 39a82f2 [Nathan Kronenfeld] Clear local copies of accumulators as soon as we're done with them --- .../main/scala/org/apache/spark/Accumulators.scala | 14 ++++++++------ .../scala/org/apache/spark/executor/Executor.scala | 3 ++- 2 files changed, 10 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index 000bbd6b532ad..5f31bfba3f8d6 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.io.{ObjectInputStream, Serializable} import java.util.concurrent.atomic.AtomicLong +import java.lang.ThreadLocal import scala.collection.generic.Growable import scala.collection.mutable.Map @@ -278,10 +279,12 @@ object AccumulatorParam { // TODO: The multi-thread support in accumulators is kind of lame; check // if there's a more intuitive way of doing it right -private object Accumulators { +private[spark] object Accumulators { // TODO: Use soft references? => need to make readObject work properly then val originals = Map[Long, Accumulable[_, _]]() - val localAccums = Map[Thread, Map[Long, Accumulable[_, _]]]() + val localAccums = new ThreadLocal[Map[Long, Accumulable[_, _]]]() { + override protected def initialValue() = Map[Long, Accumulable[_, _]]() + } var lastId: Long = 0 def newId(): Long = synchronized { @@ -293,22 +296,21 @@ private object Accumulators { if (original) { originals(a.id) = a } else { - val accums = localAccums.getOrElseUpdate(Thread.currentThread, Map()) - accums(a.id) = a + localAccums.get()(a.id) = a } } // Clear the local (non-original) accumulators for the current thread def clear() { synchronized { - localAccums.remove(Thread.currentThread) + localAccums.get.clear } } // Get the values of the local accumulators for the current thread (by ID) def values: Map[Long, Any] = synchronized { val ret = Map[Long, Any]() - for ((id, accum) <- localAccums.getOrElse(Thread.currentThread, Map())) { + for ((id, accum) <- localAccums.get) { ret(id) = accum.localValue } return ret diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 835157fc520aa..52de6980ecbf8 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -172,7 +172,6 @@ private[spark] class Executor( val startGCTime = gcTime try { - Accumulators.clear() val (taskFiles, taskJars, taskBytes) = Task.deserializeWithDependencies(serializedTask) updateDependencies(taskFiles, taskJars) task = ser.deserialize[Task[Any]](taskBytes, Thread.currentThread.getContextClassLoader) @@ -278,6 +277,8 @@ private[spark] class Executor( env.shuffleMemoryManager.releaseMemoryForThisThread() // Release memory used by this thread for unrolling blocks env.blockManager.memoryStore.releaseUnrollMemoryForThisThread() + // Release memory used by this thread for accumulators + Accumulators.clear() runningTasks.remove(taskId) } } From 742e7093eca8865225c29bacf4344f2e89bfea41 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Wed, 10 Dec 2014 12:24:04 -0800 Subject: [PATCH 161/305] [SPARK-4161]Spark shell class path is not correctly set if "spark.driver.extraClassPath" is set in defaults.conf Author: GuoQiang Li Closes #3050 from witgo/SPARK-4161 and squashes the following commits: abb6fa4 [GuoQiang Li] move usejavacp opt to spark-shell 89e39e7 [GuoQiang Li] review commit c2a6f04 [GuoQiang Li] Spark shell class path is not correctly set if "spark.driver.extraClassPath" is set in defaults.conf --- bin/spark-shell | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/bin/spark-shell b/bin/spark-shell index 4a0670fc6c8aa..cca5aa0676123 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -45,6 +45,13 @@ source "$FWDIR"/bin/utils.sh SUBMIT_USAGE_FUNCTION=usage gatherSparkSubmitOpts "$@" +# SPARK-4161: scala does not assume use of the java classpath, +# so we need to add the "-Dscala.usejavacp=true" flag mnually. We +# do this specifically for the Spark shell because the scala REPL +# has its own class loader, and any additional classpath specified +# through spark.driver.extraClassPath is not automatically propagated. +SPARK_SUBMIT_OPTS="$SPARK_SUBMIT_OPTS -Dscala.usejavacp=true" + function main() { if $cygwin; then # Workaround for issue involving JLine and Cygwin From 0fc637b4c27f9afdf5c829d26c7a86efd8681490 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Wed, 10 Dec 2014 12:29:00 -0800 Subject: [PATCH 162/305] [SPARK-4329][WebUI] HistoryPage pagenation Current HistoryPage have links only to previous page or next page. I suggest to add index to access history pages easily. I implemented like following pics. If there are many pages, current page +/- N pages, head page and last page are indexed. ![2014-11-10 16 13 25](https://cloud.githubusercontent.com/assets/4736016/4986246/9c7bbac4-6937-11e4-8695-8634d039d5b6.png) ![2014-11-10 16 03 21](https://cloud.githubusercontent.com/assets/4736016/4986210/3951bb74-6937-11e4-8b4e-9f90d266d736.png) ![2014-11-10 16 03 39](https://cloud.githubusercontent.com/assets/4736016/4986211/3b196ad8-6937-11e4-9f81-74bc0a6dad5b.png) ![2014-11-10 16 03 49](https://cloud.githubusercontent.com/assets/4736016/4986213/40686138-6937-11e4-86c0-41100f0404f6.png) ![2014-11-10 16 04 04](https://cloud.githubusercontent.com/assets/4736016/4986215/4326c9b4-6937-11e4-87ac-0f30c86ec6e3.png) Author: Kousuke Saruta Closes #3194 from sarutak/history-page-indexing and squashes the following commits: 15d3d2d [Kousuke Saruta] Simplified code c93932e [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing 1c2f605 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing 76b05e3 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing b2240f8 [Kousuke Saruta] Fixed style ec7922e [Kousuke Saruta] Simplified code 755a004 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into history-page-indexing cfa242b [Kousuke Saruta] Added index to HistoryPage --- .../spark/deploy/history/HistoryPage.scala | 41 +++++++++++++++++-- 1 file changed, 37 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 5fdc350cd8512..0d5dcfb1ddffe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -26,6 +26,7 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { private val pageSize = 20 + private val plusOrMinus = 2 def render(request: HttpServletRequest): Seq[Node] = { val requestedPage = Option(request.getParameter("page")).getOrElse("1").toInt @@ -39,6 +40,9 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { val last = Math.min(actualFirst + pageSize, allApps.size) - 1 val pageCount = allApps.size / pageSize + (if (allApps.size % pageSize > 0) 1 else 0) + val secondPageFromLeft = 2 + val secondPageFromRight = pageCount - 1 + val appTable = UIUtils.listingTable(appHeader, appRow, apps) val providerConfig = parent.getProviderConfig() val content = @@ -48,13 +52,38 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { {providerConfig.map { case (k, v) =>
  • {k}: {v}
  • }} { + // This displays the indices of pages that are within `plusOrMinus` pages of + // the current page. Regardless of where the current page is, this also links + // to the first and last page. If the current page +/- `plusOrMinus` is greater + // than the 2nd page from the first page or less than the 2nd page from the last + // page, `...` will be displayed. if (allApps.size > 0) { + val leftSideIndices = + rangeIndices(actualPage - plusOrMinus until actualPage, 1 < _) + val rightSideIndices = + rangeIndices(actualPage + 1 to actualPage + plusOrMinus, _ < pageCount) +

    Showing {actualFirst + 1}-{last + 1} of {allApps.size} - - {if (actualPage > 1) <} - {if (actualPage < pageCount) >} - + + { + if (actualPage > 1) { + < + 1 + } + } + {if (actualPage - plusOrMinus > secondPageFromLeft) " ... "} + {leftSideIndices} + {actualPage} + {rightSideIndices} + {if (actualPage + plusOrMinus < secondPageFromRight) " ... "} + { + if (actualPage < pageCount) { + {pageCount} + > + } + } +

    ++ appTable } else { @@ -81,6 +110,10 @@ private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { "Spark User", "Last Updated") + private def rangeIndices(range: Seq[Int], condition: Int => Boolean): Seq[Node] = { + range.filter(condition).map(nextPage => {nextPage} ) + } + private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { val uiAddress = HistoryServer.UI_PATH_PREFIX + s"/${info.id}" val startTime = UIUtils.formatDate(info.startTime) From 56212831c6436e287a19908e82c26117cbcb16b0 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Dec 2014 12:41:36 -0800 Subject: [PATCH 163/305] [SPARK-4771][Docs] Document standalone cluster supervise mode tdas looks like streaming already refers to the supervise mode. The link from there is broken though. Author: Andrew Or Closes #3627 from andrewor14/document-supervise and squashes the following commits: 9ca0908 [Andrew Or] Wording changes 2b55ed2 [Andrew Or] Document standalone cluster supervise mode --- docs/spark-standalone.md | 11 ++++++++++- 1 file changed, 10 insertions(+), 1 deletion(-) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index ae7b81d5bb71f..5c6084fb46255 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -257,7 +257,7 @@ To run an interactive Spark shell against the cluster, run the following command You can also pass an option `--total-executor-cores ` to control the number of cores that spark-shell uses on the cluster. -# Launching Compiled Spark Applications +# Launching Spark Applications The [`spark-submit` script](submitting-applications.html) provides the most straightforward way to submit a compiled Spark application to the cluster. For standalone clusters, Spark currently @@ -272,6 +272,15 @@ should specify them through the `--jars` flag using comma as a delimiter (e.g. ` To control the application's configuration or execution environment, see [Spark Configuration](configuration.html). +Additionally, standalone `cluster` mode supports restarting your application automatically if it +exited with non-zero exit code. To use this feature, you may pass in the `--supervise` flag to +`spark-submit` when launching your application. Then, if you wish to kill an application that is +failing repeatedly, you may do so through: + + ./bin/spark-class org.apache.spark.deploy.Client kill + +You can find the driver ID through the standalone Master web UI at `http://:8080`. + # Resource Scheduling The standalone cluster mode currently only supports a simple FIFO scheduler across applications. From faa8fd8178642ef8fce14186abc45a189042efd4 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Dec 2014 12:48:24 -0800 Subject: [PATCH 164/305] [SPARK-4215] Allow requesting / killing executors only in YARN mode Currently this doesn't do anything in other modes, so we might as well just disable it rather than having the user mistakenly rely on it. Author: Andrew Or Closes #3615 from andrewor14/dynamic-allocation-yarn-only and squashes the following commits: ce6487a [Andrew Or] Allow requesting / killing executors only in YARN mode --- .../src/main/scala/org/apache/spark/SparkContext.scala | 10 +++++++++- .../apache/spark/ExecutorAllocationManagerSuite.scala | 1 + 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index aded7c12e274e..8e5378ecc08de 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -357,8 +357,12 @@ class SparkContext(config: SparkConf) extends Logging { } // Optionally scale number of executors dynamically based on workload. Exposed for testing. + private val dynamicAllocationEnabled = conf.getBoolean("spark.dynamicAllocation.enabled", false) + private val dynamicAllocationTesting = conf.getBoolean("spark.dynamicAllocation.testing", false) private[spark] val executorAllocationManager: Option[ExecutorAllocationManager] = - if (conf.getBoolean("spark.dynamicAllocation.enabled", false)) { + if (dynamicAllocationEnabled) { + assert(master.contains("yarn") || dynamicAllocationTesting, + "Dynamic allocation of executors is currently only supported in YARN mode") Some(new ExecutorAllocationManager(this)) } else { None @@ -989,6 +993,8 @@ class SparkContext(config: SparkConf) extends Logging { */ @DeveloperApi def requestExecutors(numAdditionalExecutors: Int): Boolean = { + assert(master.contains("yarn") || dynamicAllocationTesting, + "Requesting executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => b.requestExecutors(numAdditionalExecutors) @@ -1005,6 +1011,8 @@ class SparkContext(config: SparkConf) extends Logging { */ @DeveloperApi def killExecutors(executorIds: Seq[String]): Boolean = { + assert(master.contains("yarn") || dynamicAllocationTesting, + "Killing executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => b.killExecutors(executorIds) diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index ce804f94f3267..c817f6dcede75 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -35,6 +35,7 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext { .setMaster("local") .setAppName("test-executor-allocation-manager") .set("spark.dynamicAllocation.enabled", "true") + .set("spark.dynamicAllocation.testing", "true") intercept[SparkException] { new SparkContext(conf) } SparkEnv.get.stop() // cleanup the created environment SparkContext.clearActiveContext() From e230da18f8c354b4b80416aa5277420397acf4f2 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 10 Dec 2014 13:29:27 -0800 Subject: [PATCH 165/305] [SPARK-4793] [Deploy] ensure .jar at end of line sometimes I switch between different version and do not want to rebuild spark, so I rename assembly.jar into .jar.bak, but still caught by `compute-classpath.sh` Author: Daoyuan Wang Closes #3641 from adrian-wang/jar and squashes the following commits: 45cbfd0 [Daoyuan Wang] ensure .jar at end of line --- bin/compute-classpath.sh | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 298641f2684de..685051eeed9f1 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -68,14 +68,14 @@ else assembly_folder="$ASSEMBLY_DIR" fi -num_jars="$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar" | wc -l)" +num_jars="$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*\.jar$" | wc -l)" if [ "$num_jars" -eq "0" ]; then echo "Failed to find Spark assembly in $assembly_folder" echo "You need to build Spark before running this program." exit 1 fi if [ "$num_jars" -gt "1" ]; then - jars_list=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*.jar") + jars_list=$(ls "$assembly_folder" | grep "spark-assembly.*hadoop.*.jar$") echo "Found multiple Spark assembly jars in $assembly_folder:" echo "$jars_list" echo "Please remove all but one jar." @@ -108,7 +108,7 @@ else datanucleus_dir="$FWDIR"/lib_managed/jars fi -datanucleus_jars="$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar")" +datanucleus_jars="$(find "$datanucleus_dir" 2>/dev/null | grep "datanucleus-.*\\.jar$")" datanucleus_jars="$(echo "$datanucleus_jars" | tr "\n" : | sed s/:$//g)" if [ -n "$datanucleus_jars" ]; then From 447ae2de5d4c2af865fdb63f8b876b865de60f74 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Wed, 10 Dec 2014 14:19:37 -0800 Subject: [PATCH 166/305] [SPARK-4569] Rename 'externalSorting' in Aggregator Hi all - I've renamed the unhelpfully named variable and added a comment clarifying what's actually happening. Author: Ilya Ganelin Closes #3666 from ilganeli/SPARK-4569B and squashes the following commits: 1810394 [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator e2d2092 [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator d7cefec [Ilya Ganelin] [SPARK-4569] Rename 'externalSorting' in Aggregator 5b3f39c [Ilya Ganelin] [SPARK-4569] Rename in Aggregator --- core/src/main/scala/org/apache/spark/Aggregator.scala | 10 ++++++---- 1 file changed, 6 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index 79c9c451d273d..09eb9605fb799 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -34,7 +34,9 @@ case class Aggregator[K, V, C] ( mergeValue: (C, V) => C, mergeCombiners: (C, C) => C) { - private val externalSorting = SparkEnv.get.conf.getBoolean("spark.shuffle.spill", true) + // When spilling is enabled sorting will happen externally, but not necessarily with an + // ExternalSorter. + private val isSpillEnabled = SparkEnv.get.conf.getBoolean("spark.shuffle.spill", true) @deprecated("use combineValuesByKey with TaskContext argument", "0.9.0") def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]]): Iterator[(K, C)] = @@ -42,7 +44,7 @@ case class Aggregator[K, V, C] ( def combineValuesByKey(iter: Iterator[_ <: Product2[K, V]], context: TaskContext): Iterator[(K, C)] = { - if (!externalSorting) { + if (!isSpillEnabled) { val combiners = new AppendOnlyMap[K,C] var kv: Product2[K, V] = null val update = (hadValue: Boolean, oldValue: C) => { @@ -71,9 +73,9 @@ case class Aggregator[K, V, C] ( combineCombinersByKey(iter, null) def combineCombinersByKey(iter: Iterator[_ <: Product2[K, C]], context: TaskContext) - : Iterator[(K, C)] = + : Iterator[(K, C)] = { - if (!externalSorting) { + if (!isSpillEnabled) { val combiners = new AppendOnlyMap[K,C] var kc: Product2[K, C] = null val update = (hadValue: Boolean, oldValue: C) => { From 4f93d0cabe5d1fc7c0fd0a33d992fd85df1fecb4 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 10 Dec 2014 14:27:53 -0800 Subject: [PATCH 167/305] [SPARK-4759] Fix driver hanging from coalescing partitions The driver hangs sometimes when we coalesce RDD partitions. See JIRA for more details and reproduction. This is because our use of empty string as default preferred location in `CoalescedRDDPartition` causes the `TaskSetManager` to schedule the corresponding task on host `""` (empty string). The intended semantics here, however, is that the partition does not have a preferred location, and the TSM should schedule the corresponding task accordingly. Author: Andrew Or Closes #3633 from andrewor14/coalesce-preferred-loc and squashes the following commits: e520d6b [Andrew Or] Oops 3ebf8bd [Andrew Or] A few comments f370a4e [Andrew Or] Fix tests 2f7dfb6 [Andrew Or] Avoid using empty string as default preferred location --- .../org/apache/spark/rdd/CoalescedRDD.scala | 36 +++++++++++-------- .../scala/org/apache/spark/rdd/RDDSuite.scala | 2 +- 2 files changed, 22 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 9fab1d78abb04..b073eba8a1574 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -35,11 +35,10 @@ import org.apache.spark.util.Utils * @param preferredLocation the preferred location for this partition */ private[spark] case class CoalescedRDDPartition( - index: Int, - @transient rdd: RDD[_], - parentsIndices: Array[Int], - @transient preferredLocation: String = "" - ) extends Partition { + index: Int, + @transient rdd: RDD[_], + parentsIndices: Array[Int], + @transient preferredLocation: Option[String] = None) extends Partition { var parents: Seq[Partition] = parentsIndices.map(rdd.partitions(_)) @throws(classOf[IOException]) @@ -55,9 +54,10 @@ private[spark] case class CoalescedRDDPartition( * @return locality of this coalesced partition between 0 and 1 */ def localFraction: Double = { - val loc = parents.count(p => - rdd.context.getPreferredLocs(rdd, p.index).map(tl => tl.host).contains(preferredLocation)) - + val loc = parents.count { p => + val parentPreferredLocations = rdd.context.getPreferredLocs(rdd, p.index).map(_.host) + preferredLocation.exists(parentPreferredLocations.contains) + } if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble) } } @@ -73,9 +73,9 @@ private[spark] case class CoalescedRDDPartition( * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance */ private[spark] class CoalescedRDD[T: ClassTag]( - @transient var prev: RDD[T], - maxPartitions: Int, - balanceSlack: Double = 0.10) + @transient var prev: RDD[T], + maxPartitions: Int, + balanceSlack: Double = 0.10) extends RDD[T](prev.context, Nil) { // Nil since we implement getDependencies override def getPartitions: Array[Partition] = { @@ -113,7 +113,7 @@ private[spark] class CoalescedRDD[T: ClassTag]( * @return the machine most preferred by split */ override def getPreferredLocations(partition: Partition): Seq[String] = { - List(partition.asInstanceOf[CoalescedRDDPartition].preferredLocation) + partition.asInstanceOf[CoalescedRDDPartition].preferredLocation.toSeq } } @@ -147,7 +147,7 @@ private[spark] class CoalescedRDD[T: ClassTag]( * */ -private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { +private class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanceSlack: Double) { def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.size < o2.size def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = @@ -341,8 +341,14 @@ private[spark] class PartitionCoalescer(maxPartitions: Int, prev: RDD[_], balanc } } -private[spark] case class PartitionGroup(prefLoc: String = "") { +private case class PartitionGroup(prefLoc: Option[String] = None) { var arr = mutable.ArrayBuffer[Partition]() - def size = arr.size } + +private object PartitionGroup { + def apply(prefLoc: String): PartitionGroup = { + require(prefLoc != "", "Preferred location must not be empty") + PartitionGroup(Some(prefLoc)) + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 46fcb80fa1845..6836e9ab0fd6b 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -294,7 +294,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("coalesced RDDs with locality") { val data3 = sc.makeRDD(List((1,List("a","c")), (2,List("a","b","c")), (3,List("b")))) val coal3 = data3.coalesce(3) - val list3 = coal3.partitions.map(p => p.asInstanceOf[CoalescedRDDPartition].preferredLocation) + val list3 = coal3.partitions.flatMap(_.asInstanceOf[CoalescedRDDPartition].preferredLocation) assert(list3.sorted === Array("a","b","c"), "Locality preferences are dropped") // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 From 36bdb5b748ff670a9bafd787e40c9e142c9a85b9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 10 Dec 2014 14:41:16 -0800 Subject: [PATCH 168/305] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #2883 (close requested by 'pwendell') Closes #3364 (close requested by 'pwendell') Closes #4458 (close requested by 'pwendell') Closes #1574 (close requested by 'andrewor14') Closes #2546 (close requested by 'andrewor14') Closes #2516 (close requested by 'andrewor14') Closes #154 (close requested by 'andrewor14') From 652b781a9b543cb17d7da91f5c3bebe5a02e0478 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Wed, 10 Dec 2014 15:01:15 -0800 Subject: [PATCH 169/305] SPARK-3526 Add section about data locality to the tuning guide cc kayousterhout I have a few outstanding questions from compiling this documentation: - What's the difference between NO_PREF and ANY? I understand the implications of the ordering but don't know what an example of each would be - Why is NO_PREF ahead of RACK_LOCAL? I would think it'd be better to schedule rack-local tasks ahead of no preference if you could only do one or the other. Is the idea to wait longer and hope for the rack-local tasks to turn into node-local or better? - Will there be a datacenter-local locality level in the future? Apache Cassandra for example has this level Author: Andrew Ash Closes #2519 from ash211/SPARK-3526 and squashes the following commits: 44cff28 [Andrew Ash] Link to spark.locality parameters rather than copying the list 6d5d966 [Andrew Ash] Stay focused on Spark, no astronaut architecture mumbo-jumbo 20e0e31 [Andrew Ash] SPARK-3526 Add section about data locality to the tuning guide --- docs/tuning.md | 33 +++++++++++++++++++++++++++++++++ 1 file changed, 33 insertions(+) diff --git a/docs/tuning.md b/docs/tuning.md index 0e2447dd46394..c4ca766328c1e 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -233,6 +233,39 @@ Spark prints the serialized size of each task on the master, so you can look at decide whether your tasks are too large; in general tasks larger than about 20 KB are probably worth optimizing. +## Data Locality + +Data locality can have a major impact on the performance of Spark jobs. If data and the code that +operates on it are together than computation tends to be fast. But if code and data are separated, +one must move to the other. Typically it is faster to ship serialized code from place to place than +a chunk of data because code size is much smaller than data. Spark builds its scheduling around +this general principle of data locality. + +Data locality is how close data is to the code processing it. There are several levels of +locality based on the data's current location. In order from closest to farthest: + +- `PROCESS_LOCAL` data is in the same JVM as the running code. This is the best locality + possible +- `NODE_LOCAL` data is on the same node. Examples might be in HDFS on the same node, or in + another executor on the same node. This is a little slower than `PROCESS_LOCAL` because the data + has to travel between processes +- `NO_PREF` data is accessed equally quickly from anywhere and has no locality preference +- `RACK_LOCAL` data is on the same rack of servers. Data is on a different server on the same rack + so needs to be sent over the network, typically through a single switch +- `ANY` data is elsewhere on the network and not in the same rack + +Spark prefers to schedule all tasks at the best locality level, but this is not always possible. In +situations where there is no unprocessed data on any idle executor, Spark switches to lower locality +levels. There are two options: a) wait until a busy CPU frees up to start a task on data on the same +server, or b) immediately start a new task in a farther away place that requires moving data there. + +What Spark typically does is wait a bit in the hopes that a busy CPU frees up. Once that timeout +expires, it starts moving the data from far away to the free CPU. The wait timeout for fallback +between each level can be configured individually or all together in one parameter; see the +`spark.locality` parameters on the [configuration page](configuration.html#scheduling) for details. +You should increase these settings if your tasks are long and see poor locality, but the default +usually works well. + # Summary This has been a short guide to point out the main concerns you should know about when tuning a From 57d37f9c7193f1cb04a22d7f512178696b655b71 Mon Sep 17 00:00:00 2001 From: "Zhang, Liye" Date: Wed, 10 Dec 2014 20:44:59 -0800 Subject: [PATCH 170/305] [CORE]codeStyle: uniform ConcurrentHashMap define in StorageLevel.scala with other places Author: Zhang, Liye Closes #2793 from liyezhang556520/uniformHashMap and squashes the following commits: 5884735 [Zhang, Liye] [CORE]codeStyle: uniform ConcurrentHashMap define in StorageLevel.scala --- .../main/scala/org/apache/spark/storage/StorageLevel.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 56edc4fe2e4ad..e5e1cf5a69a19 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -18,6 +18,7 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} +import java.util.concurrent.ConcurrentHashMap import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils @@ -220,8 +221,7 @@ object StorageLevel { getCachedStorageLevel(obj) } - private[spark] val storageLevelCache = - new java.util.concurrent.ConcurrentHashMap[StorageLevel, StorageLevel]() + private[spark] val storageLevelCache = new ConcurrentHashMap[StorageLevel, StorageLevel]() private[spark] def getCachedStorageLevel(level: StorageLevel): StorageLevel = { storageLevelCache.putIfAbsent(level, level) From 2a5b5fd4ccf28fab5b7e32a54170be92d5d23ba6 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Wed, 10 Dec 2014 23:41:15 -0800 Subject: [PATCH 171/305] [SPARK-4791] [sql] Infer schema from case class with multiple constructors Modified ScalaReflection.schemaFor to take primary constructor of Product when there are multiple constructors. Added test to suite which failed before but works now. Needed for [https://github.com/apache/spark/pull/3637] CC: marmbrus Author: Joseph K. Bradley Closes #3646 from jkbradley/sql-reflection and squashes the following commits: 796b2e4 [Joseph K. Bradley] Modified ScalaReflection.schemaFor to take primary constructor of Product when there are multiple constructors. Added test to suite which failed before but works now. --- .../spark/sql/catalyst/ScalaReflection.scala | 14 +++++++++++++- .../spark/sql/catalyst/ScalaReflectionSuite.scala | 14 ++++++++++++++ 2 files changed, 27 insertions(+), 1 deletion(-) 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 71034c2c43c77..2cf241de61f7a 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 @@ -118,7 +118,19 @@ trait ScalaReflection { case t if t <:< typeOf[Product] => val formalTypeArgs = t.typeSymbol.asClass.typeParams val TypeRef(_, _, actualTypeArgs) = t - val params = t.member(nme.CONSTRUCTOR).asMethod.paramss + val constructorSymbol = t.member(nme.CONSTRUCTOR) + val params = if (constructorSymbol.isMethod) { + constructorSymbol.asMethod.paramss + } else { + // Find the primary constructor, and use its parameter ordering. + val primaryConstructorSymbol: Option[Symbol] = constructorSymbol.asTerm.alternatives.find( + s => s.isMethod && s.asMethod.isPrimaryConstructor) + if (primaryConstructorSymbol.isEmpty) { + sys.error("Internal SQL error: Product object did not have a primary constructor.") + } else { + primaryConstructorSymbol.get.asMethod.paramss + } + } Schema(StructType( params.head.map { p => val Schema(dataType, nullable) = 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 ddc3d44869c98..7be24bea7d5a6 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 @@ -68,6 +68,10 @@ case class ComplexData( case class GenericData[A]( genericField: A) +case class MultipleConstructorsData(a: Int, b: String, c: Double) { + def this(b: String, a: Int) = this(a, b, c = 1.0) +} + class ScalaReflectionSuite extends FunSuite { import ScalaReflection._ @@ -253,4 +257,14 @@ class ScalaReflectionSuite extends FunSuite { Row(1, 1, 1, 1, 1, 1, true)) assert(convertToCatalyst(data, dataType) === convertedData) } + + test("infer schema from case class with multiple constructors") { + val dataType = schemaFor[MultipleConstructorsData].dataType + dataType match { + case s: StructType => + // Schema should have order: a: Int, b: String, c: Double + assert(s.fieldNames === Seq("a", "b", "c")) + assert(s.fields.map(_.dataType) === Seq(IntegerType, StringType, DoubleType)) + } + } } From b004150adb503ddbb54d5cd544e39ad974497c41 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 11 Dec 2014 06:21:23 -0800 Subject: [PATCH 172/305] [SPARK-4806] Streaming doc update for 1.2 Important updates to the streaming programming guide - Make the fault-tolerance properties easier to understand, with information about write ahead logs - Update the information about deploying the spark streaming app with information about Driver HA - Update Receiver guide to discuss reliable vs unreliable receivers. Author: Tathagata Das Author: Josh Rosen Author: Josh Rosen Closes #3653 from tdas/streaming-doc-update-1.2 and squashes the following commits: f53154a [Tathagata Das] Addressed Josh's comments. ce299e4 [Tathagata Das] Minor update. ca19078 [Tathagata Das] Minor change f746951 [Tathagata Das] Mentioned performance problem with WAL 7787209 [Tathagata Das] Merge branch 'streaming-doc-update-1.2' of github.com:tdas/spark into streaming-doc-update-1.2 2184729 [Tathagata Das] Updated Kafka and Flume guides with reliability information. 2f3178c [Tathagata Das] Added more information about writing reliable receivers in the custom receiver guide. 91aa5aa [Tathagata Das] Improved API Docs menu 5707581 [Tathagata Das] Added Pythn API badge b9c8c24 [Tathagata Das] Merge pull request #26 from JoshRosen/streaming-programming-guide b8c8382 [Josh Rosen] minor fixes a4ef126 [Josh Rosen] Restructure parts of the fault-tolerance section to read a bit nicer when skipping over the headings 65f66cd [Josh Rosen] Fix broken link to fault-tolerance semantics section. f015397 [Josh Rosen] Minor grammar / pluralization fixes. 3019f3a [Josh Rosen] Fix minor Markdown formatting issues aa8bb87 [Tathagata Das] Small update. 195852c [Tathagata Das] Updated based on Josh's comments, updated receiver reliability and deploying section, and also updated configuration. 17b99fb [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-doc-update-1.2 a0217c0 [Tathagata Das] Changed Deploying menu layout 67fcffc [Tathagata Das] Added cluster mode + supervise example to submitting application guide. e45453b [Tathagata Das] Update streaming guide, added deploying section. 192c7a7 [Tathagata Das] Added more info about Python API, and rewrote the checkpointing section. --- docs/_layouts/global.html | 13 +- docs/configuration.md | 133 ++-- docs/streaming-custom-receivers.md | 90 ++- docs/streaming-flume-integration.md | 13 +- docs/streaming-kafka-integration.md | 17 + docs/streaming-programming-guide.md | 1068 +++++++++++++++------------ docs/submitting-applications.md | 36 +- 7 files changed, 819 insertions(+), 551 deletions(-) diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 627ed37de4a9c..8841f7675d35e 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -33,7 +33,7 @@