From f7e21dd1ec4541be54eb01d8b15cfcc6714feed0 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 8 Apr 2015 10:14:52 -0700 Subject: [PATCH 001/128] [SPARK-6506] [pyspark] Do not try to retrieve SPARK_HOME when not needed... .... In particular, this makes pyspark in yarn-cluster mode fail unless SPARK_HOME is set, when it's not really needed. Author: Marcelo Vanzin Closes #5405 from vanzin/SPARK-6506 and squashes the following commits: e184507 [Marcelo Vanzin] [SPARK-6506] [pyspark] Do not try to retrieve SPARK_HOME when not needed. --- python/pyspark/java_gateway.py | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 0a16cbd8bff62..2a5e84a7dfdb4 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -29,11 +29,10 @@ def launch_gateway(): - SPARK_HOME = os.environ["SPARK_HOME"] - if "PYSPARK_GATEWAY_PORT" in os.environ: gateway_port = int(os.environ["PYSPARK_GATEWAY_PORT"]) else: + SPARK_HOME = os.environ["SPARK_HOME"] # Launch the Py4j gateway using Spark's run command so that we pick up the # proper classpath and settings from spark-env.sh on_windows = platform.system() == "Windows" From 9d44ddce1d1e19011026605549c37d0db6d6afa1 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 8 Apr 2015 10:26:45 -0700 Subject: [PATCH 002/128] [SPARK-6753] Clone SparkConf in ShuffleSuite tests Prior to this change, the unit test for SPARK-3426 did not clone the original SparkConf, which meant that that test did not use the options set by suites that subclass ShuffleSuite.scala. This commit fixes that problem. JoshRosen would be great if you could take a look at this, since you wrote this test originally. Author: Kay Ousterhout Closes #5401 from kayousterhout/SPARK-6753 and squashes the following commits: 368c540 [Kay Ousterhout] [SPARK-6753] Clone SparkConf in ShuffleSuite tests --- core/src/test/scala/org/apache/spark/ShuffleSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index f57921b768310..30b6184c77839 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -242,14 +242,14 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex shuffleSpillCompress <- Set(true, false); shuffleCompress <- Set(true, false) ) { - val conf = new SparkConf() + val myConf = conf.clone() .setAppName("test") .setMaster("local") .set("spark.shuffle.spill.compress", shuffleSpillCompress.toString) .set("spark.shuffle.compress", shuffleCompress.toString) .set("spark.shuffle.memoryFraction", "0.001") resetSparkContext() - sc = new SparkContext(conf) + sc = new SparkContext(myConf) try { sc.parallelize(0 until 100000).map(i => (i / 4, i)).groupByKey().collect() } catch { From 8d812f9986f2edf420a18ca822711c9765f480e2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 8 Apr 2015 11:31:48 -0700 Subject: [PATCH 003/128] [SPARK-6765] Fix test code style for graphx. So we can turn style checker on for test code. Author: Reynold Xin Closes #5410 from rxin/test-style-graphx and squashes the following commits: 89e253a [Reynold Xin] [SPARK-6765] Fix test code style for graphx. --- .../org/apache/spark/graphx/GraphSuite.scala | 71 ++++++++++--------- .../spark/graphx/LocalSparkContext.scala | 2 +- .../apache/spark/graphx/VertexRDDSuite.scala | 26 +++---- .../graphx/lib/ConnectedComponentsSuite.scala | 18 ++--- .../spark/graphx/lib/PageRankSuite.scala | 33 ++++----- .../StronglyConnectedComponentsSuite.scala | 23 +++--- 6 files changed, 88 insertions(+), 85 deletions(-) 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 8d15150458d26..a570e4ed75fc3 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -38,12 +38,12 @@ class GraphSuite extends FunSuite with LocalSparkContext { val doubleRing = ring ++ ring val graph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1) assert(graph.edges.count() === doubleRing.size) - assert(graph.edges.collect.forall(e => e.attr == 1)) + assert(graph.edges.collect().forall(e => e.attr == 1)) // uniqueEdges option should uniquify edges and store duplicate count in edge attributes val uniqueGraph = Graph.fromEdgeTuples(sc.parallelize(doubleRing), 1, Some(RandomVertexCut)) assert(uniqueGraph.edges.count() === ring.size) - assert(uniqueGraph.edges.collect.forall(e => e.attr == 2)) + assert(uniqueGraph.edges.collect().forall(e => e.attr == 2)) } } @@ -64,7 +64,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert( graph.edges.count() === rawEdges.size ) // Vertices not explicitly provided but referenced by edges should be created automatically assert( graph.vertices.count() === 100) - graph.triplets.collect.map { et => + graph.triplets.collect().map { et => assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) } @@ -75,15 +75,17 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val star = starGraph(sc, n) - assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect.toSet === - (1 to n).map(x => (0: VertexId, x: VertexId, "v", "v")).toSet) + assert(star.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)).collect().toSet + === (1 to n).map(x => (0: VertexId, x: VertexId, "v", "v")).toSet) } } test("partitionBy") { withSpark { sc => - def mkGraph(edges: List[(Long, Long)]) = Graph.fromEdgeTuples(sc.parallelize(edges, 2), 0) - def nonemptyParts(graph: Graph[Int, Int]) = { + def mkGraph(edges: List[(Long, Long)]): Graph[Int, Int] = { + Graph.fromEdgeTuples(sc.parallelize(edges, 2), 0) + } + def nonemptyParts(graph: Graph[Int, Int]): RDD[List[Edge[Int]]] = { graph.edges.partitionsRDD.mapPartitions { iter => Iterator(iter.next()._2.iterator.toList) }.filter(_.nonEmpty) @@ -102,7 +104,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(nonemptyParts(mkGraph(sameSrcEdges).partitionBy(EdgePartition1D)).count === 1) // partitionBy(CanonicalRandomVertexCut) puts edges that are identical modulo direction into // the same partition - assert(nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)).count === 1) + assert( + nonemptyParts(mkGraph(canonicalEdges).partitionBy(CanonicalRandomVertexCut)).count === 1) // partitionBy(EdgePartition2D) puts identical edges in the same partition assert(nonemptyParts(mkGraph(identicalEdges).partitionBy(EdgePartition2D)).count === 1) @@ -140,10 +143,10 @@ class GraphSuite extends FunSuite with LocalSparkContext { val g = Graph( sc.parallelize(List((0L, "a"), (1L, "b"), (2L, "c"))), sc.parallelize(List(Edge(0L, 1L, 1), Edge(0L, 2L, 1)), 2)) - assert(g.triplets.collect.map(_.toTuple).toSet === + assert(g.triplets.collect().map(_.toTuple).toSet === Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1))) val gPart = g.partitionBy(EdgePartition2D) - assert(gPart.triplets.collect.map(_.toTuple).toSet === + assert(gPart.triplets.collect().map(_.toTuple).toSet === Set(((0L, "a"), (1L, "b"), 1), ((0L, "a"), (2L, "c"), 1))) } } @@ -154,10 +157,10 @@ class GraphSuite extends FunSuite with LocalSparkContext { val star = starGraph(sc, n) // mapVertices preserving type val mappedVAttrs = star.mapVertices((vid, attr) => attr + "2") - assert(mappedVAttrs.vertices.collect.toSet === (0 to n).map(x => (x: VertexId, "v2")).toSet) + assert(mappedVAttrs.vertices.collect().toSet === (0 to n).map(x => (x: VertexId, "v2")).toSet) // mapVertices changing type val mappedVAttrs2 = star.mapVertices((vid, attr) => attr.length) - assert(mappedVAttrs2.vertices.collect.toSet === (0 to n).map(x => (x: VertexId, 1)).toSet) + assert(mappedVAttrs2.vertices.collect().toSet === (0 to n).map(x => (x: VertexId, 1)).toSet) } } @@ -177,12 +180,12 @@ class GraphSuite extends FunSuite with LocalSparkContext { // Trigger initial vertex replication graph0.triplets.foreach(x => {}) // Change type of replicated vertices, but preserve erased type - val graph1 = graph0.mapVertices { - case (vid, integerOpt) => integerOpt.map((x: java.lang.Integer) => (x.toDouble): java.lang.Double) + val graph1 = graph0.mapVertices { case (vid, integerOpt) => + integerOpt.map((x: java.lang.Integer) => x.toDouble: java.lang.Double) } // Access replicated vertices, exposing the erased type val graph2 = graph1.mapTriplets(t => t.srcAttr.get) - assert(graph2.edges.map(_.attr).collect.toSet === Set[java.lang.Double](1.0, 2.0, 3.0)) + assert(graph2.edges.map(_.attr).collect().toSet === Set[java.lang.Double](1.0, 2.0, 3.0)) } } @@ -202,7 +205,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val star = starGraph(sc, n) - assert(star.mapTriplets(et => et.srcAttr + et.dstAttr).edges.collect.toSet === + assert(star.mapTriplets(et => et.srcAttr + et.dstAttr).edges.collect().toSet === (1L to n).map(x => Edge(0, x, "vv")).toSet) } } @@ -211,7 +214,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { withSpark { sc => val n = 5 val star = starGraph(sc, n) - assert(star.reverse.outDegrees.collect.toSet === (1 to n).map(x => (x: VertexId, 1)).toSet) + assert(star.reverse.outDegrees.collect().toSet === (1 to n).map(x => (x: VertexId, 1)).toSet) } } @@ -221,7 +224,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val edges: RDD[Edge[Int]] = sc.parallelize(Array(Edge(1L, 2L, 0))) val graph = Graph(vertices, edges).reverse val result = graph.mapReduceTriplets[Int](et => Iterator((et.dstId, et.srcAttr)), _ + _) - assert(result.collect.toSet === Set((1L, 2))) + assert(result.collect().toSet === Set((1L, 2))) } } @@ -237,7 +240,8 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert(subgraph.vertices.collect().toSet === (0 to n by 2).map(x => (x, "v")).toSet) // And 4 edges. - assert(subgraph.edges.map(_.copy()).collect().toSet === (2 to n by 2).map(x => Edge(0, x, 1)).toSet) + assert(subgraph.edges.map(_.copy()).collect().toSet === + (2 to n by 2).map(x => Edge(0, x, 1)).toSet) } } @@ -273,9 +277,9 @@ class GraphSuite extends FunSuite with LocalSparkContext { sc.parallelize((1 to n).flatMap(x => List((0: VertexId, x: VertexId), (0: VertexId, x: VertexId))), 1), "v") val star2 = doubleStar.groupEdges { (a, b) => a} - assert(star2.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int]) === - star.edges.collect.toArray.sorted(Edge.lexicographicOrdering[Int])) - assert(star2.vertices.collect.toSet === star.vertices.collect.toSet) + assert(star2.edges.collect().toArray.sorted(Edge.lexicographicOrdering[Int]) === + star.edges.collect().toArray.sorted(Edge.lexicographicOrdering[Int])) + assert(star2.vertices.collect().toSet === star.vertices.collect().toSet) } } @@ -300,21 +304,23 @@ class GraphSuite extends FunSuite with LocalSparkContext { throw new Exception("map ran on edge with dst vid %d, which is odd".format(et.dstId)) } Iterator((et.srcId, 1)) - }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect.toSet + }, (a: Int, b: Int) => a + b, Some((active, EdgeDirection.In))).collect().toSet assert(numEvenNeighbors === (1 to n).map(x => (x: VertexId, n / 2)).toSet) // outerJoinVertices followed by mapReduceTriplets(activeSetOpt) - val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexId, (x+1) % n: VertexId)), 3) + val ringEdges = sc.parallelize((0 until n).map(x => (x: VertexId, (x + 1) % n: VertexId)), 3) val ring = Graph.fromEdgeTuples(ringEdges, 0) .mapVertices((vid, attr) => vid).cache() val changed = ring.vertices.filter { case (vid, attr) => attr % 2 == 1 }.mapValues(-_).cache() - val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => newOpt.getOrElse(old) } + val changedGraph = ring.outerJoinVertices(changed) { (vid, old, newOpt) => + newOpt.getOrElse(old) + } val numOddNeighbors = changedGraph.mapReduceTriplets(et => { // Map function should only run on edges with source in the active set if (et.srcId % 2 != 1) { throw new Exception("map ran on edge with src vid %d, which is even".format(et.dstId)) } Iterator((et.dstId, 1)) - }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect.toSet + }, (a: Int, b: Int) => a + b, Some(changed, EdgeDirection.Out)).collect().toSet assert(numOddNeighbors === (2 to n by 2).map(x => (x: VertexId, 1)).toSet) } @@ -340,17 +346,18 @@ class GraphSuite extends FunSuite with LocalSparkContext { val n = 5 val reverseStar = starGraph(sc, n).reverse.cache() // outerJoinVertices changing type - val reverseStarDegrees = - reverseStar.outerJoinVertices(reverseStar.outDegrees) { (vid, a, bOpt) => bOpt.getOrElse(0) } + val reverseStarDegrees = reverseStar.outerJoinVertices(reverseStar.outDegrees) { + (vid, a, bOpt) => bOpt.getOrElse(0) + } val neighborDegreeSums = reverseStarDegrees.mapReduceTriplets( et => Iterator((et.srcId, et.dstAttr), (et.dstId, et.srcAttr)), - (a: Int, b: Int) => a + b).collect.toSet + (a: Int, b: Int) => a + b).collect().toSet assert(neighborDegreeSums === Set((0: VertexId, n)) ++ (1 to n).map(x => (x: VertexId, 0))) // outerJoinVertices preserving type val messages = reverseStar.vertices.mapValues { (vid, attr) => vid.toString } val newReverseStar = reverseStar.outerJoinVertices(messages) { (vid, a, bOpt) => a + bOpt.getOrElse("") } - assert(newReverseStar.vertices.map(_._2).collect.toSet === + assert(newReverseStar.vertices.map(_._2).collect().toSet === (0 to n).map(x => "v%d".format(x)).toSet) } } @@ -361,7 +368,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val edges = sc.parallelize(List(Edge(1, 2, 0), Edge(2, 1, 0)), 2) val graph = Graph(verts, edges) val triplets = graph.triplets.map(et => (et.srcId, et.dstId, et.srcAttr, et.dstAttr)) - .collect.toSet + .collect().toSet assert(triplets === Set((1: VertexId, 2: VertexId, "a", "b"), (2: VertexId, 1: VertexId, "b", "a"))) } @@ -417,7 +424,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { val graph = Graph.fromEdgeTuples(edges, 1) val neighborAttrSums = graph.mapReduceTriplets[Int]( et => Iterator((et.dstId, et.srcAttr)), _ + _) - assert(neighborAttrSums.collect.toSet === Set((0: VertexId, n))) + assert(neighborAttrSums.collect().toSet === Set((0: VertexId, n))) } finally { sc.stop() } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala index a3e28efc75a98..d2ad9be555770 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/LocalSparkContext.scala @@ -26,7 +26,7 @@ import org.apache.spark.SparkContext */ trait LocalSparkContext { /** Runs `f` on a new SparkContext and ensures that it is stopped afterwards. */ - def withSpark[T](f: SparkContext => T) = { + def withSpark[T](f: SparkContext => T): T = { val conf = new SparkConf() GraphXUtils.registerKryoClasses(conf) val sc = new SparkContext("local", "test", conf) diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index c9443d11c76cf..d0a7198d691d7 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.storage.StorageLevel class VertexRDDSuite extends FunSuite with LocalSparkContext { - def vertices(sc: SparkContext, n: Int) = { + private def vertices(sc: SparkContext, n: Int) = { VertexRDD(sc.parallelize((0 to n).map(x => (x.toLong, x)), 5)) } @@ -52,7 +52,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexA = VertexRDD(sc.parallelize(0 until 75, 2).map(i => (i.toLong, 0))).cache() val vertexB = VertexRDD(sc.parallelize(25 until 100, 2).map(i => (i.toLong, 1))).cache() val vertexC = vertexA.minus(vertexB) - assert(vertexC.map(_._1).collect.toSet === (0 until 25).toSet) + assert(vertexC.map(_._1).collect().toSet === (0 until 25).toSet) } } @@ -62,7 +62,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexB: RDD[(VertexId, Int)] = sc.parallelize(25 until 100, 2).map(i => (i.toLong, 1)).cache() val vertexC = vertexA.minus(vertexB) - assert(vertexC.map(_._1).collect.toSet === (0 until 25).toSet) + assert(vertexC.map(_._1).collect().toSet === (0 until 25).toSet) } } @@ -72,7 +72,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexB = VertexRDD(sc.parallelize(50 until 100, 2).map(i => (i.toLong, 1))) assert(vertexA.partitions.size != vertexB.partitions.size) val vertexC = vertexA.minus(vertexB) - assert(vertexC.map(_._1).collect.toSet === (0 until 50).toSet) + assert(vertexC.map(_._1).collect().toSet === (0 until 50).toSet) } } @@ -106,7 +106,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexB = VertexRDD(sc.parallelize(8 until 16, 2).map(i => (i.toLong, 1))) assert(vertexA.partitions.size != vertexB.partitions.size) val vertexC = vertexA.diff(vertexB) - assert(vertexC.map(_._1).collect.toSet === (8 until 16).toSet) + assert(vertexC.map(_._1).collect().toSet === (8 until 16).toSet) } } @@ -116,11 +116,11 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val verts = vertices(sc, n).cache() val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // leftJoin with another VertexRDD - assert(verts.leftJoin(evens) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === + assert(verts.leftJoin(evens) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) // leftJoin with an RDD val evensRDD = evens.map(identity) - assert(verts.leftJoin(evensRDD) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect.toSet === + assert(verts.leftJoin(evensRDD) { (id, a, bOpt) => a - bOpt.getOrElse(0) }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet ++ (1 to n by 2).map(x => (x.toLong, x)).toSet) } } @@ -134,7 +134,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexC = vertexA.leftJoin(vertexB) { (vid, old, newOpt) => old - newOpt.getOrElse(0) } - assert(vertexC.filter(v => v._2 != 0).map(_._1).collect.toSet == (1 to 99 by 2).toSet) + assert(vertexC.filter(v => v._2 != 0).map(_._1).collect().toSet == (1 to 99 by 2).toSet) } } @@ -144,11 +144,11 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val verts = vertices(sc, n).cache() val evens = verts.filter(q => ((q._2 % 2) == 0)).cache() // innerJoin with another VertexRDD - assert(verts.innerJoin(evens) { (id, a, b) => a - b }.collect.toSet === + assert(verts.innerJoin(evens) { (id, a, b) => a - b }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet) // innerJoin with an RDD val evensRDD = evens.map(identity) - assert(verts.innerJoin(evensRDD) { (id, a, b) => a - b }.collect.toSet === + assert(verts.innerJoin(evensRDD) { (id, a, b) => a - b }.collect().toSet === (0 to n by 2).map(x => (x.toLong, 0)).toSet) } } @@ -161,7 +161,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val vertexC = vertexA.innerJoin(vertexB) { (vid, old, newVal) => old - newVal } - assert(vertexC.filter(v => v._2 == 0).map(_._1).collect.toSet == (0 to 98 by 2).toSet) + assert(vertexC.filter(v => v._2 == 0).map(_._1).collect().toSet == (0 to 98 by 2).toSet) } } @@ -171,7 +171,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val verts = vertices(sc, n) val messageTargets = (0 to n) ++ (0 to n by 2) val messages = sc.parallelize(messageTargets.map(x => (x.toLong, 1))) - assert(verts.aggregateUsingIndex[Int](messages, _ + _).collect.toSet === + assert(verts.aggregateUsingIndex[Int](messages, _ + _).collect().toSet === (0 to n).map(x => (x.toLong, if (x % 2 == 0) 2 else 1)).toSet) } } @@ -183,7 +183,7 @@ class VertexRDDSuite extends FunSuite with LocalSparkContext { val edges = EdgeRDD.fromEdges(sc.parallelize(List.empty[Edge[Int]])) val rdd = VertexRDD(verts, edges, 0, (a: Int, b: Int) => a + b) // test merge function - assert(rdd.collect.toSet == Set((0L, 0), (1L, 3), (2L, 9))) + assert(rdd.collect().toSet == Set((0L, 0), (1L, 3), (2L, 9))) } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala index 3915be15b3434..4cc30a96408f8 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/ConnectedComponentsSuite.scala @@ -32,7 +32,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10) val ccGraph = gridGraph.connectedComponents() - val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum() assert(maxCCid === 0) } } // end of Grid connected components @@ -42,7 +42,7 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { withSpark { sc => val gridGraph = GraphGenerators.gridGraph(sc, 10, 10).reverse val ccGraph = gridGraph.connectedComponents() - val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum + val maxCCid = ccGraph.vertices.map { case (vid, ccId) => ccId }.sum() assert(maxCCid === 0) } } // end of Grid connected components @@ -50,8 +50,8 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Chain Connected Components") { withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val chain2 = (10 until 20).map(x => (x, x+1) ) + val chain1 = (0 until 9).map(x => (x, x + 1)) + val chain2 = (10 until 20).map(x => (x, x + 1)) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, 1.0) val ccGraph = twoChains.connectedComponents() @@ -73,12 +73,12 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { test("Reverse Chain Connected Components") { withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val chain2 = (10 until 20).map(x => (x, x+1) ) + val chain1 = (0 until 9).map(x => (x, x + 1)) + val chain2 = (10 until 20).map(x => (x, x + 1)) val rawEdges = sc.parallelize(chain1 ++ chain2, 3).map { case (s,d) => (s.toLong, d.toLong) } val twoChains = Graph.fromEdgeTuples(rawEdges, true).reverse val ccGraph = twoChains.connectedComponents() - val vertices = ccGraph.vertices.collect + val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { if (id < 10) { assert(cc === 0) @@ -120,9 +120,9 @@ class ConnectedComponentsSuite extends FunSuite with LocalSparkContext { // Build the initial Graph val graph = Graph(users, relationships, defaultUser) val ccGraph = graph.connectedComponents() - val vertices = ccGraph.vertices.collect + val vertices = ccGraph.vertices.collect() for ( (id, cc) <- vertices ) { - assert(cc == 0) + assert(cc === 0) } } } // end of toy connected components diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala index fc491ae327c2a..95804b07b1db0 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala @@ -19,15 +19,12 @@ package org.apache.spark.graphx.lib import org.scalatest.FunSuite -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ import org.apache.spark.graphx._ -import org.apache.spark.graphx.lib._ import org.apache.spark.graphx.util.GraphGenerators -import org.apache.spark.rdd._ + object GridPageRank { - def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double) = { + def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double): Seq[(VertexId, Double)] = { val inNbrs = Array.fill(nRows * nCols)(collection.mutable.MutableList.empty[Int]) val outDegree = Array.fill(nRows * nCols)(0) // Convert row column address into vertex ids (row major order) @@ -35,13 +32,13 @@ object GridPageRank { // Make the grid graph for (r <- 0 until nRows; c <- 0 until nCols) { val ind = sub2ind(r,c) - if (r+1 < nRows) { + if (r + 1 < nRows) { outDegree(ind) += 1 - inNbrs(sub2ind(r+1,c)) += ind + inNbrs(sub2ind(r + 1,c)) += ind } - if (c+1 < nCols) { + if (c + 1 < nCols) { outDegree(ind) += 1 - inNbrs(sub2ind(r,c+1)) += ind + inNbrs(sub2ind(r,c + 1)) += ind } } // compute the pagerank @@ -64,7 +61,7 @@ class PageRankSuite extends FunSuite with LocalSparkContext { def compareRanks(a: VertexRDD[Double], b: VertexRDD[Double]): Double = { a.leftJoin(b) { case (id, a, bOpt) => (a - bOpt.getOrElse(0.0)) * (a - bOpt.getOrElse(0.0)) } - .map { case (id, error) => error }.sum + .map { case (id, error) => error }.sum() } test("Star PageRank") { @@ -80,12 +77,12 @@ class PageRankSuite extends FunSuite with LocalSparkContext { // Static PageRank should only take 2 iterations to converge val notMatching = staticRanks1.innerZipJoin(staticRanks2) { (vid, pr1, pr2) => if (pr1 != pr2) 1 else 0 - }.map { case (vid, test) => test }.sum + }.map { case (vid, test) => test }.sum() assert(notMatching === 0) val staticErrors = staticRanks2.map { case (vid, pr) => - val correct = (vid > 0 && pr == resetProb) || - (vid == 0 && math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) < 1.0E-5) + val p = math.abs(pr - (resetProb + (1.0 - resetProb) * (resetProb * (nVertices - 1)) )) + val correct = (vid > 0 && pr == resetProb) || (vid == 0L && p < 1.0E-5) if (!correct) 1 else 0 } assert(staticErrors.sum === 0) @@ -95,8 +92,6 @@ class PageRankSuite extends FunSuite with LocalSparkContext { } } // end of test Star PageRank - - test("Grid PageRank") { withSpark { sc => val rows = 10 @@ -109,18 +104,18 @@ class PageRankSuite extends FunSuite with LocalSparkContext { val staticRanks = gridGraph.staticPageRank(numIter, resetProb).vertices.cache() val dynamicRanks = gridGraph.pageRank(tol, resetProb).vertices.cache() - val referenceRanks = VertexRDD(sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() + val referenceRanks = VertexRDD( + sc.parallelize(GridPageRank(rows, cols, numIter, resetProb))).cache() assert(compareRanks(staticRanks, referenceRanks) < errorTol) assert(compareRanks(dynamicRanks, referenceRanks) < errorTol) } } // end of Grid PageRank - test("Chain PageRank") { withSpark { sc => - val chain1 = (0 until 9).map(x => (x, x+1) ) - val rawEdges = sc.parallelize(chain1, 1).map { case (s,d) => (s.toLong, d.toLong) } + val chain1 = (0 until 9).map(x => (x, x + 1)) + val rawEdges = sc.parallelize(chain1, 1).map { case (s, d) => (s.toLong, d.toLong) } val chain = Graph.fromEdgeTuples(rawEdges, 1.0).cache() val resetProb = 0.15 val tol = 0.0001 diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala index df54aa37cad68..1f658c371ffcf 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/StronglyConnectedComponentsSuite.scala @@ -34,8 +34,8 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val edges = sc.parallelize(Seq.empty[Edge[Int]]) val graph = Graph(vertices, edges) val sccGraph = graph.stronglyConnectedComponents(5) - for ((id, scc) <- sccGraph.vertices.collect) { - assert(id == scc) + for ((id, scc) <- sccGraph.vertices.collect()) { + assert(id === scc) } } } @@ -45,8 +45,8 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val rawEdges = sc.parallelize((0L to 6L).map(x => (x, (x + 1) % 7))) val graph = Graph.fromEdgeTuples(rawEdges, -1) val sccGraph = graph.stronglyConnectedComponents(20) - for ((id, scc) <- sccGraph.vertices.collect) { - assert(0L == scc) + for ((id, scc) <- sccGraph.vertices.collect()) { + assert(0L === scc) } } } @@ -60,13 +60,14 @@ class StronglyConnectedComponentsSuite extends FunSuite with LocalSparkContext { val rawEdges = sc.parallelize(edges) val graph = Graph.fromEdgeTuples(rawEdges, -1) val sccGraph = graph.stronglyConnectedComponents(20) - for ((id, scc) <- sccGraph.vertices.collect) { - if (id < 3) - assert(0L == scc) - else if (id < 6) - assert(3L == scc) - else - assert(id == scc) + for ((id, scc) <- sccGraph.vertices.collect()) { + if (id < 3) { + assert(0L === scc) + } else if (id < 6) { + assert(3L === scc) + } else { + assert(id === scc) + } } } } From 66159c35010af35098dd1ec75475bb5d4d0fd6ca Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 8 Apr 2015 11:32:44 -0700 Subject: [PATCH 004/128] [SPARK-6765] Fix test code style for mllib. So we can turn style checker on for test code. Author: Reynold Xin Closes #5411 from rxin/test-style-mllib and squashes the following commits: d8a2569 [Reynold Xin] [SPARK-6765] Fix test code style for mllib. --- .../org/apache/spark/ml/param/TestParams.scala | 2 +- .../mllib/classification/NaiveBayesSuite.scala | 2 +- .../StreamingLogisticRegressionSuite.scala | 2 +- .../spark/mllib/clustering/KMeansSuite.scala | 10 +++++++--- .../spark/mllib/clustering/LDASuite.scala | 4 ++-- .../clustering/StreamingKMeansSuite.scala | 4 ++-- .../spark/mllib/random/RandomRDDsSuite.scala | 5 +++-- .../spark/mllib/recommendation/ALSSuite.scala | 3 +++ .../regression/RidgeRegressionSuite.scala | 2 +- .../StreamingLinearRegressionSuite.scala | 2 +- .../apache/spark/mllib/util/TestingUtils.scala | 14 +++++++------- .../spark/mllib/util/TestingUtilsSuite.scala | 18 +++++++++++------- 12 files changed, 40 insertions(+), 28 deletions(-) diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala index 1a65883d78a71..ce52f2f230085 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala @@ -28,7 +28,7 @@ class TestParams extends Params { def setInputCol(value: String): this.type = { set(inputCol, value); this } def getInputCol: String = get(inputCol) - override def validate(paramMap: ParamMap) = { + override def validate(paramMap: ParamMap): Unit = { val m = this.paramMap ++ paramMap require(m(maxIter) >= 0) require(m.contains(inputCol)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index f9fe3e006ccb8..ea89b17b7c08f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -102,7 +102,7 @@ class NaiveBayesSuite extends FunSuite with MLlibTestSparkContext { def validateModelFit( piData: Array[Double], thetaData: Array[Array[Double]], - model: NaiveBayesModel) = { + model: NaiveBayesModel): Unit = { def closeFit(d1: Double, d2: Double, precision: Double): Boolean = { (d1 - d2).abs <= precision } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala index d50c43d439187..5683b55e8500a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/StreamingLogisticRegressionSuite.scala @@ -30,7 +30,7 @@ import org.apache.spark.streaming.TestSuiteBase class StreamingLogisticRegressionSuite extends FunSuite with TestSuiteBase { // use longer wait time to ensure job completion - override def maxWaitTimeMillis = 30000 + override def maxWaitTimeMillis: Int = 30000 // Test if we can accurately learn B for Y = logistic(BX) on streaming data test("parameter accuracy") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 7bf250eb5a383..0f2b26d462ad2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -199,9 +199,13 @@ class KMeansSuite extends FunSuite with MLlibTestSparkContext { test("k-means|| initialization") { case class VectorWithCompare(x: Vector) extends Ordered[VectorWithCompare] { - @Override def compare(that: VectorWithCompare): Int = { - if(this.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x) > - that.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x)) -1 else 1 + override def compare(that: VectorWithCompare): Int = { + if (this.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x) > + that.x.toArray.foldLeft[Double](0.0)((acc, x) => acc + x * x)) { + -1 + } else { + 1 + } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala index 302d751eb8a94..15de10fd13a19 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/LDASuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.clustering import org.scalatest.FunSuite -import org.apache.spark.mllib.linalg.{DenseMatrix, Matrix, Vectors} +import org.apache.spark.mllib.linalg.{Vector, DenseMatrix, Matrix, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ @@ -141,7 +141,7 @@ private[clustering] object LDASuite { (terms.toArray, termWeights.toArray) } - def tinyCorpus = Array( + def tinyCorpus: Array[(Long, Vector)] = Array( Vectors.dense(1, 3, 0, 2, 8), Vectors.dense(0, 2, 1, 0, 4), Vectors.dense(2, 3, 12, 3, 1), diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala index 850c9fce507cd..f90025d535e45 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.random.XORShiftRandom class StreamingKMeansSuite extends FunSuite with TestSuiteBase { - override def maxWaitTimeMillis = 30000 + override def maxWaitTimeMillis: Int = 30000 test("accuracy for single center and equivalence to grand average") { // set parameters @@ -59,7 +59,7 @@ class StreamingKMeansSuite extends FunSuite with TestSuiteBase { // estimated center from streaming should exactly match the arithmetic mean of all data points // because the decay factor is set to 1.0 val grandMean = - input.flatten.map(x => x.toBreeze).reduce(_+_) / (numBatches * numPoints).toDouble + input.flatten.map(x => x.toBreeze).reduce(_ + _) / (numBatches * numPoints).toDouble assert(model.latestModel().clusterCenters(0) ~== Vectors.dense(grandMean.toArray) absTol 1E-5) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala index 6395188a0842a..63f2ea916d457 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala @@ -181,7 +181,8 @@ class RandomRDDsSuite extends FunSuite with MLlibTestSparkContext with Serializa val poisson = RandomRDDs.poissonVectorRDD(sc, poissonMean, rows, cols, parts, seed) testGeneratedVectorRDD(poisson, rows, cols, parts, poissonMean, math.sqrt(poissonMean), 0.1) - val exponential = RandomRDDs.exponentialVectorRDD(sc, exponentialMean, rows, cols, parts, seed) + val exponential = + RandomRDDs.exponentialVectorRDD(sc, exponentialMean, rows, cols, parts, seed) testGeneratedVectorRDD(exponential, rows, cols, parts, exponentialMean, exponentialMean, 0.1) val gamma = RandomRDDs.gammaVectorRDD(sc, gammaShape, gammaScale, rows, cols, parts, seed) @@ -197,7 +198,7 @@ private[random] class MockDistro extends RandomDataGenerator[Double] { // This allows us to check that each partition has a different seed override def nextValue(): Double = seed.toDouble - override def setSeed(seed: Long) = this.seed = seed + override def setSeed(seed: Long): Unit = this.seed = seed override def copy(): MockDistro = new MockDistro } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 8775c0ca9df84..b3798940ddc38 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -203,6 +203,7 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext { * @param numProductBlocks number of product blocks to partition products into * @param negativeFactors whether the generated user/product factors can have negative entries */ + // scalastyle:off def testALS( users: Int, products: Int, @@ -216,6 +217,8 @@ class ALSSuite extends FunSuite with MLlibTestSparkContext { numUserBlocks: Int = -1, numProductBlocks: Int = -1, negativeFactors: Boolean = true) { + // scalastyle:on + val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products, features, samplingRate, implicitPrefs, negativeWeights, negativeFactors) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index 43d61151e2471..d6c93cc0e49cd 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -35,7 +35,7 @@ private object RidgeRegressionSuite { class RidgeRegressionSuite extends FunSuite with MLlibTestSparkContext { - def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]) = { + def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]): Double = { predictions.zip(input).map { case (prediction, expected) => (prediction - expected.label) * (prediction - expected.label) }.reduceLeft(_ + _) / predictions.size diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala index 24fd8df691817..26604dbe6c1ef 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/StreamingLinearRegressionSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.streaming.TestSuiteBase class StreamingLinearRegressionSuite extends FunSuite with TestSuiteBase { // use longer wait time to ensure job completion - override def maxWaitTimeMillis = 20000 + override def maxWaitTimeMillis: Int = 20000 // Assert that two values are equal within tolerance epsilon def assertEqual(v1: Double, v2: Double, epsilon: Double) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala index e957fa5d25f4c..352193a67860c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtils.scala @@ -95,16 +95,16 @@ object TestingUtils { /** * Comparison using absolute tolerance. */ - def absTol(eps: Double): CompareDoubleRightSide = CompareDoubleRightSide(AbsoluteErrorComparison, - x, eps, ABS_TOL_MSG) + def absTol(eps: Double): CompareDoubleRightSide = + CompareDoubleRightSide(AbsoluteErrorComparison, x, eps, ABS_TOL_MSG) /** * Comparison using relative tolerance. */ - def relTol(eps: Double): CompareDoubleRightSide = CompareDoubleRightSide(RelativeErrorComparison, - x, eps, REL_TOL_MSG) + def relTol(eps: Double): CompareDoubleRightSide = + CompareDoubleRightSide(RelativeErrorComparison, x, eps, REL_TOL_MSG) - override def toString = x.toString + override def toString: String = x.toString } case class CompareVectorRightSide( @@ -166,7 +166,7 @@ object TestingUtils { x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) }, x, eps, REL_TOL_MSG) - override def toString = x.toString + override def toString: String = x.toString } case class CompareMatrixRightSide( @@ -229,7 +229,7 @@ object TestingUtils { x.toArray.zip(y.toArray).forall(x => x._1 ~= x._2 relTol eps) }, x, eps, REL_TOL_MSG) - override def toString = x.toString + override def toString: String = x.toString } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala index b0ecb33c28483..59e6c778806f4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/TestingUtilsSuite.scala @@ -88,16 +88,20 @@ class TestingUtilsSuite extends FunSuite { assert(!(17.8 ~= 17.59 absTol 0.2)) // Comparisons of numbers very close to zero, and both side of zeros - assert(Double.MinPositiveValue ~== 4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) - assert(Double.MinPositiveValue !~== 6 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) - - assert(-Double.MinPositiveValue ~== 3 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) - assert(Double.MinPositiveValue !~== -4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert( + Double.MinPositiveValue ~== 4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert( + Double.MinPositiveValue !~== 6 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + + assert( + -Double.MinPositiveValue ~== 3 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) + assert( + Double.MinPositiveValue !~== -4 * Double.MinPositiveValue absTol 5 * Double.MinPositiveValue) } test("Comparing vectors using relative error.") { - //Comparisons of two dense vectors + // Comparisons of two dense vectors assert(Vectors.dense(Array(3.1, 3.5)) ~== Vectors.dense(Array(3.130, 3.534)) relTol 0.01) assert(Vectors.dense(Array(3.1, 3.5)) !~== Vectors.dense(Array(3.135, 3.534)) relTol 0.01) assert(Vectors.dense(Array(3.1, 3.5)) ~= Vectors.dense(Array(3.130, 3.534)) relTol 0.01) @@ -130,7 +134,7 @@ class TestingUtilsSuite extends FunSuite { test("Comparing vectors using absolute error.") { - //Comparisons of two dense vectors + // Comparisons of two dense vectors assert(Vectors.dense(Array(3.1, 3.5, 0.0)) ~== Vectors.dense(Array(3.1 + 1E-8, 3.5 + 2E-7, 1E-8)) absTol 1E-6) From 6ada4f6f52cf1d992c7ab0c32318790cf08b0a0d Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 8 Apr 2015 13:31:45 -0700 Subject: [PATCH 005/128] [SPARK-6781] [SQL] use sqlContext in python shell Use `sqlContext` in PySpark shell, make it consistent with SQL programming guide. `sqlCtx` is also kept for compatibility. Author: Davies Liu Closes #5425 from davies/sqlCtx and squashes the following commits: af67340 [Davies Liu] sqlCtx -> sqlContext 15a278f [Davies Liu] use sqlContext in python shell --- docs/ml-guide.md | 2 +- docs/sql-programming-guide.md | 4 +- .../spark/examples/sql/JavaSparkSQL.java | 20 ++--- .../ml/simple_text_classification_pipeline.py | 2 +- .../src/main/python/mllib/dataset_example.py | 6 +- python/pyspark/ml/classification.py | 4 +- python/pyspark/ml/feature.py | 4 +- python/pyspark/shell.py | 6 +- python/pyspark/sql/context.py | 79 +++++++++---------- python/pyspark/sql/dataframe.py | 6 +- python/pyspark/sql/functions.py | 2 +- python/pyspark/sql/types.py | 4 +- 12 files changed, 69 insertions(+), 70 deletions(-) diff --git a/docs/ml-guide.md b/docs/ml-guide.md index c08c76d226713..771a07183e26f 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -493,7 +493,7 @@ from pyspark.ml.feature import HashingTF, Tokenizer from pyspark.sql import Row, SQLContext sc = SparkContext(appName="SimpleTextClassificationPipeline") -sqlCtx = SQLContext(sc) +sqlContext = SQLContext(sc) # Prepare training documents, which are labeled. LabeledDocument = Row("id", "text", "label") diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 4441d6a000a02..663f656883721 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1642,7 +1642,7 @@ moved into the udf object in `SQLContext`.
{% highlight java %} -sqlCtx.udf.register("strLen", (s: String) => s.length()) +sqlContext.udf.register("strLen", (s: String) => s.length()) {% endhighlight %}
@@ -1650,7 +1650,7 @@ sqlCtx.udf.register("strLen", (s: String) => s.length())
{% highlight java %} -sqlCtx.udf().register("strLen", (String s) -> { s.length(); }); +sqlContext.udf().register("strLen", (String s) -> { s.length(); }); {% endhighlight %}
diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index dee794840a3e1..8159ffbe2d269 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -55,7 +55,7 @@ public void setAge(int age) { public static void main(String[] args) throws Exception { SparkConf sparkConf = new SparkConf().setAppName("JavaSparkSQL"); JavaSparkContext ctx = new JavaSparkContext(sparkConf); - SQLContext sqlCtx = new SQLContext(ctx); + SQLContext sqlContext = new SQLContext(ctx); System.out.println("=== Data source: RDD ==="); // Load a text file and convert each line to a Java Bean. @@ -74,11 +74,11 @@ public Person call(String line) { }); // Apply a schema to an RDD of Java Beans and register it as a table. - DataFrame schemaPeople = sqlCtx.createDataFrame(people, Person.class); + DataFrame schemaPeople = sqlContext.createDataFrame(people, Person.class); schemaPeople.registerTempTable("people"); // SQL can be run over RDDs that have been registered as tables. - DataFrame teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + DataFrame teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // The results of SQL queries are DataFrames and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -99,12 +99,12 @@ public String call(Row row) { // Read in the parquet file created above. // Parquet files are self-describing so the schema is preserved. // The result of loading a parquet file is also a DataFrame. - DataFrame parquetFile = sqlCtx.parquetFile("people.parquet"); + DataFrame parquetFile = sqlContext.parquetFile("people.parquet"); //Parquet files can also be registered as tables and then used in SQL statements. parquetFile.registerTempTable("parquetFile"); DataFrame teenagers2 = - sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); + sqlContext.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); teenagerNames = teenagers2.toJavaRDD().map(new Function() { @Override public String call(Row row) { @@ -120,7 +120,7 @@ public String call(Row row) { // The path can be either a single text file or a directory storing text files. String path = "examples/src/main/resources/people.json"; // Create a DataFrame from the file(s) pointed by path - DataFrame peopleFromJsonFile = sqlCtx.jsonFile(path); + DataFrame peopleFromJsonFile = sqlContext.jsonFile(path); // Because the schema of a JSON dataset is automatically inferred, to write queries, // it is better to take a look at what is the schema. @@ -133,8 +133,8 @@ public String call(Row row) { // Register this DataFrame as a table. peopleFromJsonFile.registerTempTable("people"); - // SQL statements can be run by using the sql methods provided by sqlCtx. - DataFrame teenagers3 = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + // SQL statements can be run by using the sql methods provided by sqlContext. + DataFrame teenagers3 = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); // The results of SQL queries are DataFrame and support all the normal RDD operations. // The columns of a row in the result can be accessed by ordinal. @@ -151,7 +151,7 @@ public String call(Row row) { List jsonData = Arrays.asList( "{\"name\":\"Yin\",\"address\":{\"city\":\"Columbus\",\"state\":\"Ohio\"}}"); JavaRDD anotherPeopleRDD = ctx.parallelize(jsonData); - DataFrame peopleFromJsonRDD = sqlCtx.jsonRDD(anotherPeopleRDD.rdd()); + DataFrame peopleFromJsonRDD = sqlContext.jsonRDD(anotherPeopleRDD.rdd()); // Take a look at the schema of this new DataFrame. peopleFromJsonRDD.printSchema(); @@ -164,7 +164,7 @@ public String call(Row row) { peopleFromJsonRDD.registerTempTable("people2"); - DataFrame peopleWithCity = sqlCtx.sql("SELECT name, address.city FROM people2"); + DataFrame peopleWithCity = sqlContext.sql("SELECT name, address.city FROM people2"); List nameAndCity = peopleWithCity.toJavaRDD().map(new Function() { @Override public String call(Row row) { diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index d281f4fa44282..c73edb7fd6b20 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -33,7 +33,7 @@ if __name__ == "__main__": sc = SparkContext(appName="SimpleTextClassificationPipeline") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) # Prepare training documents, which are labeled. LabeledDocument = Row("id", "text", "label") diff --git a/examples/src/main/python/mllib/dataset_example.py b/examples/src/main/python/mllib/dataset_example.py index b5a70db2b9a3c..fcbf56cbf0c52 100644 --- a/examples/src/main/python/mllib/dataset_example.py +++ b/examples/src/main/python/mllib/dataset_example.py @@ -44,19 +44,19 @@ def summarize(dataset): print >> sys.stderr, "Usage: dataset_example.py " exit(-1) sc = SparkContext(appName="DatasetExample") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) if len(sys.argv) == 2: input = sys.argv[1] else: input = "data/mllib/sample_libsvm_data.txt" points = MLUtils.loadLibSVMFile(sc, input) - dataset0 = sqlCtx.inferSchema(points).setName("dataset0").cache() + dataset0 = sqlContext.inferSchema(points).setName("dataset0").cache() summarize(dataset0) tempdir = tempfile.NamedTemporaryFile(delete=False).name os.unlink(tempdir) print "Save dataset as a Parquet file to %s." % tempdir dataset0.saveAsParquetFile(tempdir) print "Load it back and summarize it again." - dataset1 = sqlCtx.parquetFile(tempdir).setName("dataset1").cache() + dataset1 = sqlContext.parquetFile(tempdir).setName("dataset1").cache() summarize(dataset1) shutil.rmtree(tempdir) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 4ff7463498cce..7f42de531f3b4 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -91,9 +91,9 @@ class LogisticRegressionModel(JavaModel): # The small batch size here ensures that we see multiple batches, # even in these small test examples: sc = SparkContext("local[2]", "ml.feature tests") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) globs['sc'] = sc - globs['sqlCtx'] = sqlCtx + globs['sqlContext'] = sqlContext (failure_count, test_count) = doctest.testmod( globs=globs, optionflags=doctest.ELLIPSIS) sc.stop() diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 433b4fb5d22bf..1cfcd019dfb18 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -117,9 +117,9 @@ def setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): # The small batch size here ensures that we see multiple batches, # even in these small test examples: sc = SparkContext("local[2]", "ml.feature tests") - sqlCtx = SQLContext(sc) + sqlContext = SQLContext(sc) globs['sc'] = sc - globs['sqlCtx'] = sqlCtx + globs['sqlContext'] = sqlContext (failure_count, test_count) = doctest.testmod( globs=globs, optionflags=doctest.ELLIPSIS) sc.stop() diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 1a02fece9c5a5..81aa970a32f76 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -53,9 +53,9 @@ try: # Try to access HiveConf, it will raise exception if Hive is not added sc._jvm.org.apache.hadoop.hive.conf.HiveConf() - sqlCtx = HiveContext(sc) + sqlCtx = sqlContext = HiveContext(sc) except py4j.protocol.Py4JError: - sqlCtx = SQLContext(sc) + sqlCtx = sqlContext = SQLContext(sc) print("""Welcome to ____ __ @@ -68,7 +68,7 @@ platform.python_version(), platform.python_build()[0], platform.python_build()[1])) -print("SparkContext available as sc, %s available as sqlCtx." % sqlCtx.__class__.__name__) +print("SparkContext available as sc, %s available as sqlContext." % sqlContext.__class__.__name__) if add_files is not None: print("Warning: ADD_FILES environment variable is deprecated, use --py-files argument instead") diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index c2d81ba804110..93e2d176a5b6f 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -37,12 +37,12 @@ __all__ = ["SQLContext", "HiveContext", "UDFRegistration"] -def _monkey_patch_RDD(sqlCtx): +def _monkey_patch_RDD(sqlContext): def toDF(self, schema=None, sampleRatio=None): """ Converts current :class:`RDD` into a :class:`DataFrame` - This is a shorthand for ``sqlCtx.createDataFrame(rdd, schema, sampleRatio)`` + This is a shorthand for ``sqlContext.createDataFrame(rdd, schema, sampleRatio)`` :param schema: a StructType or list of names of columns :param samplingRatio: the sample ratio of rows used for inferring @@ -51,7 +51,7 @@ def toDF(self, schema=None, sampleRatio=None): >>> rdd.toDF().collect() [Row(name=u'Alice', age=1)] """ - return sqlCtx.createDataFrame(self, schema, sampleRatio) + return sqlContext.createDataFrame(self, schema, sampleRatio) RDD.toDF = toDF @@ -75,13 +75,13 @@ def __init__(self, sparkContext, sqlContext=None): """Creates a new SQLContext. >>> from datetime import datetime - >>> sqlCtx = SQLContext(sc) + >>> sqlContext = SQLContext(sc) >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... time=datetime(2014, 8, 1, 14, 1, 5))]) >>> df = allTypes.toDF() >>> df.registerTempTable("allTypes") - >>> sqlCtx.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' + >>> sqlContext.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)] >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, @@ -133,18 +133,18 @@ def registerFunction(self, name, f, returnType=StringType()): :param samplingRatio: lambda function :param returnType: a :class:`DataType` object - >>> sqlCtx.registerFunction("stringLengthString", lambda x: len(x)) - >>> sqlCtx.sql("SELECT stringLengthString('test')").collect() + >>> sqlContext.registerFunction("stringLengthString", lambda x: len(x)) + >>> sqlContext.sql("SELECT stringLengthString('test')").collect() [Row(c0=u'4')] >>> from pyspark.sql.types import IntegerType - >>> sqlCtx.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) - >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() + >>> sqlContext.registerFunction("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlContext.sql("SELECT stringLengthInt('test')").collect() [Row(c0=4)] >>> from pyspark.sql.types import IntegerType - >>> sqlCtx.udf.register("stringLengthInt", lambda x: len(x), IntegerType()) - >>> sqlCtx.sql("SELECT stringLengthInt('test')").collect() + >>> sqlContext.udf.register("stringLengthInt", lambda x: len(x), IntegerType()) + >>> sqlContext.sql("SELECT stringLengthInt('test')").collect() [Row(c0=4)] """ func = lambda _, it: imap(lambda x: f(*x), it) @@ -229,26 +229,26 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): :param samplingRatio: the sample ratio of rows used for inferring >>> l = [('Alice', 1)] - >>> sqlCtx.createDataFrame(l).collect() + >>> sqlContext.createDataFrame(l).collect() [Row(_1=u'Alice', _2=1)] - >>> sqlCtx.createDataFrame(l, ['name', 'age']).collect() + >>> sqlContext.createDataFrame(l, ['name', 'age']).collect() [Row(name=u'Alice', age=1)] >>> d = [{'name': 'Alice', 'age': 1}] - >>> sqlCtx.createDataFrame(d).collect() + >>> sqlContext.createDataFrame(d).collect() [Row(age=1, name=u'Alice')] >>> rdd = sc.parallelize(l) - >>> sqlCtx.createDataFrame(rdd).collect() + >>> sqlContext.createDataFrame(rdd).collect() [Row(_1=u'Alice', _2=1)] - >>> df = sqlCtx.createDataFrame(rdd, ['name', 'age']) + >>> df = sqlContext.createDataFrame(rdd, ['name', 'age']) >>> df.collect() [Row(name=u'Alice', age=1)] >>> from pyspark.sql import Row >>> Person = Row('name', 'age') >>> person = rdd.map(lambda r: Person(*r)) - >>> df2 = sqlCtx.createDataFrame(person) + >>> df2 = sqlContext.createDataFrame(person) >>> df2.collect() [Row(name=u'Alice', age=1)] @@ -256,11 +256,11 @@ def createDataFrame(self, data, schema=None, samplingRatio=None): >>> schema = StructType([ ... StructField("name", StringType(), True), ... StructField("age", IntegerType(), True)]) - >>> df3 = sqlCtx.createDataFrame(rdd, schema) + >>> df3 = sqlContext.createDataFrame(rdd, schema) >>> df3.collect() [Row(name=u'Alice', age=1)] - >>> sqlCtx.createDataFrame(df.toPandas()).collect() # doctest: +SKIP + >>> sqlContext.createDataFrame(df.toPandas()).collect() # doctest: +SKIP [Row(name=u'Alice', age=1)] """ if isinstance(data, DataFrame): @@ -316,7 +316,7 @@ def registerDataFrameAsTable(self, df, tableName): Temporary tables exist only during the lifetime of this instance of :class:`SQLContext`. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") + >>> sqlContext.registerDataFrameAsTable(df, "table1") """ if (df.__class__ is DataFrame): self._ssql_ctx.registerDataFrameAsTable(df._jdf, tableName) @@ -330,7 +330,7 @@ def parquetFile(self, *paths): >>> parquetFile = tempfile.mkdtemp() >>> shutil.rmtree(parquetFile) >>> df.saveAsParquetFile(parquetFile) - >>> df2 = sqlCtx.parquetFile(parquetFile) + >>> df2 = sqlContext.parquetFile(parquetFile) >>> sorted(df.collect()) == sorted(df2.collect()) True """ @@ -352,7 +352,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): >>> shutil.rmtree(jsonFile) >>> with open(jsonFile, 'w') as f: ... f.writelines(jsonStrings) - >>> df1 = sqlCtx.jsonFile(jsonFile) + >>> df1 = sqlContext.jsonFile(jsonFile) >>> df1.printSchema() root |-- field1: long (nullable = true) @@ -365,7 +365,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): ... StructField("field2", StringType()), ... StructField("field3", ... StructType([StructField("field5", ArrayType(IntegerType()))]))]) - >>> df2 = sqlCtx.jsonFile(jsonFile, schema) + >>> df2 = sqlContext.jsonFile(jsonFile, schema) >>> df2.printSchema() root |-- field2: string (nullable = true) @@ -386,11 +386,11 @@ def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): If the schema is provided, applies the given schema to this JSON dataset. Otherwise, it samples the dataset with ratio ``samplingRatio`` to determine the schema. - >>> df1 = sqlCtx.jsonRDD(json) + >>> df1 = sqlContext.jsonRDD(json) >>> df1.first() Row(field1=1, field2=u'row1', field3=Row(field4=11, field5=None), field6=None) - >>> df2 = sqlCtx.jsonRDD(json, df1.schema) + >>> df2 = sqlContext.jsonRDD(json, df1.schema) >>> df2.first() Row(field1=1, field2=u'row1', field3=Row(field4=11, field5=None), field6=None) @@ -400,7 +400,7 @@ def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): ... StructField("field3", ... StructType([StructField("field5", ArrayType(IntegerType()))])) ... ]) - >>> df3 = sqlCtx.jsonRDD(json, schema) + >>> df3 = sqlContext.jsonRDD(json, schema) >>> df3.first() Row(field2=u'row1', field3=Row(field5=None)) """ @@ -480,8 +480,8 @@ def createExternalTable(self, tableName, path=None, source=None, def sql(self, sqlQuery): """Returns a :class:`DataFrame` representing the result of the given query. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> df2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> df2 = sqlContext.sql("SELECT field1 AS f1, field2 as f2 from table1") >>> df2.collect() [Row(f1=1, f2=u'row1'), Row(f1=2, f2=u'row2'), Row(f1=3, f2=u'row3')] """ @@ -490,8 +490,8 @@ def sql(self, sqlQuery): def table(self, tableName): """Returns the specified table as a :class:`DataFrame`. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> df2 = sqlCtx.table("table1") + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> df2 = sqlContext.table("table1") >>> sorted(df.collect()) == sorted(df2.collect()) True """ @@ -505,8 +505,8 @@ def tables(self, dbName=None): The returned DataFrame has two columns: ``tableName`` and ``isTemporary`` (a column with :class:`BooleanType` indicating if a table is a temporary one or not). - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> df2 = sqlCtx.tables() + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> df2 = sqlContext.tables() >>> df2.filter("tableName = 'table1'").first() Row(tableName=u'table1', isTemporary=True) """ @@ -520,10 +520,10 @@ def tableNames(self, dbName=None): If ``dbName`` is not specified, the current database will be used. - >>> sqlCtx.registerDataFrameAsTable(df, "table1") - >>> "table1" in sqlCtx.tableNames() + >>> sqlContext.registerDataFrameAsTable(df, "table1") + >>> "table1" in sqlContext.tableNames() True - >>> "table1" in sqlCtx.tableNames("db") + >>> "table1" in sqlContext.tableNames("db") True """ if dbName is None: @@ -578,11 +578,11 @@ def _get_hive_ctx(self): class UDFRegistration(object): """Wrapper for user-defined function registration.""" - def __init__(self, sqlCtx): - self.sqlCtx = sqlCtx + def __init__(self, sqlContext): + self.sqlContext = sqlContext def register(self, name, f, returnType=StringType()): - return self.sqlCtx.registerFunction(name, f, returnType) + return self.sqlContext.registerFunction(name, f, returnType) register.__doc__ = SQLContext.registerFunction.__doc__ @@ -595,13 +595,12 @@ def _test(): globs = pyspark.sql.context.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = sqlCtx = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['rdd'] = rdd = sc.parallelize( [Row(field1=1, field2="row1"), Row(field1=2, field2="row2"), Row(field1=3, field2="row3")] ) - _monkey_patch_RDD(sqlCtx) globs['df'] = rdd.toDF() jsonStrings = [ '{"field1": 1, "field2": "row1", "field3":{"field4":11}}', diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index c30326ebd133e..ef91a9c4f522d 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -110,7 +110,7 @@ def saveAsParquetFile(self, path): >>> parquetFile = tempfile.mkdtemp() >>> shutil.rmtree(parquetFile) >>> df.saveAsParquetFile(parquetFile) - >>> df2 = sqlCtx.parquetFile(parquetFile) + >>> df2 = sqlContext.parquetFile(parquetFile) >>> sorted(df2.collect()) == sorted(df.collect()) True """ @@ -123,7 +123,7 @@ def registerTempTable(self, name): that was used to create this :class:`DataFrame`. >>> df.registerTempTable("people") - >>> df2 = sqlCtx.sql("select * from people") + >>> df2 = sqlContext.sql("select * from people") >>> sorted(df.collect()) == sorted(df2.collect()) True """ @@ -1180,7 +1180,7 @@ def _test(): globs = pyspark.sql.dataframe.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['df'] = sc.parallelize([(2, 'Alice'), (5, 'Bob')])\ .toDF(StructType([StructField('age', IntegerType()), StructField('name', StringType())])) diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index 146ba6f3e0d98..daeb6916b58bc 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -161,7 +161,7 @@ def _test(): globs = pyspark.sql.functions.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['df'] = sc.parallelize([Row(name='Alice', age=2), Row(name='Bob', age=5)]).toDF() (failure_count, test_count) = doctest.testmod( pyspark.sql.functions, globs=globs, diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 45eb8b945dcb0..7e0124b13671b 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -434,7 +434,7 @@ def _parse_datatype_json_string(json_string): >>> def check_datatype(datatype): ... pickled = pickle.loads(pickle.dumps(datatype)) ... assert datatype == pickled - ... scala_datatype = sqlCtx._ssql_ctx.parseDataType(datatype.json()) + ... scala_datatype = sqlContext._ssql_ctx.parseDataType(datatype.json()) ... python_datatype = _parse_datatype_json_string(scala_datatype.json()) ... assert datatype == python_datatype >>> for cls in _all_primitive_types.values(): @@ -1237,7 +1237,7 @@ def _test(): globs = pyspark.sql.types.__dict__.copy() sc = SparkContext('local[4]', 'PythonTest') globs['sc'] = sc - globs['sqlCtx'] = sqlCtx = SQLContext(sc) + globs['sqlContext'] = SQLContext(sc) globs['ExamplePoint'] = ExamplePoint globs['ExamplePointUDT'] = ExamplePointUDT (failure_count, test_count) = doctest.testmod( From 2f482d706b9d38820472c3152dbd1612c98729bd Mon Sep 17 00:00:00 2001 From: Tijo Thomas Date: Wed, 8 Apr 2015 13:42:29 -0700 Subject: [PATCH 006/128] [SPARK-6767][SQL] Fixed Query DSL error in spark sql Readme Fixed the following error query.where('key > 30).select(avg('key)).collect() :43: error: value > is not a member of Symbol query.where('key > 30).select(avg('key)).collect() Author: Tijo Thomas Closes #5415 from tijoparacka/ERROR_SQL_DATAFRAME_EXAMPLE and squashes the following commits: 234751e [Tijo Thomas] Fixed Query DSL error in spark sql Readme --- sql/README.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/README.md b/sql/README.md index fbb3200a3a4b4..237620e3fa808 100644 --- a/sql/README.md +++ b/sql/README.md @@ -56,6 +56,6 @@ res2: Array[org.apache.spark.sql.Row] = Array([238,val_238], [86,val_86], [311,v You can also build further queries on top of these `DataFrames` using the query DSL. ``` -scala> query.where('key > 30).select(avg('key)).collect() +scala> query.where(query("key") > 30).select(avg(query("key"))).collect() res3: Array[org.apache.spark.sql.Row] = Array([274.79025423728814]) ``` From 86403f5525782bc9656ab11790f7020baa6b2c1f Mon Sep 17 00:00:00 2001 From: Michelangelo D'Agostino Date: Wed, 8 Apr 2015 16:48:45 -0400 Subject: [PATCH 007/128] [SPARK-5242]: Add --private-ips flag to EC2 script The `spark_ec2.py` script currently references the `ip_address` and `public_dns_name` attributes of an instance. On private networks, these fields aren't set, so we have problems. This PR introduces a `--private-ips` flag that instead refers to the `private_ip_address` attribute in both cases. Author: Michelangelo D'Agostino Closes #5244 from mdagost/ec2_private_nets and squashes the following commits: b684c67 [Michelangelo D'Agostino] STY: A few python lint changes. a4a2eac [Michelangelo D'Agostino] ENH: Fix IP's typo and refactor conditional logic into functions. c004604 [Michelangelo D'Agostino] ENH: Add --private-ips flag. --- ec2/spark_ec2.py | 64 +++++++++++++++++++++++++++++++++++------------- 1 file changed, 47 insertions(+), 17 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 879a52cef8ff0..0c1f24761d0de 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -282,6 +282,10 @@ def parse_args(): parser.add_option( "--vpc-id", default=None, help="VPC to launch instances in") + parser.add_option( + "--private-ips", action="store_true", default=False, + help="Use private IPs for instances rather than public if VPC/subnet " + + "requires that.") (opts, args) = parser.parse_args() if len(args) != 2: @@ -707,7 +711,7 @@ def get_instances(group_names): # Deploy configuration files and run setup scripts on a newly launched # or started EC2 cluster. def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): - master = master_nodes[0].public_dns_name + master = get_dns_name(master_nodes[0], opts.private_ips) if deploy_ssh_key: print "Generating cluster's SSH key on master..." key_setup = """ @@ -719,8 +723,9 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): dot_ssh_tar = ssh_read(master, opts, ['tar', 'c', '.ssh']) print "Transferring cluster's SSH key to slaves..." for slave in slave_nodes: - print slave.public_dns_name - ssh_write(slave.public_dns_name, opts, ['tar', 'x'], dot_ssh_tar) + slave_address = get_dns_name(slave, opts.private_ips) + print slave_address + ssh_write(slave_address, opts, ['tar', 'x'], dot_ssh_tar) modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs', 'mapreduce', 'spark-standalone', 'tachyon'] @@ -809,7 +814,8 @@ def is_cluster_ssh_available(cluster_instances, opts): Check if SSH is available on all the instances in a cluster. """ for i in cluster_instances: - if not is_ssh_available(host=i.public_dns_name, opts=opts): + dns_name = get_dns_name(i, opts.private_ips) + if not is_ssh_available(host=dns_name, opts=opts): return False else: return True @@ -923,7 +929,7 @@ def get_num_disks(instance_type): # # root_dir should be an absolute path to the directory with the files we want to deploy. def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): - active_master = master_nodes[0].public_dns_name + active_master = get_dns_name(master_nodes[0], opts.private_ips) num_disks = get_num_disks(opts.instance_type) hdfs_data_dirs = "/mnt/ephemeral-hdfs/data" @@ -948,10 +954,12 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): print "Deploying Spark via git hash; Tachyon won't be set up" modules = filter(lambda x: x != "tachyon", modules) + master_addresses = [get_dns_name(i, opts.private_ips) for i in master_nodes] + slave_addresses = [get_dns_name(i, opts.private_ips) for i in slave_nodes] template_vars = { - "master_list": '\n'.join([i.public_dns_name for i in master_nodes]), + "master_list": '\n'.join(master_addresses), "active_master": active_master, - "slave_list": '\n'.join([i.public_dns_name for i in slave_nodes]), + "slave_list": '\n'.join(slave_addresses), "cluster_url": cluster_url, "hdfs_data_dirs": hdfs_data_dirs, "mapred_local_dirs": mapred_local_dirs, @@ -1011,7 +1019,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): # # root_dir should be an absolute path. def deploy_user_files(root_dir, opts, master_nodes): - active_master = master_nodes[0].public_dns_name + active_master = get_dns_name(master_nodes[0], opts.private_ips) command = [ 'rsync', '-rv', '-e', stringify_command(ssh_command(opts)), @@ -1122,6 +1130,20 @@ def get_partition(total, num_partitions, current_partitions): return num_slaves_this_zone +# Gets the IP address, taking into account the --private-ips flag +def get_ip_address(instance, private_ips=False): + ip = instance.ip_address if not private_ips else \ + instance.private_ip_address + return ip + + +# Gets the DNS name, taking into account the --private-ips flag +def get_dns_name(instance, private_ips=False): + dns = instance.public_dns_name if not private_ips else \ + instance.private_ip_address + return dns + + def real_main(): (opts, action, cluster_name) = parse_args() @@ -1230,7 +1252,7 @@ def real_main(): if any(master_nodes + slave_nodes): print "The following instances will be terminated:" for inst in master_nodes + slave_nodes: - print "> %s" % inst.public_dns_name + print "> %s" % get_dns_name(inst, opts.private_ips) print "ALL DATA ON ALL NODES WILL BE LOST!!" msg = "Are you sure you want to destroy the cluster {c}? (y/N) ".format(c=cluster_name) @@ -1294,13 +1316,17 @@ def real_main(): elif action == "login": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - master = master_nodes[0].public_dns_name - print "Logging into master " + master + "..." - proxy_opt = [] - if opts.proxy_port is not None: - proxy_opt = ['-D', opts.proxy_port] - subprocess.check_call( - ssh_command(opts) + proxy_opt + ['-t', '-t', "%s@%s" % (opts.user, master)]) + if not master_nodes[0].public_dns_name and not opts.private_ips: + print "Master has no public DNS name. Maybe you meant to specify " \ + "--private-ips?" + else: + master = get_dns_name(master_nodes[0], opts.private_ips) + print "Logging into master " + master + "..." + proxy_opt = [] + if opts.proxy_port is not None: + proxy_opt = ['-D', opts.proxy_port] + subprocess.check_call( + ssh_command(opts) + proxy_opt + ['-t', '-t', "%s@%s" % (opts.user, master)]) elif action == "reboot-slaves": response = raw_input( @@ -1318,7 +1344,11 @@ def real_main(): elif action == "get-master": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - print master_nodes[0].public_dns_name + if not master_nodes[0].public_dns_name and not opts.private_ips: + print "Master has no public DNS name. Maybe you meant to specify " \ + "--private-ips?" + else: + print get_dns_name(master_nodes[0], opts.private_ips) elif action == "stop": response = raw_input( From 55a92ef34c0b57b6e379523d5d79baa05392de37 Mon Sep 17 00:00:00 2001 From: unknown Date: Wed, 8 Apr 2015 13:56:42 -0700 Subject: [PATCH 008/128] [SPARK-4346][SPARK-3596][YARN] Commonize the monitor logic 1. YarnClientSchedulerBack.asyncMonitorApplication use Client.monitorApplication so that commonize the monitor logic 2. Support changing the yarn client monitor interval, see #5292 3. More details see discussion on https://github.com/apache/spark/pull/3143 Author: unknown Author: Sephiroth-Lin Closes #5305 from Sephiroth-Lin/SPARK-4346_3596 and squashes the following commits: 47c0014 [unknown] Edit conflicts 52b29fe [unknown] Interrupt thread when we call stop() d4298a1 [unknown] Unused, don't push aaacb42 [Sephiroth-Lin] don't wrap the entire block in the try ee2b2fd [Sephiroth-Lin] update 6483a2a [unknown] Catch exception 6b47ff7 [unknown] Update code 568f46f [unknown] YarnClientSchedulerBack.asyncMonitorApplication should be common with Client.monitorApplication --- .../org/apache/spark/deploy/yarn/Client.scala | 10 +++++- .../cluster/YarnClientSchedulerBackend.scala | 32 ++++++------------- 2 files changed, 18 insertions(+), 24 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 79d55a09eb671..7219852c0a752 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -40,6 +40,7 @@ import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException import org.apache.hadoop.yarn.util.Records import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkException} @@ -561,7 +562,14 @@ private[spark] class Client( var lastState: YarnApplicationState = null while (true) { Thread.sleep(interval) - val report = getApplicationReport(appId) + val report: ApplicationReport = + try { + getApplicationReport(appId) + } catch { + case e: ApplicationNotFoundException => + logError(s"Application $appId not found.") + return (YarnApplicationState.KILLED, FinalApplicationStatus.KILLED) + } val state = report.getYarnApplicationState if (logApplicationReport) { diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 8abdc26b43806..407dc1ac4d37d 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -34,7 +34,7 @@ private[spark] class YarnClientSchedulerBackend( private var client: Client = null private var appId: ApplicationId = null - @volatile private var stopping: Boolean = false + private var monitorThread: Thread = null /** * Create a Yarn client to submit an application to the ResourceManager. @@ -57,7 +57,8 @@ private[spark] class YarnClientSchedulerBackend( client = new Client(args, conf) appId = client.submitApplication() waitForApplication() - asyncMonitorApplication() + monitorThread = asyncMonitorApplication() + monitorThread.start() } /** @@ -123,34 +124,19 @@ private[spark] class YarnClientSchedulerBackend( * If the application has exited for any reason, stop the SparkContext. * This assumes both `client` and `appId` have already been set. */ - private def asyncMonitorApplication(): Unit = { + private def asyncMonitorApplication(): Thread = { assert(client != null && appId != null, "Application has not been submitted yet!") val t = new Thread { override def run() { - while (!stopping) { - var state: YarnApplicationState = null - try { - val report = client.getApplicationReport(appId) - state = report.getYarnApplicationState() - } catch { - case e: ApplicationNotFoundException => - state = YarnApplicationState.KILLED - } - if (state == YarnApplicationState.FINISHED || - state == YarnApplicationState.KILLED || - state == YarnApplicationState.FAILED) { - logError(s"Yarn application has already exited with state $state!") - sc.stop() - stopping = true - } - Thread.sleep(1000L) - } + val (state, _) = client.monitorApplication(appId, logApplicationReport = false) + logError(s"Yarn application has already exited with state $state!") + sc.stop() Thread.currentThread().interrupt() } } t.setName("Yarn application state monitor") t.setDaemon(true) - t.start() + t } /** @@ -158,7 +144,7 @@ private[spark] class YarnClientSchedulerBackend( */ override def stop() { assert(client != null, "Attempted to stop this scheduler before starting it!") - stopping = true + monitorThread.interrupt() super.stop() client.stop() logInfo("Stopped") From 9418280547f962eaf309bfff9986cdd848409643 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 8 Apr 2015 13:57:01 -0700 Subject: [PATCH 009/128] [SQL][minor] remove duplicated resolveGetField and update comment It's after https://github.com/apache/spark/pull/5189 Author: Wenchen Fan Closes #5304 from cloud-fan/tmp and squashes the following commits: c58c9b3 [Wenchen Fan] remove duplicated code and update comment --- .../sql/catalyst/analysis/Analyzer.scala | 32 +------------------ .../catalyst/plans/logical/LogicalPlan.scala | 13 +++----- 2 files changed, 6 insertions(+), 39 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index 119cb9c3a4400..b3aba4f68ddf9 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 @@ -293,7 +293,7 @@ class Analyzer( logDebug(s"Resolving $u to $result") result case UnresolvedGetField(child, fieldName) if child.resolved => - resolveGetField(child, fieldName) + q.resolveGetField(child, fieldName, resolver) } } @@ -313,36 +313,6 @@ class Analyzer( */ protected def containsStar(exprs: Seq[Expression]): Boolean = exprs.exists(_.collect { case _: Star => true }.nonEmpty) - - /** - * Returns the resolved `GetField`, and report error if no desired field or over one - * desired fields are found. - */ - protected def resolveGetField(expr: Expression, fieldName: String): Expression = { - def findField(fields: Array[StructField]): Int = { - val checkField = (f: StructField) => resolver(f.name, fieldName) - val ordinal = fields.indexWhere(checkField) - if (ordinal == -1) { - throw new AnalysisException( - s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") - } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { - throw new AnalysisException( - s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") - } else { - ordinal - } - } - expr.dataType match { - case StructType(fields) => - val ordinal = findField(fields) - StructGetField(expr, fields(ordinal), ordinal) - case ArrayType(StructType(fields), containsNull) => - val ordinal = findField(fields) - ArrayGetField(expr, fields(ordinal), ordinal, containsNull) - case otherType => - throw new AnalysisException(s"GetField is not valid on fields of type $otherType") - } - } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 2e9f3aa4ec4ad..d8f5858f5033e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -205,11 +205,10 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // One match, but we also need to extract the requested nested field. case Seq((a, nestedFields)) => try { - - // The foldLeft adds UnresolvedGetField for every remaining parts of the name, - // and aliased it with the last part of the name. - // For example, consider name "a.b.c", where "a" is resolved to an existing attribute. - // Then this will add UnresolvedGetField("b") and UnresolvedGetField("c"), and alias + // The foldLeft adds GetFields for every remaining parts of the identifier, + // and aliases it with the last part of the identifier. + // For example, consider "a.b.c", where "a" is resolved to an existing attribute. + // Then this will add GetField("c", GetField("b", a)), and alias // the final expression as "c". val fieldExprs = nestedFields.foldLeft(a: Expression)(resolveGetField(_, _, resolver)) val aliasName = nestedFields.last @@ -234,10 +233,8 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { /** * Returns the resolved `GetField`, and report error if no desired field or over one * desired fields are found. - * - * TODO: this code is duplicated from Analyzer and should be refactored to avoid this. */ - protected def resolveGetField( + def resolveGetField( expr: Expression, fieldName: String, resolver: Resolver): Expression = { From 7d7384c781ea72e1eabab3daca2e237e3b0fc666 Mon Sep 17 00:00:00 2001 From: Venkata Ramana Gollamudi Date: Wed, 8 Apr 2015 18:42:34 -0700 Subject: [PATCH 010/128] [SPARK-6451][SQL] supported code generation for CombineSum Author: Venkata Ramana Gollamudi Closes #5138 from gvramana/sum_fix_codegen and squashes the following commits: 95f5fe4 [Venkata Ramana Gollamudi] rebase merge changes 12f45a5 [Venkata Ramana Gollamudi] Combined and added code generations tests as per comment d6a76ac [Venkata Ramana Gollamudi] added support for codegeneration for CombineSum and tests --- .../sql/execution/GeneratedAggregate.scala | 44 ++++++++- .../spark/sql/execution/SparkStrategies.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 92 ++++++++++++++++++- 3 files changed, 133 insertions(+), 5 deletions(-) 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 a8018b9213f2b..861a2c21ad9a0 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 @@ -99,7 +99,10 @@ case class GeneratedAggregate( // but really, common sub expression elimination would be better.... val zero = Cast(Literal(0), calcType) val updateFunction = Coalesce( - Add(Coalesce(currentSum :: zero :: Nil), Cast(expr, calcType)) :: currentSum :: Nil) + Add( + Coalesce(currentSum :: zero :: Nil), + Cast(expr, calcType) + ) :: currentSum :: zero :: Nil) val result = expr.dataType match { case DecimalType.Fixed(_, _) => @@ -109,6 +112,45 @@ case class GeneratedAggregate( AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result) + case cs @ CombineSum(expr) => + val calcType = expr.dataType + expr.dataType match { + case DecimalType.Fixed(_, _) => + DecimalType.Unlimited + case _ => + expr.dataType + } + + val currentSum = AttributeReference("currentSum", calcType, nullable = true)() + val initialValue = Literal.create(null, calcType) + + // Coalasce avoids double calculation... + // but really, common sub expression elimination would be better.... + val zero = Cast(Literal(0), calcType) + // If we're evaluating UnscaledValue(x), we can do Count on x directly, since its + // UnscaledValue will be null if and only if x is null; helps with Average on decimals + val actualExpr = expr match { + case UnscaledValue(e) => e + case _ => expr + } + // partial sum result can be null only when no input rows present + val updateFunction = If( + IsNotNull(actualExpr), + Coalesce( + Add( + Coalesce(currentSum :: zero :: Nil), + Cast(expr, calcType)) :: currentSum :: zero :: Nil), + currentSum) + + val result = + expr.dataType match { + case DecimalType.Fixed(_, _) => + Cast(currentSum, cs.dataType) + case _ => currentSum + } + + AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result) + case a @ Average(expr) => val calcType = expr.dataType match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index f754fa770d1b5..23f7e5609414b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -155,7 +155,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } def canBeCodeGened(aggs: Seq[AggregateExpression]): Boolean = !aggs.exists { - case _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false + case _: CombineSum | _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false // The generated set implementation is pretty limited ATM. case CollectHashSet(exprs) if exprs.size == 1 && Seq(IntegerType, LongType).contains(exprs.head.dataType) => false 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 87e7cf8c8af9f..1ad92a3941187 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql import org.apache.spark.sql.test.TestSQLContext import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.functions._ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan @@ -102,14 +103,99 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { sql("SELECT ABS(2.5)"), Row(2.5)) } - + test("aggregation with codegen") { val originalValue = conf.codegenEnabled setConf(SQLConf.CODEGEN_ENABLED, "true") - sql("SELECT key FROM testData GROUP BY key").collect() + // Prepare a table that we can group some rows. + table("testData") + .unionAll(table("testData")) + .unionAll(table("testData")) + .registerTempTable("testData3x") + + def testCodeGen(sqlText: String, expectedResults: Seq[Row]): Unit = { + val df = sql(sqlText) + // First, check if we have GeneratedAggregate. + var hasGeneratedAgg = false + df.queryExecution.executedPlan.foreach { + case generatedAgg: GeneratedAggregate => hasGeneratedAgg = true + case _ => + } + if (!hasGeneratedAgg) { + fail( + s""" + |Codegen is enabled, but query $sqlText does not have GeneratedAggregate in the plan. + |${df.queryExecution.simpleString} + """.stripMargin) + } + // Then, check results. + checkAnswer(df, expectedResults) + } + + // Just to group rows. + testCodeGen( + "SELECT key FROM testData3x GROUP BY key", + (1 to 100).map(Row(_))) + // COUNT + testCodeGen( + "SELECT key, count(value) FROM testData3x GROUP BY key", + (1 to 100).map(i => Row(i, 3))) + testCodeGen( + "SELECT count(key) FROM testData3x", + Row(300) :: Nil) + // COUNT DISTINCT ON int + testCodeGen( + "SELECT value, count(distinct key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, 1))) + testCodeGen( + "SELECT count(distinct key) FROM testData3x", + Row(100) :: Nil) + // SUM + testCodeGen( + "SELECT value, sum(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, 3 * i))) + testCodeGen( + "SELECT sum(key), SUM(CAST(key as Double)) FROM testData3x", + Row(5050 * 3, 5050 * 3.0) :: Nil) + // AVERAGE + testCodeGen( + "SELECT value, avg(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, i))) + testCodeGen( + "SELECT avg(key) FROM testData3x", + Row(50.5) :: Nil) + // MAX + testCodeGen( + "SELECT value, max(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, i))) + testCodeGen( + "SELECT max(key) FROM testData3x", + Row(100) :: Nil) + // Some combinations. + testCodeGen( + """ + |SELECT + | value, + | sum(key), + | max(key), + | avg(key), + | count(key), + | count(distinct key) + |FROM testData3x + |GROUP BY value + """.stripMargin, + (1 to 100).map(i => Row(i.toString, i*3, i, i, 3, 1))) + testCodeGen( + "SELECT max(key), avg(key), count(key), count(distinct key) FROM testData3x", + Row(100, 50.5, 300, 100) :: Nil) + // Aggregate with Code generation handling all null values + testCodeGen( + "SELECT sum('a'), avg('a'), count(null) FROM testData", + Row(0, null, 0) :: Nil) + + dropTempTable("testData3x") setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString) } - test("Add Parser of SQL COALESCE()") { checkAnswer( sql("""SELECT COALESCE(1, 2)"""), From 891ada5be1e7fdd796380e2626d80843f2ef6017 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 8 Apr 2015 18:47:39 -0700 Subject: [PATCH 011/128] [SPARK-6696] [SQL] Adds HiveContext.refreshTable to PySpark [Review on Reviewable](https://reviewable.io/reviews/apache/spark/5349) Author: Cheng Lian Closes #5349 from liancheng/py-refresh-table and squashes the following commits: 004bec0 [Cheng Lian] Adds HiveContext.refreshTable to PySpark --- python/pyspark/sql/context.py | 9 +++++++++ 1 file changed, 9 insertions(+) diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index 93e2d176a5b6f..e8529a8f8e3a4 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -574,6 +574,15 @@ def _ssql_ctx(self): def _get_hive_ctx(self): return self._jvm.HiveContext(self._jsc.sc()) + def refreshTable(self, tableName): + """Invalidate and refresh all the cached the metadata of the given + table. For performance reasons, Spark SQL or the external data source + library it uses might cache certain metadata about a table, such as the + location of blocks. When those change outside of Spark SQL, users should + call this function to invalidate the cache. + """ + self._ssql_ctx.refreshTable(tableName) + class UDFRegistration(object): """Wrapper for user-defined function registration.""" From 1b2aab8d5b9cc2ff702506038bd71aa8debe7ca0 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 8 Apr 2015 20:35:29 -0700 Subject: [PATCH 012/128] [SPARK-6765] Fix test code style for SQL So we can turn style checker on for test code. Author: Reynold Xin Closes #5412 from rxin/test-style-sql and squashes the following commits: 9098a31 [Reynold Xin] One more compilation error ... 8c7250a [Reynold Xin] Fix compilation. 82d0944 [Reynold Xin] Indentation. 0b03fbb [Reynold Xin] code review. f2f4348 [Reynold Xin] oops. ef4ec48 [Reynold Xin] Hive module. 7e0db5e [Reynold Xin] sql module 04ec7ac [Reynold Xin] catalyst module --- .../sql/catalyst/DistributionSuite.scala | 3 +- .../sql/catalyst/analysis/AnalysisSuite.scala | 10 +- .../analysis/HiveTypeCoercionSuite.scala | 8 +- .../ExpressionEvaluationSuite.scala | 134 ++++++++++++------ .../optimizer/ConstantFoldingSuite.scala | 51 ++++--- .../optimizer/FilterPushdownSuite.scala | 3 +- .../catalyst/optimizer/OptimizeInSuite.scala | 2 +- .../spark/sql/catalyst/plans/PlanTest.scala | 5 +- .../sql/catalyst/plans/SameResultSuite.scala | 2 +- .../sql/catalyst/trees/TreeNodeSuite.scala | 8 +- .../apache/spark/sql/CachedTableSuite.scala | 3 +- .../org/apache/spark/sql/DataFrameSuite.scala | 3 +- .../org/apache/spark/sql/QueryTest.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 30 ++-- .../sql/ScalaReflectionRelationSuite.scala | 5 +- .../spark/sql/UserDefinedTypeSuite.scala | 2 +- .../sql/columnar/ColumnarTestUtils.scala | 4 +- .../NullableColumnAccessorSuite.scala | 3 +- .../columnar/NullableColumnBuilderSuite.scala | 3 +- .../TestCompressibleColumnBuilder.scala | 2 +- .../sql/execution/debug/DebuggingSuite.scala | 2 +- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 98 ++++++------- .../org/apache/spark/sql/json/JsonSuite.scala | 17 ++- .../spark/sql/parquet/ParquetIOSuite.scala | 2 +- .../sql/parquet/ParquetSchemaSuite.scala | 2 + .../spark/sql/sources/DDLTestSuite.scala | 8 +- .../spark/sql/sources/FilteredScanSuite.scala | 3 +- .../spark/sql/sources/PrunedScanSuite.scala | 5 +- .../spark/sql/sources/SaveLoadSuite.scala | 2 +- .../spark/sql/sources/TableScanSuite.scala | 9 +- .../spark/sql/hive/ErrorPositionSuite.scala | 2 +- .../spark/sql/hive/HiveInspectorSuite.scala | 33 +++-- .../sql/hive/InsertIntoHiveTableSuite.scala | 57 ++++++-- .../spark/sql/hive/StatisticsSuite.scala | 2 +- .../execution/BigDataBenchmarkSuite.scala | 12 +- .../hive/execution/HiveComparisonTest.scala | 27 ++-- .../hive/execution/HiveQueryFileTest.scala | 11 +- .../sql/hive/execution/HiveQuerySuite.scala | 13 +- .../hive/execution/HiveResolutionSuite.scala | 3 +- .../sql/hive/execution/HiveSerDeSuite.scala | 3 +- .../execution/HiveTypeCoercionSuite.scala | 6 +- .../sql/hive/execution/HiveUdfSuite.scala | 16 ++- .../sql/hive/execution/PruningSuite.scala | 2 +- .../sql/hive/execution/SQLQuerySuite.scala | 4 +- .../apache/spark/sql/hive/parquetSuites.scala | 7 +- 45 files changed, 395 insertions(+), 234 deletions(-) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala index 46b2250aab231..ea82cd2622de9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -30,7 +30,7 @@ class DistributionSuite extends FunSuite { inputPartitioning: Partitioning, requiredDistribution: Distribution, satisfied: Boolean) { - if (inputPartitioning.satisfies(requiredDistribution) != satisfied) + if (inputPartitioning.satisfies(requiredDistribution) != satisfied) { fail( s""" |== Input Partitioning == @@ -40,6 +40,7 @@ class DistributionSuite extends FunSuite { |== Does input partitioning satisfy required distribution? == |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} """.stripMargin) + } } test("HashPartitioning is the output partitioning") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index ee7b14c7a157c..6e3d6b9263e86 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -27,6 +27,8 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ +import scala.collection.immutable + class AnalysisSuite extends FunSuite with BeforeAndAfter { val caseSensitiveCatalog = new SimpleCatalog(true) val caseInsensitiveCatalog = new SimpleCatalog(false) @@ -41,10 +43,10 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { } - def caseSensitiveAnalyze(plan: LogicalPlan) = + def caseSensitiveAnalyze(plan: LogicalPlan): Unit = caseSensitiveAnalyzer.checkAnalysis(caseSensitiveAnalyzer(plan)) - def caseInsensitiveAnalyze(plan: LogicalPlan) = + def caseInsensitiveAnalyze(plan: LogicalPlan): Unit = caseInsensitiveAnalyzer.checkAnalysis(caseInsensitiveAnalyzer(plan)) val testRelation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)()) @@ -147,7 +149,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { name: String, plan: LogicalPlan, errorMessages: Seq[String], - caseSensitive: Boolean = true) = { + caseSensitive: Boolean = true): Unit = { test(name) { val error = intercept[AnalysisException] { if(caseSensitive) { @@ -202,7 +204,7 @@ class AnalysisSuite extends FunSuite with BeforeAndAfter { case class UnresolvedTestPlan() extends LeafNode { override lazy val resolved = false - override def output = Nil + override def output: Seq[Attribute] = Nil } errorTest( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index 70aef1cac421a..fcd745f43cfbf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -96,7 +96,9 @@ class HiveTypeCoercionSuite extends PlanTest { widenTest(StringType, TimestampType, None) // ComplexType - widenTest(NullType, MapType(IntegerType, StringType, false), Some(MapType(IntegerType, StringType, false))) + widenTest(NullType, + MapType(IntegerType, StringType, false), + Some(MapType(IntegerType, StringType, false))) widenTest(NullType, StructType(Seq()), Some(StructType(Seq()))) widenTest(StringType, MapType(IntegerType, StringType, true), None) widenTest(ArrayType(IntegerType), StructType(Seq()), None) @@ -113,7 +115,9 @@ class HiveTypeCoercionSuite extends PlanTest { // Remove superflous boolean -> boolean casts. ruleTest(Cast(Literal(true), BooleanType), Literal(true)) // Stringify boolean when casting to string. - ruleTest(Cast(Literal(false), StringType), If(Literal(false), Literal("true"), Literal("false"))) + ruleTest( + Cast(Literal(false), StringType), + If(Literal(false), Literal("true"), Literal("false"))) } test("coalesce casts") { 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 3dbefa40d2808..d2b1090a0cdd5 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 @@ -82,10 +82,13 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { assert(BitwiseNot(1.toByte).eval(EmptyRow).isInstanceOf[Byte]) } + // scalastyle:off /** * Checks for three-valued-logic. Based on: * http://en.wikipedia.org/wiki/Null_(SQL)#Comparisons_with_NULL_and_the_three-valued_logic_.283VL.29 - * I.e. in flat cpo "False -> Unknown -> True", OR is lowest upper bound, AND is greatest lower bound. + * I.e. in flat cpo "False -> Unknown -> True", + * OR is lowest upper bound, + * AND is greatest lower bound. * p q p OR q p AND q p = q * True True True True True * True False True False False @@ -102,7 +105,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { * False True * Unknown Unknown */ - + // scalastyle:on val notTrueTable = (true, false) :: (false, true) :: @@ -165,7 +168,9 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false) - checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), true) + checkEvaluation( + In(Literal(1), Seq(Literal(1), Literal(2))) && In(Literal(2), Seq(Literal(1), Literal(2))), + true) } test("Divide") { @@ -180,7 +185,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(0)), null) checkEvaluation(Divide(Literal.create(null, DoubleType), Literal(0.0)), null) checkEvaluation(Divide(Literal.create(null, IntegerType), Literal(1)), null) - checkEvaluation(Divide(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null) + checkEvaluation(Divide(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), + null) } test("Remainder") { @@ -195,7 +201,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(0)), null) checkEvaluation(Remainder(Literal.create(null, DoubleType), Literal(0.0)), null) checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal(1)), null) - checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null) + checkEvaluation(Remainder(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), + null) } test("INSET") { @@ -264,7 +271,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation("ab" like regEx, true, new GenericRow(Array[Any]("a%b"))) checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a%b"))) - checkEvaluation(Literal.create(null, StringType) like regEx, null, new GenericRow(Array[Any]("bc%"))) + checkEvaluation(Literal.create(null, StringType) like regEx, null, + new GenericRow(Array[Any]("bc%"))) } test("RLIKE literal Regular Expression") { @@ -507,8 +515,10 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("array casting") { - val array = Literal.create(Seq("123", "abc", "", null), ArrayType(StringType, containsNull = true)) - val array_notNull = Literal.create(Seq("123", "abc", ""), ArrayType(StringType, containsNull = false)) + val array = Literal.create(Seq("123", "abc", "", null), + ArrayType(StringType, containsNull = true)) + val array_notNull = Literal.create(Seq("123", "abc", ""), + ArrayType(StringType, containsNull = false)) { val cast = Cast(array, ArrayType(IntegerType, containsNull = true)) @@ -765,7 +775,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Coalesce(Literal.create(null, StringType) :: Nil), null, row) checkEvaluation(Coalesce(Literal.create(null, StringType) :: c1 :: c2 :: Nil), "^Ba*n", row) - checkEvaluation(If(c3, Literal.create("a", StringType), Literal.create("b", StringType)), "a", row) + checkEvaluation( + If(c3, Literal.create("a", StringType), Literal.create("b", StringType)), "a", row) checkEvaluation(If(c3, c1, c2), "^Ba*n", row) checkEvaluation(If(c4, c2, c1), "^Ba*n", row) checkEvaluation(If(Literal.create(null, BooleanType), c2, c1), "^Ba*n", row) @@ -842,18 +853,20 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(GetItem(BoundReference(3, typeMap, true), Literal("aa")), "bb", row) checkEvaluation(GetItem(Literal.create(null, typeMap), Literal("aa")), null, row) - checkEvaluation(GetItem(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row) + checkEvaluation( + GetItem(Literal.create(null, typeMap), Literal.create(null, StringType)), null, row) checkEvaluation(GetItem(BoundReference(3, typeMap, true), Literal.create(null, StringType)), null, row) checkEvaluation(GetItem(BoundReference(4, typeArray, true), Literal(1)), "bb", row) checkEvaluation(GetItem(Literal.create(null, typeArray), Literal(1)), null, row) - checkEvaluation(GetItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + GetItem(Literal.create(null, typeArray), Literal.create(null, IntegerType)), null, row) checkEvaluation(GetItem(BoundReference(4, typeArray, true), Literal.create(null, IntegerType)), null, row) - def quickBuildGetField(expr: Expression, fieldName: String) = { + def quickBuildGetField(expr: Expression, fieldName: String): StructGetField = { expr.dataType match { case StructType(fields) => val field = fields.find(_.name == fieldName).get @@ -861,7 +874,9 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } } - def quickResolve(u: UnresolvedGetField) = quickBuildGetField(u.child, u.fieldName) + def quickResolve(u: UnresolvedGetField): StructGetField = { + quickBuildGetField(u.child, u.fieldName) + } checkEvaluation(quickBuildGetField(BoundReference(2, typeS, nullable = true), "a"), "aa", row) checkEvaluation(quickBuildGetField(Literal.create(null, typeS), "a"), null, row) @@ -872,7 +887,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { ) assert(quickBuildGetField(BoundReference(2,typeS, nullable = true), "a").nullable === true) - assert(quickBuildGetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable === false) + assert(quickBuildGetField(BoundReference(2, typeS_notNullable, nullable = false), "a").nullable + === false) assert(quickBuildGetField(Literal.create(null, typeS), "a").nullable === true) assert(quickBuildGetField(Literal.create(null, typeS_notNullable), "a").nullable === true) @@ -896,7 +912,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Add(c1, c2), 3, row) checkEvaluation(Add(c1, Literal.create(null, IntegerType)), null, row) checkEvaluation(Add(Literal.create(null, IntegerType), c2), null, row) - checkEvaluation(Add(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + Add(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(-c1, -1, row) checkEvaluation(c1 + c2, 3, row) @@ -919,7 +936,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(Add(c1, c2), 3.1, row) checkEvaluation(Add(c1, Literal.create(null, DoubleType)), null, row) checkEvaluation(Add(Literal.create(null, DoubleType), c2), null, row) - checkEvaluation(Add(Literal.create(null, DoubleType), Literal.create(null, DoubleType)), null, row) + checkEvaluation( + Add(Literal.create(null, DoubleType), Literal.create(null, DoubleType)), null, row) checkEvaluation(-c1, -1.1, row) checkEvaluation(c1 + c2, 3.1, row) @@ -942,7 +960,8 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(LessThan(c1, c2), true, row) checkEvaluation(LessThan(c1, Literal.create(null, IntegerType)), null, row) checkEvaluation(LessThan(Literal.create(null, IntegerType), c2), null, row) - checkEvaluation(LessThan(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + LessThan(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(c1 < c2, true, row) checkEvaluation(c1 <= c2, true, row) @@ -985,54 +1004,84 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { val s = 'a.string.at(0) // substring from zero position with less-than-full length - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)), "ex", row) - checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(2, IntegerType)), "ex", row) + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)), "ex", row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(2, IntegerType)), "ex", row) // substring from zero position with full length - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(7, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(7, IntegerType)), "example", row) + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(7, IntegerType)), "example", row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(7, IntegerType)), "example", row) // substring from zero position with greater-than-full length - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(100, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(100, IntegerType)), "example", row) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(100, IntegerType)), + "example", row) + checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(100, IntegerType)), + "example", row) // substring from nonzero position with less-than-full length - checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(2, IntegerType)), "xa", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(2, IntegerType)), + "xa", row) // substring from nonzero position with full length - checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(6, IntegerType)), "xample", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(6, IntegerType)), + "xample", row) // substring from nonzero position with greater-than-full length - checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(100, IntegerType)), "xample", row) + checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(100, IntegerType)), + "xample", row) // zero-length substring (within string bounds) - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(0, IntegerType)), "", row) + checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(0, IntegerType)), + "", row) // zero-length substring (beyond string bounds) - checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), "", row) + checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), + "", row) // substring(null, _, _) -> null - checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), null, new GenericRow(Array[Any](null))) + checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), + null, new GenericRow(Array[Any](null))) // substring(_, null, _) -> null - checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)), null, row) + checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)), + null, row) // substring(_, _, null) -> null - checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + Substring(s, Literal.create(100, IntegerType), Literal.create(null, IntegerType)), + null, + row) // 2-arg substring from zero position - checkEvaluation(Substring(s, Literal.create(0, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), "example", row) - checkEvaluation(Substring(s, Literal.create(1, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), "example", row) + checkEvaluation( + Substring(s, Literal.create(0, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "example", + row) + checkEvaluation( + Substring(s, Literal.create(1, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "example", + row) // 2-arg substring from nonzero position - checkEvaluation(Substring(s, Literal.create(2, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), "xample", row) + checkEvaluation( + Substring(s, Literal.create(2, IntegerType), Literal.create(Integer.MAX_VALUE, IntegerType)), + "xample", + row) val s_notNull = 'a.string.notNull.at(0) - assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable === true) - assert(Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable === false) - assert(Substring(s_notNull, Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable === true) - assert(Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable === true) + assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable + === true) + assert( + Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable + === false) + assert(Substring(s_notNull, + Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable === true) + assert(Substring(s_notNull, + Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable === true) checkEvaluation(s.substr(0, 2), "ex", row) checkEvaluation(s.substr(0), "example", row) @@ -1065,17 +1114,20 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(BitwiseAnd(c1, c4), null, row) checkEvaluation(BitwiseAnd(c1, c2), 0, row) checkEvaluation(BitwiseAnd(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation(BitwiseAnd(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseAnd(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseOr(c1, c4), null, row) checkEvaluation(BitwiseOr(c1, c2), 3, row) checkEvaluation(BitwiseOr(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation(BitwiseOr(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseOr(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseXor(c1, c4), null, row) checkEvaluation(BitwiseXor(c1, c2), 3, row) checkEvaluation(BitwiseXor(c1, Literal.create(null, IntegerType)), null, row) - checkEvaluation(BitwiseXor(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) + checkEvaluation( + BitwiseXor(Literal.create(null, IntegerType), Literal.create(null, IntegerType)), null, row) checkEvaluation(BitwiseNot(c4), null, row) checkEvaluation(BitwiseNot(c1), -2, row) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala index a0efe9e2e7f6b..4396bd0dda9a9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -176,40 +176,39 @@ class ConstantFoldingSuite extends PlanTest { } test("Constant folding test: expressions have null literals") { - val originalQuery = - testRelation - .select( - IsNull(Literal(null)) as 'c1, - IsNotNull(Literal(null)) as 'c2, + val originalQuery = testRelation.select( + IsNull(Literal(null)) as 'c1, + IsNotNull(Literal(null)) as 'c2, - GetItem(Literal.create(null, ArrayType(IntegerType)), 1) as 'c3, - GetItem(Literal.create(Seq(1), ArrayType(IntegerType)), Literal.create(null, IntegerType)) as 'c4, - UnresolvedGetField( - Literal.create(null, StructType(Seq(StructField("a", IntegerType, true)))), - "a") as 'c5, + GetItem(Literal.create(null, ArrayType(IntegerType)), 1) as 'c3, + GetItem( + Literal.create(Seq(1), ArrayType(IntegerType)), Literal.create(null, IntegerType)) as 'c4, + UnresolvedGetField( + Literal.create(null, StructType(Seq(StructField("a", IntegerType, true)))), + "a") as 'c5, - UnaryMinus(Literal.create(null, IntegerType)) as 'c6, - Cast(Literal(null), IntegerType) as 'c7, - Not(Literal.create(null, BooleanType)) as 'c8, + UnaryMinus(Literal.create(null, IntegerType)) as 'c6, + Cast(Literal(null), IntegerType) as 'c7, + Not(Literal.create(null, BooleanType)) as 'c8, - Add(Literal.create(null, IntegerType), 1) as 'c9, - Add(1, Literal.create(null, IntegerType)) as 'c10, + Add(Literal.create(null, IntegerType), 1) as 'c9, + Add(1, Literal.create(null, IntegerType)) as 'c10, - EqualTo(Literal.create(null, IntegerType), 1) as 'c11, - EqualTo(1, Literal.create(null, IntegerType)) as 'c12, + EqualTo(Literal.create(null, IntegerType), 1) as 'c11, + EqualTo(1, Literal.create(null, IntegerType)) as 'c12, - Like(Literal.create(null, StringType), "abc") as 'c13, - Like("abc", Literal.create(null, StringType)) as 'c14, + Like(Literal.create(null, StringType), "abc") as 'c13, + Like("abc", Literal.create(null, StringType)) as 'c14, - Upper(Literal.create(null, StringType)) as 'c15, + Upper(Literal.create(null, StringType)) as 'c15, - Substring(Literal.create(null, StringType), 0, 1) as 'c16, - Substring("abc", Literal.create(null, IntegerType), 1) as 'c17, - Substring("abc", 0, Literal.create(null, IntegerType)) as 'c18, + Substring(Literal.create(null, StringType), 0, 1) as 'c16, + Substring("abc", Literal.create(null, IntegerType), 1) as 'c17, + Substring("abc", 0, Literal.create(null, IntegerType)) as 'c18, - Contains(Literal.create(null, StringType), "abc") as 'c19, - Contains("abc", Literal.create(null, StringType)) as 'c20 - ) + Contains(Literal.create(null, StringType), "abc") as 'c19, + Contains("abc", Literal.create(null, StringType)) as 'c20 + ) val optimized = Optimize(originalQuery.analyze) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 55c6766520a1e..1448098c770aa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -432,7 +432,8 @@ class FilterPushdownSuite extends PlanTest { val originalQuery = { z.join(x.join(y)) - .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && ("z.a".attr >= 3) && ("z.a".attr === "x.b".attr)) + .where(("x.b".attr === "y.b".attr) && ("x.a".attr === 1) && + ("z.a".attr >= 3) && ("z.a".attr === "x.b".attr)) } val optimized = Optimize(originalQuery.analyze) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala index 233e329cb2038..966bc9ada1e6e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala @@ -52,7 +52,7 @@ class OptimizeInSuite extends PlanTest { val optimized = Optimize(originalQuery.analyze) val correctAnswer = testRelation - .where(InSet(UnresolvedAttribute("a"), HashSet[Any]()+1+2)) + .where(InSet(UnresolvedAttribute("a"), HashSet[Any]() + 1 + 2)) .analyze comparePlans(optimized, correctAnswer) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala index 129d091ca03e3..e7cafcc96de87 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala @@ -45,12 +45,13 @@ class PlanTest extends FunSuite { protected def comparePlans(plan1: LogicalPlan, plan2: LogicalPlan) { val normalized1 = normalizeExprIds(plan1) val normalized2 = normalizeExprIds(plan2) - if (normalized1 != normalized2) + if (normalized1 != normalized2) { fail( s""" |== FAIL: Plans do not match === |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} - """.stripMargin) + """.stripMargin) + } } /** Fails the test if the two expressions do not match */ diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala index 11e6831b24768..1273921f6394c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala @@ -32,7 +32,7 @@ class SameResultSuite extends FunSuite { val testRelation = LocalRelation('a.int, 'b.int, 'c.int) val testRelation2 = LocalRelation('a.int, 'b.int, 'c.int) - def assertSameResult(a: LogicalPlan, b: LogicalPlan, result: Boolean = true) = { + def assertSameResult(a: LogicalPlan, b: LogicalPlan, result: Boolean = true): Unit = { val aAnalyzed = a.analyze val bAnalyzed = b.analyze diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 274f3ede0045c..4eb8708335dcf 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -25,12 +25,12 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.{StringType, NullType} case class Dummy(optKey: Option[Expression]) extends Expression { - def children = optKey.toSeq - def nullable = true - def dataType = NullType + def children: Seq[Expression] = optKey.toSeq + def nullable: Boolean = true + def dataType: NullType = NullType override lazy val resolved = true type EvaluatedType = Any - def eval(input: Row) = null.asInstanceOf[Any] + def eval(input: Row): Any = null.asInstanceOf[Any] } class TreeNodeSuite extends FunSuite { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index c240f2be955ca..f7b5f08beb92f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -92,7 +92,8 @@ class CachedTableSuite extends QueryTest { test("too big for memory") { val data = "*" * 10000 - sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF().registerTempTable("bigData") + sparkContext.parallelize(1 to 200000, 1).map(_ => BigData(data)).toDF() + .registerTempTable("bigData") table("bigData").persist(StorageLevel.MEMORY_AND_DISK) assert(table("bigData").count() === 200000L) table("bigData").unpersist(blocking = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 1db0cf7daac03..f5df8c6a59f10 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -329,8 +329,9 @@ class DataFrameSuite extends QueryTest { checkAnswer( decimalData.agg(avg('a cast DecimalType(10, 2))), Row(new java.math.BigDecimal(2.0))) + // non-partial checkAnswer( - decimalData.agg(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), // non-partial + decimalData.agg(avg('a cast DecimalType(10, 2)), sumDistinct('a cast DecimalType(10, 2))), Row(new java.math.BigDecimal(2.0), new java.math.BigDecimal(6)) :: Nil) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 9b4dd6c620fec..9a81fc5d72819 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -67,7 +67,7 @@ class QueryTest extends PlanTest { checkAnswer(df, Seq(expectedAnswer)) } - def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext): Unit = { + def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext) { test(sqlString) { checkAnswer(sqlContext.sql(sqlString), expectedAnswer) } 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 1ad92a3941187..1392b4819131b 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 @@ -268,7 +268,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.002"))) checkAnswer(sql( - "SELECT time FROM timestamps WHERE time IN ('1969-12-31 16:00:00.001','1969-12-31 16:00:00.002')"), + """ + |SELECT time FROM timestamps + |WHERE time IN ('1969-12-31 16:00:00.001','1969-12-31 16:00:00.002') + """.stripMargin), Seq(Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001")), Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.002")))) @@ -334,7 +337,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row("1")) } - def sortTest() = { + def sortTest(): Unit = { checkAnswer( sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC"), Seq(Row(1,1), Row(1,2), Row(2,1), Row(2,2), Row(3,1), Row(3,2))) @@ -413,7 +416,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { test("from follow multiple brackets") { checkAnswer(sql( - "select key from ((select * from testData limit 1) union all (select * from testData limit 1)) x limit 1"), + """ + |select key from ((select * from testData limit 1) + | union all (select * from testData limit 1)) x limit 1 + """.stripMargin), Row(1) ) @@ -423,7 +429,11 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { ) checkAnswer(sql( - "select key from (select * from testData limit 1 union all select * from testData limit 1) x limit 1"), + """ + |select key from + | (select * from testData limit 1 union all select * from testData limit 1) x + | limit 1 + """.stripMargin), Row(1) ) } @@ -470,7 +480,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Seq(Row(1, 0), Row(2, 1))) checkAnswer( - sql("SELECT COUNT(a), COUNT(b), COUNT(1), COUNT(DISTINCT a), COUNT(DISTINCT b) FROM testData3"), + sql( + """ + |SELECT COUNT(a), COUNT(b), COUNT(1), COUNT(DISTINCT a), COUNT(DISTINCT b) FROM testData3 + """.stripMargin), Row(2, 1, 2, 2, 1)) } @@ -1083,7 +1096,8 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-3483 Special chars in column names") { - val data = sparkContext.parallelize(Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) + val data = sparkContext.parallelize( + Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) jsonRDD(data).registerTempTable("records") sql("SELECT `key?number1` FROM records") } @@ -1168,8 +1182,8 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } test("SPARK-6145: ORDER BY test for nested fields") { - jsonRDD(sparkContext.makeRDD( - """{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)).registerTempTable("nestedOrder") + jsonRDD(sparkContext.makeRDD("""{"a": {"b": 1, "a": {"a": 1}}, "c": [{"d": 1}]}""" :: Nil)) + .registerTempTable("nestedOrder") checkAnswer(sql("SELECT 1 FROM nestedOrder ORDER BY a.b"), Row(1)) checkAnswer(sql("SELECT a.b FROM nestedOrder ORDER BY a.b"), Row(1)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 17e923ca48502..3fa00fd9d0ccb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -80,7 +80,7 @@ class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, - new java.math.BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3)) + new java.math.BigDecimal(1), new Date(12345), new Timestamp(12345), Seq(1,2,3)) val rdd = sparkContext.parallelize(data :: Nil) rdd.toDF().registerTempTable("reflectData") @@ -103,7 +103,8 @@ class ScalaReflectionRelationSuite extends FunSuite { val rdd = sparkContext.parallelize(data :: Nil) rdd.toDF().registerTempTable("reflectOptionalData") - assert(sql("SELECT * FROM reflectOptionalData").collect().head === Row.fromSeq(Seq.fill(7)(null))) + assert(sql("SELECT * FROM reflectOptionalData").collect().head === + Row.fromSeq(Seq.fill(7)(null))) } // Equality is broken for Arrays, so we test that separately. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index fe618e0e8e767..902da5c3ba6d6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -63,7 +63,7 @@ private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { } } - override def userClass = classOf[MyDenseVector] + override def userClass: Class[MyDenseVector] = classOf[MyDenseVector] private[spark] override def asNullable: MyDenseVectorUDT = this } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index c7a40845db16c..b301818a008e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.types.{Decimal, DataType, NativeType} object ColumnarTestUtils { - def makeNullRow(length: Int) = { + def makeNullRow(length: Int): GenericMutableRow = { val row = new GenericMutableRow(length) (0 until length).foreach(row.setNullAt) row @@ -93,7 +93,7 @@ object ColumnarTestUtils { def makeUniqueValuesAndSingleValueRows[T <: NativeType]( columnType: NativeColumnType[T], - count: Int) = { + count: Int): (Seq[T#JvmType], Seq[GenericMutableRow]) = { val values = makeUniqueRandomValues(columnType, count) val rows = values.map { value => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index bb305355276bf..a0702144f942c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -31,7 +31,8 @@ class TestNullableColumnAccessor[T <: DataType, JvmType]( with NullableColumnAccessor object TestNullableColumnAccessor { - def apply[T <: DataType, JvmType](buffer: ByteBuffer, columnType: ColumnType[T, JvmType]) = { + def apply[T <: DataType, JvmType](buffer: ByteBuffer, columnType: ColumnType[T, JvmType]) + : TestNullableColumnAccessor[T, JvmType] = { // Skips the column type ID buffer.getInt() new TestNullableColumnAccessor(buffer, columnType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index 75a47498683f4..3a5605d2335d7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -27,7 +27,8 @@ class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T with NullableColumnBuilder object TestNullableColumnBuilder { - def apply[T <: DataType, JvmType](columnType: ColumnType[T, JvmType], initialSize: Int = 0) = { + def apply[T <: DataType, JvmType](columnType: ColumnType[T, JvmType], initialSize: Int = 0) + : TestNullableColumnBuilder[T, JvmType] = { val builder = new TestNullableColumnBuilder(columnType) builder.initialize(initialSize) builder diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index 0b18b4119268f..fc8ff3b41d0e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -35,7 +35,7 @@ object TestCompressibleColumnBuilder { def apply[T <: NativeType]( columnStats: ColumnStats, columnType: NativeColumnType[T], - scheme: CompressionScheme) = { + scheme: CompressionScheme): TestCompressibleColumnBuilder[T] = { val builder = new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme)) builder.initialize(0, "", useCompression = true) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala index 4e9472c60249e..358d8cf06e463 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/debug/DebuggingSuite.scala @@ -30,4 +30,4 @@ class DebuggingSuite extends FunSuite { test("DataFrame.typeCheck()") { testData.typeCheck() } -} \ No newline at end of file +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 592ed4b23b7d3..3596b183d4328 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -45,10 +45,12 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { conn = DriverManager.getConnection(url, properties) conn.prepareStatement("create schema test").executeUpdate() - conn.prepareStatement("create table test.people (name TEXT(32) NOT NULL, theid INTEGER NOT NULL)").executeUpdate() + conn.prepareStatement( + "create table test.people (name TEXT(32) NOT NULL, theid INTEGER NOT NULL)").executeUpdate() conn.prepareStatement("insert into test.people values ('fred', 1)").executeUpdate() conn.prepareStatement("insert into test.people values ('mary', 2)").executeUpdate() - conn.prepareStatement("insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate() + conn.prepareStatement( + "insert into test.people values ('joe ''foo'' \"bar\"', 3)").executeUpdate() conn.commit() sql( @@ -132,25 +134,25 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { } test("SELECT *") { - assert(sql("SELECT * FROM foobar").collect().size == 3) + assert(sql("SELECT * FROM foobar").collect().size === 3) } test("SELECT * WHERE (simple predicates)") { - assert(sql("SELECT * FROM foobar WHERE THEID < 1").collect().size == 0) - assert(sql("SELECT * FROM foobar WHERE THEID != 2").collect().size == 2) - assert(sql("SELECT * FROM foobar WHERE THEID = 1").collect().size == 1) - assert(sql("SELECT * FROM foobar WHERE NAME = 'fred'").collect().size == 1) - assert(sql("SELECT * FROM foobar WHERE NAME > 'fred'").collect().size == 2) - assert(sql("SELECT * FROM foobar WHERE NAME != 'fred'").collect().size == 2) + assert(sql("SELECT * FROM foobar WHERE THEID < 1").collect().size === 0) + assert(sql("SELECT * FROM foobar WHERE THEID != 2").collect().size === 2) + assert(sql("SELECT * FROM foobar WHERE THEID = 1").collect().size === 1) + assert(sql("SELECT * FROM foobar WHERE NAME = 'fred'").collect().size === 1) + assert(sql("SELECT * FROM foobar WHERE NAME > 'fred'").collect().size === 2) + assert(sql("SELECT * FROM foobar WHERE NAME != 'fred'").collect().size === 2) } test("SELECT * WHERE (quoted strings)") { - assert(sql("select * from foobar").where('NAME === "joe 'foo' \"bar\"").collect().size == 1) + assert(sql("select * from foobar").where('NAME === "joe 'foo' \"bar\"").collect().size === 1) } test("SELECT first field") { val names = sql("SELECT NAME FROM foobar").collect().map(x => x.getString(0)).sortWith(_ < _) - assert(names.size == 3) + assert(names.size === 3) assert(names(0).equals("fred")) assert(names(1).equals("joe 'foo' \"bar\"")) assert(names(2).equals("mary")) @@ -158,10 +160,10 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { test("SELECT second field") { val ids = sql("SELECT THEID FROM foobar").collect().map(x => x.getInt(0)).sortWith(_ < _) - assert(ids.size == 3) - assert(ids(0) == 1) - assert(ids(1) == 2) - assert(ids(2) == 3) + assert(ids.size === 3) + assert(ids(0) === 1) + assert(ids(1) === 2) + assert(ids(2) === 3) } test("SELECT * partitioned") { @@ -169,46 +171,46 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { } test("SELECT WHERE (simple predicates) partitioned") { - assert(sql("SELECT * FROM parts WHERE THEID < 1").collect().size == 0) - assert(sql("SELECT * FROM parts WHERE THEID != 2").collect().size == 2) - assert(sql("SELECT THEID FROM parts WHERE THEID = 1").collect().size == 1) + assert(sql("SELECT * FROM parts WHERE THEID < 1").collect().size === 0) + assert(sql("SELECT * FROM parts WHERE THEID != 2").collect().size === 2) + assert(sql("SELECT THEID FROM parts WHERE THEID = 1").collect().size === 1) } test("SELECT second field partitioned") { val ids = sql("SELECT THEID FROM parts").collect().map(x => x.getInt(0)).sortWith(_ < _) - assert(ids.size == 3) - assert(ids(0) == 1) - assert(ids(1) == 2) - assert(ids(2) == 3) + assert(ids.size === 3) + assert(ids(0) === 1) + assert(ids(1) === 2) + assert(ids(2) === 3) } test("Basic API") { - assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE").collect.size == 3) + assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE").collect().size === 3) } test("Partitioning via JDBCPartitioningInfo API") { assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", "THEID", 0, 4, 3) - .collect.size == 3) + .collect.size === 3) } test("Partitioning via list-of-where-clauses API") { val parts = Array[String]("THEID < 2", "THEID >= 2") - assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts).collect.size == 3) + assert(TestSQLContext.jdbc(urlWithUserAndPass, "TEST.PEOPLE", parts).collect().size === 3) } test("H2 integral types") { val rows = sql("SELECT * FROM inttypes WHERE A IS NOT NULL").collect() - assert(rows.size == 1) - assert(rows(0).getInt(0) == 1) - assert(rows(0).getBoolean(1) == false) - assert(rows(0).getInt(2) == 3) - assert(rows(0).getInt(3) == 4) - assert(rows(0).getLong(4) == 1234567890123L) + assert(rows.size === 1) + assert(rows(0).getInt(0) === 1) + assert(rows(0).getBoolean(1) === false) + assert(rows(0).getInt(2) === 3) + assert(rows(0).getInt(3) === 4) + assert(rows(0).getLong(4) === 1234567890123L) } test("H2 null entries") { val rows = sql("SELECT * FROM inttypes WHERE A IS NULL").collect() - assert(rows.size == 1) + assert(rows.size === 1) assert(rows(0).isNullAt(0)) assert(rows(0).isNullAt(1)) assert(rows(0).isNullAt(2)) @@ -230,27 +232,27 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { val rows = sql("SELECT * FROM timetypes").collect() val cal = new GregorianCalendar(java.util.Locale.ROOT) cal.setTime(rows(0).getAs[java.sql.Timestamp](0)) - assert(cal.get(Calendar.HOUR_OF_DAY) == 12) - assert(cal.get(Calendar.MINUTE) == 34) - assert(cal.get(Calendar.SECOND) == 56) + assert(cal.get(Calendar.HOUR_OF_DAY) === 12) + assert(cal.get(Calendar.MINUTE) === 34) + assert(cal.get(Calendar.SECOND) === 56) cal.setTime(rows(0).getAs[java.sql.Timestamp](1)) - assert(cal.get(Calendar.YEAR) == 1996) - assert(cal.get(Calendar.MONTH) == 0) - assert(cal.get(Calendar.DAY_OF_MONTH) == 1) + assert(cal.get(Calendar.YEAR) === 1996) + assert(cal.get(Calendar.MONTH) === 0) + assert(cal.get(Calendar.DAY_OF_MONTH) === 1) cal.setTime(rows(0).getAs[java.sql.Timestamp](2)) - assert(cal.get(Calendar.YEAR) == 2002) - assert(cal.get(Calendar.MONTH) == 1) - assert(cal.get(Calendar.DAY_OF_MONTH) == 20) - assert(cal.get(Calendar.HOUR) == 11) - assert(cal.get(Calendar.MINUTE) == 22) - assert(cal.get(Calendar.SECOND) == 33) - assert(rows(0).getAs[java.sql.Timestamp](2).getNanos == 543543543) + assert(cal.get(Calendar.YEAR) === 2002) + assert(cal.get(Calendar.MONTH) === 1) + assert(cal.get(Calendar.DAY_OF_MONTH) === 20) + assert(cal.get(Calendar.HOUR) === 11) + assert(cal.get(Calendar.MINUTE) === 22) + assert(cal.get(Calendar.SECOND) === 33) + assert(rows(0).getAs[java.sql.Timestamp](2).getNanos === 543543543) } test("H2 floating-point types") { val rows = sql("SELECT * FROM flttypes").collect() - assert(rows(0).getDouble(0) == 1.00000000000000022) // Yes, I meant ==. - assert(rows(0).getDouble(1) == 1.00000011920928955) // Yes, I meant ==. + assert(rows(0).getDouble(0) === 1.00000000000000022) // Yes, I meant ==. + assert(rows(0).getDouble(1) === 1.00000011920928955) // Yes, I meant ==. assert(rows(0).getAs[BigDecimal](2) .equals(new BigDecimal("123456789012345.54321543215432100000"))) } @@ -264,7 +266,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { | user 'testUser', password 'testPass') """.stripMargin.replaceAll("\n", " ")) val rows = sql("SELECT * FROM hack").collect() - assert(rows(0).getDouble(0) == 1.00000011920928955) // Yes, I meant ==. + assert(rows(0).getDouble(0) === 1.00000011920928955) // Yes, I meant ==. // For some reason, H2 computes this square incorrectly... assert(math.abs(rows(0).getDouble(1) - 1.00000023841859331) < 1e-12) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 706c966ee05f5..1fe0b76c00be3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -380,8 +380,10 @@ class JsonSuite extends QueryTest { sql("select * from jsonTable"), Row("true", 11L, null, 1.1, "13.1", "str1") :: Row("12", null, new java.math.BigDecimal("21474836470.9"), null, null, "true") :: - Row("false", 21474836470L, new java.math.BigDecimal("92233720368547758070"), 100, "str1", "false") :: - Row(null, 21474836570L, new java.math.BigDecimal("1.1"), 21474836470L, "92233720368547758070", null) :: Nil + Row("false", 21474836470L, + new java.math.BigDecimal("92233720368547758070"), 100, "str1", "false") :: + Row(null, 21474836570L, + new java.math.BigDecimal("1.1"), 21474836470L, "92233720368547758070", null) :: Nil ) // Number and Boolean conflict: resolve the type as number in this query. @@ -404,7 +406,8 @@ class JsonSuite extends QueryTest { // Widening to DecimalType checkAnswer( sql("select num_num_2 + 1.2 from jsonTable where num_num_2 > 1.1"), - Row(new java.math.BigDecimal("21474836472.1")) :: Row(new java.math.BigDecimal("92233720368547758071.2")) :: Nil + Row(new java.math.BigDecimal("21474836472.1")) :: + Row(new java.math.BigDecimal("92233720368547758071.2")) :: Nil ) // Widening to DoubleType @@ -913,8 +916,10 @@ class JsonSuite extends QueryTest { df1.registerTempTable("applySchema1") val df2 = df1.toDF val result = df2.toJSON.collect() + // scalastyle:off assert(result(0) === "{\"f1\":1,\"f2\":\"A1\",\"f3\":true,\"f4\":[\"1\",\" A1\",\" true\",\" null\"]}") assert(result(3) === "{\"f1\":4,\"f2\":\"D4\",\"f3\":true,\"f4\":[\"4\",\" D4\",\" true\",\" 2147483644\"],\"f5\":2147483644}") + // scalastyle:on val schema2 = StructType( StructField("f1", StructType( @@ -968,7 +973,8 @@ class JsonSuite extends QueryTest { // Access elements of a BigInteger array (we use DecimalType internally). checkAnswer( - sql("select arrayOfBigInteger[0], arrayOfBigInteger[1], arrayOfBigInteger[2] from complexTable"), + sql("select arrayOfBigInteger[0], arrayOfBigInteger[1], arrayOfBigInteger[2] " + + " from complexTable"), Row(new java.math.BigDecimal("922337203685477580700"), new java.math.BigDecimal("-922337203685477580800"), null) ) @@ -1008,7 +1014,8 @@ class JsonSuite extends QueryTest { // Access elements of an array field of a struct. checkAnswer( - sql("select structWithArrayFields.field1[1], structWithArrayFields.field2[3] from complexTable"), + sql("select structWithArrayFields.field1[1], structWithArrayFields.field2[3] " + + "from complexTable"), Row(5, null) ) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 203bc79f153dd..97c0f439acf13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -218,7 +218,7 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } test("compression codec") { - def compressionCodecFor(path: String) = { + def compressionCodecFor(path: String): String = { val codecs = ParquetTypesConverter .readMetaData(new Path(path), Some(configuration)) .getBlocks diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala index 61f1cf347ab0f..c964b6d984557 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetSchemaSuite.scala @@ -180,10 +180,12 @@ class ParquetSchemaSuite extends FunSuite with ParquetTest { val caseClassString = "StructType(List(StructField(c1,IntegerType,false), StructField(c2,BinaryType,true)))" + // scalastyle:off val jsonString = """ |{"type":"struct","fields":[{"name":"c1","type":"integer","nullable":false,"metadata":{}},{"name":"c2","type":"binary","nullable":true,"metadata":{}}]} """.stripMargin + // scalastyle:on val fromCaseClassString = ParquetTypesConverter.convertFromString(caseClassString) val fromJson = ParquetTypesConverter.convertFromString(jsonString) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala index 54af50c6e10ad..3f24a497390c1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.sources +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -31,7 +32,7 @@ class DDLScanSource extends RelationProvider { case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) extends BaseRelation with TableScan { - override def schema = + override def schema: StructType = StructType(Seq( StructField("intType", IntegerType, nullable = false, new MetadataBuilder().putString("comment", "test comment").build()), @@ -57,8 +58,9 @@ case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLConte )) - override def buildScan() = sqlContext.sparkContext.parallelize(from to to). - map(e => Row(s"people$e", e * 2)) + override def buildScan(): RDD[Row] = { + sqlContext.sparkContext.parallelize(from to to).map(e => Row(s"people$e", e * 2)) + } } class DDLTestSuite extends DataSourceTest { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index 773bd1602d5e5..cb5e5147ff189 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import scala.language.existentials +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -41,7 +42,7 @@ case class SimpleFilteredScan(from: Int, to: Int)(@transient val sqlContext: SQL StructField("b", IntegerType, nullable = false) :: StructField("c", StringType, nullable = false) :: Nil) - override def buildScan(requiredColumns: Array[String], filters: Array[Filter]) = { + override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { val rowBuilders = requiredColumns.map { case "a" => (i: Int) => Seq(i) case "b" => (i: Int) => Seq(i * 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index 08fb5380dc026..6a1ddf2f8e98b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import scala.language.existentials +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -34,12 +35,12 @@ case class SimplePrunedScan(from: Int, to: Int)(@transient val sqlContext: SQLCo extends BaseRelation with PrunedScan { - override def schema = + override def schema: StructType = StructType( StructField("a", IntegerType, nullable = false) :: StructField("b", IntegerType, nullable = false) :: Nil) - override def buildScan(requiredColumns: Array[String]) = { + override def buildScan(requiredColumns: Array[String]): RDD[Row] = { val rowBuilders = requiredColumns.map { case "a" => (i: Int) => Seq(i) case "b" => (i: Int) => Seq(i * 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index 43bc8eb2d11a7..cb287ba85c1f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -114,4 +114,4 @@ class SaveLoadSuite extends DataSourceTest with BeforeAndAfterAll { message.contains("Append mode is not supported"), "We should complain that 'Append mode is not supported' for JSON source.") } -} \ No newline at end of file +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 7928600ac2fb5..60c8c00bda4d5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.sources import java.sql.{Timestamp, Date} +import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.types._ @@ -35,10 +36,10 @@ class SimpleScanSource extends RelationProvider { case class SimpleScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) extends BaseRelation with TableScan { - override def schema = + override def schema: StructType = StructType(StructField("i", IntegerType, nullable = false) :: Nil) - override def buildScan() = sqlContext.sparkContext.parallelize(from to to).map(Row(_)) + override def buildScan(): RDD[Row] = sqlContext.sparkContext.parallelize(from to to).map(Row(_)) } class AllDataTypesScanSource extends SchemaRelationProvider { @@ -57,9 +58,9 @@ case class AllDataTypesScan( extends BaseRelation with TableScan { - override def schema = userSpecifiedSchema + override def schema: StructType = userSpecifiedSchema - override def buildScan() = { + override def buildScan(): RDD[Row] = { sqlContext.sparkContext.parallelize(from to to).map { i => Row( s"str_$i", diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala index 968557c9c4686..d960a30e00738 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ErrorPositionSuite.scala @@ -136,7 +136,7 @@ class ErrorPositionSuite extends QueryTest with BeforeAndAfter { * @param query the query to analyze * @param token a unique token in the string that should be indicated by the exception */ - def positionTest(name: String, query: String, token: String) = { + def positionTest(name: String, query: String, token: String): Unit = { def parseTree = Try(quietly(HiveQl.dumpTree(HiveQl.getAst(query)))).getOrElse("") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala index c482c6de8a736..2a7374cc172b7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveInspectorSuite.scala @@ -116,21 +116,20 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { } def checkDataType(dt1: Seq[DataType], dt2: Seq[DataType]): Unit = { - dt1.zip(dt2).map { - case (dd1, dd2) => - assert(dd1.getClass === dd2.getClass) // DecimalType doesn't has the default precision info + dt1.zip(dt2).foreach { case (dd1, dd2) => + assert(dd1.getClass === dd2.getClass) // DecimalType doesn't has the default precision info } } def checkValues(row1: Seq[Any], row2: Seq[Any]): Unit = { - row1.zip(row2).map { - case (r1, r2) => checkValue(r1, r2) + row1.zip(row2).foreach { case (r1, r2) => + checkValue(r1, r2) } } def checkValues(row1: Seq[Any], row2: Row): Unit = { - row1.zip(row2.toSeq).map { - case (r1, r2) => checkValue(r1, r2) + row1.zip(row2.toSeq).foreach { case (r1, r2) => + checkValue(r1, r2) } } @@ -141,7 +140,7 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { assert(r1.compare(r2) === 0) case (r1: Array[Byte], r2: Array[Byte]) if r1 != null && r2 != null && r1.length == r2.length => - r1.zip(r2).map { case (b1, b2) => assert(b1 === b2) } + r1.zip(r2).foreach { case (b1, b2) => assert(b1 === b2) } case (r1, r2) => assert(r1 === r2) } } @@ -166,7 +165,8 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val constantData = constantExprs.map(_.eval()) val constantNullData = constantData.map(_ => null) val constantWritableOIs = constantExprs.map(e => toWritableInspector(e.dataType)) - val constantNullWritableOIs = constantExprs.map(e => toInspector(Literal.create(null, e.dataType))) + val constantNullWritableOIs = + constantExprs.map(e => toInspector(Literal.create(null, e.dataType))) checkValues(constantData, constantData.zip(constantWritableOIs).map { case (d, oi) => unwrap(wrap(d, oi), oi) @@ -202,7 +202,8 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { case (t, idx) => StructField(s"c_$idx", t) }) - checkValues(row, unwrap(wrap(Row.fromSeq(row), toInspector(dt)), toInspector(dt)).asInstanceOf[Row]) + checkValues(row, + unwrap(wrap(Row.fromSeq(row), toInspector(dt)), toInspector(dt)).asInstanceOf[Row]) checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) } @@ -212,8 +213,10 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val d = row(0) :: row(0) :: Nil checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValue(d, unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) - checkValue(d, unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, + unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, + unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) } test("wrap / unwrap Map Type") { @@ -222,7 +225,9 @@ class HiveInspectorSuite extends FunSuite with HiveInspectors { val d = Map(row(0) -> row(1)) checkValue(d, unwrap(wrap(d, toInspector(dt)), toInspector(dt))) checkValue(null, unwrap(wrap(null, toInspector(dt)), toInspector(dt))) - checkValue(d, unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) - checkValue(d, unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, + unwrap(wrap(d, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) + checkValue(d, + unwrap(wrap(null, toInspector(Literal.create(d, dt))), toInspector(Literal.create(d, dt)))) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index 8011952e0d535..ecb990e8aac91 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -115,11 +115,36 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { test("SPARK-4203:random partition directory order") { sql("CREATE TABLE tmp_table (key int, value string)") val tmpDir = Utils.createTempDir() - sql(s"CREATE TABLE table_with_partition(c1 string) PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) location '${tmpDir.toURI.toString}' ") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='1') SELECT 'blarr' FROM tmp_table") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='2') SELECT 'blarr' FROM tmp_table") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='3') SELECT 'blarr' FROM tmp_table") - sql("INSERT OVERWRITE TABLE table_with_partition partition (p1='a',p2='b',p3='c',p4='c',p5='4') SELECT 'blarr' FROM tmp_table") + sql( + s""" + |CREATE TABLE table_with_partition(c1 string) + |PARTITIONED by (p1 string,p2 string,p3 string,p4 string,p5 string) + |location '${tmpDir.toURI.toString}' + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='1') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='2') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='3') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (p1='a',p2='b',p3='c',p4='c',p5='4') + |SELECT 'blarr' FROM tmp_table + """.stripMargin) def listFolders(path: File, acc: List[String]): List[List[String]] = { val dir = path.listFiles() val folders = dir.filter(_.isDirectory).toList @@ -196,34 +221,42 @@ class InsertIntoHiveTableSuite extends QueryTest with BeforeAndAfter { testData.registerTempTable("testData") val testDatawithNull = TestHive.sparkContext.parallelize( - (1 to 10).map(i => ThreeCloumntable(i, i.toString,null))).toDF() + (1 to 10).map(i => ThreeCloumntable(i, i.toString, null))).toDF() val tmpDir = Utils.createTempDir() - sql(s"CREATE TABLE table_with_partition(key int,value string) PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') SELECT key,value FROM testData") + sql( + s""" + |CREATE TABLE table_with_partition(key int,value string) + |PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' + """.stripMargin) + sql( + """ + |INSERT OVERWRITE TABLE table_with_partition + |partition (ds='1') SELECT key,value FROM testData + """.stripMargin) // test schema the same between partition and table sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT") checkAnswer(sql("select key,value from table_with_partition where ds='1' "), - testData.collect.toSeq + testData.collect().toSeq ) // test difference type of field sql("ALTER TABLE table_with_partition CHANGE COLUMN key key BIGINT") checkAnswer(sql("select key,value from table_with_partition where ds='1' "), - testData.collect.toSeq + testData.collect().toSeq ) // add column to table sql("ALTER TABLE table_with_partition ADD COLUMNS(key1 string)") checkAnswer(sql("select key,value,key1 from table_with_partition where ds='1' "), - testDatawithNull.collect.toSeq + testDatawithNull.collect().toSeq ) // change column name to table sql("ALTER TABLE table_with_partition CHANGE COLUMN key keynew BIGINT") checkAnswer(sql("select keynew,value from table_with_partition where ds='1' "), - testData.collect.toSeq + testData.collect().toSeq ) sql("DROP TABLE table_with_partition") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala index ccd0e5aa51f95..00a69de9e4262 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/StatisticsSuite.scala @@ -142,7 +142,7 @@ class StatisticsSuite extends QueryTest with BeforeAndAfterAll { after: () => Unit, query: String, expectedAnswer: Seq[Row], - ct: ClassTag[_]) = { + ct: ClassTag[_]): Unit = { before() var df = sql(query) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index 42a82c1fbf5c7..a3f5921a0cb23 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.hive.test.TestHive._ class BigDataBenchmarkSuite extends HiveComparisonTest { val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") + val userVisitPath = new File(testDataDirectory, "uservisits").getCanonicalPath val testTables = Seq( TestTable( "rankings", @@ -63,7 +64,7 @@ class BigDataBenchmarkSuite extends HiveComparisonTest { | searchWord STRING, | duration INT) | ROW FORMAT DELIMITED FIELDS TERMINATED BY "," - | STORED AS TEXTFILE LOCATION "${new File(testDataDirectory, "uservisits").getCanonicalPath}" + | STORED AS TEXTFILE LOCATION "$userVisitPath" """.stripMargin.cmd), TestTable( "documents", @@ -83,7 +84,10 @@ class BigDataBenchmarkSuite extends HiveComparisonTest { "SELECT pageURL, pageRank FROM rankings WHERE pageRank > 1") createQueryTest("query2", - "SELECT SUBSTR(sourceIP, 1, 10), SUM(adRevenue) FROM uservisits GROUP BY SUBSTR(sourceIP, 1, 10)") + """ + |SELECT SUBSTR(sourceIP, 1, 10), SUM(adRevenue) FROM uservisits + |GROUP BY SUBSTR(sourceIP, 1, 10) + """.stripMargin) createQueryTest("query3", """ @@ -113,8 +117,8 @@ class BigDataBenchmarkSuite extends HiveComparisonTest { |CREATE TABLE url_counts_total AS | SELECT SUM(count) AS totalCount, destpage | FROM url_counts_partial GROUP BY destpage - |-- The following queries run, but generate different results in HIVE likely because the UDF is not deterministic - |-- given different input splits. + |-- The following queries run, but generate different results in HIVE + |-- likely because the UDF is not deterministic given different input splits. |-- SELECT CAST(SUM(count) AS INT) FROM url_counts_partial |-- SELECT COUNT(*) FROM url_counts_partial |-- SELECT * FROM url_counts_partial diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index a5ec312ee430c..027056d4b865f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -255,8 +255,9 @@ abstract class HiveComparisonTest .filterNot(_ contains "hive.outerjoin.supports.filters") .filterNot(_ contains "hive.exec.post.hooks") - if (allQueries != queryList) + if (allQueries != queryList) { logWarning(s"Simplifications made on unsupported operations for test $testCaseName") + } lazy val consoleTestCase = { val quotes = "\"\"\"" @@ -305,13 +306,16 @@ abstract class HiveComparisonTest try { // Hooks often break the harness and don't really affect our test anyway, don't // even try running them. - if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) + if (installHooksCommand.findAllMatchIn(queryString).nonEmpty) { sys.error("hive exec hooks not supported for tests.") + } - logWarning(s"Running query ${i+1}/${queryList.size} with hive.") + logWarning(s"Running query ${i + 1}/${queryList.size} with hive.") // Analyze the query with catalyst to ensure test tables are loaded. val answer = hiveQuery.analyzed match { - case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. + case _: ExplainCommand => + // No need to execute EXPLAIN queries as we don't check the output. + Nil case _ => TestHive.runSqlHive(queryString) } @@ -394,21 +398,24 @@ abstract class HiveComparisonTest case tf: org.scalatest.exceptions.TestFailedException => throw tf case originalException: Exception => if (System.getProperty("spark.hive.canarytest") != null) { - // When we encounter an error we check to see if the environment is still okay by running a simple query. - // If this fails then we halt testing since something must have gone seriously wrong. + // When we encounter an error we check to see if the environment is still + // okay by running a simple query. If this fails then we halt testing since + // something must have gone seriously wrong. try { new TestHive.HiveQLQueryExecution("SELECT key FROM src").stringResult() TestHive.runSqlHive("SELECT key FROM src") } catch { case e: Exception => - logError(s"FATAL ERROR: Canary query threw $e This implies that the testing environment has likely been corrupted.") - // The testing setup traps exits so wait here for a long time so the developer can see when things started - // to go wrong. + logError(s"FATAL ERROR: Canary query threw $e This implies that the " + + "testing environment has likely been corrupted.") + // The testing setup traps exits so wait here for a long time so the developer + // can see when things started to go wrong. Thread.sleep(1000000) } } - // If the canary query didn't fail then the environment is still okay, so just throw the original exception. + // If the canary query didn't fail then the environment is still okay, + // so just throw the original exception. throw originalException } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index 02518d516261b..f7b37dae0a5f3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -24,8 +24,9 @@ import org.apache.spark.sql.catalyst.util._ /** * A framework for running the query tests that are listed as a set of text files. * - * TestSuites that derive from this class must provide a map of testCaseName -> testCaseFiles that should be included. - * Additionally, there is support for whitelisting and blacklisting tests as development progresses. + * TestSuites that derive from this class must provide a map of testCaseName -> testCaseFiles + * that should be included. Additionally, there is support for whitelisting and blacklisting + * tests as development progresses. */ abstract class HiveQueryFileTest extends HiveComparisonTest { /** A list of tests deemed out of scope and thus completely disregarded */ @@ -54,15 +55,17 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { case (testCaseName, testCaseFile) => if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { logDebug(s"Blacklisted test skipped $testCaseName") - } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || runAll) { + } else if (realWhiteList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_) || + runAll) { // Build a test case and submit it to scala test framework... val queriesString = fileToString(testCaseFile) createQueryTest(testCaseName, queriesString) } else { // Only output warnings for the built in whitelist as this clutters the output when the user // trying to execute a single test from the commandline. - if(System.getProperty(whiteListProperty) == null && !runAll) + if (System.getProperty(whiteListProperty) == null && !runAll) { ignore(testCaseName) {} + } } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index de140fc72a2c3..af781a502e9f3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -37,7 +37,8 @@ import org.apache.spark.sql.hive.test.TestHive._ case class TestData(a: Int, b: String) /** - * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. + * A set of test cases expressed in Hive QL that are not covered by the tests + * included in the hive distribution. */ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { private val originalTimeZone = TimeZone.getDefault @@ -237,7 +238,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } createQueryTest("modulus", - "SELECT 11 % 10, IF((101.1 % 100.0) BETWEEN 1.01 AND 1.11, \"true\", \"false\"), (101 / 2) % 10 FROM src LIMIT 1") + "SELECT 11 % 10, IF((101.1 % 100.0) BETWEEN 1.01 AND 1.11, \"true\", \"false\"), " + + "(101 / 2) % 10 FROM src LIMIT 1") test("Query expressed in SQL") { setConf("spark.sql.dialect", "sql") @@ -309,7 +311,8 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { "SELECT * FROM src a JOIN src b ON a.key = b.key") createQueryTest("small.cartesian", - "SELECT a.key, b.key FROM (SELECT key FROM src WHERE key < 1) a JOIN (SELECT key FROM src WHERE key = 2) b") + "SELECT a.key, b.key FROM (SELECT key FROM src WHERE key < 1) a JOIN " + + "(SELECT key FROM src WHERE key = 2) b") createQueryTest("length.udf", "SELECT length(\"test\") FROM src LIMIT 1") @@ -457,6 +460,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("lateral view3", "FROM src SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX") + // scalastyle:off createQueryTest("lateral view4", """ |create table src_lv1 (key string, value string); @@ -466,6 +470,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { |insert overwrite table src_lv1 SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX |insert overwrite table src_lv2 SELECT key, D.* lateral view explode(array(key+3, key+4)) D as CX """.stripMargin) + // scalastyle:on createQueryTest("lateral view5", "FROM src SELECT explode(array(key+3, key+4))") @@ -584,7 +589,7 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { } } - def isExplanation(result: DataFrame) = { + def isExplanation(result: DataFrame): Boolean = { val explanation = result.select('plan).collect().map { case Row(plan: String) => plan } explanation.contains("== Physical Plan ==") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index f4440e5b7846a..8ad3627504229 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -25,7 +25,8 @@ case class Nested(a: Int, B: Int) case class Data(a: Int, B: Int, n: Nested, nestedArray: Seq[Nested]) /** - * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. + * A set of test cases expressed in Hive QL that are not covered by the tests + * included in the hive distribution. */ class HiveResolutionSuite extends HiveComparisonTest { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index 7486bfa82b00b..d05e11fcf281b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -26,8 +26,9 @@ import org.apache.spark.sql.hive.test.TestHive */ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { - override def beforeAll() = { + override def beforeAll(): Unit = { TestHive.cacheTables = false + super.beforeAll() } createQueryTest( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala index ab0e0443c7faa..f0f04f8c73fb4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala @@ -35,8 +35,10 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { val nullVal = "null" baseTypes.init.foreach { i => - createQueryTest(s"case when then $i else $nullVal end ", s"SELECT case when true then $i else $nullVal end FROM src limit 1") - createQueryTest(s"case when then $nullVal else $i end ", s"SELECT case when true then $nullVal else $i end FROM src limit 1") + createQueryTest(s"case when then $i else $nullVal end ", + s"SELECT case when true then $i else $nullVal end FROM src limit 1") + createQueryTest(s"case when then $nullVal else $i end ", + s"SELECT case when true then $nullVal else $i end FROM src limit 1") } test("[SPARK-2210] boolean cast on boolean value should be removed") { 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 d7c5d1a25a82b..7f49eac490572 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 @@ -123,9 +123,10 @@ class HiveUdfSuite extends QueryTest { IntegerCaseClass(1) :: IntegerCaseClass(2) :: Nil).toDF() testData.registerTempTable("integerTable") - sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '${classOf[UDFIntegerToString].getName}'") + val udfName = classOf[UDFIntegerToString].getName + sql(s"CREATE TEMPORARY FUNCTION testUDFIntegerToString AS '$udfName'") checkAnswer( - sql("SELECT testUDFIntegerToString(i) FROM integerTable"), //.collect(), + sql("SELECT testUDFIntegerToString(i) FROM integerTable"), Seq(Row("1"), Row("2"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFIntegerToString") @@ -141,7 +142,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testUDFListListInt AS '${classOf[UDFListListInt].getName}'") checkAnswer( - sql("SELECT testUDFListListInt(lli) FROM listListIntTable"), //.collect(), + sql("SELECT testUDFListListInt(lli) FROM listListIntTable"), Seq(Row(0), Row(2), Row(13))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFListListInt") @@ -156,7 +157,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testUDFListString AS '${classOf[UDFListString].getName}'") checkAnswer( - sql("SELECT testUDFListString(l) FROM listStringTable"), //.collect(), + sql("SELECT testUDFListString(l) FROM listStringTable"), Seq(Row("a,b,c"), Row("d,e"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFListString") @@ -170,7 +171,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testStringStringUdf AS '${classOf[UDFStringString].getName}'") checkAnswer( - sql("SELECT testStringStringUdf(\"hello\", s) FROM stringTable"), //.collect(), + sql("SELECT testStringStringUdf(\"hello\", s) FROM stringTable"), Seq(Row("hello world"), Row("hello goodbye"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testStringStringUdf") @@ -187,7 +188,7 @@ class HiveUdfSuite extends QueryTest { sql(s"CREATE TEMPORARY FUNCTION testUDFTwoListList AS '${classOf[UDFTwoListList].getName}'") checkAnswer( - sql("SELECT testUDFTwoListList(lli, lli) FROM TwoListTable"), //.collect(), + sql("SELECT testUDFTwoListList(lli, lli) FROM TwoListTable"), Seq(Row("0, 0"), Row("2, 2"), Row("13, 13"))) sql("DROP TEMPORARY FUNCTION IF EXISTS testUDFTwoListList") @@ -247,7 +248,8 @@ class PairUdf extends GenericUDF { override def initialize(p1: Array[ObjectInspector]): ObjectInspector = ObjectInspectorFactory.getStandardStructObjectInspector( Seq("id", "value"), - Seq(PrimitiveObjectInspectorFactory.javaIntObjectInspector, PrimitiveObjectInspectorFactory.javaIntObjectInspector) + Seq(PrimitiveObjectInspectorFactory.javaIntObjectInspector, + PrimitiveObjectInspectorFactory.javaIntObjectInspector) ) override def evaluate(args: Array[DeferredObject]): AnyRef = { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 8474d850c9c6c..067b577f1560e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -143,7 +143,7 @@ class PruningSuite extends HiveComparisonTest with BeforeAndAfter { sql: String, expectedOutputColumns: Seq[String], expectedScannedColumns: Seq[String], - expectedPartValues: Seq[Seq[String]]) = { + expectedPartValues: Seq[Seq[String]]): Unit = { test(s"$testCaseName - pruning test") { val plan = new TestHive.HiveQLQueryExecution(sql).executedPlan val actualOutputColumns = plan.output.map(_.name) 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 817b9dcb8f505..7811bd2e9effb 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 @@ -422,7 +422,7 @@ class SQLQuerySuite extends QueryTest { } test("resolve udtf with single alias") { - val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i+1}]}""")) + val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) jsonRDD(rdd).registerTempTable("data") val df = sql("SELECT explode(a) AS val FROM data") val col = df("val") @@ -435,7 +435,7 @@ class SQLQuerySuite extends QueryTest { // is not in a valid state (cannot be executed). Because of this bug, the analysis rule of // PreInsertionCasts will actually start to work before ImplicitGenerate and then // generates an invalid query plan. - val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i+1}]}""")) + val rdd = sparkContext.makeRDD((1 to 5).map(i => s"""{"a":[$i, ${i + 1}]}""")) jsonRDD(rdd).registerTempTable("data") val originalConf = getConf("spark.sql.hive.convertCTAS", "false") setConf("spark.sql.hive.convertCTAS", "false") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala index 5f71e1bbc2d2e..d5dd0bf58e702 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/parquetSuites.scala @@ -1,4 +1,3 @@ - /* * Licensed to the Apache Software Foundation (ASF) under one or more * contributor license agreements. See the NOTICE file distributed with @@ -887,7 +886,11 @@ abstract class ParquetPartitioningTest extends QueryTest with BeforeAndAfterAll test(s"SPARK-5775 read struct from $table") { checkAnswer( - sql(s"SELECT p, structField.intStructField, structField.stringStructField FROM $table WHERE p = 1"), + sql( + s""" + |SELECT p, structField.intStructField, structField.stringStructField + |FROM $table WHERE p = 1 + """.stripMargin), (1 to 10).map(i => Row(1, i, f"${i}_string"))) } From 2fe0a1aaeebbf7f60bd4130847d738c29f1e3d53 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Wed, 8 Apr 2015 22:45:40 -0700 Subject: [PATCH 013/128] [SPARK-5654] Integrate SparkR This pull requests integrates SparkR, an R frontend for Spark. The SparkR package contains both RDD and DataFrame APIs in R and is integrated with Spark's submission scripts to work on different cluster managers. Some integration points that would be great to get feedback on: 1. Build procedure: SparkR requires R to be installed on the machine to be built. Right now we have a new Maven profile `-PsparkR` that can be used to enable SparkR builds 2. YARN cluster mode: The R package that is built needs to be present on the driver and all the worker nodes during execution. The R package location is currently set using SPARK_HOME, but this might not work on YARN cluster mode. The SparkR package represents the work of many contributors and attached below is a list of people along with areas they worked on edwardt (edwart) - Documentation improvements Felix Cheung (felixcheung) - Documentation improvements Hossein Falaki (falaki) - Documentation improvements Chris Freeman (cafreeman) - DataFrame API, Programming Guide Todd Gao (7c00) - R worker Internals Ryan Hafen (hafen) - SparkR Internals Qian Huang (hqzizania) - RDD API Hao Lin (hlin09) - RDD API, Closure cleaner Evert Lammerts (evertlammerts) - DataFrame API Davies Liu (davies) - DataFrame API, R worker internals, Merging with Spark Yi Lu (lythesia) - RDD API, Worker internals Matt Massie (massie) - Jenkins build Harihar Nahak (hnahak87) - SparkR examples Oscar Olmedo (oscaroboto) - Spark configuration Antonio Piccolboni (piccolbo) - SparkR examples, Namespace bug fixes Dan Putler (dputler) - Dataframe API, SparkR Install Guide Ashutosh Raina (ashutoshraina) - Build improvements Josh Rosen (joshrosen) - Travis CI build Sun Rui (sun-rui)- RDD API, JVM Backend, Shuffle improvements Shivaram Venkataraman (shivaram) - RDD API, JVM Backend, Worker Internals Zongheng Yang (concretevitamin) - RDD API, Pipelined RDDs, Examples and EC2 guide Author: Shivaram Venkataraman Author: Shivaram Venkataraman Author: Zongheng Yang Author: cafreeman Author: Shivaram Venkataraman Author: Davies Liu Author: Davies Liu Author: hlin09 Author: Sun Rui Author: lythesia Author: oscaroboto Author: Antonio Piccolboni Author: root Author: edwardt Author: hqzizania Author: dputler Author: Todd Gao Author: Chris Freeman Author: Felix Cheung Author: Hossein Author: Evert Lammerts Author: Felix Cheung Author: felixcheung Author: Ryan Hafen Author: Ashutosh Raina Author: Oscar Olmedo Author: Josh Rosen Author: Yi Lu Author: Harihar Nahak Closes #5096 from shivaram/R and squashes the following commits: da64742 [Davies Liu] fix Date serialization 59266d1 [Davies Liu] check exclusive of primary-py-file and primary-r-file 55808e4 [Davies Liu] fix tests 5581c75 [Davies Liu] update author of SparkR f731b48 [Shivaram Venkataraman] Only run SparkR tests if R is installed 64eda24 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R d7c3f22 [Shivaram Venkataraman] Address code review comments Changes include 1. Adding SparkR docs to API docs generated 2. Style fixes in SparkR scala files 3. Clean up of shell scripts and explanation of install-dev.sh 377151f [Shivaram Venkataraman] Merge remote-tracking branch 'apache/master' into R eb5da53 [Shivaram Venkataraman] Merge pull request #3 from davies/R2 a18ff5c [Davies Liu] Update sparkR.R 5133f3a [Shivaram Venkataraman] Merge pull request #7 from hqzizania/R3 940b631 [hqzizania] [SPARKR-92] Phase 2: implement sum(rdd) 0e788c0 [Shivaram Venkataraman] Merge pull request #5 from hlin09/doc-fix 3487461 [hlin09] Add tests log in .gitignore. 1d1802e [Shivaram Venkataraman] Merge pull request #4 from felixcheung/r-require 11981b7 [felixcheung] Update R to fail early if SparkR package is missing c300e08 [Davies Liu] remove duplicated file b045701 [Davies Liu] Merge branch 'remote_r' into R 19c9368 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into remote_r f8fa8af [Davies Liu] mute logging when start/stop context e7104b6 [Davies Liu] remove ::: in SparkR a1777eb [Davies Liu] move rules into R/.gitignore e88b649 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R 6e20e71 [Davies Liu] address comments b433817 [Davies Liu] Merge branch 'master' of github.com:apache/spark into R a1cedad [Shivaram Venkataraman] Merge pull request #228 from felixcheung/doc e089151 [Davies Liu] Merge pull request #225 from sun-rui/SPARKR-154_2 463e28c [Davies Liu] Merge pull request #2 from shivaram/doc-fixes bc2d6d8 [Shivaram Venkataraman] Remove arg from sparkR.stop and update docs d425363 [Shivaram Venkataraman] Some doc fixes for column, generics, group 1f1a7e0 [Shivaram Venkataraman] Some fixes to DataFrame, RDD, SQLContext docs 104ad4e [Shivaram Venkataraman] Check the right env in exists cf5cd99 [Shivaram Venkataraman] Remove unused numCols argument 85a50ec [Shivaram Venkataraman] Merge pull request #226 from RevolutionAnalytics/master 3eacfc0 [Davies Liu] fix flaky test 733380d [Davies Liu] update R examples (remove master from args) b21a0da [Davies Liu] Merge pull request #1 from shivaram/log4j-tests a1493d7 [Shivaram Venkataraman] Address comments e1f83ab [Shivaram Venkataraman] Send Spark INFO logs to a file in SparkR tests 58276f5 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R 52cc92d [Shivaram Venkataraman] Add license to create-docs.sh 6ff5ea2 [Shivaram Venkataraman] Add instructions to generate docs 1f478c5 [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R 02b4833 [Shivaram Venkataraman] Add a script to generate R docs (Rd, html) Also fix some issues with our documentation d6d3729 [Davies Liu] enable spark and pyspark tests 0e5a83f [Davies Liu] fix code style afd8a77 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R d87a181 [Davies Liu] fix flaky tests 7100fb9 [Shivaram Venkataraman] Fix libPaths in README bdf3a14 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R 05e7375 [Davies Liu] sort generics b44e371 [Shivaram Venkataraman] Include RStudio instructions in README 855537f [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R 9fb6af3 [Davies Liu] mark R classes/objects are private 423ea3c [Shivaram Venkataraman] Ignore unknown jobj in cleanup 974e4ea [Davies Liu] fix flaky test 410ec18 [Davies Liu] fix zipRDD() tests d8b24fc [Davies Liu] disable spark and python tests temporary ce3ca62 [Davies Liu] fix license check 7da0049 [Davies Liu] fix build 2892e29 [Davies Liu] support R in YARN cluster ebd4d07 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R 38cbf59 [Davies Liu] fix test of zipRDD() 756ece0 [Shivaram Venkataraman] Update README remove outdated TODO d436f26 [Davies Liu] add missing files 40d193a [Shivaram Venkataraman] Merge pull request #224 from sun-rui/SPARKR-224-new 1a16cd6 [Davies Liu] rm PROJECT_HOME 56670ef [Davies Liu] rm man page ba4b80b [Davies Liu] Merge branch 'remote_r' into R f04080c [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into remote_r 028cbfb [Davies Liu] fix exit code of sparkr unit test 42d8b4c [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R ef26015 [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R a1870e8 [Shivaram Venkataraman] Merge pull request #214 from sun-rui/SPARKR-156_3 cb6e5e3 [Shivaram Venkataraman] Add scripts to start SparkR on windows 8030847 [Shivaram Venkataraman] Set windows file separators, install dirs 05afef0 [Shivaram Venkataraman] Only stop backend JVM if R launched it 95d2de3 [Davies Liu] fix spark-submit with R scripot baefd9e [Shivaram Venkataraman] Make bin/sparkR use spark-submit As a part of this move the R initialization functions into first.R and first-submit.R d6f2bdd [Shivaram Venkataraman] Fix run-tests path ea90fab [Davies Liu] fix spark-submit with R path and sparkR -h 0e2412c [Davies Liu] fix bin/sparkR 9f6aa1f [Davies Liu] Merge branch 'R' of github.com:amplab-extras/spark into R 479e3fe [Davies Liu] change println() to logging 52ca6e5 [Shivaram Venkataraman] Add missing comma 716b16f [Shivaram Venkataraman] Merge branch 'R' of https://github.com/amplab-extras/spark into R 2d235d4 [Shivaram Venkataraman] Build SparkR with Maven profile aae881b [Davies Liu] fix rat ff776aa [Shivaram Venkataraman] Fix style e4f1937 [Shivaram Venkataraman] Remove DFC example f7b6936 [Davies Liu] remove Spark prefix for class 043959e [Davies Liu] cleanup ba53b09 [Davies Liu] support R in spark-submit f403b4a [Davies Liu] rm .travis.yml c4a5bdf [Davies Liu] run sparkr tests in Spark e8fc7ca [Davies Liu] fix .gitignore 35e5755 [Davies Liu] reduce size of example data 50bff63 [Davies Liu] add LICENSE header for R sources facb6e0 [Davies Liu] add .gitignore for .o, .so, .Rd 18e5eed [Davies Liu] update docs 0a0e632 [Davies Liu] move sparkR into bin/ a76472f [Davies Liu] fix path of assembly jar df3eeea [Davies Liu] move R/examples into examples/src/main/r 3415cc7 [Davies Liu] move Scala source into core/ and sql/ 180fc9c [Davies Liu] move scala 014d253 [Davies Liu] delete man pages 49a8133 [Davies Liu] Merge branch 'remote_r' into R 44994c2 [Davies Liu] Moved files to R/ 2fc553f [Shivaram Venkataraman] Merge pull request #222 from davies/column2 b043876 [Davies Liu] fix test 5e610cb [Davies Liu] add more API for Column 6f95d49 [Shivaram Venkataraman] Merge pull request #221 from shivaram/sparkr-stop-start 3214c6d [Shivaram Venkataraman] Merge pull request #217 from hlin09/cleanClosureFix f5d3355 [Shivaram Venkataraman] Merge pull request #218 from davies/merge 70f620c [Davies Liu] address comments 4b1628d [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into merge 3139325 [Shivaram Venkataraman] Merge pull request #212 from davies/toDF 6122e0e [Davies Liu] handle NULL bc2ff38 [Davies Liu] handle NULL 7f5e70c [Davies Liu] Update SerDe.scala 46454e4 [Davies Liu] address comments dd52cbc [Shivaram Venkataraman] Merge pull request #220 from shivaram/sparkr-utils-include 662938a [Shivaram Venkataraman] Include utils before SparkR for `head` to work Before this change calling `head` on a DataFrame would not work from the sparkR script as utils would be loaded after SparkR and placed ahead in the search list. This change requires utils to be loaded before SparkR 1bc2998 [Shivaram Venkataraman] Merge pull request #179 from evertlammerts/sparkr-sql 7695d36 [Evert Lammerts] added tests 8190127 [Evert Lammerts] fixed parquetFile signature d8c8fcc [Shivaram Venkataraman] Merge pull request #219 from shivaram/sparkr-build-final 963c7ee [Davies Liu] Merge branch 'master' into merge 8bff523 [Shivaram Venkataraman] Remove staging repo now that 1.3 is released e52258f [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into toDF 05b9126 [Shivaram Venkataraman] Merge pull request #215 from davies/agg 8e1497d [Davies Liu] Update DataFrame.R 72adb14 [Davies Liu] Update SQLContext.R 66cc92a [Davies Liu] address commets 55c38bc [Shivaram Venkataraman] Merge pull request #216 from davies/select2 3e0555d [Shivaram Venkataraman] Merge pull request #193 from davies/daemon 0467474 [Davies Liu] add more selecter for DataFrame 9a6be74 [Davies Liu] include grouping columns in agg() e87bb98 [Davies Liu] improve comment and logging a6dc435 [Davies Liu] remove dependency of jsonlite 26a3621 [Davies Liu] support date.frame and Date/Time 4e4908a [Davies Liu] createDataFrame from rdd 5757b95 [Shivaram Venkataraman] Merge pull request #196 from davies/die 90f2692 [Shivaram Venkataraman] Merge pull request #211 from hlin09/generics 8583968 [Davies Liu] readFully() 46cea3d [Davies Liu] retry 01aa5ee [Davies Liu] add config for using daemon, refactor ff948db [hlin09] Remove missingOrInteger. ecdfda1 [hlin09] Remove duplication. 411b751 [Davies Liu] make RStudio happy 8f8813f [Davies Liu] switch back to use parallel 6bccbbf [hlin09] Move roxygen doc back to implementation. ffd6e8e [Shivaram Venkataraman] Merge pull request #210 from hlin09/hlin09 471c794 [hlin09] Move getJRDD and broadcast's value to 00-generic.R. 89b886d [hlin09] Move setGeneric() to 00-generics.R. 97dde1a [hlin09] Add a test for access operators. 09ff163 [Shivaram Venkataraman] Merge pull request #204 from cafreeman/sparkr-sql 15a713f [cafreeman] Fix example for `dropTempTable` dc1291b [hlin09] Add checks for namespace access operators in cleanClosure. b4c0b2e [Davies Liu] use fork package 3db5649 [cafreeman] Merge branch 'sparkr-sql' of https://github.com/amplab-extras/SparkR-pkg into sparkr-sql 789be97 [Shivaram Venkataraman] Merge pull request #207 from shivaram/err-remove e60578a [cafreeman] update tests to guarantee row order 5eec6fc [Shivaram Venkataraman] Merge pull request #206 from sun-rui/SPARKR-156_2 3f7aed6 [Sun Rui] Fix minor typos in the function description. a8cebf0 [Shivaram Venkataraman] Remove print statement in SparkRBackendHandler This print statement is noisy for SQL methods which have multiple APIs (like loadDF). We already have a better error message when no valid methods are found 5e3a576 [Sun Rui] Fix indentation. f3d99a6 [Sun Rui] [SPARKR-156] phase 2: implement zipWithIndex() of the RDD class. a582810 [cafreeman] Merge branch 'dfMethods' into sparkr-sql 7a5d6fd [cafreeman] `withColumn` and `withColumnRenamed` c5fa3b9 [cafreeman] New `select` method bcb0bf5 [Shivaram Venkataraman] Merge pull request #180 from davies/group 9dd6a5a [Davies Liu] Update SparkRBackendHandler.scala e6fb8d8 [Davies Liu] improve logging 428a99a [Davies Liu] remove test, catch exception fef99de [cafreeman] `intersect`, `subtract`, `unionAll` befbd32 [cafreeman] `insertInto` 9d01bcd [cafreeman] `dropTempTable` d8c1c09 [Davies Liu] add test to start and stop context multiple times 18c6004 [Shivaram Venkataraman] Merge pull request #201 from sun-rui/SPARKR-156_1 dfb399a [Davies Liu] address comments f06ccec [Sun Rui] Use mapply() instead of for statement. 3c7674f [Davies Liu] Merge branch 'die' of github.com:davies/SparkR-pkg into die ac8a852 [Davies Liu] close monitor connection in sparkR.stop() 4d0fb56 [Shivaram Venkataraman] Merge pull request #203 from shivaram/sparkr-hive-fix 62b0760 [Shivaram Venkataraman] Fix test hive context package name 47a613f [Shivaram Venkataraman] Fix HiveContext package name fb3b139 [Davies Liu] fix tests d0d4626 [Shivaram Venkataraman] Merge pull request #199 from davies/load 8b7fb67 [Davies Liu] fix HiveContext bb46832 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load e9e2a03 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into group b875b4f [Davies Liu] fix style de2abfa [Shivaram Venkataraman] Merge pull request #202 from cafreeman/sparkr-sql 3675fcf [cafreeman] Update `explain` and fixed doc for `toJSON` 5fd9575 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load 6fac596 [Davies Liu] support Column expression in agg() f10a24e [Davies Liu] address comments ff8b005 [cafreeman] 'saveAsParquetFile` a5c2887 [cafreeman] fix test 3fab0f8 [cafreeman] `showDF` 779c102 [cafreeman] `isLocal` 68b11cf [cafreeman] `toJSON` 0ac4abc [cafreeman] 'explain` 20242c4 [cafreeman] clean up docs 6a1fe64 [Shivaram Venkataraman] Merge pull request #198 from cafreeman/sparkr-sql 198c130 [Shivaram Venkataraman] Merge pull request #200 from shivaram/sparkr-sql-build 870acd4 [Shivaram Venkataraman] Use rc2 explicitly 8b9a963 [cafreeman] Merge branch 'sparkr-sql' of https://github.com/amplab-extras/SparkR-pkg into sparkr-sql bc90115 [cafreeman] Fixed docs 3865f39 [Sun Rui] [SPARKR-156] phase 1: implement zipWithUniqueId() of the RDD class. a37fd80 [Davies Liu] Update sparkR.R d18f9d3 [Shivaram Venkataraman] Remove SparkR snapshot build We now have 1.3.0 RC2 on Apache Staging 8de958d [Davies Liu] Update SparkRBackend.scala 4e0becc [Shivaram Venkataraman] Merge pull request #194 from davies/api 197a79b [Davies Liu] add HiveContext (commented) 32aa01d [Shivaram Venkataraman] Merge pull request #191 from felixcheung/doc 5073e07 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into load 7918634 [cafreeman] Fix test acea146 [cafreeman] remove extra line 74269f3 [cafreeman] Merge branch 'dfMethods' into sparkr-sql cd7ac8a [Shivaram Venkataraman] Merge pull request #197 from cafreeman/sparkr-sql 494a4dd [cafreeman] update export e14c328 [cafreeman] `selectExpr` 32b37d1 [cafreeman] Fixed indent in `join` test. 2e7b190 [Felix Cheung] small update on yarn deploy mode. 8ff29d6 [Davies Liu] fix tests 12a6db2 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into api 294ca4a [cafreeman] `join`, `sort`, and `filter` 4fa6343 [cafreeman] Refactor `join` generic for use with `DataFrame` 3f22c8d [Shivaram Venkataraman] Merge pull request #195 from cafreeman/sparkr-sql 2b6f980 [Davies Liu] shutdown the JVM after R process die e8639c3 [cafreeman] New 1.3 repo and updates to `column.R` ed9a89f [Davies Liu] address comments 03bcf20 [Davies Liu] Merge branch 'group' of github.com:davies/SparkR-pkg into group 39c253d [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into group 98cc97a [Davies Liu] fix test and docs e2d144a [Felix Cheung] Fixed small typos 3beadcf [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into api 06cbc2d [Davies Liu] launch R worker by a daemon 8a676b1 [Shivaram Venkataraman] Merge pull request #188 from davies/column 524c122 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into column f798402 [Davies Liu] Update column.R 1d0f2ae [Davies Liu] Update DataFrame.R 03402eb [Felix Cheung] Updates as per feedback on sparkR-submit 76cf2e0 [Shivaram Venkataraman] Merge pull request #192 from cafreeman/sparkr-sql 1955a09 [cafreeman] return object instead of a list of one object f585929 [cafreeman] Fix brackets e998356 [cafreeman] define generic for 'first' in RDD API 71d66a1 [Davies Liu] fix first(0 8ec21af [Davies Liu] fix signature acae527 [Davies Liu] refactor d7b17a4 [Davies Liu] fix approxCountDistinct 7dfe27d [Davies Liu] fix cyclic namespace dependency 8caf5bb [Davies Liu] use S4 methods 5c0bb24 [Felix Cheung] Doc updates: build and running on YARN 773baf0 [Zongheng Yang] Merge pull request #178 from davies/random 862f07c [Shivaram Venkataraman] Merge pull request #190 from shivaram/SPARKR-79 b457833 [Shivaram Venkataraman] Merge pull request #189 from shivaram/stdErrFix f7caeb8 [Davies Liu] Update SparkRBackend.scala 8c4deae [Shivaram Venkataraman] Remove unused function 6e51c7f [Shivaram Venkataraman] Fix stderr redirection on executors 7afa4c9 [Shivaram Venkataraman] Merge pull request #186 from hlin09/funcDep3 4d36ab1 [hlin09] Add tests for broadcast variables. 3f57e56 [hlin09] Fix comments. 7b72487 [hlin09] Fix comments. ae05bf1 [Davies Liu] Merge branch 'sparkr-sql' of github.com:amplab-extras/SparkR-pkg into column abb4bb9 [Davies Liu] add Column and expression eb8ac11 [Shivaram Venkataraman] Set Spark version 1.3.0 in Windows build 5c72e73 [Davies Liu] wait atmost 100 seconds e425437 [Shivaram Venkataraman] Merge pull request #177 from lythesia/master a00f502 [lythesia] fix indents 0346e5f [Davies Liu] address comment 6134649 [Shivaram Venkataraman] Merge pull request #187 from cafreeman/sparkr-sql ad0935e [lythesia] minor fixes b0e7f73 [cafreeman] Update `sampleDF` test 7b0d070 [lythesia] keep partitions check 889c265 [cafreeman] numToInt utility function 27dd3a0 [lythesia] modify tests for repartition cad0f0c [cafreeman] Fix docs and indents 2808dcf [cafreeman] Three more DataFrame methods 5ef66fb [Davies Liu] send back the port via temporary file 3b46429 [Davies Liu] Merge branch 'master' of github.com:amplab-extras/SparkR-pkg into random 798f453 [cafreeman] Merge branch 'sparkr-sql' into dev 9aa4acf [Shivaram Venkataraman] Merge pull request #184 from davies/socket 020bce8 [Shivaram Venkataraman] Merge pull request #183 from cafreeman/sparkr-sql 222e06b [cafreeman] Lazy evaluation and formatting changes e776324 [Davies Liu] fix import 211cc15 [cafreeman] Merge branch 'sparkr-sql' into dev 3351afd [hlin09] Replaces getDependencies with cleanClosure, to serialize UDFs to workers. e7c56d6 [lythesia] fix random partition key 50c74b1 [Davies Liu] address comments 083c89f [cafreeman] Remove commented lines an unused import dfa119b [hlin09] Improve the coverage of processClosure. a41c9b9 [cafreeman] Merge branch 'wrapper' into sparkr-sql 1cd714f [cafreeman] Wrapper function docs. db0cd9e [cafreeman] Clean up for wrapper functions 818c19f [cafreeman] Update schema-related functions a57884e [cafreeman] Remove unused import d72e830 [cafreeman] Add wrapper for `StructField` and `StructType` 2ea2ecf [lythesia] use generic arg 09b9512 [hlin09] add docs f4f077c [hlin09] Add recursive cleanClosure for function access. f84ad27 [hlin09] Merge remote-tracking branch 'upstream/master' into funcDep2 5300766 [Shivaram Venkataraman] Merge pull request #185 from hlin09/hlin09 07aa7c0 [hlin09] Unifies the implementation of lapply with lapplyParitionsWithIndex. f4dbb0b [Davies Liu] use socket in worker 8282c59 [Davies Liu] Update DataFrame.R ba495a8 [Davies Liu] Update NAMESPACE 36dffb3 [cafreeman] Add 'head` and `first` 534a95f [cafreeman] Schema-related methods 64f488d [cafreeman] Cache and Persist Methods 30d71fd [cafreeman] Standardize method arguments for DataFrame methods 785898b [Shivaram Venkataraman] Merge pull request #182 from cafreeman/sparkr-sql 2619003 [Shivaram Venkataraman] Merge pull request #181 from cafreeman/master a9bbe0b [cafreeman] Update existing SparkSQL functions 8c241a3 [cafreeman] Merge with master, include changes to method args 68d6de4 [cafreeman] Fix typos 8d2ec6e [Davies Liu] add sum/max/min/avg/mean 774e687 [Davies Liu] add missing API in SQLContext 1e72b4b [Davies Liu] missing API in SQLContext 3294949 [Chris Freeman] Restore `rdd` argument to `getJRDD` 3a58ebc [Davies Liu] rm unrelated file 8bd93b5 [Davies Liu] fix signature c652b4c [cafreeman] Update method signatures to use generic arg 48c8827 [Davies Liu] update NAMESPACE 84e2d8c [Davies Liu] groupBy and agg() 7c3ddbd [Davies Liu] create jmode in JVM 9465426 [Davies Liu] load and save 982f342 [lythesia] fix numeric issue 7651d84 [lythesia] fix coalesce 4e712e1 [Davies Liu] use random port in backend 041d22b [Shivaram Venkataraman] Merge pull request #172 from cafreeman/sparkr-sql 0d07770 [cafreeman] Added `limit` and updated `take` 301d8e5 [cafreeman] Remove extraneous map functions 0387db2 [cafreeman] Remove colNames 04c4b65 [lythesia] add repartition/coalesce 231deab [cafreeman] Change reserialize to serializeToBytes acf7e1a [cafreeman] Rework the Scala to R DataFrame Conversion 481ae37 [cafreeman] Updated stale comments and standardized arg names 21d4a97 [hlin09] Adds cleanClosure to capture the function closures. d24ffb4 [hlin09] Merge remote-tracking branch 'upstream/master' into funcDep2 8be02de [hlin09] Revert "loop 1-12 test pass." fddb9cc [hlin09] Revert "add docs" f8ef0ab [hlin09] Revert "More docs" 8e4b3da [hlin09] Revert "More docs" 57e005b [hlin09] Revert "fix tests." c10148e [Shivaram Venkataraman] Merge pull request #174 from shivaram/sparkr-runner 910e3be [Shivaram Venkataraman] Add a timeout for initialization Also move sparkRBackend.stop into a finally block bf52b17 [Shivaram Venkataraman] Merge remote-tracking branch 'amplab-sparkr/master' into sparkr-runner 08102b0 [Shivaram Venkataraman] Merge pull request #176 from lythesia/master 9c77b20 [Chris Freeman] Merge pull request #2 from shivaram/sparkr-sql 179ab38 [lythesia] add try counts and increase time interval 71a73b2 [Shivaram Venkataraman] Use a getter for serialization mode This change encapsulates the semantics of serialization mode for RDDs inside a getter function. For PipelinedRDDs if a backing JavaRDD is available we use that else we fall back to a default serialization mode 06bf250 [Shivaram Venkataraman] Merge pull request #173 from shivaram/windows-space-fix 88bf97f [Shivaram Venkataraman] Create SparkContext for R shell launch f9268d9 [Shivaram Venkataraman] Fix code review comments e6ad12d [Shivaram Venkataraman] Update comment describing sparkR-submit 17eda4c [Shivaram Venkataraman] Merge pull request #175 from falaki/docfix ba2b72b [Hossein] Spark 1.1.0 is default 4cd7d3f [lythesia] retry backend connection 749e2d0 [Hossein] Updated README bc04cf4 [Shivaram Venkataraman] Use SPARKR_BACKEND_PORT in sparkR.R as default Change SparkRRunner to use EXISTING_SPARKR_BACKEND_PORT to differentiate between the two 22a19ac [Shivaram Venkataraman] Use a semaphore to wait for backend to initalize Also pick a random port to avoid collisions 7f1f0f8 [cafreeman] Move comments to fit 100 char line length 8b84e4e [cafreeman] Make if statements more explicit ce5d5ab [cafreeman] New tests for Union and Object File b063320 [cafreeman] Changed 'serialized' to 'serializedMode' 0981dff [Zongheng Yang] Merge pull request #168 from sun-rui/SPARKR-153_2 86fc639 [Shivaram Venkataraman] Move sparkR-submit into pkg/inst fd8f8a9 [Shivaram Venkataraman] Merge branch 'hqzizania-master' a33dbea [Shivaram Venkataraman] Merge branch 'master' of https://github.com/hqzizania/SparkR-pkg into hqzizania-master 384e6e2 [Shivaram Venkataraman] Merge pull request #171 from hlin09/hlin09 1f5a6ac [hlin09] fixed comments 7f7596a [cafreeman] Additional handling for "row" serialization 8c3b8c5 [cafreeman] Add test for UnionRDD on "row" serialization b1141f8 [cafreeman] Fixed formatting issues. 5db30bf [cafreeman] Changed serialized from bool to string 2f0c0b8 [cafreeman] Add check for serialized type d243dfb [cafreeman] Clean up code 5ff63a2 [cafreeman] Change test from boolean to string 77fec1a [cafreeman] Updated .Rd files 9224989 [cafreeman] Various updates for DataFrame to RRDD 26af62b [cafreeman] DataFrame to RRDD e004481 [cafreeman] Update UnionRDD test 5292be7 [hlin09] Adds support of pipeRDD(). e2a7560 [Shivaram Venkataraman] Merge pull request #170 from cafreeman/sparkr-sql 5d537f4 [cafreeman] Add pairRDD to Description b6fa88e [cafreeman] Updating to current master 0cda231 [Sun Rui] [SPARKR-153] phase 2: implement aggregateByKey() and foldByKey(). 95ee6b4 [Shivaram Venkataraman] Merge remote-tracking branch 'amplab-sparkr/master' into sparkr-runner 67fbc60 [Shivaram Venkataraman] Add support for SparkR shell to use spark-submit This ensures that SparkConf options are read in both in batch and interactive modes 2271030 [Shivaram Venkataraman] Merge pull request #167 from sun-rui/removePartionByInRDD 7fcb46a [Sun Rui] Remove partitionBy() in RDD. 52f94c4 [Shivaram Venkataraman] Merge pull request #160 from lythesia/master 59e2d54 [lythesia] merge with upstream 5836650 [Zongheng Yang] Merge pull request #163 from sun-rui/SPARKR-153_1 141723e [Sun Rui] fix comments. f73a07e [Shivaram Venkataraman] Merge pull request #165 from shivaram/sparkr-sql-build 10ffc6d [Shivaram Venkataraman] Set Spark version to 1.3 using staging dependency Also fix the maven build c91ede2 [Shivaram Venkataraman] Merge pull request #164 from hlin09/hlin09 9d335a9 [hlin09] Makes git to ignore Eclipse meta files. 94066bf [Sun Rui] [SPARKR-153] phase 1: implement fold() and aggregate(). 9c391c7 [hqzizania] Merge remote-tracking branch 'upstream/master' 5f29551 [hqzizania] modified: pkg/R/RDD.R modified: pkg/R/context.R d968664 [lythesia] fix comment 7972858 [Shivaram Venkataraman] Merge pull request #159 from sun-rui/SPARKR-150_2 7690878 [lythesia] separate out pair RDD functions f4573c1 [Sun Rui] Use reduce() instead of sortBy().take() to get the ordered elements. 63e62ed [Sun Rui] [SPARKR-150] phase 2: implement takeOrdered() and top(). 050390b [Shivaram Venkataraman] Fix bugs in inferring R file 8398f2e [Shivaram Venkataraman] Add sparkR-submit helper script Also adjust R file path for YARN cluster mode bd6705b [Zongheng Yang] Merge pull request #154 from sun-rui/SPARKR-150 c7964c9 [Sun Rui] Merge with upstream master. 7feac38 [Sun Rui] Use default arguments for sortBy() and sortKeyBy(). de2bfb3 [Sun Rui] Fix minor comments and add more test cases. 0c6e071 [Zongheng Yang] Merge pull request #157 from lythesia/master f5038c0 [lythesia] pull out anonymous functions in groupByKey ba6f044 [lythesia] fixes for reduceByKeyLocally 343b6ab [Oscar Olmedo] Export sparkR.stop Closes #156 from oscaroboto/master 25639cf [Shivaram Venkataraman] Replace tabs with spaces bb25920 [Shivaram Venkataraman] Merge branch 'dputler-master' fd836db [hlin09] fix tests. 24a7f13 [hlin09] More docs a465165 [hlin09] More docs 6ad4fc3 [hlin09] add docs b082a35 [lythesia] add reduceByKeyLocally 7ca6512 [Shivaram Venkataraman] First cut of SparkRRunner 193f5fe [hlin09] loop 1-12 test pass. 345f1b8 [dputler] [SPARKR-195] Implemented project style guidelines for if-else statements 8043559 [Sun Rui] Add a TODO to use binary search in the range partitioner. 91b2fd6 [Sun Rui] Add more test cases. e8ebbe4 [Shivaram Venkataraman] Merge pull request #152 from cafreeman/sparkr-sql 0c53d6c [dputler] Data frames now coerced to lists, and messages issued for a data frame or matrix on how they are parallelized 6d57ec0 [cafreeman] Remove json test file since we're using a temp ac1ef09 [cafreeman] Update registerTempTable test d9da451 [Sun Rui] [SPARKR-150] phase 1: implement sortBy() and sortByKey(). 08ff30b [Shivaram Venkataraman] Merge pull request #153 from hqzizania/master 9767e8e [hqzizania] modified: pkg/man/collect-methods.Rd 5d69f0a [hqzizania] modified: pkg/R/RDD.R 4914091 [hqzizania] modified: pkg/inst/tests/test_rdd.R 742a68b [cafreeman] Update test_sparkRSQL.R a95823e [hqzizania] modified: pkg/R/RDD.R 2d04526 [cafreeman] Formatting fae9bdd [cafreeman] Renamed to SQLUtils.scala 39888ea [Chris Freeman] Update test_sparkSQL.R fce2453 [cafreeman] Updated documentation for SQLContext 13fbf12 [cafreeman] Regenerated .Rd files 51ecf41 [cafreeman] Updated Scala object 30d7337 [cafreeman] Added SparkSQL test 74b3ed6 [cafreeman] Incorporate code feedback 554bda0 [Zongheng Yang] Merge pull request #147 from shivaram/sparkr-ec2-fixes a5f4f8f [cafreeman] Squashed commit of the following: f34bb88 [Shivaram Venkataraman] Remove profiling information from this PR c662f29 [Zongheng Yang] Merge pull request #146 from shivaram/spark-1.2-build 21e9b74 [Zongheng Yang] Merge pull request #145 from lythesia/master 76f6b9e [Shivaram Venkataraman] Merge pull request #149 from hqzizania/master 1c2dbec [lythesia] minor fix for refactoring join code 5b380d3 [hqzizania] modified: pkg/man/combineByKey.Rd modified: pkg/man/groupByKey.Rd modified: pkg/man/partitionBy.Rd modified: pkg/man/reduceByKey.Rd 98794fe [hqzizania] modified: pkg/R/RDD.R b66534d [Zongheng Yang] Merge pull request #144 from shivaram/fix-rd-files 60da1df [Shivaram Venkataraman] Initialize timing variables 179aa75 [Shivaram Venkataraman] Bunch of fixes for longer running jobs 1. Increase the timeout for socket connection to wait for long jobs 2. Add some profiling information in worker.R 3. Put temp file writes before stdin writes in RRDD.scala 06d99f0 [Shivaram Venkataraman] Fix URI to have right number of slashes add97f5 [Shivaram Venkataraman] Use URL encode to create valid URIs for jars 4eec962 [lythesia] refactor join functions 73430c6 [Shivaram Venkataraman] Make SparkR work on paths with spaces on Windows aaf8f47 [Shivaram Venkataraman] Exclude hadoop client from Spark dependency 227ee42 [Zongheng Yang] Merge pull request #141 from shivaram/SPARKR-140 ac5ceb1 [Shivaram Venkataraman] Fix code review comments 32394de [Shivaram Venkataraman] Regenerate Rd files for SparkR This fixes a number of issues in SparkR man pages. The main changes are 1. Don't export or generate docs for PipelineRDD 2. Fix variable names for Filter, count to match base methods 3. Document missing arguments for sparkR.init, print.jobj etc. e157bf6 [Shivaram Venkataraman] Use prev_serialized to track if JRDD is serialized This changes introduces a new variable in PipelineRDD environment to track if the prev_jrdd is serialized or not. 7428a7e [Zongheng Yang] Merge pull request #143 from shivaram/SPARKR-181 7dd1797 [Shivaram Venkataraman] Address code review comments 8f81c45 [Shivaram Venkataraman] Remove roxygen export for PipelinedRDD 0cb90f1 [Zongheng Yang] Merge pull request #142 from shivaram/SPARKR-169 d1c6e6c [Shivaram Venkataraman] Buffer stderr from R and return it on Exception This change buffers the last 100 lines from R process and passes these lines back to the driver if we have an exception. This will help users debug why their tasks failed on the cluster d6c1393 [Shivaram Venkataraman] Suppress warnings from normalizePath a382835 [Shivaram Venkataraman] Fix serialization tracking in pipelined RDDs When creating a pipeline RDD, we need to check if the JavaRDD belonging to the parent is serialized. da39529 [Zongheng Yang] Merge pull request #140 from sun-rui/SPARKR-183 2814caa [Sun Rui] Merge with upstream master. cd2a5b3 [Sun Rui] Add reference to Nagle's algorithm and clean code. 52356b6 [Shivaram Venkataraman] Merge pull request #139 from shivaram/fix-backend-exit 97e5a1f [Sun Rui] [SPARKR-183] Fix the issue that parallelize collect tests are slow. a9f8e8e [Shivaram Venkataraman] Merge pull request #138 from concretevitamin/fix-collect-test 125ae43 [Shivaram Venkataraman] Fix SparkR backend to exit in more cases This change has two fixes 1. When the workspace is saved (from R or RStudio) the backend connection seems to be closed before the finalizer is run. In such cases we reopen the connection and stop the backend 2. With RStudio when R is restarted, there are port-conflicts which appear due to a race condition between the JVM and rsession restart. This change adds a 1 sec sleep to avoid this race. 12c102a [Zongheng Yang] Simplify a unit test. 9c0637a [Zongheng Yang] Merge pull request #137 from shivaram/fix-docs 0df0e18 [Shivaram Venkataraman] Fix documentation for includePackage 7549f88 [Zongheng Yang] Merge pull request #136 from shivaram/man-updates 7edbe46 [Shivaram Venkataraman] Add missing man pages 9cb9567 [Shivaram Venkataraman] Merge pull request #131 from shivaram/rJavaExpt 1fa722e [Shivaram Venkataraman] Rename to SerDe now 2fcb051 [Shivaram Venkataraman] Rename to SerDeJVMR d112cf0 [Shivaram Venkataraman] Style fixes 9fd01cc [Shivaram Venkataraman] Remove unnecessary braces 0881931 [Shivaram Venkataraman] Some more style fixes f00b531 [Shivaram Venkataraman] Address code review comments. Big changes include style fixes throughout for named arguments c09ba05 [Shivaram Venkataraman] Change jobj id to be just an integer Add a new print.jobj that gets the class name and prints it Also add a utility function isInstanceOf be05b16 [Shivaram Venkataraman] Check if context, connection exist before stopping d596a23 [Shivaram Venkataraman] Address code review comments 396e7ac [Shivaram Venkataraman] Changes to make new backend work on Windows This change uses file.path to construct the Java binary path in a OS agnostic way and uses system2 to handle quoting binary paths correctly. Tests pass on Mac OSX and a Windows EC2 instance. e7a4e03 [Shivaram Venkataraman] Remove unused file BACKEND.md 62f380b [Shivaram Venkataraman] Update worker.R to use new deserialization call 8b9c4e6 [Shivaram Venkataraman] Change RDD name, setName to use new backend 6dcd5c5 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into rJavaExpt 0873397 [Shivaram Venkataraman] Refactor java object tracking into a new singleton. Also add comments describing each class 95db964 [Shivaram Venkataraman] Add comments, cleanup new R code bcd4258 [Zongheng Yang] Merge pull request #130 from lythesia/master 74dbc5e [Sun Rui] Match method using parameter types. 7ad4a4d [Sun Rui] Use 1 char to represent types on the backend->client direction. bace887 [Sun Rui] Use an integer count for the backend java object ID because Uniqueness isn't guaranteed by System.identityHashCode(). b38d04f [Sun Rui] Use 1 char to represent types on the client -> backend direction. f88bc68 [lythesia] Merge branch 'master' of github.com:lythesia/SparkR-pkg 71d41f5 [lythesia] add test case for fullOuterJoin eb4f423 [lythesia] --amend cffecc5 [lythesia] add test case for fullOuterJoin a547dd2 [Shivaram Venkataraman] Move classTag, rddRef into newJObject call This avoids them getting eagerly garbage collected 1255391 [Shivaram Venkataraman] Add a finalizer for jobj objects This enables Java objects to be garbage collected on the backend when they are no longer referenced in R. Also rename newJava to newJObject to be more consistent with callJMethod 70fa409 [Sun Rui] Add YARN Conf Dir to the class path when launching the backend. a1108ca [lythesia] add fullOuterJoin in RDD.R 2152727 [Shivaram Venkataraman] Remove empty file cd08bee [Shivaram Venkataraman] Update all functions to use new backend All unit tests pass. 9de49b7 [Shivaram Venkataraman] Add high level calls for methods, constructors Also update BACKEND.md 5a97ea4 [Shivaram Venkataraman] Add jobj S3 class that holds backend refs e071d3e [Shivaram Venkataraman] Change SparkRBackend to use general method calls This change uses a custom protocl + JNI to invoke any method on a given object type. Also update serializers, deserializers to make code more concise 49f0404 [Shivaram Venkataraman] Merge pull request #129 from lythesia/master 7f8cd82 [lythesia] update man 4715ed2 [Yi Lu] Update RDD.R 5a53801 [lythesia] fix name,setName 4f3870b [lythesia] add name,setName in RDD.R 1c25700 [Shivaram Venkataraman] Merge pull request #128 from sun-rui/SPARKR-165 c8507d8 [Sun Rui] [SPARKR-165] IS_SCALAR is not present in R before 3.1 2cff2bd [Sun Rui] Add function to invoke Java method. 7a31da1 [Shivaram Venkataraman] Merge branch 'dputler-master'. Closes #119 0ceba82 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/dputler/SparkR-pkg into dputler-master 735f70c [Shivaram Venkataraman] Merge pull request #125 from 7c00/rawcon fccfe6c [Shivaram Venkataraman] Merge pull request #127 from sun-rui/SPARKR-164 387bd57 [Sun Rui] [SPARKR-164] Temporary files used by SparkR accumulat as time goes on. 5f2268f [Shivaram Venkataraman] Add support to stop backend 5f745c0 [Shivaram Venkataraman] Update notes in backend 22015c1 [Shivaram Venkataraman] Add first cut of SparkR Backend 52821da [Todd Gao] switch the order of packages and function deps d7b0007 [Todd Gao] remove memCompress cb6873e [Shivaram Venkataraman] Merge pull request #126 from sun-rui/SPARKR-147 c5962eb [Todd Gao] further optimize using rawConnection f04c6e0 [Sun Rui] [SPARKR-147] Support multiple directories as input to textFile. b7de604 [Todd Gao] optimize execFunctionDeps loading in worker.R 4d4fc30 [Shivaram Venkataraman] Merge pull request #122 from cafreeman/master b508877 [cafreeman] Update SparkR_IDE_Setup.sh 21ed9d7 [cafreeman] Update build.sbt f73ec16 [cafreeman] Delete SparkR_IDE_Setup_Guide.md d63b026 [cafreeman] Delete SparkR_Quick_Start_Guide.md 6e6cb62 [cafreeman] Update SparkR_IDE_Setup.sh bc6042b [cafreeman] Update build.sbt a8197d5 [cafreeman] Merge remote-tracking branch 'upstream/master' d671564 [Zongheng Yang] Merge pull request #123 from shivaram/jcheck-void 76b8d00 [Zongheng Yang] Merge pull request #124 from shivaram/master b690d58 [Shivaram Venkataraman] Specify how to change Spark versions in README 0fb003d [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into jcheck-void 1c227b4 [Shivaram Venkataraman] Also add a check in context.R 96812b6 [Shivaram Venkataraman] Check for exceptions after void method calls f5c216d [cafreeman] Merge remote-tracking branch 'upstream/master' 90c8933 [Zongheng Yang] Merge pull request #121 from shivaram/fix-sort-order bd0e3b4 [Shivaram Venkataraman] Fix saveAsTextFile test case 2e55f67 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into fix-sort-order f10c607 [Shivaram Venkataraman] Merge pull request #118 from sun-rui/saveAsTextFile 6c9bfc0 [Sun Rui] Merge remote-tracking branch 'SparkR_upstream/master' into saveAsTextFile 6faedbe [cafreeman] Update SparkR_IDE_Setup_Guide.md 57008bc [cafreeman] Update SparkR_IDE_Setup.sh bb1c17d [cafreeman] Update SparkR_IDE_Setup.sh 538bfdb [cafreeman] Update SparkR_Quick_Start_Guide.md 31322c6 [cafreeman] Update SparkR_IDE_Setup.sh ca3f593 [Sun Rui] Refactor RRDD code. df58d95 [cafreeman] Update SparkR_Quick_Start_Guide.md b488c88 [cafreeman] Rename Spark_IDE_Setup.sh to SparkR_IDE_Setup.sh b2545a4 [cafreeman] Added IDE Setup Guide 0ffb5de [cafreeman] Merge branch 'master' of https://github.com/cafreeman/SparkR-pkg bd8fbfb [cafreeman] Merge remote-tracking branch 'upstream/master' 98efa5b [cafreeman] Added Quick Start Guide 3cf88f2 [Shivaram Venkataraman] Sort lists before comparing in unit tests Since Spark doesn't guarantee that shuffle results will always be in the same order, we need to sort the results before comparing for deterministic behavior d621dbc [Shivaram Venkataraman] Merge pull request #120 from sun-rui/objectFile c4a44d7 [Sun Rui] Add @seealso in comments and extract some common code into a function. 724e3a4 [cafreeman] Update Spark_IDE_Setup.sh 8153e5a [Sun Rui] [SPARKR-146] Support read/save object files in SparkR. 17f9909 [cafreeman] Update Spark_IDE_Setup.sh a9eb080 [cafreeman] IDE Shell Script 64d800c [dputler] Merge remote branch 'upstream/master' 1fbdb2e [dputler] Added the ability for the user to specify a text file location throught the use of tilde expansion or just the file name if it is in the working directory. d83c017 [Shivaram Venkataraman] Merge pull request #113 from sun-rui/stringHashCodeInC a7d9cdb [Sun Rui] Fix build on Windows. 7d81b05 [Shivaram Venkataraman] Merge pull request #114 from hlin09/hlin09 47c4bb7 [hlin09] fix reviews a457f7f [Shivaram Venkataraman] Merge pull request #116 from dputler/master 0fa48d1 [Shivaram Venkataraman] Merge pull request #117 from sun-rui/keyBy 85cfeb4 [Sun Rui] [SPARKR-144] Implement saveAsTextFile() in the RDD class. 09083d9 [Sun Rui] Add keyBy() to the RDD class. caad5d7 [dputler] Adding the script to install software on the Cloudera Quick Start VM. dca3d05 [hlin09] Minor fix. ece5f7d [hlin09] Merge remote-tracking branch 'upstream/master' into hlin09 a40874b [hlin09] Use extendible accumulators aggregate the cogroup values. d0347ce [Zongheng Yang] Merge pull request #112 from sun-rui/outer_join 492f76e [Sun Rui] Refine code and add description. ba01358 [Shivaram Venkataraman] Merge pull request #115 from sun-rui/SPARKR-130 5c8e46e [Sun Rui] Fix per the review comments. 7190a2c [Sun Rui] Update comment to add a reference to storage levels. 1da705e [hlin09] Fix the review comments. c4b77be [Sun Rui] [SPARKR-130] Add persist(storageLevel) API to RDD. b424a1a [hlin09] Add function cogroup(). 9770312 [Shivaram Venkataraman] Merge pull request #111 from hlin09/hlin09 cead7df [hlin09] fix review comments. 54f712e [Sun Rui] Implement string hash code in C. 425f0c6 [Sun Rui] Add leftOuterJoin() and rightOuterJoin() to the RDD class. 39509c7 [hlin09] add Rd file for foreach and foreachPartition. 63d6ac7 [hlin09] Adds function foreach() and foreachPartition(). 9c954df [Zongheng Yang] Merge pull request #105 from sun-rui/join c71228d [Sun Rui] Pre-allocate list with fixed length. Add test case for join() using string key. bc3e9f6 [Shivaram Venkataraman] Merge pull request #108 from concretevitamin/take-optimize c06fc90 [Zongheng Yang] Fix: only optimize for unserialized dataset case. d399aeb [Zongheng Yang] Apply size-capping on logical representation instead of physical. e4217dd [Zongheng Yang] Merge pull request #107 from shivaram/master 7952180 [Shivaram Venkataraman] Copy, use getLocalDirs from Spark Utils.scala 08e24c3 [Zongheng Yang] Merge pull request #109 from hlin09/hlin09 97d4e02 [Zongheng Yang] Min() upper-bound size with actual size. bb779bf [hlin09] Rename the filter function to filterRDD to follow the API consistency. Filter() is also kept. ce1661f [Zongheng Yang] Fix slow take(): deserialize only up to necessary # of elements. 4dca9b1 [Shivaram Venkataraman] Merge pull request #106 from hlin09/hlin09 1220d92 [hlin09] Adds function numPartitions(). 2326a65 [Shivaram Venkataraman] Use SPARK_LOCAL_DIRS to create tmp files e119757 [hlin09] Minor fix. 9c24c8b [hlin09] Adds function countByKey(). 48fce67 [hlin09] Adds countByValue(). 6679eef [Sun Rui] Update documentation for join(). 70586b4 [Sun Rui] Add join() to the RDD class. e6fb999 [Zongheng Yang] Merge pull request #103 from shivaram/rlibdir-fix a21f146 [Shivaram Venkataraman] Merge pull request #102 from hlin09/hlin09 32eb619 [Shivaram Venkataraman] Merge pull request #104 from sun-rui/add_keys_values d8692e9 [Sun Rui] Add keys() and values() for the RDD class. 18b9be1 [Shivaram Venkataraman] Allow users to set where SparkR is installed This also adds a warning if somebody tries to call sparkR.init multiple times. a17f135 [hlin09] Adds tests for flatMap and flatMapValues. 4bcf59b [hlin09] Adds function flatMapValues. 4a193ef [Zongheng Yang] Merge pull request #101 from ashutoshraina/master 60d22f2 [Ashutosh Raina] changed sbt version 5400793 [Zongheng Yang] Merge pull request #98 from shivaram/windows-fixes-build 36d61a7 [Shivaram Venkataraman] Merge pull request #97 from hlin09/hlin09 f7d7d89 [hlin09] Remove redundant code in test. 6bbe823 [hlin09] minor style fix. 9b47f3a [Shivaram Venkataraman] Merge pull request #100 from hnahak87/patch-1 7f6e4ea [Harihar Nahak] Update logistic_regression.R a605047 [Shivaram Venkataraman] Merge pull request #99 from hlin09/makefile 323151d [hlin09] Fix yar flag in Makefile to remove build error in Maven. 8911897 [hlin09] Make reserialize() private function in package. 79aee73 [Shivaram Venkataraman] Add notes on how to build SparkR on windows 49a99e7 [Shivaram Venkataraman] Clean up some commented code ddc271b [Shivaram Venkataraman] Only append file:/// to non empty jar paths a53952e [Shivaram Venkataraman] Add windows build scripts 325b179 [hlin09] Merge remote-tracking branch 'upstream/master' into hlin09 daf5040 [hlin09] Add reserialize() before union if two RDDs are not both serialized. 536afb1 [hlin09] Add new function of union(). 7044677 [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into windows-fixes d22a02d [Zongheng Yang] Merge pull request #94 from shivaram/windows-fixes-stdin 51924f7 [Shivaram Venkataraman] Merge pull request #90 from oscaroboto/master eb97d85 [Shivaram Venkataraman] Merge pull request #96 from sun-rui/add_clarification_readme 5a128f4 [Sun Rui] Add clarification on setting Spark master when launching the SparkR shell. 187526a [oscaroboto] Update sparkR.R 32c567b [Shivaram Venkataraman] Merge pull request #95 from concretevitamin/master 4cd2d5e [Zongheng Yang] Notes about spark-ec2. 1c28e3b [Shivaram Venkataraman] Merge branch 'master' of https://github.com/amplab-extras/SparkR-pkg into windows-fixes 8e8a029 [Zongheng Yang] Merge pull request #92 from shivaram/sparkr-yarn 721043b [Zongheng Yang] Update README.md with YARN instructions. 1681f58 [Shivaram Venkataraman] Use temporary files for input instead of stdin This fixes a bug for Windows where stdin would get truncated b084314 [oscaroboto] removed ... from example 44c93d4 [oscaroboto] Added example to SparkR.R be82dcc [Shivaram Venkataraman] Merge pull request #93 from hlin09/hlin09 868554d [oscaroboto] Update sparkR.R 488ac47 [hlin09] Add generated Rd file of previous added functions, distinct() and mapValues(). b2740ad [hlin09] Add test for filter all elements. Add filter() as alias. 08d3631 [hlin09] Minor style fixes. 2c0e34f [hlin09] Adds function Filter(), which extracts the elements that satisfy a predicate. 5951d3b [Shivaram Venkataraman] Remove SBT plugin 4e70ced [oscaroboto] changed ExecutorEnv to sparkExecutorEnvMap, to make it consistent with sparkEnvirMap 903d18a [oscaroboto] changed executorEnv to sparkExecutorEnvMap, will do the same in R f97346e [oscaroboto] executorEnv to lower-case e 88a524e [oscaroboto] Added LD_LIBRARY_PATH to the ExecutorEnv. This is need so that the nodes can find libjvm.so, or if the master has a different LD_LIBRARY_PATH then the nodes. Make sure to export LD_LIBRARY_PATH that includes the path to libjvm.so in the nodes. 1d208ae [oscaroboto] added the YARN_CONF_DIR to the classpath 8a9b75c [oscaroboto] forgot to change hm and ee inside the for loops 579db58 [Shivaram Venkataraman] Merge pull request #91 from sun-rui/add_max_min 4381efa [Sun Rui] use reduce() to implemement max() and min(). a5459c5 [Shivaram Venkataraman] Consolidate yarn flags 86b04eb [Shivaram Venkataraman] Don't use quotes around yarn bf0797f [Shivaram Venkataraman] Add dependency on spark yarn module af5fe77 [Shivaram Venkataraman] Fix SBT build, add dependency tree plugin 4917607 [Sun Rui] Add maximum() and minimum() API to RDD. 51bbbe4 [Shivaram Venkataraman] Changes to make SparkR work with YARN 9d5e3ab [oscaroboto] a few stylistic changes. Also change vars to sparkEnvirMap and eevars to ExecutorEnv, to match sparkR.R 578f545 [oscaroboto] a few stylistic changes 39eea2f [oscaroboto] Modification to dynamically create a sparkContext with YARN. Added .setExecutorEnv to the sparkConf in createSparkContext within the RRDD object. This modification was made together with sparkR.R 17ec42e [oscaroboto] A modification to dynamically create a sparkContext with YARN. sparkR.R modified to pass custom Jar file names and EnvironmentEnv to the sparkConf. RRDD.scala was also modified to accept the new inputs to creatSparkContext. 624ac9d [Shivaram Venkataraman] Merge pull request #87 from sun-rui/SPARKR-125 4f213db [Shivaram Venkataraman] Merge pull request #89 from sun-rui/SPARKR-108 eb833c5 [Shivaram Venkataraman] Merge pull request #88 from hlin09/hlin09 07bf971 [Sun Rui] [SPARKR-108] Implement map-side reduction for reduceByKey(). 4accba1 [hlin09] Fixes style and adds an optional param 'numPartition' in distinct(). 80d303a [hlin09] typo fixed. e37a9b5 [hlin09] Adds function distinct() and mapValues(). 08dac06 [Sun Rui] [SPARKR-125] Get the iterator of the parent RDD before launching a R worker process in compute() of RRDD/PairwiseRRDD c4ba53c [Shivaram Venkataraman] Merge pull request #85 from edwardt/master 72a9d27 [root] reorder to keep relative ordering the same f3fcb10 [root] fix up build.sbt also to match pom.xml 5ecbe3e [root] Make spark verison configurable in build script per ISSUE122 a44e63d [Shivaram Venkataraman] Merge pull request #84 from sun-rui/SPARKR-94 fbb5663 [Sun Rui] Add {} to one-line functions and add a test case for lookup where no match is found. 95beb4e [Shivaram Venkataraman] Merge pull request #82 from edwardt/master 36776c5 [edwardt] missed one 0.9.0 revert b26deec [Sun Rui] [SPARKR-94] Add a method to get an element of a pair RDD object by key. 1ba256e [edwardt] Keep 0.9.0 and says uses 1.1.0 by default 5380c43 [root] missed one version 21f74da [root] upgrade to spark version 1.1.0 to match lastest merge list ddfcde9 [root] merge 67d067a [Shivaram Venkataraman] Merge pull request #81 from sun-rui/SparkR-117 993868f [Sun Rui] [SPARKR-117] Update Spark dependency to 1.1.0 d20661a [Zongheng Yang] Merge pull request #80 from sun-rui/master 0b2da9f [Sun Rui] Update Rd file and add a test case for mapPartitions. 5879648 [Sun Rui] Add mapPartitions() method to RDD for API consistency. c033461 [Shivaram Venkataraman] Merge pull request #79 from sun-rui/fix-kmeans f62b77e [Sun Rui] Adjust coding style. b40911d [Sun Rui] Fix syntax error in examples/kmeans.R. 5304451 [Shivaram Venkataraman] Merge pull request #78 from sun-rui/master 70ffbfb [Sun Rui] Fix a bug that modifications to build.sbt won't trigger rebuilding. a25696c [Shivaram Venkataraman] Merge pull request #76 from edwardt/addjira b8bbd93 [edwardt] Update README.md 615d930 [edwardt] Update README.md e522e69 [edwardt] Update README.md 03e6ced [edwardt] Update README.md 3007015 [root] don't check in gedit buffer file' c35c9a6 [root] Add where to enter bugs ad feeback 469eae3 [edwardt] Update README.md 61b4a43 [edwardt] Update Makefile (style uniformity) ce3337d [edwardt] Update README.md 7ff68fc [root] Merge branch 'master' of https://github.com/edwardt/SparkR-pkg 16353f5 [root] add links to devtools and install_github 513b9e5 [Shivaram Venkataraman] Merge pull request #72 from edwardt/master 31608a4 [edwardt] Update Makefile (style uniformity) 4ffe146 [root] Makefile: factor out SPARKR_VERSION to reduce potential copy&paste error; cp & rm called with -f in build/clean phase; .gitignore includes checkpoints and unit test log generated by run-tests.sh 715275f [Zongheng Yang] Merge pull request #68 from shivaram/master 90e2083 [Shivaram Venkataraman] Add return type to hasNext 8eb983d [Shivaram Venkataraman] Fix up comment 2206164 [Shivaram Venkataraman] Delete temporary files after they are read This change deletes temporary files used for communication between Rscript and the JVM once they have been completely read. 5881da7 [Zongheng Yang] Merge pull request #67 from shivaram/improve-shuffle 81251e2 [Shivaram Venkataraman] Address code review comments a5f573f [Shivaram Venkataraman] Use a better list append in shuffles This is helpful in scenarios where we have a large number of values in a bucket 388e64d [Shivaram Venkataraman] Merge pull request #55 from RevolutionAnalytics/master e1f95b6 [Zongheng Yang] Merge pull request #65 from concretevitamin/parallelize-fix fc1a71a [Zongheng Yang] Fix that collect(parallelize(sc,1:72,15)) drops elements. b8204c5 [Zongheng Yang] Minor: update a URL in README. 86f30c3 [Antonio Piccolboni] better fix for amplab-extras/SparkR-pkg#53 b3c318d [Antonio Piccolboni] delayed loading to have all namespaces available. f323e97 [Antonio Piccolboni] tentative fix for amplab-extras/SparkR-pkg#53 6f82269 [Zongheng Yang] Merge pull request #48 from shivaram/master 8f433e5 [Shivaram Venkataraman] Move up Hadoop in pom.xml and add back protobufs As Hadoop 1.0.4 doesn't use protobufs, we can't exclude protobufs from Spark always. This change tries to order the dependencies so that the shader first picks up Hadoop's protobufs over Mesos. bfe7e26 [Shivaram Venkataraman] Merge pull request #36 from RevolutionAnalytics/vectorize-examples 059ae41 [Antonio Piccolboni] and more formatting 9dbd531 [Antonio Piccolboni] more formatting per committer request 948738a [Antonio Piccolboni] converted tabs to spaces per project request 49f5f5a [Shivaram Venkataraman] Merge pull request #35 from shivaram/master 3eb5ad3 [Shivaram Venkataraman] on_failure -> after_failure in travis.yml 139bdee [Shivaram Venkataraman] Cache sbt, maven, ivy dependencies 4ebced2 [Shivaram Venkataraman] Merge pull request #34 from shivaram/master 8437061 [Shivaram Venkataraman] Exclude protobuf from Spark dependency in Maven This avoids pulling in multiple versions of protobuf from Mesos and Hadoop. 91aa527 [Antonio Piccolboni] vectorized version, 36s 10 slices 10^6 per slice. The older version takes 30 sec on 1/10th of data. f137a57 [Antonio Piccolboni] for rstudio users 1f7ffb0 [Antonio Piccolboni] implemented using matrices and vectorized calls wherever possible 46b23df [Antonio Piccolboni] replace require with library b15d7db [Antonio Piccolboni] faster parsing 8b7aeb3 [Antonio Piccolboni] 22x speed improvement, 3X mem impovement c5bce07 [Zongheng Yang] Merge pull request #30 from shivaram/string-tests 21fa2d8 [Shivaram Venkataraman] Fix bug where serialized was not changed for RRRD Reason: When an RRDD is created in getJRDD we have converted any possibly unserialized RDD to a serialized RDD. 9d1ea20 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg into string-tests 7b9348c [Shivaram Venkataraman] Add tests for partition with string keys Add two tests one with a string array and one from a textFile to test both codepaths aacd726 [Shivaram Venkataraman] Update README with maven proxy instructions 803e62c [Shivaram Venkataraman] Merge pull request #28 from concretevitamin/master 7c093e6 [Zongheng Yang] Use inherits() to test an object's class. 061c591 [Shivaram Venkataraman] Merge pull request #26 from hafen/master 90f9fda [Ryan Hafen] Fix isRdd() to properly check for class 5b10cc7 [Zongheng Yang] Merge pull request #24 from shivaram/master 7014f83 [Shivaram Venkataraman] Remove unused transformers in maven's pom.xml b00cea5 [Shivaram Venkataraman] Add support for a Maven build 11ec9b2 [Shivaram Venkataraman] Merge pull request #12 from concretevitamin/pipelined 6b18a90 [Zongheng Yang] Merge branch 'master' into pipelined 57127b8 [Zongheng Yang] Merge pull request #23 from shivaram/master 1ac3940 [Zongheng Yang] Review feedback. a06fb34 [Zongheng Yang] Remove outdated comment. 0a1fc13 [Shivaram Venkataraman] Fixes for using SparkR with Hadoop2. 1. Exclude ASM, Netty from Hadoop similar to Spark. 2. Concat services files to ensure HDFS filesystems work. 3. Update README with an example 9a1db44 [Zongheng Yang] Merge pull request #22 from shivaram/master e462448 [Shivaram Venkataraman] Use `$` for calling `put` instead of .jrcall ed4559a [Shivaram Venkataraman] Add support for passing Spark environment vars This change creates a new `createSparkContext` method in RRDD as we can't pass Map through rJava. Also use SPARK_MEM in local mode to increase heap size and update the README with some examples. 10228fb [Shivaram Venkataraman] Merge pull request #20 from concretevitamin/digit-ex 1398d9f [Zongheng Yang] Add linear_solver_mnist to examples/. d484c2a [Zongheng Yang] Add tests for actions on PipelinedRDD. d9cb95c [Zongheng Yang] Add setCheckpointDir() to context.R; comment fix. f8bc8a9 [Zongheng Yang] Minor edits per Shivaram's comments. 8cd67f7 [Shivaram Venkataraman] Merge pull request #15 from shivaram/master d4468a9 [Shivaram Venkataraman] Remove trailing comma e2714b8 [Shivaram Venkataraman] Remove Apache Staging repo and update README 334eace [Zongheng Yang] Add a multi-transformation test to benchmark on pipelining. 5650ad7 [Zongheng Yang] Put serialized field inside env for both RDD and PipelinedRDD. 0b9e8bb [Zongheng Yang] First cut at PipelinedRDD. a4c431e [Zongheng Yang] Add `isCheckpointed` field and checkpoint(). dac0795 [Zongheng Yang] Minor inline comment style fix. bfb8e26 [Zongheng Yang] Add isCached field (inside an env) and unpersist(). 295bff6 [Zongheng Yang] Merge pull request #11 from shivaram/master 4cb209c [Shivaram Venkataraman] Search rLibDir in worker before libPaths This ensures we pick up the SparkR intended and not an older version installed on the same machine ef198ff [Zongheng Yang] Merge pull request #10 from shivaram/unit-tests e0557a8 [Shivaram Venkataraman] Update travis to install plyr 8b18bc1 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg into unit-tests 4a9ca31 [Shivaram Venkataraman] Use smaller broadcast and plyr instead of Matrix Matrix package takes around 2s to load and slows down unit tests. 21c6a61 [Zongheng Yang] Merge pull request #8 from shivaram/master 08c2947 [Shivaram Venkataraman] Move dev install directory to front of libPaths bda42ee [Shivaram Venkataraman] Merge pull request #7 from JoshRosen/travis cc5f5c0 [Josh Rosen] Add Travis CI integration (using craigcitro/r-travis) b6c864b [Shivaram Venkataraman] Merge pull request #6 from concretevitamin/env-style-fix 4fcef22 [Zongheng Yang] Use one style ($) for accessing names in environments. 8a948c6 [Shivaram Venkataraman] Merge pull request #4 from shivaram/master 24978eb [Shivaram Venkataraman] Update README to use install_github 8899db4 [Shivaram Venkataraman] Update TODO.md 91792de [Shivaram Venkataraman] Update Spark requirements f34f4bf [Shivaram Venkataraman] Check tests for failures and output error msg cd750d3 [Shivaram Venkataraman] Update run-tests to use new path 1877b7c [Shivaram Venkataraman] Unset R_TESTS to make tests work with R CMD check Also silence Akka remoting logs and update Makefile to build on log4j changes e60e18a [Shivaram Venkataraman] Update README to remove Spark installation notes 4450189 [Shivaram Venkataraman] Add Spark 0.9 dependency from Apache Staging Also clean up assembly jar from inst on make clean 5eb2131 [Shivaram Venkataraman] Update repo path in README ec8210e [Shivaram Venkataraman] Remove broadcastId hack as it is public in Spark 9f0e080 [Shivaram Venkataraman] Merge branch 'install-github' 5c88fbd [Shivaram Venkataraman] Add helper script to run tests 77450a1 [Shivaram Venkataraman] Remove dependency on Spark Logging 6cb00d1 [Shivaram Venkataraman] Update README and add helper script install-dev.sh 28346ca [Shivaram Venkataraman] Only normalize if SPARK_HOME is not empty 0fd6571 [Shivaram Venkataraman] Normalize SPARK_HOME before passing it ff96d5c [Shivaram Venkataraman] Pass in SPARK_HOME and jar file path 34c4dce [Shivaram Venkataraman] Move src into pkg and update Makefile This enables the package to be installed using install_github using devtools and automates the build procedure. b25afed [Shivaram Venkataraman] Change package name to edu.berkeley.cs.amplab c691464 [Shivaram Venkataraman] Add Apache 2.0 License file 27a4a4b [Shivaram Venkataraman] Add notes on how to compile roxygen2 docs ca63844 [Shivaram Venkataraman] Add broadcast documentation Also generate documentation for sample, takeSample etc. e4dd976 [Shivaram Venkataraman] Update TODO.md e42d435 [Shivaram Venkataraman] Add support for broadcast variables 6b638e7 [Shivaram Venkataraman] Add the assembly jar to SparkContext bf24e32 [Shivaram Venkataraman] Merge branch 'master' of github.com:amplab/SparkR-pkg 43c05ce [Zongheng Yang] Fix a flaky/incorrect test for sampleRDD(). c6a9dfc [Zongheng Yang] Initial port of the kmeans example. 6885581 [Zongheng Yang] Implement element-level sampleRDD() and takeSample() with tests. d3a4987 [Zongheng Yang] Add a test for lapplyPartitionsWithIndex on pairwise RDD. c7899c1 [Zongheng Yang] Add lapplyPartitionsWithIndex, with a test and an alias function. a9a7436 [Shivaram Venkataraman] Add DFC example from Tselil, Benjamin and Jonah fbc5a95 [Zongheng Yang] Implement take() and takeSample(). c4a3409 [Shivaram Venkataraman] Use RDD instead of RRDD dfad3f5 [Zongheng Yang] Add test_utils.R: a unit test for convertJListToRList(). a45227d [Zongheng Yang] Update .gitignore. 238fe6e [Zongheng Yang] Add a unit test for textFile(). a88898b [Zongheng Yang] Rename test_rrd to test_rrdd 10c8baa [Shivaram Venkataraman] Make SparkR work as a standalone package. Changes include: 1. Adding a new `sbt` project that builds RRDD.scala 2. Change the onLoad functions to load the assembly jar for SparkR 3. Set rLibDir in RRDD.scala and worker.R to load things correctly 78adcd8 [Shivaram Venkataraman] Add a gitignore ca6108f [Shivaram Venkataraman] Merge branch 'SparkR-scalacode' of ../SparkR 999bd61 [Shivaram Venkataraman] Update collectPartition in R and use ClassTag c58f63e [Shivaram Venkataraman] Update collectPartition in R and use ClassTag 48265fd [Shivaram Venkataraman] Use new version of collectPartitions in take d4fe086 [Shivaram Venkataraman] Move collectPartitions to JavaRDDLike Also remove numPartitions in JavaRDD and update R code bfecd7b [Shivaram Venkataraman] Scala 2.10 changes 1. Update sparkR script 2. Use classTag instead of classManifest 092a4b3 [Shivaram Venkataraman] Add combineByKey, update TODO ac0d81d [Shivaram Venkataraman] Add more documentation d1dc3fa [Shivaram Venkataraman] Add more documentation c515e3a [Shivaram Venkataraman] Update TODO db56a34 [Shivaram Venkataraman] Add a test case for include package 41cea51 [Shivaram Venkataraman] Ensure all parent environments are serialized. Also add a test case with an inline function a978e84 [Shivaram Venkataraman] Add support to include packages in the worker 12bf8ce [Shivaram Venkataraman] Add support to include packages in the worker fb7e72c [Shivaram Venkataraman] Cleanup TODO 16ac314 [Shivaram Venkataraman] Add documentation for functions in context, sparkR 85b1d25 [Shivaram Venkataraman] Set license to Apache 88f1101 [Shivaram Venkataraman] Add unit test running instructions c40768e [Shivaram Venkataraman] Update TODO 0c7efbf [Shivaram Venkataraman] Refactor RRDD.scala and add comments to functions 5880d42 [Shivaram Venkataraman] Refactor RRDD.scala and add comments to functions 2dee36c [Shivaram Venkataraman] Remove empty test file a82219b [Shivaram Venkataraman] Update TODOs 5db00dc [Shivaram Venkataraman] Add reduceByKey, groupByKey and refactor shuffle Other changes include 1. Adding unit tests for basic RDD functions and shuffle 2. Add a word count example 3. Change the dependency serialization to handle double loading of SparkR package 4. Allow partitionBy to operate on any RDDs to create pair-wise RDD. f196479 [Shivaram Venkataraman] Add reduceByKey, groupByKey and refactor shuffle Other changes include 1. Adding unit tests for basic RDD functions and shuffle 2. Add a word count example 3. Change the dependency serialization to handle double loading of SparkR package 4. Allow partitionBy to operate on any RDDs to create pair-wise RDD. 987e36f [Shivaram Venkataraman] Add perf todo 0b03265 [Shivaram Venkataraman] Update TODO with testing, docs todo 685aaad [Zongheng Yang] First cut at refactoring worker.R. Remove pairwiseWorker.R. 95b9ddc [Zongheng Yang] First cut at refactoring worker.R. Remove pairwiseWorker.R. 4f00895 [Zongheng Yang] Remove the unnecessary `pairwise' flag in RRDD class. Reasons: 75d36d9 [Zongheng Yang] Working versions: partitionBy() and collectPartition() for RRDD. e3fbd9d [Zongheng Yang] Working versions: partitionBy() and collectPartition() for RRDD. 67a4335 [Zongheng Yang] Add unit test for parallelize() and collect() pairwise data. 100ae65 [Zongheng Yang] Properly parallelize() and collect() pairwise data. cd0a5e2 [Zongheng Yang] Properly parallelize() and collect() pairwise data. aea16c3 [Zongheng Yang] WIP: second cut at partitionBy. Running into R/Scala communication issues. 45eb943 [Zongheng Yang] WIP: second cut at partitionBy. Running into R/Scala communication issues. 11c893b [Zongheng Yang] WIP: need to figure out the logic of (whether or not) shipping a hash func 82c201a [Zongheng Yang] WIP: need to figure out the logic of (whether or not) shipping a hash func b3bfad2 [Zongheng Yang] Update TODO: take() done. 0e45293 [Zongheng Yang] Add ability to parallelize key-val collections in R. f60406a [Zongheng Yang] Add ability to parallelize key-val collections in R. 7d7fe3b [Zongheng Yang] Re-implement take(): take a partition at a time and append. a054e55 [Zongheng Yang] Fix take() tests(): mode difference. 9de0935 [Zongheng Yang] Implement take() for RRDD. 1e4427e [Zongheng Yang] Implement take() for RRDD. ec3cd67 [Shivaram Venkataraman] Use temp file in Spark to pipe output 417aaed [Shivaram Venkataraman] Use temp file in Spark to pipe output bb0a3c3 [Shivaram Venkataraman] Add conf directory to classpath 9594d8a [Shivaram Venkataraman] Clean up LR example 3b26b58 [Shivaram Venkataraman] Add a list of things to do. cabce68 [Shivaram Venkataraman] Fix warnings from package check fde3f9c [Shivaram Venkataraman] Flatten by default and disable recursive unlist ab2e061 [Shivaram Venkataraman] Create LIB_DIR before installing SparkR package 555220a [Shivaram Venkataraman] Add readme and update Makefile 1319cda [Shivaram Venkataraman] Make standalone programs run with sparkR ae19fa8 [Shivaram Venkataraman] Add support for cache and use `tempfile` 4e89ca4 [Shivaram Venkataraman] Add support for apply, reduce, count Also serialize closures using `save` and add two examples 25a0bea [Shivaram Venkataraman] Add support for apply, reduce, count Also serialize closures using `save` and add two examples f50223f [Zongheng Yang] Make parallelize() and collect() use lists. Add a few more tests for them. fc7693f [Zongheng Yang] Refactor and enhance the previously added unit test a little bit. 6de9b81 [Zongheng Yang] Add a simple unit test for parallelize(). 8b95155 [Zongheng Yang] Add testthat skeleton infrastructure ef305bf [Zongheng Yang] parallelize() followed by collect() now work for vectors/lists of strings and numerics (should work for other primitives as well). dc16af4 [Zongheng Yang] Comment: toArray() allocates memory for a copy f50121e [Zongheng Yang] Make parallelize() return JavaRDD[Array[Byte]]. Add RRDD.scala with a helper function in the singleton object. 46eb063 [Zongheng Yang] Make parallelize() return JavaRDD[Array[Byte]]. Add RRDD.scala with a helper function in the singleton object. 6b4938a [Zongheng Yang] parallelize(): a raw can be parallelized by JavaSparkContext and get back JavaRDD 978aa0f [Zongheng Yang] Add parallelize() skeleton: only return serialized slices now 84c1fd2 [Zongheng Yang] Use .jsimplify() to get around generic List's get() type erasure problem f16b891 [Zongheng Yang] Convert a few reflectionc alls to .jcall 1284c13 [Zongheng Yang] WIP on collect(): JavaListToRList() failed with errors. 4c2e516 [Zongheng Yang] Add simple prototype of S4 class RRDD. Make TextFile() returns an RRDD. 82aa17a [Zongheng Yang] Add textFile() 83ce63f [Zongheng Yang] Create a JavaSparkContext and save it in .sparkEnv using sparkR.init() 01cdf0e [Zongheng Yang] Add Makefile for SparkR fc9cae2 [Shivaram Venkataraman] Add skeleton R package --- .gitignore | 2 + .rat-excludes | 2 + R/.gitignore | 6 + R/DOCUMENTATION.md | 12 + R/README.md | 67 + R/WINDOWS.md | 13 + R/create-docs.sh | 46 + R/install-dev.bat | 27 + R/install-dev.sh | 36 + R/log4j.properties | 28 + R/pkg/DESCRIPTION | 35 + R/pkg/NAMESPACE | 182 ++ R/pkg/R/DataFrame.R | 1270 ++++++++++++++ R/pkg/R/RDD.R | 1539 +++++++++++++++++ R/pkg/R/SQLContext.R | 520 ++++++ R/pkg/R/SQLTypes.R | 64 + R/pkg/R/backend.R | 115 ++ R/pkg/R/broadcast.R | 86 + R/pkg/R/client.R | 57 + R/pkg/R/column.R | 199 +++ R/pkg/R/context.R | 225 +++ R/pkg/R/deserialize.R | 184 ++ R/pkg/R/generics.R | 543 ++++++ R/pkg/R/group.R | 132 ++ R/pkg/R/jobj.R | 101 ++ R/pkg/R/pairRDD.R | 789 +++++++++ R/pkg/R/serialize.R | 195 +++ R/pkg/R/sparkR.R | 266 +++ R/pkg/R/utils.R | 467 +++++ R/pkg/R/zzz.R | 21 + R/pkg/inst/profile/general.R | 22 + R/pkg/inst/profile/shell.R | 31 + R/pkg/inst/tests/test_binaryFile.R | 90 + R/pkg/inst/tests/test_binary_function.R | 68 + R/pkg/inst/tests/test_broadcast.R | 48 + R/pkg/inst/tests/test_context.R | 50 + R/pkg/inst/tests/test_includePackage.R | 57 + R/pkg/inst/tests/test_parallelize_collect.R | 109 ++ R/pkg/inst/tests/test_rdd.R | 644 +++++++ R/pkg/inst/tests/test_shuffle.R | 209 +++ R/pkg/inst/tests/test_sparkSQL.R | 695 ++++++++ R/pkg/inst/tests/test_take.R | 67 + R/pkg/inst/tests/test_textFile.R | 162 ++ R/pkg/inst/tests/test_utils.R | 137 ++ R/pkg/inst/worker/daemon.R | 52 + R/pkg/inst/worker/worker.R | 128 ++ R/pkg/src/Makefile | 27 + R/pkg/src/Makefile.win | 27 + R/pkg/src/string_hash_code.c | 49 + R/pkg/tests/run-all.R | 21 + R/run-tests.sh | 39 + bin/sparkR | 39 + bin/sparkR.cmd | 23 + bin/sparkR2.cmd | 26 + core/pom.xml | 51 + .../org/apache/spark/api/r/RBackend.scala | 145 ++ .../apache/spark/api/r/RBackendHandler.scala | 223 +++ .../scala/org/apache/spark/api/r/RRDD.scala | 450 +++++ .../scala/org/apache/spark/api/r/SerDe.scala | 340 ++++ .../org/apache/spark/deploy/RRunner.scala | 92 + .../org/apache/spark/deploy/SparkSubmit.scala | 73 +- .../spark/deploy/SparkSubmitArguments.scala | 8 +- dev/run-tests | 15 + dev/run-tests-codes.sh | 1 + dev/run-tests-jenkins | 2 + docs/README.md | 12 +- docs/_layouts/global.html | 1 + docs/_plugins/copy_api_dirs.rb | 15 +- examples/src/main/r/kmeans.R | 93 + examples/src/main/r/linear_solver_mnist.R | 107 ++ examples/src/main/r/logistic_regression.R | 62 + examples/src/main/r/pi.R | 46 + examples/src/main/r/wordcount.R | 42 + .../spark/launcher/CommandBuilderUtils.java | 8 +- .../launcher/SparkSubmitCommandBuilder.java | 87 +- .../launcher/CommandBuilderUtilsSuite.java | 6 +- pom.xml | 3 + .../org/apache/spark/sql/GroupedData.scala | 2 +- .../org/apache/spark/sql/api/r/SQLUtils.scala | 127 ++ .../spark/deploy/yarn/ApplicationMaster.scala | 3 + .../yarn/ApplicationMasterArguments.scala | 11 + .../org/apache/spark/deploy/yarn/Client.scala | 13 +- .../spark/deploy/yarn/ClientArguments.scala | 11 + 83 files changed, 12043 insertions(+), 55 deletions(-) create mode 100644 R/.gitignore create mode 100644 R/DOCUMENTATION.md create mode 100644 R/README.md create mode 100644 R/WINDOWS.md create mode 100755 R/create-docs.sh create mode 100644 R/install-dev.bat create mode 100755 R/install-dev.sh create mode 100644 R/log4j.properties create mode 100644 R/pkg/DESCRIPTION create mode 100644 R/pkg/NAMESPACE create mode 100644 R/pkg/R/DataFrame.R create mode 100644 R/pkg/R/RDD.R create mode 100644 R/pkg/R/SQLContext.R create mode 100644 R/pkg/R/SQLTypes.R create mode 100644 R/pkg/R/backend.R create mode 100644 R/pkg/R/broadcast.R create mode 100644 R/pkg/R/client.R create mode 100644 R/pkg/R/column.R create mode 100644 R/pkg/R/context.R create mode 100644 R/pkg/R/deserialize.R create mode 100644 R/pkg/R/generics.R create mode 100644 R/pkg/R/group.R create mode 100644 R/pkg/R/jobj.R create mode 100644 R/pkg/R/pairRDD.R create mode 100644 R/pkg/R/serialize.R create mode 100644 R/pkg/R/sparkR.R create mode 100644 R/pkg/R/utils.R create mode 100644 R/pkg/R/zzz.R create mode 100644 R/pkg/inst/profile/general.R create mode 100644 R/pkg/inst/profile/shell.R create mode 100644 R/pkg/inst/tests/test_binaryFile.R create mode 100644 R/pkg/inst/tests/test_binary_function.R create mode 100644 R/pkg/inst/tests/test_broadcast.R create mode 100644 R/pkg/inst/tests/test_context.R create mode 100644 R/pkg/inst/tests/test_includePackage.R create mode 100644 R/pkg/inst/tests/test_parallelize_collect.R create mode 100644 R/pkg/inst/tests/test_rdd.R create mode 100644 R/pkg/inst/tests/test_shuffle.R create mode 100644 R/pkg/inst/tests/test_sparkSQL.R create mode 100644 R/pkg/inst/tests/test_take.R create mode 100644 R/pkg/inst/tests/test_textFile.R create mode 100644 R/pkg/inst/tests/test_utils.R create mode 100644 R/pkg/inst/worker/daemon.R create mode 100644 R/pkg/inst/worker/worker.R create mode 100644 R/pkg/src/Makefile create mode 100644 R/pkg/src/Makefile.win create mode 100644 R/pkg/src/string_hash_code.c create mode 100644 R/pkg/tests/run-all.R create mode 100755 R/run-tests.sh create mode 100755 bin/sparkR create mode 100644 bin/sparkR.cmd create mode 100644 bin/sparkR2.cmd create mode 100644 core/src/main/scala/org/apache/spark/api/r/RBackend.scala create mode 100644 core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala create mode 100644 core/src/main/scala/org/apache/spark/api/r/RRDD.scala create mode 100644 core/src/main/scala/org/apache/spark/api/r/SerDe.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/RRunner.scala create mode 100644 examples/src/main/r/kmeans.R create mode 100644 examples/src/main/r/linear_solver_mnist.R create mode 100644 examples/src/main/r/logistic_regression.R create mode 100644 examples/src/main/r/pi.R create mode 100644 examples/src/main/r/wordcount.R create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala diff --git a/.gitignore b/.gitignore index d162fa9cca994..d54d21b802be8 100644 --- a/.gitignore +++ b/.gitignore @@ -63,6 +63,8 @@ ec2/lib/ rat-results.txt scalastyle.txt scalastyle-output.xml +R-unit-tests.log +R/unit-tests.out # For Hive metastore_db/ diff --git a/.rat-excludes b/.rat-excludes index 8c61e67a0c7d1..8aca5a7f7a967 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -67,3 +67,5 @@ logs .*scalastyle-output.xml .*dependency-reduced-pom.xml known_translations +DESCRIPTION +NAMESPACE diff --git a/R/.gitignore b/R/.gitignore new file mode 100644 index 0000000000000..9a5889ba28b2a --- /dev/null +++ b/R/.gitignore @@ -0,0 +1,6 @@ +*.o +*.so +*.Rd +lib +pkg/man +pkg/html diff --git a/R/DOCUMENTATION.md b/R/DOCUMENTATION.md new file mode 100644 index 0000000000000..931d01549b265 --- /dev/null +++ b/R/DOCUMENTATION.md @@ -0,0 +1,12 @@ +# SparkR Documentation + +SparkR documentation is generated using in-source comments annotated using using +`roxygen2`. After making changes to the documentation, to generate man pages, +you can run the following from an R console in the SparkR home directory + + library(devtools) + devtools::document(pkg="./pkg", roclets=c("rd")) + +You can verify if your changes are good by running + + R CMD check pkg/ diff --git a/R/README.md b/R/README.md new file mode 100644 index 0000000000000..a6970e39b55f3 --- /dev/null +++ b/R/README.md @@ -0,0 +1,67 @@ +# R on Spark + +SparkR is an R package that provides a light-weight frontend to use Spark from R. + +### SparkR development + +#### Build Spark + +Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#building-with-buildmvn) and include the `-PsparkR` profile to build the R package. For example to use the default Hadoop versions you can run +``` + build/mvn -DskipTests -Psparkr package +``` + +#### Running sparkR + +You can start using SparkR by launching the SparkR shell with + + ./bin/sparkR + +The `sparkR` script automatically creates a SparkContext with Spark by default in +local mode. To specify the Spark master of a cluster for the automatically created +SparkContext, you can run + + ./bin/sparkR --master "local[2]" + +To set other options like driver memory, executor memory etc. you can pass in the [spark-submit](http://spark.apache.org/docs/latest/submitting-applications.html) arguments to `./bin/sparkR` + +#### Using SparkR from RStudio + +If you wish to use SparkR from RStudio or other R frontends you will need to set some environment variables which point SparkR to your Spark installation. For example +``` +# Set this to where Spark is installed +Sys.setenv(SPARK_HOME="/Users/shivaram/spark") +# This line loads SparkR from the installed directory +.libPaths(c(file.path(Sys.getenv("SPARK_HOME"), "R", "lib"), .libPaths())) +library(SparkR) +sc <- sparkR.init(master="local") +``` + +#### Making changes to SparkR + +The [instructions](https://cwiki.apache.org/confluence/display/SPARK/Contributing+to+Spark) for making contributions to Spark also apply to SparkR. +If you only make R file changes (i.e. no Scala changes) then you can just re-install the R package using `R/install-dev.sh` and test your changes. +Once you have made your changes, please include unit tests for them and run existing unit tests using the `run-tests.sh` script as described below. + +#### Generating documentation + +The SparkR documentation (Rd files and HTML files) are not a part of the source repository. To generate them you can run the script `R/create-docs.sh`. This script uses `devtools` and `knitr` to generate the docs and these packages need to be installed on the machine before using the script. + +### Examples, Unit tests + +SparkR comes with several sample programs in the `examples/src/main/r` directory. +To run one of them, use `./bin/sparkR `. For example: + + ./bin/sparkR examples/src/main/r/pi.R local[2] + +You can also run the unit-tests for SparkR by running (you need to install the [testthat](http://cran.r-project.org/web/packages/testthat/index.html) package first): + + R -e 'install.packages("testthat", repos="http://cran.us.r-project.org")' + ./R/run-tests.sh + +### Running on YARN +The `./bin/spark-submit` and `./bin/sparkR` can also be used to submit jobs to YARN clusters. You will need to set YARN conf dir before doing so. For example on CDH you can run +``` +export YARN_CONF_DIR=/etc/hadoop/conf +./bin/spark-submit --master yarn examples/src/main/r/pi.R 4 +``` diff --git a/R/WINDOWS.md b/R/WINDOWS.md new file mode 100644 index 0000000000000..3f889c0ca3d1e --- /dev/null +++ b/R/WINDOWS.md @@ -0,0 +1,13 @@ +## Building SparkR on Windows + +To build SparkR on Windows, the following steps are required + +1. Install R (>= 3.1) and [Rtools](http://cran.r-project.org/bin/windows/Rtools/). Make sure to +include Rtools and R in `PATH`. +2. Install +[JDK7](http://www.oracle.com/technetwork/java/javase/downloads/jdk7-downloads-1880260.html) and set +`JAVA_HOME` in the system environment variables. +3. Download and install [Maven](http://maven.apache.org/download.html). Also include the `bin` +directory in Maven in `PATH`. +4. Set `MAVEN_OPTS` as described in [Building Spark](http://spark.apache.org/docs/latest/building-spark.html). +5. Open a command shell (`cmd`) in the Spark directory and run `mvn -DskipTests -Psparkr package` diff --git a/R/create-docs.sh b/R/create-docs.sh new file mode 100755 index 0000000000000..4194172a2e115 --- /dev/null +++ b/R/create-docs.sh @@ -0,0 +1,46 @@ +#!/bin/bash + +# +# 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. +# + +# Script to create API docs for SparkR +# This requires `devtools` and `knitr` to be installed on the machine. + +# After running this script the html docs can be found in +# $SPARK_HOME/R/pkg/html + +# Figure out where the script is +export FWDIR="$(cd "`dirname "$0"`"; pwd)" +pushd $FWDIR + +# Generate Rd file +Rscript -e 'library(devtools); devtools::document(pkg="./pkg", roclets=c("rd"))' + +# Install the package +./install-dev.sh + +# Now create HTML files + +# knit_rd puts html in current working directory +mkdir -p pkg/html +pushd pkg/html + +Rscript -e 'library(SparkR, lib.loc="../../lib"); library(knitr); knit_rd("SparkR")' + +popd + +popd diff --git a/R/install-dev.bat b/R/install-dev.bat new file mode 100644 index 0000000000000..008a5c668bc45 --- /dev/null +++ b/R/install-dev.bat @@ -0,0 +1,27 @@ +@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 + +rem Install development version of SparkR +rem + +set SPARK_HOME=%~dp0.. + +MKDIR %SPARK_HOME%\R\lib + +R.exe CMD INSTALL --library="%SPARK_HOME%\R\lib" %SPARK_HOME%\R\pkg\ diff --git a/R/install-dev.sh b/R/install-dev.sh new file mode 100755 index 0000000000000..55ed6f4be1a4a --- /dev/null +++ b/R/install-dev.sh @@ -0,0 +1,36 @@ +#!/bin/bash + +# +# 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 scripts packages the SparkR source files (R and C files) and +# creates a package that can be loaded in R. The package is by default installed to +# $FWDIR/lib and the package can be loaded by using the following command in R: +# +# library(SparkR, lib.loc="$FWDIR/lib") +# +# NOTE(shivaram): Right now we use $SPARK_HOME/R/lib to be the installation directory +# to load the SparkR package on the worker nodes. + + +FWDIR="$(cd `dirname $0`; pwd)" +LIB_DIR="$FWDIR/lib" + +mkdir -p $LIB_DIR + +# Install R +R CMD INSTALL --library=$LIB_DIR $FWDIR/pkg/ diff --git a/R/log4j.properties b/R/log4j.properties new file mode 100644 index 0000000000000..701adb2a3da1d --- /dev/null +++ b/R/log4j.properties @@ -0,0 +1,28 @@ +# +# 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. +# + +# Set everything to be logged to the file target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=true +log4j.appender.file.file=R-unit-tests.log +log4j.appender.file.layout=org.apache.log4j.PatternLayout +log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + +# Ignore messages below warning level from Jetty, because it's a bit verbose +log4j.logger.org.eclipse.jetty=WARN +org.eclipse.jetty.LEVEL=WARN diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION new file mode 100644 index 0000000000000..1842b97d43651 --- /dev/null +++ b/R/pkg/DESCRIPTION @@ -0,0 +1,35 @@ +Package: SparkR +Type: Package +Title: R frontend for Spark +Version: 1.4.0 +Date: 2013-09-09 +Author: The Apache Software Foundation +Maintainer: Shivaram Venkataraman +Imports: + methods +Depends: + R (>= 3.0), + methods, +Suggests: + testthat +Description: R frontend for Spark +License: Apache License (== 2.0) +Collate: + 'generics.R' + 'jobj.R' + 'SQLTypes.R' + 'RDD.R' + 'pairRDD.R' + 'column.R' + 'group.R' + 'DataFrame.R' + 'SQLContext.R' + 'broadcast.R' + 'context.R' + 'deserialize.R' + 'serialize.R' + 'sparkR.R' + 'backend.R' + 'client.R' + 'utils.R' + 'zzz.R' diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE new file mode 100644 index 0000000000000..a354cdce74afa --- /dev/null +++ b/R/pkg/NAMESPACE @@ -0,0 +1,182 @@ +#exportPattern("^[[:alpha:]]+") +exportClasses("RDD") +exportClasses("Broadcast") +exportMethods( + "aggregateByKey", + "aggregateRDD", + "cache", + "checkpoint", + "coalesce", + "cogroup", + "collect", + "collectAsMap", + "collectPartition", + "combineByKey", + "count", + "countByKey", + "countByValue", + "distinct", + "Filter", + "filterRDD", + "first", + "flatMap", + "flatMapValues", + "fold", + "foldByKey", + "foreach", + "foreachPartition", + "fullOuterJoin", + "glom", + "groupByKey", + "join", + "keyBy", + "keys", + "length", + "lapply", + "lapplyPartition", + "lapplyPartitionsWithIndex", + "leftOuterJoin", + "lookup", + "map", + "mapPartitions", + "mapPartitionsWithIndex", + "mapValues", + "maximum", + "minimum", + "numPartitions", + "partitionBy", + "persist", + "pipeRDD", + "reduce", + "reduceByKey", + "reduceByKeyLocally", + "repartition", + "rightOuterJoin", + "sampleRDD", + "saveAsTextFile", + "saveAsObjectFile", + "sortBy", + "sortByKey", + "sumRDD", + "take", + "takeOrdered", + "takeSample", + "top", + "unionRDD", + "unpersist", + "value", + "values", + "zipRDD", + "zipWithIndex", + "zipWithUniqueId" + ) + +# S3 methods exported +export( + "textFile", + "objectFile", + "parallelize", + "hashCode", + "includePackage", + "broadcast", + "setBroadcastValue", + "setCheckpointDir" + ) +export("sparkR.init") +export("sparkR.stop") +export("print.jobj") +useDynLib(SparkR, stringHashCode) +importFrom(methods, setGeneric, setMethod, setOldClass) + +# SparkRSQL + +exportClasses("DataFrame") + +exportMethods("columns", + "distinct", + "dtypes", + "explain", + "filter", + "groupBy", + "head", + "insertInto", + "intersect", + "isLocal", + "limit", + "orderBy", + "names", + "printSchema", + "registerTempTable", + "repartition", + "sampleDF", + "saveAsParquetFile", + "saveAsTable", + "saveDF", + "schema", + "select", + "selectExpr", + "show", + "showDF", + "sortDF", + "subtract", + "toJSON", + "toRDD", + "unionAll", + "where", + "withColumn", + "withColumnRenamed") + +exportClasses("Column") + +exportMethods("abs", + "alias", + "approxCountDistinct", + "asc", + "avg", + "cast", + "contains", + "countDistinct", + "desc", + "endsWith", + "getField", + "getItem", + "isNotNull", + "isNull", + "last", + "like", + "lower", + "max", + "mean", + "min", + "rlike", + "sqrt", + "startsWith", + "substr", + "sum", + "sumDistinct", + "upper") + +exportClasses("GroupedData") +exportMethods("agg") + +export("sparkRSQL.init", + "sparkRHive.init") + +export("cacheTable", + "clearCache", + "createDataFrame", + "createExternalTable", + "dropTempTable", + "jsonFile", + "jsonRDD", + "loadDF", + "parquetFile", + "sql", + "table", + "tableNames", + "tables", + "toDF", + "uncacheTable") + +export("print.structType", + "print.structField") diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R new file mode 100644 index 0000000000000..feafd56909a67 --- /dev/null +++ b/R/pkg/R/DataFrame.R @@ -0,0 +1,1270 @@ +# +# 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. +# + +# DataFrame.R - DataFrame class and methods implemented in S4 OO classes + +#' @include jobj.R SQLTypes.R RDD.R pairRDD.R column.R group.R +NULL + +setOldClass("jobj") + +#' @title S4 class that represents a DataFrame +#' @description DataFrames can be created using functions like +#' \code{jsonFile}, \code{table} etc. +#' @rdname DataFrame +#' @seealso jsonFile, table +#' +#' @param env An R environment that stores bookkeeping states of the DataFrame +#' @param sdf A Java object reference to the backing Scala DataFrame +#' @export +setClass("DataFrame", + slots = list(env = "environment", + sdf = "jobj")) + +setMethod("initialize", "DataFrame", function(.Object, sdf, isCached) { + .Object@env <- new.env() + .Object@env$isCached <- isCached + + .Object@sdf <- sdf + .Object +}) + +#' @rdname DataFrame +#' @export +dataFrame <- function(sdf, isCached = FALSE) { + new("DataFrame", sdf, isCached) +} + +############################ DataFrame Methods ############################################## + +#' Print Schema of a DataFrame +#' +#' Prints out the schema in tree format +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname printSchema +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' printSchema(df) +#'} +setMethod("printSchema", + signature(x = "DataFrame"), + function(x) { + schemaString <- callJMethod(schema(x)$jobj, "treeString") + cat(schemaString) + }) + +#' Get schema object +#' +#' Returns the schema of this DataFrame as a structType object. +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname schema +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' dfSchema <- schema(df) +#'} +setMethod("schema", + signature(x = "DataFrame"), + function(x) { + structType(callJMethod(x@sdf, "schema")) + }) + +#' Explain +#' +#' Print the logical and physical Catalyst plans to the console for debugging. +#' +#' @param x A SparkSQL DataFrame +#' @param extended Logical. If extended is False, explain() only prints the physical plan. +#' @rdname explain +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' explain(df, TRUE) +#'} +setMethod("explain", + signature(x = "DataFrame"), + function(x, extended = FALSE) { + queryExec <- callJMethod(x@sdf, "queryExecution") + if (extended) { + cat(callJMethod(queryExec, "toString")) + } else { + execPlan <- callJMethod(queryExec, "executedPlan") + cat(callJMethod(execPlan, "toString")) + } + }) + +#' isLocal +#' +#' Returns True if the `collect` and `take` methods can be run locally +#' (without any Spark executors). +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname isLocal +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' isLocal(df) +#'} +setMethod("isLocal", + signature(x = "DataFrame"), + function(x) { + callJMethod(x@sdf, "isLocal") + }) + +#' ShowDF +#' +#' Print the first numRows rows of a DataFrame +#' +#' @param x A SparkSQL DataFrame +#' @param numRows The number of rows to print. Defaults to 20. +#' +#' @rdname showDF +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' showDF(df) +#'} +setMethod("showDF", + signature(x = "DataFrame"), + function(x, numRows = 20) { + cat(callJMethod(x@sdf, "showString", numToInt(numRows)), "\n") + }) + +#' show +#' +#' Print the DataFrame column names and types +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname show +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' show(df) +#'} +setMethod("show", "DataFrame", + function(object) { + cols <- lapply(dtypes(object), function(l) { + paste(l, collapse = ":") + }) + s <- paste(cols, collapse = ", ") + cat(paste("DataFrame[", s, "]\n", sep = "")) + }) + +#' DataTypes +#' +#' Return all column names and their data types as a list +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname dtypes +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' dtypes(df) +#'} +setMethod("dtypes", + signature(x = "DataFrame"), + function(x) { + lapply(schema(x)$fields(), function(f) { + c(f$name(), f$dataType.simpleString()) + }) + }) + +#' Column names +#' +#' Return all column names as a list +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname columns +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' columns(df) +#'} +setMethod("columns", + signature(x = "DataFrame"), + function(x) { + sapply(schema(x)$fields(), function(f) { + f$name() + }) + }) + +#' @rdname columns +#' @export +setMethod("names", + signature(x = "DataFrame"), + function(x) { + columns(x) + }) + +#' Register Temporary Table +#' +#' Registers a DataFrame as a Temporary Table in the SQLContext +#' +#' @param x A SparkSQL DataFrame +#' @param tableName A character vector containing the name of the table +#' +#' @rdname registerTempTable +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "json_df") +#' new_df <- sql(sqlCtx, "SELECT * FROM json_df") +#'} +setMethod("registerTempTable", + signature(x = "DataFrame", tableName = "character"), + function(x, tableName) { + callJMethod(x@sdf, "registerTempTable", tableName) + }) + +#' insertInto +#' +#' Insert the contents of a DataFrame into a table registered in the current SQL Context. +#' +#' @param x A SparkSQL DataFrame +#' @param tableName A character vector containing the name of the table +#' @param overwrite A logical argument indicating whether or not to overwrite +#' the existing rows in the table. +#' +#' @rdname insertInto +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- loadDF(sqlCtx, path, "parquet") +#' df2 <- loadDF(sqlCtx, path2, "parquet") +#' registerTempTable(df, "table1") +#' insertInto(df2, "table1", overwrite = TRUE) +#'} +setMethod("insertInto", + signature(x = "DataFrame", tableName = "character"), + function(x, tableName, overwrite = FALSE) { + callJMethod(x@sdf, "insertInto", tableName, overwrite) + }) + +#' Cache +#' +#' Persist with the default storage level (MEMORY_ONLY). +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname cache-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' cache(df) +#'} +setMethod("cache", + signature(x = "DataFrame"), + function(x) { + cached <- callJMethod(x@sdf, "cache") + x@env$isCached <- TRUE + x + }) + +#' Persist +#' +#' Persist this DataFrame with the specified storage level. For details of the +#' supported storage levels, refer to +#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. +#' +#' @param x The DataFrame to persist +#' @rdname persist +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' persist(df, "MEMORY_AND_DISK") +#'} +setMethod("persist", + signature(x = "DataFrame", newLevel = "character"), + function(x, newLevel) { + callJMethod(x@sdf, "persist", getStorageLevel(newLevel)) + x@env$isCached <- TRUE + x + }) + +#' Unpersist +#' +#' Mark this DataFrame as non-persistent, and remove all blocks for it from memory and +#' disk. +#' +#' @param x The DataFrame to unpersist +#' @param blocking Whether to block until all blocks are deleted +#' @rdname unpersist-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' persist(df, "MEMORY_AND_DISK") +#' unpersist(df) +#'} +setMethod("unpersist", + signature(x = "DataFrame"), + function(x, blocking = TRUE) { + callJMethod(x@sdf, "unpersist", blocking) + x@env$isCached <- FALSE + x + }) + +#' Repartition +#' +#' Return a new DataFrame that has exactly numPartitions partitions. +#' +#' @param x A SparkSQL DataFrame +#' @param numPartitions The number of partitions to use. +#' @rdname repartition +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newDF <- repartition(df, 2L) +#'} +setMethod("repartition", + signature(x = "DataFrame", numPartitions = "numeric"), + function(x, numPartitions) { + sdf <- callJMethod(x@sdf, "repartition", numToInt(numPartitions)) + dataFrame(sdf) + }) + +#' toJSON +#' +#' Convert the rows of a DataFrame into JSON objects and return an RDD where +#' each element contains a JSON string. +#' +#' @param x A SparkSQL DataFrame +#' @return A StringRRDD of JSON objects +#' @rdname tojson +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newRDD <- toJSON(df) +#'} +setMethod("toJSON", + signature(x = "DataFrame"), + function(x) { + rdd <- callJMethod(x@sdf, "toJSON") + jrdd <- callJMethod(rdd, "toJavaRDD") + RDD(jrdd, serializedMode = "string") + }) + +#' saveAsParquetFile +#' +#' Save the contents of a DataFrame as a Parquet file, preserving the schema. Files written out +#' with this method can be read back in as a DataFrame using parquetFile(). +#' +#' @param x A SparkSQL DataFrame +#' @param path The directory where the file is saved +#' @rdname saveAsParquetFile +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' saveAsParquetFile(df, "/tmp/sparkr-tmp/") +#'} +setMethod("saveAsParquetFile", + signature(x = "DataFrame", path = "character"), + function(x, path) { + invisible(callJMethod(x@sdf, "saveAsParquetFile", path)) + }) + +#' Distinct +#' +#' Return a new DataFrame containing the distinct rows in this DataFrame. +#' +#' @param x A SparkSQL DataFrame +#' @rdname distinct +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' distinctDF <- distinct(df) +#'} +setMethod("distinct", + signature(x = "DataFrame"), + function(x) { + sdf <- callJMethod(x@sdf, "distinct") + dataFrame(sdf) + }) + +#' SampleDF +#' +#' Return a sampled subset of this DataFrame using a random seed. +#' +#' @param x A SparkSQL DataFrame +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @rdname sampleDF +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' collect(sampleDF(df, FALSE, 0.5)) +#' collect(sampleDF(df, TRUE, 0.5)) +#'} +setMethod("sampleDF", + # TODO : Figure out how to send integer as java.lang.Long to JVM so + # we can send seed as an argument through callJMethod + signature(x = "DataFrame", withReplacement = "logical", + fraction = "numeric"), + function(x, withReplacement, fraction) { + if (fraction < 0.0) stop(cat("Negative fraction value:", fraction)) + sdf <- callJMethod(x@sdf, "sample", withReplacement, fraction) + dataFrame(sdf) + }) + +#' Count +#' +#' Returns the number of rows in a DataFrame +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname count +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' count(df) +#' } +setMethod("count", + signature(x = "DataFrame"), + function(x) { + callJMethod(x@sdf, "count") + }) + +#' Collects all the elements of a Spark DataFrame and coerces them into an R data.frame. +#' +#' @param x A SparkSQL DataFrame +#' @param stringsAsFactors (Optional) A logical indicating whether or not string columns +#' should be converted to factors. FALSE by default. + +#' @rdname collect-methods +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' collected <- collect(df) +#' firstName <- collected[[1]]$name +#' } +setMethod("collect", + signature(x = "DataFrame"), + function(x, stringsAsFactors = FALSE) { + # listCols is a list of raw vectors, one per column + listCols <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToCols", x@sdf) + cols <- lapply(listCols, function(col) { + objRaw <- rawConnection(col) + numRows <- readInt(objRaw) + col <- readCol(objRaw, numRows) + close(objRaw) + col + }) + names(cols) <- columns(x) + do.call(cbind.data.frame, list(cols, stringsAsFactors = stringsAsFactors)) + }) + +#' Limit +#' +#' Limit the resulting DataFrame to the number of rows specified. +#' +#' @param x A SparkSQL DataFrame +#' @param num The number of rows to return +#' @return A new DataFrame containing the number of rows specified. +#' +#' @rdname limit +#' @export +#' @examples +#' \dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' limitedDF <- limit(df, 10) +#' } +setMethod("limit", + signature(x = "DataFrame", num = "numeric"), + function(x, num) { + res <- callJMethod(x@sdf, "limit", as.integer(num)) + dataFrame(res) + }) + +# Take the first NUM rows of a DataFrame and return a the results as a data.frame + +#' @rdname take +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' take(df, 2) +#' } +setMethod("take", + signature(x = "DataFrame", num = "numeric"), + function(x, num) { + limited <- limit(x, num) + collect(limited) + }) + +#' Head +#' +#' Return the first NUM rows of a DataFrame as a data.frame. If NUM is NULL, +#' then head() returns the first 6 rows in keeping with the current data.frame +#' convention in R. +#' +#' @param x A SparkSQL DataFrame +#' @param num The number of rows to return. Default is 6. +#' @return A data.frame +#' +#' @rdname head +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' head(df) +#' } +setMethod("head", + signature(x = "DataFrame"), + function(x, num = 6L) { + # Default num is 6L in keeping with R's data.frame convention + take(x, num) + }) + +#' Return the first row of a DataFrame +#' +#' @param x A SparkSQL DataFrame +#' +#' @rdname first +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' first(df) +#' } +setMethod("first", + signature(x = "DataFrame"), + function(x) { + take(x, 1) + }) + +#' toRDD() +#' +#' Converts a Spark DataFrame to an RDD while preserving column names. +#' +#' @param x A Spark DataFrame +#' +#' @rdname DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' rdd <- toRDD(df) +#' } +setMethod("toRDD", + signature(x = "DataFrame"), + function(x) { + jrdd <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToRowRDD", x@sdf) + colNames <- callJMethod(x@sdf, "columns") + rdd <- RDD(jrdd, serializedMode = "row") + lapply(rdd, function(row) { + names(row) <- colNames + row + }) + }) + +#' GroupBy +#' +#' Groups the DataFrame using the specified columns, so we can run aggregation on them. +#' +#' @param x a DataFrame +#' @return a GroupedData +#' @seealso GroupedData +#' @rdname DataFrame +#' @export +#' @examples +#' \dontrun{ +#' # Compute the average for all numeric columns grouped by department. +#' avg(groupBy(df, "department")) +#' +#' # Compute the max age and average salary, grouped by department and gender. +#' agg(groupBy(df, "department", "gender"), salary="avg", "age" -> "max") +#' } +setMethod("groupBy", + signature(x = "DataFrame"), + function(x, ...) { + cols <- list(...) + if (length(cols) >= 1 && class(cols[[1]]) == "character") { + sgd <- callJMethod(x@sdf, "groupBy", cols[[1]], listToSeq(cols[-1])) + } else { + jcol <- lapply(cols, function(c) { c@jc }) + sgd <- callJMethod(x@sdf, "groupBy", listToSeq(jcol)) + } + groupedData(sgd) + }) + +#' Agg +#' +#' Compute aggregates by specifying a list of columns +#' +#' @rdname DataFrame +#' @export +setMethod("agg", + signature(x = "DataFrame"), + function(x, ...) { + agg(groupBy(x), ...) + }) + + +############################## RDD Map Functions ################################## +# All of the following functions mirror the existing RDD map functions, # +# but allow for use with DataFrames by first converting to an RRDD before calling # +# the requested map function. # +################################################################################### + +#' @rdname lapply +setMethod("lapply", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + rdd <- toRDD(X) + lapply(rdd, FUN) + }) + +#' @rdname lapply +setMethod("map", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + lapply(X, FUN) + }) + +#' @rdname flatMap +setMethod("flatMap", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + rdd <- toRDD(X) + flatMap(rdd, FUN) + }) + +#' @rdname lapplyPartition +setMethod("lapplyPartition", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + rdd <- toRDD(X) + lapplyPartition(rdd, FUN) + }) + +#' @rdname lapplyPartition +setMethod("mapPartitions", + signature(X = "DataFrame", FUN = "function"), + function(X, FUN) { + lapplyPartition(X, FUN) + }) + +#' @rdname foreach +setMethod("foreach", + signature(x = "DataFrame", func = "function"), + function(x, func) { + rdd <- toRDD(x) + foreach(rdd, func) + }) + +#' @rdname foreach +setMethod("foreachPartition", + signature(x = "DataFrame", func = "function"), + function(x, func) { + rdd <- toRDD(x) + foreachPartition(rdd, func) + }) + + +############################## SELECT ################################## + +getColumn <- function(x, c) { + column(callJMethod(x@sdf, "col", c)) +} + +#' @rdname select +setMethod("$", signature(x = "DataFrame"), + function(x, name) { + getColumn(x, name) + }) + +setMethod("$<-", signature(x = "DataFrame"), + function(x, name, value) { + stopifnot(class(value) == "Column") + cols <- columns(x) + if (name %in% cols) { + cols <- lapply(cols, function(c) { + if (c == name) { + alias(value, name) + } else { + col(c) + } + }) + nx <- select(x, cols) + } else { + nx <- withColumn(x, name, value) + } + x@sdf <- nx@sdf + x + }) + +#' @rdname select +setMethod("[[", signature(x = "DataFrame"), + function(x, i) { + if (is.numeric(i)) { + cols <- columns(x) + i <- cols[[i]] + } + getColumn(x, i) + }) + +#' @rdname select +setMethod("[", signature(x = "DataFrame", i = "missing"), + function(x, i, j, ...) { + if (is.numeric(j)) { + cols <- columns(x) + j <- cols[j] + } + if (length(j) > 1) { + j <- as.list(j) + } + select(x, j) + }) + +#' Select +#' +#' Selects a set of columns with names or Column expressions. +#' @param x A DataFrame +#' @param col A list of columns or single Column or name +#' @return A new DataFrame with selected columns +#' @export +#' @rdname select +#' @examples +#' \dontrun{ +#' select(df, "*") +#' select(df, "col1", "col2") +#' select(df, df$name, df$age + 1) +#' select(df, c("col1", "col2")) +#' select(df, list(df$name, df$age + 1)) +#' # Columns can also be selected using `[[` and `[` +#' df[[2]] == df[["age"]] +#' df[,2] == df[,"age"] +#' # Similar to R data frames columns can also be selected using `$` +#' df$age +#' } +setMethod("select", signature(x = "DataFrame", col = "character"), + function(x, col, ...) { + sdf <- callJMethod(x@sdf, "select", col, toSeq(...)) + dataFrame(sdf) + }) + +#' @rdname select +#' @export +setMethod("select", signature(x = "DataFrame", col = "Column"), + function(x, col, ...) { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + sdf <- callJMethod(x@sdf, "select", listToSeq(jcols)) + dataFrame(sdf) + }) + +#' @rdname select +#' @export +setMethod("select", + signature(x = "DataFrame", col = "list"), + function(x, col) { + cols <- lapply(col, function(c) { + if (class(c)== "Column") { + c@jc + } else { + col(c)@jc + } + }) + sdf <- callJMethod(x@sdf, "select", listToSeq(cols)) + dataFrame(sdf) + }) + +#' SelectExpr +#' +#' Select from a DataFrame using a set of SQL expressions. +#' +#' @param x A DataFrame to be selected from. +#' @param expr A string containing a SQL expression +#' @param ... Additional expressions +#' @return A DataFrame +#' @rdname selectExpr +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' selectExpr(df, "col1", "(col2 * 5) as newCol") +#' } +setMethod("selectExpr", + signature(x = "DataFrame", expr = "character"), + function(x, expr, ...) { + exprList <- list(expr, ...) + sdf <- callJMethod(x@sdf, "selectExpr", listToSeq(exprList)) + dataFrame(sdf) + }) + +#' WithColumn +#' +#' Return a new DataFrame with the specified column added. +#' +#' @param x A DataFrame +#' @param colName A string containing the name of the new column. +#' @param col A Column expression. +#' @return A DataFrame with the new column added. +#' @rdname withColumn +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newDF <- withColumn(df, "newCol", df$col1 * 5) +#' } +setMethod("withColumn", + signature(x = "DataFrame", colName = "character", col = "Column"), + function(x, colName, col) { + select(x, x$"*", alias(col, colName)) + }) + +#' WithColumnRenamed +#' +#' Rename an existing column in a DataFrame. +#' +#' @param x A DataFrame +#' @param existingCol The name of the column you want to change. +#' @param newCol The new column name. +#' @return A DataFrame with the column name changed. +#' @rdname withColumnRenamed +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' newDF <- withColumnRenamed(df, "col1", "newCol1") +#' } +setMethod("withColumnRenamed", + signature(x = "DataFrame", existingCol = "character", newCol = "character"), + function(x, existingCol, newCol) { + cols <- lapply(columns(x), function(c) { + if (c == existingCol) { + alias(col(c), newCol) + } else { + col(c) + } + }) + select(x, cols) + }) + +setClassUnion("characterOrColumn", c("character", "Column")) + +#' SortDF +#' +#' Sort a DataFrame by the specified column(s). +#' +#' @param x A DataFrame to be sorted. +#' @param col Either a Column object or character vector indicating the field to sort on +#' @param ... Additional sorting fields +#' @return A DataFrame where all elements are sorted. +#' @rdname sortDF +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' sortDF(df, df$col1) +#' sortDF(df, "col1") +#' sortDF(df, asc(df$col1), desc(abs(df$col2))) +#' } +setMethod("sortDF", + signature(x = "DataFrame", col = "characterOrColumn"), + function(x, col, ...) { + if (class(col) == "character") { + sdf <- callJMethod(x@sdf, "sort", col, toSeq(...)) + } else if (class(col) == "Column") { + jcols <- lapply(list(col, ...), function(c) { + c@jc + }) + sdf <- callJMethod(x@sdf, "sort", listToSeq(jcols)) + } + dataFrame(sdf) + }) + +#' @rdname sortDF +#' @export +setMethod("orderBy", + signature(x = "DataFrame", col = "characterOrColumn"), + function(x, col) { + sortDF(x, col) + }) + +#' Filter +#' +#' Filter the rows of a DataFrame according to a given condition. +#' +#' @param x A DataFrame to be sorted. +#' @param condition The condition to sort on. This may either be a Column expression +#' or a string containing a SQL statement +#' @return A DataFrame containing only the rows that meet the condition. +#' @rdname filter +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' filter(df, "col1 > 0") +#' filter(df, df$col2 != "abcdefg") +#' } +setMethod("filter", + signature(x = "DataFrame", condition = "characterOrColumn"), + function(x, condition) { + if (class(condition) == "Column") { + condition <- condition@jc + } + sdf <- callJMethod(x@sdf, "filter", condition) + dataFrame(sdf) + }) + +#' @rdname filter +#' @export +setMethod("where", + signature(x = "DataFrame", condition = "characterOrColumn"), + function(x, condition) { + filter(x, condition) + }) + +#' Join +#' +#' Join two DataFrames based on the given join expression. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @param joinExpr (Optional) The expression used to perform the join. joinExpr must be a +#' Column expression. If joinExpr is omitted, join() wil perform a Cartesian join +#' @param joinType The type of join to perform. The following join types are available: +#' 'inner', 'outer', 'left_outer', 'right_outer', 'semijoin'. The default joinType is "inner". +#' @return A DataFrame containing the result of the join operation. +#' @rdname join +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' join(df1, df2) # Performs a Cartesian +#' join(df1, df2, df1$col1 == df2$col2) # Performs an inner join based on expression +#' join(df1, df2, df1$col1 == df2$col2, "right_outer") +#' } +setMethod("join", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y, joinExpr = NULL, joinType = NULL) { + if (is.null(joinExpr)) { + sdf <- callJMethod(x@sdf, "join", y@sdf) + } else { + if (class(joinExpr) != "Column") stop("joinExpr must be a Column") + if (is.null(joinType)) { + sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc) + } else { + if (joinType %in% c("inner", "outer", "left_outer", "right_outer", "semijoin")) { + sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc, joinType) + } else { + stop("joinType must be one of the following types: ", + "'inner', 'outer', 'left_outer', 'right_outer', 'semijoin'") + } + } + } + dataFrame(sdf) + }) + +#' UnionAll +#' +#' Return a new DataFrame containing the union of rows in this DataFrame +#' and another DataFrame. This is equivalent to `UNION ALL` in SQL. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @return A DataFrame containing the result of the union. +#' @rdname unionAll +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' unioned <- unionAll(df, df2) +#' } +setMethod("unionAll", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y) { + unioned <- callJMethod(x@sdf, "unionAll", y@sdf) + dataFrame(unioned) + }) + +#' Intersect +#' +#' Return a new DataFrame containing rows only in both this DataFrame +#' and another DataFrame. This is equivalent to `INTERSECT` in SQL. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @return A DataFrame containing the result of the intersect. +#' @rdname intersect +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' intersectDF <- intersect(df, df2) +#' } +setMethod("intersect", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y) { + intersected <- callJMethod(x@sdf, "intersect", y@sdf) + dataFrame(intersected) + }) + +#' Subtract +#' +#' Return a new DataFrame containing rows in this DataFrame +#' but not in another DataFrame. This is equivalent to `EXCEPT` in SQL. +#' +#' @param x A Spark DataFrame +#' @param y A Spark DataFrame +#' @return A DataFrame containing the result of the subtract operation. +#' @rdname subtract +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df1 <- jsonFile(sqlCtx, path) +#' df2 <- jsonFile(sqlCtx, path2) +#' subtractDF <- subtract(df, df2) +#' } +setMethod("subtract", + signature(x = "DataFrame", y = "DataFrame"), + function(x, y) { + subtracted <- callJMethod(x@sdf, "except", y@sdf) + dataFrame(subtracted) + }) + +#' Save the contents of the DataFrame to a data source +#' +#' The data source is specified by the `source` and a set of options (...). +#' If `source` is not specified, the default data source configured by +#' spark.sql.sources.default will be used. +#' +#' Additionally, mode is used to specify the behavior of the save operation when +#' data already exists in the data source. There are four modes: +#' append: Contents of this DataFrame are expected to be appended to existing data. +#' overwrite: Existing data is expected to be overwritten by the contents of +# this DataFrame. +#' error: An exception is expected to be thrown. +#' ignore: The save operation is expected to not save the contents of the DataFrame +# and to not change the existing data. +#' +#' @param df A SparkSQL DataFrame +#' @param path A name for the table +#' @param source A name for external data source +#' @param mode One of 'append', 'overwrite', 'error', 'ignore' +#' +#' @rdname saveAsTable +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' saveAsTable(df, "myfile") +#' } +setMethod("saveDF", + signature(df = "DataFrame", path = 'character', source = 'character', + mode = 'character'), + function(df, path = NULL, source = NULL, mode = "append", ...){ + if (is.null(source)) { + sqlCtx <- get(".sparkRSQLsc", envir = .sparkREnv) + source <- callJMethod(sqlCtx, "getConf", "spark.sql.sources.default", + "org.apache.spark.sql.parquet") + } + allModes <- c("append", "overwrite", "error", "ignore") + if (!(mode %in% allModes)) { + stop('mode should be one of "append", "overwrite", "error", "ignore"') + } + jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode) + options <- varargsToEnv(...) + if (!is.null(path)) { + options[['path']] = path + } + callJMethod(df@sdf, "save", source, jmode, options) + }) + + +#' saveAsTable +#' +#' Save the contents of the DataFrame to a data source as a table +#' +#' The data source is specified by the `source` and a set of options (...). +#' If `source` is not specified, the default data source configured by +#' spark.sql.sources.default will be used. +#' +#' Additionally, mode is used to specify the behavior of the save operation when +#' data already exists in the data source. There are four modes: +#' append: Contents of this DataFrame are expected to be appended to existing data. +#' overwrite: Existing data is expected to be overwritten by the contents of +# this DataFrame. +#' error: An exception is expected to be thrown. +#' ignore: The save operation is expected to not save the contents of the DataFrame +# and to not change the existing data. +#' +#' @param df A SparkSQL DataFrame +#' @param tableName A name for the table +#' @param source A name for external data source +#' @param mode One of 'append', 'overwrite', 'error', 'ignore' +#' +#' @rdname saveAsTable +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' saveAsTable(df, "myfile") +#' } +setMethod("saveAsTable", + signature(df = "DataFrame", tableName = 'character', source = 'character', + mode = 'character'), + function(df, tableName, source = NULL, mode="append", ...){ + if (is.null(source)) { + sqlCtx <- get(".sparkRSQLsc", envir = .sparkREnv) + source <- callJMethod(sqlCtx, "getConf", "spark.sql.sources.default", + "org.apache.spark.sql.parquet") + } + allModes <- c("append", "overwrite", "error", "ignore") + if (!(mode %in% allModes)) { + stop('mode should be one of "append", "overwrite", "error", "ignore"') + } + jmode <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "saveMode", mode) + options <- varargsToEnv(...) + callJMethod(df@sdf, "saveAsTable", tableName, source, jmode, options) + }) + diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R new file mode 100644 index 0000000000000..604ad03c407b9 --- /dev/null +++ b/R/pkg/R/RDD.R @@ -0,0 +1,1539 @@ +# +# 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. +# + +# RDD in R implemented in S4 OO system. + +setOldClass("jobj") + +#' @title S4 class that represents an RDD +#' @description RDD can be created using functions like +#' \code{parallelize}, \code{textFile} etc. +#' @rdname RDD +#' @seealso parallelize, textFile +#' +#' @slot env An R environment that stores bookkeeping states of the RDD +#' @slot jrdd Java object reference to the backing JavaRDD +#' to an RDD +#' @export +setClass("RDD", + slots = list(env = "environment", + jrdd = "jobj")) + +setClass("PipelinedRDD", + slots = list(prev = "RDD", + func = "function", + prev_jrdd = "jobj"), + contains = "RDD") + +setMethod("initialize", "RDD", function(.Object, jrdd, serializedMode, + isCached, isCheckpointed) { + # Check that RDD constructor is using the correct version of serializedMode + stopifnot(class(serializedMode) == "character") + stopifnot(serializedMode %in% c("byte", "string", "row")) + # RDD has three serialization types: + # byte: The RDD stores data serialized in R. + # string: The RDD stores data as strings. + # row: The RDD stores the serialized rows of a DataFrame. + + # We use an environment to store mutable states inside an RDD object. + # Note that R's call-by-value semantics makes modifying slots inside an + # object (passed as an argument into a function, such as cache()) difficult: + # i.e. one needs to make a copy of the RDD object and sets the new slot value + # there. + + # The slots are inheritable from superclass. Here, both `env' and `jrdd' are + # inherited from RDD, but only the former is used. + .Object@env <- new.env() + .Object@env$isCached <- isCached + .Object@env$isCheckpointed <- isCheckpointed + .Object@env$serializedMode <- serializedMode + + .Object@jrdd <- jrdd + .Object +}) + +setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) { + .Object@env <- new.env() + .Object@env$isCached <- FALSE + .Object@env$isCheckpointed <- FALSE + .Object@env$jrdd_val <- jrdd_val + if (!is.null(jrdd_val)) { + # This tracks the serialization mode for jrdd_val + .Object@env$serializedMode <- prev@env$serializedMode + } + + .Object@prev <- prev + + isPipelinable <- function(rdd) { + e <- rdd@env + !(e$isCached || e$isCheckpointed) + } + + if (!inherits(prev, "PipelinedRDD") || !isPipelinable(prev)) { + # This transformation is the first in its stage: + .Object@func <- func + .Object@prev_jrdd <- getJRDD(prev) + .Object@env$prev_serializedMode <- prev@env$serializedMode + # NOTE: We use prev_serializedMode to track the serialization mode of prev_JRDD + # prev_serializedMode is used during the delayed computation of JRDD in getJRDD + } else { + pipelinedFunc <- function(split, iterator) { + func(split, prev@func(split, iterator)) + } + .Object@func <- pipelinedFunc + .Object@prev_jrdd <- prev@prev_jrdd # maintain the pipeline + # Get the serialization mode of the parent RDD + .Object@env$prev_serializedMode <- prev@env$prev_serializedMode + } + + .Object +}) + +#' @rdname RDD +#' @export +#' +#' @param jrdd Java object reference to the backing JavaRDD +#' @param serializedMode Use "byte" if the RDD stores data serialized in R, "string" if the RDD +#' stores strings, and "row" if the RDD stores the rows of a DataFrame +#' @param isCached TRUE if the RDD is cached +#' @param isCheckpointed TRUE if the RDD has been checkpointed +RDD <- function(jrdd, serializedMode = "byte", isCached = FALSE, + isCheckpointed = FALSE) { + new("RDD", jrdd, serializedMode, isCached, isCheckpointed) +} + +PipelinedRDD <- function(prev, func) { + new("PipelinedRDD", prev, func, NULL) +} + +# Return the serialization mode for an RDD. +setGeneric("getSerializedMode", function(rdd, ...) { standardGeneric("getSerializedMode") }) +# For normal RDDs we can directly read the serializedMode +setMethod("getSerializedMode", signature(rdd = "RDD"), function(rdd) rdd@env$serializedMode ) +# For pipelined RDDs if jrdd_val is set then serializedMode should exist +# if not we return the defaultSerialization mode of "byte" as we don't know the serialization +# mode at this point in time. +setMethod("getSerializedMode", signature(rdd = "PipelinedRDD"), + function(rdd) { + if (!is.null(rdd@env$jrdd_val)) { + return(rdd@env$serializedMode) + } else { + return("byte") + } + }) + +# The jrdd accessor function. +setMethod("getJRDD", signature(rdd = "RDD"), function(rdd) rdd@jrdd ) +setMethod("getJRDD", signature(rdd = "PipelinedRDD"), + function(rdd, serializedMode = "byte") { + if (!is.null(rdd@env$jrdd_val)) { + return(rdd@env$jrdd_val) + } + + computeFunc <- function(split, part) { + rdd@func(split, part) + } + + packageNamesArr <- serialize(.sparkREnv[[".packages"]], + connection = NULL) + + broadcastArr <- lapply(ls(.broadcastNames), + function(name) { get(name, .broadcastNames) }) + + serializedFuncArr <- serialize(computeFunc, connection = NULL) + + prev_jrdd <- rdd@prev_jrdd + + if (serializedMode == "string") { + rddRef <- newJObject("org.apache.spark.api.r.StringRRDD", + callJMethod(prev_jrdd, "rdd"), + serializedFuncArr, + rdd@env$prev_serializedMode, + packageNamesArr, + as.character(.sparkREnv[["libname"]]), + broadcastArr, + callJMethod(prev_jrdd, "classTag")) + } else { + rddRef <- newJObject("org.apache.spark.api.r.RRDD", + callJMethod(prev_jrdd, "rdd"), + serializedFuncArr, + rdd@env$prev_serializedMode, + serializedMode, + packageNamesArr, + as.character(.sparkREnv[["libname"]]), + broadcastArr, + callJMethod(prev_jrdd, "classTag")) + } + # Save the serialization flag after we create a RRDD + rdd@env$serializedMode <- serializedMode + rdd@env$jrdd_val <- callJMethod(rddRef, "asJavaRDD") # rddRef$asJavaRDD() + rdd@env$jrdd_val + }) + +setValidity("RDD", + function(object) { + jrdd <- getJRDD(object) + cls <- callJMethod(jrdd, "getClass") + className <- callJMethod(cls, "getName") + if (grep("spark.api.java.*RDD*", className) == 1) { + TRUE + } else { + paste("Invalid RDD class ", className) + } + }) + + +############ Actions and Transformations ############ + +#' Persist an RDD +#' +#' Persist this RDD with the default storage level (MEMORY_ONLY). +#' +#' @param x The RDD to cache +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) +#'} +#' @rdname cache-methods +#' @aliases cache,RDD-method +setMethod("cache", + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "cache") + x@env$isCached <- TRUE + x + }) + +#' Persist an RDD +#' +#' Persist this RDD with the specified storage level. For details of the +#' supported storage levels, refer to +#' http://spark.apache.org/docs/latest/programming-guide.html#rdd-persistence. +#' +#' @param x The RDD to persist +#' @param newLevel The new storage level to be assigned +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' persist(rdd, "MEMORY_AND_DISK") +#'} +#' @rdname persist +#' @aliases persist,RDD-method +setMethod("persist", + signature(x = "RDD", newLevel = "character"), + function(x, newLevel) { + callJMethod(getJRDD(x), "persist", getStorageLevel(newLevel)) + x@env$isCached <- TRUE + x + }) + +#' Unpersist an RDD +#' +#' Mark the RDD as non-persistent, and remove all blocks for it from memory and +#' disk. +#' +#' @param x The RDD to unpersist +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' cache(rdd) # rdd@@env$isCached == TRUE +#' unpersist(rdd) # rdd@@env$isCached == FALSE +#'} +#' @rdname unpersist-methods +#' @aliases unpersist,RDD-method +setMethod("unpersist", + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "unpersist") + x@env$isCached <- FALSE + x + }) + +#' Checkpoint an RDD +#' +#' Mark this RDD for checkpointing. It will be saved to a file inside the +#' checkpoint directory set with setCheckpointDir() and all references to its +#' parent RDDs will be removed. This function must be called before any job has +#' been executed on this RDD. It is strongly recommended that this RDD is +#' persisted in memory, otherwise saving it on a file will require recomputation. +#' +#' @param x The RDD to checkpoint +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' setCheckpointDir(sc, "checkpoints") +#' rdd <- parallelize(sc, 1:10, 2L) +#' checkpoint(rdd) +#'} +#' @rdname checkpoint-methods +#' @aliases checkpoint,RDD-method +setMethod("checkpoint", + signature(x = "RDD"), + function(x) { + jrdd <- getJRDD(x) + callJMethod(jrdd, "checkpoint") + x@env$isCheckpointed <- TRUE + x + }) + +#' Gets the number of partitions of an RDD +#' +#' @param x A RDD. +#' @return the number of partitions of rdd as an integer. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' numPartitions(rdd) # 2L +#'} +#' @rdname numPartitions +#' @aliases numPartitions,RDD-method +setMethod("numPartitions", + signature(x = "RDD"), + function(x) { + jrdd <- getJRDD(x) + partitions <- callJMethod(jrdd, "splits") + callJMethod(partitions, "size") + }) + +#' Collect elements of an RDD +#' +#' @description +#' \code{collect} returns a list that contains all of the elements in this RDD. +#' +#' @param x The RDD to collect +#' @param ... Other optional arguments to collect +#' @param flatten FALSE if the list should not flattened +#' @return a list containing elements in the RDD +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2L) +#' collect(rdd) # list from 1 to 10 +#' collectPartition(rdd, 0L) # list from 1 to 5 +#'} +#' @rdname collect-methods +#' @aliases collect,RDD-method +setMethod("collect", + signature(x = "RDD"), + function(x, flatten = TRUE) { + # Assumes a pairwise RDD is backed by a JavaPairRDD. + collected <- callJMethod(getJRDD(x), "collect") + convertJListToRList(collected, flatten, + serializedMode = getSerializedMode(x)) + }) + + +#' @description +#' \code{collectPartition} returns a list that contains all of the elements +#' in the specified partition of the RDD. +#' @param partitionId the partition to collect (starts from 0) +#' @rdname collect-methods +#' @aliases collectPartition,integer,RDD-method +setMethod("collectPartition", + signature(x = "RDD", partitionId = "integer"), + function(x, partitionId) { + jPartitionsList <- callJMethod(getJRDD(x), + "collectPartitions", + as.list(as.integer(partitionId))) + + jList <- jPartitionsList[[1]] + convertJListToRList(jList, flatten = TRUE, + serializedMode = getSerializedMode(x)) + }) + +#' @description +#' \code{collectAsMap} returns a named list as a map that contains all of the elements +#' in a key-value pair RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4)), 2L) +#' collectAsMap(rdd) # list(`1` = 2, `3` = 4) +#'} +#' @rdname collect-methods +#' @aliases collectAsMap,RDD-method +setMethod("collectAsMap", + signature(x = "RDD"), + function(x) { + pairList <- collect(x) + map <- new.env() + lapply(pairList, function(i) { assign(as.character(i[[1]]), i[[2]], envir = map) }) + as.list(map) + }) + +#' Return the number of elements in the RDD. +#' +#' @param x The RDD to count +#' @return number of elements in the RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' count(rdd) # 10 +#' length(rdd) # Same as count +#'} +#' @rdname count +#' @aliases count,RDD-method +setMethod("count", + signature(x = "RDD"), + function(x) { + countPartition <- function(part) { + as.integer(length(part)) + } + valsRDD <- lapplyPartition(x, countPartition) + vals <- collect(valsRDD) + sum(as.integer(vals)) + }) + +#' Return the number of elements in the RDD +#' @export +#' @rdname count +setMethod("length", + signature(x = "RDD"), + function(x) { + count(x) + }) + +#' Return the count of each unique value in this RDD as a list of +#' (value, count) pairs. +#' +#' Same as countByValue in Spark. +#' +#' @param x The RDD to count +#' @return list of (value, count) pairs, where count is number of each unique +#' value in rdd. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,3,2,1)) +#' countByValue(rdd) # (1,2L), (2,2L), (3,1L) +#'} +#' @rdname countByValue +#' @aliases countByValue,RDD-method +setMethod("countByValue", + signature(x = "RDD"), + function(x) { + ones <- lapply(x, function(item) { list(item, 1L) }) + collect(reduceByKey(ones, `+`, numPartitions(x))) + }) + +#' Apply a function to all elements +#' +#' This function creates a new RDD by applying the given transformation to all +#' elements of the given RDD +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each element +#' @return a new RDD created by the transformation. +#' @rdname lapply +#' @aliases lapply +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' multiplyByTwo <- lapply(rdd, function(x) { x * 2 }) +#' collect(multiplyByTwo) # 2,4,6... +#'} +setMethod("lapply", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + func <- function(split, iterator) { + lapply(iterator, FUN) + } + lapplyPartitionsWithIndex(X, func) + }) + +#' @rdname lapply +#' @aliases map,RDD,function-method +setMethod("map", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapply(X, FUN) + }) + +#' Flatten results after apply a function to all elements +#' +#' This function return a new RDD by first applying a function to all +#' elements of this RDD, and then flattening the results. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each element +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' multiplyByTwo <- flatMap(rdd, function(x) { list(x*2, x*10) }) +#' collect(multiplyByTwo) # 2,20,4,40,6,60... +#'} +#' @rdname flatMap +#' @aliases flatMap,RDD,function-method +setMethod("flatMap", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + partitionFunc <- function(part) { + unlist( + lapply(part, FUN), + recursive = F + ) + } + lapplyPartition(X, partitionFunc) + }) + +#' Apply a function to each partition of an RDD +#' +#' Return a new RDD by applying a function to each partition of this RDD. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' partitionSum <- lapplyPartition(rdd, function(part) { Reduce("+", part) }) +#' collect(partitionSum) # 15, 40 +#'} +#' @rdname lapplyPartition +#' @aliases lapplyPartition,RDD,function-method +setMethod("lapplyPartition", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapplyPartitionsWithIndex(X, function(s, part) { FUN(part) }) + }) + +#' mapPartitions is the same as lapplyPartition. +#' +#' @rdname lapplyPartition +#' @aliases mapPartitions,RDD,function-method +setMethod("mapPartitions", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapplyPartition(X, FUN) + }) + +#' Return a new RDD by applying a function to each partition of this RDD, while +#' tracking the index of the original partition. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on each partition; takes the partition +#' index and a list of elements in the particular partition. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 5L) +#' prod <- lapplyPartitionsWithIndex(rdd, function(split, part) { +#' split * Reduce("+", part) }) +#' collect(prod, flatten = FALSE) # 0, 7, 22, 45, 76 +#'} +#' @rdname lapplyPartitionsWithIndex +#' @aliases lapplyPartitionsWithIndex,RDD,function-method +setMethod("lapplyPartitionsWithIndex", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + FUN <- cleanClosure(FUN) + closureCapturingFunc <- function(split, part) { + FUN(split, part) + } + PipelinedRDD(X, closureCapturingFunc) + }) + +#' @rdname lapplyPartitionsWithIndex +#' @aliases mapPartitionsWithIndex,RDD,function-method +setMethod("mapPartitionsWithIndex", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + lapplyPartitionsWithIndex(X, FUN) + }) + +#' This function returns a new RDD containing only the elements that satisfy +#' a predicate (i.e. returning TRUE in a given logical function). +#' The same as `filter()' in Spark. +#' +#' @param x The RDD to be filtered. +#' @param f A unary predicate function. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' unlist(collect(filterRDD(rdd, function (x) { x < 3 }))) # c(1, 2) +#'} +#' @rdname filterRDD +#' @aliases filterRDD,RDD,function-method +setMethod("filterRDD", + signature(x = "RDD", f = "function"), + function(x, f) { + filter.func <- function(part) { + Filter(f, part) + } + lapplyPartition(x, filter.func) + }) + +#' @rdname filterRDD +#' @aliases Filter +setMethod("Filter", + signature(f = "function", x = "RDD"), + function(f, x) { + filterRDD(x, f) + }) + +#' Reduce across elements of an RDD. +#' +#' This function reduces the elements of this RDD using the +#' specified commutative and associative binary operator. +#' +#' @param x The RDD to reduce +#' @param func Commutative and associative function to apply on elements +#' of the RDD. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' reduce(rdd, "+") # 55 +#'} +#' @rdname reduce +#' @aliases reduce,RDD,ANY-method +setMethod("reduce", + signature(x = "RDD", func = "ANY"), + function(x, func) { + + reducePartition <- function(part) { + Reduce(func, part) + } + + partitionList <- collect(lapplyPartition(x, reducePartition), + flatten = FALSE) + Reduce(func, partitionList) + }) + +#' Get the maximum element of an RDD. +#' +#' @param x The RDD to get the maximum element from +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' maximum(rdd) # 10 +#'} +#' @rdname maximum +#' @aliases maximum,RDD +setMethod("maximum", + signature(x = "RDD"), + function(x) { + reduce(x, max) + }) + +#' Get the minimum element of an RDD. +#' +#' @param x The RDD to get the minimum element from +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' minimum(rdd) # 1 +#'} +#' @rdname minimum +#' @aliases minimum,RDD +setMethod("minimum", + signature(x = "RDD"), + function(x) { + reduce(x, min) + }) + +#' Add up the elements in an RDD. +#' +#' @param x The RDD to add up the elements in +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' sumRDD(rdd) # 55 +#'} +#' @rdname sumRDD +#' @aliases sumRDD,RDD +setMethod("sumRDD", + signature(x = "RDD"), + function(x) { + reduce(x, "+") + }) + +#' Applies a function to all elements in an RDD, and force evaluation. +#' +#' @param x The RDD to apply the function +#' @param func The function to be applied. +#' @return invisible NULL. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreach(rdd, function(x) { save(x, file=...) }) +#'} +#' @rdname foreach +#' @aliases foreach,RDD,function-method +setMethod("foreach", + signature(x = "RDD", func = "function"), + function(x, func) { + partition.func <- function(x) { + lapply(x, func) + NULL + } + invisible(collect(mapPartitions(x, partition.func))) + }) + +#' Applies a function to each partition in an RDD, and force evaluation. +#' +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' foreachPartition(rdd, function(part) { save(part, file=...); NULL }) +#'} +#' @rdname foreach +#' @aliases foreachPartition,RDD,function-method +setMethod("foreachPartition", + signature(x = "RDD", func = "function"), + function(x, func) { + invisible(collect(mapPartitions(x, func))) + }) + +#' Take elements from an RDD. +#' +#' This function takes the first NUM elements in the RDD and +#' returns them in a list. +#' +#' @param x The RDD to take elements from +#' @param num Number of elements to take +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' take(rdd, 2L) # list(1, 2) +#'} +#' @rdname take +#' @aliases take,RDD,numeric-method +setMethod("take", + signature(x = "RDD", num = "numeric"), + function(x, num) { + resList <- list() + index <- -1 + jrdd <- getJRDD(x) + numPartitions <- numPartitions(x) + + # TODO(shivaram): Collect more than one partition based on size + # estimates similar to the scala version of `take`. + while (TRUE) { + index <- index + 1 + + if (length(resList) >= num || index >= numPartitions) + break + + # a JList of byte arrays + partitionArr <- callJMethod(jrdd, "collectPartitions", as.list(as.integer(index))) + partition <- partitionArr[[1]] + + size <- num - length(resList) + # elems is capped to have at most `size` elements + elems <- convertJListToRList(partition, + flatten = TRUE, + logicalUpperBound = size, + serializedMode = getSerializedMode(x)) + # TODO: Check if this append is O(n^2)? + resList <- append(resList, elems) + } + resList + }) + +#' First +#' +#' Return the first element of an RDD +#' +#' @rdname first +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' first(rdd) +#' } +setMethod("first", + signature(x = "RDD"), + function(x) { + take(x, 1)[[1]] + }) + +#' Removes the duplicates from RDD. +#' +#' This function returns a new RDD containing the distinct elements in the +#' given RDD. The same as `distinct()' in Spark. +#' +#' @param x The RDD to remove duplicates from. +#' @param numPartitions Number of partitions to create. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, c(1,2,2,3,3,3)) +#' sort(unlist(collect(distinct(rdd)))) # c(1, 2, 3) +#'} +#' @rdname distinct +#' @aliases distinct,RDD-method +setMethod("distinct", + signature(x = "RDD"), + function(x, numPartitions = SparkR::numPartitions(x)) { + identical.mapped <- lapply(x, function(x) { list(x, NULL) }) + reduced <- reduceByKey(identical.mapped, + function(x, y) { x }, + numPartitions) + resRDD <- lapply(reduced, function(x) { x[[1]] }) + resRDD + }) + +#' Return an RDD that is a sampled subset of the given RDD. +#' +#' The same as `sample()' in Spark. (We rename it due to signature +#' inconsistencies with the `sample()' function in R's base package.) +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param fraction The (rough) sample target fraction +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) # ensure each num is in its own split +#' collect(sampleRDD(rdd, FALSE, 0.5, 1618L)) # ~5 distinct elements +#' collect(sampleRDD(rdd, TRUE, 0.5, 9L)) # ~5 elements possibly with duplicates +#'} +#' @rdname sampleRDD +#' @aliases sampleRDD,RDD +setMethod("sampleRDD", + signature(x = "RDD", withReplacement = "logical", + fraction = "numeric", seed = "integer"), + function(x, withReplacement, fraction, seed) { + + # The sampler: takes a partition and returns its sampled version. + samplingFunc <- function(split, part) { + set.seed(seed) + res <- vector("list", length(part)) + len <- 0 + + # Discards some random values to ensure each partition has a + # different random seed. + runif(split) + + for (elem in part) { + if (withReplacement) { + count <- rpois(1, fraction) + if (count > 0) { + res[(len + 1):(len + count)] <- rep(list(elem), count) + len <- len + count + } + } else { + if (runif(1) < fraction) { + len <- len + 1 + res[[len]] <- elem + } + } + } + + # TODO(zongheng): look into the performance of the current + # implementation. Look into some iterator package? Note that + # Scala avoids many calls to creating an empty list and PySpark + # similarly achieves this using `yield'. + if (len > 0) + res[1:len] + else + list() + } + + lapplyPartitionsWithIndex(x, samplingFunc) + }) + +#' Return a list of the elements that are a sampled subset of the given RDD. +#' +#' @param x The RDD to sample elements from +#' @param withReplacement Sampling with replacement or not +#' @param num Number of elements to return +#' @param seed Randomness seed value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:100) +#' # exactly 5 elements sampled, which may not be distinct +#' takeSample(rdd, TRUE, 5L, 1618L) +#' # exactly 5 distinct elements sampled +#' takeSample(rdd, FALSE, 5L, 16181618L) +#'} +#' @rdname takeSample +#' @aliases takeSample,RDD +setMethod("takeSample", signature(x = "RDD", withReplacement = "logical", + num = "integer", seed = "integer"), + function(x, withReplacement, num, seed) { + # This function is ported from RDD.scala. + fraction <- 0.0 + total <- 0 + multiplier <- 3.0 + initialCount <- count(x) + maxSelected <- 0 + MAXINT <- .Machine$integer.max + + if (num < 0) + stop(paste("Negative number of elements requested")) + + if (initialCount > MAXINT - 1) { + maxSelected <- MAXINT - 1 + } else { + maxSelected <- initialCount + } + + if (num > initialCount && !withReplacement) { + total <- maxSelected + fraction <- multiplier * (maxSelected + 1) / initialCount + } else { + total <- num + fraction <- multiplier * (num + 1) / initialCount + } + + set.seed(seed) + samples <- collect(sampleRDD(x, withReplacement, fraction, + as.integer(ceiling(runif(1, + -MAXINT, + MAXINT))))) + # If the first sample didn't turn out large enough, keep trying to + # take samples; this shouldn't happen often because we use a big + # multiplier for thei initial size + while (length(samples) < total) + samples <- collect(sampleRDD(x, withReplacement, fraction, + as.integer(ceiling(runif(1, + -MAXINT, + MAXINT))))) + + # TODO(zongheng): investigate if this call is an in-place shuffle? + sample(samples)[1:total] + }) + +#' Creates tuples of the elements in this RDD by applying a function. +#' +#' @param x The RDD. +#' @param func The function to be applied. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3)) +#' collect(keyBy(rdd, function(x) { x*x })) # list(list(1, 1), list(4, 2), list(9, 3)) +#'} +#' @rdname keyBy +#' @aliases keyBy,RDD +setMethod("keyBy", + signature(x = "RDD", func = "function"), + function(x, func) { + apply.func <- function(x) { + list(func(x), x) + } + lapply(x, apply.func) + }) + +#' Return a new RDD that has exactly numPartitions partitions. +#' Can increase or decrease the level of parallelism in this RDD. Internally, +#' this uses a shuffle to redistribute data. +#' If you are decreasing the number of partitions in this RDD, consider using +#' coalesce, which can avoid performing a shuffle. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. +#' @seealso coalesce +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5, 6, 7), 4L) +#' numPartitions(rdd) # 4 +#' numPartitions(repartition(rdd, 2L)) # 2 +#'} +#' @rdname repartition +#' @aliases repartition,RDD +setMethod("repartition", + signature(x = "RDD", numPartitions = "numeric"), + function(x, numPartitions) { + coalesce(x, numToInt(numPartitions), TRUE) + }) + +#' Return a new RDD that is reduced into numPartitions partitions. +#' +#' @param x The RDD. +#' @param numPartitions Number of partitions to create. +#' @seealso repartition +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5), 3L) +#' numPartitions(rdd) # 3 +#' numPartitions(coalesce(rdd, 1L)) # 1 +#'} +#' @rdname coalesce +#' @aliases coalesce,RDD +setMethod("coalesce", + signature(x = "RDD", numPartitions = "numeric"), + function(x, numPartitions, shuffle = FALSE) { + numPartitions <- numToInt(numPartitions) + if (shuffle || numPartitions > SparkR::numPartitions(x)) { + func <- function(s, part) { + set.seed(s) # split as seed + start <- as.integer(sample(numPartitions, 1) - 1) + lapply(seq_along(part), + function(i) { + pos <- (start + i) %% numPartitions + list(pos, part[[i]]) + }) + } + shuffled <- lapplyPartitionsWithIndex(x, func) + repartitioned <- partitionBy(shuffled, numPartitions) + values(repartitioned) + } else { + jrdd <- callJMethod(getJRDD(x), "coalesce", numPartitions, shuffle) + RDD(jrdd) + } + }) + +#' Save this RDD as a SequenceFile of serialized objects. +#' +#' @param x The RDD to save +#' @param path The directory where the file is saved +#' @seealso objectFile +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsObjectFile(rdd, "/tmp/sparkR-tmp") +#'} +#' @rdname saveAsObjectFile +#' @aliases saveAsObjectFile,RDD +setMethod("saveAsObjectFile", + signature(x = "RDD", path = "character"), + function(x, path) { + # If serializedMode == "string" we need to serialize the data before saving it since + # objectFile() assumes serializedMode == "byte". + if (getSerializedMode(x) != "byte") { + x <- serializeToBytes(x) + } + # Return nothing + invisible(callJMethod(getJRDD(x), "saveAsObjectFile", path)) + }) + +#' Save this RDD as a text file, using string representations of elements. +#' +#' @param x The RDD to save +#' @param path The directory where the splits of the text file are saved +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' saveAsTextFile(rdd, "/tmp/sparkR-tmp") +#'} +#' @rdname saveAsTextFile +#' @aliases saveAsTextFile,RDD +setMethod("saveAsTextFile", + signature(x = "RDD", path = "character"), + function(x, path) { + func <- function(str) { + toString(str) + } + stringRdd <- lapply(x, func) + # Return nothing + invisible( + callJMethod(getJRDD(stringRdd, serializedMode = "string"), "saveAsTextFile", path)) + }) + +#' Sort an RDD by the given key function. +#' +#' @param x An RDD to be sorted. +#' @param func A function used to compute the sort key for each element. +#' @param ascending A flag to indicate whether the sorting is ascending or descending. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where all elements are sorted. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(3, 2, 1)) +#' collect(sortBy(rdd, function(x) { x })) # list (1, 2, 3) +#'} +#' @rdname sortBy +#' @aliases sortBy,RDD,RDD-method +setMethod("sortBy", + signature(x = "RDD", func = "function"), + function(x, func, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) { + values(sortByKey(keyBy(x, func), ascending, numPartitions)) + }) + +# Helper function to get first N elements from an RDD in the specified order. +# Param: +# x An RDD. +# num Number of elements to return. +# ascending A flag to indicate whether the sorting is ascending or descending. +# Return: +# A list of the first N elements from the RDD in the specified order. +# +takeOrderedElem <- function(x, num, ascending = TRUE) { + if (num <= 0L) { + return(list()) + } + + partitionFunc <- function(part) { + if (num < length(part)) { + # R limitation: order works only on primitive types! + ord <- order(unlist(part, recursive = FALSE), decreasing = !ascending) + list(part[ord[1:num]]) + } else { + list(part) + } + } + + reduceFunc <- function(elems, part) { + newElems <- append(elems, part) + # R limitation: order works only on primitive types! + ord <- order(unlist(newElems, recursive = FALSE), decreasing = !ascending) + newElems[ord[1:num]] + } + + newRdd <- mapPartitions(x, partitionFunc) + reduce(newRdd, reduceFunc) +} + +#' Returns the first N elements from an RDD in ascending order. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The first N elements from the RDD in ascending order. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' takeOrdered(rdd, 6L) # list(1, 2, 3, 4, 5, 6) +#'} +#' @rdname takeOrdered +#' @aliases takeOrdered,RDD,RDD-method +setMethod("takeOrdered", + signature(x = "RDD", num = "integer"), + function(x, num) { + takeOrderedElem(x, num) + }) + +#' Returns the top N elements from an RDD. +#' +#' @param x An RDD. +#' @param num Number of elements to return. +#' @return The top N elements from the RDD. +#' @rdname top +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(10, 1, 2, 9, 3, 4, 5, 6, 7)) +#' top(rdd, 6L) # list(10, 9, 7, 6, 5, 4) +#'} +#' @rdname top +#' @aliases top,RDD,RDD-method +setMethod("top", + signature(x = "RDD", num = "integer"), + function(x, num) { + takeOrderedElem(x, num, FALSE) + }) + +#' Fold an RDD using a given associative function and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using a given associative function and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param op An associative function for the folding operation. +#' @return The folding result. +#' @rdname fold +#' @seealso reduce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4, 5)) +#' fold(rdd, 0, "+") # 15 +#'} +#' @rdname fold +#' @aliases fold,RDD,RDD-method +setMethod("fold", + signature(x = "RDD", zeroValue = "ANY", op = "ANY"), + function(x, zeroValue, op) { + aggregateRDD(x, zeroValue, op, op) + }) + +#' Aggregate an RDD using the given combine functions and a neutral "zero value". +#' +#' Aggregate the elements of each partition, and then the results for all the +#' partitions, using given combine functions and a neutral "zero value". +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param seqOp A function to aggregate the RDD elements. It may return a different +#' result type from the type of the RDD elements. +#' @param combOp A function to aggregate results of seqOp. +#' @return The aggregation result. +#' @rdname aggregateRDD +#' @seealso reduce +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1, 2, 3, 4)) +#' zeroValue <- list(0, 0) +#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +#' aggregateRDD(rdd, zeroValue, seqOp, combOp) # list(10, 4) +#'} +#' @rdname aggregateRDD +#' @aliases aggregateRDD,RDD,RDD-method +setMethod("aggregateRDD", + signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", combOp = "ANY"), + function(x, zeroValue, seqOp, combOp) { + partitionFunc <- function(part) { + Reduce(seqOp, part, zeroValue) + } + + partitionList <- collect(lapplyPartition(x, partitionFunc), + flatten = FALSE) + Reduce(combOp, partitionList, zeroValue) + }) + +#' Pipes elements to a forked external process. +#' +#' The same as 'pipe()' in Spark. +#' +#' @param x The RDD whose elements are piped to the forked external process. +#' @param command The command to fork an external process. +#' @param env A named list to set environment variables of the external process. +#' @return A new RDD created by piping all elements to a forked external process. +#' @rdname pipeRDD +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' collect(pipeRDD(rdd, "more") +#' Output: c("1", "2", ..., "10") +#'} +#' @rdname pipeRDD +#' @aliases pipeRDD,RDD,character-method +setMethod("pipeRDD", + signature(x = "RDD", command = "character"), + function(x, command, env = list()) { + func <- function(part) { + trim.trailing.func <- function(x) { + sub("[\r\n]*$", "", toString(x)) + } + input <- unlist(lapply(part, trim.trailing.func)) + res <- system2(command, stdout = TRUE, input = input, env = env) + lapply(res, trim.trailing.func) + } + lapplyPartition(x, func) + }) + +# TODO: Consider caching the name in the RDD's environment +#' Return an RDD's name. +#' +#' @param x The RDD whose name is returned. +#' @rdname name +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' name(rdd) # NULL (if not set before) +#'} +#' @rdname name +#' @aliases name,RDD +setMethod("name", + signature(x = "RDD"), + function(x) { + callJMethod(getJRDD(x), "name") + }) + +#' Set an RDD's name. +#' +#' @param x The RDD whose name is to be set. +#' @param name The RDD name to be set. +#' @return a new RDD renamed. +#' @rdname setName +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(1,2,3)) +#' setName(rdd, "myRDD") +#' name(rdd) # "myRDD" +#'} +#' @rdname setName +#' @aliases setName,RDD +setMethod("setName", + signature(x = "RDD", name = "character"), + function(x, name) { + callJMethod(getJRDD(x), "setName", name) + x + }) + +#' Zip an RDD with generated unique Long IDs. +#' +#' Items in the kth partition will get ids k, n+k, 2*n+k, ..., where +#' n is the number of partitions. So there may exist gaps, but this +#' method won't trigger a spark job, which is different from +#' zipWithIndex. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @seealso zipWithIndex +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithUniqueId(rdd)) +#' # list(list("a", 0), list("b", 3), list("c", 1), list("d", 4), list("e", 2)) +#'} +#' @rdname zipWithUniqueId +#' @aliases zipWithUniqueId,RDD +setMethod("zipWithUniqueId", + signature(x = "RDD"), + function(x) { + n <- numPartitions(x) + + partitionFunc <- function(split, part) { + mapply( + function(item, index) { + list(item, (index - 1) * n + split) + }, + part, + seq_along(part), + SIMPLIFY = FALSE) + } + + lapplyPartitionsWithIndex(x, partitionFunc) + }) + +#' Zip an RDD with its element indices. +#' +#' The ordering is first based on the partition index and then the +#' ordering of items within each partition. So the first item in +#' the first partition gets index 0, and the last item in the last +#' partition receives the largest index. +#' +#' This method needs to trigger a Spark job when this RDD contains +#' more than one partition. +#' +#' @param x An RDD to be zipped. +#' @return An RDD with zipped items. +#' @seealso zipWithUniqueId +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) +#' collect(zipWithIndex(rdd)) +#' # list(list("a", 0), list("b", 1), list("c", 2), list("d", 3), list("e", 4)) +#'} +#' @rdname zipWithIndex +#' @aliases zipWithIndex,RDD +setMethod("zipWithIndex", + signature(x = "RDD"), + function(x) { + n <- numPartitions(x) + if (n > 1) { + nums <- collect(lapplyPartition(x, + function(part) { + list(length(part)) + })) + startIndices <- Reduce("+", nums, accumulate = TRUE) + } + + partitionFunc <- function(split, part) { + if (split == 0) { + startIndex <- 0 + } else { + startIndex <- startIndices[[split]] + } + + mapply( + function(item, index) { + list(item, index - 1 + startIndex) + }, + part, + seq_along(part), + SIMPLIFY = FALSE) + } + + lapplyPartitionsWithIndex(x, partitionFunc) + }) + +#' Coalesce all elements within each partition of an RDD into a list. +#' +#' @param x An RDD. +#' @return An RDD created by coalescing all elements within +#' each partition into a list. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, as.list(1:4), 2L) +#' collect(glom(rdd)) +#' # list(list(1, 2), list(3, 4)) +#'} +#' @rdname glom +#' @aliases glom,RDD +setMethod("glom", + signature(x = "RDD"), + function(x) { + partitionFunc <- function(part) { + list(part) + } + + lapplyPartition(x, partitionFunc) + }) + +############ Binary Functions ############# + +#' Return the union RDD of two RDDs. +#' The same as union() in Spark. +#' +#' @param x An RDD. +#' @param y An RDD. +#' @return a new RDD created by performing the simple union (witout removing +#' duplicates) of two input RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:3) +#' unionRDD(rdd, rdd) # 1, 2, 3, 1, 2, 3 +#'} +#' @rdname unionRDD +#' @aliases unionRDD,RDD,RDD-method +setMethod("unionRDD", + signature(x = "RDD", y = "RDD"), + function(x, y) { + if (getSerializedMode(x) == getSerializedMode(y)) { + jrdd <- callJMethod(getJRDD(x), "union", getJRDD(y)) + union.rdd <- RDD(jrdd, getSerializedMode(x)) + } else { + # One of the RDDs is not serialized, we need to serialize it first. + if (getSerializedMode(x) != "byte") x <- serializeToBytes(x) + if (getSerializedMode(y) != "byte") y <- serializeToBytes(y) + jrdd <- callJMethod(getJRDD(x), "union", getJRDD(y)) + union.rdd <- RDD(jrdd, "byte") + } + union.rdd + }) + +#' Zip an RDD with another RDD. +#' +#' Zips this RDD with another one, returning key-value pairs with the +#' first element in each RDD second element in each RDD, etc. Assumes +#' that the two RDDs have the same number of partitions and the same +#' number of elements in each partition (e.g. one was made through +#' a map on the other). +#' +#' @param x An RDD to be zipped. +#' @param other Another RDD to be zipped. +#' @return An RDD zipped from the two RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, 0:4) +#' rdd2 <- parallelize(sc, 1000:1004) +#' collect(zipRDD(rdd1, rdd2)) +#' # list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004)) +#'} +#' @rdname zipRDD +#' @aliases zipRDD,RDD +setMethod("zipRDD", + signature(x = "RDD", other = "RDD"), + function(x, other) { + n1 <- numPartitions(x) + n2 <- numPartitions(other) + if (n1 != n2) { + stop("Can only zip RDDs which have the same number of partitions.") + } + + if (getSerializedMode(x) != getSerializedMode(other) || + getSerializedMode(x) == "byte") { + # Append the number of elements in each partition to that partition so that we can later + # check if corresponding partitions of both RDDs have the same number of elements. + # + # Note that this appending also serves the purpose of reserialization, because even if + # any RDD is serialized, we need to reserialize it to make sure its partitions are encoded + # as a single byte array. For example, partitions of an RDD generated from partitionBy() + # may be encoded as multiple byte arrays. + appendLength <- function(part) { + part[[length(part) + 1]] <- length(part) + 1 + part + } + x <- lapplyPartition(x, appendLength) + other <- lapplyPartition(other, appendLength) + } + + zippedJRDD <- callJMethod(getJRDD(x), "zip", getJRDD(other)) + # The zippedRDD's elements are of scala Tuple2 type. The serialized + # flag Here is used for the elements inside the tuples. + serializerMode <- getSerializedMode(x) + zippedRDD <- RDD(zippedJRDD, serializerMode) + + partitionFunc <- function(split, part) { + len <- length(part) + if (len > 0) { + if (serializerMode == "byte") { + lengthOfValues <- part[[len]] + lengthOfKeys <- part[[len - lengthOfValues]] + stopifnot(len == lengthOfKeys + lengthOfValues) + + # check if corresponding partitions of both RDDs have the same number of elements. + if (lengthOfKeys != lengthOfValues) { + stop("Can only zip RDDs with same number of elements in each pair of corresponding partitions.") + } + + if (lengthOfKeys > 1) { + keys <- part[1 : (lengthOfKeys - 1)] + values <- part[(lengthOfKeys + 1) : (len - 1)] + } else { + keys <- list() + values <- list() + } + } else { + # Keys, values must have same length here, because this has + # been validated inside the JavaRDD.zip() function. + keys <- part[c(TRUE, FALSE)] + values <- part[c(FALSE, TRUE)] + } + mapply( + function(k, v) { + list(k, v) + }, + keys, + values, + SIMPLIFY = FALSE, + USE.NAMES = FALSE) + } else { + part + } + } + + PipelinedRDD(zippedRDD, partitionFunc) + }) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R new file mode 100644 index 0000000000000..930ada22f4c38 --- /dev/null +++ b/R/pkg/R/SQLContext.R @@ -0,0 +1,520 @@ +# +# 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. +# + +# SQLcontext.R: SQLContext-driven functions + +#' infer the SQL type +infer_type <- function(x) { + if (is.null(x)) { + stop("can not infer type from NULL") + } + + # class of POSIXlt is c("POSIXlt" "POSIXt") + type <- switch(class(x)[[1]], + integer = "integer", + character = "string", + logical = "boolean", + double = "double", + numeric = "double", + raw = "binary", + list = "array", + environment = "map", + Date = "date", + POSIXlt = "timestamp", + POSIXct = "timestamp", + stop(paste("Unsupported type for DataFrame:", class(x)))) + + if (type == "map") { + stopifnot(length(x) > 0) + key <- ls(x)[[1]] + list(type = "map", + keyType = "string", + valueType = infer_type(get(key, x)), + valueContainsNull = TRUE) + } else if (type == "array") { + stopifnot(length(x) > 0) + names <- names(x) + if (is.null(names)) { + list(type = "array", elementType = infer_type(x[[1]]), containsNull = TRUE) + } else { + # StructType + types <- lapply(x, infer_type) + fields <- lapply(1:length(x), function(i) { + list(name = names[[i]], type = types[[i]], nullable = TRUE) + }) + list(type = "struct", fields = fields) + } + } else if (length(x) > 1) { + list(type = "array", elementType = type, containsNull = TRUE) + } else { + type + } +} + +#' dump the schema into JSON string +tojson <- function(x) { + if (is.list(x)) { + names <- names(x) + if (!is.null(names)) { + items <- lapply(names, function(n) { + safe_n <- gsub('"', '\\"', n) + paste(tojson(safe_n), ':', tojson(x[[n]]), sep = '') + }) + d <- paste(items, collapse = ', ') + paste('{', d, '}', sep = '') + } else { + l <- paste(lapply(x, tojson), collapse = ', ') + paste('[', l, ']', sep = '') + } + } else if (is.character(x)) { + paste('"', x, '"', sep = '') + } else if (is.logical(x)) { + if (x) "true" else "false" + } else { + stop(paste("unexpected type:", class(x))) + } +} + +#' Create a DataFrame from an RDD +#' +#' Converts an RDD to a DataFrame by infer the types. +#' +#' @param sqlCtx A SQLContext +#' @param data An RDD or list or data.frame +#' @param schema a list of column names or named list (StructType), optional +#' @return an DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) +#' df <- createDataFrame(sqlCtx, rdd) +#' } + +# TODO(davies): support sampling and infer type from NA +createDataFrame <- function(sqlCtx, data, schema = NULL, samplingRatio = 1.0) { + if (is.data.frame(data)) { + # get the names of columns, they will be put into RDD + schema <- names(data) + n <- nrow(data) + m <- ncol(data) + # get rid of factor type + dropFactor <- function(x) { + if (is.factor(x)) { + as.character(x) + } else { + x + } + } + data <- lapply(1:n, function(i) { + lapply(1:m, function(j) { dropFactor(data[i,j]) }) + }) + } + if (is.list(data)) { + sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sqlCtx) + rdd <- parallelize(sc, data) + } else if (inherits(data, "RDD")) { + rdd <- data + } else { + stop(paste("unexpected type:", class(data))) + } + + if (is.null(schema) || is.null(names(schema))) { + row <- first(rdd) + names <- if (is.null(schema)) { + names(row) + } else { + as.list(schema) + } + if (is.null(names)) { + names <- lapply(1:length(row), function(x) { + paste("_", as.character(x), sep = "") + }) + } + + # SPAKR-SQL does not support '.' in column name, so replace it with '_' + # TODO(davies): remove this once SPARK-2775 is fixed + names <- lapply(names, function(n) { + nn <- gsub("[.]", "_", n) + if (nn != n) { + warning(paste("Use", nn, "instead of", n, " as column name")) + } + nn + }) + + types <- lapply(row, infer_type) + fields <- lapply(1:length(row), function(i) { + list(name = names[[i]], type = types[[i]], nullable = TRUE) + }) + schema <- list(type = "struct", fields = fields) + } + + stopifnot(class(schema) == "list") + stopifnot(schema$type == "struct") + stopifnot(class(schema$fields) == "list") + schemaString <- tojson(schema) + + jrdd <- getJRDD(lapply(rdd, function(x) x), "row") + srdd <- callJMethod(jrdd, "rdd") + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", + srdd, schemaString, sqlCtx) + dataFrame(sdf) +} + +#' toDF +#' +#' Converts an RDD to a DataFrame by infer the types. +#' +#' @param x An RDD +#' +#' @rdname DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- lapply(parallelize(sc, 1:10), function(x) list(a=x, b=as.character(x))) +#' df <- toDF(rdd) +#' } + +setGeneric("toDF", function(x, ...) { standardGeneric("toDF") }) + +setMethod("toDF", signature(x = "RDD"), + function(x, ...) { + sqlCtx <- if (exists(".sparkRHivesc", envir = .sparkREnv)) { + get(".sparkRHivesc", envir = .sparkREnv) + } else if (exists(".sparkRSQLsc", envir = .sparkREnv)) { + get(".sparkRSQLsc", envir = .sparkREnv) + } else { + stop("no SQL context available") + } + createDataFrame(sqlCtx, x, ...) + }) + +#' Create a DataFrame from a JSON file. +#' +#' Loads a JSON file (one object per line), returning the result as a DataFrame +#' It goes through the entire dataset once to determine the schema. +#' +#' @param sqlCtx SQLContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' } + +jsonFile <- function(sqlCtx, path) { + # Allow the user to have a more flexible definiton of the text file path + path <- normalizePath(path) + # Convert a string vector of paths to a string containing comma separated paths + path <- paste(path, collapse = ",") + sdf <- callJMethod(sqlCtx, "jsonFile", path) + dataFrame(sdf) +} + + +#' JSON RDD +#' +#' Loads an RDD storing one JSON object per string as a DataFrame. +#' +#' @param sqlCtx SQLContext to use +#' @param rdd An RDD of JSON string +#' @param schema A StructType object to use as schema +#' @param samplingRatio The ratio of simpling used to infer the schema +#' @return A DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' rdd <- texFile(sc, "path/to/json") +#' df <- jsonRDD(sqlCtx, rdd) +#' } + +# TODO: support schema +jsonRDD <- function(sqlCtx, rdd, schema = NULL, samplingRatio = 1.0) { + rdd <- serializeToString(rdd) + if (is.null(schema)) { + sdf <- callJMethod(sqlCtx, "jsonRDD", callJMethod(getJRDD(rdd), "rdd"), samplingRatio) + dataFrame(sdf) + } else { + stop("not implemented") + } +} + + +#' Create a DataFrame from a Parquet file. +#' +#' Loads a Parquet file, returning the result as a DataFrame. +#' +#' @param sqlCtx SQLContext to use +#' @param ... Path(s) of parquet file(s) to read. +#' @return DataFrame +#' @export + +# TODO: Implement saveasParquetFile and write examples for both +parquetFile <- function(sqlCtx, ...) { + # Allow the user to have a more flexible definiton of the text file path + paths <- lapply(list(...), normalizePath) + sdf <- callJMethod(sqlCtx, "parquetFile", paths) + dataFrame(sdf) +} + +#' SQL Query +#' +#' Executes a SQL query using Spark, returning the result as a DataFrame. +#' +#' @param sqlCtx SQLContext to use +#' @param sqlQuery A character vector containing the SQL query +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' new_df <- sql(sqlCtx, "SELECT * FROM table") +#' } + +sql <- function(sqlCtx, sqlQuery) { + sdf <- callJMethod(sqlCtx, "sql", sqlQuery) + dataFrame(sdf) +} + + +#' Create a DataFrame from a SparkSQL Table +#' +#' Returns the specified Table as a DataFrame. The Table must have already been registered +#' in the SQLContext. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The SparkSQL Table to convert to a DataFrame. +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' new_df <- table(sqlCtx, "table") +#' } + +table <- function(sqlCtx, tableName) { + sdf <- callJMethod(sqlCtx, "table", tableName) + dataFrame(sdf) +} + + +#' Tables +#' +#' Returns a DataFrame containing names of tables in the given database. +#' +#' @param sqlCtx SQLContext to use +#' @param databaseName name of the database +#' @return a DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' tables(sqlCtx, "hive") +#' } + +tables <- function(sqlCtx, databaseName = NULL) { + jdf <- if (is.null(databaseName)) { + callJMethod(sqlCtx, "tables") + } else { + callJMethod(sqlCtx, "tables", databaseName) + } + dataFrame(jdf) +} + + +#' Table Names +#' +#' Returns the names of tables in the given database as an array. +#' +#' @param sqlCtx SQLContext to use +#' @param databaseName name of the database +#' @return a list of table names +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' tableNames(sqlCtx, "hive") +#' } + +tableNames <- function(sqlCtx, databaseName = NULL) { + if (is.null(databaseName)) { + callJMethod(sqlCtx, "tableNames") + } else { + callJMethod(sqlCtx, "tableNames", databaseName) + } +} + + +#' Cache Table +#' +#' Caches the specified table in-memory. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The name of the table being cached +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' cacheTable(sqlCtx, "table") +#' } + +cacheTable <- function(sqlCtx, tableName) { + callJMethod(sqlCtx, "cacheTable", tableName) +} + +#' Uncache Table +#' +#' Removes the specified table from the in-memory cache. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The name of the table being uncached +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' path <- "path/to/file.json" +#' df <- jsonFile(sqlCtx, path) +#' registerTempTable(df, "table") +#' uncacheTable(sqlCtx, "table") +#' } + +uncacheTable <- function(sqlCtx, tableName) { + callJMethod(sqlCtx, "uncacheTable", tableName) +} + +#' Clear Cache +#' +#' Removes all cached tables from the in-memory cache. +#' +#' @param sqlCtx SQLContext to use +#' @examples +#' \dontrun{ +#' clearCache(sqlCtx) +#' } + +clearCache <- function(sqlCtx) { + callJMethod(sqlCtx, "clearCache") +} + +#' Drop Temporary Table +#' +#' Drops the temporary table with the given table name in the catalog. +#' If the table has been cached/persisted before, it's also unpersisted. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName The name of the SparkSQL table to be dropped. +#' @examples +#' \dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- loadDF(sqlCtx, path, "parquet") +#' registerTempTable(df, "table") +#' dropTempTable(sqlCtx, "table") +#' } + +dropTempTable <- function(sqlCtx, tableName) { + if (class(tableName) != "character") { + stop("tableName must be a string.") + } + callJMethod(sqlCtx, "dropTempTable", tableName) +} + +#' Load an DataFrame +#' +#' Returns the dataset in a data source as a DataFrame +#' +#' The data source is specified by the `source` and a set of options(...). +#' If `source` is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. +#' +#' @param sqlCtx SQLContext to use +#' @param path The path of files to load +#' @param source the name of external data source +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- load(sqlCtx, "path/to/file.json", source = "json") +#' } + +loadDF <- function(sqlCtx, path = NULL, source = NULL, ...) { + options <- varargsToEnv(...) + if (!is.null(path)) { + options[['path']] <- path + } + sdf <- callJMethod(sqlCtx, "load", source, options) + dataFrame(sdf) +} + +#' Create an external table +#' +#' Creates an external table based on the dataset in a data source, +#' Returns the DataFrame associated with the external table. +#' +#' The data source is specified by the `source` and a set of options(...). +#' If `source` is not specified, the default data source configured by +#' "spark.sql.sources.default" will be used. +#' +#' @param sqlCtx SQLContext to use +#' @param tableName A name of the table +#' @param path The path of files to load +#' @param source the name of external data source +#' @return DataFrame +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#' df <- sparkRSQL.createExternalTable(sqlCtx, "myjson", path="path/to/json", source="json") +#' } + +createExternalTable <- function(sqlCtx, tableName, path = NULL, source = NULL, ...) { + options <- varargsToEnv(...) + if (!is.null(path)) { + options[['path']] <- path + } + sdf <- callJMethod(sqlCtx, "createExternalTable", tableName, source, options) + dataFrame(sdf) +} diff --git a/R/pkg/R/SQLTypes.R b/R/pkg/R/SQLTypes.R new file mode 100644 index 0000000000000..962fba5b3cf03 --- /dev/null +++ b/R/pkg/R/SQLTypes.R @@ -0,0 +1,64 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Utility functions for handling SparkSQL DataTypes. + +# Handler for StructType +structType <- function(st) { + obj <- structure(new.env(parent = emptyenv()), class = "structType") + obj$jobj <- st + obj$fields <- function() { lapply(callJMethod(st, "fields"), structField) } + obj +} + +#' Print a Spark StructType. +#' +#' This function prints the contents of a StructType returned from the +#' SparkR JVM backend. +#' +#' @param x A StructType object +#' @param ... further arguments passed to or from other methods +print.structType <- function(x, ...) { + fieldsList <- lapply(x$fields(), function(i) { i$print() }) + print(fieldsList) +} + +# Handler for StructField +structField <- function(sf) { + obj <- structure(new.env(parent = emptyenv()), class = "structField") + obj$jobj <- sf + obj$name <- function() { callJMethod(sf, "name") } + obj$dataType <- function() { callJMethod(sf, "dataType") } + obj$dataType.toString <- function() { callJMethod(obj$dataType(), "toString") } + obj$dataType.simpleString <- function() { callJMethod(obj$dataType(), "simpleString") } + obj$nullable <- function() { callJMethod(sf, "nullable") } + obj$print <- function() { paste("StructField(", + paste(obj$name(), obj$dataType.toString(), obj$nullable(), sep = ", "), + ")", sep = "") } + obj +} + +#' Print a Spark StructField. +#' +#' This function prints the contents of a StructField returned from the +#' SparkR JVM backend. +#' +#' @param x A StructField object +#' @param ... further arguments passed to or from other methods +print.structField <- function(x, ...) { + cat(x$print()) +} diff --git a/R/pkg/R/backend.R b/R/pkg/R/backend.R new file mode 100644 index 0000000000000..2fb6fae55f28c --- /dev/null +++ b/R/pkg/R/backend.R @@ -0,0 +1,115 @@ +# +# 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. +# + +# Methods to call into SparkRBackend. + + +# Returns TRUE if object is an instance of given class +isInstanceOf <- function(jobj, className) { + stopifnot(class(jobj) == "jobj") + cls <- callJStatic("java.lang.Class", "forName", className) + callJMethod(cls, "isInstance", jobj) +} + +# Call a Java method named methodName on the object +# specified by objId. objId should be a "jobj" returned +# from the SparkRBackend. +callJMethod <- function(objId, methodName, ...) { + stopifnot(class(objId) == "jobj") + if (!isValidJobj(objId)) { + stop("Invalid jobj ", objId$id, + ". If SparkR was restarted, Spark operations need to be re-executed.") + } + invokeJava(isStatic = FALSE, objId$id, methodName, ...) +} + +# Call a static method on a specified className +callJStatic <- function(className, methodName, ...) { + invokeJava(isStatic = TRUE, className, methodName, ...) +} + +# Create a new object of the specified class name +newJObject <- function(className, ...) { + invokeJava(isStatic = TRUE, className, methodName = "", ...) +} + +# Remove an object from the SparkR backend. This is done +# automatically when a jobj is garbage collected. +removeJObject <- function(objId) { + invokeJava(isStatic = TRUE, "SparkRHandler", "rm", objId) +} + +isRemoveMethod <- function(isStatic, objId, methodName) { + isStatic == TRUE && objId == "SparkRHandler" && methodName == "rm" +} + +# Invoke a Java method on the SparkR backend. Users +# should typically use one of the higher level methods like +# callJMethod, callJStatic etc. instead of using this. +# +# isStatic - TRUE if the method to be called is static +# objId - String that refers to the object on which method is invoked +# Should be a jobj id for non-static methods and the classname +# for static methods +# methodName - name of method to be invoked +invokeJava <- function(isStatic, objId, methodName, ...) { + if (!exists(".sparkRCon", .sparkREnv)) { + stop("No connection to backend found. Please re-run sparkR.init") + } + + # If this isn't a removeJObject call + if (!isRemoveMethod(isStatic, objId, methodName)) { + objsToRemove <- ls(.toRemoveJobjs) + if (length(objsToRemove) > 0) { + sapply(objsToRemove, + function(e) { + removeJObject(e) + }) + rm(list = objsToRemove, envir = .toRemoveJobjs) + } + } + + + rc <- rawConnection(raw(0), "r+") + + writeBoolean(rc, isStatic) + writeString(rc, objId) + writeString(rc, methodName) + + args <- list(...) + writeInt(rc, length(args)) + writeArgs(rc, args) + + # Construct the whole request message to send it once, + # avoiding write-write-read pattern in case of Nagle's algorithm. + # Refer to http://en.wikipedia.org/wiki/Nagle%27s_algorithm for the details. + bytesToSend <- rawConnectionValue(rc) + close(rc) + rc <- rawConnection(raw(0), "r+") + writeInt(rc, length(bytesToSend)) + writeBin(bytesToSend, rc) + requestMessage <- rawConnectionValue(rc) + close(rc) + + conn <- get(".sparkRCon", .sparkREnv) + writeBin(requestMessage, conn) + + # TODO: check the status code to output error information + returnStatus <- readInt(conn) + stopifnot(returnStatus == 0) + readObject(conn) +} diff --git a/R/pkg/R/broadcast.R b/R/pkg/R/broadcast.R new file mode 100644 index 0000000000000..583fa2e7fdcfd --- /dev/null +++ b/R/pkg/R/broadcast.R @@ -0,0 +1,86 @@ +# +# 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. +# + +# S4 class representing Broadcast variables + +# Hidden environment that holds values for broadcast variables +# This will not be serialized / shipped by default +.broadcastNames <- new.env() +.broadcastValues <- new.env() +.broadcastIdToName <- new.env() + +#' @title S4 class that represents a Broadcast variable +#' @description Broadcast variables can be created using the broadcast +#' function from a \code{SparkContext}. +#' @rdname broadcast-class +#' @seealso broadcast +#' +#' @param id Id of the backing Spark broadcast variable +#' @export +setClass("Broadcast", slots = list(id = "character")) + +#' @rdname broadcast-class +#' @param value Value of the broadcast variable +#' @param jBroadcastRef reference to the backing Java broadcast object +#' @param objName name of broadcasted object +#' @export +Broadcast <- function(id, value, jBroadcastRef, objName) { + .broadcastValues[[id]] <- value + .broadcastNames[[as.character(objName)]] <- jBroadcastRef + .broadcastIdToName[[id]] <- as.character(objName) + new("Broadcast", id = id) +} + +#' @description +#' \code{value} can be used to get the value of a broadcast variable inside +#' a distributed function. +#' +#' @param bcast The broadcast variable to get +#' @rdname broadcast +#' @aliases value,Broadcast-method +setMethod("value", + signature(bcast = "Broadcast"), + function(bcast) { + if (exists(bcast@id, envir = .broadcastValues)) { + get(bcast@id, envir = .broadcastValues) + } else { + NULL + } + }) + +#' Internal function to set values of a broadcast variable. +#' +#' This function is used internally by Spark to set the value of a broadcast +#' variable on workers. Not intended for use outside the package. +#' +#' @rdname broadcast-internal +#' @seealso broadcast, value + +#' @param bcastId The id of broadcast variable to set +#' @param value The value to be set +#' @export +setBroadcastValue <- function(bcastId, value) { + bcastIdStr <- as.character(bcastId) + .broadcastValues[[bcastIdStr]] <- value +} + +#' Helper function to clear the list of broadcast variables we know about +#' Should be called when the SparkR JVM backend is shutdown +clearBroadcastVariables <- function() { + bcasts <- ls(.broadcastNames) + rm(list = bcasts, envir = .broadcastNames) +} diff --git a/R/pkg/R/client.R b/R/pkg/R/client.R new file mode 100644 index 0000000000000..1281c41213e32 --- /dev/null +++ b/R/pkg/R/client.R @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Client code to connect to SparkRBackend + +# Creates a SparkR client connection object +# if one doesn't already exist +connectBackend <- function(hostname, port, timeout = 6000) { + if (exists(".sparkRcon", envir = .sparkREnv)) { + if (isOpen(.sparkREnv[[".sparkRCon"]])) { + cat("SparkRBackend client connection already exists\n") + return(get(".sparkRcon", envir = .sparkREnv)) + } + } + + con <- socketConnection(host = hostname, port = port, server = FALSE, + blocking = TRUE, open = "wb", timeout = timeout) + + assign(".sparkRCon", con, envir = .sparkREnv) + con +} + +launchBackend <- function(args, sparkHome, jars, sparkSubmitOpts) { + if (.Platform$OS.type == "unix") { + sparkSubmitBinName = "spark-submit" + } else { + sparkSubmitBinName = "spark-submit.cmd" + } + + if (sparkHome != "") { + sparkSubmitBin <- file.path(sparkHome, "bin", sparkSubmitBinName) + } else { + sparkSubmitBin <- sparkSubmitBinName + } + + if (jars != "") { + jars <- paste("--jars", jars) + } + + combinedArgs <- paste(jars, sparkSubmitOpts, args, sep = " ") + cat("Launching java with spark-submit command", sparkSubmitBin, combinedArgs, "\n") + invisible(system2(sparkSubmitBin, combinedArgs, wait = F)) +} diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R new file mode 100644 index 0000000000000..e196305186b9a --- /dev/null +++ b/R/pkg/R/column.R @@ -0,0 +1,199 @@ +# +# 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. +# + +# Column Class + +#' @include generics.R jobj.R +NULL + +setOldClass("jobj") + +#' @title S4 class that represents a DataFrame column +#' @description The column class supports unary, binary operations on DataFrame columns + +#' @rdname column +#' +#' @param jc reference to JVM DataFrame column +#' @export +setClass("Column", + slots = list(jc = "jobj")) + +setMethod("initialize", "Column", function(.Object, jc) { + .Object@jc <- jc + .Object +}) + +column <- function(jc) { + new("Column", jc) +} + +col <- function(x) { + column(callJStatic("org.apache.spark.sql.functions", "col", x)) +} + +#' @rdname show +setMethod("show", "Column", + function(object) { + cat("Column", callJMethod(object@jc, "toString"), "\n") + }) + +operators <- list( + "+" = "plus", "-" = "minus", "*" = "multiply", "/" = "divide", "%%" = "mod", + "==" = "equalTo", ">" = "gt", "<" = "lt", "!=" = "notEqual", "<=" = "leq", ">=" = "geq", + # we can not override `&&` and `||`, so use `&` and `|` instead + "&" = "and", "|" = "or" #, "!" = "unary_$bang" +) +column_functions1 <- c("asc", "desc", "isNull", "isNotNull") +column_functions2 <- c("like", "rlike", "startsWith", "endsWith", "getField", "getItem", "contains") +functions <- c("min", "max", "sum", "avg", "mean", "count", "abs", "sqrt", + "first", "last", "lower", "upper", "sumDistinct") + +createOperator <- function(op) { + setMethod(op, + signature(e1 = "Column"), + function(e1, e2) { + jc <- if (missing(e2)) { + if (op == "-") { + callJMethod(e1@jc, "unary_$minus") + } else { + callJMethod(e1@jc, operators[[op]]) + } + } else { + if (class(e2) == "Column") { + e2 <- e2@jc + } + callJMethod(e1@jc, operators[[op]], e2) + } + column(jc) + }) +} + +createColumnFunction1 <- function(name) { + setMethod(name, + signature(x = "Column"), + function(x) { + column(callJMethod(x@jc, name)) + }) +} + +createColumnFunction2 <- function(name) { + setMethod(name, + signature(x = "Column"), + function(x, data) { + if (class(data) == "Column") { + data <- data@jc + } + jc <- callJMethod(x@jc, name, data) + column(jc) + }) +} + +createStaticFunction <- function(name) { + setMethod(name, + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", name, x@jc) + column(jc) + }) +} + +createMethods <- function() { + for (op in names(operators)) { + createOperator(op) + } + for (name in column_functions1) { + createColumnFunction1(name) + } + for (name in column_functions2) { + createColumnFunction2(name) + } + for (x in functions) { + createStaticFunction(x) + } +} + +createMethods() + +#' alias +#' +#' Set a new name for a column +setMethod("alias", + signature(object = "Column"), + function(object, data) { + if (is.character(data)) { + column(callJMethod(object@jc, "as", data)) + } else { + stop("data should be character") + } + }) + +#' An expression that returns a substring. +#' +#' @param start starting position +#' @param stop ending position +setMethod("substr", signature(x = "Column"), + function(x, start, stop) { + jc <- callJMethod(x@jc, "substr", as.integer(start - 1), as.integer(stop - start + 1)) + column(jc) + }) + +#' Casts the column to a different data type. +#' @examples +#' \dontrun{ +#' cast(df$age, "string") +#' cast(df$name, list(type="array", elementType="byte", containsNull = TRUE)) +#' } +setMethod("cast", + signature(x = "Column"), + function(x, dataType) { + if (is.character(dataType)) { + column(callJMethod(x@jc, "cast", dataType)) + } else if (is.list(dataType)) { + json <- tojson(dataType) + jdataType <- callJStatic("org.apache.spark.sql.types.DataType", "fromJson", json) + column(callJMethod(x@jc, "cast", jdataType)) + } else { + stop("dataType should be character or list") + } + }) + +#' Approx Count Distinct +#' +#' Returns the approximate number of distinct items in a group. +#' +setMethod("approxCountDistinct", + signature(x = "Column"), + function(x, rsd = 0.95) { + jc <- callJStatic("org.apache.spark.sql.functions", "approxCountDistinct", x@jc, rsd) + column(jc) + }) + +#' Count Distinct +#' +#' returns the number of distinct items in a group. +#' +setMethod("countDistinct", + signature(x = "Column"), + function(x, ...) { + jcol <- lapply(list(...), function (x) { + x@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "countDistinct", x@jc, + listToSeq(jcol)) + column(jc) + }) + diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R new file mode 100644 index 0000000000000..2fc0bb294bcce --- /dev/null +++ b/R/pkg/R/context.R @@ -0,0 +1,225 @@ +# +# 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. +# + +# context.R: SparkContext driven functions + +getMinSplits <- function(sc, minSplits) { + if (is.null(minSplits)) { + defaultParallelism <- callJMethod(sc, "defaultParallelism") + minSplits <- min(defaultParallelism, 2) + } + as.integer(minSplits) +} + +#' Create an RDD from a text file. +#' +#' This function reads a text file from HDFS, a local file system (available on all +#' nodes), or any Hadoop-supported file system URI, and creates an +#' RDD of strings from it. +#' +#' @param sc SparkContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param minSplits Minimum number of splits to be created. If NULL, the default +#' value is chosen based on available parallelism. +#' @return RDD where each item is of type \code{character} +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' lines <- textFile(sc, "myfile.txt") +#'} +textFile <- function(sc, path, minSplits = NULL) { + # Allow the user to have a more flexible definiton of the text file path + path <- suppressWarnings(normalizePath(path)) + #' Convert a string vector of paths to a string containing comma separated paths + path <- paste(path, collapse = ",") + + jrdd <- callJMethod(sc, "textFile", path, getMinSplits(sc, minSplits)) + # jrdd is of type JavaRDD[String] + RDD(jrdd, "string") +} + +#' Load an RDD saved as a SequenceFile containing serialized objects. +#' +#' The file to be loaded should be one that was previously generated by calling +#' saveAsObjectFile() of the RDD class. +#' +#' @param sc SparkContext to use +#' @param path Path of file to read. A vector of multiple paths is allowed. +#' @param minSplits Minimum number of splits to be created. If NULL, the default +#' value is chosen based on available parallelism. +#' @return RDD containing serialized R objects. +#' @seealso saveAsObjectFile +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- objectFile(sc, "myfile") +#'} +objectFile <- function(sc, path, minSplits = NULL) { + # Allow the user to have a more flexible definiton of the text file path + path <- suppressWarnings(normalizePath(path)) + #' Convert a string vector of paths to a string containing comma separated paths + path <- paste(path, collapse = ",") + + jrdd <- callJMethod(sc, "objectFile", path, getMinSplits(sc, minSplits)) + # Assume the RDD contains serialized R objects. + RDD(jrdd, "byte") +} + +#' Create an RDD from a homogeneous list or vector. +#' +#' This function creates an RDD from a local homogeneous list in R. The elements +#' in the list are split into \code{numSlices} slices and distributed to nodes +#' in the cluster. +#' +#' @param sc SparkContext to use +#' @param coll collection to parallelize +#' @param numSlices number of partitions to create in the RDD +#' @return an RDD created from this collection +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10, 2) +#' # The RDD should contain 10 elements +#' length(rdd) +#'} +parallelize <- function(sc, coll, numSlices = 1) { + # TODO: bound/safeguard numSlices + # TODO: unit tests for if the split works for all primitives + # TODO: support matrix, data frame, etc + if ((!is.list(coll) && !is.vector(coll)) || is.data.frame(coll)) { + if (is.data.frame(coll)) { + message(paste("context.R: A data frame is parallelized by columns.")) + } else { + if (is.matrix(coll)) { + message(paste("context.R: A matrix is parallelized by elements.")) + } else { + message(paste("context.R: parallelize() currently only supports lists and vectors.", + "Calling as.list() to coerce coll into a list.")) + } + } + coll <- as.list(coll) + } + + if (numSlices > length(coll)) + numSlices <- length(coll) + + sliceLen <- ceiling(length(coll) / numSlices) + slices <- split(coll, rep(1:(numSlices + 1), each = sliceLen)[1:length(coll)]) + + # Serialize each slice: obtain a list of raws, or a list of lists (slices) of + # 2-tuples of raws + serializedSlices <- lapply(slices, serialize, connection = NULL) + + jrdd <- callJStatic("org.apache.spark.api.r.RRDD", + "createRDDFromArray", sc, serializedSlices) + + RDD(jrdd, "byte") +} + +#' Include this specified package on all workers +#' +#' This function can be used to include a package on all workers before the +#' user's code is executed. This is useful in scenarios where other R package +#' functions are used in a function passed to functions like \code{lapply}. +#' NOTE: The package is assumed to be installed on every node in the Spark +#' cluster. +#' +#' @param sc SparkContext to use +#' @param pkg Package name +#' +#' @export +#' @examples +#'\dontrun{ +#' library(Matrix) +#' +#' sc <- sparkR.init() +#' # Include the matrix library we will be using +#' includePackage(sc, Matrix) +#' +#' generateSparse <- function(x) { +#' sparseMatrix(i=c(1, 2, 3), j=c(1, 2, 3), x=c(1, 2, 3)) +#' } +#' +#' rdd <- lapplyPartition(parallelize(sc, 1:2, 2L), generateSparse) +#' collect(rdd) +#'} +includePackage <- function(sc, pkg) { + pkg <- as.character(substitute(pkg)) + if (exists(".packages", .sparkREnv)) { + packages <- .sparkREnv$.packages + } else { + packages <- list() + } + packages <- c(packages, pkg) + .sparkREnv$.packages <- packages +} + +#' @title Broadcast a variable to all workers +#' +#' @description +#' Broadcast a read-only variable to the cluster, returning a \code{Broadcast} +#' object for reading it in distributed functions. +#' +#' @param sc Spark Context to use +#' @param object Object to be broadcast +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:2, 2L) +#' +#' # Large Matrix object that we want to broadcast +#' randomMat <- matrix(nrow=100, ncol=10, data=rnorm(1000)) +#' randomMatBr <- broadcast(sc, randomMat) +#' +#' # Use the broadcast variable inside the function +#' useBroadcast <- function(x) { +#' sum(value(randomMatBr) * x) +#' } +#' sumRDD <- lapply(rdd, useBroadcast) +#'} +broadcast <- function(sc, object) { + objName <- as.character(substitute(object)) + serializedObj <- serialize(object, connection = NULL) + + jBroadcast <- callJMethod(sc, "broadcast", serializedObj) + id <- as.character(callJMethod(jBroadcast, "id")) + + Broadcast(id, object, jBroadcast, objName) +} + +#' @title Set the checkpoint directory +#' +#' Set the directory under which RDDs are going to be checkpointed. The +#' directory must be a HDFS path if running on a cluster. +#' +#' @param sc Spark Context to use +#' @param dirName Directory path +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' setCheckpointDir(sc, "~/checkpoints") +#' rdd <- parallelize(sc, 1:2, 2L) +#' checkpoint(rdd) +#'} +setCheckpointDir <- function(sc, dirName) { + invisible(callJMethod(sc, "setCheckpointDir", suppressWarnings(normalizePath(dirName)))) +} diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R new file mode 100644 index 0000000000000..257b435607ce8 --- /dev/null +++ b/R/pkg/R/deserialize.R @@ -0,0 +1,184 @@ +# +# 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. +# + +# Utility functions to deserialize objects from Java. + +# Type mapping from Java to R +# +# void -> NULL +# Int -> integer +# String -> character +# Boolean -> logical +# Double -> double +# Long -> double +# Array[Byte] -> raw +# Date -> Date +# Time -> POSIXct +# +# Array[T] -> list() +# Object -> jobj + +readObject <- function(con) { + # Read type first + type <- readType(con) + readTypedObject(con, type) +} + +readTypedObject <- function(con, type) { + switch (type, + "i" = readInt(con), + "c" = readString(con), + "b" = readBoolean(con), + "d" = readDouble(con), + "r" = readRaw(con), + "D" = readDate(con), + "t" = readTime(con), + "l" = readList(con), + "n" = NULL, + "j" = getJobj(readString(con)), + stop(paste("Unsupported type for deserialization", type))) +} + +readString <- function(con) { + stringLen <- readInt(con) + string <- readBin(con, raw(), stringLen, endian = "big") + rawToChar(string) +} + +readInt <- function(con) { + readBin(con, integer(), n = 1, endian = "big") +} + +readDouble <- function(con) { + readBin(con, double(), n = 1, endian = "big") +} + +readBoolean <- function(con) { + as.logical(readInt(con)) +} + +readType <- function(con) { + rawToChar(readBin(con, "raw", n = 1L)) +} + +readDate <- function(con) { + as.Date(readString(con)) +} + +readTime <- function(con) { + t <- readDouble(con) + as.POSIXct(t, origin = "1970-01-01") +} + +# We only support lists where all elements are of same type +readList <- function(con) { + type <- readType(con) + len <- readInt(con) + if (len > 0) { + l <- vector("list", len) + for (i in 1:len) { + l[[i]] <- readTypedObject(con, type) + } + l + } else { + list() + } +} + +readRaw <- function(con) { + dataLen <- readInt(con) + data <- readBin(con, raw(), as.integer(dataLen), endian = "big") +} + +readRawLen <- function(con, dataLen) { + data <- readBin(con, raw(), as.integer(dataLen), endian = "big") +} + +readDeserialize <- function(con) { + # We have two cases that are possible - In one, the entire partition is + # encoded as a byte array, so we have only one value to read. If so just + # return firstData + dataLen <- readInt(con) + firstData <- unserialize( + readBin(con, raw(), as.integer(dataLen), endian = "big")) + + # Else, read things into a list + dataLen <- readInt(con) + if (length(dataLen) > 0 && dataLen > 0) { + data <- list(firstData) + while (length(dataLen) > 0 && dataLen > 0) { + data[[length(data) + 1L]] <- unserialize( + readBin(con, raw(), as.integer(dataLen), endian = "big")) + dataLen <- readInt(con) + } + unlist(data, recursive = FALSE) + } else { + firstData + } +} + +readDeserializeRows <- function(inputCon) { + # readDeserializeRows will deserialize a DataOutputStream composed of + # a list of lists. Since the DOS is one continuous stream and + # the number of rows varies, we put the readRow function in a while loop + # that termintates when the next row is empty. + data <- list() + while(TRUE) { + row <- readRow(inputCon) + if (length(row) == 0) { + break + } + data[[length(data) + 1L]] <- row + } + data # this is a list of named lists now +} + +readRowList <- function(obj) { + # readRowList is meant for use inside an lapply. As a result, it is + # necessary to open a standalone connection for the row and consume + # the numCols bytes inside the read function in order to correctly + # deserialize the row. + rawObj <- rawConnection(obj, "r+") + on.exit(close(rawObj)) + readRow(rawObj) +} + +readRow <- function(inputCon) { + numCols <- readInt(inputCon) + if (length(numCols) > 0 && numCols > 0) { + lapply(1:numCols, function(x) { + obj <- readObject(inputCon) + if (is.null(obj)) { + NA + } else { + obj + } + }) # each row is a list now + } else { + list() + } +} + +# Take a single column as Array[Byte] and deserialize it into an atomic vector +readCol <- function(inputCon, numRows) { + # sapply can not work with POSIXlt + do.call(c, lapply(1:numRows, function(x) { + value <- readObject(inputCon) + # Replace NULL with NA so we can coerce to vectors + if (is.null(value)) NA else value + })) +} diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R new file mode 100644 index 0000000000000..5fb1ccaa84ee2 --- /dev/null +++ b/R/pkg/R/generics.R @@ -0,0 +1,543 @@ +# +# 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. +# + +############ RDD Actions and Transformations ############ + +#' @rdname aggregateRDD +#' @seealso reduce +#' @export +setGeneric("aggregateRDD", function(x, zeroValue, seqOp, combOp) { standardGeneric("aggregateRDD") }) + +#' @rdname cache-methods +#' @export +setGeneric("cache", function(x) { standardGeneric("cache") }) + +#' @rdname coalesce +#' @seealso repartition +#' @export +setGeneric("coalesce", function(x, numPartitions, ...) { standardGeneric("coalesce") }) + +#' @rdname checkpoint-methods +#' @export +setGeneric("checkpoint", function(x) { standardGeneric("checkpoint") }) + +#' @rdname collect-methods +#' @export +setGeneric("collect", function(x, ...) { standardGeneric("collect") }) + +#' @rdname collect-methods +#' @export +setGeneric("collectAsMap", function(x) { standardGeneric("collectAsMap") }) + +#' @rdname collect-methods +#' @export +setGeneric("collectPartition", + function(x, partitionId) { + standardGeneric("collectPartition") + }) + +#' @rdname count +#' @export +setGeneric("count", function(x) { standardGeneric("count") }) + +#' @rdname countByValue +#' @export +setGeneric("countByValue", function(x) { standardGeneric("countByValue") }) + +#' @rdname distinct +#' @export +setGeneric("distinct", function(x, numPartitions = 1L) { standardGeneric("distinct") }) + +#' @rdname filterRDD +#' @export +setGeneric("filterRDD", function(x, f) { standardGeneric("filterRDD") }) + +#' @rdname first +#' @export +setGeneric("first", function(x) { standardGeneric("first") }) + +#' @rdname flatMap +#' @export +setGeneric("flatMap", function(X, FUN) { standardGeneric("flatMap") }) + +#' @rdname fold +#' @seealso reduce +#' @export +setGeneric("fold", function(x, zeroValue, op) { standardGeneric("fold") }) + +#' @rdname foreach +#' @export +setGeneric("foreach", function(x, func) { standardGeneric("foreach") }) + +#' @rdname foreach +#' @export +setGeneric("foreachPartition", function(x, func) { standardGeneric("foreachPartition") }) + +# The jrdd accessor function. +setGeneric("getJRDD", function(rdd, ...) { standardGeneric("getJRDD") }) + +#' @rdname glom +#' @export +setGeneric("glom", function(x) { standardGeneric("glom") }) + +#' @rdname keyBy +#' @export +setGeneric("keyBy", function(x, func) { standardGeneric("keyBy") }) + +#' @rdname lapplyPartition +#' @export +setGeneric("lapplyPartition", function(X, FUN) { standardGeneric("lapplyPartition") }) + +#' @rdname lapplyPartitionsWithIndex +#' @export +setGeneric("lapplyPartitionsWithIndex", + function(X, FUN) { + standardGeneric("lapplyPartitionsWithIndex") + }) + +#' @rdname lapply +#' @export +setGeneric("map", function(X, FUN) { standardGeneric("map") }) + +#' @rdname lapplyPartition +#' @export +setGeneric("mapPartitions", function(X, FUN) { standardGeneric("mapPartitions") }) + +#' @rdname lapplyPartitionsWithIndex +#' @export +setGeneric("mapPartitionsWithIndex", + function(X, FUN) { standardGeneric("mapPartitionsWithIndex") }) + +#' @rdname maximum +#' @export +setGeneric("maximum", function(x) { standardGeneric("maximum") }) + +#' @rdname minimum +#' @export +setGeneric("minimum", function(x) { standardGeneric("minimum") }) + +#' @rdname sumRDD +#' @export +setGeneric("sumRDD", function(x) { standardGeneric("sumRDD") }) + +#' @rdname name +#' @export +setGeneric("name", function(x) { standardGeneric("name") }) + +#' @rdname numPartitions +#' @export +setGeneric("numPartitions", function(x) { standardGeneric("numPartitions") }) + +#' @rdname persist +#' @export +setGeneric("persist", function(x, newLevel) { standardGeneric("persist") }) + +#' @rdname pipeRDD +#' @export +setGeneric("pipeRDD", function(x, command, env = list()) { standardGeneric("pipeRDD")}) + +#' @rdname reduce +#' @export +setGeneric("reduce", function(x, func) { standardGeneric("reduce") }) + +#' @rdname repartition +#' @seealso coalesce +#' @export +setGeneric("repartition", function(x, numPartitions) { standardGeneric("repartition") }) + +#' @rdname sampleRDD +#' @export +setGeneric("sampleRDD", + function(x, withReplacement, fraction, seed) { + standardGeneric("sampleRDD") + }) + +#' @rdname saveAsObjectFile +#' @seealso objectFile +#' @export +setGeneric("saveAsObjectFile", function(x, path) { standardGeneric("saveAsObjectFile") }) + +#' @rdname saveAsTextFile +#' @export +setGeneric("saveAsTextFile", function(x, path) { standardGeneric("saveAsTextFile") }) + +#' @rdname setName +#' @export +setGeneric("setName", function(x, name) { standardGeneric("setName") }) + +#' @rdname sortBy +#' @export +setGeneric("sortBy", + function(x, func, ascending = TRUE, numPartitions = 1L) { + standardGeneric("sortBy") + }) + +#' @rdname take +#' @export +setGeneric("take", function(x, num) { standardGeneric("take") }) + +#' @rdname takeOrdered +#' @export +setGeneric("takeOrdered", function(x, num) { standardGeneric("takeOrdered") }) + +#' @rdname takeSample +#' @export +setGeneric("takeSample", + function(x, withReplacement, num, seed) { + standardGeneric("takeSample") + }) + +#' @rdname top +#' @export +setGeneric("top", function(x, num) { standardGeneric("top") }) + +#' @rdname unionRDD +#' @export +setGeneric("unionRDD", function(x, y) { standardGeneric("unionRDD") }) + +#' @rdname unpersist-methods +#' @export +setGeneric("unpersist", function(x, ...) { standardGeneric("unpersist") }) + +#' @rdname zipRDD +#' @export +setGeneric("zipRDD", function(x, other) { standardGeneric("zipRDD") }) + +#' @rdname zipWithIndex +#' @seealso zipWithUniqueId +#' @export +setGeneric("zipWithIndex", function(x) { standardGeneric("zipWithIndex") }) + +#' @rdname zipWithUniqueId +#' @seealso zipWithIndex +#' @export +setGeneric("zipWithUniqueId", function(x) { standardGeneric("zipWithUniqueId") }) + + +############ Binary Functions ############# + +#' @rdname countByKey +#' @export +setGeneric("countByKey", function(x) { standardGeneric("countByKey") }) + +#' @rdname flatMapValues +#' @export +setGeneric("flatMapValues", function(X, FUN) { standardGeneric("flatMapValues") }) + +#' @rdname keys +#' @export +setGeneric("keys", function(x) { standardGeneric("keys") }) + +#' @rdname lookup +#' @export +setGeneric("lookup", function(x, key) { standardGeneric("lookup") }) + +#' @rdname mapValues +#' @export +setGeneric("mapValues", function(X, FUN) { standardGeneric("mapValues") }) + +#' @rdname values +#' @export +setGeneric("values", function(x) { standardGeneric("values") }) + + + +############ Shuffle Functions ############ + +#' @rdname aggregateByKey +#' @seealso foldByKey, combineByKey +#' @export +setGeneric("aggregateByKey", + function(x, zeroValue, seqOp, combOp, numPartitions) { + standardGeneric("aggregateByKey") + }) + +#' @rdname cogroup +#' @export +setGeneric("cogroup", + function(..., numPartitions) { + standardGeneric("cogroup") + }, + signature = "...") + +#' @rdname combineByKey +#' @seealso groupByKey, reduceByKey +#' @export +setGeneric("combineByKey", + function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { + standardGeneric("combineByKey") + }) + +#' @rdname foldByKey +#' @seealso aggregateByKey, combineByKey +#' @export +setGeneric("foldByKey", + function(x, zeroValue, func, numPartitions) { + standardGeneric("foldByKey") + }) + +#' @rdname join-methods +#' @export +setGeneric("fullOuterJoin", function(x, y, numPartitions) { standardGeneric("fullOuterJoin") }) + +#' @rdname groupByKey +#' @seealso reduceByKey +#' @export +setGeneric("groupByKey", function(x, numPartitions) { standardGeneric("groupByKey") }) + +#' @rdname join-methods +#' @export +setGeneric("join", function(x, y, ...) { standardGeneric("join") }) + +#' @rdname join-methods +#' @export +setGeneric("leftOuterJoin", function(x, y, numPartitions) { standardGeneric("leftOuterJoin") }) + +#' @rdname partitionBy +#' @export +setGeneric("partitionBy", function(x, numPartitions, ...) { standardGeneric("partitionBy") }) + +#' @rdname reduceByKey +#' @seealso groupByKey +#' @export +setGeneric("reduceByKey", function(x, combineFunc, numPartitions) { standardGeneric("reduceByKey")}) + +#' @rdname reduceByKeyLocally +#' @seealso reduceByKey +#' @export +setGeneric("reduceByKeyLocally", + function(x, combineFunc) { + standardGeneric("reduceByKeyLocally") + }) + +#' @rdname join-methods +#' @export +setGeneric("rightOuterJoin", function(x, y, numPartitions) { standardGeneric("rightOuterJoin") }) + +#' @rdname sortByKey +#' @export +setGeneric("sortByKey", function(x, ascending = TRUE, numPartitions = 1L) { + standardGeneric("sortByKey") +}) + + +################### Broadcast Variable Methods ################# + +#' @rdname broadcast +#' @export +setGeneric("value", function(bcast) { standardGeneric("value") }) + + + +#################### DataFrame Methods ######################## + +#' @rdname schema +#' @export +setGeneric("columns", function(x) {standardGeneric("columns") }) + +#' @rdname schema +#' @export +setGeneric("dtypes", function(x) { standardGeneric("dtypes") }) + +#' @rdname explain +#' @export +setGeneric("explain", function(x, ...) { standardGeneric("explain") }) + +#' @rdname filter +#' @export +setGeneric("filter", function(x, condition) { standardGeneric("filter") }) + +#' @rdname DataFrame +#' @export +setGeneric("groupBy", function(x, ...) { standardGeneric("groupBy") }) + +#' @rdname insertInto +#' @export +setGeneric("insertInto", function(x, tableName, ...) { standardGeneric("insertInto") }) + +#' @rdname intersect +#' @export +setGeneric("intersect", function(x, y) { standardGeneric("intersect") }) + +#' @rdname isLocal +#' @export +setGeneric("isLocal", function(x) { standardGeneric("isLocal") }) + +#' @rdname limit +#' @export +setGeneric("limit", function(x, num) {standardGeneric("limit") }) + +#' @rdname sortDF +#' @export +setGeneric("orderBy", function(x, col) { standardGeneric("orderBy") }) + +#' @rdname schema +#' @export +setGeneric("printSchema", function(x) { standardGeneric("printSchema") }) + +#' @rdname registerTempTable +#' @export +setGeneric("registerTempTable", function(x, tableName) { standardGeneric("registerTempTable") }) + +#' @rdname sampleDF +#' @export +setGeneric("sampleDF", + function(x, withReplacement, fraction, seed) { + standardGeneric("sampleDF") + }) + +#' @rdname saveAsParquetFile +#' @export +setGeneric("saveAsParquetFile", function(x, path) { standardGeneric("saveAsParquetFile") }) + +#' @rdname saveAsTable +#' @export +setGeneric("saveAsTable", function(df, tableName, source, mode, ...) { + standardGeneric("saveAsTable") +}) + +#' @rdname saveAsTable +#' @export +setGeneric("saveDF", function(df, path, source, mode, ...) { standardGeneric("saveDF") }) + +#' @rdname schema +#' @export +setGeneric("schema", function(x) { standardGeneric("schema") }) + +#' @rdname select +#' @export +setGeneric("select", function(x, col, ...) { standardGeneric("select") } ) + +#' @rdname select +#' @export +setGeneric("selectExpr", function(x, expr, ...) { standardGeneric("selectExpr") }) + +#' @rdname showDF +#' @export +setGeneric("showDF", function(x,...) { standardGeneric("showDF") }) + +#' @rdname sortDF +#' @export +setGeneric("sortDF", function(x, col, ...) { standardGeneric("sortDF") }) + +#' @rdname subtract +#' @export +setGeneric("subtract", function(x, y) { standardGeneric("subtract") }) + +#' @rdname tojson +#' @export +setGeneric("toJSON", function(x) { standardGeneric("toJSON") }) + +#' @rdname DataFrame +#' @export +setGeneric("toRDD", function(x) { standardGeneric("toRDD") }) + +#' @rdname unionAll +#' @export +setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") }) + +#' @rdname filter +#' @export +setGeneric("where", function(x, condition) { standardGeneric("where") }) + +#' @rdname withColumn +#' @export +setGeneric("withColumn", function(x, colName, col) { standardGeneric("withColumn") }) + +#' @rdname withColumnRenamed +#' @export +setGeneric("withColumnRenamed", function(x, existingCol, newCol) { + standardGeneric("withColumnRenamed") }) + + +###################### Column Methods ########################## + +#' @rdname column +#' @export +setGeneric("approxCountDistinct", function(x, ...) { standardGeneric("approxCountDistinct") }) + +#' @rdname column +#' @export +setGeneric("asc", function(x) { standardGeneric("asc") }) + +#' @rdname column +#' @export +setGeneric("avg", function(x, ...) { standardGeneric("avg") }) + +#' @rdname column +#' @export +setGeneric("cast", function(x, dataType) { standardGeneric("cast") }) + +#' @rdname column +#' @export +setGeneric("contains", function(x, ...) { standardGeneric("contains") }) +#' @rdname column +#' @export +setGeneric("countDistinct", function(x, ...) { standardGeneric("countDistinct") }) + +#' @rdname column +#' @export +setGeneric("desc", function(x) { standardGeneric("desc") }) + +#' @rdname column +#' @export +setGeneric("endsWith", function(x, ...) { standardGeneric("endsWith") }) + +#' @rdname column +#' @export +setGeneric("getField", function(x, ...) { standardGeneric("getField") }) + +#' @rdname column +#' @export +setGeneric("getItem", function(x, ...) { standardGeneric("getItem") }) + +#' @rdname column +#' @export +setGeneric("isNull", function(x) { standardGeneric("isNull") }) + +#' @rdname column +#' @export +setGeneric("isNotNull", function(x) { standardGeneric("isNotNull") }) + +#' @rdname column +#' @export +setGeneric("last", function(x) { standardGeneric("last") }) + +#' @rdname column +#' @export +setGeneric("like", function(x, ...) { standardGeneric("like") }) + +#' @rdname column +#' @export +setGeneric("lower", function(x) { standardGeneric("lower") }) + +#' @rdname column +#' @export +setGeneric("rlike", function(x, ...) { standardGeneric("rlike") }) + +#' @rdname column +#' @export +setGeneric("startsWith", function(x, ...) { standardGeneric("startsWith") }) + +#' @rdname column +#' @export +setGeneric("sumDistinct", function(x) { standardGeneric("sumDistinct") }) + +#' @rdname column +#' @export +setGeneric("upper", function(x) { standardGeneric("upper") }) + diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R new file mode 100644 index 0000000000000..09fc0a7abe48a --- /dev/null +++ b/R/pkg/R/group.R @@ -0,0 +1,132 @@ +# +# 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. +# + +# group.R - GroupedData class and methods implemented in S4 OO classes + +setOldClass("jobj") + +#' @title S4 class that represents a GroupedData +#' @description GroupedDatas can be created using groupBy() on a DataFrame +#' @rdname GroupedData +#' @seealso groupBy +#' +#' @param sgd A Java object reference to the backing Scala GroupedData +#' @export +setClass("GroupedData", + slots = list(sgd = "jobj")) + +setMethod("initialize", "GroupedData", function(.Object, sgd) { + .Object@sgd <- sgd + .Object +}) + +#' @rdname DataFrame +groupedData <- function(sgd) { + new("GroupedData", sgd) +} + + +#' @rdname show +setMethod("show", "GroupedData", + function(object) { + cat("GroupedData\n") + }) + +#' Count +#' +#' Count the number of rows for each group. +#' The resulting DataFrame will also contain the grouping columns. +#' +#' @param x a GroupedData +#' @return a DataFrame +#' @export +#' @examples +#' \dontrun{ +#' count(groupBy(df, "name")) +#' } +setMethod("count", + signature(x = "GroupedData"), + function(x) { + dataFrame(callJMethod(x@sgd, "count")) + }) + +#' Agg +#' +#' Aggregates on the entire DataFrame without groups. +#' The resulting DataFrame will also contain the grouping columns. +#' +#' df2 <- agg(df, = ) +#' df2 <- agg(df, newColName = aggFunction(column)) +#' +#' @param x a GroupedData +#' @return a DataFrame +#' @rdname agg +#' @examples +#' \dontrun{ +#' df2 <- agg(df, age = "sum") # new column name will be created as 'SUM(age#0)' +#' df2 <- agg(df, ageSum = sum(df$age)) # Creates a new column named ageSum +#' } +setGeneric("agg", function (x, ...) { standardGeneric("agg") }) + +setMethod("agg", + signature(x = "GroupedData"), + function(x, ...) { + cols = list(...) + stopifnot(length(cols) > 0) + if (is.character(cols[[1]])) { + cols <- varargsToEnv(...) + sdf <- callJMethod(x@sgd, "agg", cols) + } else if (class(cols[[1]]) == "Column") { + ns <- names(cols) + if (!is.null(ns)) { + for (n in ns) { + if (n != "") { + cols[[n]] = alias(cols[[n]], n) + } + } + } + jcols <- lapply(cols, function(c) { c@jc }) + # the GroupedData.agg(col, cols*) API does not contain grouping Column + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "aggWithGrouping", + x@sgd, listToSeq(jcols)) + } else { + stop("agg can only support Column or character") + } + dataFrame(sdf) + }) + + +# sum/mean/avg/min/max +methods <- c("sum", "mean", "avg", "min", "max") + +createMethod <- function(name) { + setMethod(name, + signature(x = "GroupedData"), + function(x, ...) { + sdf <- callJMethod(x@sgd, name, toSeq(...)) + dataFrame(sdf) + }) +} + +createMethods <- function() { + for (name in methods) { + createMethod(name) + } +} + +createMethods() + diff --git a/R/pkg/R/jobj.R b/R/pkg/R/jobj.R new file mode 100644 index 0000000000000..4180f146b7fbc --- /dev/null +++ b/R/pkg/R/jobj.R @@ -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. +# + +# References to objects that exist on the JVM backend +# are maintained using the jobj. + +# Maintain a reference count of Java object references +# This allows us to GC the java object when it is safe +.validJobjs <- new.env(parent = emptyenv()) + +# List of object ids to be removed +.toRemoveJobjs <- new.env(parent = emptyenv()) + +# Check if jobj was created with the current SparkContext +isValidJobj <- function(jobj) { + if (exists(".scStartTime", envir = .sparkREnv)) { + jobj$appId == get(".scStartTime", envir = .sparkREnv) + } else { + FALSE + } +} + +getJobj <- function(objId) { + newObj <- jobj(objId) + if (exists(objId, .validJobjs)) { + .validJobjs[[objId]] <- .validJobjs[[objId]] + 1 + } else { + .validJobjs[[objId]] <- 1 + } + newObj +} + +# Handler for a java object that exists on the backend. +jobj <- function(objId) { + if (!is.character(objId)) { + stop("object id must be a character") + } + # NOTE: We need a new env for a jobj as we can only register + # finalizers for environments or external references pointers. + obj <- structure(new.env(parent = emptyenv()), class = "jobj") + obj$id <- objId + obj$appId <- get(".scStartTime", envir = .sparkREnv) + + # Register a finalizer to remove the Java object when this reference + # is garbage collected in R + reg.finalizer(obj, cleanup.jobj) + obj +} + +#' Print a JVM object reference. +#' +#' This function prints the type and id for an object stored +#' in the SparkR JVM backend. +#' +#' @param x The JVM object reference +#' @param ... further arguments passed to or from other methods +print.jobj <- function(x, ...) { + cls <- callJMethod(x, "getClass") + name <- callJMethod(cls, "getName") + cat("Java ref type", name, "id", x$id, "\n", sep = " ") +} + +cleanup.jobj <- function(jobj) { + if (isValidJobj(jobj)) { + objId <- jobj$id + # If we don't know anything about this jobj, ignore it + if (exists(objId, envir = .validJobjs)) { + .validJobjs[[objId]] <- .validJobjs[[objId]] - 1 + + if (.validJobjs[[objId]] == 0) { + rm(list = objId, envir = .validJobjs) + # NOTE: We cannot call removeJObject here as the finalizer may be run + # in the middle of another RPC. Thus we queue up this object Id to be removed + # and then run all the removeJObject when the next RPC is called. + .toRemoveJobjs[[objId]] <- 1 + } + } + } +} + +clearJobjs <- function() { + valid <- ls(.validJobjs) + rm(list = valid, envir = .validJobjs) + + removeList <- ls(.toRemoveJobjs) + rm(list = removeList, envir = .toRemoveJobjs) +} diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R new file mode 100644 index 0000000000000..c2396c32a7548 --- /dev/null +++ b/R/pkg/R/pairRDD.R @@ -0,0 +1,789 @@ +# +# 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. +# + +# Operations supported on RDDs contains pairs (i.e key, value) + +############ Actions and Transformations ############ + +#' Look up elements of a key in an RDD +#' +#' @description +#' \code{lookup} returns a list of values in this RDD for key key. +#' +#' @param x The RDD to collect +#' @param key The key to look up for +#' @return a list of values in this RDD for key key +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(c(1, 1), c(2, 2), c(1, 3)) +#' rdd <- parallelize(sc, pairs) +#' lookup(rdd, 1) # list(1, 3) +#'} +#' @rdname lookup +#' @aliases lookup,RDD-method +setMethod("lookup", + signature(x = "RDD", key = "ANY"), + function(x, key) { + partitionFunc <- function(part) { + filtered <- part[unlist(lapply(part, function(i) { identical(key, i[[1]]) }))] + lapply(filtered, function(i) { i[[2]] }) + } + valsRDD <- lapplyPartition(x, partitionFunc) + collect(valsRDD) + }) + +#' Count the number of elements for each key, and return the result to the +#' master as lists of (key, count) pairs. +#' +#' Same as countByKey in Spark. +#' +#' @param x The RDD to count keys. +#' @return list of (key, count) pairs, where count is number of each key in rdd. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(c("a", 1), c("b", 1), c("a", 1))) +#' countByKey(rdd) # ("a", 2L), ("b", 1L) +#'} +#' @rdname countByKey +#' @aliases countByKey,RDD-method +setMethod("countByKey", + signature(x = "RDD"), + function(x) { + keys <- lapply(x, function(item) { item[[1]] }) + countByValue(keys) + }) + +#' Return an RDD with the keys of each tuple. +#' +#' @param x The RDD from which the keys of each tuple is returned. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(keys(rdd)) # list(1, 3) +#'} +#' @rdname keys +#' @aliases keys,RDD +setMethod("keys", + signature(x = "RDD"), + function(x) { + func <- function(k) { + k[[1]] + } + lapply(x, func) + }) + +#' Return an RDD with the values of each tuple. +#' +#' @param x The RDD from which the values of each tuple is returned. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) +#' collect(values(rdd)) # list(2, 4) +#'} +#' @rdname values +#' @aliases values,RDD +setMethod("values", + signature(x = "RDD"), + function(x) { + func <- function(v) { + v[[2]] + } + lapply(x, func) + }) + +#' Applies a function to all values of the elements, without modifying the keys. +#' +#' The same as `mapValues()' in Spark. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on the value of each element. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, 1:10) +#' makePairs <- lapply(rdd, function(x) { list(x, x) }) +#' collect(mapValues(makePairs, function(x) { x * 2) }) +#' Output: list(list(1,2), list(2,4), list(3,6), ...) +#'} +#' @rdname mapValues +#' @aliases mapValues,RDD,function-method +setMethod("mapValues", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + func <- function(x) { + list(x[[1]], FUN(x[[2]])) + } + lapply(X, func) + }) + +#' Pass each value in the key-value pair RDD through a flatMap function without +#' changing the keys; this also retains the original RDD's partitioning. +#' +#' The same as 'flatMapValues()' in Spark. +#' +#' @param X The RDD to apply the transformation. +#' @param FUN the transformation to apply on the value of each element. +#' @return a new RDD created by the transformation. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) +#' collect(flatMapValues(rdd, function(x) { x })) +#' Output: list(list(1,1), list(1,2), list(2,3), list(2,4)) +#'} +#' @rdname flatMapValues +#' @aliases flatMapValues,RDD,function-method +setMethod("flatMapValues", + signature(X = "RDD", FUN = "function"), + function(X, FUN) { + flatMapFunc <- function(x) { + lapply(FUN(x[[2]]), function(v) { list(x[[1]], v) }) + } + flatMap(X, flatMapFunc) + }) + +############ Shuffle Functions ############ + +#' Partition an RDD by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' For each element of this RDD, the partitioner is used to compute a hash +#' function and the RDD is partitioned using this hash value. +#' +#' @param x The RDD to partition. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param numPartitions Number of partitions to create. +#' @param ... Other optional arguments to partitionBy. +#' +#' @param partitionFunc The partition function to use. Uses a default hashCode +#' function if not provided +#' @return An RDD partitioned using the specified partitioner. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- partitionBy(rdd, 2L) +#' collectPartition(parts, 0L) # First partition should contain list(1, 2) and list(1, 4) +#'} +#' @rdname partitionBy +#' @aliases partitionBy,RDD,integer-method +setMethod("partitionBy", + signature(x = "RDD", numPartitions = "integer"), + function(x, numPartitions, partitionFunc = hashCode) { + + #if (missing(partitionFunc)) { + # partitionFunc <- hashCode + #} + + partitionFunc <- cleanClosure(partitionFunc) + serializedHashFuncBytes <- serialize(partitionFunc, connection = NULL) + + packageNamesArr <- serialize(.sparkREnv$.packages, + connection = NULL) + broadcastArr <- lapply(ls(.broadcastNames), function(name) { + get(name, .broadcastNames) }) + jrdd <- getJRDD(x) + + # We create a PairwiseRRDD that extends RDD[(Array[Byte], + # Array[Byte])], where the key is the hashed split, the value is + # the content (key-val pairs). + pairwiseRRDD <- newJObject("org.apache.spark.api.r.PairwiseRRDD", + callJMethod(jrdd, "rdd"), + as.integer(numPartitions), + serializedHashFuncBytes, + getSerializedMode(x), + packageNamesArr, + as.character(.sparkREnv$libname), + broadcastArr, + callJMethod(jrdd, "classTag")) + + # Create a corresponding partitioner. + rPartitioner <- newJObject("org.apache.spark.HashPartitioner", + as.integer(numPartitions)) + + # Call partitionBy on the obtained PairwiseRDD. + javaPairRDD <- callJMethod(pairwiseRRDD, "asJavaPairRDD") + javaPairRDD <- callJMethod(javaPairRDD, "partitionBy", rPartitioner) + + # Call .values() on the result to get back the final result, the + # shuffled acutal content key-val pairs. + r <- callJMethod(javaPairRDD, "values") + + RDD(r, serializedMode = "byte") + }) + +#' Group values by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and group values for each key in the RDD into a single sequence. +#' +#' @param x The RDD to group. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, list(V)) +#' @seealso reduceByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- groupByKey(rdd, 2L) +#' grouped <- collect(parts) +#' grouped[[1]] # Should be a list(1, list(2, 4)) +#'} +#' @rdname groupByKey +#' @aliases groupByKey,RDD,integer-method +setMethod("groupByKey", + signature(x = "RDD", numPartitions = "integer"), + function(x, numPartitions) { + shuffled <- partitionBy(x, numPartitions) + groupVals <- function(part) { + vals <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + appendList <- function(acc, i) { + addItemToAccumulator(acc, i) + acc + } + makeList <- function(i) { + acc <- initAccumulator() + addItemToAccumulator(acc, i) + acc + } + # Each item in the partition is list of (K, V) + lapply(part, + function(item) { + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, vals, pred, + appendList, makeList) + }) + # extract out data field + vals <- eapply(vals, + function(i) { + length(i$data) <- i$counter + i$data + }) + # Every key in the environment contains a list + # Convert that to list(K, Seq[V]) + convertEnvsToList(keys, vals) + } + lapplyPartition(shuffled, groupVals) + }) + +#' Merge values by key +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and merges the values for each key using an associative reduce function. +#' +#' @param x The RDD to reduce by key. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param combineFunc The associative reduce function to use. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, V') where V' is the merged +#' value +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- reduceByKey(rdd, "+", 2L) +#' reduced <- collect(parts) +#' reduced[[1]] # Should be a list(1, 6) +#'} +#' @rdname reduceByKey +#' @aliases reduceByKey,RDD,integer-method +setMethod("reduceByKey", + signature(x = "RDD", combineFunc = "ANY", numPartitions = "integer"), + function(x, combineFunc, numPartitions) { + reduceVals <- function(part) { + vals <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, vals, pred, combineFunc, identity) + }) + convertEnvsToList(keys, vals) + } + locallyReduced <- lapplyPartition(x, reduceVals) + shuffled <- partitionBy(locallyReduced, numPartitions) + lapplyPartition(shuffled, reduceVals) + }) + +#' Merge values by key locally +#' +#' This function operates on RDDs where every element is of the form list(K, V) or c(K, V). +#' and merges the values for each key using an associative reduce function, but return the +#' results immediately to the driver as an R list. +#' +#' @param x The RDD to reduce by key. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param combineFunc The associative reduce function to use. +#' @return A list of elements of type list(K, V') where V' is the merged value for each key +#' @seealso reduceByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' reduced <- reduceByKeyLocally(rdd, "+") +#' reduced # list(list(1, 6), list(1.1, 3)) +#'} +#' @rdname reduceByKeyLocally +#' @aliases reduceByKeyLocally,RDD,integer-method +setMethod("reduceByKeyLocally", + signature(x = "RDD", combineFunc = "ANY"), + function(x, combineFunc) { + reducePart <- function(part) { + vals <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(hashCode(item[[1]])) + updateOrCreatePair(item, keys, vals, pred, combineFunc, identity) + }) + list(list(keys, vals)) # return hash to avoid re-compute in merge + } + mergeParts <- function(accum, x) { + pred <- function(item) { + exists(item$hash, accum[[1]]) + } + lapply(ls(x[[1]]), + function(name) { + item <- list(x[[1]][[name]], x[[2]][[name]]) + item$hash <- name + updateOrCreatePair(item, accum[[1]], accum[[2]], pred, combineFunc, identity) + }) + accum + } + reduced <- mapPartitions(x, reducePart) + merged <- reduce(reduced, mergeParts) + convertEnvsToList(merged[[1]], merged[[2]]) + }) + +#' Combine values by key +#' +#' Generic function to combine the elements for each key using a custom set of +#' aggregation functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], +#' for a "combined type" C. Note that V and C can be different -- for example, one +#' might group an RDD of type (Int, Int) into an RDD of type (Int, Seq[Int]). + +#' Users provide three functions: +#' \itemize{ +#' \item createCombiner, which turns a V into a C (e.g., creates a one-element list) +#' \item mergeValue, to merge a V into a C (e.g., adds it to the end of a list) - +#' \item mergeCombiners, to combine two C's into a single one (e.g., concatentates +#' two lists). +#' } +#' +#' @param x The RDD to combine. Should be an RDD where each element is +#' list(K, V) or c(K, V). +#' @param createCombiner Create a combiner (C) given a value (V) +#' @param mergeValue Merge the given value (V) with an existing combiner (C) +#' @param mergeCombiners Merge two combiners and return a new combiner +#' @param numPartitions Number of partitions to create. +#' @return An RDD where each element is list(K, C) where C is the combined type +#' +#' @seealso groupByKey, reduceByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' pairs <- list(list(1, 2), list(1.1, 3), list(1, 4)) +#' rdd <- parallelize(sc, pairs) +#' parts <- combineByKey(rdd, function(x) { x }, "+", "+", 2L) +#' combined <- collect(parts) +#' combined[[1]] # Should be a list(1, 6) +#'} +#' @rdname combineByKey +#' @aliases combineByKey,RDD,ANY,ANY,ANY,integer-method +setMethod("combineByKey", + signature(x = "RDD", createCombiner = "ANY", mergeValue = "ANY", + mergeCombiners = "ANY", numPartitions = "integer"), + function(x, createCombiner, mergeValue, mergeCombiners, numPartitions) { + combineLocally <- function(part) { + combiners <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(item[[1]]) + updateOrCreatePair(item, keys, combiners, pred, mergeValue, createCombiner) + }) + convertEnvsToList(keys, combiners) + } + locallyCombined <- lapplyPartition(x, combineLocally) + shuffled <- partitionBy(locallyCombined, numPartitions) + mergeAfterShuffle <- function(part) { + combiners <- new.env() + keys <- new.env() + pred <- function(item) exists(item$hash, keys) + lapply(part, + function(item) { + item$hash <- as.character(item[[1]]) + updateOrCreatePair(item, keys, combiners, pred, mergeCombiners, identity) + }) + convertEnvsToList(keys, combiners) + } + lapplyPartition(shuffled, mergeAfterShuffle) + }) + +#' Aggregate a pair RDD by each key. +#' +#' Aggregate the values of each key in an RDD, using given combine functions +#' and a neutral "zero value". This function can return a different result type, +#' U, than the type of the values in this RDD, V. Thus, we need one operation +#' for merging a V into a U and one operation for merging two U's, The former +#' operation is used for merging values within a partition, and the latter is +#' used for merging values between partitions. To avoid memory allocation, both +#' of these functions are allowed to modify and return their first argument +#' instead of creating a new U. +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param seqOp A function to aggregate the values of each key. It may return +#' a different result type from the type of the values. +#' @param combOp A function to aggregate results of seqOp. +#' @return An RDD containing the aggregation result. +#' @seealso foldByKey, combineByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +#' zeroValue <- list(0, 0) +#' seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } +#' combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } +#' aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) +#' # list(list(1, list(3, 2)), list(2, list(7, 2))) +#'} +#' @rdname aggregateByKey +#' @aliases aggregateByKey,RDD,ANY,ANY,ANY,integer-method +setMethod("aggregateByKey", + signature(x = "RDD", zeroValue = "ANY", seqOp = "ANY", + combOp = "ANY", numPartitions = "integer"), + function(x, zeroValue, seqOp, combOp, numPartitions) { + createCombiner <- function(v) { + do.call(seqOp, list(zeroValue, v)) + } + + combineByKey(x, createCombiner, seqOp, combOp, numPartitions) + }) + +#' Fold a pair RDD by each key. +#' +#' Aggregate the values of each key in an RDD, using an associative function "func" +#' and a neutral "zero value" which may be added to the result an arbitrary +#' number of times, and must not change the result (e.g., 0 for addition, or +#' 1 for multiplication.). +#' +#' @param x An RDD. +#' @param zeroValue A neutral "zero value". +#' @param func An associative function for folding values of each key. +#' @return An RDD containing the aggregation result. +#' @seealso aggregateByKey, combineByKey +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) +#' foldByKey(rdd, 0, "+", 2L) # list(list(1, 3), list(2, 7)) +#'} +#' @rdname foldByKey +#' @aliases foldByKey,RDD,ANY,ANY,integer-method +setMethod("foldByKey", + signature(x = "RDD", zeroValue = "ANY", + func = "ANY", numPartitions = "integer"), + function(x, zeroValue, func, numPartitions) { + aggregateByKey(x, zeroValue, func, func, numPartitions) + }) + +############ Binary Functions ############# + +#' Join two RDDs +#' +#' @description +#' \code{join} This function joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return a new RDD containing all pairs of elements with matching keys in +#' two input RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' join(rdd1, rdd2, 2L) # list(list(1, list(1, 2)), list(1, list(1, 3)) +#'} +#' @rdname join-methods +#' @aliases join,RDD,RDD-method +setMethod("join", + signature(x = "RDD", y = "RDD"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(FALSE, FALSE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numToInt(numPartitions)), + doJoin) + }) + +#' Left outer join two RDDs +#' +#' @description +#' \code{leftouterjoin} This function left-outer-joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, v) in x, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, w) in rdd2, or the pair (k, (v, NULL)) +#' if no elements in rdd2 have key k. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' leftOuterJoin(rdd1, rdd2, 2L) +#' # list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) +#'} +#' @rdname join-methods +#' @aliases leftOuterJoin,RDD,RDD-method +setMethod("leftOuterJoin", + signature(x = "RDD", y = "RDD", numPartitions = "integer"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(FALSE, TRUE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) + }) + +#' Right outer join two RDDs +#' +#' @description +#' \code{rightouterjoin} This function right-outer-joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, w) in y, the resulting RDD will either contain +#' all pairs (k, (v, w)) for (k, v) in x, or the pair (k, (NULL, w)) +#' if no elements in x have key k. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rightOuterJoin(rdd1, rdd2, 2L) +#' # list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) +#'} +#' @rdname join-methods +#' @aliases rightOuterJoin,RDD,RDD-method +setMethod("rightOuterJoin", + signature(x = "RDD", y = "RDD", numPartitions = "integer"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(TRUE, FALSE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) + }) + +#' Full outer join two RDDs +#' +#' @description +#' \code{fullouterjoin} This function full-outer-joins two RDDs where every element is of the form list(K, V). +#' The key types of the two RDDs should be the same. +#' +#' @param x An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param y An RDD to be joined. Should be an RDD where each element is +#' list(K, V). +#' @param numPartitions Number of partitions to create. +#' @return For each element (k, v) in x and (k, w) in y, the resulting RDD +#' will contain all pairs (k, (v, w)) for both (k, v) in x and +#' (k, w) in y, or the pair (k, (NULL, w))/(k, (v, NULL)) if no elements +#' in x/y have key k. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 2), list(1, 3), list(3, 3))) +#' rdd2 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' fullOuterJoin(rdd1, rdd2, 2L) # list(list(1, list(2, 1)), +#' # list(1, list(3, 1)), +#' # list(2, list(NULL, 4))) +#' # list(3, list(3, NULL)), +#'} +#' @rdname join-methods +#' @aliases fullOuterJoin,RDD,RDD-method +setMethod("fullOuterJoin", + signature(x = "RDD", y = "RDD", numPartitions = "integer"), + function(x, y, numPartitions) { + xTagged <- lapply(x, function(i) { list(i[[1]], list(1L, i[[2]])) }) + yTagged <- lapply(y, function(i) { list(i[[1]], list(2L, i[[2]])) }) + + doJoin <- function(v) { + joinTaggedList(v, list(TRUE, TRUE)) + } + + joined <- flatMapValues(groupByKey(unionRDD(xTagged, yTagged), numPartitions), doJoin) + }) + +#' For each key k in several RDDs, return a resulting RDD that +#' whose values are a list of values for the key in all RDDs. +#' +#' @param ... Several RDDs. +#' @param numPartitions Number of partitions to create. +#' @return a new RDD containing all pairs of elements with values in a list +#' in all RDDs. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) +#' rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) +#' cogroup(rdd1, rdd2, numPartitions = 2L) +#' # list(list(1, list(1, list(2, 3))), list(2, list(list(4), list())) +#'} +#' @rdname cogroup +#' @aliases cogroup,RDD-method +setMethod("cogroup", + "RDD", + function(..., numPartitions) { + rdds <- list(...) + rddsLen <- length(rdds) + for (i in 1:rddsLen) { + rdds[[i]] <- lapply(rdds[[i]], + function(x) { list(x[[1]], list(i, x[[2]])) }) + # TODO(hao): As issue [SparkR-142] mentions, the right value of i + # will not be captured into UDF if getJRDD is not invoked. + # It should be resolved together with that issue. + getJRDD(rdds[[i]]) # Capture the closure. + } + union.rdd <- Reduce(unionRDD, rdds) + group.func <- function(vlist) { + res <- list() + length(res) <- rddsLen + for (x in vlist) { + i <- x[[1]] + acc <- res[[i]] + # Create an accumulator. + if (is.null(acc)) { + acc <- initAccumulator() + } + addItemToAccumulator(acc, x[[2]]) + res[[i]] <- acc + } + lapply(res, function(acc) { + if (is.null(acc)) { + list() + } else { + acc$data + } + }) + } + cogroup.rdd <- mapValues(groupByKey(union.rdd, numPartitions), + group.func) + }) + +#' Sort a (k, v) pair RDD by k. +#' +#' @param x A (k, v) pair RDD to be sorted. +#' @param ascending A flag to indicate whether the sorting is ascending or descending. +#' @param numPartitions Number of partitions to create. +#' @return An RDD where all (k, v) pair elements are sorted. +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' rdd <- parallelize(sc, list(list(3, 1), list(2, 2), list(1, 3))) +#' collect(sortByKey(rdd)) # list (list(1, 3), list(2, 2), list(3, 1)) +#'} +#' @rdname sortByKey +#' @aliases sortByKey,RDD,RDD-method +setMethod("sortByKey", + signature(x = "RDD"), + function(x, ascending = TRUE, numPartitions = SparkR::numPartitions(x)) { + rangeBounds <- list() + + if (numPartitions > 1) { + rddSize <- count(x) + # constant from Spark's RangePartitioner + maxSampleSize <- numPartitions * 20 + fraction <- min(maxSampleSize / max(rddSize, 1), 1.0) + + samples <- collect(keys(sampleRDD(x, FALSE, fraction, 1L))) + + # Note: the built-in R sort() function only works on atomic vectors + samples <- sort(unlist(samples, recursive = FALSE), decreasing = !ascending) + + if (length(samples) > 0) { + rangeBounds <- lapply(seq_len(numPartitions - 1), + function(i) { + j <- ceiling(length(samples) * i / numPartitions) + samples[j] + }) + } + } + + rangePartitionFunc <- function(key) { + partition <- 0 + + # TODO: Use binary search instead of linear search, similar with Spark + while (partition < length(rangeBounds) && key > rangeBounds[[partition + 1]]) { + partition <- partition + 1 + } + + if (ascending) { + partition + } else { + numPartitions - partition - 1 + } + } + + partitionFunc <- function(part) { + sortKeyValueList(part, decreasing = !ascending) + } + + newRDD <- partitionBy(x, numPartitions, rangePartitionFunc) + lapplyPartition(newRDD, partitionFunc) + }) + diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R new file mode 100644 index 0000000000000..8a9c0c652ce24 --- /dev/null +++ b/R/pkg/R/serialize.R @@ -0,0 +1,195 @@ +# +# 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. +# + +# Utility functions to serialize R objects so they can be read in Java. + +# Type mapping from R to Java +# +# NULL -> Void +# integer -> Int +# character -> String +# logical -> Boolean +# double, numeric -> Double +# raw -> Array[Byte] +# Date -> Date +# POSIXct,POSIXlt -> Time +# +# list[T] -> Array[T], where T is one of above mentioned types +# environment -> Map[String, T], where T is a native type +# jobj -> Object, where jobj is an object created in the backend + +writeObject <- function(con, object, writeType = TRUE) { + # NOTE: In R vectors have same type as objects. So we don't support + # passing in vectors as arrays and instead require arrays to be passed + # as lists. + type <- class(object)[[1]] # class of POSIXlt is c("POSIXlt", "POSIXt") + if (writeType) { + writeType(con, type) + } + switch(type, + NULL = writeVoid(con), + integer = writeInt(con, object), + character = writeString(con, object), + logical = writeBoolean(con, object), + double = writeDouble(con, object), + numeric = writeDouble(con, object), + raw = writeRaw(con, object), + list = writeList(con, object), + jobj = writeJobj(con, object), + environment = writeEnv(con, object), + Date = writeDate(con, object), + POSIXlt = writeTime(con, object), + POSIXct = writeTime(con, object), + stop(paste("Unsupported type for serialization", type))) +} + +writeVoid <- function(con) { + # no value for NULL +} + +writeJobj <- function(con, value) { + if (!isValidJobj(value)) { + stop("invalid jobj ", value$id) + } + writeString(con, value$id) +} + +writeString <- function(con, value) { + writeInt(con, as.integer(nchar(value) + 1)) + writeBin(value, con, endian = "big") +} + +writeInt <- function(con, value) { + writeBin(as.integer(value), con, endian = "big") +} + +writeDouble <- function(con, value) { + writeBin(value, con, endian = "big") +} + +writeBoolean <- function(con, value) { + # TRUE becomes 1, FALSE becomes 0 + writeInt(con, as.integer(value)) +} + +writeRawSerialize <- function(outputCon, batch) { + outputSer <- serialize(batch, ascii = FALSE, connection = NULL) + writeRaw(outputCon, outputSer) +} + +writeRowSerialize <- function(outputCon, rows) { + invisible(lapply(rows, function(r) { + bytes <- serializeRow(r) + writeRaw(outputCon, bytes) + })) +} + +serializeRow <- function(row) { + rawObj <- rawConnection(raw(0), "wb") + on.exit(close(rawObj)) + writeRow(rawObj, row) + rawConnectionValue(rawObj) +} + +writeRow <- function(con, row) { + numCols <- length(row) + writeInt(con, numCols) + for (i in 1:numCols) { + writeObject(con, row[[i]]) + } +} + +writeRaw <- function(con, batch) { + writeInt(con, length(batch)) + writeBin(batch, con, endian = "big") +} + +writeType <- function(con, class) { + type <- switch(class, + NULL = "n", + integer = "i", + character = "c", + logical = "b", + double = "d", + numeric = "d", + raw = "r", + list = "l", + jobj = "j", + environment = "e", + Date = "D", + POSIXlt = 't', + POSIXct = 't', + stop(paste("Unsupported type for serialization", class))) + writeBin(charToRaw(type), con) +} + +# Used to pass arrays where all the elements are of the same type +writeList <- function(con, arr) { + # All elements should be of same type + elemType <- unique(sapply(arr, function(elem) { class(elem) })) + stopifnot(length(elemType) <= 1) + + # TODO: Empty lists are given type "character" right now. + # This may not work if the Java side expects array of any other type. + if (length(elemType) == 0) { + elemType <- class("somestring") + } + + writeType(con, elemType) + writeInt(con, length(arr)) + + if (length(arr) > 0) { + for (a in arr) { + writeObject(con, a, FALSE) + } + } +} + +# Used to pass in hash maps required on Java side. +writeEnv <- function(con, env) { + len <- length(env) + + writeInt(con, len) + if (len > 0) { + writeList(con, as.list(ls(env))) + vals <- lapply(ls(env), function(x) { env[[x]] }) + writeList(con, as.list(vals)) + } +} + +writeDate <- function(con, date) { + writeString(con, as.character(date)) +} + +writeTime <- function(con, time) { + writeDouble(con, as.double(time)) +} + +# Used to serialize in a list of objects where each +# object can be of a different type. Serialization format is +# for each object +writeArgs <- function(con, args) { + if (length(args) > 0) { + for (a in args) { + writeObject(con, a) + } + } +} + +writeStrings <- function(con, stringList) { + writeLines(unlist(stringList), con) +} diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R new file mode 100644 index 0000000000000..bc82df01f0fff --- /dev/null +++ b/R/pkg/R/sparkR.R @@ -0,0 +1,266 @@ +# +# 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. +# + +.sparkREnv <- new.env() + +sparkR.onLoad <- function(libname, pkgname) { + .sparkREnv$libname <- libname +} + +# Utility function that returns TRUE if we have an active connection to the +# backend and FALSE otherwise +connExists <- function(env) { + tryCatch({ + exists(".sparkRCon", envir = env) && isOpen(env[[".sparkRCon"]]) + }, error = function(err) { + return(FALSE) + }) +} + +#' Stop the Spark context. +#' +#' Also terminates the backend this R session is connected to +sparkR.stop <- function() { + env <- .sparkREnv + if (exists(".sparkRCon", envir = env)) { + # cat("Stopping SparkR\n") + if (exists(".sparkRjsc", envir = env)) { + sc <- get(".sparkRjsc", envir = env) + callJMethod(sc, "stop") + rm(".sparkRjsc", envir = env) + } + + if (exists(".backendLaunched", envir = env)) { + callJStatic("SparkRHandler", "stopBackend") + } + + # Also close the connection and remove it from our env + conn <- get(".sparkRCon", envir = env) + close(conn) + + rm(".sparkRCon", envir = env) + rm(".scStartTime", envir = env) + } + + if (exists(".monitorConn", envir = env)) { + conn <- get(".monitorConn", envir = env) + close(conn) + rm(".monitorConn", envir = env) + } + + # Clear all broadcast variables we have + # as the jobj will not be valid if we restart the JVM + clearBroadcastVariables() + + # Clear jobj maps + clearJobjs() +} + +#' Initialize a new Spark Context. +#' +#' This function initializes a new SparkContext. +#' +#' @param master The Spark master URL. +#' @param appName Application name to register with cluster manager +#' @param sparkHome Spark Home directory +#' @param sparkEnvir Named list of environment variables to set on worker nodes. +#' @param sparkExecutorEnv Named list of environment variables to be used when launching executors. +#' @param sparkJars Character string vector of jar files to pass to the worker nodes. +#' @param sparkRLibDir The path where R is installed on the worker nodes. +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark") +#' sc <- sparkR.init("local[2]", "SparkR", "/home/spark", +#' list(spark.executor.memory="1g")) +#' sc <- sparkR.init("yarn-client", "SparkR", "/home/spark", +#' list(spark.executor.memory="1g"), +#' list(LD_LIBRARY_PATH="/directory of JVM libraries (libjvm.so) on workers/"), +#' c("jarfile1.jar","jarfile2.jar")) +#'} + +sparkR.init <- function( + master = "", + appName = "SparkR", + sparkHome = Sys.getenv("SPARK_HOME"), + sparkEnvir = list(), + sparkExecutorEnv = list(), + sparkJars = "", + sparkRLibDir = "") { + + if (exists(".sparkRjsc", envir = .sparkREnv)) { + cat("Re-using existing Spark Context. Please stop SparkR with sparkR.stop() or restart R to create a new Spark Context\n") + return(get(".sparkRjsc", envir = .sparkREnv)) + } + + sparkMem <- Sys.getenv("SPARK_MEM", "512m") + jars <- suppressWarnings(normalizePath(as.character(sparkJars))) + + # Classpath separator is ";" on Windows + # URI needs four /// as from http://stackoverflow.com/a/18522792 + if (.Platform$OS.type == "unix") { + collapseChar <- ":" + uriSep <- "//" + } else { + collapseChar <- ";" + uriSep <- "////" + } + + existingPort <- Sys.getenv("EXISTING_SPARKR_BACKEND_PORT", "") + if (existingPort != "") { + backendPort <- existingPort + } else { + path <- tempfile(pattern = "backend_port") + launchBackend( + args = path, + sparkHome = sparkHome, + jars = jars, + sparkSubmitOpts = Sys.getenv("SPARKR_SUBMIT_ARGS", "sparkr-shell")) + # wait atmost 100 seconds for JVM to launch + wait <- 0.1 + for (i in 1:25) { + Sys.sleep(wait) + if (file.exists(path)) { + break + } + wait <- wait * 1.25 + } + if (!file.exists(path)) { + stop("JVM is not ready after 10 seconds") + } + f <- file(path, open='rb') + backendPort <- readInt(f) + monitorPort <- readInt(f) + close(f) + file.remove(path) + if (length(backendPort) == 0 || backendPort == 0 || + length(monitorPort) == 0 || monitorPort == 0) { + stop("JVM failed to launch") + } + assign(".monitorConn", socketConnection(port = monitorPort), envir = .sparkREnv) + assign(".backendLaunched", 1, envir = .sparkREnv) + } + + .sparkREnv$backendPort <- backendPort + tryCatch({ + connectBackend("localhost", backendPort) + }, error = function(err) { + stop("Failed to connect JVM\n") + }) + + if (nchar(sparkHome) != 0) { + sparkHome <- normalizePath(sparkHome) + } + + if (nchar(sparkRLibDir) != 0) { + .sparkREnv$libname <- sparkRLibDir + } + + sparkEnvirMap <- new.env() + for (varname in names(sparkEnvir)) { + sparkEnvirMap[[varname]] <- sparkEnvir[[varname]] + } + + sparkExecutorEnvMap <- new.env() + if (!any(names(sparkExecutorEnv) == "LD_LIBRARY_PATH")) { + sparkExecutorEnvMap[["LD_LIBRARY_PATH"]] <- paste0("$LD_LIBRARY_PATH:",Sys.getenv("LD_LIBRARY_PATH")) + } + for (varname in names(sparkExecutorEnv)) { + sparkExecutorEnvMap[[varname]] <- sparkExecutorEnv[[varname]] + } + + nonEmptyJars <- Filter(function(x) { x != "" }, jars) + localJarPaths <- sapply(nonEmptyJars, function(j) { utils::URLencode(paste("file:", uriSep, j, sep = "")) }) + + # Set the start time to identify jobjs + # Seconds resolution is good enough for this purpose, so use ints + assign(".scStartTime", as.integer(Sys.time()), envir = .sparkREnv) + + assign( + ".sparkRjsc", + callJStatic( + "org.apache.spark.api.r.RRDD", + "createSparkContext", + master, + appName, + as.character(sparkHome), + as.list(localJarPaths), + sparkEnvirMap, + sparkExecutorEnvMap), + envir = .sparkREnv + ) + + sc <- get(".sparkRjsc", envir = .sparkREnv) + + # Register a finalizer to sleep 1 seconds on R exit to make RStudio happy + reg.finalizer(.sparkREnv, function(x) { Sys.sleep(1) }, onexit = TRUE) + + sc +} + +#' Initialize a new SQLContext. +#' +#' This function creates a SparkContext from an existing JavaSparkContext and +#' then uses it to initialize a new SQLContext +#' +#' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRSQL.init(sc) +#'} + +sparkRSQL.init <- function(jsc) { + if (exists(".sparkRSQLsc", envir = .sparkREnv)) { + return(get(".sparkRSQLsc", envir = .sparkREnv)) + } + + sqlCtx <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createSQLContext", + jsc) + assign(".sparkRSQLsc", sqlCtx, envir = .sparkREnv) + sqlCtx +} + +#' Initialize a new HiveContext. +#' +#' This function creates a HiveContext from an existing JavaSparkContext +#' +#' @param jsc The existing JavaSparkContext created with SparkR.init() +#' @export +#' @examples +#'\dontrun{ +#' sc <- sparkR.init() +#' sqlCtx <- sparkRHive.init(sc) +#'} + +sparkRHive.init <- function(jsc) { + if (exists(".sparkRHivesc", envir = .sparkREnv)) { + return(get(".sparkRHivesc", envir = .sparkREnv)) + } + + ssc <- callJMethod(jsc, "sc") + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.HiveContext", ssc) + }, error = function(err) { + stop("Spark SQL is not built with Hive support") + }) + + assign(".sparkRHivesc", hiveCtx, envir = .sparkREnv) + hiveCtx +} diff --git a/R/pkg/R/utils.R b/R/pkg/R/utils.R new file mode 100644 index 0000000000000..c337fb0751e72 --- /dev/null +++ b/R/pkg/R/utils.R @@ -0,0 +1,467 @@ +# +# 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. +# + +# Utilities and Helpers + +# Given a JList, returns an R list containing the same elements, the number +# of which is optionally upper bounded by `logicalUpperBound` (by default, +# return all elements). Takes care of deserializations and type conversions. +convertJListToRList <- function(jList, flatten, logicalUpperBound = NULL, + serializedMode = "byte") { + arrSize <- callJMethod(jList, "size") + + # Datasets with serializedMode == "string" (such as an RDD directly generated by textFile()): + # each partition is not dense-packed into one Array[Byte], and `arrSize` + # here corresponds to number of logical elements. Thus we can prune here. + if (serializedMode == "string" && !is.null(logicalUpperBound)) { + arrSize <- min(arrSize, logicalUpperBound) + } + + results <- if (arrSize > 0) { + lapply(0:(arrSize - 1), + function(index) { + obj <- callJMethod(jList, "get", as.integer(index)) + + # Assume it is either an R object or a Java obj ref. + if (inherits(obj, "jobj")) { + if (isInstanceOf(obj, "scala.Tuple2")) { + # JavaPairRDD[Array[Byte], Array[Byte]]. + + keyBytes = callJMethod(obj, "_1") + valBytes = callJMethod(obj, "_2") + res <- list(unserialize(keyBytes), + unserialize(valBytes)) + } else { + stop(paste("utils.R: convertJListToRList only supports", + "RDD[Array[Byte]] and", + "JavaPairRDD[Array[Byte], Array[Byte]] for now")) + } + } else { + if (inherits(obj, "raw")) { + if (serializedMode == "byte") { + # RDD[Array[Byte]]. `obj` is a whole partition. + res <- unserialize(obj) + # For serialized datasets, `obj` (and `rRaw`) here corresponds to + # one whole partition dense-packed together. We deserialize the + # whole partition first, then cap the number of elements to be returned. + } else if (serializedMode == "row") { + res <- readRowList(obj) + # For DataFrames that have been converted to RRDDs, we call readRowList + # which will read in each row of the RRDD as a list and deserialize + # each element. + flatten <<- FALSE + # Use global assignment to change the flatten flag. This means + # we don't have to worry about the default argument in other functions + # e.g. collect + } + # TODO: is it possible to distinguish element boundary so that we can + # unserialize only what we need? + if (!is.null(logicalUpperBound)) { + res <- head(res, n = logicalUpperBound) + } + } else { + # obj is of a primitive Java type, is simplified to R's + # corresponding type. + res <- list(obj) + } + } + res + }) + } else { + list() + } + + if (flatten) { + as.list(unlist(results, recursive = FALSE)) + } else { + as.list(results) + } +} + +# Returns TRUE if `name` refers to an RDD in the given environment `env` +isRDD <- function(name, env) { + obj <- get(name, envir = env) + inherits(obj, "RDD") +} + +#' Compute the hashCode of an object +#' +#' Java-style function to compute the hashCode for the given object. Returns +#' an integer value. +#' +#' @details +#' This only works for integer, numeric and character types right now. +#' +#' @param key the object to be hashed +#' @return the hash code as an integer +#' @export +#' @examples +#' hashCode(1L) # 1 +#' hashCode(1.0) # 1072693248 +#' hashCode("1") # 49 +hashCode <- function(key) { + if (class(key) == "integer") { + as.integer(key[[1]]) + } else if (class(key) == "numeric") { + # Convert the double to long and then calculate the hash code + rawVec <- writeBin(key[[1]], con = raw()) + intBits <- packBits(rawToBits(rawVec), "integer") + as.integer(bitwXor(intBits[2], intBits[1])) + } else if (class(key) == "character") { + .Call("stringHashCode", key) + } else { + warning(paste("Could not hash object, returning 0", sep = "")) + as.integer(0) + } +} + +# Create a new RDD with serializedMode == "byte". +# Return itself if already in "byte" format. +serializeToBytes <- function(rdd) { + if (!inherits(rdd, "RDD")) { + stop("Argument 'rdd' is not an RDD type.") + } + if (getSerializedMode(rdd) != "byte") { + ser.rdd <- lapply(rdd, function(x) { x }) + return(ser.rdd) + } else { + return(rdd) + } +} + +# Create a new RDD with serializedMode == "string". +# Return itself if already in "string" format. +serializeToString <- function(rdd) { + if (!inherits(rdd, "RDD")) { + stop("Argument 'rdd' is not an RDD type.") + } + if (getSerializedMode(rdd) != "string") { + ser.rdd <- lapply(rdd, function(x) { toString(x) }) + # force it to create jrdd using "string" + getJRDD(ser.rdd, serializedMode = "string") + return(ser.rdd) + } else { + return(rdd) + } +} + +# Fast append to list by using an accumulator. +# http://stackoverflow.com/questions/17046336/here-we-go-again-append-an-element-to-a-list-in-r +# +# The accumulator should has three fields size, counter and data. +# This function amortizes the allocation cost by doubling +# the size of the list every time it fills up. +addItemToAccumulator <- function(acc, item) { + if(acc$counter == acc$size) { + acc$size <- acc$size * 2 + length(acc$data) <- acc$size + } + acc$counter <- acc$counter + 1 + acc$data[[acc$counter]] <- item +} + +initAccumulator <- function() { + acc <- new.env() + acc$counter <- 0 + acc$data <- list(NULL) + acc$size <- 1 + acc +} + +# Utility function to sort a list of key value pairs +# Used in unit tests +sortKeyValueList <- function(kv_list, decreasing = FALSE) { + keys <- sapply(kv_list, function(x) x[[1]]) + kv_list[order(keys, decreasing = decreasing)] +} + +# Utility function to generate compact R lists from grouped rdd +# Used in Join-family functions +# param: +# tagged_list R list generated via groupByKey with tags(1L, 2L, ...) +# cnull Boolean list where each element determines whether the corresponding list should +# be converted to list(NULL) +genCompactLists <- function(tagged_list, cnull) { + len <- length(tagged_list) + lists <- list(vector("list", len), vector("list", len)) + index <- list(1, 1) + + for (x in tagged_list) { + tag <- x[[1]] + idx <- index[[tag]] + lists[[tag]][[idx]] <- x[[2]] + index[[tag]] <- idx + 1 + } + + len <- lapply(index, function(x) x - 1) + for (i in (1:2)) { + if (cnull[[i]] && len[[i]] == 0) { + lists[[i]] <- list(NULL) + } else { + length(lists[[i]]) <- len[[i]] + } + } + + lists +} + +# Utility function to merge compact R lists +# Used in Join-family functions +# param: +# left/right Two compact lists ready for Cartesian product +mergeCompactLists <- function(left, right) { + result <- list() + length(result) <- length(left) * length(right) + index <- 1 + for (i in left) { + for (j in right) { + result[[index]] <- list(i, j) + index <- index + 1 + } + } + result +} + +# Utility function to wrapper above two operations +# Used in Join-family functions +# param (same as genCompactLists): +# tagged_list R list generated via groupByKey with tags(1L, 2L, ...) +# cnull Boolean list where each element determines whether the corresponding list should +# be converted to list(NULL) +joinTaggedList <- function(tagged_list, cnull) { + lists <- genCompactLists(tagged_list, cnull) + mergeCompactLists(lists[[1]], lists[[2]]) +} + +# Utility function to reduce a key-value list with predicate +# Used in *ByKey functions +# param +# pair key-value pair +# keys/vals env of key/value with hashes +# updateOrCreatePred predicate function +# updateFn update or merge function for existing pair, similar with `mergeVal` @combineByKey +# createFn create function for new pair, similar with `createCombiner` @combinebykey +updateOrCreatePair <- function(pair, keys, vals, updateOrCreatePred, updateFn, createFn) { + # assume hashVal bind to `$hash`, key/val with index 1/2 + hashVal <- pair$hash + key <- pair[[1]] + val <- pair[[2]] + if (updateOrCreatePred(pair)) { + assign(hashVal, do.call(updateFn, list(get(hashVal, envir = vals), val)), envir = vals) + } else { + assign(hashVal, do.call(createFn, list(val)), envir = vals) + assign(hashVal, key, envir = keys) + } +} + +# Utility function to convert key&values envs into key-val list +convertEnvsToList <- function(keys, vals) { + lapply(ls(keys), + function(name) { + list(keys[[name]], vals[[name]]) + }) +} + +# Utility function to capture the varargs into environment object +varargsToEnv <- function(...) { + pairs <- as.list(substitute(list(...)))[-1L] + env <- new.env() + for (name in names(pairs)) { + env[[name]] <- pairs[[name]] + } + env +} + +getStorageLevel <- function(newLevel = c("DISK_ONLY", + "DISK_ONLY_2", + "MEMORY_AND_DISK", + "MEMORY_AND_DISK_2", + "MEMORY_AND_DISK_SER", + "MEMORY_AND_DISK_SER_2", + "MEMORY_ONLY", + "MEMORY_ONLY_2", + "MEMORY_ONLY_SER", + "MEMORY_ONLY_SER_2", + "OFF_HEAP")) { + match.arg(newLevel) + storageLevel <- switch(newLevel, + "DISK_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY"), + "DISK_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "DISK_ONLY_2"), + "MEMORY_AND_DISK" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK"), + "MEMORY_AND_DISK_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_2"), + "MEMORY_AND_DISK_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER"), + "MEMORY_AND_DISK_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_AND_DISK_SER_2"), + "MEMORY_ONLY" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY"), + "MEMORY_ONLY_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_2"), + "MEMORY_ONLY_SER" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER"), + "MEMORY_ONLY_SER_2" = callJStatic("org.apache.spark.storage.StorageLevel", "MEMORY_ONLY_SER_2"), + "OFF_HEAP" = callJStatic("org.apache.spark.storage.StorageLevel", "OFF_HEAP")) +} + +# Utility function for functions where an argument needs to be integer but we want to allow +# the user to type (for example) `5` instead of `5L` to avoid a confusing error message. +numToInt <- function(num) { + if (as.integer(num) != num) { + warning(paste("Coercing", as.list(sys.call())[[2]], "to integer.")) + } + as.integer(num) +} + +# create a Seq in JVM +toSeq <- function(...) { + callJStatic("org.apache.spark.sql.api.r.SQLUtils", "toSeq", list(...)) +} + +# create a Seq in JVM from a list +listToSeq <- function(l) { + callJStatic("org.apache.spark.sql.api.r.SQLUtils", "toSeq", l) +} + +# Utility function to recursively traverse the Abstract Syntax Tree (AST) of a +# user defined function (UDF), and to examine variables in the UDF to decide +# if their values should be included in the new function environment. +# param +# node The current AST node in the traversal. +# oldEnv The original function environment. +# defVars An Accumulator of variables names defined in the function's calling environment, +# including function argument and local variable names. +# checkedFunc An environment of function objects examined during cleanClosure. It can +# be considered as a "name"-to-"list of functions" mapping. +# newEnv A new function environment to store necessary function dependencies, an output argument. +processClosure <- function(node, oldEnv, defVars, checkedFuncs, newEnv) { + nodeLen <- length(node) + + if (nodeLen > 1 && typeof(node) == "language") { + # Recursive case: current AST node is an internal node, check for its children. + if (length(node[[1]]) > 1) { + for (i in 1:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else { # if node[[1]] is length of 1, check for some R special functions. + nodeChar <- as.character(node[[1]]) + if (nodeChar == "{" || nodeChar == "(") { # Skip start symbol. + for (i in 2:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else if (nodeChar == "<-" || nodeChar == "=" || + nodeChar == "<<-") { # Assignment Ops. + defVar <- node[[2]] + if (length(defVar) == 1 && typeof(defVar) == "symbol") { + # Add the defined variable name into defVars. + addItemToAccumulator(defVars, as.character(defVar)) + } else { + processClosure(node[[2]], oldEnv, defVars, checkedFuncs, newEnv) + } + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else if (nodeChar == "function") { # Function definition. + # Add parameter names. + newArgs <- names(node[[2]]) + lapply(newArgs, function(arg) { addItemToAccumulator(defVars, arg) }) + for (i in 3:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } else if (nodeChar == "$") { # Skip the field. + processClosure(node[[2]], oldEnv, defVars, checkedFuncs, newEnv) + } else if (nodeChar == "::" || nodeChar == ":::") { + processClosure(node[[3]], oldEnv, defVars, checkedFuncs, newEnv) + } else { + for (i in 1:nodeLen) { + processClosure(node[[i]], oldEnv, defVars, checkedFuncs, newEnv) + } + } + } + } else if (nodeLen == 1 && + (typeof(node) == "symbol" || typeof(node) == "language")) { + # Base case: current AST node is a leaf node and a symbol or a function call. + nodeChar <- as.character(node) + if (!nodeChar %in% defVars$data) { # Not a function parameter or local variable. + func.env <- oldEnv + topEnv <- parent.env(.GlobalEnv) + # Search in function environment, and function's enclosing environments + # up to global environment. There is no need to look into package environments + # above the global or namespace environment that is not SparkR below the global, + # as they are assumed to be loaded on workers. + while (!identical(func.env, topEnv)) { + # Namespaces other than "SparkR" will not be searched. + if (!isNamespace(func.env) || + (getNamespaceName(func.env) == "SparkR" && + !(nodeChar %in% getNamespaceExports("SparkR")))) { # Only include SparkR internals. + # Set parameter 'inherits' to FALSE since we do not need to search in + # attached package environments. + if (tryCatch(exists(nodeChar, envir = func.env, inherits = FALSE), + error = function(e) { FALSE })) { + obj <- get(nodeChar, envir = func.env, inherits = FALSE) + if (is.function(obj)) { # If the node is a function call. + funcList <- mget(nodeChar, envir = checkedFuncs, inherits = F, + ifnotfound = list(list(NULL)))[[1]] + found <- sapply(funcList, function(func) { + ifelse(identical(func, obj), TRUE, FALSE) + }) + if (sum(found) > 0) { # If function has been examined, ignore. + break + } + # Function has not been examined, record it and recursively clean its closure. + assign(nodeChar, + if (is.null(funcList[[1]])) { + list(obj) + } else { + append(funcList, obj) + }, + envir = checkedFuncs) + obj <- cleanClosure(obj, checkedFuncs) + } + assign(nodeChar, obj, envir = newEnv) + break + } + } + + # Continue to search in enclosure. + func.env <- parent.env(func.env) + } + } + } +} + +# Utility function to get user defined function (UDF) dependencies (closure). +# More specifically, this function captures the values of free variables defined +# outside a UDF, and stores them in the function's environment. +# param +# func A function whose closure needs to be captured. +# checkedFunc An environment of function objects examined during cleanClosure. It can be +# considered as a "name"-to-"list of functions" mapping. +# return value +# a new version of func that has an correct environment (closure). +cleanClosure <- function(func, checkedFuncs = new.env()) { + if (is.function(func)) { + newEnv <- new.env(parent = .GlobalEnv) + func.body <- body(func) + oldEnv <- environment(func) + # defVars is an Accumulator of variables names defined in the function's calling + # environment. First, function's arguments are added to defVars. + defVars <- initAccumulator() + argNames <- names(as.list(args(func))) + for (i in 1:(length(argNames) - 1)) { # Remove the ending NULL in pairlist. + addItemToAccumulator(defVars, argNames[i]) + } + # Recursively examine variables in the function body. + processClosure(func.body, oldEnv, defVars, checkedFuncs, newEnv) + environment(func) <- newEnv + } + func +} diff --git a/R/pkg/R/zzz.R b/R/pkg/R/zzz.R new file mode 100644 index 0000000000000..80d796d467943 --- /dev/null +++ b/R/pkg/R/zzz.R @@ -0,0 +1,21 @@ +# +# 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. +# + +.onLoad <- function(libname, pkgname) { + sparkR.onLoad(libname, pkgname) +} + diff --git a/R/pkg/inst/profile/general.R b/R/pkg/inst/profile/general.R new file mode 100644 index 0000000000000..8fe711b622086 --- /dev/null +++ b/R/pkg/inst/profile/general.R @@ -0,0 +1,22 @@ +# +# 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. +# + +.First <- function() { + home <- Sys.getenv("SPARK_HOME") + .libPaths(c(file.path(home, "R", "lib"), .libPaths())) + Sys.setenv(NOAWT=1) +} diff --git a/R/pkg/inst/profile/shell.R b/R/pkg/inst/profile/shell.R new file mode 100644 index 0000000000000..7a7f2031152a0 --- /dev/null +++ b/R/pkg/inst/profile/shell.R @@ -0,0 +1,31 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +.First <- function() { + home <- Sys.getenv("SPARK_HOME") + .libPaths(c(file.path(home, "R", "lib"), .libPaths())) + Sys.setenv(NOAWT=1) + + library(utils) + library(SparkR) + sc <- sparkR.init(Sys.getenv("MASTER", unset = "")) + assign("sc", sc, envir=.GlobalEnv) + sqlCtx <- sparkRSQL.init(sc) + assign("sqlCtx", sqlCtx, envir=.GlobalEnv) + cat("\n Welcome to SparkR!") + cat("\n Spark context is available as sc, SQL context is available as sqlCtx\n") +} diff --git a/R/pkg/inst/tests/test_binaryFile.R b/R/pkg/inst/tests/test_binaryFile.R new file mode 100644 index 0000000000000..4bb5f58d83dc9 --- /dev/null +++ b/R/pkg/inst/tests/test_binaryFile.R @@ -0,0 +1,90 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("functions on binary files") + +# JavaSparkContext handle +sc <- sparkR.init() + +mockFile = c("Spark is pretty.", "Spark is awesome.") + +test_that("saveAsObjectFile()/objectFile() following textFile() works", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1) + saveAsObjectFile(rdd, fileName2) + rdd <- objectFile(sc, fileName2) + expect_equal(collect(rdd), as.list(mockFile)) + + unlink(fileName1) + unlink(fileName2, recursive = TRUE) +}) + +test_that("saveAsObjectFile()/objectFile() works on a parallelized list", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + + l <- list(1, 2, 3) + rdd <- parallelize(sc, l) + saveAsObjectFile(rdd, fileName) + rdd <- objectFile(sc, fileName) + expect_equal(collect(rdd), l) + + unlink(fileName, recursive = TRUE) +}) + +test_that("saveAsObjectFile()/objectFile() following RDD transformations works", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1) + + words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + counts <- reduceByKey(wordCount, "+", 2L) + + saveAsObjectFile(counts, fileName2) + counts <- objectFile(sc, fileName2) + + output <- collect(counts) + expected <- list(list("awesome.", 1), list("Spark", 2), list("pretty.", 1), + list("is", 2)) + expect_equal(sortKeyValueList(output), sortKeyValueList(expected)) + + unlink(fileName1) + unlink(fileName2, recursive = TRUE) +}) + +test_that("saveAsObjectFile()/objectFile() works with multiple paths", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + + rdd1 <- parallelize(sc, "Spark is pretty.") + saveAsObjectFile(rdd1, fileName1) + rdd2 <- parallelize(sc, "Spark is awesome.") + saveAsObjectFile(rdd2, fileName2) + + rdd <- objectFile(sc, c(fileName1, fileName2)) + expect_true(count(rdd) == 2) + + unlink(fileName1, recursive = TRUE) + unlink(fileName2, recursive = TRUE) +}) + diff --git a/R/pkg/inst/tests/test_binary_function.R b/R/pkg/inst/tests/test_binary_function.R new file mode 100644 index 0000000000000..c15553ba28517 --- /dev/null +++ b/R/pkg/inst/tests/test_binary_function.R @@ -0,0 +1,68 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("binary functions") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Data +nums <- 1:10 +rdd <- parallelize(sc, nums, 2L) + +# File content +mockFile <- c("Spark is pretty.", "Spark is awesome.") + +test_that("union on two RDDs", { + actual <- collect(unionRDD(rdd, rdd)) + expect_equal(actual, as.list(rep(nums, 2))) + + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + text.rdd <- textFile(sc, fileName) + union.rdd <- unionRDD(rdd, text.rdd) + actual <- collect(union.rdd) + expect_equal(actual, c(as.list(nums), mockFile)) + expect_true(getSerializedMode(union.rdd) == "byte") + + rdd<- map(text.rdd, function(x) {x}) + union.rdd <- unionRDD(rdd, text.rdd) + actual <- collect(union.rdd) + expect_equal(actual, as.list(c(mockFile, mockFile))) + expect_true(getSerializedMode(union.rdd) == "byte") + + unlink(fileName) +}) + +test_that("cogroup on two RDDs", { + rdd1 <- parallelize(sc, list(list(1, 1), list(2, 4))) + rdd2 <- parallelize(sc, list(list(1, 2), list(1, 3))) + cogroup.rdd <- cogroup(rdd1, rdd2, numPartitions = 2L) + actual <- collect(cogroup.rdd) + expect_equal(actual, + list(list(1, list(list(1), list(2, 3))), list(2, list(list(4), list())))) + + rdd1 <- parallelize(sc, list(list("a", 1), list("a", 4))) + rdd2 <- parallelize(sc, list(list("b", 2), list("a", 3))) + cogroup.rdd <- cogroup(rdd1, rdd2, numPartitions = 2L) + actual <- collect(cogroup.rdd) + + expected <- list(list("b", list(list(), list(2))), list("a", list(list(1, 4), list(3)))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) +}) diff --git a/R/pkg/inst/tests/test_broadcast.R b/R/pkg/inst/tests/test_broadcast.R new file mode 100644 index 0000000000000..fee91a427d6d5 --- /dev/null +++ b/R/pkg/inst/tests/test_broadcast.R @@ -0,0 +1,48 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("broadcast variables") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Partitioned data +nums <- 1:2 +rrdd <- parallelize(sc, nums, 2L) + +test_that("using broadcast variable", { + randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100)) + randomMatBr <- broadcast(sc, randomMat) + + useBroadcast <- function(x) { + sum(value(randomMatBr) * x) + } + actual <- collect(lapply(rrdd, useBroadcast)) + expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) + expect_equal(actual, expected) +}) + +test_that("without using broadcast variable", { + randomMat <- matrix(nrow=10, ncol=10, data=rnorm(100)) + + useBroadcast <- function(x) { + sum(randomMat * x) + } + actual <- collect(lapply(rrdd, useBroadcast)) + expected <- list(sum(randomMat) * 1, sum(randomMat) * 2) + expect_equal(actual, expected) +}) diff --git a/R/pkg/inst/tests/test_context.R b/R/pkg/inst/tests/test_context.R new file mode 100644 index 0000000000000..e4aab37436a74 --- /dev/null +++ b/R/pkg/inst/tests/test_context.R @@ -0,0 +1,50 @@ +# +# 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. +# + +context("test functions in sparkR.R") + +test_that("repeatedly starting and stopping SparkR", { + for (i in 1:4) { + sc <- sparkR.init() + rdd <- parallelize(sc, 1:20, 2L) + expect_equal(count(rdd), 20) + sparkR.stop() + } +}) + +test_that("rdd GC across sparkR.stop", { + sparkR.stop() + sc <- sparkR.init() # sc should get id 0 + rdd1 <- parallelize(sc, 1:20, 2L) # rdd1 should get id 1 + rdd2 <- parallelize(sc, 1:10, 2L) # rdd2 should get id 2 + sparkR.stop() + + sc <- sparkR.init() # sc should get id 0 again + + # GC rdd1 before creating rdd3 and rdd2 after + rm(rdd1) + gc() + + rdd3 <- parallelize(sc, 1:20, 2L) # rdd3 should get id 1 now + rdd4 <- parallelize(sc, 1:10, 2L) # rdd4 should get id 2 now + + rm(rdd2) + gc() + + count(rdd3) + count(rdd4) +}) diff --git a/R/pkg/inst/tests/test_includePackage.R b/R/pkg/inst/tests/test_includePackage.R new file mode 100644 index 0000000000000..8152b448d0870 --- /dev/null +++ b/R/pkg/inst/tests/test_includePackage.R @@ -0,0 +1,57 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +context("include R packages") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Partitioned data +nums <- 1:2 +rdd <- parallelize(sc, nums, 2L) + +test_that("include inside function", { + # Only run the test if plyr is installed. + if ("plyr" %in% rownames(installed.packages())) { + suppressPackageStartupMessages(library(plyr)) + generateData <- function(x) { + suppressPackageStartupMessages(library(plyr)) + attach(airquality) + result <- transform(Ozone, logOzone = log(Ozone)) + result + } + + data <- lapplyPartition(rdd, generateData) + actual <- collect(data) + } +}) + +test_that("use include package", { + # Only run the test if plyr is installed. + if ("plyr" %in% rownames(installed.packages())) { + suppressPackageStartupMessages(library(plyr)) + generateData <- function(x) { + attach(airquality) + result <- transform(Ozone, logOzone = log(Ozone)) + result + } + + includePackage(sc, plyr) + data <- lapplyPartition(rdd, generateData) + actual <- collect(data) + } +}) diff --git a/R/pkg/inst/tests/test_parallelize_collect.R b/R/pkg/inst/tests/test_parallelize_collect.R new file mode 100644 index 0000000000000..fff028657db37 --- /dev/null +++ b/R/pkg/inst/tests/test_parallelize_collect.R @@ -0,0 +1,109 @@ +# +# 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. +# + +context("parallelize() and collect()") + +# Mock data +numVector <- c(-10:97) +numList <- list(sqrt(1), sqrt(2), sqrt(3), 4 ** 10) +strVector <- c("Dexter Morgan: I suppose I should be upset, even feel", + "violated, but I'm not. No, in fact, I think this is a friendly", + "message, like \"Hey, wanna play?\" and yes, I want to play. ", + "I really, really do.") +strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge, ", + "other times it helps me control the chaos.", + "Dexter Morgan: Harry and Dorris Morgan did a wonderful job ", + "raising me. But they're both dead now. I didn't kill them. Honest.") + +numPairs <- list(list(1, 1), list(1, 2), list(2, 2), list(2, 3)) +strPairs <- list(list(strList, strList), list(strList, strList)) + +# JavaSparkContext handle +jsc <- sparkR.init() + +# Tests + +test_that("parallelize() on simple vectors and lists returns an RDD", { + numVectorRDD <- parallelize(jsc, numVector, 1) + numVectorRDD2 <- parallelize(jsc, numVector, 10) + numListRDD <- parallelize(jsc, numList, 1) + numListRDD2 <- parallelize(jsc, numList, 4) + strVectorRDD <- parallelize(jsc, strVector, 2) + strVectorRDD2 <- parallelize(jsc, strVector, 3) + strListRDD <- parallelize(jsc, strList, 4) + strListRDD2 <- parallelize(jsc, strList, 1) + + rdds <- c(numVectorRDD, + numVectorRDD2, + numListRDD, + numListRDD2, + strVectorRDD, + strVectorRDD2, + strListRDD, + strListRDD2) + + for (rdd in rdds) { + expect_true(inherits(rdd, "RDD")) + expect_true(.hasSlot(rdd, "jrdd") + && inherits(rdd@jrdd, "jobj") + && isInstanceOf(rdd@jrdd, "org.apache.spark.api.java.JavaRDD")) + } +}) + +test_that("collect(), following a parallelize(), gives back the original collections", { + numVectorRDD <- parallelize(jsc, numVector, 10) + expect_equal(collect(numVectorRDD), as.list(numVector)) + + numListRDD <- parallelize(jsc, numList, 1) + numListRDD2 <- parallelize(jsc, numList, 4) + expect_equal(collect(numListRDD), as.list(numList)) + expect_equal(collect(numListRDD2), as.list(numList)) + + strVectorRDD <- parallelize(jsc, strVector, 2) + strVectorRDD2 <- parallelize(jsc, strVector, 3) + expect_equal(collect(strVectorRDD), as.list(strVector)) + expect_equal(collect(strVectorRDD2), as.list(strVector)) + + strListRDD <- parallelize(jsc, strList, 4) + strListRDD2 <- parallelize(jsc, strList, 1) + expect_equal(collect(strListRDD), as.list(strList)) + expect_equal(collect(strListRDD2), as.list(strList)) +}) + +test_that("regression: collect() following a parallelize() does not drop elements", { + # 10 %/% 6 = 1, ceiling(10 / 6) = 2 + collLen <- 10 + numPart <- 6 + expected <- runif(collLen) + actual <- collect(parallelize(jsc, expected, numPart)) + expect_equal(actual, as.list(expected)) +}) + +test_that("parallelize() and collect() work for lists of pairs (pairwise data)", { + # use the pairwise logical to indicate pairwise data + numPairsRDDD1 <- parallelize(jsc, numPairs, 1) + numPairsRDDD2 <- parallelize(jsc, numPairs, 2) + numPairsRDDD3 <- parallelize(jsc, numPairs, 3) + expect_equal(collect(numPairsRDDD1), numPairs) + expect_equal(collect(numPairsRDDD2), numPairs) + expect_equal(collect(numPairsRDDD3), numPairs) + # can also leave out the parameter name, if the params are supplied in order + strPairsRDDD1 <- parallelize(jsc, strPairs, 1) + strPairsRDDD2 <- parallelize(jsc, strPairs, 2) + expect_equal(collect(strPairsRDDD1), strPairs) + expect_equal(collect(strPairsRDDD2), strPairs) +}) diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R new file mode 100644 index 0000000000000..f75e0817b9406 --- /dev/null +++ b/R/pkg/inst/tests/test_rdd.R @@ -0,0 +1,644 @@ +# +# 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. +# + +context("basic RDD functions") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Data +nums <- 1:10 +rdd <- parallelize(sc, nums, 2L) + +intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200)) +intRdd <- parallelize(sc, intPairs, 2L) + +test_that("get number of partitions in RDD", { + expect_equal(numPartitions(rdd), 2) + expect_equal(numPartitions(intRdd), 2) +}) + +test_that("first on RDD", { + expect_true(first(rdd) == 1) + newrdd <- lapply(rdd, function(x) x + 1) + expect_true(first(newrdd) == 2) +}) + +test_that("count and length on RDD", { + expect_equal(count(rdd), 10) + expect_equal(length(rdd), 10) +}) + +test_that("count by values and keys", { + mods <- lapply(rdd, function(x) { x %% 3 }) + actual <- countByValue(mods) + expected <- list(list(0, 3L), list(1, 4L), list(2, 3L)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + actual <- countByKey(intRdd) + expected <- list(list(2L, 2L), list(1L, 2L)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("lapply on RDD", { + multiples <- lapply(rdd, function(x) { 2 * x }) + actual <- collect(multiples) + expect_equal(actual, as.list(nums * 2)) +}) + +test_that("lapplyPartition on RDD", { + sums <- lapplyPartition(rdd, function(part) { sum(unlist(part)) }) + actual <- collect(sums) + expect_equal(actual, list(15, 40)) +}) + +test_that("mapPartitions on RDD", { + sums <- mapPartitions(rdd, function(part) { sum(unlist(part)) }) + actual <- collect(sums) + expect_equal(actual, list(15, 40)) +}) + +test_that("flatMap() on RDDs", { + flat <- flatMap(intRdd, function(x) { list(x, x) }) + actual <- collect(flat) + expect_equal(actual, rep(intPairs, each=2)) +}) + +test_that("filterRDD on RDD", { + filtered.rdd <- filterRDD(rdd, function(x) { x %% 2 == 0 }) + actual <- collect(filtered.rdd) + expect_equal(actual, list(2, 4, 6, 8, 10)) + + filtered.rdd <- Filter(function(x) { x[[2]] < 0 }, intRdd) + actual <- collect(filtered.rdd) + expect_equal(actual, list(list(1L, -1))) + + # Filter out all elements. + filtered.rdd <- filterRDD(rdd, function(x) { x > 10 }) + actual <- collect(filtered.rdd) + expect_equal(actual, list()) +}) + +test_that("lookup on RDD", { + vals <- lookup(intRdd, 1L) + expect_equal(vals, list(-1, 200)) + + vals <- lookup(intRdd, 3L) + expect_equal(vals, list()) +}) + +test_that("several transformations on RDD (a benchmark on PipelinedRDD)", { + rdd2 <- rdd + for (i in 1:12) + rdd2 <- lapplyPartitionsWithIndex( + rdd2, function(split, part) { + part <- as.list(unlist(part) * split + i) + }) + rdd2 <- lapply(rdd2, function(x) x + x) + actual <- collect(rdd2) + expected <- list(24, 24, 24, 24, 24, + 168, 170, 172, 174, 176) + expect_equal(actual, expected) +}) + +test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkpoint()", { + # RDD + rdd2 <- rdd + # PipelinedRDD + rdd2 <- lapplyPartitionsWithIndex( + rdd2, + function(split, part) { + part <- as.list(unlist(part) * split) + }) + + cache(rdd2) + expect_true(rdd2@env$isCached) + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + + unpersist(rdd2) + expect_false(rdd2@env$isCached) + + persist(rdd2, "MEMORY_AND_DISK") + expect_true(rdd2@env$isCached) + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + + unpersist(rdd2) + expect_false(rdd2@env$isCached) + + setCheckpointDir(sc, "checkpoints") + checkpoint(rdd2) + expect_true(rdd2@env$isCheckpointed) + + rdd2 <- lapply(rdd2, function(x) x) + expect_false(rdd2@env$isCached) + expect_false(rdd2@env$isCheckpointed) + + # make sure the data is collectable + collect(rdd2) + + unlink("checkpoints") +}) + +test_that("reduce on RDD", { + sum <- reduce(rdd, "+") + expect_equal(sum, 55) + + # Also test with an inline function + sumInline <- reduce(rdd, function(x, y) { x + y }) + expect_equal(sumInline, 55) +}) + +test_that("lapply with dependency", { + fa <- 5 + multiples <- lapply(rdd, function(x) { fa * x }) + actual <- collect(multiples) + + expect_equal(actual, as.list(nums * 5)) +}) + +test_that("lapplyPartitionsWithIndex on RDDs", { + func <- function(splitIndex, part) { list(splitIndex, Reduce("+", part)) } + actual <- collect(lapplyPartitionsWithIndex(rdd, func), flatten = FALSE) + expect_equal(actual, list(list(0, 15), list(1, 40))) + + pairsRDD <- parallelize(sc, list(list(1, 2), list(3, 4), list(4, 8)), 1L) + partitionByParity <- function(key) { if (key %% 2 == 1) 0 else 1 } + mkTup <- function(splitIndex, part) { list(splitIndex, part) } + actual <- collect(lapplyPartitionsWithIndex( + partitionBy(pairsRDD, 2L, partitionByParity), + mkTup), + FALSE) + expect_equal(actual, list(list(0, list(list(1, 2), list(3, 4))), + list(1, list(list(4, 8))))) +}) + +test_that("sampleRDD() on RDDs", { + expect_equal(unlist(collect(sampleRDD(rdd, FALSE, 1.0, 2014L))), nums) +}) + +test_that("takeSample() on RDDs", { + # ported from RDDSuite.scala, modified seeds + data <- parallelize(sc, 1:100, 2L) + for (seed in 4:5) { + s <- takeSample(data, FALSE, 20L, seed) + expect_equal(length(s), 20L) + expect_equal(length(unique(s)), 20L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, FALSE, 200L, seed) + expect_equal(length(s), 100L) + expect_equal(length(unique(s)), 100L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 20L, seed) + expect_equal(length(s), 20L) + for (elem in s) { + expect_true(elem >= 1 && elem <= 100) + } + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 100L, seed) + expect_equal(length(s), 100L) + # Chance of getting all distinct elements is astronomically low, so test we + # got < 100 + expect_true(length(unique(s)) < 100L) + } + for (seed in 4:5) { + s <- takeSample(data, TRUE, 200L, seed) + expect_equal(length(s), 200L) + # Chance of getting all distinct elements is still quite low, so test we + # got < 100 + expect_true(length(unique(s)) < 100L) + } +}) + +test_that("mapValues() on pairwise RDDs", { + multiples <- mapValues(intRdd, function(x) { x * 2 }) + actual <- collect(multiples) + expected <- lapply(intPairs, function(x) { + list(x[[1]], x[[2]] * 2) + }) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("flatMapValues() on pairwise RDDs", { + l <- parallelize(sc, list(list(1, c(1,2)), list(2, c(3,4)))) + actual <- collect(flatMapValues(l, function(x) { x })) + expect_equal(actual, list(list(1,1), list(1,2), list(2,3), list(2,4))) + + # Generate x to x+1 for every value + actual <- collect(flatMapValues(intRdd, function(x) { x:(x + 1) })) + expect_equal(actual, + list(list(1L, -1), list(1L, 0), list(2L, 100), list(2L, 101), + list(2L, 1), list(2L, 2), list(1L, 200), list(1L, 201))) +}) + +test_that("reduceByKeyLocally() on PairwiseRDDs", { + pairs <- parallelize(sc, list(list(1, 2), list(1.1, 3), list(1, 4)), 2L) + actual <- reduceByKeyLocally(pairs, "+") + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, 6), list(1.1, 3)))) + + pairs <- parallelize(sc, list(list("abc", 1.2), list(1.1, 0), list("abc", 1.3), + list("bb", 5)), 4L) + actual <- reduceByKeyLocally(pairs, "+") + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("abc", 2.5), list(1.1, 0), list("bb", 5)))) +}) + +test_that("distinct() on RDDs", { + nums.rep2 <- rep(1:10, 2) + rdd.rep2 <- parallelize(sc, nums.rep2, 2L) + uniques <- distinct(rdd.rep2) + actual <- sort(unlist(collect(uniques))) + expect_equal(actual, nums) +}) + +test_that("maximum() on RDDs", { + max <- maximum(rdd) + expect_equal(max, 10) +}) + +test_that("minimum() on RDDs", { + min <- minimum(rdd) + expect_equal(min, 1) +}) + +test_that("sumRDD() on RDDs", { + sum <- sumRDD(rdd) + expect_equal(sum, 55) +}) + +test_that("keyBy on RDDs", { + func <- function(x) { x*x } + keys <- keyBy(rdd, func) + actual <- collect(keys) + expect_equal(actual, lapply(nums, function(x) { list(func(x), x) })) +}) + +test_that("repartition/coalesce on RDDs", { + rdd <- parallelize(sc, 1:20, 4L) # each partition contains 5 elements + + # repartition + r1 <- repartition(rdd, 2) + expect_equal(numPartitions(r1), 2L) + count <- length(collectPartition(r1, 0L)) + expect_true(count >= 8 && count <= 12) + + r2 <- repartition(rdd, 6) + expect_equal(numPartitions(r2), 6L) + count <- length(collectPartition(r2, 0L)) + expect_true(count >=0 && count <= 4) + + # coalesce + r3 <- coalesce(rdd, 1) + expect_equal(numPartitions(r3), 1L) + count <- length(collectPartition(r3, 0L)) + expect_equal(count, 20) +}) + +test_that("sortBy() on RDDs", { + sortedRdd <- sortBy(rdd, function(x) { x * x }, ascending = FALSE) + actual <- collect(sortedRdd) + expect_equal(actual, as.list(sort(nums, decreasing = TRUE))) + + rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L) + sortedRdd2 <- sortBy(rdd2, function(x) { x * x }) + actual <- collect(sortedRdd2) + expect_equal(actual, as.list(nums)) +}) + +test_that("takeOrdered() on RDDs", { + l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7) + rdd <- parallelize(sc, l) + actual <- takeOrdered(rdd, 6L) + expect_equal(actual, as.list(sort(unlist(l)))[1:6]) + + l <- list("e", "d", "c", "d", "a") + rdd <- parallelize(sc, l) + actual <- takeOrdered(rdd, 3L) + expect_equal(actual, as.list(sort(unlist(l)))[1:3]) +}) + +test_that("top() on RDDs", { + l <- list(10, 1, 2, 9, 3, 4, 5, 6, 7) + rdd <- parallelize(sc, l) + actual <- top(rdd, 6L) + expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:6]) + + l <- list("e", "d", "c", "d", "a") + rdd <- parallelize(sc, l) + actual <- top(rdd, 3L) + expect_equal(actual, as.list(sort(unlist(l), decreasing = TRUE))[1:3]) +}) + +test_that("fold() on RDDs", { + actual <- fold(rdd, 0, "+") + expect_equal(actual, Reduce("+", nums, 0)) + + rdd <- parallelize(sc, list()) + actual <- fold(rdd, 0, "+") + expect_equal(actual, 0) +}) + +test_that("aggregateRDD() on RDDs", { + rdd <- parallelize(sc, list(1, 2, 3, 4)) + zeroValue <- list(0, 0) + seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } + combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } + actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) + expect_equal(actual, list(10, 4)) + + rdd <- parallelize(sc, list()) + actual <- aggregateRDD(rdd, zeroValue, seqOp, combOp) + expect_equal(actual, list(0, 0)) +}) + +test_that("zipWithUniqueId() on RDDs", { + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) + actual <- collect(zipWithUniqueId(rdd)) + expected <- list(list("a", 0), list("b", 3), list("c", 1), + list("d", 4), list("e", 2)) + expect_equal(actual, expected) + + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) + actual <- collect(zipWithUniqueId(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) +}) + +test_that("zipWithIndex() on RDDs", { + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 3L) + actual <- collect(zipWithIndex(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) + + rdd <- parallelize(sc, list("a", "b", "c", "d", "e"), 1L) + actual <- collect(zipWithIndex(rdd)) + expected <- list(list("a", 0), list("b", 1), list("c", 2), + list("d", 3), list("e", 4)) + expect_equal(actual, expected) +}) + +test_that("glom() on RDD", { + rdd <- parallelize(sc, as.list(1:4), 2L) + actual <- collect(glom(rdd)) + expect_equal(actual, list(list(1, 2), list(3, 4))) +}) + +test_that("keys() on RDDs", { + keys <- keys(intRdd) + actual <- collect(keys) + expect_equal(actual, lapply(intPairs, function(x) { x[[1]] })) +}) + +test_that("values() on RDDs", { + values <- values(intRdd) + actual <- collect(values) + expect_equal(actual, lapply(intPairs, function(x) { x[[2]] })) +}) + +test_that("pipeRDD() on RDDs", { + actual <- collect(pipeRDD(rdd, "more")) + expected <- as.list(as.character(1:10)) + expect_equal(actual, expected) + + trailed.rdd <- parallelize(sc, c("1", "", "2\n", "3\n\r\n")) + actual <- collect(pipeRDD(trailed.rdd, "sort")) + expected <- list("", "1", "2", "3") + expect_equal(actual, expected) + + rev.nums <- 9:0 + rev.rdd <- parallelize(sc, rev.nums, 2L) + actual <- collect(pipeRDD(rev.rdd, "sort")) + expected <- as.list(as.character(c(5:9, 0:4))) + expect_equal(actual, expected) +}) + +test_that("zipRDD() on RDDs", { + rdd1 <- parallelize(sc, 0:4, 2) + rdd2 <- parallelize(sc, 1000:1004, 2) + actual <- collect(zipRDD(rdd1, rdd2)) + expect_equal(actual, + list(list(0, 1000), list(1, 1001), list(2, 1002), list(3, 1003), list(4, 1004))) + + mockFile = c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName, 1) + actual <- collect(zipRDD(rdd, rdd)) + expected <- lapply(mockFile, function(x) { list(x ,x) }) + expect_equal(actual, expected) + + rdd1 <- parallelize(sc, 0:1, 1) + actual <- collect(zipRDD(rdd1, rdd)) + expected <- lapply(0:1, function(x) { list(x, mockFile[x + 1]) }) + expect_equal(actual, expected) + + rdd1 <- map(rdd, function(x) { x }) + actual <- collect(zipRDD(rdd, rdd1)) + expected <- lapply(mockFile, function(x) { list(x, x) }) + expect_equal(actual, expected) + + unlink(fileName) +}) + +test_that("join() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) + rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, list(1, 2)), list(1, list(1, 3))))) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",4))) + rdd2 <- parallelize(sc, list(list("a",2), list("a",3))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("a", list(1, 2)), list("a", list(1, 3))))) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(actual, list()) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(join(rdd1, rdd2, 2L)) + expect_equal(actual, list()) +}) + +test_that("leftOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,1), list(2,4))) + rdd2 <- parallelize(sc, list(list(1,2), list(1,3))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(1, 2)), list(1, list(1, 3)), list(2, list(4, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",4))) + rdd2 <- parallelize(sc, list(list("a",2), list("a",3))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(4, NULL)), list("a", list(1, 2)), list("a", list(1, 3))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(1, NULL)), list(2, list(2, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(leftOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(2, NULL)), list("a", list(1, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) +}) + +test_that("rightOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,2), list(1,3))) + rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",2), list("a",3))) + rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(3, list(NULL, 3)), list(4, list(NULL, 4))))) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(rightOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("d", list(NULL, 4)), list("c", list(NULL, 3))))) +}) + +test_that("fullOuterJoin() on pairwise RDDs", { + rdd1 <- parallelize(sc, list(list(1,2), list(1,3), list(3,3))) + rdd2 <- parallelize(sc, list(list(1,1), list(2,4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list(1, list(2, 1)), list(1, list(3, 1)), list(2, list(NULL, 4)), list(3, list(3, NULL))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list("a",2), list("a",3), list("c", 1))) + rdd2 <- parallelize(sc, list(list("a",1), list("b",4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expected <- list(list("b", list(NULL, 4)), list("a", list(2, 1)), list("a", list(3, 1)), list("c", list(1, NULL))) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(expected)) + + rdd1 <- parallelize(sc, list(list(1,1), list(2,2))) + rdd2 <- parallelize(sc, list(list(3,3), list(4,4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list(1, list(1, NULL)), list(2, list(2, NULL)), list(3, list(NULL, 3)), list(4, list(NULL, 4))))) + + rdd1 <- parallelize(sc, list(list("a",1), list("b",2))) + rdd2 <- parallelize(sc, list(list("c",3), list("d",4))) + actual <- collect(fullOuterJoin(rdd1, rdd2, 2L)) + expect_equal(sortKeyValueList(actual), + sortKeyValueList(list(list("a", list(1, NULL)), list("b", list(2, NULL)), list("d", list(NULL, 4)), list("c", list(NULL, 3))))) +}) + +test_that("sortByKey() on pairwise RDDs", { + numPairsRdd <- map(rdd, function(x) { list (x, x) }) + sortedRdd <- sortByKey(numPairsRdd, ascending = FALSE) + actual <- collect(sortedRdd) + numPairs <- lapply(nums, function(x) { list (x, x) }) + expect_equal(actual, sortKeyValueList(numPairs, decreasing = TRUE)) + + rdd2 <- parallelize(sc, sort(nums, decreasing = TRUE), 2L) + numPairsRdd2 <- map(rdd2, function(x) { list (x, x) }) + sortedRdd2 <- sortByKey(numPairsRdd2) + actual <- collect(sortedRdd2) + expect_equal(actual, numPairs) + + # sort by string keys + l <- list(list("a", 1), list("b", 2), list("1", 3), list("d", 4), list("2", 5)) + rdd3 <- parallelize(sc, l, 2L) + sortedRdd3 <- sortByKey(rdd3) + actual <- collect(sortedRdd3) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # test on the boundary cases + + # boundary case 1: the RDD to be sorted has only 1 partition + rdd4 <- parallelize(sc, l, 1L) + sortedRdd4 <- sortByKey(rdd4) + actual <- collect(sortedRdd4) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # boundary case 2: the sorted RDD has only 1 partition + rdd5 <- parallelize(sc, l, 2L) + sortedRdd5 <- sortByKey(rdd5, numPartitions = 1L) + actual <- collect(sortedRdd5) + expect_equal(actual, list(list("1", 3), list("2", 5), list("a", 1), list("b", 2), list("d", 4))) + + # boundary case 3: the RDD to be sorted has only 1 element + l2 <- list(list("a", 1)) + rdd6 <- parallelize(sc, l2, 2L) + sortedRdd6 <- sortByKey(rdd6) + actual <- collect(sortedRdd6) + expect_equal(actual, l2) + + # boundary case 4: the RDD to be sorted has 0 element + l3 <- list() + rdd7 <- parallelize(sc, l3, 2L) + sortedRdd7 <- sortByKey(rdd7) + actual <- collect(sortedRdd7) + expect_equal(actual, l3) +}) + +test_that("collectAsMap() on a pairwise RDD", { + rdd <- parallelize(sc, list(list(1, 2), list(3, 4))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1` = 2, `3` = 4)) + + rdd <- parallelize(sc, list(list("a", 1), list("b", 2))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(a = 1, b = 2)) + + rdd <- parallelize(sc, list(list(1.1, 2.2), list(1.2, 2.4))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1.1` = 2.2, `1.2` = 2.4)) + + rdd <- parallelize(sc, list(list(1, "a"), list(2, "b"))) + vals <- collectAsMap(rdd) + expect_equal(vals, list(`1` = "a", `2` = "b")) +}) diff --git a/R/pkg/inst/tests/test_shuffle.R b/R/pkg/inst/tests/test_shuffle.R new file mode 100644 index 0000000000000..d1da8232aea81 --- /dev/null +++ b/R/pkg/inst/tests/test_shuffle.R @@ -0,0 +1,209 @@ +# +# 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. +# + +context("partitionBy, groupByKey, reduceByKey etc.") + +# JavaSparkContext handle +sc <- sparkR.init() + +# Data +intPairs <- list(list(1L, -1), list(2L, 100), list(2L, 1), list(1L, 200)) +intRdd <- parallelize(sc, intPairs, 2L) + +doublePairs <- list(list(1.5, -1), list(2.5, 100), list(2.5, 1), list(1.5, 200)) +doubleRdd <- parallelize(sc, doublePairs, 2L) + +numPairs <- list(list(1L, 100), list(2L, 200), list(4L, -1), list(3L, 1), + list(3L, 0)) +numPairsRdd <- parallelize(sc, numPairs, length(numPairs)) + +strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge and ", + "Dexter Morgan: Harry and Dorris Morgan did a wonderful job ") +strListRDD <- parallelize(sc, strList, 4) + +test_that("groupByKey for integers", { + grouped <- groupByKey(intRdd, 2L) + + actual <- collect(grouped) + + expected <- list(list(2L, list(100, 1)), list(1L, list(-1, 200))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("groupByKey for doubles", { + grouped <- groupByKey(doubleRdd, 2L) + + actual <- collect(grouped) + + expected <- list(list(1.5, list(-1, 200)), list(2.5, list(100, 1))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("reduceByKey for ints", { + reduced <- reduceByKey(intRdd, "+", 2L) + + actual <- collect(reduced) + + expected <- list(list(2L, 101), list(1L, 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("reduceByKey for doubles", { + reduced <- reduceByKey(doubleRdd, "+", 2L) + actual <- collect(reduced) + + expected <- list(list(1.5, 199), list(2.5, 101)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("combineByKey for ints", { + reduced <- combineByKey(intRdd, function(x) { x }, "+", "+", 2L) + + actual <- collect(reduced) + + expected <- list(list(2L, 101), list(1L, 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("combineByKey for doubles", { + reduced <- combineByKey(doubleRdd, function(x) { x }, "+", "+", 2L) + actual <- collect(reduced) + + expected <- list(list(1.5, 199), list(2.5, 101)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("aggregateByKey", { + # test aggregateByKey for int keys + rdd <- parallelize(sc, list(list(1, 1), list(1, 2), list(2, 3), list(2, 4))) + + zeroValue <- list(0, 0) + seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } + combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } + aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) + + actual <- collect(aggregatedRDD) + + expected <- list(list(1, list(3, 2)), list(2, list(7, 2))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test aggregateByKey for string keys + rdd <- parallelize(sc, list(list("a", 1), list("a", 2), list("b", 3), list("b", 4))) + + zeroValue <- list(0, 0) + seqOp <- function(x, y) { list(x[[1]] + y, x[[2]] + 1) } + combOp <- function(x, y) { list(x[[1]] + y[[1]], x[[2]] + y[[2]]) } + aggregatedRDD <- aggregateByKey(rdd, zeroValue, seqOp, combOp, 2L) + + actual <- collect(aggregatedRDD) + + expected <- list(list("a", list(3, 2)), list("b", list(7, 2))) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) +}) + +test_that("foldByKey", { + # test foldByKey for int keys + folded <- foldByKey(intRdd, 0, "+", 2L) + + actual <- collect(folded) + + expected <- list(list(2L, 101), list(1L, 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test foldByKey for double keys + folded <- foldByKey(doubleRdd, 0, "+", 2L) + + actual <- collect(folded) + + expected <- list(list(1.5, 199), list(2.5, 101)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test foldByKey for string keys + stringKeyPairs <- list(list("a", -1), list("b", 100), list("b", 1), list("a", 200)) + + stringKeyRDD <- parallelize(sc, stringKeyPairs) + folded <- foldByKey(stringKeyRDD, 0, "+", 2L) + + actual <- collect(folded) + + expected <- list(list("b", 101), list("a", 199)) + expect_equal(sortKeyValueList(actual), sortKeyValueList(expected)) + + # test foldByKey for empty pair RDD + rdd <- parallelize(sc, list()) + folded <- foldByKey(rdd, 0, "+", 2L) + actual <- collect(folded) + expected <- list() + expect_equal(actual, expected) + + # test foldByKey for RDD with only 1 pair + rdd <- parallelize(sc, list(list(1, 1))) + folded <- foldByKey(rdd, 0, "+", 2L) + actual <- collect(folded) + expected <- list(list(1, 1)) + expect_equal(actual, expected) +}) + +test_that("partitionBy() partitions data correctly", { + # Partition by magnitude + partitionByMagnitude <- function(key) { if (key >= 3) 1 else 0 } + + resultRDD <- partitionBy(numPairsRdd, 2L, partitionByMagnitude) + + expected_first <- list(list(1, 100), list(2, 200)) # key < 3 + expected_second <- list(list(4, -1), list(3, 1), list(3, 0)) # key >= 3 + actual_first <- collectPartition(resultRDD, 0L) + actual_second <- collectPartition(resultRDD, 1L) + + expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first)) + expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second)) +}) + +test_that("partitionBy works with dependencies", { + kOne <- 1 + partitionByParity <- function(key) { if (key %% 2 == kOne) 7 else 4 } + + # Partition by parity + resultRDD <- partitionBy(numPairsRdd, numPartitions = 2L, partitionByParity) + + # keys even; 100 %% 2 == 0 + expected_first <- list(list(2, 200), list(4, -1)) + # keys odd; 3 %% 2 == 1 + expected_second <- list(list(1, 100), list(3, 1), list(3, 0)) + actual_first <- collectPartition(resultRDD, 0L) + actual_second <- collectPartition(resultRDD, 1L) + + expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first)) + expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second)) +}) + +test_that("test partitionBy with string keys", { + words <- flatMap(strListRDD, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + resultRDD <- partitionBy(wordCount, 2L) + expected_first <- list(list("Dexter", 1), list("Dexter", 1)) + expected_second <- list(list("and", 1), list("and", 1)) + + actual_first <- Filter(function(item) { item[[1]] == "Dexter" }, + collectPartition(resultRDD, 0L)) + actual_second <- Filter(function(item) { item[[1]] == "and" }, + collectPartition(resultRDD, 1L)) + + expect_equal(sortKeyValueList(actual_first), sortKeyValueList(expected_first)) + expect_equal(sortKeyValueList(actual_second), sortKeyValueList(expected_second)) +}) diff --git a/R/pkg/inst/tests/test_sparkSQL.R b/R/pkg/inst/tests/test_sparkSQL.R new file mode 100644 index 0000000000000..cf5cf6d1692af --- /dev/null +++ b/R/pkg/inst/tests/test_sparkSQL.R @@ -0,0 +1,695 @@ +# +# 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. +# + +library(testthat) + +context("SparkSQL functions") + +# Tests for SparkSQL functions in SparkR + +sc <- sparkR.init() + +sqlCtx <- sparkRSQL.init(sc) + +mockLines <- c("{\"name\":\"Michael\"}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"Justin\", \"age\":19}") +jsonPath <- tempfile(pattern="sparkr-test", fileext=".tmp") +parquetPath <- tempfile(pattern="sparkr-test", fileext=".parquet") +writeLines(mockLines, jsonPath) + +test_that("infer types", { + expect_equal(infer_type(1L), "integer") + expect_equal(infer_type(1.0), "double") + expect_equal(infer_type("abc"), "string") + expect_equal(infer_type(TRUE), "boolean") + expect_equal(infer_type(as.Date("2015-03-11")), "date") + expect_equal(infer_type(as.POSIXlt("2015-03-11 12:13:04.043")), "timestamp") + expect_equal(infer_type(c(1L, 2L)), + list(type = 'array', elementType = "integer", containsNull = TRUE)) + expect_equal(infer_type(list(1L, 2L)), + list(type = 'array', elementType = "integer", containsNull = TRUE)) + expect_equal(infer_type(list(a = 1L, b = "2")), + list(type = "struct", + fields = list(list(name = "a", type = "integer", nullable = TRUE), + list(name = "b", type = "string", nullable = TRUE)))) + e <- new.env() + assign("a", 1L, envir = e) + expect_equal(infer_type(e), + list(type = "map", keyType = "string", valueType = "integer", + valueContainsNull = TRUE)) +}) + +test_that("create DataFrame from RDD", { + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) + df <- createDataFrame(sqlCtx, rdd, list("a", "b")) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + df <- createDataFrame(sqlCtx, rdd) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("_1", "_2")) + + fields <- list(list(name = "a", type = "integer", nullable = TRUE), + list(name = "b", type = "string", nullable = TRUE)) + schema <- list(type = "struct", fields = fields) + df <- createDataFrame(sqlCtx, rdd, schema) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) }) + df <- createDataFrame(sqlCtx, rdd) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) +}) + +test_that("toDF", { + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) + df <- toDF(rdd, list("a", "b")) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + df <- toDF(rdd) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("_1", "_2")) + + fields <- list(list(name = "a", type = "integer", nullable = TRUE), + list(name = "b", type = "string", nullable = TRUE)) + schema <- list(type = "struct", fields = fields) + df <- toDF(rdd, schema) + expect_true(inherits(df, "DataFrame")) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + + rdd <- lapply(parallelize(sc, 1:10), function(x) { list(a = x, b = as.character(x)) }) + df <- toDF(rdd) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 10) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) +}) + +test_that("create DataFrame from list or data.frame", { + l <- list(list(1, 2), list(3, 4)) + df <- createDataFrame(sqlCtx, l, c("a", "b")) + expect_equal(columns(df), c("a", "b")) + + l <- list(list(a=1, b=2), list(a=3, b=4)) + df <- createDataFrame(sqlCtx, l) + expect_equal(columns(df), c("a", "b")) + + a <- 1:3 + b <- c("a", "b", "c") + ldf <- data.frame(a, b) + df <- createDataFrame(sqlCtx, ldf) + expect_equal(columns(df), c("a", "b")) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) + expect_equal(count(df), 3) + ldf2 <- collect(df) + expect_equal(ldf$a, ldf2$a) +}) + +test_that("create DataFrame with different data types", { + l <- list(a = 1L, b = 2, c = TRUE, d = "ss", e = as.Date("2012-12-13"), + f = as.POSIXct("2015-03-15 12:13:14.056")) + df <- createDataFrame(sqlCtx, list(l)) + expect_equal(dtypes(df), list(c("a", "int"), c("b", "double"), c("c", "boolean"), + c("d", "string"), c("e", "date"), c("f", "timestamp"))) + expect_equal(count(df), 1) + expect_equal(collect(df), data.frame(l, stringsAsFactors = FALSE)) +}) + +# TODO: enable this test after fix serialization for nested object +#test_that("create DataFrame with nested array and struct", { +# e <- new.env() +# assign("n", 3L, envir = e) +# l <- list(1:10, list("a", "b"), e, list(a="aa", b=3L)) +# df <- createDataFrame(sqlCtx, list(l), c("a", "b", "c", "d")) +# expect_equal(dtypes(df), list(c("a", "array"), c("b", "array"), +# c("c", "map"), c("d", "struct"))) +# expect_equal(count(df), 1) +# ldf <- collect(df) +# expect_equal(ldf[1,], l[[1]]) +#}) + +test_that("jsonFile() on a local file returns a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) +}) + +test_that("jsonRDD() on a RDD with json string", { + rdd <- parallelize(sc, mockLines) + expect_true(count(rdd) == 3) + df <- jsonRDD(sqlCtx, rdd) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) + + rdd2 <- flatMap(rdd, function(x) c(x, x)) + df <- jsonRDD(sqlCtx, rdd2) + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 6) +}) + +test_that("test cache, uncache and clearCache", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + cacheTable(sqlCtx, "table1") + uncacheTable(sqlCtx, "table1") + clearCache(sqlCtx) + dropTempTable(sqlCtx, "table1") +}) + +test_that("test tableNames and tables", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + expect_true(length(tableNames(sqlCtx)) == 1) + df <- tables(sqlCtx) + expect_true(count(df) == 1) + dropTempTable(sqlCtx, "table1") +}) + +test_that("registerTempTable() results in a queryable table and sql() results in a new DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + newdf <- sql(sqlCtx, "SELECT * FROM table1 where name = 'Michael'") + expect_true(inherits(newdf, "DataFrame")) + expect_true(count(newdf) == 1) + dropTempTable(sqlCtx, "table1") +}) + +test_that("insertInto() on a registered table", { + df <- loadDF(sqlCtx, jsonPath, "json") + saveDF(df, parquetPath, "parquet", "overwrite") + dfParquet <- loadDF(sqlCtx, parquetPath, "parquet") + + lines <- c("{\"name\":\"Bob\", \"age\":24}", + "{\"name\":\"James\", \"age\":35}") + jsonPath2 <- tempfile(pattern="jsonPath2", fileext=".tmp") + parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") + writeLines(lines, jsonPath2) + df2 <- loadDF(sqlCtx, jsonPath2, "json") + saveDF(df2, parquetPath2, "parquet", "overwrite") + dfParquet2 <- loadDF(sqlCtx, parquetPath2, "parquet") + + registerTempTable(dfParquet, "table1") + insertInto(dfParquet2, "table1") + expect_true(count(sql(sqlCtx, "select * from table1")) == 5) + expect_true(first(sql(sqlCtx, "select * from table1 order by age"))$name == "Michael") + dropTempTable(sqlCtx, "table1") + + registerTempTable(dfParquet, "table1") + insertInto(dfParquet2, "table1", overwrite = TRUE) + expect_true(count(sql(sqlCtx, "select * from table1")) == 2) + expect_true(first(sql(sqlCtx, "select * from table1 order by age"))$name == "Bob") + dropTempTable(sqlCtx, "table1") +}) + +test_that("table() returns a new DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + registerTempTable(df, "table1") + tabledf <- table(sqlCtx, "table1") + expect_true(inherits(tabledf, "DataFrame")) + expect_true(count(tabledf) == 3) + dropTempTable(sqlCtx, "table1") +}) + +test_that("toRDD() returns an RRDD", { + df <- jsonFile(sqlCtx, jsonPath) + testRDD <- toRDD(df) + expect_true(inherits(testRDD, "RDD")) + expect_true(count(testRDD) == 3) +}) + +test_that("union on two RDDs created from DataFrames returns an RRDD", { + df <- jsonFile(sqlCtx, jsonPath) + RDD1 <- toRDD(df) + RDD2 <- toRDD(df) + unioned <- unionRDD(RDD1, RDD2) + expect_true(inherits(unioned, "RDD")) + expect_true(SparkR:::getSerializedMode(unioned) == "byte") + expect_true(collect(unioned)[[2]]$name == "Andy") +}) + +test_that("union on mixed serialization types correctly returns a byte RRDD", { + # Byte RDD + nums <- 1:10 + rdd <- parallelize(sc, nums, 2L) + + # String RDD + textLines <- c("Michael", + "Andy, 30", + "Justin, 19") + textPath <- tempfile(pattern="sparkr-textLines", fileext=".tmp") + writeLines(textLines, textPath) + textRDD <- textFile(sc, textPath) + + df <- jsonFile(sqlCtx, jsonPath) + dfRDD <- toRDD(df) + + unionByte <- unionRDD(rdd, dfRDD) + expect_true(inherits(unionByte, "RDD")) + expect_true(SparkR:::getSerializedMode(unionByte) == "byte") + expect_true(collect(unionByte)[[1]] == 1) + expect_true(collect(unionByte)[[12]]$name == "Andy") + + unionString <- unionRDD(textRDD, dfRDD) + expect_true(inherits(unionString, "RDD")) + expect_true(SparkR:::getSerializedMode(unionString) == "byte") + expect_true(collect(unionString)[[1]] == "Michael") + expect_true(collect(unionString)[[5]]$name == "Andy") +}) + +test_that("objectFile() works with row serialization", { + objectPath <- tempfile(pattern="spark-test", fileext=".tmp") + df <- jsonFile(sqlCtx, jsonPath) + dfRDD <- toRDD(df) + saveAsObjectFile(coalesce(dfRDD, 1L), objectPath) + objectIn <- objectFile(sc, objectPath) + + expect_true(inherits(objectIn, "RDD")) + expect_equal(SparkR:::getSerializedMode(objectIn), "byte") + expect_equal(collect(objectIn)[[2]]$age, 30) +}) + +test_that("lapply() on a DataFrame returns an RDD with the correct columns", { + df <- jsonFile(sqlCtx, jsonPath) + testRDD <- lapply(df, function(row) { + row$newCol <- row$age + 5 + row + }) + expect_true(inherits(testRDD, "RDD")) + collected <- collect(testRDD) + expect_true(collected[[1]]$name == "Michael") + expect_true(collected[[2]]$newCol == "35") +}) + +test_that("collect() returns a data.frame", { + df <- jsonFile(sqlCtx, jsonPath) + rdf <- collect(df) + expect_true(is.data.frame(rdf)) + expect_true(names(rdf)[1] == "age") + expect_true(nrow(rdf) == 3) + expect_true(ncol(rdf) == 2) +}) + +test_that("limit() returns DataFrame with the correct number of rows", { + df <- jsonFile(sqlCtx, jsonPath) + dfLimited <- limit(df, 2) + expect_true(inherits(dfLimited, "DataFrame")) + expect_true(count(dfLimited) == 2) +}) + +test_that("collect() and take() on a DataFrame return the same number of rows and columns", { + df <- jsonFile(sqlCtx, jsonPath) + expect_true(nrow(collect(df)) == nrow(take(df, 10))) + expect_true(ncol(collect(df)) == ncol(take(df, 10))) +}) + +test_that("multiple pipeline transformations starting with a DataFrame result in an RDD with the correct values", { + df <- jsonFile(sqlCtx, jsonPath) + first <- lapply(df, function(row) { + row$age <- row$age + 5 + row + }) + second <- lapply(first, function(row) { + row$testCol <- if (row$age == 35 && !is.na(row$age)) TRUE else FALSE + row + }) + expect_true(inherits(second, "RDD")) + expect_true(count(second) == 3) + expect_true(collect(second)[[2]]$age == 35) + expect_true(collect(second)[[2]]$testCol) + expect_false(collect(second)[[3]]$testCol) +}) + +test_that("cache(), persist(), and unpersist() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + expect_false(df@env$isCached) + cache(df) + expect_true(df@env$isCached) + + unpersist(df) + expect_false(df@env$isCached) + + persist(df, "MEMORY_AND_DISK") + expect_true(df@env$isCached) + + unpersist(df) + expect_false(df@env$isCached) + + # make sure the data is collectable + expect_true(is.data.frame(collect(df))) +}) + +test_that("schema(), dtypes(), columns(), names() return the correct values/format", { + df <- jsonFile(sqlCtx, jsonPath) + testSchema <- schema(df) + expect_true(length(testSchema$fields()) == 2) + expect_true(testSchema$fields()[[1]]$dataType.toString() == "LongType") + expect_true(testSchema$fields()[[2]]$dataType.simpleString() == "string") + expect_true(testSchema$fields()[[1]]$name() == "age") + + testTypes <- dtypes(df) + expect_true(length(testTypes[[1]]) == 2) + expect_true(testTypes[[1]][1] == "age") + + testCols <- columns(df) + expect_true(length(testCols) == 2) + expect_true(testCols[2] == "name") + + testNames <- names(df) + expect_true(length(testNames) == 2) + expect_true(testNames[2] == "name") +}) + +test_that("head() and first() return the correct data", { + df <- jsonFile(sqlCtx, jsonPath) + testHead <- head(df) + expect_true(nrow(testHead) == 3) + expect_true(ncol(testHead) == 2) + + testHead2 <- head(df, 2) + expect_true(nrow(testHead2) == 2) + expect_true(ncol(testHead2) == 2) + + testFirst <- first(df) + expect_true(nrow(testFirst) == 1) +}) + +test_that("distinct() on DataFrames", { + lines <- c("{\"name\":\"Michael\"}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"Justin\", \"age\":19}", + "{\"name\":\"Justin\", \"age\":19}") + jsonPathWithDup <- tempfile(pattern="sparkr-test", fileext=".tmp") + writeLines(lines, jsonPathWithDup) + + df <- jsonFile(sqlCtx, jsonPathWithDup) + uniques <- distinct(df) + expect_true(inherits(uniques, "DataFrame")) + expect_true(count(uniques) == 3) +}) + +test_that("sampleDF on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + sampled <- sampleDF(df, FALSE, 1.0) + expect_equal(nrow(collect(sampled)), count(df)) + expect_true(inherits(sampled, "DataFrame")) + sampled2 <- sampleDF(df, FALSE, 0.1) + expect_true(count(sampled2) < 3) +}) + +test_that("select operators", { + df <- select(jsonFile(sqlCtx, jsonPath), "name", "age") + expect_true(inherits(df$name, "Column")) + expect_true(inherits(df[[2]], "Column")) + expect_true(inherits(df[["age"]], "Column")) + + expect_true(inherits(df[,1], "DataFrame")) + expect_equal(columns(df[,1]), c("name")) + expect_equal(columns(df[,"age"]), c("age")) + df2 <- df[,c("age", "name")] + expect_true(inherits(df2, "DataFrame")) + expect_equal(columns(df2), c("age", "name")) + + df$age2 <- df$age + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == df$age)), 2) + df$age2 <- df$age * 2 + expect_equal(columns(df), c("name", "age", "age2")) + expect_equal(count(where(df, df$age2 == df$age * 2)), 2) +}) + +test_that("select with column", { + df <- jsonFile(sqlCtx, jsonPath) + df1 <- select(df, "name") + expect_true(columns(df1) == c("name")) + expect_true(count(df1) == 3) + + df2 <- select(df, df$age) + expect_true(columns(df2) == c("age")) + expect_true(count(df2) == 3) +}) + +test_that("selectExpr() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + selected <- selectExpr(df, "age * 2") + expect_true(names(selected) == "(age * 2)") + expect_equal(collect(selected), collect(select(df, df$age * 2L))) + + selected2 <- selectExpr(df, "name as newName", "abs(age) as age") + expect_equal(names(selected2), c("newName", "age")) + expect_true(count(selected2) == 3) +}) + +test_that("column calculation", { + df <- jsonFile(sqlCtx, jsonPath) + d <- collect(select(df, alias(df$age + 1, "age2"))) + expect_true(names(d) == c("age2")) + df2 <- select(df, lower(df$name), abs(df$age)) + expect_true(inherits(df2, "DataFrame")) + expect_true(count(df2) == 3) +}) + +test_that("load() from json file", { + df <- loadDF(sqlCtx, jsonPath, "json") + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) +}) + +test_that("save() as parquet file", { + df <- loadDF(sqlCtx, jsonPath, "json") + saveDF(df, parquetPath, "parquet", mode="overwrite") + df2 <- loadDF(sqlCtx, parquetPath, "parquet") + expect_true(inherits(df2, "DataFrame")) + expect_true(count(df2) == 3) +}) + +test_that("test HiveContext", { + hiveCtx <- tryCatch({ + newJObject("org.apache.spark.sql.hive.test.TestHiveContext", ssc) + }, error = function(err) { + skip("Hive is not build with SparkSQL, skipped") + }) + df <- createExternalTable(hiveCtx, "json", jsonPath, "json") + expect_true(inherits(df, "DataFrame")) + expect_true(count(df) == 3) + df2 <- sql(hiveCtx, "select * from json") + expect_true(inherits(df2, "DataFrame")) + expect_true(count(df2) == 3) + + jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + saveAsTable(df, "json", "json", "append", path = jsonPath2) + df3 <- sql(hiveCtx, "select * from json") + expect_true(inherits(df3, "DataFrame")) + expect_true(count(df3) == 6) +}) + +test_that("column operators", { + c <- SparkR:::col("a") + c2 <- (- c + 1 - 2) * 3 / 4.0 + c3 <- (c + c2 - c2) * c2 %% c2 + c4 <- (c > c2) & (c2 <= c3) | (c == c2) & (c2 != c3) +}) + +test_that("column functions", { + c <- SparkR:::col("a") + c2 <- min(c) + max(c) + sum(c) + avg(c) + count(c) + abs(c) + sqrt(c) + c3 <- lower(c) + upper(c) + first(c) + last(c) + c4 <- approxCountDistinct(c) + countDistinct(c) + cast(c, "string") +}) + +test_that("string operators", { + df <- jsonFile(sqlCtx, jsonPath) + expect_equal(count(where(df, like(df$name, "A%"))), 1) + expect_equal(count(where(df, startsWith(df$name, "A"))), 1) + expect_equal(first(select(df, substr(df$name, 1, 2)))[[1]], "Mi") + expect_equal(collect(select(df, cast(df$age, "string")))[[2, 1]], "30") +}) + +test_that("group by", { + df <- jsonFile(sqlCtx, jsonPath) + df1 <- agg(df, name = "max", age = "sum") + expect_true(1 == count(df1)) + df1 <- agg(df, age2 = max(df$age)) + expect_true(1 == count(df1)) + expect_equal(columns(df1), c("age2")) + + gd <- groupBy(df, "name") + expect_true(inherits(gd, "GroupedData")) + df2 <- count(gd) + expect_true(inherits(df2, "DataFrame")) + expect_true(3 == count(df2)) + + df3 <- agg(gd, age = "sum") + expect_true(inherits(df3, "DataFrame")) + expect_true(3 == count(df3)) + + df3 <- agg(gd, age = sum(df$age)) + expect_true(inherits(df3, "DataFrame")) + expect_true(3 == count(df3)) + expect_equal(columns(df3), c("name", "age")) + + df4 <- sum(gd, "age") + expect_true(inherits(df4, "DataFrame")) + expect_true(3 == count(df4)) + expect_true(3 == count(mean(gd, "age"))) + expect_true(3 == count(max(gd, "age"))) +}) + +test_that("sortDF() and orderBy() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + sorted <- sortDF(df, df$age) + expect_true(collect(sorted)[1,2] == "Michael") + + sorted2 <- sortDF(df, "name") + expect_true(collect(sorted2)[2,"age"] == 19) + + sorted3 <- orderBy(df, asc(df$age)) + expect_true(is.na(first(sorted3)$age)) + expect_true(collect(sorted3)[2, "age"] == 19) + + sorted4 <- orderBy(df, desc(df$name)) + expect_true(first(sorted4)$name == "Michael") + expect_true(collect(sorted4)[3,"name"] == "Andy") +}) + +test_that("filter() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + filtered <- filter(df, "age > 20") + expect_true(count(filtered) == 1) + expect_true(collect(filtered)$name == "Andy") + filtered2 <- where(df, df$name != "Michael") + expect_true(count(filtered2) == 2) + expect_true(collect(filtered2)$age[2] == 19) +}) + +test_that("join() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + + mockLines2 <- c("{\"name\":\"Michael\", \"test\": \"yes\"}", + "{\"name\":\"Andy\", \"test\": \"no\"}", + "{\"name\":\"Justin\", \"test\": \"yes\"}", + "{\"name\":\"Bob\", \"test\": \"yes\"}") + jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + writeLines(mockLines2, jsonPath2) + df2 <- jsonFile(sqlCtx, jsonPath2) + + joined <- join(df, df2) + expect_equal(names(joined), c("age", "name", "name", "test")) + expect_true(count(joined) == 12) + + joined2 <- join(df, df2, df$name == df2$name) + expect_equal(names(joined2), c("age", "name", "name", "test")) + expect_true(count(joined2) == 3) + + joined3 <- join(df, df2, df$name == df2$name, "right_outer") + expect_equal(names(joined3), c("age", "name", "name", "test")) + expect_true(count(joined3) == 4) + expect_true(is.na(collect(orderBy(joined3, joined3$age))$age[2])) + + joined4 <- select(join(df, df2, df$name == df2$name, "outer"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined4), c("newAge", "name", "test")) + expect_true(count(joined4) == 4) + expect_equal(collect(orderBy(joined4, joined4$name))$newAge[3], 24) +}) + +test_that("toJSON() returns an RDD of the correct values", { + df <- jsonFile(sqlCtx, jsonPath) + testRDD <- toJSON(df) + expect_true(inherits(testRDD, "RDD")) + expect_true(SparkR:::getSerializedMode(testRDD) == "string") + expect_equal(collect(testRDD)[[1]], mockLines[1]) +}) + +test_that("showDF()", { + df <- jsonFile(sqlCtx, jsonPath) + expect_output(showDF(df), "age name \nnull Michael\n30 Andy \n19 Justin ") +}) + +test_that("isLocal()", { + df <- jsonFile(sqlCtx, jsonPath) + expect_false(isLocal(df)) +}) + +test_that("unionAll(), subtract(), and intersect() on a DataFrame", { + df <- jsonFile(sqlCtx, jsonPath) + + lines <- c("{\"name\":\"Bob\", \"age\":24}", + "{\"name\":\"Andy\", \"age\":30}", + "{\"name\":\"James\", \"age\":35}") + jsonPath2 <- tempfile(pattern="sparkr-test", fileext=".tmp") + writeLines(lines, jsonPath2) + df2 <- loadDF(sqlCtx, jsonPath2, "json") + + unioned <- sortDF(unionAll(df, df2), df$age) + expect_true(inherits(unioned, "DataFrame")) + expect_true(count(unioned) == 6) + expect_true(first(unioned)$name == "Michael") + + subtracted <- sortDF(subtract(df, df2), desc(df$age)) + expect_true(inherits(unioned, "DataFrame")) + expect_true(count(subtracted) == 2) + expect_true(first(subtracted)$name == "Justin") + + intersected <- sortDF(intersect(df, df2), df$age) + expect_true(inherits(unioned, "DataFrame")) + expect_true(count(intersected) == 1) + expect_true(first(intersected)$name == "Andy") +}) + +test_that("withColumn() and withColumnRenamed()", { + df <- jsonFile(sqlCtx, jsonPath) + newDF <- withColumn(df, "newAge", df$age + 2) + expect_true(length(columns(newDF)) == 3) + expect_true(columns(newDF)[3] == "newAge") + expect_true(first(filter(newDF, df$name != "Michael"))$newAge == 32) + + newDF2 <- withColumnRenamed(df, "age", "newerAge") + expect_true(length(columns(newDF2)) == 2) + expect_true(columns(newDF2)[1] == "newerAge") +}) + +test_that("saveDF() on DataFrame and works with parquetFile", { + df <- jsonFile(sqlCtx, jsonPath) + saveDF(df, parquetPath, "parquet", mode="overwrite") + parquetDF <- parquetFile(sqlCtx, parquetPath) + expect_true(inherits(parquetDF, "DataFrame")) + expect_equal(count(df), count(parquetDF)) +}) + +test_that("parquetFile works with multiple input paths", { + df <- jsonFile(sqlCtx, jsonPath) + saveDF(df, parquetPath, "parquet", mode="overwrite") + parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") + saveDF(df, parquetPath2, "parquet", mode="overwrite") + parquetDF <- parquetFile(sqlCtx, parquetPath, parquetPath2) + expect_true(inherits(parquetDF, "DataFrame")) + expect_true(count(parquetDF) == count(df)*2) +}) + +unlink(parquetPath) +unlink(jsonPath) diff --git a/R/pkg/inst/tests/test_take.R b/R/pkg/inst/tests/test_take.R new file mode 100644 index 0000000000000..7f4c7c315d787 --- /dev/null +++ b/R/pkg/inst/tests/test_take.R @@ -0,0 +1,67 @@ +# +# 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. +# + +context("tests RDD function take()") + +# Mock data +numVector <- c(-10:97) +numList <- list(sqrt(1), sqrt(2), sqrt(3), 4 ** 10) +strVector <- c("Dexter Morgan: I suppose I should be upset, even feel", + "violated, but I'm not. No, in fact, I think this is a friendly", + "message, like \"Hey, wanna play?\" and yes, I want to play. ", + "I really, really do.") +strList <- list("Dexter Morgan: Blood. Sometimes it sets my teeth on edge, ", + "other times it helps me control the chaos.", + "Dexter Morgan: Harry and Dorris Morgan did a wonderful job ", + "raising me. But they're both dead now. I didn't kill them. Honest.") + +# JavaSparkContext handle +jsc <- sparkR.init() + +test_that("take() gives back the original elements in correct count and order", { + numVectorRDD <- parallelize(jsc, numVector, 10) + # case: number of elements to take is less than the size of the first partition + expect_equal(take(numVectorRDD, 1), as.list(head(numVector, n = 1))) + # case: number of elements to take is the same as the size of the first partition + expect_equal(take(numVectorRDD, 11), as.list(head(numVector, n = 11))) + # case: number of elements to take is greater than all elements + expect_equal(take(numVectorRDD, length(numVector)), as.list(numVector)) + expect_equal(take(numVectorRDD, length(numVector) + 1), as.list(numVector)) + + numListRDD <- parallelize(jsc, numList, 1) + numListRDD2 <- parallelize(jsc, numList, 4) + expect_equal(take(numListRDD, 3), take(numListRDD2, 3)) + expect_equal(take(numListRDD, 5), take(numListRDD2, 5)) + expect_equal(take(numListRDD, 1), as.list(head(numList, n = 1))) + expect_equal(take(numListRDD2, 999), numList) + + strVectorRDD <- parallelize(jsc, strVector, 2) + strVectorRDD2 <- parallelize(jsc, strVector, 3) + expect_equal(take(strVectorRDD, 4), as.list(strVector)) + expect_equal(take(strVectorRDD2, 2), as.list(head(strVector, n = 2))) + + strListRDD <- parallelize(jsc, strList, 4) + strListRDD2 <- parallelize(jsc, strList, 1) + expect_equal(take(strListRDD, 3), as.list(head(strList, n = 3))) + expect_equal(take(strListRDD2, 1), as.list(head(strList, n = 1))) + + expect_true(length(take(strListRDD, 0)) == 0) + expect_true(length(take(strVectorRDD, 0)) == 0) + expect_true(length(take(numListRDD, 0)) == 0) + expect_true(length(take(numVectorRDD, 0)) == 0) +}) + diff --git a/R/pkg/inst/tests/test_textFile.R b/R/pkg/inst/tests/test_textFile.R new file mode 100644 index 0000000000000..7bb3e8003131d --- /dev/null +++ b/R/pkg/inst/tests/test_textFile.R @@ -0,0 +1,162 @@ +# +# 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. +# + +context("the textFile() function") + +# JavaSparkContext handle +sc <- sparkR.init() + +mockFile = c("Spark is pretty.", "Spark is awesome.") + +test_that("textFile() on a local file returns an RDD", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + expect_true(inherits(rdd, "RDD")) + expect_true(count(rdd) > 0) + expect_true(count(rdd) == 2) + + unlink(fileName) +}) + +test_that("textFile() followed by a collect() returns the same content", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + expect_equal(collect(rdd), as.list(mockFile)) + + unlink(fileName) +}) + +test_that("textFile() word count works as expected", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + + words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + counts <- reduceByKey(wordCount, "+", 2L) + output <- collect(counts) + expected <- list(list("pretty.", 1), list("is", 2), list("awesome.", 1), + list("Spark", 2)) + expect_equal(sortKeyValueList(output), sortKeyValueList(expected)) + + unlink(fileName) +}) + +test_that("several transformations on RDD created by textFile()", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) # RDD + for (i in 1:10) { + # PipelinedRDD initially created from RDD + rdd <- lapply(rdd, function(x) paste(x, x)) + } + collect(rdd) + + unlink(fileName) +}) + +test_that("textFile() followed by a saveAsTextFile() returns the same content", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1) + saveAsTextFile(rdd, fileName2) + rdd <- textFile(sc, fileName2) + expect_equal(collect(rdd), as.list(mockFile)) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("saveAsTextFile() on a parallelized list works as expected", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + l <- list(1, 2, 3) + rdd <- parallelize(sc, l) + saveAsTextFile(rdd, fileName) + rdd <- textFile(sc, fileName) + expect_equal(collect(rdd), lapply(l, function(x) {toString(x)})) + + unlink(fileName) +}) + +test_that("textFile() and saveAsTextFile() word count works as expected", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName1) + + rdd <- textFile(sc, fileName1) + + words <- flatMap(rdd, function(line) { strsplit(line, " ")[[1]] }) + wordCount <- lapply(words, function(word) { list(word, 1L) }) + + counts <- reduceByKey(wordCount, "+", 2L) + + saveAsTextFile(counts, fileName2) + rdd <- textFile(sc, fileName2) + + output <- collect(rdd) + expected <- list(list("awesome.", 1), list("Spark", 2), + list("pretty.", 1), list("is", 2)) + expectedStr <- lapply(expected, function(x) { toString(x) }) + expect_equal(sortKeyValueList(output), sortKeyValueList(expectedStr)) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("textFile() on multiple paths", { + fileName1 <- tempfile(pattern="spark-test", fileext=".tmp") + fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines("Spark is pretty.", fileName1) + writeLines("Spark is awesome.", fileName2) + + rdd <- textFile(sc, c(fileName1, fileName2)) + expect_true(count(rdd) == 2) + + unlink(fileName1) + unlink(fileName2) +}) + +test_that("Pipelined operations on RDDs created using textFile", { + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + rdd <- textFile(sc, fileName) + + lengths <- lapply(rdd, function(x) { length(x) }) + expect_equal(collect(lengths), list(1, 1)) + + lengthsPipelined <- lapply(lengths, function(x) { x + 10 }) + expect_equal(collect(lengthsPipelined), list(11, 11)) + + lengths30 <- lapply(lengthsPipelined, function(x) { x + 20 }) + expect_equal(collect(lengths30), list(31, 31)) + + lengths20 <- lapply(lengths, function(x) { x + 20 }) + expect_equal(collect(lengths20), list(21, 21)) + + unlink(fileName) +}) + diff --git a/R/pkg/inst/tests/test_utils.R b/R/pkg/inst/tests/test_utils.R new file mode 100644 index 0000000000000..9c5bb427932b4 --- /dev/null +++ b/R/pkg/inst/tests/test_utils.R @@ -0,0 +1,137 @@ +# +# 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. +# + +context("functions in utils.R") + +# JavaSparkContext handle +sc <- sparkR.init() + +test_that("convertJListToRList() gives back (deserializes) the original JLists + of strings and integers", { + # It's hard to manually create a Java List using rJava, since it does not + # support generics well. Instead, we rely on collect() returning a + # JList. + nums <- as.list(1:10) + rdd <- parallelize(sc, nums, 1L) + jList <- callJMethod(rdd@jrdd, "collect") + rList <- convertJListToRList(jList, flatten = TRUE) + expect_equal(rList, nums) + + strs <- as.list("hello", "spark") + rdd <- parallelize(sc, strs, 2L) + jList <- callJMethod(rdd@jrdd, "collect") + rList <- convertJListToRList(jList, flatten = TRUE) + expect_equal(rList, strs) +}) + +test_that("serializeToBytes on RDD", { + # File content + mockFile <- c("Spark is pretty.", "Spark is awesome.") + fileName <- tempfile(pattern="spark-test", fileext=".tmp") + writeLines(mockFile, fileName) + + text.rdd <- textFile(sc, fileName) + expect_true(getSerializedMode(text.rdd) == "string") + ser.rdd <- serializeToBytes(text.rdd) + expect_equal(collect(ser.rdd), as.list(mockFile)) + expect_true(getSerializedMode(ser.rdd) == "byte") + + unlink(fileName) +}) + +test_that("cleanClosure on R functions", { + y <- c(1, 2, 3) + g <- function(x) { x + 1 } + f <- function(x) { g(x) + y } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 2) # y, g + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + actual <- get("g", envir = env, inherits = FALSE) + expect_equal(actual, g) + + # Test for nested enclosures and package variables. + env2 <- new.env() + funcEnv <- new.env(parent = env2) + f <- function(x) { log(g(x) + y) } + environment(f) <- funcEnv # enclosing relationship: f -> funcEnv -> env2 -> .GlobalEnv + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 2) # "min" should not be included + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + actual <- get("g", envir = env, inherits = FALSE) + expect_equal(actual, g) + + base <- c(1, 2, 3) + l <- list(field = matrix(1)) + field <- matrix(2) + defUse <- 3 + g <- function(x) { x + y } + f <- function(x) { + defUse <- base::as.integer(x) + 1 # Test for access operators `::`. + lapply(x, g) + 1 # Test for capturing function call "g"'s closure as a argument of lapply. + l$field[1,1] <- 3 # Test for access operators `$`. + res <- defUse + l$field[1,] # Test for def-use chain of "defUse", and "" symbol. + f(res) # Test for recursive calls. + } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 3) # Only "g", "l" and "f". No "base", "field" or "defUse". + expect_true("g" %in% ls(env)) + expect_true("l" %in% ls(env)) + expect_true("f" %in% ls(env)) + expect_equal(get("l", envir = env, inherits = FALSE), l) + # "y" should be in the environemnt of g. + newG <- get("g", envir = env, inherits = FALSE) + env <- environment(newG) + expect_equal(length(ls(env)), 1) + actual <- get("y", envir = env, inherits = FALSE) + expect_equal(actual, y) + + # Test for function (and variable) definitions. + f <- function(x) { + g <- function(y) { y * 2 } + g(x) + } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(length(ls(env)), 0) # "y" and "g" should not be included. + + # Test for overriding variables in base namespace (Issue: SparkR-196). + nums <- as.list(1:10) + rdd <- parallelize(sc, nums, 2L) + t = 4 # Override base::t in .GlobalEnv. + f <- function(x) { x > t } + newF <- cleanClosure(f) + env <- environment(newF) + expect_equal(ls(env), "t") + expect_equal(get("t", envir = env, inherits = FALSE), t) + actual <- collect(lapply(rdd, f)) + expected <- as.list(c(rep(FALSE, 4), rep(TRUE, 6))) + expect_equal(actual, expected) + + # Test for broadcast variables. + a <- matrix(nrow=10, ncol=10, data=rnorm(100)) + aBroadcast <- broadcast(sc, a) + normMultiply <- function(x) { norm(aBroadcast$value) * x } + newnormMultiply <- SparkR:::cleanClosure(normMultiply) + env <- environment(newnormMultiply) + expect_equal(ls(env), "aBroadcast") + expect_equal(get("aBroadcast", envir = env, inherits = FALSE), aBroadcast) +}) diff --git a/R/pkg/inst/worker/daemon.R b/R/pkg/inst/worker/daemon.R new file mode 100644 index 0000000000000..3584b418a71a9 --- /dev/null +++ b/R/pkg/inst/worker/daemon.R @@ -0,0 +1,52 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Worker daemon + +rLibDir <- Sys.getenv("SPARKR_RLIBDIR") +script <- paste(rLibDir, "SparkR/worker/worker.R", sep = "/") + +# preload SparkR package, speedup worker +.libPaths(c(rLibDir, .libPaths())) +suppressPackageStartupMessages(library(SparkR)) + +port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) +inputCon <- socketConnection(port = port, open = "rb", blocking = TRUE, timeout = 3600) + +while (TRUE) { + ready <- socketSelect(list(inputCon)) + if (ready) { + port <- SparkR:::readInt(inputCon) + # There is a small chance that it could be interrupted by signal, retry one time + if (length(port) == 0) { + port <- SparkR:::readInt(inputCon) + if (length(port) == 0) { + cat("quitting daemon\n") + quit(save = "no") + } + } + p <- parallel:::mcfork() + if (inherits(p, "masterProcess")) { + close(inputCon) + Sys.setenv(SPARKR_WORKER_PORT = port) + source(script) + # Set SIGUSR1 so that child can exit + tools::pskill(Sys.getpid(), tools::SIGUSR1) + parallel:::mcexit(0L) + } + } +} diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R new file mode 100644 index 0000000000000..c6542928e8ddd --- /dev/null +++ b/R/pkg/inst/worker/worker.R @@ -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. +# + +# Worker class + +rLibDir <- Sys.getenv("SPARKR_RLIBDIR") +# Set libPaths to include SparkR package as loadNamespace needs this +# TODO: Figure out if we can avoid this by not loading any objects that require +# SparkR namespace +.libPaths(c(rLibDir, .libPaths())) +suppressPackageStartupMessages(library(SparkR)) + +port <- as.integer(Sys.getenv("SPARKR_WORKER_PORT")) +inputCon <- socketConnection(port = port, blocking = TRUE, open = "rb") +outputCon <- socketConnection(port = port, blocking = TRUE, open = "wb") + +# read the index of the current partition inside the RDD +partition <- SparkR:::readInt(inputCon) + +deserializer <- SparkR:::readString(inputCon) +serializer <- SparkR:::readString(inputCon) + +# Include packages as required +packageNames <- unserialize(SparkR:::readRaw(inputCon)) +for (pkg in packageNames) { + suppressPackageStartupMessages(require(as.character(pkg), character.only=TRUE)) +} + +# read function dependencies +funcLen <- SparkR:::readInt(inputCon) +computeFunc <- unserialize(SparkR:::readRawLen(inputCon, funcLen)) +env <- environment(computeFunc) +parent.env(env) <- .GlobalEnv # Attach under global environment. + +# Read and set broadcast variables +numBroadcastVars <- SparkR:::readInt(inputCon) +if (numBroadcastVars > 0) { + for (bcast in seq(1:numBroadcastVars)) { + bcastId <- SparkR:::readInt(inputCon) + value <- unserialize(SparkR:::readRaw(inputCon)) + setBroadcastValue(bcastId, value) + } +} + +# If -1: read as normal RDD; if >= 0, treat as pairwise RDD and treat the int +# as number of partitions to create. +numPartitions <- SparkR:::readInt(inputCon) + +isEmpty <- SparkR:::readInt(inputCon) + +if (isEmpty != 0) { + + if (numPartitions == -1) { + if (deserializer == "byte") { + # Now read as many characters as described in funcLen + data <- SparkR:::readDeserialize(inputCon) + } else if (deserializer == "string") { + data <- as.list(readLines(inputCon)) + } else if (deserializer == "row") { + data <- SparkR:::readDeserializeRows(inputCon) + } + output <- computeFunc(partition, data) + if (serializer == "byte") { + SparkR:::writeRawSerialize(outputCon, output) + } else if (serializer == "row") { + SparkR:::writeRowSerialize(outputCon, output) + } else { + SparkR:::writeStrings(outputCon, output) + } + } else { + if (deserializer == "byte") { + # Now read as many characters as described in funcLen + data <- SparkR:::readDeserialize(inputCon) + } else if (deserializer == "string") { + data <- readLines(inputCon) + } else if (deserializer == "row") { + data <- SparkR:::readDeserializeRows(inputCon) + } + + res <- new.env() + + # Step 1: hash the data to an environment + hashTupleToEnvir <- function(tuple) { + # NOTE: execFunction is the hash function here + hashVal <- computeFunc(tuple[[1]]) + bucket <- as.character(hashVal %% numPartitions) + acc <- res[[bucket]] + # Create a new accumulator + if (is.null(acc)) { + acc <- SparkR:::initAccumulator() + } + SparkR:::addItemToAccumulator(acc, tuple) + res[[bucket]] <- acc + } + invisible(lapply(data, hashTupleToEnvir)) + + # Step 2: write out all of the environment as key-value pairs. + for (name in ls(res)) { + SparkR:::writeInt(outputCon, 2L) + SparkR:::writeInt(outputCon, as.integer(name)) + # Truncate the accumulator list to the number of elements we have + length(res[[name]]$data) <- res[[name]]$counter + SparkR:::writeRawSerialize(outputCon, res[[name]]$data) + } + } +} + +# End of output +if (serializer %in% c("byte", "row")) { + SparkR:::writeInt(outputCon, 0L) +} + +close(outputCon) +close(inputCon) diff --git a/R/pkg/src/Makefile b/R/pkg/src/Makefile new file mode 100644 index 0000000000000..a55a56fe80e10 --- /dev/null +++ b/R/pkg/src/Makefile @@ -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. +# + +all: sharelib + +sharelib: string_hash_code.c + R CMD SHLIB -o SparkR.so string_hash_code.c + +clean: + rm -f *.o + rm -f *.so + +.PHONY: all clean diff --git a/R/pkg/src/Makefile.win b/R/pkg/src/Makefile.win new file mode 100644 index 0000000000000..aa486d8228371 --- /dev/null +++ b/R/pkg/src/Makefile.win @@ -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. +# + +all: sharelib + +sharelib: string_hash_code.c + R CMD SHLIB -o SparkR.dll string_hash_code.c + +clean: + rm -f *.o + rm -f *.dll + +.PHONY: all clean diff --git a/R/pkg/src/string_hash_code.c b/R/pkg/src/string_hash_code.c new file mode 100644 index 0000000000000..e3274b9a0c547 --- /dev/null +++ b/R/pkg/src/string_hash_code.c @@ -0,0 +1,49 @@ +/* + 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. +*/ + +/* + * A C function for R extension which implements the Java String hash algorithm. + * Refer to http://en.wikipedia.org/wiki/Java_hashCode%28%29#The_java.lang.String_hash_function + * + */ + +#include +#include + +/* for compatibility with R before 3.1 */ +#ifndef IS_SCALAR +#define IS_SCALAR(x, type) (TYPEOF(x) == (type) && XLENGTH(x) == 1) +#endif + +SEXP stringHashCode(SEXP string) { + const char* str; + R_xlen_t len, i; + int hashCode = 0; + + if (!IS_SCALAR(string, STRSXP)) { + error("invalid input"); + } + + str = CHAR(asChar(string)); + len = XLENGTH(asChar(string)); + + for (i = 0; i < len; i++) { + hashCode = (hashCode << 5) - hashCode + *str++; + } + + return ScalarInteger(hashCode); +} diff --git a/R/pkg/tests/run-all.R b/R/pkg/tests/run-all.R new file mode 100644 index 0000000000000..4f8a1ed2d83ef --- /dev/null +++ b/R/pkg/tests/run-all.R @@ -0,0 +1,21 @@ +# +# 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. +# + +library(testthat) +library(SparkR) + +test_package("SparkR") diff --git a/R/run-tests.sh b/R/run-tests.sh new file mode 100755 index 0000000000000..e82ad0ba2cd06 --- /dev/null +++ b/R/run-tests.sh @@ -0,0 +1,39 @@ +#!/bin/bash + +# +# 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. +# + +FWDIR="$(cd `dirname $0`; pwd)" + +FAILED=0 +LOGFILE=$FWDIR/unit-tests.out +rm -f $LOGFILE + +SPARK_TESTING=1 $FWDIR/../bin/sparkR --driver-java-options "-Dlog4j.configuration=file:$FWDIR/log4j.properties" $FWDIR/pkg/tests/run-all.R 2>&1 | tee -a $LOGFILE +FAILED=$((PIPESTATUS[0]||$FAILED)) + +if [[ $FAILED != 0 ]]; then + cat $LOGFILE + echo -en "\033[31m" # Red + echo "Had test failures; see logs." + echo -en "\033[0m" # No color + exit -1 +else + echo -en "\033[32m" # Green + echo "Tests passed." + echo -en "\033[0m" # No color +fi diff --git a/bin/sparkR b/bin/sparkR new file mode 100755 index 0000000000000..8c918e2b09aef --- /dev/null +++ b/bin/sparkR @@ -0,0 +1,39 @@ +#!/bin/bash + +# +# 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. +# + +# Figure out where Spark is installed +export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" + +source "$SPARK_HOME"/bin/load-spark-env.sh + +function usage() { + if [ -n "$1" ]; then + echo $1 + fi + echo "Usage: ./bin/sparkR [options]" 1>&2 + "$SPARK_HOME"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit $2 +} +export -f usage + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + usage +fi + +exec "$SPARK_HOME"/bin/spark-submit sparkr-shell-main "$@" diff --git a/bin/sparkR.cmd b/bin/sparkR.cmd new file mode 100644 index 0000000000000..d7b60183ca8e0 --- /dev/null +++ b/bin/sparkR.cmd @@ -0,0 +1,23 @@ +@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 + +rem This is the entry point for running SparkR. To avoid polluting the +rem environment, it just launches a new cmd to do the real work. + +cmd /V /E /C %~dp0sparkR2.cmd %* diff --git a/bin/sparkR2.cmd b/bin/sparkR2.cmd new file mode 100644 index 0000000000000..e47f22c7300bb --- /dev/null +++ b/bin/sparkR2.cmd @@ -0,0 +1,26 @@ +@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 + +rem Figure out where the Spark framework is installed +set SPARK_HOME=%~dp0.. + +call %SPARK_HOME%\bin\load-spark-env.cmd + + +call %SPARK_HOME%\bin\spark-submit2.cmd sparkr-shell-main %* diff --git a/core/pom.xml b/core/pom.xml index 6cd1965ec37c2..e80829b7a7f3d 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -442,4 +442,55 @@ + + + Windows + + + Windows + + + + \ + .bat + + + + unix + + + unix + + + + / + .sh + + + + sparkr + + + + org.codehaus.mojo + exec-maven-plugin + 1.3.2 + + + sparkr-pkg + compile + + exec + + + + + ..${path.separator}R${path.separator}install-dev${script.extension} + + + + + + + diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala new file mode 100644 index 0000000000000..3a2c94bd9d875 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala @@ -0,0 +1,145 @@ +/* + * 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.api.r + +import java.io.{DataOutputStream, File, FileOutputStream, IOException} +import java.net.{InetSocketAddress, ServerSocket} +import java.util.concurrent.TimeUnit + +import io.netty.bootstrap.ServerBootstrap +import io.netty.channel.{ChannelFuture, ChannelInitializer, EventLoopGroup} +import io.netty.channel.nio.NioEventLoopGroup +import io.netty.channel.socket.SocketChannel +import io.netty.channel.socket.nio.NioServerSocketChannel +import io.netty.handler.codec.LengthFieldBasedFrameDecoder +import io.netty.handler.codec.bytes.{ByteArrayDecoder, ByteArrayEncoder} + +import org.apache.spark.Logging + +/** + * Netty-based backend server that is used to communicate between R and Java. + */ +private[spark] class RBackend { + + private[this] var channelFuture: ChannelFuture = null + private[this] var bootstrap: ServerBootstrap = null + private[this] var bossGroup: EventLoopGroup = null + + def init(): Int = { + bossGroup = new NioEventLoopGroup(2) + val workerGroup = bossGroup + val handler = new RBackendHandler(this) + + bootstrap = new ServerBootstrap() + .group(bossGroup, workerGroup) + .channel(classOf[NioServerSocketChannel]) + + bootstrap.childHandler(new ChannelInitializer[SocketChannel]() { + def initChannel(ch: SocketChannel): Unit = { + ch.pipeline() + .addLast("encoder", new ByteArrayEncoder()) + .addLast("frameDecoder", + // maxFrameLength = 2G + // lengthFieldOffset = 0 + // lengthFieldLength = 4 + // lengthAdjustment = 0 + // initialBytesToStrip = 4, i.e. strip out the length field itself + new LengthFieldBasedFrameDecoder(Integer.MAX_VALUE, 0, 4, 0, 4)) + .addLast("decoder", new ByteArrayDecoder()) + .addLast("handler", handler) + } + }) + + channelFuture = bootstrap.bind(new InetSocketAddress(0)) + channelFuture.syncUninterruptibly() + channelFuture.channel().localAddress().asInstanceOf[InetSocketAddress].getPort() + } + + def run(): Unit = { + channelFuture.channel.closeFuture().syncUninterruptibly() + } + + def close(): Unit = { + if (channelFuture != null) { + // close is a local operation and should finish within milliseconds; timeout just to be safe + channelFuture.channel().close().awaitUninterruptibly(10, TimeUnit.SECONDS) + channelFuture = null + } + if (bootstrap != null && bootstrap.group() != null) { + bootstrap.group().shutdownGracefully() + } + if (bootstrap != null && bootstrap.childGroup() != null) { + bootstrap.childGroup().shutdownGracefully() + } + bootstrap = null + } + +} + +private[spark] object RBackend extends Logging { + def main(args: Array[String]): Unit = { + if (args.length < 1) { + System.err.println("Usage: RBackend ") + System.exit(-1) + } + val sparkRBackend = new RBackend() + try { + // bind to random port + val boundPort = sparkRBackend.init() + val serverSocket = new ServerSocket(0, 1) + val listenPort = serverSocket.getLocalPort() + + // tell the R process via temporary file + val path = args(0) + val f = new File(path + ".tmp") + val dos = new DataOutputStream(new FileOutputStream(f)) + dos.writeInt(boundPort) + dos.writeInt(listenPort) + dos.close() + f.renameTo(new File(path)) + + // wait for the end of stdin, then exit + new Thread("wait for socket to close") { + setDaemon(true) + override def run(): Unit = { + // any un-catched exception will also shutdown JVM + val buf = new Array[Byte](1024) + // shutdown JVM if R does not connect back in 10 seconds + serverSocket.setSoTimeout(10000) + try { + val inSocket = serverSocket.accept() + serverSocket.close() + // wait for the end of socket, closed if R process die + inSocket.getInputStream().read(buf) + } finally { + sparkRBackend.close() + System.exit(0) + } + } + }.start() + + sparkRBackend.run() + } catch { + case e: IOException => + logError("Server shutting down: failed with exception ", e) + sparkRBackend.close() + System.exit(1) + } + System.exit(0) + } +} diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala new file mode 100644 index 0000000000000..0075d963711f1 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -0,0 +1,223 @@ +/* + * 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.api.r + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import scala.collection.mutable.HashMap + +import io.netty.channel.ChannelHandler.Sharable +import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} + +import org.apache.spark.Logging +import org.apache.spark.api.r.SerDe._ + +/** + * Handler for RBackend + * TODO: This is marked as sharable to get a handle to RBackend. Is it safe to re-use + * this across connections ? + */ +@Sharable +private[r] class RBackendHandler(server: RBackend) + extends SimpleChannelInboundHandler[Array[Byte]] with Logging { + + override def channelRead0(ctx: ChannelHandlerContext, msg: Array[Byte]): Unit = { + val bis = new ByteArrayInputStream(msg) + val dis = new DataInputStream(bis) + + val bos = new ByteArrayOutputStream() + val dos = new DataOutputStream(bos) + + // First bit is isStatic + val isStatic = readBoolean(dis) + val objId = readString(dis) + val methodName = readString(dis) + val numArgs = readInt(dis) + + if (objId == "SparkRHandler") { + methodName match { + case "stopBackend" => + writeInt(dos, 0) + writeType(dos, "void") + server.close() + case "rm" => + try { + val t = readObjectType(dis) + assert(t == 'c') + val objToRemove = readString(dis) + JVMObjectTracker.remove(objToRemove) + writeInt(dos, 0) + writeObject(dos, null) + } catch { + case e: Exception => + logError(s"Removing $objId failed", e) + writeInt(dos, -1) + } + case _ => dos.writeInt(-1) + } + } else { + handleMethodCall(isStatic, objId, methodName, numArgs, dis, dos) + } + + val reply = bos.toByteArray + ctx.write(reply) + } + + override def channelReadComplete(ctx: ChannelHandlerContext): Unit = { + ctx.flush() + } + + override def exceptionCaught(ctx: ChannelHandlerContext, cause: Throwable): Unit = { + // Close the connection when an exception is raised. + cause.printStackTrace() + ctx.close() + } + + def handleMethodCall( + isStatic: Boolean, + objId: String, + methodName: String, + numArgs: Int, + dis: DataInputStream, + dos: DataOutputStream): Unit = { + var obj: Object = null + try { + val cls = if (isStatic) { + Class.forName(objId) + } else { + JVMObjectTracker.get(objId) match { + case None => throw new IllegalArgumentException("Object not found " + objId) + case Some(o) => + obj = o + o.getClass + } + } + + val args = readArgs(numArgs, dis) + + val methods = cls.getMethods + val selectedMethods = methods.filter(m => m.getName == methodName) + if (selectedMethods.length > 0) { + val methods = selectedMethods.filter { x => + matchMethod(numArgs, args, x.getParameterTypes) + } + if (methods.isEmpty) { + logWarning(s"cannot find matching method ${cls}.$methodName. " + + s"Candidates are:") + selectedMethods.foreach { method => + logWarning(s"$methodName(${method.getParameterTypes.mkString(",")})") + } + throw new Exception(s"No matched method found for $cls.$methodName") + } + val ret = methods.head.invoke(obj, args:_*) + + // Write status bit + writeInt(dos, 0) + writeObject(dos, ret.asInstanceOf[AnyRef]) + } else if (methodName == "") { + // methodName should be "" for constructor + val ctor = cls.getConstructors.filter { x => + matchMethod(numArgs, args, x.getParameterTypes) + }.head + + val obj = ctor.newInstance(args:_*) + + writeInt(dos, 0) + writeObject(dos, obj.asInstanceOf[AnyRef]) + } else { + throw new IllegalArgumentException("invalid method " + methodName + " for object " + objId) + } + } catch { + case e: Exception => + logError(s"$methodName on $objId failed", e) + writeInt(dos, -1) + } + } + + // Read a number of arguments from the data input stream + def readArgs(numArgs: Int, dis: DataInputStream): Array[java.lang.Object] = { + (0 until numArgs).map { arg => + readObject(dis) + }.toArray + } + + // Checks if the arguments passed in args matches the parameter types. + // NOTE: Currently we do exact match. We may add type conversions later. + def matchMethod( + numArgs: Int, + args: Array[java.lang.Object], + parameterTypes: Array[Class[_]]): Boolean = { + if (parameterTypes.length != numArgs) { + return false + } + + for (i <- 0 to numArgs - 1) { + val parameterType = parameterTypes(i) + var parameterWrapperType = parameterType + + // Convert native parameters to Object types as args is Array[Object] here + if (parameterType.isPrimitive) { + parameterWrapperType = parameterType match { + case java.lang.Integer.TYPE => classOf[java.lang.Integer] + case java.lang.Double.TYPE => classOf[java.lang.Double] + case java.lang.Boolean.TYPE => classOf[java.lang.Boolean] + case _ => parameterType + } + } + if (!parameterWrapperType.isInstance(args(i))) { + return false + } + } + true + } +} + +/** + * Helper singleton that tracks JVM objects returned to R. + * This is useful for referencing these objects in RPC calls. + */ +private[r] object JVMObjectTracker { + + // TODO: This map should be thread-safe if we want to support multiple + // connections at the same time + private[this] val objMap = new HashMap[String, Object] + + // TODO: We support only one connection now, so an integer is fine. + // Investigate using use atomic integer in the future. + private[this] var objCounter: Int = 0 + + def getObject(id: String): Object = { + objMap(id) + } + + def get(id: String): Option[Object] = { + objMap.get(id) + } + + def put(obj: Object): String = { + val objId = objCounter.toString + objCounter = objCounter + 1 + objMap.put(objId, obj) + objId + } + + def remove(id: String): Option[Object] = { + objMap.remove(id) + } + +} diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala new file mode 100644 index 0000000000000..5fa4d483b8342 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -0,0 +1,450 @@ +/* + * 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.api.r + +import java.io._ +import java.net.ServerSocket +import java.util.{Map => JMap} + +import scala.collection.JavaConversions._ +import scala.io.Source +import scala.reflect.ClassTag +import scala.util.Try + +import org.apache.spark._ +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( + parent: RDD[T], + numPartitions: Int, + func: Array[Byte], + deserializer: String, + serializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Broadcast[Object]]) + extends RDD[U](parent) with Logging { + override def getPartitions: Array[Partition] = parent.partitions + + override def compute(partition: Partition, context: TaskContext): Iterator[U] = { + + // The parent may be also an RRDD, so we should launch it first. + val parentIterator = firstParent[T].iterator(partition, context) + + // we expect two connections + val serverSocket = new ServerSocket(0, 2) + val listenPort = serverSocket.getLocalPort() + + // The stdout/stderr is shared by multiple tasks, because we use one daemon + // to launch child process as worker. + val errThread = RRDD.createRWorker(rLibDir, listenPort) + + // We use two sockets to separate input and output, then it's easy to manage + // the lifecycle of them to avoid deadlock. + // TODO: optimize it to use one socket + + // the socket used to send out the input of task + serverSocket.setSoTimeout(10000) + val inSocket = serverSocket.accept() + startStdinThread(inSocket.getOutputStream(), parentIterator, partition.index) + + // the socket used to receive the output of task + val outSocket = serverSocket.accept() + val inputStream = new BufferedInputStream(outSocket.getInputStream) + val dataStream = openDataStream(inputStream) + serverSocket.close() + + try { + + return new Iterator[U] { + def next(): U = { + val obj = _nextObj + if (hasNext) { + _nextObj = read() + } + obj + } + + var _nextObj = read() + + def hasNext(): Boolean = { + val hasMore = (_nextObj != null) + if (!hasMore) { + dataStream.close() + } + hasMore + } + } + } catch { + case e: Exception => + throw new SparkException("R computation failed with\n " + errThread.getLines()) + } + } + + /** + * Start a thread to write RDD data to the R process. + */ + private def startStdinThread[T]( + output: OutputStream, + iter: Iterator[T], + partition: Int): Unit = { + + val env = SparkEnv.get + val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt + val stream = new BufferedOutputStream(output, bufferSize) + + new Thread("writer for R") { + override def run(): Unit = { + try { + SparkEnv.set(env) + val dataOut = new DataOutputStream(stream) + dataOut.writeInt(partition) + + SerDe.writeString(dataOut, deserializer) + SerDe.writeString(dataOut, serializer) + + dataOut.writeInt(packageNames.length) + dataOut.write(packageNames) + + dataOut.writeInt(func.length) + dataOut.write(func) + + dataOut.writeInt(broadcastVars.length) + broadcastVars.foreach { broadcast => + // TODO(shivaram): Read a Long in R to avoid this cast + dataOut.writeInt(broadcast.id.toInt) + // TODO: Pass a byte array from R to avoid this cast ? + val broadcastByteArr = broadcast.value.asInstanceOf[Array[Byte]] + dataOut.writeInt(broadcastByteArr.length) + dataOut.write(broadcastByteArr) + } + + dataOut.writeInt(numPartitions) + + if (!iter.hasNext) { + dataOut.writeInt(0) + } else { + dataOut.writeInt(1) + } + + val printOut = new PrintStream(stream) + + def writeElem(elem: Any): Unit = { + if (deserializer == SerializationFormats.BYTE) { + val elemArr = elem.asInstanceOf[Array[Byte]] + dataOut.writeInt(elemArr.length) + dataOut.write(elemArr) + } else if (deserializer == SerializationFormats.ROW) { + dataOut.write(elem.asInstanceOf[Array[Byte]]) + } else if (deserializer == SerializationFormats.STRING) { + printOut.println(elem) + } + } + + for (elem <- iter) { + elem match { + case (key, value) => + writeElem(key) + writeElem(value) + case _ => + writeElem(elem) + } + } + stream.flush() + } catch { + // TODO: We should propogate this error to the task thread + case e: Exception => + logError("R Writer thread got an exception", e) + } finally { + Try(output.close()) + } + } + }.start() + } + + protected def openDataStream(input: InputStream): Closeable + + protected def read(): U +} + +/** + * Form an RDD[(Int, Array[Byte])] from key-value pairs returned from R. + * This is used by SparkR's shuffle operations. + */ +private class PairwiseRRDD[T: ClassTag]( + parent: RDD[T], + numPartitions: Int, + hashFunc: Array[Byte], + deserializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Object]) + extends BaseRRDD[T, (Int, Array[Byte])]( + parent, numPartitions, hashFunc, deserializer, + SerializationFormats.BYTE, packageNames, rLibDir, + broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { + + private var dataStream: DataInputStream = _ + + override protected def openDataStream(input: InputStream): Closeable = { + dataStream = new DataInputStream(input) + dataStream + } + + override protected def read(): (Int, Array[Byte]) = { + try { + val length = dataStream.readInt() + + length match { + case length if length == 2 => + val hashedKey = dataStream.readInt() + val contentPairsLength = dataStream.readInt() + val contentPairs = new Array[Byte](contentPairsLength) + dataStream.readFully(contentPairs) + (hashedKey, contentPairs) + case _ => null // End of input + } + } catch { + case eof: EOFException => { + throw new SparkException("R worker exited unexpectedly (crashed)", eof) + } + } + } + + lazy val asJavaPairRDD : JavaPairRDD[Int, Array[Byte]] = JavaPairRDD.fromRDD(this) +} + +/** + * An RDD that stores serialized R objects as Array[Byte]. + */ +private class RRDD[T: ClassTag]( + parent: RDD[T], + func: Array[Byte], + deserializer: String, + serializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Object]) + extends BaseRRDD[T, Array[Byte]]( + parent, -1, func, deserializer, serializer, packageNames, rLibDir, + broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { + + private var dataStream: DataInputStream = _ + + override protected def openDataStream(input: InputStream): Closeable = { + dataStream = new DataInputStream(input) + dataStream + } + + override protected def read(): Array[Byte] = { + try { + val length = dataStream.readInt() + + length match { + case length if length > 0 => + val obj = new Array[Byte](length) + dataStream.readFully(obj, 0, length) + obj + case _ => null + } + } catch { + case eof: EOFException => { + throw new SparkException("R worker exited unexpectedly (crashed)", eof) + } + } + } + + lazy val asJavaRDD : JavaRDD[Array[Byte]] = JavaRDD.fromRDD(this) +} + +/** + * An RDD that stores R objects as Array[String]. + */ +private class StringRRDD[T: ClassTag]( + parent: RDD[T], + func: Array[Byte], + deserializer: String, + packageNames: Array[Byte], + rLibDir: String, + broadcastVars: Array[Object]) + extends BaseRRDD[T, String]( + parent, -1, func, deserializer, SerializationFormats.STRING, packageNames, rLibDir, + broadcastVars.map(x => x.asInstanceOf[Broadcast[Object]])) { + + private var dataStream: BufferedReader = _ + + override protected def openDataStream(input: InputStream): Closeable = { + dataStream = new BufferedReader(new InputStreamReader(input)) + dataStream + } + + override protected def read(): String = { + try { + dataStream.readLine() + } catch { + case e: IOException => { + throw new SparkException("R worker exited unexpectedly (crashed)", e) + } + } + } + + lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this) +} + +private[r] class BufferedStreamThread( + in: InputStream, + name: String, + errBufferSize: Int) extends Thread(name) with Logging { + val lines = new Array[String](errBufferSize) + var lineIdx = 0 + override def run() { + for (line <- Source.fromInputStream(in).getLines) { + synchronized { + lines(lineIdx) = line + lineIdx = (lineIdx + 1) % errBufferSize + } + logInfo(line) + } + } + + def getLines(): String = synchronized { + (0 until errBufferSize).filter { x => + lines((x + lineIdx) % errBufferSize) != null + }.map { x => + lines((x + lineIdx) % errBufferSize) + }.mkString("\n") + } +} + +private[r] object RRDD { + // Because forking processes from Java is expensive, we prefer to launch + // a single R daemon (daemon.R) and tell it to fork new workers for our tasks. + // This daemon currently only works on UNIX-based systems now, so we should + // also fall back to launching workers (worker.R) directly. + private[this] var errThread: BufferedStreamThread = _ + private[this] var daemonChannel: DataOutputStream = _ + + def createSparkContext( + master: String, + appName: String, + sparkHome: String, + jars: Array[String], + sparkEnvirMap: JMap[Object, Object], + sparkExecutorEnvMap: JMap[Object, Object]): JavaSparkContext = { + + val sparkConf = new SparkConf().setAppName(appName) + .setSparkHome(sparkHome) + .setJars(jars) + + // Override `master` if we have a user-specified value + if (master != "") { + sparkConf.setMaster(master) + } else { + // If conf has no master set it to "local" to maintain + // backwards compatibility + sparkConf.setIfMissing("spark.master", "local") + } + + for ((name, value) <- sparkEnvirMap) { + sparkConf.set(name.asInstanceOf[String], value.asInstanceOf[String]) + } + for ((name, value) <- sparkExecutorEnvMap) { + sparkConf.setExecutorEnv(name.asInstanceOf[String], value.asInstanceOf[String]) + } + + new JavaSparkContext(sparkConf) + } + + /** + * Start a thread to print the process's stderr to ours + */ + private def startStdoutThread(proc: Process): BufferedStreamThread = { + val BUFFER_SIZE = 100 + val thread = new BufferedStreamThread(proc.getInputStream, "stdout reader for R", BUFFER_SIZE) + thread.setDaemon(true) + thread.start() + thread + } + + private def createRProcess(rLibDir: String, port: Int, script: String): BufferedStreamThread = { + val rCommand = "Rscript" + val rOptions = "--vanilla" + val rExecScript = rLibDir + "/SparkR/worker/" + script + val pb = new ProcessBuilder(List(rCommand, rOptions, rExecScript)) + // Unset the R_TESTS environment variable for workers. + // This is set by R CMD check as startup.Rs + // (http://svn.r-project.org/R/trunk/src/library/tools/R/testing.R) + // and confuses worker script which tries to load a non-existent file + pb.environment().put("R_TESTS", "") + pb.environment().put("SPARKR_RLIBDIR", rLibDir) + pb.environment().put("SPARKR_WORKER_PORT", port.toString) + pb.redirectErrorStream(true) // redirect stderr into stdout + val proc = pb.start() + val errThread = startStdoutThread(proc) + errThread + } + + /** + * ProcessBuilder used to launch worker R processes. + */ + def createRWorker(rLibDir: String, port: Int): BufferedStreamThread = { + val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true) + if (!Utils.isWindows && useDaemon) { + synchronized { + if (daemonChannel == null) { + // we expect one connections + val serverSocket = new ServerSocket(0, 1) + val daemonPort = serverSocket.getLocalPort + errThread = createRProcess(rLibDir, daemonPort, "daemon.R") + // the socket used to send out the input of task + serverSocket.setSoTimeout(10000) + val sock = serverSocket.accept() + daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) + serverSocket.close() + } + try { + daemonChannel.writeInt(port) + daemonChannel.flush() + } catch { + case e: IOException => + // daemon process died + daemonChannel.close() + daemonChannel = null + errThread = null + // fail the current task, retry by scheduler + throw e + } + errThread + } + } else { + createRProcess(rLibDir, port, "worker.R") + } + } + + /** + * Create an RRDD given a sequence of byte arrays. Used to create RRDD when `parallelize` is + * called from R. + */ + def createRDDFromArray(jsc: JavaSparkContext, arr: Array[Array[Byte]]): JavaRDD[Array[Byte]] = { + JavaRDD.fromRDD(jsc.sc.parallelize(arr, arr.length)) + } + +} diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala new file mode 100644 index 0000000000000..ccb2a371f4e48 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -0,0 +1,340 @@ +/* + * 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.api.r + +import java.io.{DataInputStream, DataOutputStream} +import java.sql.{Date, Time} + +import scala.collection.JavaConversions._ + +/** + * Utility functions to serialize, deserialize objects to / from R + */ +private[spark] object SerDe { + + // Type mapping from R to Java + // + // NULL -> void + // integer -> Int + // character -> String + // logical -> Boolean + // double, numeric -> Double + // raw -> Array[Byte] + // Date -> Date + // POSIXlt/POSIXct -> Time + // + // list[T] -> Array[T], where T is one of above mentioned types + // environment -> Map[String, T], where T is a native type + // jobj -> Object, where jobj is an object created in the backend + + def readObjectType(dis: DataInputStream): Char = { + dis.readByte().toChar + } + + def readObject(dis: DataInputStream): Object = { + val dataType = readObjectType(dis) + readTypedObject(dis, dataType) + } + + def readTypedObject( + dis: DataInputStream, + dataType: Char): Object = { + dataType match { + case 'n' => null + case 'i' => new java.lang.Integer(readInt(dis)) + case 'd' => new java.lang.Double(readDouble(dis)) + case 'b' => new java.lang.Boolean(readBoolean(dis)) + case 'c' => readString(dis) + case 'e' => readMap(dis) + case 'r' => readBytes(dis) + case 'l' => readList(dis) + case 'D' => readDate(dis) + case 't' => readTime(dis) + case 'j' => JVMObjectTracker.getObject(readString(dis)) + case _ => throw new IllegalArgumentException(s"Invalid type $dataType") + } + } + + def readBytes(in: DataInputStream): Array[Byte] = { + val len = readInt(in) + val out = new Array[Byte](len) + val bytesRead = in.readFully(out) + out + } + + def readInt(in: DataInputStream): Int = { + in.readInt() + } + + def readDouble(in: DataInputStream): Double = { + in.readDouble() + } + + def readString(in: DataInputStream): String = { + val len = in.readInt() + val asciiBytes = new Array[Byte](len) + in.readFully(asciiBytes) + assert(asciiBytes(len - 1) == 0) + val str = new String(asciiBytes.dropRight(1).map(_.toChar)) + str + } + + def readBoolean(in: DataInputStream): Boolean = { + val intVal = in.readInt() + if (intVal == 0) false else true + } + + def readDate(in: DataInputStream): Date = { + Date.valueOf(readString(in)) + } + + def readTime(in: DataInputStream): Time = { + val t = in.readDouble() + new Time((t * 1000L).toLong) + } + + def readBytesArr(in: DataInputStream): Array[Array[Byte]] = { + val len = readInt(in) + (0 until len).map(_ => readBytes(in)).toArray + } + + def readIntArr(in: DataInputStream): Array[Int] = { + val len = readInt(in) + (0 until len).map(_ => readInt(in)).toArray + } + + def readDoubleArr(in: DataInputStream): Array[Double] = { + val len = readInt(in) + (0 until len).map(_ => readDouble(in)).toArray + } + + def readBooleanArr(in: DataInputStream): Array[Boolean] = { + val len = readInt(in) + (0 until len).map(_ => readBoolean(in)).toArray + } + + def readStringArr(in: DataInputStream): Array[String] = { + val len = readInt(in) + (0 until len).map(_ => readString(in)).toArray + } + + def readList(dis: DataInputStream): Array[_] = { + val arrType = readObjectType(dis) + arrType match { + case 'i' => readIntArr(dis) + case 'c' => readStringArr(dis) + case 'd' => readDoubleArr(dis) + case 'b' => readBooleanArr(dis) + case 'j' => readStringArr(dis).map(x => JVMObjectTracker.getObject(x)) + case 'r' => readBytesArr(dis) + case _ => throw new IllegalArgumentException(s"Invalid array type $arrType") + } + } + + def readMap(in: DataInputStream): java.util.Map[Object, Object] = { + val len = readInt(in) + if (len > 0) { + val keysType = readObjectType(in) + val keysLen = readInt(in) + val keys = (0 until keysLen).map(_ => readTypedObject(in, keysType)) + + val valuesType = readObjectType(in) + val valuesLen = readInt(in) + val values = (0 until valuesLen).map(_ => readTypedObject(in, valuesType)) + mapAsJavaMap(keys.zip(values).toMap) + } else { + new java.util.HashMap[Object, Object]() + } + } + + // Methods to write out data from Java to R + // + // Type mapping from Java to R + // + // void -> NULL + // Int -> integer + // String -> character + // Boolean -> logical + // Double -> double + // Long -> double + // Array[Byte] -> raw + // Date -> Date + // Time -> POSIXct + // + // Array[T] -> list() + // Object -> jobj + + def writeType(dos: DataOutputStream, typeStr: String): Unit = { + typeStr match { + case "void" => dos.writeByte('n') + case "character" => dos.writeByte('c') + case "double" => dos.writeByte('d') + case "integer" => dos.writeByte('i') + case "logical" => dos.writeByte('b') + case "date" => dos.writeByte('D') + case "time" => dos.writeByte('t') + case "raw" => dos.writeByte('r') + case "list" => dos.writeByte('l') + case "jobj" => dos.writeByte('j') + case _ => throw new IllegalArgumentException(s"Invalid type $typeStr") + } + } + + def writeObject(dos: DataOutputStream, value: Object): Unit = { + if (value == null) { + writeType(dos, "void") + } else { + value.getClass.getName match { + case "java.lang.String" => + writeType(dos, "character") + writeString(dos, value.asInstanceOf[String]) + case "long" | "java.lang.Long" => + writeType(dos, "double") + writeDouble(dos, value.asInstanceOf[Long].toDouble) + case "double" | "java.lang.Double" => + writeType(dos, "double") + writeDouble(dos, value.asInstanceOf[Double]) + case "int" | "java.lang.Integer" => + writeType(dos, "integer") + writeInt(dos, value.asInstanceOf[Int]) + case "boolean" | "java.lang.Boolean" => + writeType(dos, "logical") + writeBoolean(dos, value.asInstanceOf[Boolean]) + case "java.sql.Date" => + writeType(dos, "date") + writeDate(dos, value.asInstanceOf[Date]) + case "java.sql.Time" => + writeType(dos, "time") + writeTime(dos, value.asInstanceOf[Time]) + case "[B" => + writeType(dos, "raw") + writeBytes(dos, value.asInstanceOf[Array[Byte]]) + // TODO: Types not handled right now include + // byte, char, short, float + + // Handle arrays + case "[Ljava.lang.String;" => + writeType(dos, "list") + writeStringArr(dos, value.asInstanceOf[Array[String]]) + case "[I" => + writeType(dos, "list") + writeIntArr(dos, value.asInstanceOf[Array[Int]]) + case "[J" => + writeType(dos, "list") + writeDoubleArr(dos, value.asInstanceOf[Array[Long]].map(_.toDouble)) + case "[D" => + writeType(dos, "list") + writeDoubleArr(dos, value.asInstanceOf[Array[Double]]) + case "[Z" => + writeType(dos, "list") + writeBooleanArr(dos, value.asInstanceOf[Array[Boolean]]) + case "[[B" => + writeType(dos, "list") + writeBytesArr(dos, value.asInstanceOf[Array[Array[Byte]]]) + case otherName => + // Handle array of objects + if (otherName.startsWith("[L")) { + val objArr = value.asInstanceOf[Array[Object]] + writeType(dos, "list") + writeType(dos, "jobj") + dos.writeInt(objArr.length) + objArr.foreach(o => writeJObj(dos, o)) + } else { + writeType(dos, "jobj") + writeJObj(dos, value) + } + } + } + } + + def writeInt(out: DataOutputStream, value: Int): Unit = { + out.writeInt(value) + } + + def writeDouble(out: DataOutputStream, value: Double): Unit = { + out.writeDouble(value) + } + + def writeBoolean(out: DataOutputStream, value: Boolean): Unit = { + val intValue = if (value) 1 else 0 + out.writeInt(intValue) + } + + def writeDate(out: DataOutputStream, value: Date): Unit = { + writeString(out, value.toString) + } + + def writeTime(out: DataOutputStream, value: Time): Unit = { + out.writeDouble(value.getTime.toDouble / 1000.0) + } + + + // NOTE: Only works for ASCII right now + def writeString(out: DataOutputStream, value: String): Unit = { + val len = value.length + out.writeInt(len + 1) // For the \0 + out.writeBytes(value) + out.writeByte(0) + } + + def writeBytes(out: DataOutputStream, value: Array[Byte]): Unit = { + out.writeInt(value.length) + out.write(value) + } + + def writeJObj(out: DataOutputStream, value: Object): Unit = { + val objId = JVMObjectTracker.put(value) + writeString(out, objId) + } + + def writeIntArr(out: DataOutputStream, value: Array[Int]): Unit = { + writeType(out, "integer") + out.writeInt(value.length) + value.foreach(v => out.writeInt(v)) + } + + def writeDoubleArr(out: DataOutputStream, value: Array[Double]): Unit = { + writeType(out, "double") + out.writeInt(value.length) + value.foreach(v => out.writeDouble(v)) + } + + def writeBooleanArr(out: DataOutputStream, value: Array[Boolean]): Unit = { + writeType(out, "logical") + out.writeInt(value.length) + value.foreach(v => writeBoolean(out, v)) + } + + def writeStringArr(out: DataOutputStream, value: Array[String]): Unit = { + writeType(out, "character") + out.writeInt(value.length) + value.foreach(v => writeString(out, v)) + } + + def writeBytesArr(out: DataOutputStream, value: Array[Array[Byte]]): Unit = { + writeType(out, "raw") + out.writeInt(value.length) + value.foreach(v => writeBytes(out, v)) + } +} + +private[r] object SerializationFormats { + val BYTE = "byte" + val STRING = "string" + val ROW = "row" +} diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala new file mode 100644 index 0000000000000..e99779f299785 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import java.io._ +import java.util.concurrent.{Semaphore, TimeUnit} + +import scala.collection.JavaConversions._ + +import org.apache.hadoop.fs.Path + +import org.apache.spark.api.r.RBackend +import org.apache.spark.util.RedirectThread + +/** + * Main class used to launch SparkR applications using spark-submit. It executes R as a + * subprocess and then has it connect back to the JVM to access system properties etc. + */ +object RRunner { + def main(args: Array[String]): Unit = { + val rFile = PythonRunner.formatPath(args(0)) + + val otherArgs = args.slice(1, args.length) + + // Time to wait for SparkR backend to initialize in seconds + val backendTimeout = sys.env.getOrElse("SPARKR_BACKEND_TIMEOUT", "120").toInt + val rCommand = "Rscript" + + // Check if the file path exists. + // If not, change directory to current working directory for YARN cluster mode + val rF = new File(rFile) + val rFileNormalized = if (!rF.exists()) { + new Path(rFile).getName + } else { + rFile + } + + // Launch a SparkR backend server for the R process to connect to; this will let it see our + // Java system properties etc. + val sparkRBackend = new RBackend() + @volatile var sparkRBackendPort = 0 + val initialized = new Semaphore(0) + val sparkRBackendThread = new Thread("SparkR backend") { + override def run() { + sparkRBackendPort = sparkRBackend.init() + initialized.release() + sparkRBackend.run() + } + } + + sparkRBackendThread.start() + // Wait for RBackend initialization to finish + if (initialized.tryAcquire(backendTimeout, TimeUnit.SECONDS)) { + // Launch R + val returnCode = try { + val builder = new ProcessBuilder(Seq(rCommand, rFileNormalized) ++ otherArgs) + val env = builder.environment() + env.put("EXISTING_SPARKR_BACKEND_PORT", sparkRBackendPort.toString) + val sparkHome = System.getenv("SPARK_HOME") + env.put("R_PROFILE_USER", + Seq(sparkHome, "R", "lib", "SparkR", "profile", "general.R").mkString(File.separator)) + builder.redirectErrorStream(true) // Ugly but needed for stdout and stderr to synchronize + val process = builder.start() + + new RedirectThread(process.getInputStream, System.out, "redirect R output").start() + + process.waitFor() + } finally { + sparkRBackend.close() + } + System.exit(returnCode) + } else { + System.err.println("SparkR backend did not initialize in " + backendTimeout + " seconds") + System.exit(-1) + } + } +} 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 660307d19eab4..60bc243ebf40a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -77,6 +77,7 @@ object SparkSubmit { // Special primary resource names that represent shells rather than application jars. private val SPARK_SHELL = "spark-shell" private val PYSPARK_SHELL = "pyspark-shell" + private val SPARKR_SHELL = "sparkr-shell" private val CLASS_NOT_FOUND_EXIT_STATUS = 101 @@ -284,6 +285,13 @@ object SparkSubmit { } } + // Require all R files to be local + if (args.isR && !isYarnCluster) { + if (Utils.nonLocalPaths(args.primaryResource).nonEmpty) { + printErrorAndExit(s"Only local R files are supported: $args.primaryResource") + } + } + // The following modes are not supported or applicable (clusterManager, deployMode) match { case (MESOS, CLUSTER) => @@ -291,6 +299,9 @@ object SparkSubmit { case (STANDALONE, CLUSTER) if args.isPython => printErrorAndExit("Cluster deploy mode is currently not supported for python " + "applications on standalone clusters.") + case (STANDALONE, CLUSTER) if args.isR => + printErrorAndExit("Cluster deploy mode is currently not supported for R " + + "applications on standalone clusters.") case (_, CLUSTER) if isShell(args.primaryResource) => printErrorAndExit("Cluster deploy mode is not applicable to Spark shells.") case (_, CLUSTER) if isSqlShell(args.mainClass) => @@ -317,11 +328,32 @@ object SparkSubmit { } } - // In yarn-cluster mode for a python app, add primary resource and pyFiles to files - // that can be distributed with the job - if (args.isPython && isYarnCluster) { - args.files = mergeFileLists(args.files, args.primaryResource) - args.files = mergeFileLists(args.files, args.pyFiles) + // If we're running a R app, set the main class to our specific R runner + if (args.isR && deployMode == CLIENT) { + if (args.primaryResource == SPARKR_SHELL) { + args.mainClass = "org.apache.spark.api.r.RBackend" + } else { + // If a R file is provided, add it to the child arguments and list of files to deploy. + // Usage: RRunner
[app arguments] + args.mainClass = "org.apache.spark.deploy.RRunner" + args.childArgs = ArrayBuffer(args.primaryResource) ++ args.childArgs + args.files = mergeFileLists(args.files, args.primaryResource) + } + } + + if (isYarnCluster) { + // In yarn-cluster mode for a python app, add primary resource and pyFiles to files + // that can be distributed with the job + if (args.isPython) { + args.files = mergeFileLists(args.files, args.primaryResource) + args.files = mergeFileLists(args.files, args.pyFiles) + } + + // In yarn-cluster mode for a R app, add primary resource to files + // that can be distributed with the job + if (args.isR) { + args.files = mergeFileLists(args.files, args.primaryResource) + } } // Special flag to avoid deprecation warnings at the client @@ -405,8 +437,8 @@ object SparkSubmit { // Add the application jar automatically so the user doesn't have to call sc.addJar // For YARN cluster mode, the jar is already distributed on each node as "app.jar" - // For python files, the primary resource is already distributed as a regular file - if (!isYarnCluster && !args.isPython) { + // For python and R files, the primary resource is already distributed as a regular file + if (!isYarnCluster && !args.isPython && !args.isR) { var jars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty) if (isUserJar(args.primaryResource)) { jars = jars ++ Seq(args.primaryResource) @@ -447,6 +479,10 @@ object SparkSubmit { childArgs += ("--py-files", pyFilesNames) } childArgs += ("--class", "org.apache.spark.deploy.PythonRunner") + } else if (args.isR) { + val mainFile = new Path(args.primaryResource).getName + childArgs += ("--primary-r-file", mainFile) + childArgs += ("--class", "org.apache.spark.deploy.RRunner") } else { if (args.primaryResource != SPARK_INTERNAL) { childArgs += ("--jar", args.primaryResource) @@ -591,15 +627,15 @@ object SparkSubmit { /** * Return whether the given primary resource represents a user jar. */ - private def isUserJar(primaryResource: String): Boolean = { - !isShell(primaryResource) && !isPython(primaryResource) && !isInternal(primaryResource) + private[deploy] def isUserJar(res: String): Boolean = { + !isShell(res) && !isPython(res) && !isInternal(res) && !isR(res) } /** * Return whether the given primary resource represents a shell. */ - private[deploy] def isShell(primaryResource: String): Boolean = { - primaryResource == SPARK_SHELL || primaryResource == PYSPARK_SHELL + private[deploy] def isShell(res: String): Boolean = { + (res == SPARK_SHELL || res == PYSPARK_SHELL || res == SPARKR_SHELL) } /** @@ -619,12 +655,19 @@ object SparkSubmit { /** * Return whether the given primary resource requires running python. */ - private[deploy] def isPython(primaryResource: String): Boolean = { - primaryResource.endsWith(".py") || primaryResource == PYSPARK_SHELL + private[deploy] def isPython(res: String): Boolean = { + res != null && res.endsWith(".py") || res == PYSPARK_SHELL + } + + /** + * Return whether the given primary resource requires running R. + */ + private[deploy] def isR(res: String): Boolean = { + res != null && res.endsWith(".R") || res == SPARKR_SHELL } - private[deploy] def isInternal(primaryResource: String): Boolean = { - primaryResource == SPARK_INTERNAL + private[deploy] def isInternal(res: String): Boolean = { + res == SPARK_INTERNAL } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 6eb73c43470a5..03ecf3fd99ec5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -59,6 +59,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S var verbose: Boolean = false var isPython: Boolean = false var pyFiles: String = null + var isR: Boolean = false var action: SparkSubmitAction = null val sparkProperties: HashMap[String, String] = new HashMap[String, String]() var proxyUser: String = null @@ -158,7 +159,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .getOrElse(sparkProperties.get("spark.executor.instances").orNull) // Try to set main class from JAR if no --class argument is given - if (mainClass == null && !isPython && primaryResource != null) { + if (mainClass == null && !isPython && !isR && primaryResource != null) { val uri = new URI(primaryResource) val uriScheme = uri.getScheme() @@ -211,9 +212,9 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S printUsageAndExit(-1) } if (primaryResource == null) { - SparkSubmit.printErrorAndExit("Must specify a primary resource (JAR or Python file)") + SparkSubmit.printErrorAndExit("Must specify a primary resource (JAR or Python or R file)") } - if (mainClass == null && !isPython) { + if (mainClass == null && SparkSubmit.isUserJar(primaryResource)) { SparkSubmit.printErrorAndExit("No main class set in JAR; please specify one with --class") } if (pyFiles != null && !isPython) { @@ -414,6 +415,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S opt } isPython = SparkSubmit.isPython(opt) + isR = SparkSubmit.isR(opt) false } diff --git a/dev/run-tests b/dev/run-tests index 561d7fc9e7b1f..1b6cf78b5da01 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -236,3 +236,18 @@ echo "=========================================================================" CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS ./python/run-tests + +echo "" +echo "=========================================================================" +echo "Running SparkR tests" +echo "=========================================================================" + +CURRENT_BLOCK=$BLOCK_SPARKR_UNIT_TESTS + +if [ $(command -v R) ]; then + ./R/install-dev.sh + ./R/run-tests.sh +else + echo "Ignoring SparkR tests as R was not found in PATH" +fi + diff --git a/dev/run-tests-codes.sh b/dev/run-tests-codes.sh index 8ab6db6925d6e..154e01255b2ef 100644 --- a/dev/run-tests-codes.sh +++ b/dev/run-tests-codes.sh @@ -25,3 +25,4 @@ readonly BLOCK_BUILD=14 readonly BLOCK_MIMA=15 readonly BLOCK_SPARK_UNIT_TESTS=16 readonly BLOCK_PYSPARK_UNIT_TESTS=17 +readonly BLOCK_SPARKR_UNIT_TESTS=18 diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index f10aa6b59e1af..f6372835a6dbf 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -210,6 +210,8 @@ done failing_test="Spark unit tests" elif [ "$test_result" -eq "$BLOCK_PYSPARK_UNIT_TESTS" ]; then failing_test="PySpark unit tests" + elif [ "$test_result" -eq "$BLOCK_SPARKR_UNIT_TESTS" ]; then + failing_test="SparkR unit tests" else failing_test="some tests" fi diff --git a/docs/README.md b/docs/README.md index 3773ea25c8b67..5852f972a051d 100644 --- a/docs/README.md +++ b/docs/README.md @@ -58,13 +58,19 @@ phase, use the following sytax: We use Sphinx to generate Python API docs, so you will need to install it by running `sudo pip install sphinx`. -## API Docs (Scaladoc and Sphinx) +## knitr, devtools + +SparkR documentation is written using `roxygen2` and we use `knitr`, `devtools` to generate +documentation. To install these packages you can run `install.packages(c("knitr", "devtools"))` from a +R console. + +## API Docs (Scaladoc, Sphinx, roxygen2) You can build just the Spark scaladoc by running `build/sbt unidoc` from the SPARK_PROJECT_ROOT directory. Similarly, you can build just the PySpark docs by running `make html` from the SPARK_PROJECT_ROOT/python/docs directory. Documentation is only generated for classes that are listed as -public in `__init__.py`. +public in `__init__.py`. The SparkR docs can be built by running SPARK_PROJECT_ROOT/R/create-docs.sh. When you run `jekyll` in the `docs` directory, it will also copy over the scaladoc for the various Spark subprojects into the `docs` directory (and then also into the `_site` directory). We use a @@ -72,5 +78,5 @@ jekyll plugin to run `build/sbt unidoc` before building the site so if you haven may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs [Sphinx](http://sphinx-doc.org/). -NOTE: To skip the step of building and copying over the Scala and Python API docs, run `SKIP_API=1 +NOTE: To skip the step of building and copying over the Scala, Python, R API docs, run `SKIP_API=1 jekyll`. diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 2e88b3093652d..b92c75f90b11c 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -84,6 +84,7 @@
  • Scala
  • Java
  • Python
  • +
  • R
  • diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 3c626a0b7f54b..0ea3f8eab461b 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -78,5 +78,18 @@ puts "cp -r python/docs/_build/html/. docs/api/python" cp_r("python/docs/_build/html/.", "docs/api/python") - cd("..") + # Build SparkR API docs + puts "Moving to R directory and building roxygen docs." + cd("R") + puts `./create-docs.sh` + + puts "Moving back into home dir." + cd("../") + + puts "Making directory api/R" + mkdir_p "docs/api/R" + + puts "cp -r R/pkg/html/. docs/api/R" + cp_r("R/pkg/html/.", "docs/api/R") + end diff --git a/examples/src/main/r/kmeans.R b/examples/src/main/r/kmeans.R new file mode 100644 index 0000000000000..6e6b5cb93789c --- /dev/null +++ b/examples/src/main/r/kmeans.R @@ -0,0 +1,93 @@ +# +# 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. +# + +library(SparkR) + +# Logistic regression in Spark. +# Note: unlike the example in Scala, a point here is represented as a vector of +# doubles. + +parseVectors <- function(lines) { + lines <- strsplit(as.character(lines) , " ", fixed = TRUE) + list(matrix(as.numeric(unlist(lines)), ncol = length(lines[[1]]))) +} + +dist.fun <- function(P, C) { + apply( + C, + 1, + function(x) { + colSums((t(P) - x)^2) + } + ) +} + +closestPoint <- function(P, C) { + max.col(-dist.fun(P, C)) +} +# Main program + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 3) { + print("Usage: kmeans ") + q("no") +} + +sc <- sparkR.init(appName = "RKMeans") +K <- as.integer(args[[2]]) +convergeDist <- as.double(args[[3]]) + +lines <- textFile(sc, args[[1]]) +points <- cache(lapplyPartition(lines, parseVectors)) +# kPoints <- take(points, K) +kPoints <- do.call(rbind, takeSample(points, FALSE, K, 16189L)) +tempDist <- 1.0 + +while (tempDist > convergeDist) { + closest <- lapplyPartition( + lapply(points, + function(p) { + cp <- closestPoint(p, kPoints); + mapply(list, unique(cp), split.data.frame(cbind(1, p), cp), SIMPLIFY=FALSE) + }), + function(x) {do.call(c, x) + }) + + pointStats <- reduceByKey(closest, + function(p1, p2) { + t(colSums(rbind(p1, p2))) + }, + 2L) + + newPoints <- do.call( + rbind, + collect(lapply(pointStats, + function(tup) { + point.sum <- tup[[2]][, -1] + point.count <- tup[[2]][, 1] + point.sum/point.count + }))) + + D <- dist.fun(kPoints, newPoints) + tempDist <- sum(D[cbind(1:3, max.col(-D))]) + kPoints <- newPoints + cat("Finished iteration (delta = ", tempDist, ")\n") +} + +cat("Final centers:\n") +writeLines(unlist(lapply(kPoints, paste, collapse = " "))) diff --git a/examples/src/main/r/linear_solver_mnist.R b/examples/src/main/r/linear_solver_mnist.R new file mode 100644 index 0000000000000..c864a4232d010 --- /dev/null +++ b/examples/src/main/r/linear_solver_mnist.R @@ -0,0 +1,107 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +# Instructions: https://github.com/amplab-extras/SparkR-pkg/wiki/SparkR-Example:-Digit-Recognition-on-EC2 + +library(SparkR) +library(Matrix) + +args <- commandArgs(trailing = TRUE) + +# number of random features; default to 1100 +D <- ifelse(length(args) > 0, as.integer(args[[1]]), 1100) +# number of partitions for training dataset +trainParts <- 12 +# dimension of digits +d <- 784 +# number of test examples +NTrain <- 60000 +# number of training examples +NTest <- 10000 +# scale of features +gamma <- 4e-4 + +sc <- sparkR.init(appName = "SparkR-LinearSolver") + +# You can also use HDFS path to speed things up: +# hdfs:///train-mnist-dense-with-labels.data +file <- textFile(sc, "/data/train-mnist-dense-with-labels.data", trainParts) + +W <- gamma * matrix(nrow=D, ncol=d, data=rnorm(D*d)) +b <- 2 * pi * matrix(nrow=D, ncol=1, data=runif(D)) +broadcastW <- broadcast(sc, W) +broadcastB <- broadcast(sc, b) + +includePackage(sc, Matrix) +numericLines <- lapplyPartitionsWithIndex(file, + function(split, part) { + matList <- sapply(part, function(line) { + as.numeric(strsplit(line, ",", fixed=TRUE)[[1]]) + }, simplify=FALSE) + mat <- Matrix(ncol=d+1, data=unlist(matList, F, F), + sparse=T, byrow=T) + mat + }) + +featureLabels <- cache(lapplyPartition( + numericLines, + function(part) { + label <- part[,1] + mat <- part[,-1] + ones <- rep(1, nrow(mat)) + features <- cos( + mat %*% t(value(broadcastW)) + (matrix(ncol=1, data=ones) %*% t(value(broadcastB)))) + onesMat <- Matrix(ones) + featuresPlus <- cBind(features, onesMat) + labels <- matrix(nrow=nrow(mat), ncol=10, data=-1) + for (i in 1:nrow(mat)) { + labels[i, label[i]] <- 1 + } + list(label=labels, features=featuresPlus) + })) + +FTF <- Reduce("+", collect(lapplyPartition(featureLabels, + function(part) { + t(part$features) %*% part$features + }), flatten=F)) + +FTY <- Reduce("+", collect(lapplyPartition(featureLabels, + function(part) { + t(part$features) %*% part$label + }), flatten=F)) + +# solve for the coefficient matrix +C <- solve(FTF, FTY) + +test <- Matrix(as.matrix(read.csv("/data/test-mnist-dense-with-labels.data", + header=F), sparse=T)) +testData <- test[,-1] +testLabels <- matrix(ncol=1, test[,1]) + +err <- 0 + +# contstruct the feature maps for all examples from this digit +featuresTest <- cos(testData %*% t(value(broadcastW)) + + (matrix(ncol=1, data=rep(1, NTest)) %*% t(value(broadcastB)))) +featuresTest <- cBind(featuresTest, Matrix(rep(1, NTest))) + +# extract the one vs. all assignment +results <- featuresTest %*% C +labelsGot <- apply(results, 1, which.max) +err <- sum(testLabels != labelsGot) / nrow(testLabels) + +cat("\nFinished running. The error rate is: ", err, ".\n") diff --git a/examples/src/main/r/logistic_regression.R b/examples/src/main/r/logistic_regression.R new file mode 100644 index 0000000000000..2a86aa98160d3 --- /dev/null +++ b/examples/src/main/r/logistic_regression.R @@ -0,0 +1,62 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(SparkR) + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 3) { + print("Usage: logistic_regression ") + q("no") +} + +# Initialize Spark context +sc <- sparkR.init(appName = "LogisticRegressionR") +iterations <- as.integer(args[[2]]) +D <- as.integer(args[[3]]) + +readPartition <- function(part){ + part = strsplit(part, " ", fixed = T) + list(matrix(as.numeric(unlist(part)), ncol = length(part[[1]]))) +} + +# Read data points and convert each partition to a matrix +points <- cache(lapplyPartition(textFile(sc, args[[1]]), readPartition)) + +# Initialize w to a random value +w <- runif(n=D, min = -1, max = 1) +cat("Initial w: ", w, "\n") + +# Compute logistic regression gradient for a matrix of data points +gradient <- function(partition) { + partition = partition[[1]] + Y <- partition[, 1] # point labels (first column of input file) + X <- partition[, -1] # point coordinates + + # For each point (x, y), compute gradient function + dot <- X %*% w + logit <- 1 / (1 + exp(-Y * dot)) + grad <- t(X) %*% ((logit - 1) * Y) + list(grad) +} + +for (i in 1:iterations) { + cat("On iteration ", i, "\n") + w <- w - reduce(lapplyPartition(points, gradient), "+") +} + +cat("Final w: ", w, "\n") diff --git a/examples/src/main/r/pi.R b/examples/src/main/r/pi.R new file mode 100644 index 0000000000000..aa7a833e147a0 --- /dev/null +++ b/examples/src/main/r/pi.R @@ -0,0 +1,46 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +library(SparkR) + +args <- commandArgs(trailing = TRUE) + +sc <- sparkR.init(appName = "PiR") + +slices <- ifelse(length(args) > 1, as.integer(args[[2]]), 2) + +n <- 100000 * slices + +piFunc <- function(elem) { + rands <- runif(n = 2, min = -1, max = 1) + val <- ifelse((rands[1]^2 + rands[2]^2) < 1, 1.0, 0.0) + val +} + + +piFuncVec <- function(elems) { + message(length(elems)) + rands1 <- runif(n = length(elems), min = -1, max = 1) + rands2 <- runif(n = length(elems), min = -1, max = 1) + val <- ifelse((rands1^2 + rands2^2) < 1, 1.0, 0.0) + sum(val) +} + +rdd <- parallelize(sc, 1:n, slices) +count <- reduce(lapplyPartition(rdd, piFuncVec), sum) +cat("Pi is roughly", 4.0 * count / n, "\n") +cat("Num elements in RDD ", count(rdd), "\n") diff --git a/examples/src/main/r/wordcount.R b/examples/src/main/r/wordcount.R new file mode 100644 index 0000000000000..b734cb0ecf55b --- /dev/null +++ b/examples/src/main/r/wordcount.R @@ -0,0 +1,42 @@ +# +# 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. +# + +library(SparkR) + +args <- commandArgs(trailing = TRUE) + +if (length(args) != 1) { + print("Usage: wordcount ") + q("no") +} + +# Initialize Spark context +sc <- sparkR.init(appName = "RwordCount") +lines <- textFile(sc, args[[1]]) + +words <- flatMap(lines, + function(line) { + strsplit(line, " ")[[1]] + }) +wordCount <- lapply(words, function(word) { list(word, 1L) }) + +counts <- reduceByKey(wordCount, "+", 2L) +output <- collect(counts) + +for (wordcount in output) { + cat(wordcount[[1]], ": ", wordcount[[2]], "\n") +} diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index 9b04732afee14..f4ebc25bdd32b 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -274,14 +274,14 @@ static String quoteForBatchScript(String arg) { } /** - * Quotes a string so that it can be used in a command string and be parsed back into a single - * argument by python's "shlex.split()" function. - * + * Quotes a string so that it can be used in a command string. * Basically, just add simple escapes. E.g.: * original single argument : ab "cd" ef * after: "ab \"cd\" ef" + * + * This can be parsed back into a single argument by python's "shlex.split()" function. */ - static String quoteForPython(String s) { + static String quoteForCommandString(String s) { StringBuilder quoted = new StringBuilder().append('"'); for (int i = 0; i < s.length(); i++) { int cp = s.codePointAt(i); diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java index 91dcf70f105db..a73c9c87e3126 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkSubmitCommandBuilder.java @@ -17,14 +17,9 @@ package org.apache.spark.launcher; +import java.io.File; import java.io.IOException; -import java.util.ArrayList; -import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.List; -import java.util.Map; -import java.util.Properties; +import java.util.*; import static org.apache.spark.launcher.CommandBuilderUtils.*; @@ -53,6 +48,20 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { */ static final String PYSPARK_SHELL_RESOURCE = "pyspark-shell"; + /** + * Name of the app resource used to identify the SparkR shell. The command line parser expects + * the resource name to be the very first argument to spark-submit in this case. + * + * NOTE: this cannot be "sparkr-shell" since that identifies the SparkR shell to SparkSubmit + * (see sparkR.R), and can cause this code to enter into an infinite loop. + */ + static final String SPARKR_SHELL = "sparkr-shell-main"; + + /** + * This is the actual resource name that identifies the SparkR shell to SparkSubmit. + */ + static final String SPARKR_SHELL_RESOURCE = "sparkr-shell"; + /** * This map must match the class names for available special classes, since this modifies the way * command line parsing works. This maps the class name to the resource to use when calling @@ -87,6 +96,10 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { this.allowsMixedArguments = true; appResource = PYSPARK_SHELL_RESOURCE; submitArgs = args.subList(1, args.size()); + } else if (args.size() > 0 && args.get(0).equals(SPARKR_SHELL)) { + this.allowsMixedArguments = true; + appResource = SPARKR_SHELL_RESOURCE; + submitArgs = args.subList(1, args.size()); } else { this.allowsMixedArguments = false; } @@ -98,6 +111,8 @@ class SparkSubmitCommandBuilder extends AbstractCommandBuilder { public List buildCommand(Map env) throws IOException { if (PYSPARK_SHELL_RESOURCE.equals(appResource)) { return buildPySparkShellCommand(env); + } else if (SPARKR_SHELL_RESOURCE.equals(appResource)) { + return buildSparkRCommand(env); } else { return buildSparkSubmitCommand(env); } @@ -213,36 +228,62 @@ private List buildPySparkShellCommand(Map env) throws IO return buildCommand(env); } - // When launching the pyspark shell, the spark-submit arguments should be stored in the - // PYSPARK_SUBMIT_ARGS env variable. The executable is the PYSPARK_DRIVER_PYTHON env variable - // set by the pyspark script, followed by PYSPARK_DRIVER_PYTHON_OPTS. checkArgument(appArgs.isEmpty(), "pyspark does not support any application options."); + // When launching the pyspark shell, the spark-submit arguments should be stored in the + // PYSPARK_SUBMIT_ARGS env variable. + constructEnvVarArgs(env, "PYSPARK_SUBMIT_ARGS"); + + // The executable is the PYSPARK_DRIVER_PYTHON env variable set by the pyspark script, + // followed by PYSPARK_DRIVER_PYTHON_OPTS. + List pyargs = new ArrayList(); + pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python")); + String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS"); + if (!isEmpty(pyOpts)) { + pyargs.addAll(parseOptionString(pyOpts)); + } + + return pyargs; + } + + private List buildSparkRCommand(Map env) throws IOException { + if (!appArgs.isEmpty() && appArgs.get(0).endsWith(".R")) { + appResource = appArgs.get(0); + appArgs.remove(0); + return buildCommand(env); + } + // When launching the SparkR shell, store the spark-submit arguments in the SPARKR_SUBMIT_ARGS + // env variable. + constructEnvVarArgs(env, "SPARKR_SUBMIT_ARGS"); + + // Set shell.R as R_PROFILE_USER to load the SparkR package when the shell comes up. + String sparkHome = System.getenv("SPARK_HOME"); + env.put("R_PROFILE_USER", + join(File.separator, sparkHome, "R", "lib", "SparkR", "profile", "shell.R")); + + List args = new ArrayList(); + args.add(firstNonEmpty(System.getenv("SPARKR_DRIVER_R"), "R")); + return args; + } + + private void constructEnvVarArgs( + Map env, + String submitArgsEnvVariable) throws IOException { Properties props = loadPropertiesFile(); mergeEnvPathList(env, getLibPathEnvName(), firstNonEmptyValue(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH, conf, props)); - // Store spark-submit arguments in an environment variable, since there's no way to pass - // them to shell.py on the comand line. StringBuilder submitArgs = new StringBuilder(); for (String arg : buildSparkSubmitArgs()) { if (submitArgs.length() > 0) { submitArgs.append(" "); } - submitArgs.append(quoteForPython(arg)); + submitArgs.append(quoteForCommandString(arg)); } - env.put("PYSPARK_SUBMIT_ARGS", submitArgs.toString()); - - List pyargs = new ArrayList(); - pyargs.add(firstNonEmpty(System.getenv("PYSPARK_DRIVER_PYTHON"), "python")); - String pyOpts = System.getenv("PYSPARK_DRIVER_PYTHON_OPTS"); - if (!isEmpty(pyOpts)) { - pyargs.addAll(parseOptionString(pyOpts)); - } - - return pyargs; + env.put(submitArgsEnvVariable, submitArgs.toString()); } + private boolean isClientMode(Properties userProps) { String userMaster = firstNonEmpty(master, (String) userProps.get(SparkLauncher.SPARK_MASTER)); // Default master is "local[*]", so assume client mode in that case. diff --git a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java index dba0203867372..1ae42eed8a3af 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/CommandBuilderUtilsSuite.java @@ -79,9 +79,9 @@ public void testWindowsBatchQuoting() { @Test public void testPythonArgQuoting() { - assertEquals("\"abc\"", quoteForPython("abc")); - assertEquals("\"a b c\"", quoteForPython("a b c")); - assertEquals("\"a \\\"b\\\" c\"", quoteForPython("a \"b\" c")); + assertEquals("\"abc\"", quoteForCommandString("abc")); + assertEquals("\"a b c\"", quoteForCommandString("a b c")); + assertEquals("\"a \\\"b\\\" c\"", quoteForCommandString("a \"b\" c")); } private void testOpt(String opts, List expected) { diff --git a/pom.xml b/pom.xml index 42bd926a2fcb8..70e297c4f082a 100644 --- a/pom.xml +++ b/pom.xml @@ -1749,5 +1749,8 @@ parquet-provided + + sparkr + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala index a5e6b638d2150..53ad67372e024 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/GroupedData.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.types.NumericType @Experimental class GroupedData protected[sql](df: DataFrame, groupingExprs: Seq[Expression]) { - private[this] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { + private[sql] implicit def toDF(aggExprs: Seq[NamedExpression]): DataFrame = { val namedGroupingExprs = groupingExprs.map { case expr: NamedExpression => expr case expr: Expression => Alias(expr, expr.prettyString)() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala new file mode 100644 index 0000000000000..d1ea7cc3e9162 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -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.sql.api.r + +import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} + +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.api.r.SerDe +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, NamedExpression} +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.{Column, DataFrame, GroupedData, Row, SQLContext, SaveMode} + +private[r] object SQLUtils { + def createSQLContext(jsc: JavaSparkContext): SQLContext = { + new SQLContext(jsc) + } + + def getJavaSparkContext(sqlCtx: SQLContext): JavaSparkContext = { + new JavaSparkContext(sqlCtx.sparkContext) + } + + def toSeq[T](arr: Array[T]): Seq[T] = { + arr.toSeq + } + + def createDF(rdd: RDD[Array[Byte]], schemaString: String, sqlContext: SQLContext): DataFrame = { + val schema = DataType.fromJson(schemaString).asInstanceOf[StructType] + val num = schema.fields.size + val rowRDD = rdd.map(bytesToRow) + sqlContext.createDataFrame(rowRDD, schema) + } + + // A helper to include grouping columns in Agg() + def aggWithGrouping(gd: GroupedData, exprs: Column*): DataFrame = { + val aggExprs = exprs.map { col => + col.expr match { + case expr: NamedExpression => expr + case expr: Expression => Alias(expr, expr.simpleString)() + } + } + gd.toDF(aggExprs) + } + + def dfToRowRDD(df: DataFrame): JavaRDD[Array[Byte]] = { + df.map(r => rowToRBytes(r)) + } + + private[this] def bytesToRow(bytes: Array[Byte]): Row = { + val bis = new ByteArrayInputStream(bytes) + val dis = new DataInputStream(bis) + val num = SerDe.readInt(dis) + Row.fromSeq((0 until num).map { i => + SerDe.readObject(dis) + }.toSeq) + } + + private[this] def rowToRBytes(row: Row): Array[Byte] = { + val bos = new ByteArrayOutputStream() + val dos = new DataOutputStream(bos) + + SerDe.writeInt(dos, row.length) + (0 until row.length).map { idx => + val obj: Object = row(idx).asInstanceOf[Object] + SerDe.writeObject(dos, obj) + } + bos.toByteArray() + } + + def dfToCols(df: DataFrame): Array[Array[Byte]] = { + // localDF is Array[Row] + val localDF = df.collect() + val numCols = df.columns.length + // dfCols is Array[Array[Any]] + val dfCols = convertRowsToColumns(localDF, numCols) + + dfCols.map { col => + colToRBytes(col) + } + } + + def convertRowsToColumns(localDF: Array[Row], numCols: Int): Array[Array[Any]] = { + (0 until numCols).map { colIdx => + localDF.map { row => + row(colIdx) + } + }.toArray + } + + def colToRBytes(col: Array[Any]): Array[Byte] = { + val numRows = col.length + val bos = new ByteArrayOutputStream() + val dos = new DataOutputStream(bos) + + SerDe.writeInt(dos, numRows) + + col.map { item => + val obj: Object = item.asInstanceOf[Object] + SerDe.writeObject(dos, obj) + } + bos.toByteArray() + } + + def saveMode(mode: String): SaveMode = { + mode match { + case "append" => SaveMode.Append + case "overwrite" => SaveMode.Overwrite + case "error" => SaveMode.ErrorIfExists + case "ignore" => SaveMode.Ignore + } + } +} diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 24a1e02795218..32bc4e5663062 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -469,6 +469,9 @@ private[spark] class ApplicationMaster( System.setProperty("spark.submit.pyFiles", PythonRunner.formatPaths(args.pyFiles).mkString(",")) } + if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) { + // TODO(davies): add R dependencies here + } val mainMethod = userClassLoader.loadClass(args.userClass) .getMethod("main", classOf[Array[String]]) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index e1a992af3aae7..ae6dc1094d724 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -25,6 +25,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var userJar: String = null var userClass: String = null var primaryPyFile: String = null + var primaryRFile: String = null var pyFiles: String = null var userArgs: Seq[String] = Seq[String]() var executorMemory = 1024 @@ -54,6 +55,10 @@ class ApplicationMasterArguments(val args: Array[String]) { primaryPyFile = value args = tail + case ("--primary-r-file") :: value :: tail => + primaryRFile = value + args = tail + case ("--py-files") :: value :: tail => pyFiles = value args = tail @@ -79,6 +84,11 @@ class ApplicationMasterArguments(val args: Array[String]) { } } + if (primaryPyFile != null && primaryRFile != null) { + System.err.println("Cannot have primary-py-file and primary-r-file at the same time") + System.exit(-1) + } + userArgs = userArgsBuffer.readOnly } @@ -92,6 +102,7 @@ class ApplicationMasterArguments(val args: Array[String]) { | --jar JAR_PATH Path to your application's JAR file | --class CLASS_NAME Name of your application's main class | --primary-py-file A main Python file + | --primary-r-file A main R file | --py-files PY_FILES Comma-separated list of .zip, .egg, or .py files to | place on the PYTHONPATH for Python apps. | --args ARGS Arguments to be passed to your application's main class. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 7219852c0a752..c1effd3c8a718 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -491,6 +491,12 @@ private[spark] class Client( } else { Nil } + val primaryRFile = + if (args.primaryRFile != null) { + Seq("--primary-r-file", args.primaryRFile) + } else { + Nil + } val amClass = if (isClusterMode) { Class.forName("org.apache.spark.deploy.yarn.ApplicationMaster").getName @@ -500,12 +506,15 @@ private[spark] class Client( if (args.primaryPyFile != null && args.primaryPyFile.endsWith(".py")) { args.userArgs = ArrayBuffer(args.primaryPyFile, args.pyFiles) ++ args.userArgs } + if (args.primaryRFile != null && args.primaryRFile.endsWith(".R")) { + args.userArgs = ArrayBuffer(args.primaryRFile) ++ args.userArgs + } val userArgs = args.userArgs.flatMap { arg => Seq("--arg", YarnSparkHadoopUtil.escapeForShell(arg)) } val amArgs = - Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ pyFiles ++ userArgs ++ - Seq( + Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ pyFiles ++ primaryRFile ++ + userArgs ++ Seq( "--executor-memory", args.executorMemory.toString + "m", "--executor-cores", args.executorCores.toString, "--num-executors ", args.numExecutors.toString) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 3bc7eb1abf341..da6798cb1b279 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -32,6 +32,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) var userClass: String = null var pyFiles: String = null var primaryPyFile: String = null + var primaryRFile: String = null var userArgs: ArrayBuffer[String] = new ArrayBuffer[String]() var executorMemory = 1024 // MB var executorCores = 1 @@ -150,6 +151,10 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) primaryPyFile = value args = tail + case ("--primary-r-file") :: value :: tail => + primaryRFile = value + args = tail + case ("--args" | "--arg") :: value :: tail => if (args(0) == "--args") { println("--args is deprecated. Use --arg instead.") @@ -228,6 +233,11 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) throw new IllegalArgumentException(getUsageMessage(args)) } } + + if (primaryPyFile != null && primaryRFile != null) { + throw new IllegalArgumentException("Cannot have primary-py-file and primary-r-file" + + " at the same time") + } } private def getUsageMessage(unknownParam: List[String] = null): String = { @@ -240,6 +250,7 @@ private[spark] class ClientArguments(args: Array[String], sparkConf: SparkConf) | mode) | --class CLASS_NAME Name of your application's main class (required) | --primary-py-file A main Python file + | --primary-r-file A main R file | --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) From b9c51c04932efeeda790752276078314db440634 Mon Sep 17 00:00:00 2001 From: Peter Parente Date: Thu, 9 Apr 2015 06:37:20 -0400 Subject: [PATCH 014/128] [SPARK-6343] Doc driver-worker network reqs Attempt at making the driver-worker networking requirement more explicit and up-front in the documentation (see https://issues.apache.org/jira/browse/SPARK-6343). Update cluster overview diagram to show connections from workers to driver. Add a bullet below about how driver listens / accepts connections from workers. Author: Peter Parente Closes #5382 from parente/SPARK-6343 and squashes the following commits: 0b2fb9d [Peter Parente] [SPARK-6343] Doc driver-worker network reqs --- docs/cluster-overview.md | 6 +++++- docs/img/cluster-overview.png | Bin 28011 -> 33565 bytes docs/img/cluster-overview.pptx | Bin 51771 -> 28133 bytes 3 files changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 6a75d5c457f02..7079de546e2f5 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -33,7 +33,11 @@ There are several useful things to note about this architecture: 2. Spark is agnostic to the underlying cluster manager. As long as it can acquire executor processes, and these communicate with each other, it is relatively easy to run it even on a cluster manager that also supports other applications (e.g. Mesos/YARN). -3. Because the driver schedules tasks on the cluster, it should be run close to the worker +3. The driver program must listen for and accept incoming connections from its executors throughout + its lifetime (e.g., see [spark.driver.port and spark.fileserver.port in the network config + section](configuration.html#networking)). As such, the driver program must be network + addressable from the worker nodes. +4. Because the driver schedules tasks on the cluster, it should be run close to the worker nodes, preferably on the same local area network. If you'd like to send requests to the cluster remotely, it's better to open an RPC to the driver and have it submit operations from nearby than to run a driver far away from the worker nodes. diff --git a/docs/img/cluster-overview.png b/docs/img/cluster-overview.png index 368274068e7548517e3f5572916e38b3a63b355f..317554c5f2a5b1022ac594f4bcae774e2ca62521 100644 GIT binary patch literal 33565 zcmcG0gNq2WQ(jlFKbcld-r<4dtr*wP|9^Z4` z-yiUKU0yS@=ZV?-+56t>UTfWisVGTfpc12kKp+fR8A&w|2wnpOf+L3@0WAu35$(VW z*i1}O3o>N>rp&I^cEA}Rkf19+@YdSY={3yN+RDa}-&Kg}&k_8< z`^RP$D%hVxoGgW?UMZ@;By1f_VO-3d%&b(xs4y5z(BaKnel^J#e@_QK2~n9lIoa{E zu(-ImFuQOt+d7!Bu<`Nnv9PkUu(LA(M=&|M**Lv+WwLRk{?A4JxsIf%qltrsos)&F z4ear{uZ?Y;orI{U9&hyDfB*TNP8M(fcPAUizuf{l$ny9N3mY>l%YUy8oGSR(%CGET zVG7*&@%qASf`5+uKW%^SBgpc2^MC!A|Mc`vE6`P8R6&;i4x2ElLN~282qX%Ul@!x( zh1=6doWqgE$!fp_y6@r=u5172l}?4Op0_Y@0{_pa*?6CowUu@36ItPJDL*sZvIdv- z4=P$+zO|*LeYoq*3zP)IiHSm?;J+`ok)4g+?P|=p|1|t*Hv$K$d<%dZV> zU_r$T}WT6152sJif5AS(I3jx`pD;?cG(woW>l#+*OdwuqK{f=Tub=RbpG#}Drs zhsUTj1k+GD^yMThhk`708g0lDifudYW?-t(vGgFCK!#UB-0ZLZjG7(=9F)OIx6#J= zZba_M61ks0BHBdA6N+CghwbY%sj^AbIt-K=HV<_Dpg)h{h!o73)@>jOwZn&pXK8C z@_QO&)$Qi;nELKw#pi5$l2$~-NBm=^+iU^T-=|9P_>`$7)nL9@tO@iT@$QrwaGvS~ z{QvpduT$FSbaPmv_KmFj^;wCg$z-we)>+xhDxJcrS2g|vDJ-MJkTP_`ERFJ4aqssR zHU=lW8E4A1IZD-wF)ih-ED70+!5D*K&g+@weI$~kK*iPL;gI5t@!5L zb-g#r>EYgM1^+^w586;DpE8lkX42k#-xG%8^s_&%YJyC0uGSQ7sm)_{d{D5x)qC@7 znFRq~e1l->C-L9j!K7f?wQq1)3NPol;osn!n|fcaAv?y) zlhZ0?rXoEdSJ)c);P~y{?Om~G!L3A*Zv>P<*^B>;-k))I<%O3D$IT@Zcu$xg*vnu# zkT~_(-o#%PN<-SeF_@ywsNU*oFC3y(=CnP5E8^`rkR>F5LBbtL79V%DH_ri#CB;&+ zb40Vtp%M}Mn^j9Iks`&6UhsZ*5L)K%A79blEjBPUX49+F?2P9lxA{C|or<17Kb~Ek zZcP=);xMQeMOr)ex(%mu(fi!rPSsne+ZrBK8MNgoec-o$5bZA2s*r1DX=Y!A!zNwwNe&-9;Yuu28NWztE+B;^%7 zTy9$@x1Ou^lS`x%gXzF{7bKA;pp@d^fn+9lTevbE0kkf^KYtC%;27Cb#vtQ*x~mhsCUC-(8w_zOjgYjbo&S#Qtx&1USIM& z8kVD>NA*uM5sjtJ3H6u4A|709v}Iy&{}glA8*(`_2%-Gh1>J+ z%Om?Q!5B>8J%rogoJTeHD)(A?F(>M^blqS2=(=eQXug@r48@V%)n&h`+dn9ivuazdk+{sx6Y zxOHRUlciq0R)r4!hQ?@LI80kS7~P`D^_VJq9pQ7Ka`ti_r*39Zz0wOeeHn}L&lKS@ z4rVsl*W=gxuX+oU(C>z7_j0c21Yq^DXu3 zmag9w6my43r`||0@rvJkL#EXh~f_aOiBMIZYViIY>JWzc~*1m*rXse_@ zKYu1(PMLZp9)fAjMJ(BiG!FVU2A@r(+x02+S%@755JXHU7!$jkR6uL!-;g*>?FnXrd5ct$h7?$1ZVA;uQZclZ%Lt+#-t$(T>(=e>b41~0Zs{f|EbDT-{MY$I@h9rF(7g3=XqBD@ zy7;OK_85Fr|3O;t%FZg9f=j+U5uU=t62>au0C2>ggkoVQ^u77WTOQ0?AEP&*akPEtY7QNtg z?~5Xt<*WKW7 zTzc%+y`&I~YlMm^AbO^KUL&?@AHU4OSagt)(4Di)L#U(SG=|xmqv6IT*F2A_#opa5Az05S_w1J`b%yWL%Ly^bIkYEBtmYH|z-%VN@= zBmlWc`XLjcNBe&^LBoG;gu`|_wyGS0f(WZlhu$EiR7Lf)H^~kB7ZTV+`=a_lSE+ru z`_tfuC|9P~X}w3gQ25G>Y(rGU!VF)cYTf1j8XeLI;?U+W%?P1%Tu{71h>1Bk+hOc|(D-k>L`4 z{JU|E3KlqkkHtsuFC5w_2As_YNyYs?jrhQFZwT#g`a4hmykI__E!J7^`QMEVz{XW8 zB{_olFTiUD1{Dff0}Y9brVDSvS4TURd)YqE7+>>`o-s*@e^ZSkR`4d1 zzSwVFmrqn&UubS6g^**oU%2n=A-DH%xw%8e~h2B0^xIBfflTRNN zhG{rWX9Ta$wsE%B@?}G^>P-&eu4@eBOBZW7mf9{l`_Uo;${G*G^L!e{!lPQY4+8q& z7J4c?o>;6L*TLzSsb~T?{$p&)`@ho$p3Lv!aG7HbZkx^aaHV6bVenQZl-p)SaS18% zr(d=Nq_CKyPGz@`nA1~%@(TT{yk ziih&Tfekse>nZw@l~5FoD$SSc*CO`@#i9CUsALR@JEI=q_*}Jd1hL3BG1>WVzLKbx zP7Hs1NW9W$*`<@y%0p4qy;Sz)I_+!zpiy@E5md%P$B3O^sFKHG{dcj08Q}AaZWZRN zO!ro^rrNa;!pWH!hSNCX8K$ho5KlJ?Fxx^@Ix4N{d=By4M4gsXHb zsJLua)Gexe$NZugagVj7k)tzn21B?Rjd6ko10ly`)+=7VC~?*;o2N2Z)GAl;QWkTU z*&ox{6?|5 zcHhKKvKY@UO&(Gf%FrzCIW`_YO{uczwq^Xdn>w!n^Kc(XWsvo4*gi-?$#u98pu77U zPL8nsik!<-6Pl+x29%*Z(Q#U(uf*QXSL5Rp!iUTJJ^-L;fV#)Xb0Uk*{;(GdCL5yO zw`Y9UK>9=`;1*o_R>3ls9(U(^X^ra4ctAewh)QpL2|9bwt~?%-43M5;i0KxgXhxS4 zkyR|@o-iRRdVZPl7Gv9J3dMS+Bx1JTxJD&7 zQI<|RIPnEJra`;Agus*Rygp?^m2dF#bUHsnwXUj?-rC$9P@nyrM;l1tA}drN9c7>> z>LNo{o**fV8_amdRH)|V$b}`o@y{@KFY&&h$38bvaS?Q#GIP{aGkhzRT+wJou53Ao7>YK7p{wJ`;=t=+Y&LP% zaHue8+rv?pOmr@%rQfquIJdJsylYc)93R zN4)rbeseopXru+Zyz5W-(C+#Hb!PX)3Oy+YeT7!teS(EX%8RK68^jTx`yH;0is>|Q z`aKFkZ>7eh?3wsuq3u3nH(sClh5f05L&tTXtN8eTxrdh zQF4`wXN9{FAWGXYT5r(bli+fm5=FhbyO0w&eTI+E$vbPa%udkAuawM7$#lDJJL4S$ zIljWf7{31D-o0w#ZPYD)t(;zSY1qEG{DpQ( z53BmznQvl~#o6GSDR|`o! zZ?bY$YLwdDs688Wt{Yze2q3>;p-=2-@xsFToWWFsoZ}S@Dwp&ETB{24TGVc02#; z$8Uk}XV>~6n)(zU%inFq%eA=fbDPR%`{zlbOP4hqT$Fey4HIA_sjnsX{>=q4!#eBv z&`>gjn!FqcFvu=&mV7Q#xvL2KrF3{I^SepQU&nuP*=OIGu6d^4;hZ5iYo2$TZagL- zHFmPuFPD$xv~^;k{;ZhDBt)gfAi#X;XW9VzW%MfRa+^NK$l-i0Vp7R;qH)oP;%o7c zJi&KbhHAvF&Ie=q6J$T`gR{b2 zPf`&r&gFH40(#Z&uX^3e&EvLP4ux+lDA_#cSO;3K;!d$@;eID-VxBZOytJs&lMPg9 znplc${|gSmgz(ah_a=B9-x_{=RHahinu#j0O_>o~d~)SF|4?^iMx(SNG&0zA%kNxI z#jpR%f5ORBE+w5xy7v7?8;urJO?tz4B|7QfxUx9vLxhI?5}W82+Y^4nP8xhw5`%%a z$l;w`FVV1d;>r^Qv?iM(Uedii8=dVEDSMxE!tlM@O-VkB6Dwt!g7$rRo_Xn^-!oMz zsT^=kJ|pe%mvY5g>*A6VvjxXJ^reMn-y-_S<$mpsNlw!d#w;GjuWH^(j>%Q|ceEpn zRFc7`SL+45fGLBp-|mswM&q|t2`@lyB+60+r& zFt;QB0}n+-DPv&ErOPx=+3Ek8Tt>V`|!#>&s87ma!C|>x-67!G#_&yf;|H0y&NPaCYhYJP$C3RRI^#rrSMKq&yFf--WH}7SerqSC!geZ;U znffy(i7;Z=yo_nlXcSjvSa}EgM9!IX_i=`f_IL4sA36M_QTJyy3#VC{ski;g^)_3S za#flAVp9uK%+klkUX-z(ejlFnH1E86kltbSq^VE^qt3G2yO3-Qq=paR4!zzA&ymL# z$8(yKAN(7a-a~)Q*U^3u_DVlkYLTun_B#GlqplGTkL4#8&dF7(=ifjdHBm`jJkQo@ zexRVyprVvNjs31@DH_q_SCyj2ubqmUsUjt}r0jrUUzVx7saP6Kc{QEFl55L#^6NKO zl0J(2wml^SANaU#Oo;GG12V&(Q||dEtLl^BEG*?H#8v@8o+w~Jb&ZtBsbO%hLv@GG zbOZMm)iDtsARsh*wZp&CC<*^~gT1*zzn~zgH(r(XW<9ix4W%$H8w;!oL{+|;$z`Ef@gNrQ?a5;c_I#kbQ2#3hZ zCeWD&-!hD?8U?)VYn?CJ_UX7pwpy<5g;+o~AwS@U6P6dr*Yzx`TNf>d3CnS5ZEa3{gx8mNIKZpz2-yF`+ng*b5+C;Ik zvP1}GAto6=E4LQnPk;=E1Jc5#PQ_T}$C$PbS$*cS-iu+<0x)y-k!JwAUXl}K>!r_T zaLH!eV}yZL|9q3l;b(t&QrGf1uW?UkJ2%Q@GY7K{^sO9}(P}hXgeTe_bC|5&ZBA6} zcg=D@r-iu3-W*$qbZA41^X?1z8dC~1Ak@7o*)|!nf3Avx;2zy<-YAQ@1!WB~k`R#VB zd;qYTaHgNuq{V3)8;7>lybUL|5R8KT${GcShI3^7Yeb*Uv%{TO!7WNC zjRN8NvgCJ)#2} z`G4WO1gMzfA{5*q2Llv?Oqhaf0?h^n=C^BJfRHcOKH1Tcbuk3Dy3??~9g+c<-5h|J zth1Q$lBN^7&;Xv0+s=n1dlYyvUywm?vC@ay1<``lJu79=bkRh$A_dtJwZb1>CqKWX z@qUe=$o^2fuTN`{72!KiNw)cw-+6~M&R@Mb6t+6>I0ZLasJ(E&0I@d&FJS)z+|_gm zP^YcPaoy$o9@t#(G~`m4$qJP|qyxWDB@BmlQ``EyGOm6#43s&?s97p$Hk!3zhE|{p zOdaIMsq;lYUKkZ#EEGf1IVn? zI?X&ulyQ}7Bv#}>&r^K}!_4QZfR|0S$0=1Sd=;mFfP(ai^C{%BFg)aq+tI3+iGbZI zXmp@SG`D~wU6_Cs<|W>?g8d&HYmc@bvyaq?(_ReL$rV|tWj+E6%+lu zn0pA68h%mLQ!J3rbnv0n^*#^Yj3;kKDS5>MAzD2sVAOQlp<0~3G3zoCU{Tz#!G1=* zpdrW9*NtYQEQh0+e5XnCnCgplZ-)o$y`j6vB$+*4Ey619zVzQsR7+ZallgsIpO_tnx+{f*wyl)&U*Yb~gUt;R!) zygq#paG5yYU$p3pCIi$JEQ8(Q@7j`xqY1Q6Y?~animc{qkKG+M+5cfRR(XIMUSaSw z$w47%;A%nSe&9i|uB0LA&_IhkuTHS~aYAwZ--Q2zQ7{3{2_E@ni6#`H0)}Ro4JI?y zyPdvWW;SRw*L(o9E9EAKjXXd+Q}csIqywS^F0&qQ8M_13-&DsHm^7$@R)ufHI!7U` zuE+kLl+#5iLyNSlV{PwmuQ1?~0M<3l*$hc08qx28T4k?~|IO85e%}wj&uYR!;!yNV z$D1G(K4dcsDd-WtAD~WP84FkFHb(Q1QY)fys*Ldis$XO{KGT5GPfGFr*_Z3o9%7x0 zk5mu9?r1hF7T?gw6dSE(EyoG|4Ut2!XDS*LBJUmA?)b>&SRm$ngmd0l6iW9*wJP_0rMI6^|vT0sc*(gE2e^^u&Fe#13TlT`3-^*>m%xxH`U{1dYkGs8Jr#ZjWMH|A7NrgRAQKm{Y z%P!~id;l7y1bD&)v7{(50Rkba-1kvDtk8tdJ#|XEgEU+V!8GlNIHg zje^%dHB+H0L_$WS?@Q&IlRzQt2@_gcQ7AnUY~rs0q&4yiy{7p4>s^)^nMX+!Nf?mY za^#X2#$!b8Wr#T5a!zdg)yiS6uvOOF2_5=9DKMNgw3FlXcs&K=0aK||DgcBUiNO?> z;uW8V4+O+Bqe9owruv*=-RMGCK(~(2!Ab_7cFZ>6;zBH_QNghP`x+o^d-H{ zPY5p0Th8ooB<=esfhWI40{RRZVpcjyiPYSIV~}=yj(0E$6|=^Q^K23hS#!Jl&*Z;a zzn~{`CIaW=`ilw#btcWV5SN^UOLqAIGyO+-P^n5TJg~nnnc5{q0}D#DlTxMXhLGQD zJ2(X|nJG2kK67Do&@-ET$B9geYXG-En91@=gR}pnm_=I(K#AL?H!I#Z7;Svb6OTs~ zG2e!~Cli(uAc#6G`MCH~K;h%j36A&%_r+`>fA|&>ap__EP4u@!?_|BMsV%ZF2K{DE zKr@H}^h;$xb#b9rDC7Ww2=oZDr^g#DtSwxnQT_S?t905FcmKhQ>X{-mg9I(t`-`AH zQhkX;Tl!0$U?L%9MdFzn;ZGk`du0+3EFUN5Ov+#yY?ToHNB|Iu6X^RYG3O?LWmzb4 ze;I%es0IS>_gL@gL~(*Z8Gx`rfQ?rjD{t-Zs|$lNTbbu)zn1VZ$b|t35-z3QE9$%H z-NyREOTK3JUFNT$#AzpQW%Gwzv7&CxSPZcG=gMYn=@e@N6FE#6s7KFN2Zmcpvpf4PUt#zKNdvy?8;d z+M1fk<4BB3vWeEP+3YvFHc3FQ?a!OYYNOzoyhwTzrh=FpjJlhm-|jgNVgWEpqM)0D zzYl6X@iK}*bgGzEjj>4AI4PbU55sU3E69qgwvqJU0quR~GJzxsnVUD854 z=m2S^?k65Dq&y6k0aH!=P)s%$BJ}PwW)jNsXE|&fpD?!u9lcjiP(8h`PEor;V^Q9sv zw|v4Ivi~xWLHat;=|x36%5(z+IbRbR>^8la+B~+w8UpmHBi2IC!Wgl<`xKA$`WmKXhE(;() zVrhE9bUxcrhVYG1Vv71=&ExmCnh}2%O)yA!8pn}>8~00oe@83F&w7`5`@pVxHG8=JVvdi$@h$$>aPpuQy^1fa z>9t3;G#ZOfvNYc8!Y?g1_~tPndm1HnD7{>0e&C&^H=H8;5%IFBMB&sMSIMIC{@x;I z(EL{?&;nrk`lYM?iX2CFu-e2Z#l~xPRv!x z$2S&p^Ltl%f3b1veU96FhmQ!d{oYgwj^z-18^VNd3S^b<`uD7%HVL9}Y`JRO)-+iS2D~otU43UOJ`a5Bdp;A9le!n7s1{&o^ zHY`nhN6TW6|CudsVv>NR=_|O#WNc4lW@2nRqC|ZBd!lq&^GMa**HWI;Z65n&Lya~K zxJGJqLYcaE15Dz-h2#n#13c#<&8~Oc3Ax3ihR3|?ec`?Mq0ISas!OO+C89Gj?Y8V?)T;x;`6k1$U{OqR!G0aa=Nz@(ckY-c zp5VAwuvc~a~%KrHU9J0wWfpkO5Lwmm3XDL`r`s_Foxv;pF-MaKN(}IC**># zRH`b&j**C6V{V-yh3x5m>1fh5&(x<_@+i@BcAsB({i4O+|4Jyz8UIZF?Lqh=>3aK@ zOk~9^0NX}-2iQsA1eqbdee<}v*fGOj-UA8i;K0z6ml#%?BICK@LB@`)jviaDEHB7~ zbG;L`CL0j?whz7(C$gCuyWj2wojkcCRe}%~18bM{HF2Jd=H9Qk63teu(Cvy(wyWph?4jD5ooIIJ7wY$#uM;^Q%+ObPxP3HU?o`?7 zoE>VRlI?h0*Ss=FAFJcDEcfk78{{Dhkmos=sBwHr=TbB!^IrFAM&EroM+^Z#__!?0 z5iEv)g~noUt_JMj&owDpt5a{@^>YKr{WR*{D!SiZ*02f2mIEI-a^advbsL#;m7q^r zeQSCbmfJj<1mTl00mICqFFhokMH<{g3&3Pg6k|d^=#RbvgEsfuFL1R;Y8-Z}qTN55 zINlDyz`>F&I>4HdhP{SGA2Bfk*|_ueL{Ku5Zm*Q++}qKt;es0b^az~>!#r# zeDJX549yP7t!IK+zvaCtdyr2DlWxP$OI6snF&fk~hpR<;vNIL%Q3#gLbt?||PU=YS zmd)9e9$;bg+wDWViDtsc4-CX_QWWz7?TzEp5k{9Vv3>{;&lj>SEjEP$kvqPWUU9^? z1a0o=$G++Mcd`rIKAbni&J>mvZ4;(?ZY#DYg6TDHxPDt}CsbNx$`j0v<_>&4)yCS~ zDMXvlHpLda8g(L`Xcw+nvX9llxXvq-D;+9A#(lqmwpGyE);GE6a;WdtFMK|{!KL92 zUZ|22s0eu8RYJ((ZC;bpGy&h|^!&o_(==dp=>C>E{QJujk?JtBE`P*FkWRzHpA3~j z9PqCf6I~di;S^q;zY@z5oq>%-i|!&SDg!`}7lYy#?aqtr;{Z5o+=ZZXlsj-&nPG=; z3LKk11);_|%ssX8iI{eeXg6U!Z0>3Od!0IPEm#q+UU4CeLK17_8m0v`KNckwONYT6 zMDEUT`~rC8H6l$kuZoZdp8`occ!vE=e{7{7csm0I zeGc`-NA*EEMxJ5pav|`EJlYy31k;n^w4Z=xW* zOQVkTwO3DHz9`jhV7Xl+e9&Qr=)?nnPxFMP`+q)^c02NBm##D)V+cWjg?*X*W>2~s_5=m*~l3=F6T9=sL z>0@R3ho8BM2x;BVIa2RM;m?R|KUcaOE-ToG{>({&FByM>ink_>bZ`3BPeO3PFAdZG zc$-Ze4e3W91#5i?B3M3wzM$(C#X5~9YOLHZr+a8S+@I@Alr#4OLhgkBhJk2a4hs1F zBOHg;Lsof6cJ)WHMY5Mj&TK!o8LaU@bzE-e16fKL(Zp$C)mznR^1r^FrjYJ?9G6z$ zDoY9ZT>d0-A>%l&eYr2n#gDONhqCx2(`X-!le+=FLg{N>(?{(UStqNF`l@d1xn!55 zQ=M=bXVPPi-)WbOU&F|gjIKTIPq^Atyj2fZd~zIu9Mnc@99@eH3n@;BVCq9y-N#6Z~PtSmzC3clNa0i$_3ZzC8e#waN*gun2H&B1GXD$F1 zx`8^xnh|2K4q!55*1np9kqK)oC_pgO%3m42HG!>UMN)_?+sQSi#Bg37{!+ml6Bff} z?4Ymv<(SHx>s&Y6z@*ywT)d5$e5d;pYfyJ0GOhBe$*vQ|1GsKAWa#~Sk_kOUQFgJs zBbJM$mQi_e$#C2-#YWh0t#?-065F=2ofK#WUWf?X(E+_JT2-d}PQEIDN}*KOshIn2 zx>ReE%F$beoK~JNq?xNyTs4tfb})18!>_|(4~xlV69%v-=e6YmNUG9cELdOqGen%0 zRG%&BE+4bB8Ztjom?Il5^NYw^F-gJ&X~U(^w07Xj9CY8M+cN> z0z^{L0(Z``USSoJ&D-4Dfh@b_=pYk3nh?$Br6zJD1kq;4cq3_XyT0!k=N}LXso;JK zd7fmB7&dLx5)uYn{ODwY;3*h8ZVtWnFHqE;TZ~6IB?K^;d>EdEANS5c663AuG%;nt z7cjTiwbMHGEu+8N6G_xSMa%9;_3W|b6^*5Oi!j|!$5fP`V(K?B8La@AfB0nxDWx(C#Etw5Ca-r}S) zQ;d0B>UY@_uLmNbyiu)s781Gy)1pHNIR(h0+9K2ihR7}fN7B%_SW1bxC+NZ|5H2_$ z`pcNKyBc_RK{!U2mJ2L6r1R3js)!OAI)L%&tRhcxUkl zbUzAO@oG-1xmW~N4<2C6)EwCd6`^uC07<;ESZ5&wujqk*PNlOGUupIT$3IL-z<`SAoyGeaN3sIRumJ$GqF6J$#A8Wk@PpN4a6(_WdPo2(LCFPCxF5b2)?g z%FxK>M+%+iiDDmMrek)Lp8ma{biT#=_j_~Wb~3YG;53BZ5$L6EPBtq6ESw#Df#$6W z3CJ}G;LB&32nKpOCk^s(J<33tMH4C`dWPj&r<(u53as*4`(TfMoQMZSQ!2f3dA(MvqbI@vL(rt4!i(m0j%*pB-=E)KDT82#9`NE|0uF%`D zd7MjrH@xS#_v$cX7|~dfSdLuZ6C#D6SilT@ynO5DqVF2E$aS8+M|Y6@63wldnu+4P^l1jv4~E9F-()}vKD_>>#`-_RJcUg%UH4W zpzb{U1Q5Lzkk7ceC!BV7{N(1QIbr@LLbo_a5=f3u?@_4cid#?>K96~K)fbzoch`^$ zJB;-g$|m&chtS;Ng9VY`E|a@6k*#rQ&dmLOgPx-Le6G(f8cApa#s~^{KE{hd50K+S zsOrR8Kc)zk%QJOC1Y_sd+ay-dp?iy^W-iNB=*@styz{s&)e<&X5FhS#zDo`jrW%@1 zOuGd*$U`2ER!1{in58#rcodN~9UeL9Cf$=J2^b;diSIBO;#;9uaVR~MfhqJX`kC(| zF<5t&$%Z1@$m9Wl^y(KNzrZy#)QV#u3E{;_5UE}O*Q*}wbKX?x7&4gwTcptoKv5vV z;5+Amzf=u&6f_?E-r^FoE3vB56D9r`;paB2_})f=;gPSS5s$unx+V}ljU-wyREA-M zhZswDX>E*@6$10?l*0sa#y&}%XPRrl4M*RLQBxkFAu?4W=4eO`!BFAF@pgg}r8 z$NA7!s8q-_lF`H@vB_r~V9UZroXjdl?q52a?e1)Xv(YF6_|lALfdhQJ9vY07jS<=* zFtr5{RFTbBsf3m&-A*v$C|{b4nGy#YV2S$y)Rnh10&*q7pxkA zg64}26;=MZ%|a^PguZDC+a_%n63z~?tP-7q2>5*v3y^xVc1}?Dg(E;8D^wm8Bl{E8 zhbkgdZB*6;4zDMgh;=4C+8+m8`eePnJYiD+y^>iVtPa$at zll#iNmZetw=-DcY3W0x+25ky@&H`*`ACBM=`aDoy(W|Cr$KH6+*T2jaPXjDaxDa1x zsqdEwCQJV?P}eOCrUa7*C^pkN@*xj*Zs?EKzxU?VNr&(3QngO}uqquJWoj((R^O1)q{;r=ltzN8D^N?_Wj<3rMB-UgAoy#Hu>QWP#KxcR_aI9ui70AH?F9TGer4+Q4z9<1ju3y z;<*dIX6~n1Pt3xg3}%%3#`{LrooEEibNhHdCSiF^GM$vZDIb97MF5VcQ5--EF~PU# zkEP^W#!U|KGsN>G;c99INP=o7(Un@~3`3(NOFfVuu&qu2*wOs>$5-Ytk_Q(tNYVqF zQwuRr-|+%aI!`U3*^i}WHM+1oLowkH}2Ir`B3R(|M95L(|Gv@+~a2>d}{=c9HS zRy$i|s1C3}SA6e_?-zG>8_%(y4Lf3q!j!-(XG%sdjx6ZW4l7eytFWrQu)Ouf)xdB! zkkxqr-S?e5pYM}ZTSD^#ywY{Sc%!GlcKf~T)Zk1Dl0?wyYRrC*)!C$pe0ORSdHwpL zGV&+j3l&|Z&XG-omUQ;%=-6SxyApJ)R6JmT;=;)|_`il@Ma|_SKfG71xW2qKsTdVGl*eaz>*B?_nAN$hd`oDh8t3@9 zSRv7ps4P5Kqf#KBxz{_B{$`#>t6hR6;>C#5%Oax&IXDclCBuVTNO;^6l{nB?B5oy+ zcKRPNqZ0vIJ*~7&DJG9-SPG;+yHAVv#qwCcNhHv>xzr>0DSaf}k4^lSpLkRfgyzy0 zj<;LZB_)7hRHn`x*SECO@>E+v(Ss zK{7@;`kmg|{6qn2-=l_Ny9k)1{$2ML?du)3_7$m%kGu7#ep{JS!_S)pz6~yxjtNjS zgwjTx-Hi@@I|d77akYg7Gq9?rH2<~1W`_HEf6G-RHJN-fP(l{h3y%T!Jih?3Z-hpp z+4|0NyKP2gjWq9~xz)vET*WOU`f&z5_M8_3uC*1g;rF>Vt?3;)XHD`y#Xuy6KhmDa z8@9gwp5~#LTFpaoQ~1R2#m|(ojlcC-KpY2Qb;>K^(B8#=Y4V;&G*Va7gi?X93laM^ z9X}ilwY6;x8l<~;VT*5&@yF8w0}R}4)d@Vr!cM8K6tc37@s1;7lBB_6?DqToo2wm- znUhmI1vQz!kCzU5bSTeEcm3QNNf|6DfhTy_5~=g&yGAilnWV+<42CXKp33%C!WYiZ zn6*?mHJIN^6#Qend%TtRBQWUjpb^6VpBcIhHIo=L=>UQpm(5u6v7T!i0B(CkEyN3Rz6B9icR!s$l+izs5fO z=1(4qA2UzMH+-Sq!erI{?AK%UOuYR;b^znc$|FMC??=}jt0~v2 zE74>E=0mA$Zw!LPXFmZ7yH2|&Uy9rDk07PwN4yAuXAbin6ez6HtBV`S?2z2Lu+9M= z{~&~MA5Mi_N2Q7~5l=l0tayUzc5_aaRJ#1iLSJ}n`im~T-skG?QE*EOsuzCwA00c^ zH1cG;!$}0Re4P$|Bv~eMs``UU(3138J)^JzgR{}^c`fkpFhDk20M$(7(gi~%h_Zn% zAnSWTH6NGla^_1Vo)nbOb-L&-N&2k$L;iwQC98D!rjX z?oQDnbq*VHr3P)ynl3(fzhBZs^hLW>Q8qNoDE+ZQLa4xc1`^H}_Y#F^*I9Xz;pT{{ z*&`JtF2!2qc;dYU**^J*gIW`xv%O!}>3(TgvH(1=TC;kp=|x)lH|l_g{xdo!=LLlz z`@odVa@hL=AsJIk?n3v9=CS-fbB=f>gv6BmHdg+&=ewAgEpD9lZYD+Wm78kww^zyT4w4 z?{LCt{k=UR-FeSC5hWA>N^fqJ(*!qu#95K}lF-69$XZOW!9_xjkBF+!ICcWqe`(!jdiS+6dK#3xs<#MarqmU$A3Fxbj zP{>gHk$SAcr=V3LX4qr2>)sw(dhbHXSJ<$U-O^cx^Z6^&Z!Z`z+ zXQ9$d3*7nUs`*DB<~>#+IH5s4w~7@RQ@K!vlXqX;nJi}8B{F@smc0fvJ`4Kg)LUw0 zKb9~sYE=|3F!2+)yua$$y%c}_pe~K|p<~uMk6Ur)xMQacIYuq; zXpPRY*XaeH7nIwIkeK6Raa^NYA31|w4Q89?Rr&&Vg*1y{TM?DWOrg)L?V7yJN}F;A zP%pxTH~#3o^RK^we=Vt3DcDb8Mnrts0vV=-czHR6uvj_14P^7Y1Y@t_0KqC zC`4#t;UWE+E${a)-D5FgIb!1M%QK%;VD}H-UY(}J`Z>8h14@AlfdA0z5=3XfFp?aO4pf2t^S1y9SSk}-pJ2}d zC9?;NkU=1gORKK{YF|&+b`e)<00Ddwn6>6WB&-KaK?k&`g6?ObqUS&QVn*-o0ii1G z$mQUBT2&tfR^&Q>k-+hM0LgX@2pdgGLJdGkp$d@XJnD*p?*WuA|0pBY^VlvA zNwRrepK0|Zk;FX#65Asgu-@{|6e*PZs}b1 zyeTC>t%_v>Qp^YO4n-0PyeU_b>C$eUri1gfvQZw?LSEh!}728D5-zlx22 zDk~0y19WB+fW+p6ZGGT(&SN(nQ2EglhC)or8(*dKDIA4-C(z^;faBU6;CKW-YCf49 z<%lftMW7m+o91aJ8hjKU=gaftd_o?Z%-kmbhgTimcfSDbNu$9^yB(jGMZ31goWN@= z5Urp>=hM4#vA|$7`1niUiJ!7TX3{v!egPIeT*etQb+2G_qE{UKh*QW52x0zj(f!-k zM|%M3v1wEWAuJ#mGf+bk008YTI&nTs=x3>{aTmy@W!~!xMi|gnj9{9^2hiJ|`>$9e zOkD37+zEV*#ELr9%Gyf<5@+3)dRHvJ$%M0p;b2b@9n6ZoLZZh@90`7X4;rku8N}Q` zzcC=#@``?*Rz70y&u_ETB)yJwQF{^+PC$z<(C_2yrwG5mTee`SGrAyMp!Y4WT=UCw zF($s+H&Y?yLf1!Y)E5*iTy@`Dy;8^p-4$4XigxcWw1nEQu#o5voFjctwiUG_w4Oqn zQ6z3uiq0$|TYdRE2@r~C91Bjl(knTTx!=`?LZaZ8&nk8g{L);%rt=8`8 zMx$ymQ9ubbfR<7C+{R55-sqf$yXWR!9jzf)`V4WP7n%%@J58hpmr;iUz zlO!EVzA?SN*3rejd!UhYl!{Aw&|0K%P71c7^6NMMmZ#w5av+pLz+$-bwCt`LMP$5r zSU?7cCUtfQUL2`wP*(o(L7-t>FM*CDVRob5XSQ+0vur4&Dw3&~>J6U!Lr#S1ud!UZ zcZvO;5RI5__%>6 z|F5;T?y9o;zK7v(fI|p}76fUP?(RmqyBkDGLAtx8JEXe>lm_XN7Lk+|=@jAF-1q(Y z{@%bd#u@y4f=9+V^XHsW4iBCnLV;{ICw>W(;tB4iqC^X+wNl~2tfeha;nxGCs77 z-lY%*G?h=NeMYK%`q{9?4cipT5}uo?A#VSQ2Mk4?xI>K&V$+tTqloDk&9N%xM!wyz z9ekQ}+6i1{V*;1?v+}-cxYutiH^?WQr+xDSg!pv~k*T>e_&X7Aw#=+KV;tjwyaEhAf1DPR;If=6MP1 z@te>P4CQ93J^OR(xby;F6xFmR*mN51{rc>ea07=E0vF!u?1ERoO$iR1?uw(y=4&7q zEo(%E@t|+jlh~_K5)5jGmkiez`smWCi!>J-=1wG_dB&J`PNI8XTi zqB=P~M6k1YRs?QK(vzc!*5eJ_NF3i*O&*P5-+pLZYH4zdFlL&tHO}_+mSxx^WbV?1_f*|w6j>U;8tdDf(?=@RS}g3PghlFQ~W8z46ez_oZydPl<$(m^CZlePdB z*aw2L&jZ0Jvtv~`STs-1gk7QqnYU!)oBstoc6QQT?2LDn=ac%7xCGdmp{` z1Tyq)Yjh~Oxqk}oK_I+W23?QUW)wb7>3wx1J$n=u)XNW}sSb+M0+pVt=AjofsCVKB zlDgaz1EL}{)$TQ}E2N7R0atP}ammP;H{DI8dd6K?`F#l#!r4<*fKW&4eR(H zr^`y`7$q;iZOjtICqm&PdAW`bIgLxgt=~^1^4qdPk6(;s7Aei56V4pp4y(%7)1Wcg zr4*{J6a{6rDy)oG@x8uT=$7`Wam?Zz=JV`yifhNSoNFj$g+v#j|FltM5sjKc(3!yi zG5>QyOW$?tk~ zW+A#s&v3%-QrbLD^^zP@q;L$PrY`i7(zCuJYTfXBTZw7WLH0CQ*!VX6q@OW!+a&K9 z>+vOOC;6W8%lH-jzgg+IBBTOTD|xk!kJFfoNml*p!n*@$S-lf_kWZ(#HsAQrZ&i5P z_`{cQJ5;wQa$6odXH&c>iT+SPpQ1i&5%Fk+a{TIkVD}Btb*@cD(vbT{a!z4YO?T*( z_1a@$4bvX)uZ$E-87E>>R^qo&AuIURXf@)`(l)E*L@F93Clx3wgM_*tR~vMSrB4;r zx#We-?dC^%Sw2l|=WGmc0UAOcZrL)^rIN_ zE~#5bc$VSz2#Zpe<`UcNk?taq?yyN$`2-cCnUdr+1sapyHaH!sh&UnkA7KX*PBKnL zcN-AvB(N9@-;L+xdIo(YU3qelSi%OY6y|L(`kNz`$@}rg+fN*SXkH^@K_>Bg-(r1D z6oZt|*1VcsAv;~Z@G3fOtKO84J_>f~9_P(teE1;1sK| zjO!catd3`Smq)ZsGQ~%jHz^ve1o`MT2t~qRbAm%hVO=7aY-OuXu?zZFY#{|VfRSU) zo0T~-;@kxwbNq`m0><@pn5aa^;S=bqguNkVF*d#6NW|{mTrRw>48Z9( zu`Qmal0Ky96symnwcJ$PHWhiZL0Wo!tuPz4%_Xg(rqxJ<93D9RuVWw7^r>LEL!Y1+ z@WZ}H+<&AY6nXGTt1ka?<-KDeh5RCp4%}T`H9fgcieWzk1Ows9C-c-uJ z0J7*Ll|NBBygaWRb=>BMcN`flGvW`CGvq`=RzzgH*S9yA==tb70@UaKp|=mPsy^ka zF?&cidcI4AK65s~zi=l4-{%X8TT+mbz&c@p3sefyCg_24 zIU;_`4tVs!BrN8ZJktS^&=0N_u864RZr?vl%C5B@#krve5s7bpjPrl{ua(GzWS2eG ziCu8GMO-rC=*WwFMk{M7A5aC;N{t2JUn(M;Sb-*7qiUHf*7o!)Z=1VuN7DR=)3 zj&}$G)prRz%V!#_BV|R^uzjWsG=iyG<^nzO!BSMq?Xk>(MUvC?H4080WT=>-DTxFo zyH}uK5>T3@ynpm&10)_|c6`)E*6P+237x;658E%)idp+K$#C9*%zP50>z%_5ie1Af zJLGf8jn?!~y55#L>1Ri^GLPd!k$VD=8-2&AT(&x(P3oVRs(pnn(`wOtkwvTjXIZBV zHR*UD|Hh05l>*gF8nEh;2e{V;ZR7{(nBibVrGfB{*ef$YDW#U|m)7uROp2i`gimT# znGQ+9Hnpk^hN3KkbSPIqM*HNaLe=>o6x{=86Y~Iq@y1`S(fNE&^Feho?{RzOthO)V z^29D%MlSK$Bg(NH6TTE2YH1aaalNd{e+js(AGM=_GnIHaYxP`$MNF37VZsCdQ}s?v zrIrR?A>8h-#&8A~LmY=@xrSh-qINhtm)(rX3OnNmtow41^-cpcCE7Cmt?su!(GS)l z{jzfZApR7v7K#V&HOi({F2|jxFijw8q60=kZF2<|9)Ydc?^|X+tSe}N65y9$lwF0Yq|44MaK_lit=e zayl%f0V0UdcCk_Eb60kAiE;xS41k5n%6nLF33ISclWZU=hRxW$}s`ZPKpm*}ui zG%J31K7d2_n2Eh?rfUcJWHSkQ$F!^Tl2e_h@E9(K`3>E z7#&LR_-asGY>?oAAzN&Xrr&-?ZAgCyp$b_U0KRjz1((5ScsfB)@=7p2>#b0sFn~G9 zk_ATGXqvC?;2?*S zHYH&+?hq_lD*zy9<*dZ#0J^PhEvxHm++8O9=Ii=Gd<;gYiOfKfPA(Lit9ZYMNb~`8 zGC+C|T}}fse^q#5dyZ5%JOI!jELcVgrUBO4%`vsA5cAE$>Z@dl@826~cB{z;o57)% z00U1e?LkHN)c_3c&G0>GQx+`InY^mK@69ZKbOX}5{Lrk#uV}>bhNh==e9 z3cxZEa#_Bcl)YeB=YRxM=byA#D zl~0nq4wowgu&s~=q+M`fq75RYG7dqt%F#q`dv)wz2UyvT61@yru;qh*T zGQz&|IFX34XW=rlpWiw5xO168TE$-=+mx{?MSEQxt&v(mg5iNewO!;6Hp2O{;_ZNk zu;G2}gc;zNv^ch&DgTyjA=)$!<;Wc-wS401KT8WMhpt{`|}<-L2e>J&|^H>tPt zVx51&@Z!YBIaOfu-nVB(9q);K|8~$P zu`|9MIhH$`3LE}D7xA^?B;moPYg|n9KCf3}w!_DZN7U%KD2A9roiwu8Ezv7NaPo<;V*i;!XE4B`7HZVEdiovR zJH&(yAgUaIN)(-{5P*j6D>P|9xz0^mcQA(8P$dt{#H=aus*G$fCO$q72Y#O|Q5~d6 z3a?aNg<=qm??qBdgieC;%ePkCX=@x)q6;@(E;IVN@Ozx2By!AqYDz9%U^1(+_BeKO zQY#%fcqk?9XU~Wo!gX~)Y}hehQ%mW;V|E)(THK$ z`g70^!BRY?SpI#{#EJ0d$g!EM*j*7glIYXUYhAD^0;`xj1@Mr6xQtfM*!J~wi*Wos zR5&y&i<=^v7f0|5t~Jg0{A_Dda!sJZoKy_Sk`U_Nhw_@EQB6mbXrz!L6qj)13yN9j z3euIqn&-Ro&+KvfKqH$c*`=(s%ZhDF@J7YWXz@Y-qC<$-jZgv{Nipme=Yn^ej#{Vi zuvxuO)ZejTe9`K)^s3U{4nHPP~Ob zhrfW+@eY{vULt#)zT47ndJ9`ergUj`gw|!i5%7M3Dwt-Qdu<1n`XQS-{3rVU8uAzN z7v9)s@1&jy?+R?BdrA7uC+5{I=fyga;}<-7QSz2jL(KWoAWyOK2lO7jkM!7q{tyHw z={ly}%CK;P6mij?Mt$whlA@(>Nu`MolfiQLr*romWM$K{BkRcd#<`si=O3+BBe zNSWb)nTN1~Y_BH}a##w`?L*A!fKKY$LQ(<{vu~UM$>?b)MZVtTQ!|DD1InF(oS33h zq*pba3|9_-`NWuHSYSk{u_wGkgh17`mx}BVcfG*nkb2hh9O^XTEpH~;RXhC2sEkx( zC+%gh4%0FY`w!H{pbK~jvU-9=6wI@*Lkt(yms+4f>Vx&)r3XPG~f-}$-TDWdgBFlKAF;`4g@pn z?p$mL5nftukX)dGLat5p)X&fUa!U*8 zj~5bqh|Lm(wvz+TKzUHE=I&IHu2+apq3run!_$+;EUdR$t8r<%!7Axrk485DpUl|? z+X=N{m9Ia6hfb}K^|`<%?;wmox+EfTPQ!K#f9iashwg=C92;@EEq6?~1+vzC`T=L8JDSddOu!3MMf-Yb&ICIDs}wsTh=y zP6`RL{gFM{z9N5Ep%yRY_hDzj$VlDrMx z0^Be%A$5^_3O#UzUg^AUV3YAgQm%k1SdJQi$X$PNx*{=v;E}CSyF>Q`1u+;0(MZAf zx8J=F2c)W~=MN<~^3JPx9C}U%`3(CBRcT2$0^}&w|CIkO0svE?VgbeIA~_%(F`dqf zr}NN)A_9uqA?lSOB`MOK$r?{LL%<=eFUwIvGOvbF{!hN$Ni|T*W*_t};Y<*dD}DRi zf(3R(h?s^1PbO0;1@+>s5g_L664nA1zi0`iqa6pIlR%;t;4g+yoDJeCPkmj0uJUrq z1G9|iWRkYed9rKb7T0IG7{3kqhWmzc`9knjinTOTDG}i*JZBI{Hm*?oxjp6_K)11i%hLIOBtR##ZefoXy0x43(=K}NM zBi#@_6oQ4wC@}p&3WMM4g7Gg%cu47pL~5Nh6hBaHa%(u-f@m4X>5-h`1G<3=Gci=* zNbl*xA4RwWZr)`;eZ&dekd3m)D%efPRB;}7>##KkvXnbd6v!1(c>xpk&z^-ka}TRk z;Qau5LOKruXk}mpz2UBz!)8YuLIL;2aR+L>^TU!V@&6v{^pf!ElSM8(Rv=5P zWuo2WM-Z_5ZLH>nR||MwIT;xcY;^(gUlfuEiy6_oMTX4=^JHN%!7 z4-rQF2A=i!beC!(8e6*L(i31q`rGF|(C`otT;u>_W&5#Kcw6+UUwC;Na1|QP6%nx< z%SZ+=niR;c(cKmX{Q-q=^%c~tKU|4;s`$oE{yirfQHR$d;*SABcuKB2w)3GN$)Markd^Eyy={SzqL?whsE+fFL|`1B@j8aVhVP8hdw4zFd9d0|fi z^3g7E){*ANs{YX;1zgHY)U8=2XZLxkPJYttJW#9j!&8Vh)z7VZOfwY?}~Ir5vCYuVr+s^M5z5O+#OCsn-zbzrJ@mcEm-j6aaL+$H#N(XBT9!S!PXE>`# zLkW`^MQ>V`0v<7x+Wy#UJe*YBoAiPKPk0tCX%q2uBR@v{K-r3x5o&o-e!?{dCOok0H!%e@G$wg2`T}0*` zS4x00CjX*PDkDqqFs)*u$mtM0)#gyU~{?I6B#pf z<}u6znsr1M$o{|CqfDoJcpdFDpr93GaC^d0fdziq)2hYVjL`PoD{jxuifA8RBjxJ0>HL?BtH7UG6KB@rU9qZ<27p~ zbvP5)XdYN&4_a!eD8exZvV;dmpx<$#e@T+dj8nOnv1q_k<@{i*Z8upg(DYg@b*kj$ z(B|4_Xl`llHdB#qq>k-ZO8q=G{^~}}p+uS~-8SVE8cNgCsH{%!kjS4Cz}MW{L&$n8 zEtX0qQZ}{5_jq3};YEOP`QD{8NA&FM8{jQphVIq~PpdM@SlaQ<50@QzZ(&ahb z7peF?j?2-l3ltLQ@TO~(9?`i@vG|bkyyN@Z;6W~=)?sRiu~|6RG^`5D>F(#3J-X(- z+x>1;BZSOoaTw|@=aOzm8_iPG@F`#0X}y}!(vy0RQZ=hZ_u5aBda+ZD%Z_uVSVX@x z8SYhB8vOYFl+U|{0aq*`^RWFot0Tv|^HN~KBlnMsh|l<&9CMHUz!mY$hQm8$=KD8_ zdvm2P62n|m8L=f_za3S)cE2&u?eGngTxp|oE!U3ELkvh5pbgvf<@hxzVHs*vI)rBVgV!<;xPAWhz1sU1K(Ar5Dy5>HiZsZ&Xb?w(qt7=KVIa zTou;eFG9x5>3@g*oE(3?bKwbf>tJ|P^$$Sn{-006IJl6~! zP3G^iy?MRnDqT*_dUb=3+_OJBeN9NaLO#dB-|*Wh*JNh#!Xm}R)FeUfjXa9fR6VSh z`u653*jMqZc*KBJcDuWjr>jkI$dq4saw68)zP#f-Vg0>quy23N&SNw!v%RRe<>dSy zi!Xf+<=9^-7!M^fgk`B;l5E}=$G~AtfGZ|j5h`47e)w0DO}zOqxk>+Wa0cg4*5s9l z$B&wijJbSzv65L|j}7|&S}h2l(PiXwS0bS;_#@oa;6c#cfa5^>rt++Kt9vyAbkaIV@0fr_6hbFva$K)T-_+Hc&%Xao;m#uXm&jRj^xt3d=^R|L z9;F7GnL`_eYVB6Ig0E9BfByhh%8y^LG-Ji5>bs`+^XLtA_sUqT6E$pZ9~SS+AUAp` zKAX7GGO}Ng^L+OxUOu&Euu)fJFwR(cYPn2%@6X^4+H_GV$%m25=glPi1{4nT$w8_o zKgWySg~esl7z%V!wnP5T!Z*B+ZKsqx-(kd0(^o-+-yO_o4X4mMETw$VuL*RxxyG>` zmw!0{QT8SZg^rZwx6|%_97Lr3&=Z?GBen(iKe{De|0SY1qMNg!+okAx^)`O9LH?pr zt?Pg?5uH|Wvw8C1=Jt23dB#Ru=N;wgOteIS#Yn0GjTWrmmcCNT9@{oy_tLIxWg$hGv{ML9= z8q`fb_TG)-VB8M7*rhB8yGWc={GRkW`DH=a z%g!9fmm#RmdTIGQuD4gMw-KT8FTL!nsd?Nv~B5HQe|wkKS0l zmwt1n%AGgAztG;|U&#{v%V1qK`o0&~gizJj1o~VQHYvZB?wdv9W3srltBpD-3FJL! zhii3(9FLvGm|*rR6=t8!l1iLMQ%@}h`}hL$#W@Jj={e3(_RaVqX*l8DT2FW!lBi{3 z2TN583W16P(1HlsKbnAQOW(+dJm6nZ0Y6So!@s(exD<%E?oT1DhV>)sw&W4kV(&b( zX)hU`Mmgg=fV{K^j;93fy}l=4RE8!KslOD3#~nzxaCufB_~(m^(F z%x?KMMepEGlnj}lkiymMz~Sz~3!iShJB#ro)f8_wwG#ExJ5R?U{f2Bt!)DEn>%v^o z(e$!H^=gh}uD}qe?5Zxz*T^(wq9Fi<_#Xr5&?=~%)JgU1LDuokGU3xvK&)%b)~n8&(o zYH(lpx^+v4M&XNxP}$Yh)y~jM*uKBzXqsxtsA#r|-#)kCF5j7oPz2V*dr=4!^rWJZ zB_VnGwEB&K>0eQ8ZSB7vJ;7+6t(m|KVq`L3!gcNf`n0zfyd5+qM9{XRl3{+Z)D^9L zttaFUeGJ}w?eSl6;dEI+I(3BVXk@+4G$)O|sIlo~Dk|w^JUT?5sr~&h?dGei)R|0} zx`A|6F7Bxiw2Tns#gyIM-D(BWj{;gqIBnwX0pOM3*Z-Sj1U(S}4a__mS^q$KF-}1- z$b>Y)@7H}Apr4T<6*V=;tGMFAD}XS@_BiC%FSsGVZ#1T39fiKjyK_DKFy5aI4V=AR zUx&`?If652)?*me(lf;aYH%q49~lY3AUazN1V@|g4c3|DqoI+kuE^qHe7HqX$i++l zz+vG(F-;$-A#m--gu!8ZAp5L0AHX6O_HIdH&{m2n1qYA=)%bIgAW-t0qQdMA`QMKv zz>j@zBDq}Q5MYg(0vLgNTu9jD!jKEjho9u7O)(}b6AfLf{?Zd&}|bGt9av@klD;@LC)%F zQ)@mD>xq^HR5Oe6r#i|H(?XU5+H$6lv1`XY^9vd;2}1tyMKg2rChNUU-76n_z=Y$Hm5%K&xN!8fdCKUsnMP-ZlW)8ANPM=#@#kgu=;CS&A5(ygWH`vhgi`Zp6xn z&4Ys~&uxL9`cO71MI;Cze)KemLas1#vp zg99muAygDo$D(t4p4nu};?t4Dx02~Yn-3S83k=V;+L>2j&vyb2`RzbC4z#-dDg{ zXwzjj0X*|wZe<)h|1)`4pf5^)eZJT6!j;8Egi%8)dsX7^?SLDVo&lxAA7DAP2@0G1 zM1(G8`6hJPpaN;*jQk!s0J(bcz#T?$Ptc}P8-Rqm#}Kc9 z--Eu)U=M1G6{%z2K-W5oO9DAy|5XGAQmx(7H46A(D&%gEVkYpIrCtI7l8U#!QtI)tH~1)&jA7Wf0m_;WTk;;^7RV%)y`_s#;@cW6;a=FzO4` zU@m(7;+ji`DoA&YjVS^HmGZGtJqwFMDFf4Il%X)dNPNv_We~<>g)458%%racs0Srg z8f9t_#D4LA))G;$96a@maHT;Mpiu!GtpWh?xJT^|u+G$9uc4?l`Ez-g3PjDXI8r9M zAHiG(*!e@d0v8@y*_jYO?X&-9*!|PNyYtUditFSA0XxlHl(6}lb%BYMsg6$C_?GW# z7a~(FMSZO|7G&LYP8{QMb0XJxg0b(H_^Y6jk)eqDN@Dv1XRrM^rv7i=%m?bmd;C1E zXde~|O-Pz<_?mE;r*JMge*|Prh>3d|c->vmD16Zo_gVJ?&(V8( z!FbBXNER;S@Vm-+eO&z!Sz}^qBVhsh(aElp2pSJId`a+xu72SRuCG(F1lqqslkUJW zZMu$hXigvk@~pb(IxTpEwJPGi)&Gkz~{N+w3? z&%YN*GyI?YOdYK)mj{#uxi=nKIJG;AO2ew9-rVv$R?C7;@wu`D} zxWSxJ_SHo}K>@NEYN(Hn9Z_FmK*XAE5YgDez}!v4T6UCU)aHYkIAe8nbsQJkq5GeM zW>*D*P!;uh%)P1plPiKJZZ=|EZ8BbTTD+Z`Ed__1HHS>B1)?fVo?m2^&;4e3`(Bem z@HL$MkEkD(8yX%&_wZHS09tJ}LCJpZYSf{{UZ@z+B}Z&`L+~eEud2#+-F`mzuJzME z;$=32R8r(~v380cif1u_=M*K%xz+QboUuh)gy&hUPOXdvbz-r({0!9|28Fgih%9Sy z%>cNP@oI7u?&{la(22uj8AXtSbNDviZc|QB>$Tu3lOsmHcXVL2LEIU;Vh zKivwx4ki`wkk)JG(F=8Z&CtPBF}P)jTKHW?omROzYpQZNiV$T%x5c|1>U-2WO8kAo z;S2tvm>4qpe*s=GrGEs{<6+5u?{NAC0(bM`m>s&inw!1y*(z1Bl)rU~ZXrJ={5~Ga zw%{}TWUeB)ku8oUF4X(BK2@tl5E0XEZW}qJb~0YNs~7$E_;1O&=8_}E{P;Wys6YY_9WTtD5NkpiE_&=T@|#x@*Ws8o+x{GU2UI8 z9Eo}yZ0bF8a9-Ce$dk*$sR8TyL-a)%5FZaezP#iqN`r(7XY23f3y%g%IGuO&KK~hy z3r`-wpHAYZ%Rm3SKD(}3QowmuMj!J^J2gd)@#pBB)1%>C5$5EP;Y_@RPx(J0=r~UZ zXyNU%_5-Ny}0BtmnZsqbobiy`LZS?Fk)9RdQEw3p@JnV{&}P zoTt@Og-VY$YJWH+TCaY0$ef7$H?mKQ5%m_^wJP5T{uY&_vcqg#CVHU5HNtk}DpIB~ zJfikxk3PlbFrh}nF2v16)x*V;URmRMOX%UkrJYsR?S@f>EH0EQe#X7_itcr5_)#yQ zAn1c+u~Wz=XONSKvYcFKnD{B>#(FH_Nr+k22n~4tdwC04q7R6t(;hbar+3f4@j50zv42r9`Mesw9#M6X zL7oA7FGXk@3bdozydd^oSjJ3aFT!KwZgTq=Rl*d&-d~L}ne7JmSDvt?yt%05u)8rJ zurM~jV*aCN6|>9W*kt#jnS4}O>Mu5F1RQhQ)ozX9^!cOr9Q)hD3|tH&N%wbVwnoY9 zQs>t%B|Z&X1HholtA}y>mvb11CcI)?pZ)WQWeFhTN=gayv|(VzOHIf|$Fx@k)@@g$ z+Y|lVs?sqmVdcVHuyv~aLhX9fW7&%1BQJ-4>|QyZsJVIrDnk}Wec$4;IZgv^t6MkK zX1F3+J><3DRFVAdWfTfR#G-=vd>ww|8EQMzu(swZ=HCBf>2XYK_NzkK0IcS?V( z0v~_Ka6C&~T7Erq;j_C~{J!zkt&{Aijbg_V-)2Mp4~yaq;ueQ}hmG3Z=D#cUOyiLu z70O;i#-D>QRRf(q$ay$PJlZe~HU739Vmae=$L`)^KXj@lK&MlaXz|uj^RUeDAM2VBvE7TsH$IM5#kp6@_u^>Gw=DtU1mi!=Dh@f%k zyJtZe$l<2Uq<;i%8?pEYMnwzkG13{8=%d8?lwEt|E@g3WvBQd)5wP)Kr%=lEs%2 zXx|jKgejc!QKBF(od~N~QHY9JVz>A4Cp@HD51+J*hmhxUSTglL*YdD%pn^T=|EB}RSJ-Cz`GW#xBI! zYZc?11}C_)t_lC|Hc+35ifwjfj}*WE-)$^}g4=k_d&>0R5de~<03z=IfmFr+cVmx0 zIRD*gOaEaX`sYKMhG2co=d@jK`|z{X9q3ehyb(#G@5tB8p$nnsh~6UhnjnRc$~Xi8 zAs8B2H^}}SO!30{B!K2f5X$@WQgFr{`@fL{gQ)`&#QMN)!~eU`2Y?D+V?rY4^54;Z z`0GyqE`#Cm5{Q=mef}UzeIS(nlvY=vC8i|*2ZNxA&w?gw3^4KS_gWV{f?O@rZhn># zOXlu)ZcqEa^8xH8fxh@i2fZA+dcyj@|N3v>`KNiVg%CB;O6jm+CvW&^FNi;!sg?oK zAEyU^w72btATnjeH?I&m!bYPBK7GeZyvUv5zXQ@FBlyOe z0GdKF0N!Nn?JK`HtaMbxt&#eDOpwdC4Dw$~4@?W>6A*aCA^fCg8EzDzS`bFhCN8M_ zkpIw_s6+a(bwT?Crgbz4cDOn#ek6USTSRaB=vAy~)FCmZIJ&;~yh@IiEV0vzQcT>X zf8R_HA#ccxUa3liIAMyo%`MZRAE1@IML+c3p@+`{*-*Rrx6>qIV}|1ED%^sF{5(3JUV9nRt0N^?WWJM~DH|1ZpU}p8xO0(n7mV?@H5)5i4ag zsJj$)ZL3GVnf&j43HS+K?BpfMYr+5f52p-(z{Xtpo+$itU*Oi_mp~eykoh84=Kub) r^?)t3GFu1zy9$GsJ0Y{{9_cnK$N&6iV>9qjKzJ#oBv~zP68QfC2l@U& literal 28011 zcmb?>V|1lW)9#LKO>En?ZQI7gww;M>CllM4U}AH^i8--z_B_vrcdfI|zmp$Xz3=Yo z?yBytx(ZQB3X%x0IIsW!06|(xOa%Y{#RRrrph1EEo%u4=007tyYf(`pX;D!kC1*zq zYddoQKnkHY%|i`s4Qq5;&bt^QQQ^1;4jftP7zDipwg~l`kQH+{I7$O?3~hZ;q*PO3 zj0!j{QA~8CiFkwYHn7pjfN59pMo~Y^-*>z5>F+r0bL+M3aWw7cahU4@LUd&c7lvJr z3djU)pdy)#XeDCE3-2_O_-rm<)eEJAyPE70p z0AZvP(wlS&6oaFQ27@Tk!+?S>kq+H@bScbO#_A}jM1$Nc3Ymwf*p%|Om(mb~c-PJ>80GLxVW<`&4(V(6dk2o|9KjaIDu&4_OT8 zZ;tL7!dHyj;$VxnNO5K7?B9!k<=AN~h;cXLkwrm%j1VbQ;u$bSWcnH6^b5)v`fIQc zBJ>U`A{;IdpB(_+a5r89o#%GxAV$$5dT+*vm;zuXYXsHFuGTvK0Z~E7JzsgY60H?Wf9suO95Ir?!JyFs8r*l^lrK zLHESYfy3(0-jQiXbO_(rVZAng;-CvN9Nec0Bq>}}hNQwp$3KXH75!2CO_^lTMh&G3 zR6W8**sbVAiBnl+k#^CN4O3oxVV1)R$rYIgXG)k+v`!+upn48!*8bSwPW}$r8~z<% zbl}*ihUxMPrUeKEL=&EBWM%)NX}syQv4g1u6Jfnp4bL+CSv11ni19)_Z3Fo#{;Gx@ zr#pZ*7JqctDT|9Xhmrw)bR_C)SUZg#>m}PIqyzXPjN;(gKK>t&C(mv`enMS8=bqxO z<4*oACA17US}-qhY(IGoQ3~n<3J=T*G-)VI5zAccv2-j+8OjO@Z4~{m%W%Z72N~*k z;%|v8MH%v1a`kVBNeqeMQu*UFX0nZp+(C52jwDx<9?4fp+sahvs<0ubiYTfa$~aV( z)Ctu2=oPT7uyhdL2XVr1!Z^ZkVe2qpqA8-~e*B2;h{lMP#Hge0RUx6uqf5q{>V0~j%^z@CIC0Vw1?uzO)~1j=kiw9tA^ecb@#ye4#vG;|w`|Cs+V4y~ ze82S{WHNEWzK^w(gR{VC%6wiQ-IDz6vUyZo*L6j+?I4(9SYViB*w+^w&>Yb6xepi$ zObILtY!Eu?PYwDdoKcWiKrb9Zr;i!2)QoHhoNz>h)) zLwRE+F%RfE+qG=Jj_f7NqNmrV!j?{KrrYasP{5)rSqf{%H1Z5rN$om|n=m{}v!q<4>LuvhsGZDmc(74&STkF!+ zn!4~a=I6GfZsUc?kMn5#ji?RsjndS@RDAUj_3Y|ijoWI+m8Zq#<Q!pQcBFUsz3bSDG(%2=_YoKkm05;2va=#PE*o zo$Rfw7Om;*gR{xxi>F|wR*+us&#fA)*A}X_bbO)lzTtU)<@$4-OG+&19Y3GYipR>W z;gjRtU4dYuJIAp+?U?R9>;|6$EikumGUxTixv~A=f$h4KqsFW49qn1Qv-taDb7BA2 z;H9Re$~62V-HDfz`@ouR&#lkvOju5rlfcf~nt;Xj30xjlCqt(p-`6Pr%J;^JgI3yU z&3Hq%UhnSh(!PtVvzGWXhZSC^L-?G|UBCTF`M&HU#|8)eUjD0x{@i0;4=6OqQjZRg z{OyVM$3y>z_b4b^LK6Z^eid)EKUzm$e_dhrwaMt0P)59kf9H$xy)nvQeY;7h5$uge z=PTu_^(_0W+3ncB{XQ2scgBd>`^Agem-|oEwc>%GtzmYbzysEP9jz(ch{RUpmRv8r zSNXHU=j+;)&&bdyVZhiM)1QXpv#yEdXEmD1I^DRo$*sj(&2#L4~K^hS@;iiUI&2xz=i0u3GYP zJf@EJ48~@TCgu#D_D(>M3jpwW@&KFm=B~y>p7wSQE#-y0k;2sW+Wl{XNapU zKZ%yS5|OB*vpEqv12Y2)i2y7S5fPuWnFWuEn8bgP1Ap<8Sh>17@h~!acz7^)urfG0 zTQV|pb8|B?u`sf*&;v)%yLdUc8hg?^xRCxk$$!ryX6|C@Z0+P~?dU-CcV1%?M>kh~ z5|Y0Q{mgtKAcKs5dl;D+m>B*DM)}H2eT4L7r<_<2v zH3ZmM+4%k$|9|!T?~4CHsr|o{EbL7GN%$2{QD*WF4`ZH8^S3SUQixi=3&eh{?W1GyEK*B5kVsynTJofE3*q7~;;);_h?F zyXBjC&ik6T%{|lMT2)aY#AO7|1o{Oe79dfu3;kBTEA9jXYy(IGQTm~gp=X)k|LvPX z>0cnW-vURanXOc%RF<(55u5A(`j<)_W#{&}qocz}upY)*eG`{lPFuS}v(CUsq(bOV zKShbM)}9D(4j~t!Yv2pQ<8iIwUf6?r+PGY2&3!m)#~^=A!nNxzs}r(VyGD)XP?)TAZrfuKt= zvMZvhe^S1$8`%RdTF^rnK!2ehv2fz6qJHcm^4=DYA1Jdn)w$`L=5oBgqt0o$CN2KgrlB>geO5zScU(Sy z%n7SNSoK)LMCS)HvC}gWd9dYn19OsY8(FOJln<(}g8;;TPjw09=_o%SCpUMuWXqd8 zJr4wT@Qxb1Xz+_)XfRifJ#>RFcj7e%WDylf9be3ajC<<~9cFUu>M$Bav|j<>jTH3k z&VXY6k^XxH)l7^T3N;6_SE&#h_*+m_Rf(mUL0qu3QW*`pi*HnL4m5J{6^^n(3D)Z% z)i7mLWN#@~NT#W>iL?X8!OZGnaa^u`DNm$$BL;2m4w-#neSN*{%J&2Ue(#5KY`itN zBYKhBpl?#^=s#ZmK5cw(ka0OFaZF=TE$LE>AIy2PewONL$lG#;#~=Irg`y9Ww2G89 zfCZS8P-&;Q&}sg}dW@GKX(J38E-^*IjC3sNFW6<4JSlZ7BE8a~1CvZ)OBO9%^(~Cp{1xP~Hx7htu&O~cB;97KJRvuZ_aVtoeo(e<%^lUpO-F&i zoUI}xtrMh>l89%%ttL+?4~nuqqFKn*D-c|%(9ROo4wgSr@{w|EXd0-bvtLmPtPvy9 zU&qdpG())DYj%w&)Lci;NZrNmTA&ddIp)%;UF8C_Q%lQWijg3GE=;Ihl0@S#OvBuq z<9n!XIW@jo_WERsgm!ssKYF-1oA@81^hV=Ah-W5#yRBZ3-aIOwJ5M=)=P+IVL**-1a6fjN9mW=L(VM8 zh6+g>itwBocCNk5od5% zBq^7`xR`8;?C_{AFp4mi6hg#l^LQd+?-tgq6J|>}3#mo+Tra_;(|1DUOczzyL3+b=PKm@lM z$`~{81f5cIxHZ4!CnGm0>1pH!^eft!LxVu>fZlm%5k8hZ;?&qw9Dk!t4b)^QN+H1s zdJw#?OiUBxSrAa7u1w%$uH|4fi_sK^tzU>me&h%p8)*0b=An|JpSl6jRl?w#K|X7t zlH{aKjMgj(am>|A690`pg0|(PK}6T;osBM*McT-;5T#QF@!i8%IJlQ4@zT|XS>0U4 znrFACZoY(1pnkqgh-88K8vA4y>fOW_CY=ZS?!rz;9oMEXo$Ew>u;iqR!NZ$UsjdLRGp}0)vU};enns9)wNfgJ1sO8e;bQ6K<<&k=KOof>>zzUSuce~_W#e(3 zi8B>3tx)=i8NsMCh?dYqsg}$YIi0MS*2KxLXB5NHg*XfS=hB&hg4l_e6=l|c`$t;} z;h^*bZSw!W?ruggiYyq~D3oI-=<~Iv>>=WKN;}TorD{#aJi+vjQ!5)AgXng;!o1Z` zDQI+xzxGQ9BS;YGlY7C}*%0A0=RpJGm&o9BV?M%L>(TE&raF-UP4#}7TFv`AU&pnL z(MY&8GEjk%wc-coaXV%6tE$ZVu5cD*XDpqNZ{|^44B=ANTAxGF5b!|KCrknUDv|}x z$q@Sw?>sD^tD&yHmq*=3fy*+1%8LN96DbBF;7tDltDyK;00%LC$Nkq|%zt5)P%<(Q z6G>n14*qLbNdKY*Ne4eYkAHy&88{F{q58Hn{c{Y*W6(Q(|4V%$Qb5J{DE-l;c;L*> z_KLJ*#3s@a+|AefO-IoDSt7+=48=|a2Ly*97`U@$X3#eeFE73zybD<*4F=>fX77Lv z(ZV=(+vX>vOx*O1%h*p91CN(0%hskBzskMznoiSvk+{R}+DDouoFxufhm+PB?o-s* zw}b!i6(>&gezg>Swr7*(Zl2x&UzfS6HzSUHd7GdB_drm(sD~&1+`Yy#<8C<;h#z!$ z3*VB0eD$zB8eq`3&=~Ofw0$Rj(uU`4DsX zd^cM^LDCR{Sj)K{`#sslCw?GMoFQS+*+W&vNU(|6^Kk5jjbrz;kP{_Jb*3>!+wPJ( z6dzDJuj#YSwqIDaoj9C-DWTVuao4Q|jAd4ilj+OMuDDg_gLW;PjY1un05>=v1wcZ$ zC|s&rX?thyy^wsvoi#+WXpnJ6MS6w!HT^6_LoXEv&x_|f@>2abZR#iYndp(998m$18(}9G~ zAu@BN59VMC?GS~|S!8tGGTd%!;E??gQLd4O1zW|bGI>B^=5SwZ)%G!UcabEnxDxnI z<&C@^Fb_LgFDGuE(mrb0+9BS*uh@U3{DhdBdA4~KphV~Q+p4YRF>QlLEhtFkv4*vw zOG)R&CZ_!8I0-5rwF(`AE_+h1ffbCpphlE+(y}IQtX7j$nzaEY37uLl0(haM&_J1B zcmIHc3iQc6A$7`-w@kOKbl~8g2gECl_n4B>;Z1j|Y7`cZ?q)ap=Ulm`Z{r{EG8$Ft zKndr^&Az^~M&L3!$xGfTdi0dNb2+c{Jicx6!fEj8jTgltd*}XcTBKRW*2d(C{*Iy2vLMGn{e7=aNyVPly#kN z(08`>{wnWt&vibkSfs5kbT=m3ZUk_*B5>^fI4AC0?m?uxiJFPkW^|O>2ngy%cenvG z16>ox*QA^ui}ccL+I|X-_YJz2`^GwFE7ZL@HE~^}6ppiGunHUK-an1e^BJwiVzFu! z1w-e5p4Ic*Se%&}9v+na>h>e2p1r-@K?i0VtBW^Z*p^}+Zs-C*CAnp{=Mn#Hi zVSb4}UsqRC4@9oIIn<^Q3my`IEmF=5<`X2vE(EEQL5BaD^#!PyNCMlrfZWegK=)G( zypZ`+$#AbplBgNPvq8CosHcQ$k6tOT>G=T)u^UD)s|TKlZa<{Q+THxH-VLw`DZ-0( zPh{p*O}#R*8}mNE4m6Js-1np4RCX*OqiT`z&vz!CE65(l{gOD%|9PwZZ1L~6^Ik7Y^_+# zp{0ovWMB>?Z4rvgreW8Y}r21925(*o~0Kji03ETSVlL_ z|Erk2PymL|3wGxTCWa4PRqo}X<4OEE1#hf>w4V0Z=R$5!SXOsz3OMS! zHeHNvL_LfdrBsn6D)&qTt)1%+M9K+8SB)25aS8`%<{LXlP}RXr`q=LzNt<0Xk&jYx zo!*za&~V-JelmHiP4mM&n0DGuJFqfENF`?e(WHBcO?g4Wj!`;@Xit>@~!mZ6hdz`4F}U!lm%nVKzep8HllU!300foca*~7kIsW@*Ih4LaA*Z+Ni?FgezoTP*u@Sx+=n#$x zNlc?W;r{p0LKz>i614HqTYITY!LRL27P{MnreyJt!};aa%RN!Ci*W^S)qulUSWJ=KB*IHCyi)@ ze}>q5V`ZccXavI*EKZ8d4oNW^oDyIs{jTjPn)B0;^68jB1dY^SCpvyh%C zBW@kF7lW+$H*svSqjT$W!FS$`mh^V_yJ2!BO?xc_RWMb;+(Sf+J}35$TKc>Uzlk?6 zukK#eyOJ2c(Jk}lwf-^2fe`XNy*Z$rEd=9ilBt2*xD-%6cWClF)GG>XeeO73^Najq zZ8YYHgFpw%{il@Y#kYOb@i31 z%Gb|HT+9$VB+P!d?F&G{M$;hB_cR&^o?3AE(8Iw~KkqgnpD-g=78<2pwfr#Jxi%e+ z^IP7>hi61_c>X%!@orrmV-;)T z*X9pwa;e2gVtnHGu5CcZUD=-)n8xpmo7kCi{Rs{hZ~?Rp+8TE#c1xTxKXb!+T>1#< z_E@Kgb0w~7N&wk>HLc1uT~^=M@w4qby~{w3BlSp!v^5sEC=m zo{^lNRgVn^)ZIRTT`G7OIFJhH@~8k2v1pTcH>y^x61`K09zA_+dwV;BekaqH*y&u} ziHV8NGsDks^_NSYyHNkYG@=;j*#XbUIbGq>?8rX_(8u~}!-@09BqI?93;mJCzb=#O zd3%0wGJhwM0AgV(Mn**$85vpGWQl`%1t83nBt{htg8|~fAPOQSEv=_|8a~f!B6fCl zB4pRo#qrI}ZWz6BOP#_WtbXQc^@a8vHZ|1xV z)cop*Jf`Gk%ucp;xutoQ$^oJC0z!7@8Yc3;`dtlNA&(E{0gPt>u|3ylT%=kU?1d9! z0F&YK(r@>|hx!{FFy{8(gUP@LhlW<_43X%3{rsAmn%W$+n?VPePo(|o_3QF^ zyxf6P*3d{Jo@%Hv6bC4p`C*XA{f&OI5rAa0@;Lv7j+1~xKnO#_z@3St%k;eZ5dIVj zExC1CRj3YVT3NwWDUPSEJn8!Mkx0OcUws~@obJ6Sf#_3>P>*X`i|DjgIQRKsWorwH$=QW2}vgm;=X>CaZz zBwjbi@+XC+I@i;;<39@&gTa2p^5Do=e?638qU+0FyjsCO_Ww}GXEpmgIxwSiRq7J~ z{g5sz2|bwDKsI6kRL4vfJ1r%p<@L@0F)?wGY<-Eee2pwC(Y2dlWV3frR^biH0I0+z zOCcpN8bTztvtk$!7Zs(ftjrzo;oIi$EA)hG{sXyUA=LNf?j%3pqwpN6mBfDOIyySK zJ_VvAgm|E}q*e!H!X6pa=i&VMc1Cd518yJbOBh=P9Yq>9_;7Y&m&X+j4vtE0`lNd0 zFDm#npW#RpPKV7vxai8jTHpoWe9%dnNN23mZi2Kq+TUmWx;OlwBd^Cu47w)|%{4GN zIcbDUQ!3U#w^^1E@ zb06#;PL#M%EDVZl1lk5Hh(JCPDbkmSK<;x$)6JwA@BV$I5?1aWe(!t8sGJ> z@$nIk3&YO~;WFzjRN5^>mnzn$S*y&=Oy23&WRQZbJX zUp@J%yi(YvXub{i^n2NIohGA9L$IBKrkB%$47!-DvthUBq-uFdMShlXh6*0>N27b_lRG_(FCF+(+NIq8C zkCuTQ#^5~2pqZJO^;XDfMgs>SPRMC@OL|HaMXGZlvmePCT;b+4G{wXG0!>r&pacOT z$5RqL9ZmWSDKu2wvDeRQt~x?Abydilwu8J4-s|w?xj#;gVs$tJ(V&P=R*ibgK)4wAM{$ongINB4!`5;)p9ElSsqIPD+jTe?Cw7iH)801 zRJtz#{ZzlZu6kczulkIC7>UZX!PIuIy{x8tby#yc8jg?4$I7+`&Z;Si6~+*V@M3A7 zv$3u5Ev=ee#25A$VJuTZKt`{f5`-jPR)N`@2?}vz&vrmiG(PFI@3Q2>iO^vo*sb(IVn${7rU-AX8>aXvO5At2gfIQ*gUH6LkE@7 zI$k2x2qDti+Ep{Bm%Ez)?twj?ix9XgSPC%>g=y`?%klQ+CeNH)9I~2SS3fBr(7yTQX4oMu7lsDcp%1SCHGkkRw3xWKa-K{39A>=mw|2d3 zw0U%+o<{vY^#0$VsFG<7rD_EU8QD#L4X!(svPB-EfTaQK;YaY`uG$Y`%uGnlw9z zy#=w{p4#S9G?1=j-J^_%5LCyuH)ac@3RH5pvl<+|sx9kBCl`FaS&Lz=ERF6NRmkTQ zNAlGZq8l985YA2!!3drC$a-H|S{#?WH{aqVeTZ+*xi-1qSs_M&NG`Lcgn%0M{c>{3 z5|)qR#_{Ivplp3LV5;3zW~HIS>|kd#7WaE|-XY{@ie*yPvrN!OM$teft}*yLG%|)$ z)ey_=jNfx}re+RI4SX}UKGKqO)bC&$8;IrQ!VuJ-Bs-`m<1|j0NXR4c+ zr#bbz-$myZD_)XuDK^SGVJnK(t5ma8#5Hp@JXKA3*V&?hQrri#>(CSa#t?>VC?*&W zoTfJyL0(ldKKgFay@eq@dizuPnchQIQYakkQyY^(u+}6~XS8pr*UJ9HF|{q(e@z0L{Jw-AW^&}pD@GeRf$!c`k{eX z%jpAyZXmd+-v8x<($LUQyF%J>V@GR)M*Q{N2qoXz;I1L|AM8KUZ-bl3zUXQ!U4~II zBpcAiuoy*A1aVMaC#cfc8!bB#fYjo|*ieB^`p(nVFP6p?xdgdU0L=IBEn;}P#`_zo z7>J8!1i^xG;_1ha=)nj|ex99#ZUm>Jc3~B=YLLR2AO*c&%hULN!NgQo_8yVmHXZ!X zQ_B^-Z}agqkqiwBkBA5lha_AR=qabps9F<5B0gH~H}iCcm8BIK#Rq>KxEZIV{DS2Y zjLc))MjD-s4E->anv3DK$#`yToQQuspQ>dpGC851il95v15?M8=SW-h_7bWzrL{pqTNEF`Kj{qNeb~z9$wlx3rdjxWA#;^jG%zv*o)nNRcsQAvoRkD&FaiW0 zKPOc6YipJwDYoS^h1U~KvyxBLABw#-vIV8vrs+9gD}#KnU5f(r=ijVv@of(iD&Jjqe+ATm4BzDz# z!QzY703T%rZ9xgNuBO%ePDM=}77#1{X_OFr*P&JdpekLSpT+t|r!r*_sA;)b2tzZm1yqfZI$`JN` znl^M$dUn_(Qaq7VVp7)#Q?^E!x5AfZRdtT%T26cBM$2b>wL6KIq~vB|`FP>Px*Ki> zA@`}^&5B>;a4kz1^Hr(U&w!Z5Q>44^JwZFlBe>wVR$^mFykinoWm3XmU-61>@h8W3 zhlY-!GsZ|@!~}0HOr?Gy>w<*$8mPS{zQ`F z>t*h8Z0@0%x+Mv<7SG8Sq;l`~ercV^VLx1m)&}WTG=pCfggM66b;=o|8`u*2I!C8N zLikKAaBPr_i_qw@)z9T!4w}}=;+gOvi<5DJ@Un-u<0fX6U!5M2L}kaO{c<9D>x4CK zQ4+*$NP_dt!GnNUa5{?}&H*n&FrYUE?)iw4G|QA-7b*yoh1jarT>Z0p#`*qN_xy6s z4Zl$BOR|AYmG^a?;XP&@%hc9V-o-_SmC8YAV)WceNkJ)T4dwCBDRl;wb2PVcxk$}j z)kx5usQ*vq9a-U;Z4isfP*K?MrQI5j+;kqDgi<~1JWH-i38R)p26J()X*TmX2oHgU z@%RobUkWi~`f3i!c}lC`eJ%W=iri{iG=oXHfZPY8Ug&@#q5Ihun}-^27XLwLDNfKI zKYlQsdwzbrXE2+-z0-aYl<1n*=-ZOczEOl0w$&7wXye}-(Z!&S7vqMit3gvQm3ykI zdBPA97g-o#XfS7w9>7>d)4>01oUQ(fTv~R9kF`sYH8iWMrQC2W3+opcVz^i*z-K_M zV#ah#2KlzB^%)c90E6LQl5;+5?(@Dal!Rl>9*0kmwjnId9S*BJJCv%6*f}Tm#EL25 zhoZrVW5_nh4D{@Y0d0$u*_@2@^cNeBef%Dmcy6zaY*b$K6n}DW3Fix>5%5uyRVA@M zlt`1jyK|q6qSQ{`h{#Pp7`LOAvnx@oIX_&t5l~7YGHp&^}pfG=WvFKUSw(uw#;i#*&1tHBlir{D! zj(l5KoQi2OLA-d}K1A%2&d3W&qQ$~7b;W_BWT4!D28VgB)fQCmv8!;owQES0s)Mz@3He z1$v~75iQ}+?|()KSuG~5z@hX?_W!!Pqt<7bbtgbqbJdPxvM_|q##SLFW*F3l7+^89 zxTvPDpW8@?&_Xhk$DhsZa(Fb8Ut3e7ryrG!3ABfcA;8q$ePBovPI!M99G208>CY5u z1lsV*Y>c3cj4$nq$cPK?;qdi@O3t(@Dk>%p4hFzThWFiZAYhM61dj>y{%`W{{b^?t zk_fPmgoc)^-{sC;08e4>0M5tHZwxTX8KMQ03tB25c&b~$h3e9km)>yX#6Kv9(%&qBYy#01B zKHbz@u_OXKd`aOqUXRP*uC-EOH0?Q%k)-;R>DATtfB-I(oJY`{`0YL$%3 zYUrI7EyJ)1H|PT59Xt19B{Dpmksm+S)=*Y(?be&e2a8UtqL=4d9H@8HhWuMPcTtjA zeID!(Bf#7rfxdb@u^PNM99he>bPs(2mqipX7_%4zFy|&~KAy=Jj7aSQR)jg>M|O4j z;M09=Y$gRv5KiZCgXsk0!Y_;YzCAU#u{lgq{eHOEaPm|*J6=I4IX*kHwX$*z&D!IU z!3CY26Y>Bm*zC`@BS9u868lqQKQb|Zf~rBUZ%5+O?ob2=2Zs}D2AKepapj0Oj-AMk z{g1IRD^M4^{olJP?8u(aEcx8Vdjla*wk7+Lu|l$>fT0va|7VC!+*L7923K+MBwQRF z9gf3oUubAWRZMdj+tZJ$&ySgj>h;QyBJe2;$9kBce0)02z9-JD!C!pxJFiC#l8QS5 zhQM1<8`uAGf42K3R*z>S*dN{8>Pc6}jT{J5SgBe*X#KM#bAWjd=&Q7}wtmlDils*2 zAfBSaZTg#}U55o}4@mSD4|}G~_+dlKUcXsJ3b+DE*`}qTfn|USfpDW#$SGKD9mIG9 zmbLXjFhj)##u6tcCK}n=!pZn}dIn}7H5FvO4wW+)KnE3z zW9$qs5T*g1ohU&R6$8wKpn_yD8`c|vbXN>MlRG7$=a|nQE(PX6Z`tn3h(!jz0}zOt zJ2Y6T0B%iWcUQmw?UVsT{UdTg=FS)xG;-nJeVr?rhX36cpsZFzR1|n5T7O_HsareJ zX32vPXAgwI;bB^Gd#75y=|fZ`GlM9$+cuKh2(Kc~M*~B!vi&h$!00Sa5Rj=M;@*Ni zU>FR>vO_lNyIQ)KpG=fYV>M)ku@b=u6utKxzp=!a5J;FLapP;qK$a;unfT6Dk53FH zJ*W*QXIt796;Y*vx~p11=uutOwbfgjle6>}YreWpSrMNo3t-t^G-SLf_&PD@^YinL zNW}iOG4XO5y>3ZUI6Du;iJA&UGwADFmeHOmrum*`Q0g$WR5SqZHZMJ$vIKGhT^ z45(aAMY z#2_Fb2nBq!qM_7@Lc9*RHy76g6VIEqSl6t6#ggz;BK{EkXFr7XfcCx?7IZ?)eLo5#U92=B!9SenJ=9Y`^^;c*XD@ZjE`MQ0cYdz@91 z-A9gym7Pw_zc^~;rfFjqWd1i05tm1>w)BJiJR9PqEP_x&F)fpl>(b<0%a=W5vN?T{ zdL5RcgwA^#RUMy{w8=|V3p2cUl&ZR&qPM#GHXXe`W#KmBIDdDEf>Fke^RuwmCubkzV^thK{btJ2eDSh+Itw&g!jtBe0H^q?=yA=Rr;blWWkR-tBC8 zDqoNgk*kUjv!O#ddsNq3a^qQ|m!7^3ScWK@P%k={_JTM*``y|39mVF_R`6oM+A(2q zL_dO}T!^OQ(Ii;v45m@BHZJ_`eyOAo#|mKd)(WhaQ$ig(c##aBhpJKhxG##f*Rd|sGDZbaTWShm5okvb{Zn35OT<%8vc=(>J2(27GC@>qvn~bmTlpNT= zz{`{S@O+|HF5KYk>agD;@Xci62@Xz~4Pan+ThB2+c1 zh=?T5uwy<7*;TuqPJF3(>`BuqgR@7Wzp2}!v3VZ4ey{L*Hm_t((sU^6W)P>z+RmO~S-L1BqWTHl zbGO&^as2YktB+ubo#aM|h#^YFd#Iwu2mOtZi9U8(>w2Xc(RYrhIx1au@JDRHHim}uv{V68<>R9WdT->Lo1YkBv9G$mumQ` z`hoi{6IKdhl_bJP-bPkPiJpi{68!n|Cj<&9JcJMCf*fwM{E-Vfx1ow-bdX6)!@!2T zfH{#U!bCO8Wyx7t=#9?BZC*6V#Vxg^*xoqoBX`OIjj!llo8#rj))oOYx408lUu$#3 zmp#wbCi~#%IJRlVGnYB%iyvlj;7@^U+*3e zrQ&>#&10Xit2<8vAxzfR3Oz-nyI8gTF~L|wG*7fHAzIGwVzk3;3-MfZyBi)7jhNS~ z`r5lz=JUQl3SYN4m6F^Cz_z6hyi^18OWXBnSPE*PfHtHhL*%2MJfMsD_1C|?r&f`k z3xuK^&lT{7WY>pM7olibAo^)zKX!zE3Xq+X*sv_VI8nWhC#NKkKryo?+ftv=|JU;Otrdc3xvqE%F}v&e2hr2DEMbwbT;|GPz$9lkTNE)=!o>Bchf!A^DxGrZpM#6^6C4{aft*gxE{(4~#H@ zM0+%;R0=I-Ke(Wh6fikIsna&*(Bu)$D5>yRR)rVn4uNVaL~z&=!i*HjvVdnIh&8f< zuXf0drf4ov7-7srX1ef{W01^#--w8g<(G{XRt#Ci)Fg|b=A60HXN z-yJJw^ab;i4DQ37(MPbF69n~re&G7kGm+0`u_vALW8c}>6(Hp$xpD;z&+E(52TKu{ z;&ISFO--Nc@_b9C9Km~aETO|Mi^Zd;J2qvpmHiEse--ReKEvyDABi#7@I9|=VT104 zpYWcfysF$Dmj8eWHa^VXodw;EQ8xOH-N4Ccy)3yL&%dNFp>U;+gw!z($E}Du5ysrO z3$MSQtmk?;?9Z%Y6XpYFb_@Al$Trz90PZNqedsFyp_YA(V~pn6Eox9sG2c2uZuYX8 zm#&$i9eZ2MXoLd*;%s>Q-Gt42V=2LB@?ra0fP2fW1y_Z~6@YCI@BF1STmk}LYEE#h z8^etLnePLb;oSK_%+IeM*-%+Y-40C4UjoZLfhi!lI1wWvP{s`8iZ=Ks^ti~8Af(K- z$~>us6JvDHo>V2vFJW&lXLUPcC}oR;mFzIu($hzOtYYiF51CjYEJ*rbQ9BRv*;^)j zT5Fy(f{^VT{Jz6tPP*Isz<-)-_w;;qbE*Oc)r>X^U$P>81`fJ>ihwmdb{~%4FE+V| zoBmqkox+DDRTLx54l!-s<0NAHrZm$}esYnsS>BM&xYdV?DHQ7&YTSF^dFruJYZ(w4 znU6QrU)Vgmr>$Uje9`|Q*8k(Xm#}_Y_ z+Pkpbo5V>#MMDF#W(*p(=wlE0&3Q*jyxml$x#x5@u!#xu8>!IyIeEDwIJx}FV1wPu z;JIYN6$PcCz)_2%=w>hS5oH@5$_upRgI3^spwNmzquXTusA2m2RJ>C@pQ3CR4nbPx zpKA$EfC$ClrNzx~VL<~bv)dN#4+pmRuy4!2Dru0QWl6NcMHU1O|C$G<`8^!jqs zrTM)*G1JXDdw4`(!)WawNxH=3HOMKq?ozhfhw7uA8}d5s2!&W|v`FBP{`P!?nGsSf z@%cRxe<`3L$n^fCH<#rmK4=Iym9dzeeB_hM$M!y?+ZdY5k}`If=sG2_!5q^>JnTxR zz7jxr&VB9T;id)_;iIX@neY7)x1+);V7~QDYJ0!%joA$O7-Lt-rtoNrW+Pn`9YwVZ z>Wxbd1I8EaoVQ{p%}#BhadVnYuHk<6@M%t@Ane(lm%Xutd6D3-$O!%h<0l+egXBK# zzUOH^@PJH5R_*e^XbIhBQWVn17oHpPcLVxrQQ7BYi%`ierrOA z;BnbKlHQZ9kEA2)=Zf_?S#}e!y>-2a+j9>JX_XU2uaFpV+5S&sUmX=k_ifpD-Bo>Wox11L zJ?HGbdq;kz1qn-_X&2dg5R6Ktb-C%9tWd;HR#6nrum3S$Yn;?3+^Sxb8Rumf+Il!4 z`e6)_xvShU661Xi7rd`eER{TmTCLA1 zFO3v{oMlzD3a6eXWlF z^-@NQo)Jmtz55^H6VJWR!zcw0GJ>nTjp~#yoV}a4q86Rxfd;c{9xId+f=C&8EJ_>% zUQzt}GxWhN3biI?@CwF}&yS(RI%V&N6YX$t`1((VnGufn;!oT8zq_0$$axnO7JP8!^*ZY!vKz6ohGI)Zn3yV1_s^u@(D%DEG$@qKqLRdO7SD7v1&{fKlRQg zA4hw6%yEL(zLC*GaX{H_DCKQ>d*;&RAYxAR-dm*!_IeB{cKOn7OUPdPj3q-e7ModV z37Pu!gMaX+v7zZ#cAni-)R+iVZ+|~#@7&f149>ucF#PKlOWPT)Ma@w4@v`ar&FNLN zUM%8N>+1XWazi%InyZ(eA$!h(4R(FxV%~hZ>r0v^>k9=)Z~?D)21-1m$1G`d@%bdZ zQ6cFrc6KN{`X4@g;I@Po!;k9%42l~4dP42odWP!6%h0&swxFxs{L(ZY4^N)n2fbak zzw))+h6}v{0Wz}_F>*^C62wgJ@3V4yi9rD-c_wW!D9UyY_E04sAEm#2bSV`}`C;FU zf4j&Qs45K2^0!*kT!MMpnWz*yzd)UO7mD8ukB8K$>%Akgj}WvN5v`1Sn*tHiiR~iG zPR&n++$JC!NM_s#)-T7)EqzH(%P%}=KVX!{3V3&#^ZQKRZQ$DY=R5^zK7{PEP7FDe zxj3MuF1zbmMOJnXvyz)gML`<#i(jW80j&ml81OXE&kS-d+< zr?UNq9kCV!*}@C2okbK2E+jf^x80@xvo6?)Qp(75vpzkdFMCWmSV$r7TX^;JmASrc9$)9ftAUuMlC%5L)57e!n=r&wm@j|VGjPE>YtZ6oKSgX`l@PqR%M-QOL?tUp6K$z6Kq> z|5_Jj75+d*Nh#vdTA!z_7y05Wvb^N6JM4CbLI|jjZ{_j!U%QBxeQf3T68)*oC0vN# zAAeZ{q62yZxoTWEK02OP$40YEI5xOa$97IZy~YxSc_i9ZDth^NyQP$eTzc}#gX2xj zKmncFb;Mlz38m|g5*ib$zHFVu&)JotqJD!(D$R3F3eShH#Z|hsh zodjoaKLsp=((COc?oG9^a_u=<6n~@$hprZftCNwD&3Q~oQq;*~ai~Bn2(YnKoncW^pr|S z3U8OAyrS`1s>3|x3YxDWF3e1~p*zD$=C3$XivJ*}OKKm?$kNpXUru`>*_!lTNdX8< zttG`U#QS8?PDxG0T(_Z+m*8t#Frk-b#(lC$r=-r4urm7ab-3$J8aPz9GK3=VoNo3f zc16DvvRh>zzKnuGtk>sBrz&)b{Ls))`DBX2`Q}PF!eD%<(0#kb7hl+l!tCUHLXJn8 z>}bi_7PeN4FK4Pq(qk427z$TW}>i_ z(S=s0yNmtyXBeX0G8{>fSRf&xXEK=p7U}2t8fy^`N~z zG3u$MO_VL$o-rPW)Pm3Ddvn__mv3JhOXL_M!;P=M3}ZZr9JswcrEO(L7!aN+t}G9t zzL2UAP7!kT==xkyzRwzW#tGk)05V-#>|tfwvqWla8tIYj<-s7;GPm7yJhnMm0GC4M zED-k_DphjfQ7CyM4M!%R^;1pooL8@8L2X%m`>}2`=zi_xs5d_04>(tfQ{qXl>KI-{ z0~LB5Y0$Fz+AzeRR#QiZ*I|M#$F(+VNe+81xCQ4Vi3t5Jshd_=2+G9n>;NS-$2R$g zZbb#(FhTT&&Uzp*Oq&QOPBY;Vq474vzbGu=9(wti12HkC>H*Z>h`PHYA!73MHX<}F zO-n`w(QjeRz*ewB|L>%|FqRCg0#g=%%0|;f?{dL&CxT##A5A{JudKXU|AuaDZEc|Z zvM;megEAZdV^ff#wIBdW?6Dy75Vwchi{q{*aYjaz0UFn3ouamhcoqxILzL&1in{T7 zo&_SEO!0HI^J*VJO+|c_UnTN?dUUeuRQOJm-pUMLw~AhED)AcXXdSN234cfl2V0~E z3hiwdJr3h;8zcwT*<-?Ip8#(B4GkmtX`Ef^Mub8*JWjaqKJ_!bTez8ibk)-`urT(` zd*UGjT!Ifa5S3077sE`JbWVF^1`vn}_N)c4G8g&XrrENb3tm-TQd=9ZR^gp^P+EGr zWtW0t+>T<2s@;kx`TwSXr;%}NRw0XuM1n;QL`0}8FL+SjmTe%aO-%?c%8>Oci(tvm z{ht(Yw6h5;CkIuP@H1mu37~;<=N#(cO|lT^1?{BOqzHCdl_d>aO~0SzrFLz0&apHj z%cpSRq%Vi6PI!1UTxqw$A?;09IrET3Fus0i*yaC#8RF`J zD9j3^XSejl-xI))x?#oZEU`gnc^jm6JQe{44QwhYaQ_^&-=@U=>Do@DwUwlbiWZ-p zw@4I2xG6lbs(G#5GeId3n_?u7Pc~ifB=;S8FkDh@ZGpS^>VZffoH>LLzbw*J1&WZ! zmPnQP?#<1?x11@hZ_7YMCjUK3_y>Z5<3lQsCrNr>TRxkgb-Oz)*ow;iX@2b>;WW3N zpaYXaBEgoo_-_?Mu+;E;I!<+c0xzb)MgUH>wocT-M2!dO^%6s83UWR~ka@Gd>oIL1 zcM{oZ`SMRh3`T@lpL0FROfdrO$#KXN8P*m;m+%~}va&Mlol^V}QHg_Ulk{UY9*n(I z7gi1Tiz&AGrd(X)V5>Q5ApnWwYSk>;`IdBk%>}`blj4_Mqt1UKeu77qCUA@hVj?e09;tt{gl>_1llB1O(sQdWd8Y(*H^#4iatU!=L@7S$j{^e;$cW zPyHon9$CFpamxR$+^!(BWWRY2Np_lb>l$jQ<}XXQs2rVw?XD5R)icM4>OyECg0ntn zNVh-11{L{}p=&nAaFO7z?7>nwVW{}cTak!hF7z5`!%0$*TTF&>oBh&+QxKa0KV&k~F!~F_qnfh2*3H06xP!r%wW#W)UAawa%kxY&$=2^YZhKjq)dQu&D zBtR*t=K2ez_8jHzob%maAR+edSicTA;sWSS%mN`!BioBbpFtd8{l&fNRkA`fLV%ld zQ21fh<9v2``NvDyDfS^ABX7&i?^E&@3V*&>>p@biS5R5sFXC46GUf@ZR8Dx#pOgmA zvvmy}^EhuciYsz*hD~LVI1dY`elEKUY+^Eg-5B-d(2jMxI=XxOsHQghQ?yrKv$U8A zkNaci?EwiaWBxy>@7#+$qz@Ghs|Z}_m-K}tSi2UBOdHz9pT9eDP;;QtWK6sM;%Rbp z(sZV%_u*IT0n9Fp5C7`bD^5yoyD{<*pgW!Ge!R?r0wsGdIcX!j+kEgcPUw9qCz$&6 z_j)#)pW*q^h94uQcgnv~9|E8q8_AoGXMI>rsUI9(W?B*)wsk$2AbNgoW~?<0>Oa4R ziU}dF`Iv8dE{2i|fjvrn-my>V=_}Oe!hGPJLuJ#jogz!=1tkn5q{Qb7m;JkCOPR$c zT;!l*zNA4)-Hx)1ZcD6_$3;S_rdcL2URs$8f6KU$lA&cyh7DT8VQtbD0Avi<)I_02 zt?p^*>(91n%P{5gCpxHz8{EsM2kh&TIG}yyPQ+J1k#?Au_{G~RH&$9!3+P7w1_ zv=G0=EAj#{$Y30ae~5W2vJj1h=%sC?PdZ%-e1XO`1ccPd@&`pzY!NP!g7eIhE{hH7H9W74)*?NC-Ww?pIw zp#vQ~eGenzo+Ih5`%qgp<+s^=xK{{AuxLF0wRJB8OQdYo9@VO(W_lDXa)VDE^s$SL z)-fjxOZ84vTu5vLp6>2pheW4HH@Aj<31pSI zd$0zIJDzJ=p)jAs72zt_ctY+iIjU>D8y`b0efi9NYbg0>lQ`n)EIdB~8wv(mb)!Pi z)WuevK_mP!vPB#xB5^{u_18bn?-Wa=uNDRQ>@f(2Lw>>5rL-7Emsy|X#4~hX!&iC8H_Lc5(u>^%^Pm!KoGp1M{La1ybG-)?GfP(p~5Y5m`K zbcmcItl}Gf_F_ID4Xef-8nGJAk^^C(F*8xkFiJ~n>vu_7iv`_gq@f>sy3u(i>`@Iw zfod^_*+RwPvR?D0QnN3cfV&g9Udt1G?ITtrA?6Gf#KOpX1V9mPGm@S| zxFhn3s3kViF|t3(wd&Jh@(^@$Wyap*&?XuSXkIyoJyKNl3ie}93-8p$nd_xF`fCUr zD}25zfKc&0S`?ZS*Xn{7Z1;wQYG(oP7wll58DA0I4{!zAS{^`g7rBrRPvK+eJ8sDu zrT#{1se4GP+Pp2{i?49tgayzkM`Z#ap?$U8@!bO#hRv@`VQz43P0GnGFAVX3jI_9z zO3>DD>P`?CX~r=A<#MECpoS?(MM=XsLRDe^NlrG<@|WMfQ#id9A_$Hcf`W`3o`8?_ zi(5dT%VQIbye29&6&bNYH6MA5#eiCs83CQPq})ipQCJlmOLW*Kvr$({lJzV{pg;zH zqK3-x#X{ENFXC2{#jmX@&C)W}p|{o8Y+sN|DW2s%_yG_Wtkwu6=2k^v-#)bbnF7ci z34$q9Yl2PK;b%8{r$4zZDJbty;^3s_=Fa05{ElZN@4~)|GFJPXmWB(HvoNcP6Y*5`?d89Un1}kE?Tq)4l zUj){Pc2{#)=wg3A;@D$4$#mm;RcRp0s#uFrepa@igKt;|QV{?Tu(rTuaBQrl6&|(= zewM<*F(R48p&cg$W03NK?`fV9KOKjw-!bPzCP=j!6ZxNWHU20{^g(b72VykV3pA$| zfEb5-hq>0vD;WPJXuAHJlXgZnKunwXAyOdk&!@heso^7Kww!=nhaw`rD%0!JWAn_0 zkO5;*ij!X^bbpi_mGJoo2P*Tfj$J zToDr!6T5>|fKX0Hho~kU);yT+=cA1Y`}2ncEX3%(6oY|NrD~!B-^r z`tFRHYzIF@Snblvci4VyUqmiJ15`Tf3gecdCMNWaMaATkMXxh7%9$+YZMNbINfh90 zU*uVeTRY2~kt;-~exEfa(tZwGNzx!WYM@&gX)Jetf3H_(G%d*Boe{1HPG*Fh+}QSp zjO(&gdv|^vo1U|H@{CkEdO>En=C)PXgoY`GwNiGm@ut1w{)lPCA5M8%7<~IWa{2h3 z(4t?MlkoeH&-;9bgQ*=yU$?iH4I38Ll=L<=TkF%`WEs&Do|u^lc@dPH$8P9T5TPHT z2xH~hpfB$1#GS#T>z;L)QOn?)SUTCf>e$r?J=;Bx&Ph>M6tGTRv$!B_hVnW>_s$B@ z#T5em$LQ;mx*mL(c`R6I^-znCVmjW6C>j4uQk1Lsp?raqg%@5O~(`_+>F^ z`@wU~jN%f#{FNV2uc#(E3Q{NQ^B422HB}!#)H<6#c-FTvR~u5o?R6@se*RRDv+u6K zMU*aJKzd-2Y@$7a3@uL6gU7K+SS$dq_N(-tBB=U-Yo{3vX?o@oS8$MOb&I1 zab8<@ct{+#S%ILFLJ zzc5=C`SQB6l=W&?DqDLeiJ~xALuY93@=^$GMv9)ZmYSi7>GwI;Q2;saC&5$&h69z` z&6vpJLa=PIT0c!)MP2D(g%fELvFY6B-q*@IKc)^)d|hi>^VW*O2q_y#RZ5xdY0qyD zo4>pdUu>y6bGD>G4KcxtNyWzoOk^5Rw|qURyH zSs4ntqPb|poE%!>(WZ*)!QZ4|UnUprKmGPhPzN?N$nfm!?3|^o?i+G*J!-amYY=?o zfDAv;7IeSq8t9GPO<41fY3zR*Vg*#kA5-^?2(j{mDLVV!TnE#(O!V#P$gPXnQZ-y7>++5^*^;$BhPfLr1v&+j#*7d5Nj9=0p2Y+J^y}gQrapOm8b1J25!?8b zsr)m}17Z8q^OHdyp*?FxmkW4>kaF>6e-ZAxP*-XJ*O3E_0R z=2{;R-#^8Q&IY&N!!|n}Tc%}sd)g4%qFMP#Ddxe;8T7Aq)Yl6snH{jq!^b;l-_mq{{F^Tzu@Knj7 z724Xp>9NN7N>H*{Ih%dlv4IV0X7*_QbekRNT|a;>@%(Zo4vVH~0D#JPjA~O%(~-b# zN)1>kO|ha?^D6M*|wjKAaEbty1^`tD1DkJ~s>X&7|GpYOL%Q({C#ES3$ew-a1uY~y!{g*T ziNi+=>UNmao+aJ{6^xVXpA5vF4qm%bh3(;d*MY)c&W+Z;#gEHOF)kchcj4fj@QtC& z<+U5uzh{ag2H#rp-aCm0zwBjZyz(C{>_}@K==?8@if7SZ4NPDnA9}(l0UQ1ElJFygJ{~%DrwG%Ai==CF>@Dd`aMEdA!{S^ zq+x{FC>{Tsdg-xfbjtFVy?MenA}{~;ev@;ER#hmq=!<%O7;TcE1NoDol*Z^M7xXpr zR1)v~BCYH(Wup11;*P&hhf5(Ucy`1nCqzOD{GrUPr$kc!@5IX+6p{m@=i(PHamyj7tzg#sA)VLbv~u8l*wY z%k{T|0`q)p#MRNl@q=cd@A1;vmv>jN!49Xj80)*skW;N;-sDZKoQLYput(%gOef4> z#t7VXffxLdAVe8R@kQzQbsISBiS^Ikk7Mj9o4qrd^!&MJabR2NhD@Z5^ zk!qDW6nC9D+#f6nf`>quLUCyzXg{bxaA3)9Z4iCGHsaYor%`Eu=w+zz^Q3^YLPIM- zO0kvTE$ZjfB#VIqNhaP1;Kx94>Tqv3{=F!46zO2rc;N*(kz8!VHtJgMj{n>37X`ON zb4+2CUwjhE?fg{Ul!cQ-)zuyPU{embe=KmA;SE{t@%+_D8m8F%Gj|<)ZZi-o`8@vP zyY3~{4Q+k@-%*_xjy@Uf1P@u>x)nCgqYo{fn~}ja(e_e<%e$XNUGMlQcsJEZ*PN3E zxi5u8@FY=BiIGR4Ab4&f4Mr1Ty)~+W`2EFpnhuuMlYLv%w-iNgdu!o=72|1!Xjzqm zI3xQk=H>d-Yl}AzND01Arh29sqG?`dbhqx*7k%?wTS6icO}riheTcxVTf`#apO#vi zQp*jOH8=Z@QShvlu!;VP4Si~<)}&d{L&0EF z{Gt8D>B^yd{eDG2E%Bn6j05k{!%Cbk-uvJ^!yinuEAPsMvX%JkSV9MPBLaynTmh?l zAzlXtc{CQ*k@NUoEU+}Qq)i>O8Gez}5BQXj(smzbm>@9i#hO_cqs%?Aoo%+`2h(Jt zeqp=6%`wc}d4~8>?lt2rBnGm(omc~Di$Qu1Jj-?PTu3!D^#;bQin5hGe6)Vkz+P9i z)AZo!Mda%_;|vL7Y5vYESg`S3(r>z<`N+9?%DKkFIu)q3O zOT~zGrUprCamiM=h+trhC~&7`?GAAn7XudGON)<>OzgG}3x5}rlX>L_C$}-r@l`2N zlG#7&_*;yS<+joK4=n!K<*2K6sc?Z%)%<_Z*Ohw(*=2s2mtyfoj zrOo`zF}snAp|P=XJ~;P2=dQWyS6Oz-uistV07hwl_{ZyJdMrCpl0S*Kz1Q86f`Iqn zhcOkpJ5LNvOl90yYv~sjzBZ_%=Nqi|a!gY{oG`y`5)`uib$z}V|LImc-|XNnR`6Bc z$Acesj)_R}zwKo$xtVn$s|r@C7`eZHswcpEffxq(VYdPm)ZDBlgqSrr`l!r5?C?m8 zUDOJe7Hr|GnChf!72s3pk7}y+G)Y4M;}!!w7uzbe5f(L*SWPVo_4hR zajpe@HKyS5np>G?TT#2B>__fWJ*zky;Qnyvsz9J}M`B<|da?-Ygb7`f2EFc`{Dx#4 zw4}s-c*-WjP8%iq9Lp5cfFS1~9Ul2S_wTa6!T;}@{#V9-CIwhK6T9JUmBwzCumVCVA~p!W{U?;5c4 ze_R5XbDR<$6dLLWe?CH~tE;P5qx%IPVwfVSudlD5@JylFVJ)O^#)HP3{K8oN=SeZR z0Zih70lD-c8(&qROY4Q&yaWDI6Dn6a^!7a3yKjMCJS=*WZaYinyQ!AzM-w?$@eSLJ3jG_2nN>xT+% zTPSv8=O(tO$ggNnH_*it0wai IY7zdw01+G|`~Uy| diff --git a/docs/img/cluster-overview.pptx b/docs/img/cluster-overview.pptx index af3c462cd904d459a433ee6a8b97daab63e547b8..1b90d7ec5a7ae022db7f11402cd93f11c863b60c 100644 GIT binary patch literal 28133 zcmeF3bx>eSlCPn0cZbH^-D#k4cN%whcXxMaym5DThsNFA-Q6F3XEyfUdE0MxWB!}B zL~x=`5r>GZI{8&*)~}9?I1n&00000u062JvP|q#3>GsE+%qIW~H`P@?@=)@PG$eL5S)W(SXMRRg4f5^Y zJ_R8PSRqBv%KC>Gf~yZH!keSbbItH=|urphO8faM&IAIqhbo>GcJc zSY2Mh7frJJ*PTQ9=X()RT9-*#auCS* zgy_eV$x|;(YNU0B5mS}InLYMbVturz%9cl9nZJzB6QF2A-^UnG0s;Uae2kC2m7a{P zm9-tMo|UZujkAUMe>F-LQNyM^ywHO8uaG%y)SNzY5@{#RV)zrN%{WZOj5F8TxpRl> zPP(BrgK{f&aLO$`ebTdVMmF@&6U@{K*kr0`{eWxh8ONlB0+N-6p*~;_20lHW!bJt& zALI45N+a_qGVr+b0LO}pgpoOFSf`K);x8Btt!mR;di4*6!^2bSJp{&0OJ)6(b<&jx zUnr@|hHlae1hpYtbjSHXTJn3aO48a^sHq0UG<(ZtxW-3YH5#i6d!mxIj~Ut>H=j7^ zH{Lek{yGkv=n(D+ALFq9@eBXwanP~0{@pO7kF`N|(;)|Lu9kosS;^xFh)vUMoye?y zwS58_75y2K*XBm(i?;D_+1)d5pQ_dy8e=348EF9WY0#N-x^QE1JroLDDy3ADE6W!L zqq)9G7?&;|Lq*^FTUWeXVV*u?*0lPbL=!X*NBUKG&P@*;R;%l^t%ztw@e!4cxCp8R zf+wDPpR441_FU@t7tcGlpBd(GR(~}s*~U2~^J8iwzyUstLI$w5wx_kWHLx?VwAZmW zv9kP~X`_i6R=yvmo!?WYtkJ0Qwi9Ku4p$Q|S4-RjHQUO>%896@KO8GxJmSPV{r=2B3^=sF!}1N$nf;3ty5X>!5ohR5d<A?ee> z?mhKQvxe0W&N}8$Z7zqlZQgSQRO6n92|DyAaqn4uk|)TaJ{WgT()|)V!fQjBvNd0D zvNTc(G9_t2@ZBZrvpD@mb3`7*1SQ~sruLDTwkd64K9{_vw-qm1csAU(udIMA%m+}#=-$NrnB-ev007*-Px9Y5_m5Nkd+x}qpxf)jrD zGC26@X_oIK_=EW#ZjA9MkqE6ncOPxXGNX`wx;`_D*aIB;L&e9d3$Z+}gERQ~c5MAr^-gdJgGsNdK-mk47Y&pXL*#)rt=oT$G?Q|!z5Qg5y zEd{D38z!!@0nISV*7#br%8Vj(YrOGgvDpG72yFBoh|FsbQkSaHsHDfB(6Amz`qDo% zI&jb>L{KWk&wCsx1R{^9+*Zt*8$0UkzI~pBA^AFf%s~oge2uvENtS;Hu`B+kq!a?F zAdvKkOQjb9LYjjDd6~i7tG`Mxiyf7h_S7azf#1~Ul|xekwqp^~vg(e7LABQ@QQr~I zlF>fazm7NXuwro42PXvn&dmRCsz1Q_JCBNpYJu&hgBEmZ&%S~shXz;lpbAa~9UFRi{l`o)1QnLQ) zlTYKZ2?QxOLl1M0FOD9|uT0l}0F{@EGijnyDJ<5J;Wtg6t|Ox{k>u~BRCeWFmc{~n z?(w~PznzuZvWdHi=)xDcRvHN-A_;C(I?Kc9g^ZiPU`IT(W+WuRuUPz6OLP3}uQn$j z#hsOX*i8B-n;lIIoc`Z7NB^hIQu65V#ZTzGBhVLsvd8r?n4FVa#zVO6}Yo z!k#P#^!aH6i~J>1_(TvT+UTBaP%Iy&AM_6m*LE^3_A{)e!bXzMa*a!V{+cvGmPmL-m;m&S4!$) zdt6*WOuZ(Pz0QB99PGUIIXw>P73EbdA^oep^T0Jp1t0ceeb|focc0R;vat97C!d48 zv6ZdeZzp+}GDFFcR|8B0Gxrx4kq>i1vfvp|wpKj=X zXZfkBh1D_({9C&gzHw(ebU9I|YZY2SWn2^Ggd<=2D}ZXYMb^x)$dZHVO~*M|Nc|3( z-I9H}MK3PVl`|K&Q!D4^WcAjS+>vkcG_hP*DPvH}Ovb3YHnG%!ds+=~n#P#?u~Z~f z;|CM1Io<9qF0#B%u1nGB1%PhSiAO8WSP+%Hz20~sozgFA`&~Ctks@?l(uukN+4G}9HNzsnu$P|CzMJO#f~(nU|f^|3M7bf z^5?4S*QDwXl#ty3OLdC)a2I{%=hy@a&zI{+B8`GzuhgER`MLt!1+QmhBC-Zeh$IC` zk6HPZN>c=5R1DpLWyt`FiZe+lu)oEALRNb&6FNU zwgm1{gN0OmlrDKiwINhE$Z$b{%djx_2w7S2EIL<0WzR?iy*}S~>tAW996+ze_BBp)VR{Et^%Bf@6jfw0IvZcm4>g2(bpkn;T6yt-spiZ65D zG_th3yX2*$-eBwrka_8mLAPVyTOoL}q98T!9+quV#m;f#3bR`LQSHVCKu~$q^F}_9 z;AVxMy_I9wd<|CX8K7oCR~XzyHfV*D_q@f$6PUA&A1hnuF|%_IP^+h3gO@j}@dd{n z;9lk63AVOnd0GzA*=s^lnm&PST^K&(hLj0(Bud5bRd%_hqIx_&*-$-28}3gI3LOmi zfd+k+*V;1sdj?sRmG=sjJo9WM-w4ObRV3fjGwtnNBdO!nG|?=y$e(MdtjTDN9~bRzQ!VLXA~U z0mP=V?8<4oX7*-UlIR#J;#8CuM&e4sv4mYsgBpUBz!`c}%X1A6D{H+tE6_*7o(oIQ zpcl{HuhRq*fp2sRI}8EmzA?P9LsD7DK6+Z%Lb=xF;UqMw4AYbn)DnR7B_E{=ZLZSvYf zIQ>>lNx|QUk|+PAtTfnSr+HlI<;k>F)(g!~orc`&g5zo9a?1zG^mc%+xh*>pYoGpp zx+7mDY7myN5KxSx^am)xB{xmnnB43cRAG%m-SgBDmEvjp0f)F^3+=!k-F3gZ5zl^g zH*$(AHtY)4WLKtm1yp0cKe;|m|3esy@Ibi<@v+?Yg8=|g{a0(y-(uQ-T#0@g)Ar4_ znfB1ZgI%;q<6Bk`bB1jDlc{`UsYSVPcsti%b;N|^d7jUZpNIqTIFKz74sxH5Sr1_g z&TX`{4&1FG{JD@iCydqZ^Aj#?Ts+FXAAd-cftty5hkuqD=;4H==tW(>jWeR}afZ+U zpU$wW;KWWCKk+BYV491d=q)pQ%)KcvjX~p&Us*NkWQa>QgE&*)@1(i*_ZqC1oHlc> zdg922fXmpxzte?QDDT6=AGShb#9B&{10B?%_nG)I?8(0|7{&D^N>aMQ#vAQFBu2;a{aMU(`yCN=B>m3Y- z72zZttTa;1H)lr9i?L7HnLm$?b*S2z5sfQ5h=f^ab>CsPB%2cfKBjRF_3w_k%f!RNvGpen=Wy zSV%@k$QD5QQSD7{AEmpUUORu|`*xa;b&w%rM zogKxyh{n=VleiqX)ba)P%szKr(aUpE3k?nSrsZfHKm@squDROpaUnH5X#uoQ#PONO z?klVDHr6W~$HSmG3lu5!%a49AoRM zrAq(xn=x%>Miip4g#pb6_ciZ%vI4&JY;I++koZ{IJciXS70=)Gb6{ZMg__sE5nvr-m3Iuq)y2ELsJ+ z6`M{ROK>IxJcg76FMk|@6kAGIQm6WYHv8!XzzD6Aj!=3Fi(Pknnr!Awr3EtU5PuUm zhgzGU6;NX!2rsW)4TE)@z6zD!X^LQ9ibgplw+tV@CcSXAF$9vXX(J{;lS@Zx zB3Df=Ud3w;gJU=(nb@;#`8V+m5yspa#z5D>J>0oTAS?0%wz1RRuK`R(VSo_G&?wYJP59H3V~0_8WvGq=5DC@)9oB$u6` z@ypA|kHoq#N&gs!BAE3B&ps@Kn>d1d2vz}YHZmBhXGwNGpqzJRyXG&m@IY}5zdENz zn5kblS9goI-s$R6V`j`wR$*auPt7nGA^WtHtly*z(Io1H~v5;MAK@)o`q-aL+ zWwqEmBc6~}$b64~PRdJg8)N~Wc&pnUJVZQB6N66YQ>AT}(N;gSHqbZ^gcfZ*SMiLW2W=E2caGqU3$*=G@M5IzB>&(Qcq5@u z%^j0c)W*=#E-IO$hcdjO+pP@oZBxOJ`xqOus9++9S9lXmm%2FP0dWlS=aa%gbSXRI zPx;HT#ScmZO=M0kj2XpBjka8;t>A_r*Tk7Llxofdn-UP>p3ep(ZA;*yrB&OU=4PZj)9 zKVU{>gMNU#jM-Oh2g%NdH4i=m;8xIyOOJ2Cexlcy+SM(fWM+w@DWn=eg=543<2S^- zFBC}t2eO{FQNDaGx!|C?F@)&x!jalCpV>b;cK+Id-SvaZc1IX?NV&?{;lAug>wZ)- z-yK|mulgC?~jX^E*CDl@1=YAKA|vOMih3JGmkCs_Tx{4 zf9R+{7^m%dkh*`rIeAMOf7W^f{A;>6PAru6_)$?$`6w&?C0&%zaj|l+|9iUlx0~NC zC~Zp1wo7#I9_Px&H&XhY3$RGB5VYM)PL zP4A7z&s2x|6cg+e81icP!>#W}dV)h5MHs3uH!H49NV`R_C=iFBx`dp2nA)3LqrZ;E zaW<3BfdcwYeC6maqPGm(ZSKz z@r|bD`zrD!hf((oUZ{;Jo@=_HT!ZlyGE>oaQ1pEd^w>p_4O5DZ4Ak*^1+08S@5NPR zuDp4I;j*bc!i-B`h!nqdF2=MhjB&)h`r0(g{!*eLHDaSHQTC{yE72?}o60&;l2IW& zAyZP)XyvmKrQ9CVBHQP{n;)kBS%Fy3L`nk`ep{*-=e0IZ&b=&BUvR5PP674LWrmAk z&zv~;#ElZ=24yM@9C!!Y4VLnk@}V23PDvIA<_1DQC-JUKL05m?`&nE6WQXp_IA>OO z%~;2|0R;;`kfE@!REZ&^46oF!>Qa&(=Cm-Kp6l()O%0Y{riqQHQ}0}!%>G(FI%3Xd z2X**48d1|WRkTH7;R?$5CJbi5tY$g71b4Ll+)Yw;WY^#5l2sqFH+JMl^JDZdx_gRs zS|1&(=&Uo%6bt`U;?b#ZKoAI&G}(89h_0sU)#n0-FQ6#30KMnzX)JjfbDC46>A}u0h?$v$(yxkH7M8ynl{>t^M6f)?x6!Pr=kQXM1j`KB=6-IY3VrNWix->!eH6N8GmXFZ) zFJbS0_|l)k-amQNZ^Pb*@n5?VA7Srm0eC}dEiA!QZ5TK#O*Wi0(LBon2|k+O`dT;5ClmK$ z6Zqc;BzWF~Ww_YIjdmo8D!+pcVdlP^(I%Rz5b)MTsJ1Sb^Za!{d`3RErhmX?_rC!b zhTqkqW|g{r!KLkb;F}QOY}uTHe)| zw!U@A!}VOn?D3_V2Bi3VmHHt0S*9R5eR6U5Z~^&SWAh;~x{9!YP(ZACDC@}9aF`u; z6+uf5R-Gd_d~W&#IZfI$62G}>W$Y4TUb26adU7m>Gm|tC;|)f%I3K7i69MrqV#Dp; zch`h{E`xarjU0IS$Kz8QBW{{tA z*_w)9&%DzLr)#JR4GVGG4wevh&zldZ?&mh83Nne759$+&BODqe5`MzvFI3LBNs!hy z*OcQg=K%vR&$YPbS4WJ$0N?ZaO8Gm8Hy}`P9asZGWFu-ed5wH~14np95g?Y`7ZT5F zs-3pMt|Ms+8t2H;%iE5E*eL>5t1HNY+ z>5{b`JL!%(zID9j?8$u67Pi%x6fRt&84GP>wVkTZ^jXVmfELoFpU9Z#OG85IXWv|Q zIzOB475P^gkTL_~br*1x2YKU>MKM5`uX(#~BSK0*5LIvlB2n&}HvnuT(luu4um0 z7hcdsIu!l{{Q;?h_q7Dz_J}y=lteK2sk6z9)tcb262-oWSTV&p_YDxE_8_wHqt=~k zpZiyn6;bY7rcHJo<^muq$pj|{7yPHtT_=4FTBc9HD5_yR{S=hOf-@@AWI=T* z)Rftqieol3gN%5}&_V6E+oAC*W-q(sZ@1=3hBx_Rs3^NjJ|aWGfd`7Rkh9q#@)Q2K zklLm9o+7T)-X)rV_t~CrV)6O5pT2jU_3GJ-4i8DBxa%V&5=v2Aq)v?RX4A&+C@d)w zXAtEX_9ivRN%ZE_O5G0&hYYO{a2mm}LPcOWN;!JNt?Zvh-zYPxB$Micmw|qcx3M@| z7@15EN=_ZAuTiGd@|T*qN=6ot0Y0a$;FjAX4`o@;Gn_GV zNfB%fE02_y?5@P}PyCAfc-iFD$srm`cLg*+a?qcZd2`1W(;h9hKGn=`NOwUrGtPLe z?^009Xd!7XQ5+&t5$Mp)k|^)|$)UR@8Q1 zHXHXo_Tb?lyEwP#``{!msm(`++|?NjwRYS&bf+w0mr7QR%mUSZbfBCIl~|phFCVwP z!X>=f*iPZM&mjwasVAKoNG0Gg4t?;?_qKcVlx;$NYr;eHHFYI&?+|`V$iCMPkD6VU z8r%5EB;b{IX^ak&lPbl~eC9azN_Lbh%5{=!;d;aIebb%2y(-r6H>?9I0Aa!_r>= zP#OB4gXjP+;|GXLq5e6Dn0^PM(Z8fQ?p=WpgTPZe#Ax{wa55GMZl=Q@ zk%;jx4w(<|iVnx?=W{JwiMidS&C*Og?l%9F?aLh!^qPK)}QzpeIT0f~ss3?+W zrJ{nzh^{xN8>h#SJ)1toyyzGrC==14DLtgGr{w9=K+GrHG-cN|nEVQV2Z(q@!8$Z* z)SZ~wnca>hE0r1%DE4Zc@3C$YTS%Mlg6bvMEI>8LR2&AP;xQoxKc;OQMr6h{Tx%e* zH1mev)zbzH7bQ|T8PVloadcYYvPMb%1w4z}yzB-na+||#c9&C=$vUMNVg>J{dX}JV zcx=%o;T~&FtQV=?T!^tuMo@Q3?5yh74qzPHmHU|1MVWK zYek_?gqS1cer=)^Q0v-NKBxIWNqS;X{pMh}n5%d3L;A1a$PP7$i;unJndI?k_19b_XL<3LQY^V4F@ z&*()~7A!VRtp!_E)v8d-?eb=b)*5ATHFdze#Jd!XXu`R`7gXPT5hN*|zf9m%(kAVM z!FBx0So*^k|9!F4o>|9#`jH@!{WlUM=HFqdO>xhtA}U%C+->IhhpiWhNc)VHYKJBUZ~|~{UA;6+LSg>{wq1~E(!8Bd zcZqov-+l622|{$o;hTPnZn)r+e($ew&9 zGBGODjxyv(Xs{`beBWAY6nO{{tEK1h}g{2s3=Mqz2?qMBC;U9V5iuPj5c zVTma}Jw#lz0V3H|j>OEY$-CogpHLHvUCmgNn#jFR%8sI?xJeR(mM2hzr;j~B1*KJ3 z*<_f9+5^ zix)=+K|QD-OOrbA*Srm!9`-z+$*Dzy=@F$3eToBYi|pRKIAUGH8~M=!Kk;T* zO}PH>qJLkYP=VgZseJ&Y_G4S{pOtf2ehUx6Ug9CN~MlZ=Ni%R3q{LMGhyBa_m^vGU8AR()(VUkLt>Ga7X+QbgQY{_O*q@* z$IpDBF;xnC@yf9gHa*bo=_`dXn~#JR#p#0Hc%vUNj!qdSf^M}hgeeV0AJK^g1$ALS z-de2sG#W2znm->r5}$ZTi>rr}N8MsPMym3n6rG70{3u&p=NfVz19u7sn52mzK)*7s ze09X9GDo$*;; zQ&7T?np7??1y~N@@W6sN6seomcl~WDj1*=1;7i>!gtJ!)L)XmqG0q&CC2fN(rNkr3 z5f7Hg+f2|k2lxu~(6xeeVzG;|2=gL5v=ewO8TwNUP+Qo_a3d__pOMG=?bS~_Rd*wX zUq0oa2?(z&Bv`OrL+yUi2CHb&+e>6$cJ~c_gtedg;Zz?t>)Cut2FXG1$gpj2hq4G+ z)mmHldLDfbX-SN|Pgg2DMV(=^#7N|k`ZiU%qfryfm;&PT?m}8=^lTx_9Np~=S|9Xo zx%pnR3dU+poj!qU!j&pCZOkKL=#=Dlj28-Of+!7U;>z9@dN$1_y7%R>q?^GAktz1e zve;TVi^+KhYveI$F>2jBg?o=7PwvL1*(kdRpp0Z5`6`ls_@@ltUG&M;yL2b3uWHv% zD^ff!|8l|fk9YRp*L@+h!`^uPbieOZ6!z7g`va?5nc?!R_pERp#|BtUyM zaxc1x8MAQl^x5xzs1f%qoZhUv5iR4_a$V?(4j3u79B&UFD?!d!ADfmjxVtiR1ykmS zsh%>tY(KJy(%WVf`$CpE2XdZW+|K5yaLBGqOPzCn0^Lny>Pfga6b;rxNt#JF?m=m# ztV@{bnLHEGImDt9=gW^9OcPP1z_|KC01@Htx}hPJJL|ZVy#ZDsKgll z*qy!+7HwzTm~Gf-o3etNaj|UZba7Q*P+%=i%tF7?R+GgOB;K&)n>C~2xHe~pGrH5OhF*aDyi43CoJO`O%2~-s!ZRNV7aORLM_QI7;W{nEoVj7G=a%e+~zo>=r)>MF+ z-PZ}Kn(u6ni?{P2#R{xldhMhxj?98lpXP#nRNLg#-auC`0Lr@Ik$RjBl}=kJWMWUA zw*Dk_;Y@c4*e^f`%bsu%<2Si}=$-rT%@r>xEh<{Ox0L-1%=*BiVlEU4Px{)WhPG z5Fv>sp%v}%zX3v4)0VbSN$>~yaIcnFIzTZfY6~rMmo_*1F->y~b?Rb7W{KkLJ)o0J z7j*iJIh8IWpqI!({EiqRclx>v~_wvIhPgK!eu!`u=00+ike({ezx zHdGE_LJH{iTiGoY21BLRrZ`=K{;5Du<9fhE5T6JP}N90W=XXj--m`) z@THCEJ7q@lnh^E$pd#dqB-Tq!-Wms3N7v6UNlroa_2hGgVxDECD(4sni?qQr)HKwx zM>KXU=AbkfR*slx2T7WjXqm%x3)gwteD;ZbaM^J@(FNKY$^!2K6rLhyQ5eDi6RBnw z7EJ6aYV9mEsIcm`ve9}t)G)`<8vTrirdtE0JZ+H0@7?jqF^?? z3DqOgy;j!nO`(p{BeKAic}}r((W+*(Ohc{`s&?t4e0}J)ep`(M&t|(Km`7Y?qZ=-3 z4|ZRV%2D*X&nDuEdQ;RgX9i-I=>xiDtnfR_8p4RFw>B-Y{gGj$M{zg}i)cb_ad{=Z z!K`C4%BWBsm{4GSkkPrxPP?P^13ry+vpz%o0V-_-pDW-i{i^t8)E?VH^7ckBdtgm> zkjrj6N9Mi)e^_~cvzv42`#QYpHnQio(7}8zD`sHNMLtqgx!+h5T3|ZS84RZ=B6Q9L z%VFQ;C~K?IH~nxwp{wiK*ajc~n|+YfMmZjt%Y|=c2{`L9%n)p3O*`g|0EM~gD0?D4 z@cw~wKCs(UZ|0-eraI1e~!kiLio_`R=;lGj3v;U4TbxIa?*>v!4 zo*e{n=PZTbGexHgZl_7@Y)0o`0>Rzb{_2QDcYANo7Kk!8=Eo8RTG}-yJc~y6KGrsM zG`0%#Mb)uUf2b)g8zmUpu{Sfv8`8lE^T?%*HEmhCvGOlwcLUIixta=6ZSxTDteJBO z(@d!Y{R`O)KopJgw6Qa&#Z+Wq#G8mha%sKQcXIb#0LhK}ZPIC_2Wip5s0>*@^7%gq zlV)0|nU^dBlfnifDUuJ+Z+M=R%lfJDh}l|tI0>wnDUPx8q#kGbCdDYgCdIS2hKnx^ zHJWu)w=EXBO`k&$=>stle;|fRb5)ccN!~;rx1>lwMfVa1nB>-vBfOEwEZXnJKyPZ( z6(7njh$rh7yoZi;Pb@sDj5#!&am{7>=1{~u&GQSQWD1Ok*~g9t%Kex0w;!8~;va~a zgZZ`PK{J(qoIHkM>6{Xj8}YH_aT>VarirYA=m{Km18WTTLUz%Xk;4e?TZwyWll>Jx zpp7tOn}mFTt4TrjMBtg^lCG63X}@6-yOU-r&q5ZP4AvsNyb{K zo7QMPZ{|pzv|Xv;Dp z{-&dq4Q5+l-dXbYBZxl#RtDGDm(0XPf2&A@3ZN$7sZlvyQ7yJbA& z)_Y~T6&!uFIuz+Ni4&%#ogSo?HhBm^2?UwW&=sk7H(U|16Q*R>qt8oWz;J8TEn7ZP zV@Gm7LNSID(zf1Y{So4dh8o?!Z$*_1Yx>4GC>Sb`4dy%dQ1NhbPa;uPn>EXt1p)9a zL8NbWvvcG3=7Ufcam%HMq7NsK7x|UD;3a28d8KLL7OwQ8(7(q^yBSVi^LG=%35kV!^0tz$`)CI)5$GGlcm5dfJ)ImehK~cP`wF zj1gPw--_boi+=&_q1%5M4_T{Q2Q^))iP7&auB0qPd)w`QviwU((TxZLNpi}|8Nno) z#iyD3#a@!5$WH!=^;^IZdEj}|rSz5_@25OTI*-WwzEw)Q>8&2h8qo%TMb()S)1R*t zM$RjTDyRyKz^_vCk!0_4f<~>Z@J<}Bc#A>}WeOIhExN@5?eu+td24zcS+jKxFpHLA zyp56PLEI3g1vm6nGrz5$W2aM5@an+eaB{a0?m3sx5qv4wUyc~e|Vn{1J) zNUMkH7dKG0kX7NFGv?UHH+(8xwx4@Ih^99{@^QXSsXW(r+0>d@W1s%gQsB4!Us?*8 zKeQC*T6i&$Q?M0pfUSvCiq`PIv=rb$U4Z|gr2v3T`lY2{3wnTu4C##PjK7o5Gl zFXU)su$m`^?QJhF@}XWr-Q3Wk=!PHJWVz<=AkBHC%IHIDyL@q9;EeoFEyelv@GmXJ zOQ-bTv=mm$Vi~l2Q|hT*Y&w>O?Ipf*4Dsh&{!A~lOnc^%f74Rz{il{Fghl?frQRlls4U@BeQx)gN#Fzb^^il{Uv0KS&Vrk-q#>68u)H`Abgqp{DplPQ}^n zO|P@1uUjxkY{FHw+yZH5*lYP~6VtobakamUGyM=GclPY4&7-t;v9EZi&`=C5 z0e$l=BnSb?Bmrg1#(#ah!i`>NNhU^C6N07NEdR3aFcOoyn_ZAk`i#Izp1V1au|JzWc{E z<8YBa_#RFLwcH)`b@@fTW#hI9+MuChTc#=UV?kC!Qsp^JpyeQ9h&Hm@8!AqUNZu$+ ziI_eZVd*zhkQ|6It2z^PPBbv#2JI13bpx~GM})OV1j$Ag#+tnm$y_&HZs$>I2F}ri zEkLR#DDcm28YXI*qd6L|Un_|9#z)g-#V*9EpY+xxiuG(XR^NJb-v|kojHqR+nBBODKHZQ6$7a==B2fCXE&5&H17;@U=}CgzHVE! z`7WI;&c7o$pACy9T0u4o>RHRD7C4RgS88%P=98p%^@w})GW!2WP5zts_`_xXeeq$n zkj1h6fe$L!|Ew)qTG<=e{l~%JKi&M69;->RRv+{jSidKy>X0gfhz8D{5}U4foLW*8 zzXw#`23FAHS~d{<@;c@XG+12ZmMW9?z?MmS|2*I%N?Y!jzzZ{qJ_^$Y`CCnM1|$Y0I8w4g=UMU2N?q8@fZ*C%P#dQy5!Y z_hZTCB@6pqCGD>*&^^MHP8Ge64`T##?28))eGSUGDwl@|##`&&;cGLp4t=j7NVHtz z%|Aj3o$u6zWh@mzs@*`cgm$!wsqcPRCWTL@&Z`*JutEAp>c))3Z2)s-(Dpd4ZwlCa z2koLq)*!byVXmOu35PZtqjL0Ia%ZlGy2ZDU%&+))#xwJ{CdkvhWILHa=V)>7+W=SW z0jMZI2uiP*+LuY9IoNT~#5HO62+FM?IpIQ!@%AXe%!CoBE>-E3$d=w`Q(=wWwO-VF zjVJMy)&?3^XduXISbr1g<0JW6AY5mnvGw9)81iCR$d|C26RRS?_4d&{3!3* zV~x7+l6*4Et(V4;?W(&Z8g%YvjrRk?0+(ZD?FTYlv?P@Ul)nwk5(dIn#j_vm9Eqcz zJxRi|V@K!E+rqI4?ilk%4*Yzz&Lptq!W$p>>3Jaq@4|ye+y67D;u?a17IbeH!LBw< z(!rY=Y;PwR!@Gc^E7pgdTUV-EznG9p@oj$RU>=dt7N2^b3U-KdMwYT}uK*G|1ISsr!sV{e&@MoaM zT}FPDq{KVORqi)|ZMdxd@<~*MjTrSjx+fIksl08Az&0dz(O)2R6ETQ=-&&Vy5Ti_F z2F%`o@zGe2#$tvF+yxC&MP3mM^$W}c%is^TUcOo0e&&rY^_e7umnXYD*~4F?D%GpM z@gguA!p2+1m9f-1cQz+_yw07j{WUxLCnpYntKmNV-xG)bd*bkaPaOX5iNpUrarpoJ zi9>wOEb-Wn6n4-V008CBiT&Tt8vZu+Jk?OM%V9(HT-V}zbfAPmY|igz+$Bn1D@(~Z zOSw7YqC{UdZ=}{HO)h&qr-TTu;$P7Ob4n9%u?C((;Fx;6vB?E`xf#|#IAde@i&Q<%(3VB{JOoo3xUOMKs;3xT*LJ7?WrEyo8j>GCb5q1sK)Y`=@To4kgF{#@Yb8TZ*Jy8u%P?f7mCb%yh=tXk`(iNV-lrI(T zzrK}$e@zfm&J2B_cvn3*Sy$46B278skc;_Qj@2Y7BtKFXVBgK$yKfwPv`Q8E_blz=ot>UX=SeVJEaUiq*fzhptX;YhPY)Ib)1Tl=J-KfDq%;Aaem z*F93P4CBGiEy2PJ60bL+T?Poy35I^2??4UF9BV8N7a#EyWtSqScDnmZOYTVbGxW)W66ZGb8~JxmsP%SYeA z69r{NymfgX_s(2rg&Zco@5r@3AS2U*{Y%7~Itx`8Nzey2p z=EFAIMwcpTk&D@!lL#3Zc337~99dN2H1zE7l>hk-i~B4xc;A+$MrdMj_me1h9$5uDt5uTf}uh;dpm>v4c&zr*SpUk@0rDV|1pH zlhNXhW(n=P_;xe30$kF&W+(wR`{nwy9BZ>Fl3;41CDI`S`?xOmKrquceK{Xj`ziD4 z(rT5N?|dGB+&>H)6jukb&hFITe^L&VG)~*`DISRBZ=N_MtGtoZdgIQ_lsVN^wkUO5 zAY3z9RyKrSo#0@zwv&P#Bv*Ttaavh?nz!qAOeIhT^^m+fY~ z%I!Q=J*=*Bw>zC!HeG^F*0R=@ky`E0EFS~`TLW4j0&b)Fj*bRjFguPWNOWGARw2kj zE09fel(FJV*zmP$eOFiCnjSMS%p=g_r^;v_@~3sZs2`x*OLP905$3&I@vkme%0*W-XrtkC^g0a`xp$|8+*TKby zd?s()t%U_MlkFgECI!>eqv-Oprdtu6^s~O`8yM`{A!A?gIIYdw6-3FCa>xu~L=dS- zMve(iokV@z_p5$)cgeglXgm$b6pihzG-vQP0QwZOZ&77=V8h=?EGiyL8+GkO`DsKl=^H;qZ65)te<605w z6@Lx-)^etNjaTd8=`&ub(d>CIKHs|#kMCzbL>CC|u~dKBU8m9L`3yC`F5tq*1!Cvt z`^ekQhz~7Va2NbBjf^OD$mZk^)H0P-FkiptX^VFy`bH)AlCR`yLxNrTR!KW=gWS_- z$@?}&-DhhPhh#{0%{bHfxq@YB&wkS-Uy!?i%h5J9AC|EugFIWP|Md^7GJVGbuxB^f z(Xp_@gQAwN0CljP*CUp%yzEy{nKK4|yupPoS4%T4HkQq!HV8(fV}-(A(Ou>=Qa{sG zMJ&7O*{H>1QXIJlbzTzfG(T z4UO9nz;#5(8)ZKI9aGvu#*#JSH1rnhTa{tI67=;+ew9X5J>}d$v&l%flNCLa`^ub+ z935i@qRxa|W!^>uZ&yDH^ffhz8e~1Pdo8kgt6SLQ=61bnQiT?mWf%sW3!V9PJrzg? z2qa6r0eY2(Fx5;1{02_oT^}laPs!k@U`|Cc@rypHnMNzOj9&*8^(xAw$WX^Xwr=Ff` zpPs$9*+68{t#y#RbA*(9K32rYI-u7|K#2Fadv2i93{XT41~l@p^uM{-54hp=1?U-V zk>`fwUSDUz%N)+&FLfzn_5GeqYXF&O24w;xoHkZqAx5>NV4d`#iIx~tFCsqDPNn#k5Lp3r-5A{e?{r3i=-6he^>B26jMi=tEs zaRVyoD#)(%Mfyq;kg5_PfC`96mtKq_MT#_otcalCjiAfSFg(w_;ji#K-;Z}rhMALd zzV{t3+`Uw~8J5t;Qd2aF`c*b20eyiYp6|q}6RIDbV~c8~j1PG5RnY_PR|g zK1uo{Bcd$Ax_lGUq4unsag2yJVa^)N)Ac(#5!#K*j1y&xZ5ZctT~u%=cU|-xSBva# zB!-Za4qhW47G{ZmDo=J(Azd@LuF}XCUrH5U9WWu77(KF#Fy?-6dvA5p!G5X3!MWPg z&hM2!&Z&F!>g8%jFpjog5EG|yIjf1+2#<4R2uDgf&8DW1L%XNum#u9@Fkey$%}9ZA zN6nzTeC)t*L*T%j7S^$`vZk7%gG8=1lc9jihO&kOB6B>k34CI*;!BYZd2Nz^$NQ;X zc#aL+qgNLxIPP&uw|_nHJ0wbCcH(;r;ivsw1U#_=?A82V%Oe!ad#?^noxvw$)HsRm zwZwSVURRRG!r!_dv!2WS#`>s8BL-ckI((iue0)}z;pKbTsJi&DWC1qs zfV-RlDx-Bed<_dl&P^&G5?quzugg%pb5i#D+>_?H4mtww z$~RvNO{0!r)0MC+lPa+t7%`scm)*2e3vD+VdM)}zCN73Nux#Z*7I^oL4dIp`qrk3> zkhi3?Z_arl5ZpgU0qgAIj!1cOQgH>+Ir)!T>OzYAY(HmQztB>5_?6?-bXXM6c@q2@`r zknGv^phP}{e@aMscU2Ru0%Hj?9YKY@Gvf}gZ{NL@g&I!F@$FvS8_s zj&1pqSk(lB!f(E~m8Y5{srE~s66ek8{pS(k<+IK+0t%ncy1ArsX%Sr+k1##W?n%za z7YsO}3zR1K_C|TSwAe3+5hIhirI~a2b)RwUQGP=Lg~mJU&bHZ`VrVrsoZD zdIJ^gYl!_Qt8-QO3YD~$f`@Bq9=1>M3udv2Rzt!*GxLA6+EyFoMBn;w^)H*Xd-XNw zAi?{nH8=E|-f-b$5AO=uN5MBZN;_To^?=E4 z$@UW8fpo2*b4A_NmY;>)Uin7t>0QC0%l4bf5&Et;m`1S`6_gkMfw#wh7L`xE6n`aU zAf0>f>QJfC!qlt+DatHCjqRepmz)?H=NMIt!S$PtSz+>;3>A_t1=VrWRo%GU#ED7T7RF<$<%km^+Q|s%fcvbW50hB#X5U;d+GXDvP24Rg#jM?StG2Tpaet=rIY6gnJ)j(#nA$U5YMYNG?455nNVTy!x z{R1w#qV?Bah~^@d{hh*)CcC`uMb4=+ zg^K=S#$KyE_LaFKyMtANjFzigS)Qt>f8@ASI@N++c-#5#S*UepgqoJIvFzR#Z4aou z1fnK5#M$z1W|SkwWjY%*Ukd*7TB>fevW=X3>+y6riAqB_gYCq{g9X}s2uoJPJ_Il3 zI#WJ`P@Fr{7qMQvAS)$JA*8brH-#u~$Burl@~gM1$9EBRdT3 zxGDQTZ`e04Om^D7K3GZ!uA?R1&pp6?r_Pb~M0;P-W`4NX@JWf<@{&}BK7!Awula-z z^u%U^F}d}a%NYygi$bJ*Ix88HXDgAfCh;BduD>^oT@Y_|V(n)vb~(hHCxb|MZ<&A8 zBgE&aiJbLFOyOUNwwi6_BZCtc{L7R_s9X~Mw7Gg#T_@IvDdx=$QfO9!v0Za)*r8|^ z)AdE6tQBm~9!XnK>9mXvr`85uw@X$o3xV1_N7$d)EVwVLIPUX!lVmO-+;p4D+Hk!V#hg)0I$*i`= zY`#dpOJ1J1oHJ5=TPw)P+x0-!+rwIgOnL*Q;bw(*+>PXVMHQ{k$DPe-mv?ol)Le`9)4mUzTg~HTn|ynv0JAp3brjF-|JPFC0tVPhXaqe(u$i^vv@- zxYEY+mI{}76LkHe?ekgb$Xc>rGt%*c@jY`dH0g#{PL_7|(dRvQi_H_P08x-!Y_n^e7mvbvQ1PWHG4Ex1$R%Q?2Ol6jLXXN}Y0RgMTsu z<=3Fs=DmMoUlm#ieq;f#N`$iAvh4u|&~k9nKd@Yya-RQN{xkO z2gd6|3E;GLAYq&`0r)}KJa*XZb|?%SMGk~nQqr7%$@w>=8(I!d^#+#fQ$jd@l!G$9 zq2=I^Y+(6eO5fR!^6in?PzE@88pzm7S?>G600e$+&!L9mz#*bQTp{HOwhafEVs36| zZ%s6zWbiH)kj%xoN#4lM1f*y}3E1ajpingl7&1WUhL;_R+gWJ+UKrdqp<4!dC>iW62MX<#h9LulZg^y& zxSfUaP)^x@{~dpw^4o-N8E>IvuxAx0R8kg(3=q2E287~v7TO~ZgWD!_%VG#6gWY*R zq2mW&$N-@m#wsXoXQ2v8Ft}|(x7=D#GT7(@6smO)h71t8;m?5Lb`~0shQVzUx@G!+ zlD~oc5=nVbDOS8@J5=U8>wM7~XcNJKu=GEonfh8akV3KwnLO zJm3wiQgE*h5Qo!;!EGvqLSe^r=NZFwgH1AE#sGWg1n+tRwKbUnMiICN3jH9!>uW$n zrVR)I_y~3$d+<6Hkd|lr^N_%+R!}+M?~_1)h9d$2^X>A45iMmVO9?xn{GeG7h$QD9 G@BRyj$kC7h literal 51771 zcmeFZWpE^2b0yqjW@ct)W@hFQGcz+YGc#+%Fk%`pGxLmCBW7AXkG*Sq{YKbd#P`el z){nYXx4JvxW@Mgw@?=&k%7B2P0>A-~004jp@II##Fb4zxbix7vC;&)c9T9sw7gIYI zeN|5fQ)gXz4_h0;B2Zw;0s!#W=l^;9KWu@?^bO?!M#Qko+-p4Uc2|o&5^hum-fx~+ zNCbqfFro{j$+VRAm#u^@m?gxf=W>~dZlCs_C58*_Z?bBP;76r%NYz*fIwg{gfkXf3i%aMGbm3UM50 z_T8BPeycWlH>(z)v2c@4^>#4#+Ig@;R7LVKlTT> z7QC5Nxmg)XCbp*#NS`5?sq@Zbb7u-(k>c`4q`)356Eif1p2@w76h?Ops!HmDAD|&^ z!zldm(8Fx5BB%NFfwmB1#(`@Jx`xchW$e2|KZ%l#6hr279%7L9BL;A?3&;~_(+}a5 zt1R+RyNW>pOMQ)tUNl907VkzuVHsG%&w+2{D16#qC{#s!p;<6odTjzOE5>HVTS+)6 z36m{nGvTf#Qt1qnJz_JlVmpJp*QeleEq~VMX4`;g=Q!+Kn=Haf>p*d!G_>vH(kS`k z>D*tMhuv5$2s{RDo2Sp$q5tK?pPwKA#s7c=#pglfa9>EU{}t#kUr3yeI zp9t{(aM=I5rPn0&%MUWbhFwZ_iTrxavLg(co!3R2w}VWDV2s=VUniC3ZSLkX3{RCV z&H^1-z0SX!xx6GWJ>976t z?~n?K@3$IcL=?RYeG6XoCgllNjxyC3CQMLqXn0cI3>UE zPT%CAWQ5Tbrmapmnr)Cvz_$51aSa1%eusXvD>Wz$tLyz_;b$@YO_;ZRv`CTHI>RY$ zN4g=?LMV+WiD4RXKB#?^wJPE;kfc&Y&9|!ARIt-Hs(LA*wiSG?u7UM)27YiCHaFOl z{?~0b^iWfi%OD%CbH5Qbm34xX_tEzXQ%?fbjf$vnK5b^hWLhAx)&cK>JfUw{JiMJc|%@c;Hvv#6vH%!mka<&M$|a?*)fCT|Sx z%|!0a$Ykdi01t0LYrDWx?&FNr3=8%kAL z%kt3R-R^0Elr&qaC@wP0E!a9;SEqhi0QwwWpNLM9U&Le#m*~5X8x5`<*w9e$E^Yda zW;DI_P#rbD2)su!iOl@~HZ9*&Je%3rPG!5%VnwU>_H(VQ$^>Yp?8!=KA(ZZit1`+D zI>cuTQ3ASea48LA19!u$+cHy6rhB@YM7!B650h?ILm#xC!kgiuxTHP`h`!eF4X}}f z0wRn)g~ns?D=yPlh>L^|(5;J90{8DVv$HwDB}7COCQx<2?^o~%N+{128GQL-4$Gh} zMR$;X59Oo&Uk>u0L+!HBPg?V9lWO^*tf>DQYX7&Fou}>CWiz4#VOmp;sT{5jHN+OuU9s^i@} z=`4y`sTijTH!E9S=+0&}itP!hK`HtDSK&)53=`ayY?zAbnzW{r#j?c8e$B1&xFaSK zl%eg_R*IM#8Xc(U;V-+twBLlEmAQM)!sMx-hhDwjGheKF8JM4xOywdc^hH@m?FbGv z7Z0n{{l#s4qI%m*L}4}#rvh{}#<5EVGc9cHHB(hh*u~6n-Xz;6G)4zU-^v-$?^Hf) z&Zl5$Zc<+L5W%i_VG*n3Ex?`m{?w#R`;U((D?E#tAjMU^7kcG%z8kDj{eIL#{6YV~ z%2Uo8gWtMsPj$xygBgvKHBPAu%2-7Bbx@qPA z@4W)BYB~+iaQay~E0Zm5nzmo8i>Z2Z)tA#9Om3e{J|ssL?V~HX1~*!MMagCjR%@n^ z?zU;-TFt!QihWNwAOM2Km~)YGS0E^w?+*U-HQGP$c$3zD2meQ& z_;FkjPWY8a_3;A$X#eVo{|fs52KVPWI`$jvn7#z{pM=MJEft+H+#(0;8v!jer*^(( zWb5cfIFxHr?B~6g35!E+CFCtDrZOqvt4RV$1Pn)=xJeBDKcrA>Cp@r@)?_IdV0DeP zG9(;Zvrc+_yLfM1zt>ox&ymio%J!O;ehfZ8?sU_ZnIcXp)*G8SM{ z6RzJ<Z2f-?6?x43p1EwjY)6;CWYC?k zh%%g`xJPKo@R=kGV@M9(uP!6!?$)}wdn3y&vR|K)9ogN9`lJ*sKQEgXzGpt)4m;EL zHW|Fn#y^o-0I*u%pmi(rb!^~uo=iia8%=9KE!4D{3%|*?WpPV$vC4VdhDTWP^n@x2 zLt!;WlR;$8!fT+i9hi3vxD;i^bBS1nlfPn=4mm1jLMFL0?Y3_Z>PpdM>J%f#m_1L2zozSTCX)}Hx|d{IcGq|;N0 zsb5oUec{}=M?eXxJHb@oiwd(un_mG@W4^%hGrxAu1k6A!vi7>`1y$p0C<|SaDeOFC zwo8Sfa4+>#N4chQ9?L+8O;>=3F1%b#oWDU{LAEsu&PTlsDvcqRd<~-YXVj1rvndF2 z`BtB(##EoAl#)p#kqt~@kFeAdYL%%NQ>mnm;=NOSi@J7ehPj#!V{B8-W6OGFhw2To zfvZBarK@wIL|akE%oC_=b$%0avWyK*a(}f+o-t8`=h%R2Tx8XvD1~VYVsU8Ta8mN8 zqNbdlrXyKMV!2A(BUB&Lf|lFE2JqoE0_~O;)|Ird8jXr9O>u@N8}NXvskW<5JsBL+ zt3ZRvL=|VrI!~)%)kFPT5r8nTEEIg{*&2V)%=*%^XP*_#q8}Iyc;7rt?2@XS0KcFd z@zY%O3_CT-0JH%#;b5DMFAftO^T2(>ST_DQPZ}A3Qa0Wkc>;2gk;4tG zY=re&GtJjTM}{zwL)UMtDar6D3UhHUS_y0P5B*l-JR9jJrVF~MQOavoObSMM7kmoBYz>c2*KkmkrA#=&I1MP;}`@-aio}|+*lO-mxrfaEyEc?j@ zFE=E-!7xW<1gvS!!R1d_Te3pQqp`e8&Ph7b?Z^9jWgZhh1g{VAlgK1VzR_1|mY zn@%3dbf+Z-^gZ}xTxQvrwAAtkOC9rij(PgliZmiVInGn($f2#!pa*F9s%4-N<4-E% zV}zy8P=v|76&A|eZ>`XY+^$lK)7EJFRUsleGEj5ZUW3dQCQR$s+ntiCPPQME5J+W*GUo-51pxqT|6q z-S_rZ1oRvKr+p>21o!5b;4!_=dK|UO^3X0&)!R4V1a$o-&-`w@`2((ld&Ew;jPJrKp2Pts0p;zz}hom_3EZWBcni3l7kIAj=)O&ioItSv5 z8K6QCgV!HJScJ=`>ibZ^^7?+K&oK1+xO=9A>CnfaVSyak<@}iQ7)2&lsb*P1rlwz)j2Q}_D=<0Qv zop1O99#u{^OP|6j0)g+MYlfRAJ!%{$rPD}mz8kEX>};u8p~>KEsUJ}#7AiZ; zw2EyOwjRdzyzb97GrHr&>r-&1oA3!MsxY389)ky&A4fxCyIIbk`{zNc>E;h-eLUW- zC*0=g`#s}t1O9An+i#@@vsew~MCLclcQaF^c3oZ0iQKUwdFjuMUPy;0pcEI5B;Lxu zR$?$bX&FpWihI`R7>ZWQEKz=-7t!BKZ`Eh~E?-o~O{-?m35W8RiPmpp#u~zN+Ef~l zg6yJpu>K|pD9VjE=ApOzg!@O7Vreevy61~Z{FO)2{(DLxYv^h3>hd45$$yz*|36vP zS5682CD4gp27Mwv*v)$(CYI@mUhRtAfuxyifNzkR6d&(o8-w*9oHN?HWxZQt4<#da z?y@a50t;<}p83@6x$)Ju5^@%|ae^ygZ~v^fPmewnCOJgj?-xT4-PamSaNeXEGAn@6 z3X|;N)RjazBvz>-&f<9}SypRNvYZZhh^CTP7vWyR{&|llbL;PR%x!**IB86r@2^UU zpIge}RHj;=;q^spb9OxGt+ciwme^0N?Oy*O$!DEvpC!S`V#Io1A_t&;W8ANSUH*#=G;J~hYx3uAw|xP zNw!=4&vMF%LPtAJMXL#LZ$lKmf*XBmTd%#h#YGpy-{BRBW|mnhEK>nSX;RZk!CW%K z0pEuE3EusG)K<0rG}qPhR{fLCQgNXjS!cAUabz4?Ip-*ubkdP4k16wp0C*ovqqB7G zVa=(uqq$=b!;@kDQ%kuxTLiX6Ga=llHaV?Mw8mX#gO_A$G|o<>g|CtpZnBZlXe)VW z|KNdfMo+pil5!!!e7G~>QQB}fB5poofBpZH!*TwPbh!WTfyebf-VSvCEdXNrx5)dS zgh2n^A^w|&VEiBN5&yj{iQ}Jk_yniZ6)cFV}T&2VI9Qy{-v_A`ae#N=GcybJ5|ZNW9$t+}h;yA=tNF^hhP#F$Pr zTU%YDMk=lB)LqP9aKQ-3}>?&K!w+;C3*c!1+tGCYQYgTG#Cc{y3a@cdh z4#Ew9&ra7RH|+0x5Z`~7fMA-wFOoIb6KNBQ)6xiyl~ASgfKffGOhoyz8M9kpRb4jz$=x+642@XoqET9C;@zUy7y8D-N+ z-hNOG7F|D@hB#zs0obcTlJtHYYG`9X&eF3(Pop2Akg?LUzx3*?TaqUKXKzTTkBEy5 z0(6hEg;d*oyEm~5Lzyxv|6O9#cZ^@lL8!3nsAM|AGXWaPy#T-V+NbAZJ$-8PTQ+kX z`ek#T%U;wT?Of4tKd(9DTi3{`dO`(i8Kj%_acVG^R0an$qYmb%$YqR9l{9kb>>(_79OfFZ* z!f0x8hf9ND$(^xXq%&t>k!V_Rj~UetJ*?d|q=9KB(;4Tj0uZWI}D(F9%2rr*1xU42+7vcT!fWm+cPIhtH&OL0U%BUA!#G zllfU|vFP@wbF-@LU^Xqc~T8HdWt0E@t$5{9n(2+pFlU_yh++9TxgScdFX$ z25BgJXk~=mu|>wXJ~r>nrA*`x4GpG2W**D6cr(48H5|HJ-UNi02zf$(6u{V^88@Ey} zc%!qsY@PLTfM$S7_*7|G9IzI$*X6ZKY&ctqx!(DLjsINu>pRHGc=^%;4(b2uT>E!5 z_W!L3{0%JsP7OHM*0nnjNA}YvbSKOh5hS_JM;3>t)f$K2m5o{D5{$OdilIu{u<<+x z1v-TH4&Eg0wC5VnAyr%2LN%3PxmnlK3fhMqP-X%U3I8ADk; z*se_;-Dm8S!*mEHBRkz_jC3<s70|Cck3jJ=>tcb+jpw7;T7f>Q zAc_~(7J9ub)V9alyrfZZcDv?&oFE_iIih0Mkb*}9+*52bGFwKCMDh@V&N6xQSlW{A zEik9{tuk*a_uTuzk3&w3%*l4ww5AY-abB?6gs9_?Q>IxXS8Z~Q7PS-yGu2RTYRVDF z;oF1is!%d}n%N}L2Cv+#kX*E>m#jN$_AQGc1Y$zG%Hi0z*=UY?lukQ=pPq|J2Tytk zb`A{ILa7zM-MhYNdoVSFDUc!s>gF#i)3M1w`nDum5!#EJdPN&d?g|*dqeILvNMA?W z{=SUzvUtSX33r$2D9l|?vzVD=H#6a<4|KW$r9n<$X-1*4U;xwHxRy@4mA#bgM%K?; zx%swzm}3F7+W=I7#&00PB@%ojc2w@XA%2>%JkAHeHAb}TIYaK~72ssSiR_or<u`xe#wILwNy12$3pG*dGgue>#L;>)lsGgVrU5>th%iGyFHhM8cisvp z5(6wQhsSeTHT}<0LI2wMJC1v#a%AN~rAGPJOLQ*%rV)PjB;i&{8RKAL7(-?dvAJzoBNC}TNP&> z)*3*UBi+2;#XyIN%S)wbe@ zZz6mwhZ)|~rJG|bOpMJs+~7{;>Rf6}tl^m@jd;_t6n22lZH!9&^iNoT5AJ*A<@)@i zr;`fK5T6yhk|!H?K*@RHF5fj)u|V*AFnw%-?1t;_#2ia;O&XCqY`rqt1-u%F?@nO; zesVyd@Bi|YPjR)ZnwepQ&#YYWTEE)2=|a|6_QOlj`vwY;2KwMIGYKc zcNs5neZ89f298sQ>z@EA|2^Ej(k`%Xx4U(SSw>VV-T9fEDqVN!^H}{i1br295mmEb zPTrqlMT`L`;9dluKR7&r-AZ=GZZHyz-(yKJcACb6hQUDHpyVJU7DRZugR0j`qm32E zCUAa8*X>4;LAl5ng!V9!FxH7@vE^X}GRw*J_uqwLK}|x6a2VUM5M+O+O%Lq8sHf~(~1V;3zMQL7!T@|ZeC5C$Qj>wMvY}y9bY?*oVz)n zNa(3MGp&AE=r7&wIsqdx0r5-EF|d2Z_trv9x24g9lgKVbd7aB;J?w#FBA=&S6p@g` z)SaAD1x3o&oVe1_NMpLYQtjALIQ&;$RJv>qyzPwpt4DnefCH}TGy_OQn78diD7EUy z2_o1M>NkYhhqT-Xv2nZdunn6YqaoQ9Q?r2Yn{I4tfil!jXF;{|psM5ZfU093bZVon zXtL}e5@62RQZ|Plj)3179=f`7-5#7Dz6kIfe)JQb!kma87Xe*txMeMy1B0=ZtlXHq zf3m}0ZSu~ct_2ZU@%>;Bwu<-+cw1K?xckf_X%!+Mf(ou7mpy)T6B4t3e_t|O8_ECd zzO01@g8wNvh}60ZiJRZGP&m*)mt`DuwLb+SH0vQs$NQ2f(tnADAg$h;04Y$|Dg5=* z*VqWD8+Iic;lFF)m-!JvVkH&)@v^4pG#78J_YCsTTnu_c{8l*b z>;eiOE4X2eOyOoM-JwTkBate|HA55J_tUN9BC$?8q7HUFQ9@*1-1M_L$c?ns5V4!v z(#nn_(~PG*HWpqhTvoc0_7Jth`m`<%;Jl$F(5` zvEqp;4UMfY_oo^6P7RdK{Hf1X`z?P`DCsl6-!2+$)`fJOV_LG`6L6%RI zVeF7d%9a`Tv{wu~`7yvMwZKr~DTNM;Xn%MvvByU_qgJi-R+(+{Qa0_|y%$Jo6@>Yf z{=Q@IV!G7~#)Qq$n8=5aqg|s_b@d%Xb^u-brq#EC?r(@B^CX_!l|A{`^nCLHx8{@C zO>_Z^0|?qG6LekBx+~_|HjqDpbT?O{7&kciJ}J;rl4d}WwUe2?^Buy}f~Cqnt>o6L zHg#?}+U+t{CXDR6<5_^~QA|Xb%cAE`=Xa&uP=eXLEZ%ik?xw%5KWkuA|3+8bMhla z)no<)8wFE3_Syhr||7sAgta}n*7lUV=Ouc8i5En z=o47T5lP`Q=zxf^m|;CZG)3_60dPVaJ3r*>!;(llt0Fw|6$$jaB0T>7bUlC=j1@4= zN`~ln>PaaI$Ql$Fn$owRbovND&W;4zlsSC3q&KVi7W?SH$PxGRXO}XJs|4wJt&lJf(q_s=*Y1+i4$#Wzirtv%e)>#cJg`6CWOTvp1hO&~ zyhyQmjb136+N$#Oo`+DcW*y+R2K-Mc3Y5T>>^p{2`pa54btlc%;9>_)R`Xu8tl}38 zjLH{i8@C9vr!B;r5;{6FY#W?wfVLV6UoOt{dwpJC9-)kuQl>@PpuZp3!n{iuXN{&( z6Ftwn;CiA!Qyqb`@MOdpznYCg`jfL%k--x2sZrd~wB#o$r&+2ub>i{L8luUkL>fHau@8r}X@q-)go z?7rld4*`QqL5p5At+ew|;q!r{QbWNkP}dhgjHuW*szegYl|PCFbuE1I`6d$5(%Fk( zIcxqV(H*ZIk6YIVy4`DRu(pw^DQ(Upy_j4P&0q|cubl4NJ4bfE2umGxjD~fX-5*4` z?9qEaoXCuZ#;v=KaAE<)YD{@NEz86YylRUj{ii`N^8;;B6X4b!squPL*Ajhp865br zWqQGuXLlbrnErZd&4e_1HrVs0@S@0JY(ML)s22msC<7X{XGG?`f!1!q1@KH_ zf1&I7nge-aSgRMIbJRO1u#rHn3Q^g!5o&>u{UIY7Zr=K!AJX2GI&*L8PL46B)VrojclvfywpF!R0zrNB+j6&GvM^wFU^-2iD7%f+jI z2!kUGDwiwVESfkTaTylj#>4wiQO~6UU$NHnQLOl#P%hOgQ}<&H-hkX3gu!Z&sSd8G zTlaD9hG~^%cQL8d9g5+R{2Jn^C7RPry6!#?Nd__`))){DF*{RlX*1t(xiY&NE05Jb z`^4wzmB_5^BbY^^tQ@;-R2nJDj>!oQ@B}-zpp-|DTEBspdHVy_IEFA%&jQLP3hB=S z?JsY_h`8pC(qa6Akj^B}qv37g&80$Obj>rSgpy~ku@ zx!@tDv{JI`Ada<$>;~Fp`hs@#$d)=%3{e(WCphyge)@8}JGe5ia7xHcIi}^!gX`rT z%I3MenueILBn0=7u9t!xcS|wrGf530y3E?ix2Vn@^CPE(08O$-=}&Jlt87`i&@p+j z;~{u4qhqoKuUoPyDm}|ieq7O8(?YZ+RHfB=LJq$jY5P{j84&fSFfIJp>}2QN+pbRC zz{l`?7g;7O*%%iG^JDk=Dm!-zWonwP+Q+&96c21DS21jEdg4jnXx3a(LAKoORZ#kv zCH1DR9U~Xeosd<{Lr%MSlMR-xygG|Jnn@&&%)p|Y`r6|CpiQcTR*D?)XQg@SSXbzA zob6Jb{>SshS-StsZ1qDJt~kXH#9Fk?<^g$D9I3r4vN0QOc@j=1zl_d3E4LH++moMD z6hfF~Qq8i#=PaP~Z;bS$4aQpprUH5c`v@e4A~0iUb$4R@ICa@{eEkB*RGcjbr2?ns zi*#X~(*$DutRW+l%|Jh4l;C^WwKbU!3ZXg;Zs@*bxdeRNK~~uJ{B<&*Tl=H;+{iKC zjB5mt{u_+u@9N8>=Eh%)=3gaNvB&7p)#%Z*4d6A>iY9_SNf%mL zS1K{@3v`%s`9Y+DNsV%})e{;u zMO#`6jo622u@w~|`TN_{pzoj^G)Bz>88_Way_-0blzk^j?_T@ zZPXj`*47Gh%h1b#*ex$ar0v%I8*#sT8MQG*p`zemxrRY~W-z)4*}fYX^gFT2D<(j( zZM3unF{M;<2t>QhZ9}#kTRD?_ahfE<217WM;EWfoU`!v)__Q5sIo36LF*dGpDA8b> z5pYu5@Gnjyx2m4hMKFm_{CxTXwdRw`M`UXW5@2ADdb9O_?|T+kz12=fX8v={dnsC3 z(`)S)0gp1rE=9*qXS%*^GxHtRQX9>DxL#3}P;I{I_3TY35$2lGy(U5$(2xt>JPc5$N_ZL6wv~`S)Tk`F zCU)>C3q%0GLS8|C$iGE7Adz&yjdp`w-CoZtckTMK7S;sQ z@)Bbcv9trcAovHzlZjTnK>4vQ%lh9uc0F(D7CmXocUsTBa$uUXkx>t)6<*YaE`zmK4Io*}7-hdq~Zj&WYoK)<$zYwXEWl{jJKHXj$TSiygD9Zi-tJy4GrG(=?S|H# zy4oMVe*F*y#TEpQxeJl=;cBvA-JAn9s4v>4t@n}2=5(NVE(pe>Q{xexetY4;tGLy` zV;1lsf&(u?$7=`}Qo17y>N%rG925A5^hl3RM|I+h9!-7GqyMrT^>+tHjk?lT$3CLp zSI54f#Y_~fj++UQo8j6#BcWW-+7^$D5|WbiqU3rD2kP<8x}UuNSX@fRbR`UDtiDM! zw_n%mpFam%U27}27Zs|qtSk&&*d7t=qNp{VIi8p7?F8PEmYb*<(P7hFAZpv$&ocsB z&5`>$D#RKLlKxew7&)A`W#i+H7UCFf9``VO8vH#*nLPTHusMeDa7nWQoR_oyY z^*7h6m=?3JMv#s5`Z2tyava-J9rZGG_;OzGvUC+-_z%CE6oGaGdc<6IcX&5gA()i|91KnwmzW^p3rGxBKd$Yuo({{B9cihVNYkrY>)C_Q`j&eqM5LzStQRW7 zvA~_#Lkkeg6AJIf$8mx~BZDblZ$DIh&;tuXk~^cqY6yx0aH3tJk1pBLJCu&HDMby4 zJHX@0h4NST1@Q}eP9aP=37Jn+70Xcly?O!yG#l_f>z?~wVrMQ03lq*$tFSI~^Iluq z8NW7vPB*YTot9UA)V=|*KSHk?62-TZ7JdgSD4;e+;N2_03l}?@JkIA9Hg(ssj69Qz zph8|A0~ZNF0(u0xY;l{WC3T`J_zMuBeE=TZ*a?xP3a>$=zkis`{vcEitUM3Gp1s80 zVwBT(eMj0qAlHzmAAvG4!xxp&b7kuQ3`z)hVm2rjhR@ zk2=noITYFpyWEfx`qpUE96AiyNewbm^dU$?pVdip#Vh2ci;WZG`8u?Fpy?R|NTrvf zv!+;?oZ`zlXsBDZXOA-;So*@>4R?W$lNU8F+8w6M)`)xX>&iZ?Qc6EqW7GUP#>xFK zO{;mQ_ib>)71*j}f5CaU+g18Kzs)e{^M85=YPURA;xz9H|4?&aXmk5mlLntkFIyOM zz!I4VRr(`QebUl9N?|NO5MN}GIUN0)@)b0$%`gQ~>Vox2XHKEC{%ML-i-kB8xwMD4 z2)-kc)nZ73TBrY96&wC#&Mkbe7Dw1Seq_(&v;?fHd3?Vs6mjw^hRDD53|+j#mq1W3 zii*55TqwgBH(^o4$;zJF!RCK{m5Kgy0qG)XKhXP&HCZ74*O}7ay&C8LK?f?*pJ+@l zL0WTCBFqLoI|ssyj9V%dO)bImjs|N=Q(=<&}O(H7H*;F z6k};LfA|_$j9Nh%-OgVp;GjLYUgR~)~CL}WIG6_A^*NG`uLGGQ|oSUtJMl}kpUtYd|$YGR@QRd z&eg+o)S8k)Xm6_q^92P-zx6?@uvRN4?WZQ{(8V(vwIc1!DvEU@PaQ5~ro#JSc2$$f z(y3_F$FF#v>M$IILvu#a$g&U>aVQ9JKLGEj)fMC|@1DR)x?@CSm_M1_jxYPdvNb^fHuWH9 zc1Cc3`aS2GM^T8c344NN008G=5F23d`dSYTSoQd8f@d|{7AWeBJ~zeKg}q-M6nWY5 zcKlo!sU%s3cdV8@xHWz;&|6ekL2~*OSgDcl4sgvq5r^o{L-V{h?<$N-#8(U@)l$1V(DKK$%2EWXn+l0B?= zq92T5DYpX+cyZvquSU4{=Z*}JgNE4pu)*gzl-nMLNzO$OX#S{MeW2+1a9mZ)N3!*m zyklCwn-`GF4G}^(Y8F9IW-iL;VPNHa5J!L#ev3DJd|T)^Sst^1A+N9uSLP{yBeAQ+ znAO<>J>)_B6zN{<6NPxID2PWi?aV~X{l~~Pe#=QB@I_+Uzuq?d52f#apkV*~LO_kW zj@e6i*o|;5Zb0F&CZt#+>BTx7@UJi6a!R77qLNBRg;e}p zMQ-oh%*6Y1dhDUY(p)I55ftBbRd7T!9!W{#$8u9kf8=*cWXs6Fgpw22g}1bQ{d5pc zl4+@)x0|bp1IqMc`f5}Mi=>Hg)IGFThXfBq{;@_o3i~QG9=42Q*!d&4^Q=!6YmLSb>db4)TW`9zH zv3NllHtjnkIXCFl`1l8~90FTO5Tsk1dUr!uI@lsuqkmbZ)=clzlU47-ofYr^m_2wG zi>wg4!%$ouu<-{ZRAaxy9TXuuwtZj+?I$>1r`D4otx5K-Z-wSR+uWwg4vL#r`YK5# zY@C(;&|X(<#Iiub3kV7fs;{KHYQW68+Lpx_&HBpZVJT>d*(D-H zq@@a#YvNQTr~qvOLwbUqV^YfFB;DKq)4bh_nM5O)J1~=OnL@_Hg?^GV|B1*k6qy4X z45d>O`>S5pZmSKa14`%eEhRm_oebh-Rp&9!I zNfUY`7Wt@@QmQ1U*7Lm~2&H4dCEWyJe}{qOJuVTfS~_Za`Tib8FUj``co0mu_p5md z)_d7TNp^nl!{&HqpZGD{lllHui|J~xBx7oUnC`{$sY1&((RuVwx3ce#&-;_Tk4OCG z(+)`vy@Hi|i(~P5*Y}pH-J9Rf4v2CpKb#DiACuRvX7eW@eEqlMlTEV9lP$k(3hgt? zG8a{3`{<9m0T1q1S88d9LeZri21IaM4BHb9xmT4`Y=XjjUtsPSfYX^GmO2Vxi{}Or z$*gE~*Z>9P=Ca!kEK;Q%;aKo-1|xvaelF5UJ&9U`|4fb?+V*vd`TU0|gqE;j&>9#3 zz>fQ0N8jJQjptfhcE#eTJ#~xjf_GUh5R3*i)6^f;)TYT5*Zh5$!%Vsc$Bb@C>NFSD zE%8kX^UmPI!~UH=$A+ZuV`HyJMxHZo9XXw^Kd9tl&``uCn|h?STjYXGiQ@Oq3+?@K zz1eoz+^11ZpCl>(Ki0DX-X%fZN^ zvmZ!W#lN{dgj-V9YxY|j=>>i6`66Rs$?g42DCa_R>}R=G zNK^nbju=z#je6tnML7-1Udmy=*e<@42?7Fkq!PDtvi8UR7ZG5N- zQT}n46`l`Ga`2_Q=yzaj*RW!}#N0g(R0zkbqSd~s42c}TL{vhj`CwliW8vlZw{c5( zOA%fEb_S*jI8YN-1-4)?y4{(WU6v1SbLHfe@-~3>5o|<`a_wV2JK$Rsq7H{DR-{m? z`2CwPa!?`c7kIRhrB6Yq$v^@%G#Iu6&^fZQmQo{aM{!UPlFlS1+CoPB{ z@A)+MdB$yLNSIB3G9+#N4lxL(0!kW94OwnINo1eJ1+7%ufkNw&U7KarE19#C#U3s}lgKR^x@`t)Bk~tD8C;tAH z`2>z4lc{7oddz-Zk7cLDiEO8q4$kRp*pdp8RULLi>!Vg8^aX$=7tT8U$9NEV4e6@6 zSXI>0Rz30v1%43D9Q7|Z$b{;mi$x0%9!2&X>F950-`fgXE&#F~p)I3ob1!V#JFbh| zyi3}@sw{%x=$zOU)&f_FHN-Z)&w1f}s!@s1my+FQDMp2H1gf>A(Yrn_puvMlKz(Z{ zbttdMs8+KXWr1qxATyC>+f}M2KxObrF>qzDV6HQ%Nx}4~sbbs{diM;1;-WWZzuxM* z^br}(u?rV;BrJ-Iqr$49u)36LQ7TVr1P>6QBX=bmMAkND1Jr91TWOY(TGxOB)K`4* zqN|-^mjHdIM*LBK4bXgl4~;y358B*+4QXISJV0prWJ%ULelgI>QvjdiBDI+vjApV& z+Fx2Gq&7oIAYdIbb*KAacMD>)Sxp)WAHz{Bdon~Av9azGQ8w}fGYE|$_BUl)S`ugL zE^Rz2==tfzEjyCJ?GEaPNW6j$N zxgV`Lg=u!c-HmUHAFO+pTai+2&d3%A;cs0GXR4bRq}|>_9dT-YkLP@ElN|phywUZ!OSF^e z-b2b{kCRv%H|mQlG5JHQtSR?L#r2>y293=H;J$y>$2{yNX|`tN+`|VfIaO%|o9iHO z3Gau9H6yk+y5mT(6eIj_Za~e7O<|B0>OJ+j^2d^Ry!wc73-ft z$==@TmblP*SM5F+H_d8xel1QPqoUKsEuUiROXTZO11|AD7%km#i*zU$l`p6*jQc@0 zTa}&CA`n=6bpsU_@Xwcp0^yb_aUYHdVwJ$LJH}c89 z-xjO+k1G7KF8V$#v3}IQsqi&+r4}RGMuG3&ADka#xJ4pcQ#o}+nlv+ zM+Rk#;w~Yvr4mKM7RO`Il0S=XGy;2TrSy~s^ZBhHK5)WfaFl5X2cDO>&^){T{+9Co z+bD6^MtQh82dxNbkG2^2t}fkDji&h&wK=K@+u+xqT4^Jlz!o)Z0UK80p(ku~*$Zge z_#-W0=&Zp3U+ic!^Iu$OGR$RYr$nZwr-3YC^FuH0dQ;Wj^ljU;DH#RT5ZRe&8WZeknbeVOdj5$8iOf)ndQ?IiW;oIX zcahE_>{vLDh)64OQ$=2>Qp6+%3~tY)UU!XlE8yCL=eQNRgF5&%X4po{42sgQFuFif zHDE>>s6r5K9L!0JY|=AwH#6}2~ ztuNncMM!=%0reu%_Mw2*r6h&}S3crba85L1dJK&-3&7kTd;*VM8-j&1}*KtMo1L5PhaRgV;H&wU9Fq40ZgFu2`C*Bk5e^SDEkow`y11jvz!caCT4eygt7(+MfQ( zYu#89hDCVf74sg$R4^!Uux9-ows(oSJkU@aH%z1-Z1Kh<*}#(XWjL( zFY8T&l%zYJ6etO}z8X|-+{6E6b!caYcm5Q(!yA)uEyLxS%ly4N0Y2}oZGYyqkWTD} z?HADVJ#_BF^Us&Vz`Eh#2_W8#wq2Mq2`IRT$l>6T~*D-QSp;Z2hMBUTTv&AaP#HP3x zmA|o`*oFI=wvn%J{(@$jiRt#9J9ITf?S6elX zth8kZ5Kiu6p#~<8d}LV|M(;n*Wy963Ugq2H4QUNMl(_gc&;Nj5`OqjqrquFXZF-&a zxM@Zg^zpUT&s$MbzXC$)YszYmS=qiwnV2q`(~fm>weiyntEmxpC23zeW_S4G!Glda zufElOpobrOqUiLZkaNn1C+K9xtG7-4PDR)na{SQ6+*<|*kV*}ePvY$xsQgJ-gVpS9 zx0=x{Uz<~yl&rv{IZsEo-{ug`q9r-U)2UqL zi~ACEabrcpZp~g4^)eNfld|AHnma9gT{P}+&)IIwN4Gvb2Sx{_mv-*>aBgo|w!GW4(9XxMc@LB+t} z*vC{4aWAI9{zyl)Pc5${yj(M53$o195!pNjxGd$>Z`BHS%PX>#%Qx-PdaB~X&XsZ9 zD-f7&-#6=0^K#XxPr=|;fm6hW$Gi~Zup~#1q#|a{dBWlC1(B$T%(gEvRto!1hTWZh zD?Brmb|LpT|7VRolu`xasS792em1`vjYKTwV;8vmZ5V29zh6C=q+@JXr zA|f3laUkyTwylryTIQ3PO}{+L^tJ`$d~c!ddc33V`n!n9t@~&4)zWl7Sf)?E^~x~# z{IU^Kzldk{2?=fC9(}2(_iQTLb8PXN!8*TAW&OMRJz1Xk=rd;1AGs;*$#eL7S}q%w zEuHZF-L=ZkKfhuK@<|xHxM8~df@M=j-}O-KCep?2gP7VT?<{&snIA?Dyujx=7A3qv zD6TaOS&qk_*^_Q?fS`HzPQj(frAD7NkCK)XQr5o?CiO@^lzG~ICx(^pn0_~IM6ga? zBBsUpar)GMG*AMRDm!gK1EuYMVW9l+wf;{I6G0@+{n)k3QwB4xI>xN-bPxI2(!u9` zX(yIB_c(Kf`}bsU-wO-739E>D5tSzML7byqRqlgPU~8gf{%7{tksk@6PYT6;gm!&Y z;)HGNz0ea|?Dz}eiOhb~`{f+^u;-;9y8$J4L*E(vW*^kW?rLr+#siugm_;-4x$1y3fC#P&DP> z=d|Oye$Mi_!%L2`qCccvB0WNppd99AGImzl~OyM;#D zs3yG^UTwaR7|lAqEBWPS2FIiE-z~phINB(#02fc68}_)3;>uArHu;jN4;=q0!>3;( znc`aR?i*V2{Pz>>&Q+2M_o^_;zpfRg-OICL8n=88DE?6NISuRdvWs!v3jKc8(o-m< zCc_;4(%J02EAY1XlN*mkpU#`4!KFCCu)W7sV%`Q~5?xZHW)(%2o)t^M`wt>xW-dNg zKESnD`8L=!IqB>7>6;=yDh|b*OZ=Yt*z_zHN0OB`TRL+QSG{*1uSYh#6N4b1(p@9p zjWgv6yDRiHTl9HiojTO~urU9r-%oIC+sWV7`@Q!2X_3AC>Z0?<%D8`k=d%rnA5ULC zpIe{+f3CmJqUcJ~rvb6!_tV?3PF@JVW@c1=jPO?N^bvVpmp8iHN0u>0=Q7VT77s71 zBD?u@7p^?t+ugk8_K>3=F~zUPW2m6MsKr}66hOM-d_EFZJW^fZlm$T$p7f7iSp}y1 z17c)Z29o5oJ zs`c%m`?-4#)PMNd^&(Bfw%Dd6%i4VM_4j*1QAK>;P7ff?i`tbalqzM1zqD@%-OUVn zLRno_Jo5QEq*gXzb<`GNzkBxV#u!6Er>-Hq>SlHIP3wil39FL5yZm98a(j)_V+Jo{ z4V4@oBLiCZ0^Am7tu!v3TUW>NASTQtGN+dW?6M-sOru`$_IzjoQ9tJZ($yk1ht zKfbj5ST9P*2RFOkG9AjIp*DDa_tP_f&0+YcVJg2u*Ci?b!k3myMLu)RN>>$KhmL%^ z%>UDW#{0Y^0IEY`5Bn_i!FaaY}kiqr_SD3tLL%Q ztixz~sv>4GpjokaUjLX@dE@EeL+e;^-Z~?>h)k;xjPgyMFv=8O^;QosWMUrtDk9H4 z+yB!eV=cF70s8EhgPk{_VLUf=5B-u0Gr`|J!)M}}h&knK)@%0IqRTn!3X{5-55q<= zx1?{_+6qVEf%T960)Y`3cPOx*d(+e&(Xo1aj{JC|qdDOQJsH@A^?Ib&9m;bXar%}q z>>Qb|>1umf4qwhq7QX)U;g^ZkM9t7kK*D4nixX|rW-I*u>iPU5R-sXwAu|ymK7^#0VF3kC_3$&YuLB;t}Y6B`EM#uv&UVH=p8tMPNB%sDbx`F$=l?tQ|M-s|Jmc0K062Ee zlMnO=cJ~Ki1i07U&;3?V2mtK<4yK<854i=xJYf73*dGKE2*M|KV6Q*n3p?L>fJU|!;!pb-PCEe>^;6G^qF~D+6JOey~;O>WaWC2R{Cfv{SkBmQW z|0VlhF#qpBf4@+W=RYnP@V+@1ToT|A1e*yBS%7@NqEx_gf+7chcoR zcsbP9)&PVR0brNN%^+jjKk;|z-to4v2VrqA-2)zCYV(iuI{`N=EI}B2ch@W5K(n24 zc5r@o4>km4xwEfj*QB?n=}sGi^l$3@`S$J@{X zgu!w+gM8dBT>)W95KasVwA!f)q?z-Tr=Q7AU10f~UAIE6{DTJ0`2atQow`7Nxdc6f z|KJfUlXE`A+Xn2XJMVI7h6LIC<6SPNn{Z>(f8Zl;`TfxsLE5;AfVGA{Lb58E&mzk&l(o~ffL9j z!F7`BH29~(b(u?#>kQXf;4l}I>l~Lp*99>3ESJt-6kPciIu8S0zye6=pXJ;JLcyAM z=&%FMKb8zt`0Jg+_k&SJohkoALIV^_xs%A+~2^@7+?UlfZrb!`GZu!|7eNe zzwlK4jfXZ6GX!u0??ezU5Tx~=l>S?b-VAsw`FGBaBs>!=s20F|<-b!;j-FgOIefC< zWYNi`f2-r~68}v~oavls&KAxF&PL8pz+ui-&NfaQXBTG;XVYJ0{^z#;vy{Ku{tte4 z(ssDrX@!5*_ph?{1pEZg3rYzd6TBb@5tJ3Y{1?_?K?%WAf)@ql!5pc-+Rg9p{D0#G z4mg6c`lnR?b6qv>W+j1fC#YEe;p_Ppp;97>lByi|LC=6cKY%^ zsn~6_`@(Jm;P7t6-Dh{7+I?vU{^!WsEf40M1@A_GQUB-2`3vLUIPm-zEKn+cm80@U zuiNRXJ8y@9bo+sHdxm%-z_|ke3-Rh6|jYGWe@3ate0MJ_nb=H@^)0{#9pfVVomev1G zQvh|MSUv!}*K`jJ3j4D>aN_L+fZ&^_fBWNqj=kqW-89K&uY>xECk+6YPuOhwD>j>1 z2$ug1JRjJPO$0)W5b&D-U~ayn^}%0X;o^`4K-cUK%Jey;{(_A;6hH;OlfDZ865s&W zKX7(*_0F}UUpfAI?s~cFpBNm#OU{47Kr4s=u*U(z9qZx$=-Mgd|EPeSm)Wg=@E)L* z1K{Kk26hQ^a0+vKmpE`xQdUl0 z;e?{PhNjk;v)YCijEqe#nqIQDv9+@Yo7dgL^QMxD$L!qv z!XoA8`o<=8i}q`Khp!zz|9SmKzCa;x?Be3$_7jc;B6c4vC~#$u^U1CeB#H|iL#U(*8bq^f5+JU|3{qt#n|6`5rDm%V5b!3 z6b7IGi=_a(+;x`^Gy(p9{SeC1hem&e?Pdcv*Tj)}WQkNi)+t(6H)=mphr-jz+DnGD zsf|s|jIIcia@Kf}%0pNgZ>&DWxcC*4j}}bfqvIQS5<*jxA%b-kDfS&#;bB=$sSh^B z&GpS1;y#YwMQky2^QW_z!YG=8JWC0};uB^AU&hYy{C}S;gC+}6duU2@6U1P5z5s(i zkdlCh>|;qGyjmwUC|>ulTC6^Z49Y7|Y@DV=vM8JB6>4);8mNOl!UD0+H&wXUec=S(B5J6l_D_YgS()Dw)Aq+B0LWV>qlnF4e1LJ6h9xB)E(gj1Sk+ zB)G7FB-J`PBsqI4E_=(c#a~rVr%hYFqi-(_)0qV)=JV3-E4>k*s}a4i*&3@EB>sivjeH*3AaORon)X8@Vr3(_qDjx1*q2O2z^sPRkdF7*Ze^P)0+uYuvo)oNQac$DpuLe{X5VHD$j8d(5jffC6A7Gf?YOKDmdve6SaSb0n zys`;P0dPM>mibp4RA%Vf&1-<(e^v7=yzCc`=n+S$}tviZuPv=<99<I`{OL{91Q;zzSXGI3c5Y zczOBGIICCYL~7u3Ho&dOtRMW8dBz&mkvi5A-m96fS+&=!i5ezVoOKD4Z~f?0R=VYV z-{IYpT2I{RfzpI57Jt+VgSoM(Mi=E|Y~sis+DnZ5)B0*-Y``{q{n{W26izN{>>Z4D zJu3`g`{G}e7Tk8F#HOBeYM4h#GHXybJ z_3k6owzJNHHD{vQ4C(?CrnuCn#wPpPkF`$;V-I{?l)&VR>FQPZ*lP~Jv7$6?*{KH% zqZ&l!`lT-ntx~x5g>7uHZlh~*$=KA?w{z(cj+p&7-$Tnx1oj^Pk@)DSHMQ=C=W-Ex zmip!4kpsb(g3{mFKP^fd9QsJ}QtXTha>#!6zFSCJIH9&o3g;_Ps#ti|A}Hr-?2Vjs zpHmi=uI;c`Co%NXt?Vp@SPzvA^vl*JTSgfgQy?n`$= #ev!3s=3C2jTKm1BO7Q_ z(do=K#*`WQ-|`EQwlu06S^XaGaQ#i1hj;fAR>SRMaAj2y`51gm!HLkuyI9K1CS7wM;&P%Tdur%f^OZI{i4i=q>Ag|5Y9TLt~@l!Op}6eAmEjbZGmP8v6CIB^)>7G%kE~EqVVVQYgWNA3sYvRsi1D(6AnDH9~!UlXG_wxDmk4;AR zKm=Kb5T$MUXK6KL&h}{e23jZy!hwjN+!G5ER#A;UN>f#FH>bsre#h183REG^```Jv zdSI1^yWV-%x<&3mkaX_a3)M^giNoE9n#=QTb z>_Lkc9@F4(&7Vve`cZk!8H;&$M(GS{I(0;pc0bZ`+>Gj<#JEb7mTGnCM-gz-{-3h+ zh5AFB^<)GDm6}?@B9|^?o1ZotepCz(-HmpOa;NHvS`ReqyOPPJTN+`h!CZk;0Ef|YD#{}vVYywxWqKH zLr4;_M6#$LS<+0dGfVV$FKyR!2qoc!W3#yjoV~Nnx2SQS3rUWfUV~MP6GVqPrjl*m z&(X`0(%;0l{WIPLb#`g@mmTQh)1umnJUp@%urV+?8y?v-xeUMlL)K^G#|iU~lZ2e$ z7aLWyIU?AB*@We0^`Z8$ zBxn^xt@w%MjR}+iCG|lwbOt4Ws2YV*MI znpa;t`~^ysc<;EWMe7<37BDf0%++yNmvyf)dA;~^vFqoI!?0}Ak899puOl^{85l&x zWmsm-%sE9$!F4V{KMqCs_eh24vH{g#YU-{DlV)%@6ntV%nnXnUE{PA-P+YH#2Bk-5 zkh7@iw16p*CoInBK2)tfj*`_mDMdlWhD6D?;>gx&V@=(zrzqk%WmAhdk!DNxBARhu z7h!k=gTUBnXC+pb)DBc@=txVMWlkz3yKtJ*HL#)Y!=<&aWvnap#}-^IyROuA>eho| z@$Usvvf_m|m&cndEM}h~61e%QLw?&_DdojabbonKe9&|_zv=U@e2pKGXOCBfin%X9 zXvf-hVf%1i7mN6gr?Lc=z^Or@=$^y71alMWq#a+!3D3QP2g=jVwtiOH9Da3?QP5OR zZ_4i%+x3@6z$mcfLV7TZp?;R*YIrreNAzGmSF-^5v=xnqsK!APnnvO{)KP+Xp=6c- z9I_8-TII6^-KWpVFrnyIR4+DeC8~;;B{444YZ5{;bG_V4oqnA*y~XjC4eXsZd4R@o znS$Eaj@hunuqI9^_(~ehC9?q!`iUypEVF@7yq{bXGtCX%(2VIvGdb;W@9N&?tyv7B zR|L1N#1p0*20a>p@nov`ND2B)3yh{{x$?Q5kef=)h^<9TQ7>TVws-gRg^FB@{wRF$ zM)D^Eo&7R~ZysGw{rQDQOJaG;UoAz~kW2Y}eGgXS+XB_ec1v-uoHgZk#3&YITll1oloS5v^1asiMQ; zOQwgwCMO-)fH%R(5iv(S-`LprW_*8s0Or2Xn6ar>?U=3*^;v(xxVkw`LEb%a6z+?Y>vJU`^wk z*21UmhdZ&-NkRgSza;Ef$LRu{ag&8!s8W|oir#ykCG`I^-Z_31Q`S49<(O~vPtdZc zvNR2nfIA+?%McHtK?C|8>5C7k*0q=!^>4Olk2(zEJWz4bvV%Y37x6PfotA>IvBi9G zq;27ZVL3yLrdCPlDVcynMfBwv`XdZ=j&%vfe*SjsL9%=Ylc!Ce|BE_`4*_k5kPTC5 z%BXlJ`k`;=BJT7@l)EMW{N!QUJ^IZSY7DW(**c~Z_U;|rnwD&Y9$r}q@C3^XV|~!S z`b5sMF6!y$$LJxu!M5Z5CBsi3zBMFSv7%aIawmFJ^#sQbS*0nXA;iazM zOv@tIvhswtY-;3{e!e!XM5S&|#j+Hm+gV(+2A+0bNPIRMNYpF1cpVw;w9lcg^kBpJ zB3B;a<0Dmi!*rd^v!~(bu}0Ry%VGyVI~WrcWY1`%793O>(;FNukM?>M-RHDBUn?H! zUnA^$zP}Qr>pbVZxP<0I4Zu&ZSfjp=2* z=0-n1n%Ivn6e~1KXgIX{{~WkhT?GH^!>6L1_4MUPufmz3plkif%TVvv&C=E0us8Xs z&3e=R(6&KZDHTmSQo+1$DYVe2mAHXGqqvZUEjb3G$I$f;d{vyyGvXd$O4L)RdR7CZ zk@IVT&D2|Lpze`pajGXxcopzG>kYAXy174?t9^ zR}v3=*6n}&&C|aqFU`Kc+4lvwHXrir>LLGfyw=NGg8jQfKcY>)>{f4Q^c$CNlrhXZ&Ibd2p zk>*rC`9WpeZyk0ZdKf~!90)-)(mVA}(t2i?VT3i3<$dfa!)f|d%#>7Wtoqu6XpTW$ z;g|FJXY1ajPQLS5P@P{GyJ$mL`7EpVw9`VZ&kB;Aa;Aqn z7Y6*0!v4N0naQEDBUv&y-`!Ozm7SOOM!5^I0q8tf@>kSPgJmT)(3g#)siM9ko>RgZ zB3PU^h+cVvOlmf#P@!xvuO9F1E^nYz=RF##RWP7~-C~J?bsK)-)Z;$eeYW&|ZWu2^Cw1P^(sn7WkU{ zOl1R-?5bGnU2mM0SZ5NdR@8oNy9>fX^@F9pVFTY|zS@J9xCV-RjSc+1k)J#zsV~9$ z0;^MFh!m1a+xiYHV0FlX4U{En!N3L^*thUx%Go$Ha1Vo%)-(jGc3$W$U&v~le7wG3HTeOK#5^(B+bYvdw?&52Vkb|KN0*<_)h55Htr$`Q z4bnR#Q&ZGJ=0n#6qkCXnPAnHG81q$gWbtM5%)$!vcP-XIP;YEmd$57H3d*I4nfGy6 z8H5xmvw`?VsDQ=6;H5>(RQ`?Fncm096xiVrDb??!q`8&lci39C2B}jMRuX~ zHAj(r-!cnGeTn*d6yt(&T1^-B#L)Aa%{5`4#`uD>mS=A~VDcF3C0kGp8v8Eu&Fa5>})8z^4(-$Hf0*kYxK-yt@q24gs zQIkZZ#A`$oF~5xs>|#jOQv|cv0DoKbZ#0})CL3s<5@JBbHe+Q@XPNjmIR^SIR~0wC zhew-<;%{|b66qwqK|Hs%jypS4m0BvxJDlrk_lEZ1WHi%M=Y4~(Z{3s4XVJ$ph0Wb= zpZTQgbLMl=5fe_U-(JAmvwlN75G$#XylsBs=R%}Q(+2b_8^Gi<_rqu>U@RFmnpQd+ z$khl&mC+#E64#;RQcX56J5$IE=t6zLgMN7I7&s139k{cFhY@iT6X3N^Rlu$%vX+yREbh(~C5Y7vWKXId2@>!)Szp$P9j3wc7y&LZt!;A8g z2eNH8oLmgFM%J!gCHXw_{&>L8PBZSTO$N+M@+^{6s^>9OK`b8{Uh>k+GCS6d^0rLL zuPa@COGb6!Wb}0yVc_V0f{-Ah@Jx+RsEF@6(h8iRqR|x@j@3tN-wtl1aN?)9H(|nv zRF~nvYLyKp67_^JIzphl8mJTq zWF!ABq)u%d7cfOgAZo-{yhMi4oL?a@Gi-n%%-PQ3rl?j<9vlTH3qD_MI#Rx0UkLoI zPXE1#wAIBWyc!87I6?5OhmGkg4w{WEN*YWi?6q$H7~D?0<#4CARMDbH>89B60=XAy zDM)jlJ3ouVcaIw9m1xj15wE)rRN85_oawc(nKR+o)6i*Z)NlnqP*!?6%-XbNHMIdX z{XBZli2i=o;G_t+)=<$W_>HzSjbZHDV)06oQ%PdDxMrggT7~fdBb>a}1@_^DxMtfD z$EEDc@5X#Iso7o6Lo!F-WIUpRXZR*vuos?N7J3Z5^DlIz}SHK7atW;5q-`%sGKA%q>=NrnS9Fn2?%!HH6 zQ8iHqXKxD4881>&Ok^q^)U~bhy)&62P}G>Qm87h)AuV;u+^b{qSf12UpwRrC+=&Wj zO#As4<>0wth>EeP=kwz~Om)WO#>33aS}dD#9R_dDT26*xGo6Yn{C&P7HX(wv@D3C?adDun$VXpd@OxcjE#;U| zsq6TX2t2{;J_6nA2F{9t5+foblC^ZjZy&YhQmO~jhD}$~30rr--yD;Xdq6kG+A9Tp zrdnpq&rG#cH4R!9zhPEK3om98Y7&(p9iuqGy4nIlX{R*Q$i=xmdlEzP2i-D7@@fn* z1#+D_J_f8kX|YOv>hXQ5 zVSllBcF?S42sDrv!RW53)~y5tm{5=l|<$JK2L z9h|2+b#})w-2933D)-jx)CBd3NQ<{40U-zcHDLK2Zzkc}nU4tZNw48|w35&|{#PrT z^~!p=qg>HBKgNvEz2XBI1b>)0+M-u8-QH(uNj)$i`knpljGScCF|-#pMKjxc@g%3$2YFfCxQ}+2H zMa@^^KWYPdRXi04D?UR7va-!iwL>iTXmQZ;xrh}xTI}V;&QZdbpI4aHGx>AM9hi@LvUJM(I;s!GB4a3QDTT>!~T4O z{!o5Bd!t9IY~TvHJ4F-^3BAV#g4@}^S3L;xsC>ISZt4__D(uV#EH1JhvjO6F7`M;_ z?KD^&5J>OxV(b@2ZFntz%deSnF;PI_f7joe-$b@#@c~m?DVF0CJDFrK^BjZ=Bwhy< zn9mKaH$bOf!RiuHbtS+M_GtT8mH>li80F;>e3|xkdLp=8!fCdaTuOaIYber6($y16 zRPU9tql*{2$tH~=q1E05wFk}IGkX|@(=gG6eTz;U*r9V!OARPttNo6|BQUlZ6^?g$ zluKj-GDz4cmWO822^-Hllt*3fb`>91)77d*`5?qJxGC|^iK7$`J8iiqBn9;eOUq2@ z^P?^legw>TxLH1&^jJ`Ej>Tt4*Klt9q=pQN?!A;A=lo{tLvv%#-QaGIG z(cgTca?xqzZ3#Z20OfNx1e_Z%Ys(Yi!BtLZtt}op?D4G@_0&k2Ucp7 z0h_MUA9Q)*{>91i5G@>^9bZ5!^BPuJnY5HQAD$fuy_wm*Ptctu<3g8{GLt@sXzENU zUCa|qC&^`nL-46mz0AUC2aO4@wk=nMy4nPzbgQ(Feceui#nTl{a_?LM1a1mP6JU&^ zD{O#w4?weL3Gy)u7>;z!=vHu15(QV=g$OnpW;m1ksPCB>jHm*dGfAc}kf^eoI@{AF z;)+e45da+^tAR?!34gr`OE**nY%y$vpz`T{#E;v&CIDTF#T!NJRk8-(7NTGD8O@4T zrj65Gyz^veOO$m|SXBhk#rX=QG-b0zhTm6D3E!A?zu5?k)blwbrD;)x_`vD4dss;_ z=7C0y(0sn2vU*O3)0=a{>8`HMjQw&iT_<`#oWNb$%?9q_c(LjRMAOiN z&8oQl`Z6@jZZCY=62C`d)4U}qO5Qr^@M1;DLPZ(#?l`jrS=kw z7c9c~G+hBjyhj5^&nGkhAurX?I*Xy5lgWPOCtuJ4Tfb(M4AT6>!zYB~o(ww&UNZq2V}1IfCl=@QYO zB!!X~C+J6_2e5MF&$uGwUK+e~Ns=eAPAx`8V`#%A*ks5bd0K2Tw|v1V1>Hi;f9+&0 zFr54C)HUaxL$9j={X8p!Sv9N+_C3L$FqTL_vxDDDzK?Pi~3?90G5j(N7K-HAkwLO5e(ZRIf#Lrxe+0? zAtC*hsCJFwg+eP<5S)Gl2G0{AT8LAl#PZ4g`fs+Kh&JAiv!!DDB?(3RX zj4fFCj9?@y>-kF^B1X3~M}E#E#g-G722&c!uU71t7z?%#dC;^HUZIH}N>MiAdxule z9xA^?z&cp`7UtrTUSEnq%*>QzP)?}-pn$C^YDWD;iEY#(x?CiU~ZF`ZCVgWdLR zn%vgg47P-8ZyJtKL5m-z&3a;>u|7n?&8m^jKIe`~jMfcAMy!WT=#*JF9{I>SpL!c z$u42eQ)9u?Mnl=d=vlNE$*d{LagLU`@){pXQh{2IR=YQyKE6F1mszF|x#S?9qFnl$ zT^*^a935U*n=L~A(CW3cM*yL$S~Qo1?v`p~8l?kM5RE4oEtnXAj9#NhIJz*=>+>Z*+CT&6KV zkGw*=OTPV6$+DOj5hV9$ZE2D`oF@vh=&WO;SJQCeFzrw=Kjx-F8*FAi^lkm<5z9W z&rc4VSjhzsAHO)`{JWdYqaW@@&#x)o<>)gZXA!SyN>*nG;h_or@H;Cri`tVpkQm)F znUY-v$F;%+U_SM}CCO3ef0{+o=VfMB&z5=z29Uf?1poG)M+dYX#+Ig0ahn2OHvRhI z2-Oj~YfC+DNIvHRdT3X#SMij0=56?!MYHhTNp4<+i+3`g!;9zNE^PZ(#TTM{xr=U1 zUMqSD0nMFjENL)aG1rzqh9P1)`uM#kwa7vbQ2}5DmL?QftgexlOyNr~hLY8V3Hmm_ zJ(+XHH_FTu`!iG8(7egiZ6Tw<2{62OC=TR~Nbq6Wh#rE=6=L z4K8Mg)8byE+Rp})mpZ&Tw#&{zp5oAft+<$}x^Tw+K&l1}PMINg$6{3&dbP|X+;l`#N5k%*5<_#GBis|L znjFvRBkE|5Am!$$gNrQA5$KJzm^{hRjXfG4&~?K&FQ(*A0TsrH61l<|smMa&GhKsQ za%A_m8VA~2QhnyAK~@_}WoT{n26;9`uHo?RksNY3Ymf8mMXhaTRxcZ<)yxX_nt^a4 zj!tN#-ZL-#P`FDc^1g^N!9b8&T3!Wy{5BVSsj%aHrQN$k)(>uA63*$UPt zNNv{>`9uw+HKB#=`nPmlA8pM)9KQZh*7D`mlPGyLoebzbAS~z1MhzK>a)`=Xkgw{_K_bE8d(7Q+vQ!RS{HJIivdf z$(^0u%cgWKbG`0_I>(Ys%eYBNxNNqICc*T0m}Z1m_|2Q{A8wc~os%)8pG1n#Oz2vu zwyQWFSR9T+Uj|`Ml)y(>Qw5m`jL6pra2v!m85wfbR0>)xb&7l?UuaQZDeJ-;VfY*j z_l0qm$duc%-N)Zk1-&p(C7*fGPsQq_(CFe{J4`u=`42%$2RcTd(drZuVcZzrtRbki zM(a|FQxp7x&Me-Xis0(-PWI-1?_z{Oxy28cOEdRit$d!e5zI z^%Xn8x1)v`)a=lQSAd8qhBj`Rhbayj2Lqzw;I=?+9yT!l>?J2l8b;=1B@3;9vq`cl z*mf~&VB_*17Fi7qw9AYVs9?mkGYj-$*ub`&0sw|t?HS=U$OzF=zh?y zt6%}Bem5o-NAU!iN3oI~0L!ccsJ1gGubMd2{^o;}WJ?OqitJ2{6jD$7dmewzOyEj5 z-5QDf!q5mMnIDbfaU80H@iBZs$8L>(A-i`qIXJXfv%#~lt+KOZ66VkrC04ch-TH3S z8n}xRiJ_m5?ts1BT5*3(3wH!J2i}C(DUR(O$8EqEJ>BjF!R%z@GB! z2`Hog_cH`XiK7gW9u{vWIiZVnh?YUFr9zmQENSFc+n)b(EeQ;+QJ0J>MFz8Jah5%LrX!i^ZjZ>uHi zf(KoTT9Zc~=&F$PH2WOM;2LMMUX`PH$&#cDrpRaCHwT|=i?Y&h-yD6!B_5mmhWo0& zu@voEF5+DGZH-o>XfZ^`yRc$3UP@$#u`j0+Z>QExzPJ7Vl0kb)jVh7dTH8$916tr(q9abN*CX_{?>a z_g?}wavGokmPs|i#S;i;)A}#SQ^BPcpru==@|1Px%kuJu3!k&sBlBaX==#ArDyH_n zf-NfWI=GTCDbEIgVm8o|z>IkaF8(|VWC((*XD3va8cSC&q+}g)oemt?Jdh?Am{>;_ zfLPY3L(akw#t*)29`)N=)2QA9%O~-3j)i6Zf`Ma9dsc>FQz4UxdIV5w? z3iP!#sw)faQ42KE`N(Z&inGG$YYxaWOs_m3|kJ>>5v_sN-1&q+I@qY8k zRws1DmAQ5o`?3y=x(_f6ihr?zC6+lGIQJ`y#fwL+yG^kQ<{%rQnc&p149ewHUNKpM zG{x9YpEha5B|!G-n$gche*}N{Z7Ly}cQ7PS7->tp2lin_+QjV{HOP{7R6&*_MYBp8 z!w?zstAyBKeV;F@n~A71c`P*Vs3dC?$EtRN$~GPKMc)&o{MHoXtI*x`(D2O{7#!s- zjM3#A-2pwsvvQVx>jT|1buWY~l_1L6(*t(zx3p>91EDoEQ=wRs&eR7e@)aQ&a1G@o z@`8F54;?WN83zZTi4zOok3;K!psp=f>GaVLL2qMQ`jR4RsjvQ*-*ezhr_9}DNKj1L zToq}zDN^kOi4+F+ObU+;8TU~42+3b_+yT@pNxC{!PG;`q^V?m|BM#M%)=P|}n{koN zrIeJ?PPU!|Sk}#6(?a7pL@22LuP{`Yuh!~%-KiEd6v)aAko6i`Aa&0uI$v@Od4V=w zfz-y(66g|^GqN<|j4Y;ZL}LPVVX#{N0K&H&Lttp#`GytluNx$#iqK}dr^wl~nGX0r zH&HHPpOAG(HOZCHLJSkKTJi={1WE%J*8l_r+#xytT z7(nVC>0b_+-++ia(d?vkY$>0p6ohuf<~_{{=?%;2r55uL7e{m=(iP(hI!tH1UinzB zD8T8Z6fG6!=7^lDw7LNTVLND|Of$c>re-a@`9@}DT~%e>qef;{VasaO%_K5}ip%eq zR2ZJ3CdU{LG{eEg4RJ*I`sIPOK(snJH4a@P(Ye_8h9dBQ zW|M0-XfqmEr((R-vE8iBEao@aGJoHtxMl%A;quuoi_m1Q@2_0lU@cKw^JxnG?rE`H zdSIrCqp=q3=7%Y>rutk0=-CI*=E$dsLMsvUV1xu1q*@Um#G?sPb+@~vkNmTCmM|F+~lffr0Vidt^9CkR^fhT)#0w- z=4FL2ZL_N^i@a!{r7(e3z4YQ;@mV9C6K19w9~t+zs92hTeodqxXmWDvF;kHz$lH-o zntc{znR}G>_K^O%A@V0GWb8H~wIfTC)ubCOL(wk|uP*+aAN;a3rLI@WnA|&&5Tov2 zPYBF>w`{LC`{DG#;pP|RJ^V@^)=NJT#rDw250b6ss8&zQLO5n}44`_b<*%s!Z85Js)-Z)+4hpDr`oYMA6h%U-&5HFUl# zxbe}I^vsm^gIkBHYocy^i^wlWwMhqq0Z?!U(ahtID3VM*?P$F1@RK1XjymzX$Mu5_ zG6x-YTR%Q`wq`=Sryr8oGF8Fon@@Xo|9`7%DAp`KFq9}e38SAcWr%}KVhy3m4a?VV zxI5R_{>oQo93bO5P>)>CG7FA@hNT((Dvh&kic1;%s*)uajk7hMn*lE7!Or#TYQ1K4 z!fMw#Rl3n6^^a0CTPhLV-THDvKN_&jBcTzU!+9wspv~{}f3^43VNrEm-ykI| z9U`D~gLEU^HFP5qL$^Zw6Cu zb4~5vti8@UYwxx8KI`Y6LbOD(3^r!X56TG62=bjt(kqSg^#1B{JngA46T3*?L}c; z#gRQf7z4#-gf(|q1n#i%P%UU@QF)v9DXaad3c3NzXdEKiK8txOywF)7q-Go8HlJ-V zXcp7TleVcRFw+!@9S0KV16MJaY)8h)b*VO=$kL#I8^Fpy^ocE3@8Gn`#5tH1v&4;W zPG-;I!bhDIdH}M|EIWsLs1-|h(w4_Y;bU_%EgIG`UxG*qX~!-Y8) z^+|VDMDv0ZDAw}HC+C7JW%*3T2!&!dA?1xfBEo6GO)h$ zo}s{IGBeH1FWYdcZ|J^YuZyh~Hi-;sP+Fk1j|hysGRj^9c$wZxF&9VrO4YA1BC6JVaZ8Fr#_DmhROB*n?E!+aKQp>#Ch zdDHA*&(A&+p@8f*HryCH8(Z##k#i?A7=2V4Ayq$8e4zt--XrCoK)t;kx=C`fP7DkQ&`)md{)J2Jra$w&_aHC&P zGQ;-L=RiMmiU4C|A-Q>A!BA9xkbA6jBp=uX4OJasDPS(Ifw6M+%73uO?N)vAwTl9< zsDJ{n`s1l$J`dF_t?;+o+}1h9czPv?7FoAQvtyJ6vvL)`(J}Rd&5H_=$haZU9OaUi z&>VHA3i1d#5^2>JIYahq%Bvd*{HT85{s~rx8(?yVm=-`pmx2{u5$x*0!r)WFmig!i z0b_XZvGU7#mD0%q=Er)5$-!A{&&M_~H&$JIvS~&S-een0yZGf!ykvf}e|%c1iU0CI z@GwqS9#m8J?K55emomBZXuy{`m{0BG?30phAK0^<7=kxq%~TZhr_~~r^lEDC>M*^4 zEtk9t4gqKI-{OS`7E@rfevMl})pTu<6Tm($9P1%K`MM$~u(giHdUTlSiNvWOtYzV* z9o0P_azh)C#^5R4J20^9v|pcVo;9iBl)36&vZ^Osjc(|eC>r`Ep1HlA3NQ3sn(-^< ziVS|uBt>C))GCIrSspp^$}C;IH6Nyz!ulP@!|RqttmcbG60@Bvb@@>6D`?qKwks0s`Ufh`Ls4kGExl+!VaK{_(_@UacgtdD9Xf+U>ghK zr#cxRJaMNt(ys%%*G86-@S%63vtwh1$o}`d3ZpHCy;hbKBk4H9)#SjNSwJ zmX<`w3u8HxD%GCJ+#Wz`qJuPIrqIW4ZVN_-x!G5jWvwSt0QK9}1J=S`qWYw-9X|}= z@=;KFL83cPsI8~+)NPYt?irWVP&x9@B5GEXjU8fMAg4i&1#wU}4gE(tPkGk6>a%Y2 z(sElpg>@gZl9%#7)M6rkN;LPL7R+I2%jLG=>r*SQC$h`)w8%NJ_NInKP8oJS90AwLvT zW;m;4ZywoiW#%~I7?t&;WjI4GN%>(s=-mi|I9N(1PF|@Zx;E}b!j^}ODS-%`)$&qgQ+M8PScv$*v{-pnw5 zky0Qmqd+*tvzhOQJEh!_X=@;XXv+?iIrz__)e~B6F(*^Ppo?L3Aoa1lav$k6{<3aa z`}};@Pfx|LYkq7G`rljMf2`5|e#_6lzc={*{+rn66RHXbi1={GKrIb8xZ7~1>a;1m zK-Exqpa2p+@Q$OSD^OC)+|K;_BgX}=^M-D}JsCG~1Jw@J*wn`jdmb>Z8_1luhNTRg z_CS9m?O}V(Vml}K#0+(JvClQchgh-(s_>zAurO;C;neu{2w=ATJ(Ie@nQDkA(GoR_ zM?UB0u+q;5NsHlcB<@fzdC*zU*h+?c;!v>?S(S_o4!miD6OHy6)k26a$x8ZxtL1JB zw*4YzTy!{PYX1`Q7s(v;w)Uaua8r`**R_=VwKrX&Opzk*m&9K8M)c;a7_sKkdU;oI z_xchaLPcb|87Usu2yCjF-R?Y(*-pF7 zuZ448hY>OuO1jLZH(6Wvm{l$N@gt zL^{iUp~f_njUl>tA5QgKW6@iF_0oWFCyb+78hQ)8 z8#9#Sx|IsW20BA`qr#C`H0X}sH>#4j*gL2g)e9fN*9jk;?H%O~e-=%uhd3Rd3U+r> z)duf|D!@KR7M>j+Q)9%kZuhSD`+hsxWxpYBRnX-CF#UF{v-$ZQ5vbN4Er-Fiy=bI* z+3XZl$V3tvHur=fleM5jN}R(}_Sv=O6UO~UY|Hq0xB>-l8DX3OcJaoVVawzfCpqzq zq;n9_UZFXj6R*@p0lz?(sB~aUJN^BQ|5fy z5OL?X&wV?vucKO>#;W({3vv+}d@UANVOYCQy4U&qS?$Oxyy-NO$L56UH{Na@S$sxa zb0#eDd`)&N+xhL?a;@Y|+~ukr)?~)SARR2Hc7tHiMn+7ADp9=Ruyq9*F#%JueY`v7 zAfK?tLKc({VF!~QUO|1iyefic38sALz0DTa$U*G~qv25~Ll{lCJRuL@!V>xt5#)S$ z7}a6A74F!)IO7A9O`~)g87B;^CXeh%QSo$3l4UgMV<+$0`O2)}_uREbhDu`v4@m0XYOPsM0c|#q&;cSaO!sN2a2O;xs z!9_Y=c(mAvyJ!#(0_6ogE;3z9fCS6dh?x z6rQgI5~9+YE@mm})Lg6lc%F|XOTwKb{aW(AceGPRoxJOp0E z&DbKN2h<7tdIuY8dmyb0G@vr&Y z-J6bZ+H!bW0L#7GrnGqeiosLx5#E{eTdEr^KDmvuP#d}h@eHzR)u@3wgw_V?9CXf)q%X~X(1XT=AGYou#Y)CN*U-HSk@rTemE}m zS}`Hc&5l3Ox={1=K7q=;@+r=~u>?cvnXT0g&ycTe9_5g@K33t4qsfA#b?cCg3EZFz zCpRn(e~#JYseTFXJ&bF0n9wbdg(&fdYsreY@N3v!WIf_6;$sNl2D4~-)du?Dc8IGb< z{w;3qhYc(vJm7{K7clpv>|>U6TCSJw=Xvi$tVzw?LiH8)3mKi6ABtOfJG3kYH3=Dv zD;?B;%(=TG_BadB^fvI4w+d2Id=?bmE8fy(3^vsAL`$sHxxS5~VZz6_`TDWH(g1BD z!<*g)pw84ES@aAWzR6VppM(QdssQ@o%YoJ%Z2s`Fisi*98?Js&xrO~kEs%BUekiMc zk6sOgDDHUgT24?+ab=F$eSM?KbRXe$Y8F+`xn;wxp1yG5D)8%s{HC;u%uf>@T_nWd zI3CRp#-Fh*&HC7{r%Bpj#(E=kI%HT0f}$Bta9Qd0O3mkQ|{Op5I@QAmxB73!f@0V(?%x*gRpu?44`TVKYf{KgYe zs1vZhDGbH-yrQr1U1wpo#p}?OzN{Jld=yI}B{?uyhW&aihCBo2w zax9qzlv?j;g5;r3c$@1TG+Pqk!~GD>`yYmsk=%IwYSQh*s+v6w!7Y=~DdrQJeG|0F zLt!f~mY?LSIh5ZyQXxLDl-5CeJrh*&sYiE6pzG`HYTNPMEvcc_e9!} z8~MxTWgg6SAbcF$!kI^`%Y2A-_zVmSdHj?dy!<#_w4XY|T}m|YIyJ{$he)N{ zVRxo6a0>r;lE=SAMCJjXQ~>{Y|2fI7ruS{lL9QOQ<}Mef8KXxID}bF?Y`!k^q1BlW zi-a0AelsQ0!!>xu*xRdOR%y74)8q-*IpmUw0iA`oaW*p9%3i+=Dl#d?#kxAh1!%|6-K%~ z%xgG~+W+&d(1jS}kVcrU?R^wM_csF)Serlb+lM9rlR-m@#~)L}Q~L@nB0VMg z&-{tiLYx8+mv8~}NFz*yEuFOLv~4(Tyq5ORR`T~1v-Zd2m9zZz&*dqi4X47~cJawD z*&1bD(|!cV?BnOG(9Ljzi~1%>X)#09!%+*G_wXQ?p}WZ+pCz2>4hY+LeIZ?2AsbCO zpF^D?F&~{HOu}7BzyrHkNHd8@(LlbeOLkl{dDO+7!0tfuIz)pHl_}mL7p+|_zUiGk zsSb41`W5wUBiz%LPGnEA>DFK$PmYr4nZ1u?7u%mI-`6GjDFUE)TcDoX4Y=#@ zzt_J7YR_Jn73Et*c>2`K5_drR|9_8kjV6G>52FryT=cq#U9hARDTY*PHnBQBKeNK4 z_tUyDD?0HksL?NYPL2n)lK2#gTb75ZjufLJW5=fzKrslg)O_6sN04Y7_8~^QgRqTa z2=TN9wWMimcD}!q`{U=+JBM52WBMo}*4Gh1@kO2Qh^G^WD4DDziOt4M(#z^`aFiOd zDwxY3dolU(nA}NHrcI%*m1IX7=}f}4U|2l8an7AnQj*UmL?9E42^w9l7!FAtVU(@Q~)khgnt}OQt&)~K*Z5KWEY{+N*MK+$i9)8 zYF6rz)`XP%4I^|swfPq>!m*&NlKtogNIen74<0(o-n=htqc6!C(0~gnD9~45(UE|i zQ%$;nUPBHt;8sJFLa9O>Ryc004aU6~k=3-Xj-%5SPo(B|z#}PW)0COMUQ{9aPIs~$ z)upu&)n{;72I3l2M}%{{V95V-J^Z&haet3%TXk(RTcKo(k zrBf`0SsDFpY%ViuYe@8C$?zwYnRQ}|W2~ETO_h~5+3)#m7Ylwq^Zc?}@}Yn^ngwQ( zcnfdFj2b>Sk;h$_wlEAVKrJT*E@w^*;+T-2j5$4K1jVWLwM^BZ36Wlxqn@DD;HJIV zDz8d9f-FzsIcS>p-nl-OIMP>%DM#cJsZ~m`px!lk7PTSdc&%GeWKnuTBW;(}Q)Dzm# zNgN3&EX~D4H~bR(ZYGdm>krr3_vuRa_wP(rM72z(#=KAC<8oG?L5@@jWB70qj;+ok zKjGN)-h`@Y>!s5BFd*`53I0mFA=mDe1s_rdm`bbFlv5|_-Xrki`q&L~ljpOh^= ze{&NjL=Kl1GC*S!yL;7dto{EEn zDVZ^EhHP(hIg*7@awSCK+X=ktB}nxjXww9Nea(7~LI#G1K{s~|?UD*$(%4{Dw~%fg z5an0^f?Xhq(tFy}`WCD_(#*c&i0Wfm9v?~^To;333p-4<5AoGZ7%4APa%|4hd0NIW zcQjfLTZ*?K#~v1Ik&nyWUYbZ7I%O7kVsx@)tdp-87HLb(J4AT0%Ed&DHoEOtDB$n< z`P1;x$9JXf)aoQ&XSdt;l41NR$~SHBOy~&-i9UqH3Gw7!=mWVaUm2-|Vapq+Sen%1 zT5XfNFW$Q=U2`{?(_AQ#{WLOcUL^5U)nHNFF1Uu2`6+{iKt*F*uGy1#F*V9ZXXr#I z#&AoD>|z<)^itWK*MlUfnjCvvx*}ys0;I<1Wo04jH6G6`Z((?xkCkn$+C74;7Nr|n zPs%QO1&>f_L!??o7+lVj$R~K;d=7BGn+Da4x>?l~k*e(9|48u}zPEyov?8h*B4ebT z=Iy=?nI>1?$E z@$O`4)v8G}iHJ}sNL{Oo63ptB6`qyWHbYCrV=Uv^B;}Lq)?J$?yj!OIJkLhZulKiZ z?Zt4bOu00yQw}~eYVmkT?LpwM&zI|=@j$ulh&t`QC(dUL_IG#uXonog8e5kird`8( zjWY^mBE~|*m8k6_lg~GKLgh_mo$-m`y%pzcAHA9^D@~4INWw8-(fDA#XXylR!+8ZoBoAU=KS>Eld(>g8ZfV>UycF#IFlFAavOy>;#o!;_sXhTk*9A2%X+M0Fm8t_nIDF zjIl4|b6F;qJu+0c)n*pMRd>qm%pdI8D^_BjXb&5@n{_-};;W-qYqG>t(^uNIMe+Nv z27iqO$kG0mx&1b%|7Z{x*lIAo#~MF8`NcDR2?xM`*t;5QcsQE77+wr?stUgXq-_My zVgdjQz{m*rrJoKoV3KLyQI_nWOqIsnNT_|c2L{y1cS6!it#cbJP%1L%;cE7-yQqC0;IwpHRQ zuLJa_98k(dI{U{_3UEnWfL(>T+PSlp>|jdZhZ{wJ3f4cG1J8spf9c$hZe4{K9gN7nrSETDQc?eD_qcki`hYUYa2Gd%H6$bTvGB@fb{%KX>Js3iXe`H}+iACXyp(lQX90_Del*T!GU z%<=Ek@hddx&3^^W{+A~D*ZQCWh(7*V>P!B%f7HjX*1>e^-yq-pFJ$Hf){XvIX3qZt zSrBjr|BTG_Um#0R{u|^=O6vbZJuBS)H^_f&c3=7UIYo2zdDiZCtvb z@8IsAW58D|12iCjXg?i!K+*l*@qY6cmv}#UimPzXm1%}j0P&E3^v|zw zOTZ8O4)@yy_p^8xr|BvjW{$&7IWX^i0gOJs!nJVzJ=~?GUWMC1wFut;q|^O_bd^A8 z_;=D>n&(wGBNly)Za}(TK)PSg#e&e^!*N_*{8#ZdqZjO~{Q)Nou;Kh2 z+@-s}3U`&?@(0{=fb8e*;4Z!8Rk*7Je?Q=603M;ggS#B&zQg@BH`Na~XT!f_s=6xE z)rI&UAYIwtWxBd7e-+{C3d0Ws&Z6%KzpOW0ZGJU({i8W?$$td+HIsc6;A(#O2f+S+ z0Dj39Uv2(thVw^bU}jkMlNSGz@4VXj*EG-1t#d1XYW-W%=W6q-A@~mf7| ZX+W;3fP{R}k{0+&1KdWy#Vio+e*mTV$`1el From 53f6bb1dfec74cbe8be9ae6a670d82eb6759cc8c Mon Sep 17 00:00:00 2001 From: raschild Date: Thu, 9 Apr 2015 07:04:18 -0400 Subject: [PATCH 015/128] SPARK-4924 addendum. Minor assembly directory fix in load-spark-env-sh Set the current dir path $FWDIR and same at $ASSEMBLY_DIR1, $ASSEMBLY_DIR2 otherwise $SPARK_HOME cannot be visible from spark-env.sh -- no SPARK_HOME variable is assigned there. I am using the Spark-1.3.0 source code package and I come across with this when trying to start the master: sbin/start-master.sh Author: raschild Closes #5261 from raschild/patch-1 and squashes the following commits: b9babcd [raschild] Update load-spark-env.sh --- bin/load-spark-env.sh | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 2d7070c25d328..95779e9ddbb18 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -20,6 +20,7 @@ # This script loads spark-env.sh if it exists, and ensures it is only loaded once. # spark-env.sh is loaded from SPARK_CONF_DIR if set, or within the current directory's # conf/ subdirectory. +FWDIR="$(cd "`dirname "$0"`"/..; pwd)" if [ -z "$SPARK_ENV_LOADED" ]; then export SPARK_ENV_LOADED=1 @@ -41,8 +42,8 @@ fi if [ -z "$SPARK_SCALA_VERSION" ]; then - ASSEMBLY_DIR2="$SPARK_HOME/assembly/target/scala-2.11" - ASSEMBLY_DIR1="$SPARK_HOME/assembly/target/scala-2.10" + ASSEMBLY_DIR2="$FWDIR/assembly/target/scala-2.11" + ASSEMBLY_DIR1="$FWDIR/assembly/target/scala-2.10" if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then echo -e "Presence of build for both scala versions(SCALA 2.10 and SCALA 2.11) detected." 1>&2 From 470d7453a56c56a41b2851551fe1830065f88b2c Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 9 Apr 2015 07:07:50 -0400 Subject: [PATCH 016/128] [minor] [examples] Avoid packaging duplicate classes. Add exclusions and explicit dependencies so that the examples assembly does not duplicate classes already packaged in the main assembly. Also avoid relocating the commons-math3 package since it's already a dependency of spark-core, and thus is already available in the main assembly. Author: Marcelo Vanzin Closes #5379 from vanzin/examples-deps and squashes the following commits: 12c258e [Marcelo Vanzin] [minor] [examples] Avoid re-packaging unneeded classes. --- examples/pom.xml | 52 ++++++++++++++++++++++++++++++++++++++++++------ 1 file changed, 46 insertions(+), 6 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 7e93f0eec0b91..afd7c6d52f0dd 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -90,6 +90,12 @@ org.apache.spark spark-streaming-zeromq_${scala.binary.version} ${project.version} + + + org.spark-project.protobuf + protobuf-java + + org.apache.hbase @@ -234,6 +240,7 @@ org.apache.commons commons-math3 + provided com.twitter @@ -262,6 +269,22 @@ com.ning compress-lzf + + commons-cli + commons-cli + + + commons-codec + commons-codec + + + commons-lang + commons-lang + + + commons-logging + commons-logging + io.netty netty @@ -270,10 +293,22 @@ jline jline + + net.jpountz.lz4 + lz4 + org.apache.cassandra.deps avro + + org.apache.commons + commons-math3 + + + org.apache.thrift + libthrift + @@ -281,6 +316,17 @@ scopt_${scala.binary.version} 3.2.0 + + + + org.scala-lang + scala-library + provided + + @@ -322,12 +368,6 @@ - - - org.apache.commons.math3 - org.spark-project.commons.math3 - - From 7d92db342e01fa694d3522fb8d2254d6297a4203 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Thu, 9 Apr 2015 17:44:08 -0400 Subject: [PATCH 017/128] [SPARK-6758]block the right jetty package in log https://issues.apache.org/jira/browse/SPARK-6758 I am not sure if it is ok to block them in test resources too (as we shade jetty in assembly?). Author: WangTaoTheTonic Closes #5406 from WangTaoTheTonic/SPARK-6758 and squashes the following commits: e09605b [WangTaoTheTonic] block the right jetty package --- bagel/src/test/resources/log4j.properties | 2 +- conf/log4j.properties.template | 4 ++-- .../main/resources/org/apache/spark/log4j-defaults.properties | 4 ++-- core/src/test/resources/log4j.properties | 4 ++-- external/flume-sink/src/test/resources/log4j.properties | 2 +- external/flume/src/test/resources/log4j.properties | 2 +- external/kafka/src/test/resources/log4j.properties | 2 +- external/mqtt/src/test/resources/log4j.properties | 2 +- external/twitter/src/test/resources/log4j.properties | 2 +- external/zeromq/src/test/resources/log4j.properties | 2 +- extras/java8-tests/src/test/resources/log4j.properties | 4 ++-- extras/kinesis-asl/src/main/resources/log4j.properties | 4 ++-- extras/kinesis-asl/src/test/resources/log4j.properties | 2 +- graphx/src/test/resources/log4j.properties | 4 ++-- launcher/src/test/resources/log4j.properties | 4 ++-- mllib/src/test/resources/log4j.properties | 2 +- repl/src/test/resources/log4j.properties | 2 +- sql/catalyst/src/test/resources/log4j.properties | 4 ++-- streaming/src/test/resources/log4j.properties | 2 +- yarn/src/test/resources/log4j.properties | 2 +- 20 files changed, 28 insertions(+), 28 deletions(-) diff --git a/bagel/src/test/resources/log4j.properties b/bagel/src/test/resources/log4j.properties index 853ef0ed2986f..edbecdae92096 100644 --- a/bagel/src/test/resources/log4j.properties +++ b/bagel/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/conf/log4j.properties.template b/conf/log4j.properties.template index 89eec7d4b7f61..3a2a88219818f 100644 --- a/conf/log4j.properties.template +++ b/conf/log4j.properties.template @@ -6,7 +6,7 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n # Settings to quiet third party logs that are too verbose -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties index 89eec7d4b7f61..3a2a88219818f 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -6,7 +6,7 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n # Settings to quiet third party logs that are too verbose -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/external/flume-sink/src/test/resources/log4j.properties b/external/flume-sink/src/test/resources/log4j.properties index 2a58e99817224..42df8792f147f 100644 --- a/external/flume-sink/src/test/resources/log4j.properties +++ b/external/flume-sink/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/flume/src/test/resources/log4j.properties b/external/flume/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/flume/src/test/resources/log4j.properties +++ b/external/flume/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/kafka/src/test/resources/log4j.properties b/external/kafka/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/kafka/src/test/resources/log4j.properties +++ b/external/kafka/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/mqtt/src/test/resources/log4j.properties b/external/mqtt/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/mqtt/src/test/resources/log4j.properties +++ b/external/mqtt/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/twitter/src/test/resources/log4j.properties b/external/twitter/src/test/resources/log4j.properties index 64bfc5745088f..9a3569789d2e0 100644 --- a/external/twitter/src/test/resources/log4j.properties +++ b/external/twitter/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/zeromq/src/test/resources/log4j.properties b/external/zeromq/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/external/zeromq/src/test/resources/log4j.properties +++ b/external/zeromq/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/extras/java8-tests/src/test/resources/log4j.properties b/extras/java8-tests/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/extras/java8-tests/src/test/resources/log4j.properties +++ b/extras/java8-tests/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/extras/kinesis-asl/src/main/resources/log4j.properties b/extras/kinesis-asl/src/main/resources/log4j.properties index 97348fb5b6123..6cdc9286c5d76 100644 --- a/extras/kinesis-asl/src/main/resources/log4j.properties +++ b/extras/kinesis-asl/src/main/resources/log4j.properties @@ -31,7 +31,7 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n # Settings to quiet third party logs that are too verbose -log4j.logger.org.eclipse.jetty=WARN -log4j.logger.org.eclipse.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.spark-project.jetty=WARN +log4j.logger.org.spark-project.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO \ No newline at end of file diff --git a/extras/kinesis-asl/src/test/resources/log4j.properties b/extras/kinesis-asl/src/test/resources/log4j.properties index 853ef0ed2986f..edbecdae92096 100644 --- a/extras/kinesis-asl/src/test/resources/log4j.properties +++ b/extras/kinesis-asl/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/graphx/src/test/resources/log4j.properties +++ b/graphx/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/launcher/src/test/resources/log4j.properties b/launcher/src/test/resources/log4j.properties index 00c20ad69cd4d..67a6a98217118 100644 --- a/launcher/src/test/resources/log4j.properties +++ b/launcher/src/test/resources/log4j.properties @@ -27,5 +27,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index e7e4a4113174a..e2ee9c963a4da 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/sql/catalyst/src/test/resources/log4j.properties b/sql/catalyst/src/test/resources/log4j.properties index 287c8e3563503..eb3b1999eb996 100644 --- a/sql/catalyst/src/test/resources/log4j.properties +++ b/sql/catalyst/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN -org.eclipse.jetty.LEVEL=WARN +log4j.logger.org.spark-project.jetty=WARN +org.spark-project.jetty.LEVEL=WARN diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index 9697237bfa1a3..75e3b53a093f6 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/yarn/src/test/resources/log4j.properties b/yarn/src/test/resources/log4j.properties index aab41fa49430f..6b8a5dbf6373e 100644 --- a/yarn/src/test/resources/log4j.properties +++ b/yarn/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN log4j.logger.org.apache.hadoop=WARN From a0411aebee7c134f0426f0c2b2cb4c1c7856a291 Mon Sep 17 00:00:00 2001 From: Yanbo Liang Date: Thu, 9 Apr 2015 15:10:10 -0700 Subject: [PATCH 018/128] [SPARK-6264] [MLLIB] Support FPGrowth algorithm in Python API Support FPGrowth algorithm in Python API. Should we remove "Experimental" which were marked for FPGrowth and FPGrowthModel in Scala? jkbradley Author: Yanbo Liang Closes #5213 from yanboliang/spark-6264 and squashes the following commits: ed62ead [Yanbo Liang] trigger jenkins 8ce0359 [Yanbo Liang] fix docstring style 544c725 [Yanbo Liang] address comments a2d7cf7 [Yanbo Liang] add doc for FPGrowth.train() dcf7d73 [Yanbo Liang] add python doc b18fd07 [Yanbo Liang] trigger jenkins 2c951b8 [Yanbo Liang] fix typos 7f62c8f [Yanbo Liang] add fpm to __init__.py b96206a [Yanbo Liang] Support FPGrowth algorithm in Python API --- .../api/python/FPGrowthModelWrapper.scala | 33 ++++++++ .../mllib/api/python/PythonMLLibAPI.scala | 23 +++++- python/docs/pyspark.mllib.rst | 7 ++ python/pyspark/mllib/__init__.py | 2 +- python/pyspark/mllib/fpm.py | 81 +++++++++++++++++++ python/run-tests | 1 + 6 files changed, 143 insertions(+), 4 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala create mode 100644 python/pyspark/mllib/fpm.py diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala new file mode 100644 index 0000000000000..ee933f4cfcafd --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/FPGrowthModelWrapper.scala @@ -0,0 +1,33 @@ +/* + * 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.api.python + +import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel} +import org.apache.spark.rdd.RDD + +/** + * A Wrapper of FPGrowthModel to provide helper method for Python + */ +private[python] class FPGrowthModelWrapper(model: FPGrowthModel[Any]) + extends FPGrowthModel(model.freqItemsets) { + + def getFreqItemsets: RDD[Array[Any]] = { + SerDe.fromTuple2RDD(model.freqItemsets.map(x => (x.javaItems, x.freq))) + } +} 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 6c386cacfb7ca..1faa3def0e042 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 @@ -34,6 +34,7 @@ import org.apache.spark.api.python.SerDeUtil import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ import org.apache.spark.mllib.feature._ +import org.apache.spark.mllib.fpm.{FPGrowth, FPGrowthModel} import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.random.{RandomRDDs => RG} @@ -358,9 +359,7 @@ private[python] class PythonMLLibAPI extends Serializable { val model = new GaussianMixtureModel(weight, gaussians) model.predictSoft(data) } - - - + /** * Java stub for Python mllib ALS.train(). This stub returns a handle * to the Java object instead of the content of the Java object. Extra care @@ -420,6 +419,24 @@ private[python] class PythonMLLibAPI extends Serializable { new MatrixFactorizationModelWrapper(model) } + /** + * Java stub for Python mllib FPGrowth.train(). This stub returns a handle + * to the Java object instead of the content of the Java object. Extra care + * needs to be taken in the Python code to ensure it gets freed on exit; see + * the Py4J documentation. + */ + def trainFPGrowthModel( + data: JavaRDD[java.lang.Iterable[Any]], + minSupport: Double, + numPartitions: Int): FPGrowthModel[Any] = { + val fpg = new FPGrowth() + .setMinSupport(minSupport) + .setNumPartitions(numPartitions) + + val model = fpg.run(data.rdd.map(_.asScala.toArray)) + new FPGrowthModelWrapper(model) + } + /** * Java stub for Normalizer.transform() */ diff --git a/python/docs/pyspark.mllib.rst b/python/docs/pyspark.mllib.rst index 15101470afc07..26ece4c2c389a 100644 --- a/python/docs/pyspark.mllib.rst +++ b/python/docs/pyspark.mllib.rst @@ -31,6 +31,13 @@ pyspark.mllib.feature module :undoc-members: :show-inheritance: +pyspark.mllib.fpm module +------------------------ + +.. automodule:: pyspark.mllib.fpm + :members: + :undoc-members: + pyspark.mllib.linalg module --------------------------- diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index 6449800d9c120..f2ef573fe9f6f 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -25,7 +25,7 @@ if numpy.version.version < '1.4': raise Exception("MLlib requires NumPy 1.4+") -__all__ = ['classification', 'clustering', 'feature', 'linalg', 'random', +__all__ = ['classification', 'clustering', 'feature', 'fpm', 'linalg', 'random', 'recommendation', 'regression', 'stat', 'tree', 'util'] import sys diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py new file mode 100644 index 0000000000000..3aa6d79d7093c --- /dev/null +++ b/python/pyspark/mllib/fpm.py @@ -0,0 +1,81 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from pyspark import SparkContext +from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc + +__all__ = ['FPGrowth', 'FPGrowthModel'] + + +@inherit_doc +class FPGrowthModel(JavaModelWrapper): + + """ + .. note:: Experimental + + A FP-Growth model for mining frequent itemsets + using the Parallel FP-Growth algorithm. + + >>> data = [["a", "b", "c"], ["a", "b", "d", "e"], ["a", "c", "e"], ["a", "c", "f"]] + >>> rdd = sc.parallelize(data, 2) + >>> model = FPGrowth.train(rdd, 0.6, 2) + >>> sorted(model.freqItemsets().collect()) + [([u'a'], 4), ([u'c'], 3), ([u'c', u'a'], 3)] + """ + + def freqItemsets(self): + """ + Get the frequent itemsets of this model + """ + return self.call("getFreqItemsets") + + +class FPGrowth(object): + """ + .. note:: Experimental + + A Parallel FP-growth algorithm to mine frequent itemsets. + """ + + @classmethod + def train(cls, data, minSupport=0.3, numPartitions=-1): + """ + Computes an FP-Growth model that contains frequent itemsets. + :param data: The input data set, each element + contains a transaction. + :param minSupport: The minimal support level + (default: `0.3`). + :param numPartitions: The number of partitions used by parallel + FP-growth (default: same as input data). + """ + model = callMLlibFunc("trainFPGrowthModel", data, float(minSupport), int(numPartitions)) + return FPGrowthModel(model) + + +def _test(): + import doctest + import pyspark.mllib.fpm + globs = pyspark.mllib.fpm.__dict__.copy() + globs['sc'] = SparkContext('local[4]', 'PythonTest') + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) + globs['sc'].stop() + if failure_count: + exit(-1) + + +if __name__ == "__main__": + _test() diff --git a/python/run-tests b/python/run-tests index b7630c356cfae..f569a56fb7a9a 100755 --- a/python/run-tests +++ b/python/run-tests @@ -77,6 +77,7 @@ function run_mllib_tests() { run_test "pyspark/mllib/clustering.py" run_test "pyspark/mllib/evaluation.py" run_test "pyspark/mllib/feature.py" + run_test "pyspark/mllib/fpm.py" run_test "pyspark/mllib/linalg.py" run_test "pyspark/mllib/rand.py" run_test "pyspark/mllib/recommendation.py" From 9c67049b4ef416a80803ccb958bbac1dd02cc380 Mon Sep 17 00:00:00 2001 From: Yuhao Yang Date: Thu, 9 Apr 2015 15:37:45 -0700 Subject: [PATCH 019/128] [Spark-6693][MLlib]add tostring with max lines and width for matrix jira: https://issues.apache.org/jira/browse/SPARK-6693 It's kind of annoying when debugging and found you cannot print out the matrix as you want. original toString of Matrix only print like following, 0.17810102596909183 0.5616906241468385 ... (10 total) 0.9692861997823815 0.015558159784155756 ... 0.8513015122819192 0.031523763918528847 ... 0.5396875653953941 0.3267864552779176 ... The def toString(maxLines : Int, maxWidth : Int) is useful when debuging, logging and saving matrix to files. Author: Yuhao Yang Closes #5344 from hhbyyh/addToString and squashes the following commits: 19a6836 [Yuhao Yang] remove extra line 6314b21 [Yuhao Yang] add exclude 736c324 [Yuhao Yang] add ut and exclude 420da39 [Yuhao Yang] Merge remote-tracking branch 'upstream/master' into addToString c22f352 [Yuhao Yang] style change 64a9e0f [Yuhao Yang] add specific to string to matrix --- .../org/apache/spark/mllib/linalg/Matrices.scala | 3 +++ .../spark/mllib/linalg/MatricesSuite.scala | 16 ++++++++++++++++ project/MimaExcludes.scala | 4 ++++ 3 files changed, 23 insertions(+) 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 d1a174063caba..3fa5e068d16d4 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 @@ -87,6 +87,9 @@ sealed trait Matrix extends Serializable { /** A human readable representation of the matrix */ override def toString: String = toBreeze.toString() + /** A human readable representation of the matrix with maximum lines and width */ + def toString(maxLines: Int, maxLineWidth: Int): String = toBreeze.toString(maxLines, maxLineWidth) + /** Map the values of this matrix using a function. Generates a new matrix. Performs the * function on only the backing array. For example, an operation such as addition or * subtraction will only be performed on the non-zero values in a `SparseMatrix`. */ 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 0d2cec58e2c03..86119ec38101e 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 @@ -439,4 +439,20 @@ class MatricesSuite extends FunSuite { assert(mUDT.typeName == "matrix") assert(mUDT.simpleString == "matrix") } + + test("toString") { + val empty = Matrices.ones(0, 0) + empty.toString(0, 0) + + val mat = Matrices.rand(5, 10, new Random()) + mat.toString(-1, -5) + mat.toString(0, 0) + mat.toString(Int.MinValue, Int.MinValue) + mat.toString(Int.MaxValue, Int.MaxValue) + var lines = mat.toString(6, 50).lines.toArray + assert(lines.size == 5 && lines.forall(_.size <= 50)) + + lines = mat.toString(5, 100).lines.toArray + assert(lines.size == 5 && lines.forall(_.size <= 100)) + } } diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index c2d828f982fe0..1564babefa62f 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -64,6 +64,10 @@ object MimaExcludes { // SPARK-6492 Fix deadlock in SparkContext.stop() ProblemFilters.exclude[MissingMethodProblem]("org.apache.spark.SparkContext.org$" + "apache$spark$SparkContext$$SPARK_CONTEXT_CONSTRUCTOR_LOCK") + )++ Seq( + // SPARK-6693 add tostring with max lines and width for matrix + ProblemFilters.exclude[MissingMethodProblem]( + "org.apache.spark.mllib.linalg.Matrix.toString") ) case v if v.startsWith("1.3") => From b5c51c8df480f1a82a82e4d597d8eea631bffb4e Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 9 Apr 2015 17:07:23 -0700 Subject: [PATCH 020/128] [SPARK-3074] [PySpark] support groupByKey() with single huge key This patch change groupByKey() to use external sort based approach, so it can support single huge key. For example, it can group by a dataset including one hot key with 40 millions values (strings), using 500M memory for Python worker, finished in about 2 minutes. (it will need 6G memory in hash based approach). During groupByKey(), it will do in-memory groupBy first. If the dataset can not fit in memory, then data will be partitioned by hash. If one partition still can not fit in memory, it will switch to sort based groupBy(). Author: Davies Liu Author: Davies Liu Closes #1977 from davies/groupby and squashes the following commits: af3713a [Davies Liu] make sure it's iterator 67772dd [Davies Liu] fix tests e78c15c [Davies Liu] address comments 0b0fde8 [Davies Liu] address comments 0dcf320 [Davies Liu] address comments, rollback changes in ResultIterable e3b8eab [Davies Liu] fix narrow dependency 2a1857a [Davies Liu] typo d2f053b [Davies Liu] add repr for FlattedValuesSerializer c6a2f8d [Davies Liu] address comments 9e2df24 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby 2b9c261 [Davies Liu] fix typo in comments 70aadcd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby a14b4bd [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby ab5515b [Davies Liu] Merge branch 'master' into groupby 651f891 [Davies Liu] simplify GroupByKey 1578f2e [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby 1f69f93 [Davies Liu] fix tests 0d3395f [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby 341f1e0 [Davies Liu] add comments, refactor 47918b8 [Davies Liu] remove unused code 6540948 [Davies Liu] address comments: 17f4ec6 [Davies Liu] Merge branch 'master' of github.com:apache/spark into groupby 4d4bc86 [Davies Liu] bugfix 8ef965e [Davies Liu] Merge branch 'master' into groupby fbc504a [Davies Liu] Merge branch 'master' into groupby 779ed03 [Davies Liu] fix merge conflict 2c1d05b [Davies Liu] refactor, minor turning b48cda5 [Davies Liu] Merge branch 'master' into groupby 85138e6 [Davies Liu] Merge branch 'master' into groupby acd8e1b [Davies Liu] fix memory when groupByKey().count() 905b233 [Davies Liu] Merge branch 'sort' into groupby 1f075ed [Davies Liu] Merge branch 'master' into sort 4b07d39 [Davies Liu] compress the data while spilling 0a081c6 [Davies Liu] Merge branch 'master' into groupby f157fe7 [Davies Liu] Merge branch 'sort' into groupby eb53ca6 [Davies Liu] Merge branch 'master' into sort b2dc3bf [Davies Liu] Merge branch 'sort' into groupby 644abaf [Davies Liu] add license in LICENSE 19f7873 [Davies Liu] improve tests 11ba318 [Davies Liu] typo 085aef8 [Davies Liu] Merge branch 'master' into groupby 3ee58e5 [Davies Liu] switch to sort based groupBy, based on size of data 1ea0669 [Davies Liu] choose sort based groupByKey() automatically b40bae7 [Davies Liu] bugfix efa23df [Davies Liu] refactor, add spark.shuffle.sort=False 250be4e [Davies Liu] flatten the combined values when dumping into disks d05060d [Davies Liu] group the same key before shuffle, reduce the comparison during sorting 083d842 [Davies Liu] sorted based groupByKey() 55602ee [Davies Liu] use external sort in sortBy() and sortByKey() --- python/pyspark/join.py | 13 +- python/pyspark/rdd.py | 48 ++- python/pyspark/resultiterable.py | 7 +- python/pyspark/serializers.py | 25 +- python/pyspark/shuffle.py | 531 ++++++++++++++++++++++++------- python/pyspark/tests.py | 50 ++- 6 files changed, 531 insertions(+), 143 deletions(-) diff --git a/python/pyspark/join.py b/python/pyspark/join.py index efc1ef9396412..c3491defb2b29 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -48,7 +48,7 @@ def dispatch(seq): vbuf.append(v) elif n == 2: wbuf.append(v) - return [(v, w) for v in vbuf for w in wbuf] + return ((v, w) for v in vbuf for w in wbuf) return _do_python_join(rdd, other, numPartitions, dispatch) @@ -62,7 +62,7 @@ def dispatch(seq): wbuf.append(v) if not vbuf: vbuf.append(None) - return [(v, w) for v in vbuf for w in wbuf] + return ((v, w) for v in vbuf for w in wbuf) return _do_python_join(rdd, other, numPartitions, dispatch) @@ -76,7 +76,7 @@ def dispatch(seq): wbuf.append(v) if not wbuf: wbuf.append(None) - return [(v, w) for v in vbuf for w in wbuf] + return ((v, w) for v in vbuf for w in wbuf) return _do_python_join(rdd, other, numPartitions, dispatch) @@ -104,8 +104,9 @@ def make_mapper(i): rdd_len = len(vrdds) def dispatch(seq): - bufs = [[] for i in range(rdd_len)] - for (n, v) in seq: + bufs = [[] for _ in range(rdd_len)] + for n, v in seq: bufs[n].append(v) - return tuple(map(ResultIterable, bufs)) + return tuple(ResultIterable(vs) for vs in bufs) + return union_vrdds.groupByKey(numPartitions).mapValues(dispatch) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 2d05611321ed6..1b18789040360 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -41,7 +41,7 @@ from pyspark.storagelevel import StorageLevel from pyspark.resultiterable import ResultIterable from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, \ - get_used_memory, ExternalSorter + get_used_memory, ExternalSorter, ExternalGroupBy from pyspark.traceback_utils import SCCallSiteSync from py4j.java_collections import ListConverter, MapConverter @@ -573,8 +573,8 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): if numPartitions is None: numPartitions = self._defaultReducePartitions() - spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() == 'true') - memory = _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m")) + spill = self._can_spill() + memory = self._memory_limit() serializer = self._jrdd_deserializer def sortPartition(iterator): @@ -1699,10 +1699,8 @@ def combineByKey(self, createCombiner, mergeValue, mergeCombiners, numPartitions = self._defaultReducePartitions() serializer = self.ctx.serializer - spill = (self.ctx._conf.get("spark.shuffle.spill", 'True').lower() - == 'true') - memory = _parse_memory(self.ctx._conf.get( - "spark.python.worker.memory", "512m")) + spill = self._can_spill() + memory = self._memory_limit() agg = Aggregator(createCombiner, mergeValue, mergeCombiners) def combineLocally(iterator): @@ -1755,21 +1753,28 @@ def createZero(): return self.combineByKey(lambda v: func(createZero(), v), func, func, numPartitions) + def _can_spill(self): + return self.ctx._conf.get("spark.shuffle.spill", "True").lower() == "true" + + def _memory_limit(self): + return _parse_memory(self.ctx._conf.get("spark.python.worker.memory", "512m")) + # TODO: support variant with custom partitioner def groupByKey(self, numPartitions=None): """ Group the values for each key in the RDD into a single sequence. - Hash-partitions the resulting RDD with into numPartitions partitions. + Hash-partitions the resulting RDD with numPartitions partitions. Note: If you are grouping in order to perform an aggregation (such as a sum or average) over each key, using reduceByKey or aggregateByKey will provide much better performance. >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> map((lambda (x,y): (x, list(y))), sorted(x.groupByKey().collect())) + >>> sorted(x.groupByKey().mapValues(len).collect()) + [('a', 2), ('b', 1)] + >>> sorted(x.groupByKey().mapValues(list).collect()) [('a', [1, 1]), ('b', [1])] """ - def createCombiner(x): return [x] @@ -1781,8 +1786,27 @@ def mergeCombiners(a, b): a.extend(b) return a - return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numPartitions).mapValues(lambda x: ResultIterable(x)) + spill = self._can_spill() + memory = self._memory_limit() + serializer = self._jrdd_deserializer + agg = Aggregator(createCombiner, mergeValue, mergeCombiners) + + def combine(iterator): + merger = ExternalMerger(agg, memory * 0.9, serializer) \ + if spill else InMemoryMerger(agg) + merger.mergeValues(iterator) + return merger.iteritems() + + locally_combined = self.mapPartitions(combine, preservesPartitioning=True) + shuffled = locally_combined.partitionBy(numPartitions) + + def groupByKey(it): + merger = ExternalGroupBy(agg, memory, serializer)\ + if spill else InMemoryMerger(agg) + merger.mergeCombiners(it) + return merger.iteritems() + + return shuffled.mapPartitions(groupByKey, True).mapValues(ResultIterable) def flatMapValues(self, f): """ diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py index ef04c82866e6c..1ab5ce14c3531 100644 --- a/python/pyspark/resultiterable.py +++ b/python/pyspark/resultiterable.py @@ -15,15 +15,16 @@ # limitations under the License. # -__all__ = ["ResultIterable"] - import collections +__all__ = ["ResultIterable"] + class ResultIterable(collections.Iterable): """ - A special result iterable. This is used because the standard iterator can not be pickled + A special result iterable. This is used because the standard + iterator can not be pickled """ def __init__(self, data): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 0ffb41d02f6f6..4afa82f4b2973 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -220,6 +220,29 @@ def __repr__(self): return "BatchedSerializer(%s, %d)" % (str(self.serializer), self.batchSize) +class FlattenedValuesSerializer(BatchedSerializer): + + """ + Serializes a stream of list of pairs, split the list of values + which contain more than a certain number of objects to make them + have similar sizes. + """ + def __init__(self, serializer, batchSize=10): + BatchedSerializer.__init__(self, serializer, batchSize) + + def _batched(self, iterator): + n = self.batchSize + for key, values in iterator: + for i in xrange(0, len(values), n): + yield key, values[i:i + n] + + def load_stream(self, stream): + return self.serializer.load_stream(stream) + + def __repr__(self): + return "FlattenedValuesSerializer(%d)" % self.batchSize + + class AutoBatchedSerializer(BatchedSerializer): """ Choose the size of batch automatically based on the size of object @@ -251,7 +274,7 @@ def __eq__(self, other): return (isinstance(other, AutoBatchedSerializer) and other.serializer == self.serializer and other.bestSize == self.bestSize) - def __str__(self): + def __repr__(self): return "AutoBatchedSerializer(%s)" % str(self.serializer) diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 10a7ccd502000..8a6fc627eb383 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -16,28 +16,35 @@ # import os -import sys import platform import shutil import warnings import gc import itertools +import operator import random import pyspark.heapq3 as heapq -from pyspark.serializers import AutoBatchedSerializer, PickleSerializer +from pyspark.serializers import BatchedSerializer, PickleSerializer, FlattenedValuesSerializer, \ + CompressedSerializer, AutoBatchedSerializer + try: import psutil + process = None + def get_used_memory(): """ Return the used memory in MB """ - process = psutil.Process(os.getpid()) + global process + if process is None or process._pid != os.getpid(): + process = psutil.Process(os.getpid()) if hasattr(process, "memory_info"): info = process.memory_info() else: info = process.get_memory_info() return info.rss >> 20 + except ImportError: def get_used_memory(): @@ -46,6 +53,7 @@ def get_used_memory(): for line in open('/proc/self/status'): if line.startswith('VmRSS:'): return int(line.split()[1]) >> 10 + else: warnings.warn("Please install psutil to have better " "support with spilling") @@ -54,6 +62,7 @@ def get_used_memory(): rss = resource.getrusage(resource.RUSAGE_SELF).ru_maxrss return rss >> 20 # TODO: support windows + return 0 @@ -148,10 +157,16 @@ def mergeCombiners(self, iterator): d[k] = comb(d[k], v) if k in d else v def iteritems(self): - """ Return the merged items ad iterator """ + """ Return the merged items as iterator """ return self.data.iteritems() +def _compressed_serializer(self, serializer=None): + # always use PickleSerializer to simplify implementation + ser = PickleSerializer() + return AutoBatchedSerializer(CompressedSerializer(ser)) + + class ExternalMerger(Merger): """ @@ -173,7 +188,7 @@ class ExternalMerger(Merger): dict. Repeat this again until combine all the items. - Before return any items, it will load each partition and - combine them seperately. Yield them before loading next + combine them separately. Yield them before loading next partition. - During loading a partition, if the memory goes over limit, @@ -182,7 +197,7 @@ class ExternalMerger(Merger): `data` and `pdata` are used to hold the merged items in memory. At first, all the data are merged into `data`. Once the used - memory goes over limit, the items in `data` are dumped indo + memory goes over limit, the items in `data` are dumped into disks, `data` will be cleared, all rest of items will be merged into `pdata` and then dumped into disks. Before returning, all the items in `pdata` will be dumped into disks. @@ -193,16 +208,16 @@ class ExternalMerger(Merger): >>> agg = SimpleAggregator(lambda x, y: x + y) >>> merger = ExternalMerger(agg, 10) >>> N = 10000 - >>> merger.mergeValues(zip(xrange(N), xrange(N)) * 10) + >>> merger.mergeValues(zip(xrange(N), xrange(N))) >>> assert merger.spills > 0 >>> sum(v for k,v in merger.iteritems()) - 499950000 + 49995000 >>> merger = ExternalMerger(agg, 10) - >>> merger.mergeCombiners(zip(xrange(N), xrange(N)) * 10) + >>> merger.mergeCombiners(zip(xrange(N), xrange(N))) >>> assert merger.spills > 0 >>> sum(v for k,v in merger.iteritems()) - 499950000 + 49995000 """ # the max total partitions created recursively @@ -212,8 +227,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, localdirs=None, scale=1, partitions=59, batch=1000): Merger.__init__(self, aggregator) self.memory_limit = memory_limit - # default serializer is only used for tests - self.serializer = serializer or AutoBatchedSerializer(PickleSerializer()) + self.serializer = _compressed_serializer(serializer) self.localdirs = localdirs or _get_local_dirs(str(id(self))) # number of partitions when spill data into disks self.partitions = partitions @@ -221,7 +235,7 @@ def __init__(self, aggregator, memory_limit=512, serializer=None, self.batch = batch # scale is used to scale down the hash of key for recursive hash map self.scale = scale - # unpartitioned merged data + # un-partitioned merged data self.data = {} # partitioned merged data, list of dicts self.pdata = [] @@ -244,72 +258,63 @@ def _next_limit(self): def mergeValues(self, iterator): """ Combine the items by creator and combiner """ - iterator = iter(iterator) # speedup attribute lookup creator, comb = self.agg.createCombiner, self.agg.mergeValue - d, c, batch = self.data, 0, self.batch + c, data, pdata, hfun, batch = 0, self.data, self.pdata, self._partition, self.batch + limit = self.memory_limit for k, v in iterator: + d = pdata[hfun(k)] if pdata else data d[k] = comb(d[k], v) if k in d else creator(v) c += 1 - if c % batch == 0 and get_used_memory() > self.memory_limit: - self._spill() - self._partitioned_mergeValues(iterator, self._next_limit()) - break + if c >= batch: + if get_used_memory() >= limit: + self._spill() + limit = self._next_limit() + batch /= 2 + c = 0 + else: + batch *= 1.5 + + if get_used_memory() >= limit: + self._spill() def _partition(self, key): """ Return the partition for key """ return hash((key, self._seed)) % self.partitions - def _partitioned_mergeValues(self, iterator, limit=0): - """ Partition the items by key, then combine them """ - # speedup attribute lookup - creator, comb = self.agg.createCombiner, self.agg.mergeValue - c, pdata, hfun, batch = 0, self.pdata, self._partition, self.batch - - for k, v in iterator: - d = pdata[hfun(k)] - d[k] = comb(d[k], v) if k in d else creator(v) - if not limit: - continue - - c += 1 - if c % batch == 0 and get_used_memory() > limit: - self._spill() - limit = self._next_limit() + def _object_size(self, obj): + """ How much of memory for this obj, assume that all the objects + consume similar bytes of memory + """ + return 1 - def mergeCombiners(self, iterator, check=True): + def mergeCombiners(self, iterator, limit=None): """ Merge (K,V) pair by mergeCombiner """ - iterator = iter(iterator) + if limit is None: + limit = self.memory_limit # speedup attribute lookup - d, comb, batch = self.data, self.agg.mergeCombiners, self.batch - c = 0 - for k, v in iterator: - d[k] = comb(d[k], v) if k in d else v - if not check: - continue - - c += 1 - if c % batch == 0 and get_used_memory() > self.memory_limit: - self._spill() - self._partitioned_mergeCombiners(iterator, self._next_limit()) - break - - def _partitioned_mergeCombiners(self, iterator, limit=0): - """ Partition the items by key, then merge them """ - comb, pdata = self.agg.mergeCombiners, self.pdata - c, hfun = 0, self._partition + comb, hfun, objsize = self.agg.mergeCombiners, self._partition, self._object_size + c, data, pdata, batch = 0, self.data, self.pdata, self.batch for k, v in iterator: - d = pdata[hfun(k)] + d = pdata[hfun(k)] if pdata else data d[k] = comb(d[k], v) if k in d else v if not limit: continue - c += 1 - if c % self.batch == 0 and get_used_memory() > limit: - self._spill() - limit = self._next_limit() + c += objsize(v) + if c > batch: + if get_used_memory() > limit: + self._spill() + limit = self._next_limit() + batch /= 2 + c = 0 + else: + batch *= 1.5 + + if limit and get_used_memory() >= limit: + self._spill() def _spill(self): """ @@ -335,7 +340,7 @@ def _spill(self): for k, v in self.data.iteritems(): h = self._partition(k) - # put one item in batch, make it compatitable with load_stream + # put one item in batch, make it compatible with load_stream # it will increase the memory if dump them in batch self.serializer.dump_stream([(k, v)], streams[h]) @@ -344,7 +349,7 @@ def _spill(self): s.close() self.data.clear() - self.pdata = [{} for i in range(self.partitions)] + self.pdata.extend([{} for i in range(self.partitions)]) else: for i in range(self.partitions): @@ -370,29 +375,12 @@ def _external_items(self): assert not self.data if any(self.pdata): self._spill() - hard_limit = self._next_limit() + # disable partitioning and spilling when merge combiners from disk + self.pdata = [] try: for i in range(self.partitions): - self.data = {} - for j in range(self.spills): - path = self._get_spill_dir(j) - p = os.path.join(path, str(i)) - # do not check memory during merging - self.mergeCombiners(self.serializer.load_stream(open(p)), - False) - - # limit the total partitions - if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS - and j < self.spills - 1 - and get_used_memory() > hard_limit): - self.data.clear() # will read from disk again - gc.collect() # release the memory as much as possible - for v in self._recursive_merged_items(i): - yield v - return - - for v in self.data.iteritems(): + for v in self._merged_items(i): yield v self.data.clear() @@ -400,53 +388,56 @@ def _external_items(self): for j in range(self.spills): path = self._get_spill_dir(j) os.remove(os.path.join(path, str(i))) - finally: self._cleanup() - def _cleanup(self): - """ Clean up all the files in disks """ - for d in self.localdirs: - shutil.rmtree(d, True) + def _merged_items(self, index): + self.data = {} + limit = self._next_limit() + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + # do not check memory during merging + self.mergeCombiners(self.serializer.load_stream(open(p)), 0) + + # limit the total partitions + if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS + and j < self.spills - 1 + and get_used_memory() > limit): + self.data.clear() # will read from disk again + gc.collect() # release the memory as much as possible + return self._recursive_merged_items(index) - def _recursive_merged_items(self, start): + return self.data.iteritems() + + def _recursive_merged_items(self, index): """ merge the partitioned items and return the as iterator If one partition can not be fit in memory, then them will be partitioned and merged recursively. """ - # make sure all the data are dumps into disks. - assert not self.data - if any(self.pdata): - self._spill() - assert self.spills > 0 - - for i in range(start, self.partitions): - subdirs = [os.path.join(d, "parts", str(i)) - for d in self.localdirs] - m = ExternalMerger(self.agg, self.memory_limit, self.serializer, - subdirs, self.scale * self.partitions, self.partitions) - m.pdata = [{} for _ in range(self.partitions)] - limit = self._next_limit() - - for j in range(self.spills): - path = self._get_spill_dir(j) - p = os.path.join(path, str(i)) - m._partitioned_mergeCombiners( - self.serializer.load_stream(open(p))) - - if get_used_memory() > limit: - m._spill() - limit = self._next_limit() + subdirs = [os.path.join(d, "parts", str(index)) for d in self.localdirs] + m = ExternalMerger(self.agg, self.memory_limit, self.serializer, subdirs, + self.scale * self.partitions, self.partitions, self.batch) + m.pdata = [{} for _ in range(self.partitions)] + limit = self._next_limit() + + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + m.mergeCombiners(self.serializer.load_stream(open(p)), 0) + + if get_used_memory() > limit: + m._spill() + limit = self._next_limit() - for v in m._external_items(): - yield v + return m._external_items() - # remove the merged partition - for j in range(self.spills): - path = self._get_spill_dir(j) - os.remove(os.path.join(path, str(i))) + def _cleanup(self): + """ Clean up all the files in disks """ + for d in self.localdirs: + shutil.rmtree(d, True) class ExternalSorter(object): @@ -457,6 +448,7 @@ class ExternalSorter(object): The spilling will only happen when the used memory goes above the limit. + >>> sorter = ExternalSorter(1) # 1M >>> import random >>> l = range(1024) @@ -469,7 +461,7 @@ class ExternalSorter(object): def __init__(self, memory_limit, serializer=None): self.memory_limit = memory_limit self.local_dirs = _get_local_dirs("sort") - self.serializer = serializer or AutoBatchedSerializer(PickleSerializer()) + self.serializer = _compressed_serializer(serializer) def _get_path(self, n): """ Choose one directory for spill by number n """ @@ -515,6 +507,7 @@ def sorted(self, iterator, key=None, reverse=False): limit = self._next_limit() MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 DiskBytesSpilled += os.path.getsize(path) + os.unlink(path) # data will be deleted after close elif not chunks: batch = min(batch * 2, 10000) @@ -529,6 +522,310 @@ def sorted(self, iterator, key=None, reverse=False): return heapq.merge(chunks, key=key, reverse=reverse) +class ExternalList(object): + """ + ExternalList can have many items which cannot be hold in memory in + the same time. + + >>> l = ExternalList(range(100)) + >>> len(l) + 100 + >>> l.append(10) + >>> len(l) + 101 + >>> for i in range(20240): + ... l.append(i) + >>> len(l) + 20341 + >>> import pickle + >>> l2 = pickle.loads(pickle.dumps(l)) + >>> len(l2) + 20341 + >>> list(l2)[100] + 10 + """ + LIMIT = 10240 + + def __init__(self, values): + self.values = values + self.count = len(values) + self._file = None + self._ser = None + + def __getstate__(self): + if self._file is not None: + self._file.flush() + f = os.fdopen(os.dup(self._file.fileno())) + f.seek(0) + serialized = f.read() + else: + serialized = '' + return self.values, self.count, serialized + + def __setstate__(self, item): + self.values, self.count, serialized = item + if serialized: + self._open_file() + self._file.write(serialized) + else: + self._file = None + self._ser = None + + def __iter__(self): + if self._file is not None: + self._file.flush() + # read all items from disks first + with os.fdopen(os.dup(self._file.fileno()), 'r') as f: + f.seek(0) + for v in self._ser.load_stream(f): + yield v + + for v in self.values: + yield v + + def __len__(self): + return self.count + + def append(self, value): + self.values.append(value) + self.count += 1 + # dump them into disk if the key is huge + if len(self.values) >= self.LIMIT: + self._spill() + + def _open_file(self): + dirs = _get_local_dirs("objects") + d = dirs[id(self) % len(dirs)] + if not os.path.exists(d): + os.makedirs(d) + p = os.path.join(d, str(id)) + self._file = open(p, "w+", 65536) + self._ser = BatchedSerializer(CompressedSerializer(PickleSerializer()), 1024) + os.unlink(p) + + def _spill(self): + """ dump the values into disk """ + global MemoryBytesSpilled, DiskBytesSpilled + if self._file is None: + self._open_file() + + used_memory = get_used_memory() + pos = self._file.tell() + self._ser.dump_stream(self.values, self._file) + self.values = [] + gc.collect() + DiskBytesSpilled += self._file.tell() - pos + MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 + + +class ExternalListOfList(ExternalList): + """ + An external list for list. + + >>> l = ExternalListOfList([[i, i] for i in range(100)]) + >>> len(l) + 200 + >>> l.append(range(10)) + >>> len(l) + 210 + >>> len(list(l)) + 210 + """ + + def __init__(self, values): + ExternalList.__init__(self, values) + self.count = sum(len(i) for i in values) + + def append(self, value): + ExternalList.append(self, value) + # already counted 1 in ExternalList.append + self.count += len(value) - 1 + + def __iter__(self): + for values in ExternalList.__iter__(self): + for v in values: + yield v + + +class GroupByKey(object): + """ + Group a sorted iterator as [(k1, it1), (k2, it2), ...] + + >>> k = [i/3 for i in range(6)] + >>> v = [[i] for i in range(6)] + >>> g = GroupByKey(iter(zip(k, v))) + >>> [(k, list(it)) for k, it in g] + [(0, [0, 1, 2]), (1, [3, 4, 5])] + """ + + def __init__(self, iterator): + self.iterator = iter(iterator) + self.next_item = None + + def __iter__(self): + return self + + def next(self): + key, value = self.next_item if self.next_item else next(self.iterator) + values = ExternalListOfList([value]) + try: + while True: + k, v = next(self.iterator) + if k != key: + self.next_item = (k, v) + break + values.append(v) + except StopIteration: + self.next_item = None + return key, values + + +class ExternalGroupBy(ExternalMerger): + + """ + Group by the items by key. If any partition of them can not been + hold in memory, it will do sort based group by. + + This class works as follows: + + - It repeatedly group the items by key and save them in one dict in + memory. + + - When the used memory goes above memory limit, it will split + the combined data into partitions by hash code, dump them + into disk, one file per partition. If the number of keys + in one partitions is smaller than 1000, it will sort them + by key before dumping into disk. + + - Then it goes through the rest of the iterator, group items + by key into different dict by hash. Until the used memory goes over + memory limit, it dump all the dicts into disks, one file per + dict. Repeat this again until combine all the items. It + also will try to sort the items by key in each partition + before dumping into disks. + + - It will yield the grouped items partitions by partitions. + If the data in one partitions can be hold in memory, then it + will load and combine them in memory and yield. + + - If the dataset in one partition cannot be hold in memory, + it will sort them first. If all the files are already sorted, + it merge them by heap.merge(), so it will do external sort + for all the files. + + - After sorting, `GroupByKey` class will put all the continuous + items with the same key as a group, yield the values as + an iterator. + """ + SORT_KEY_LIMIT = 1000 + + def flattened_serializer(self): + assert isinstance(self.serializer, BatchedSerializer) + ser = self.serializer + return FlattenedValuesSerializer(ser, 20) + + def _object_size(self, obj): + return len(obj) + + def _spill(self): + """ + dump already partitioned data into disks. + """ + global MemoryBytesSpilled, DiskBytesSpilled + path = self._get_spill_dir(self.spills) + if not os.path.exists(path): + os.makedirs(path) + + used_memory = get_used_memory() + if not self.pdata: + # The data has not been partitioned, it will iterator the + # data once, write them into different files, has no + # additional memory. It only called when the memory goes + # above limit at the first time. + + # open all the files for writing + streams = [open(os.path.join(path, str(i)), 'w') + for i in range(self.partitions)] + + # If the number of keys is small, then the overhead of sort is small + # sort them before dumping into disks + self._sorted = len(self.data) < self.SORT_KEY_LIMIT + if self._sorted: + self.serializer = self.flattened_serializer() + for k in sorted(self.data.keys()): + h = self._partition(k) + self.serializer.dump_stream([(k, self.data[k])], streams[h]) + else: + for k, v in self.data.iteritems(): + h = self._partition(k) + self.serializer.dump_stream([(k, v)], streams[h]) + + for s in streams: + DiskBytesSpilled += s.tell() + s.close() + + self.data.clear() + # self.pdata is cached in `mergeValues` and `mergeCombiners` + self.pdata.extend([{} for i in range(self.partitions)]) + + else: + for i in range(self.partitions): + p = os.path.join(path, str(i)) + with open(p, "w") as f: + # dump items in batch + if self._sorted: + # sort by key only (stable) + sorted_items = sorted(self.pdata[i].iteritems(), key=operator.itemgetter(0)) + self.serializer.dump_stream(sorted_items, f) + else: + self.serializer.dump_stream(self.pdata[i].iteritems(), f) + self.pdata[i].clear() + DiskBytesSpilled += os.path.getsize(p) + + self.spills += 1 + gc.collect() # release the memory as much as possible + MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 + + def _merged_items(self, index): + size = sum(os.path.getsize(os.path.join(self._get_spill_dir(j), str(index))) + for j in range(self.spills)) + # if the memory can not hold all the partition, + # then use sort based merge. Because of compression, + # the data on disks will be much smaller than needed memory + if (size >> 20) >= self.memory_limit / 10: + return self._merge_sorted_items(index) + + self.data = {} + for j in range(self.spills): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + # do not check memory during merging + self.mergeCombiners(self.serializer.load_stream(open(p)), 0) + return self.data.iteritems() + + def _merge_sorted_items(self, index): + """ load a partition from disk, then sort and group by key """ + def load_partition(j): + path = self._get_spill_dir(j) + p = os.path.join(path, str(index)) + return self.serializer.load_stream(open(p, 'r', 65536)) + + disk_items = [load_partition(j) for j in range(self.spills)] + + if self._sorted: + # all the partitions are already sorted + sorted_items = heapq.merge(disk_items, key=operator.itemgetter(0)) + + else: + # Flatten the combined values, so it will not consume huge + # memory during merging sort. + ser = self.flattened_serializer() + sorter = ExternalSorter(self.memory_limit, ser) + sorted_items = sorter.sorted(itertools.chain(*disk_items), + key=operator.itemgetter(0)) + return ((k, vs) for k, vs in GroupByKey(sorted_items)) + + if __name__ == "__main__": import doctest doctest.testmod() diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index dd8d3b1c53733..0bd5d20f7877f 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -31,6 +31,7 @@ import time import zipfile import random +import itertools import threading import hashlib @@ -76,7 +77,7 @@ class MergerTests(unittest.TestCase): def setUp(self): - self.N = 1 << 14 + self.N = 1 << 12 self.l = [i for i in xrange(self.N)] self.data = zip(self.l, self.l) self.agg = Aggregator(lambda x: [x], @@ -108,7 +109,7 @@ def test_small_dataset(self): sum(xrange(self.N))) def test_medium_dataset(self): - m = ExternalMerger(self.agg, 10) + m = ExternalMerger(self.agg, 30) m.mergeValues(self.data) self.assertTrue(m.spills >= 1) self.assertEqual(sum(sum(v) for k, v in m.iteritems()), @@ -124,10 +125,36 @@ def test_huge_dataset(self): m = ExternalMerger(self.agg, 10, partitions=3) m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10)) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(len(v) for k, v in m._recursive_merged_items(0)), + self.assertEqual(sum(len(v) for k, v in m.iteritems()), self.N * 10) m._cleanup() + def test_group_by_key(self): + + def gen_data(N, step): + for i in range(1, N + 1, step): + for j in range(i): + yield (i, [j]) + + def gen_gs(N, step=1): + return shuffle.GroupByKey(gen_data(N, step)) + + self.assertEqual(1, len(list(gen_gs(1)))) + self.assertEqual(2, len(list(gen_gs(2)))) + self.assertEqual(100, len(list(gen_gs(100)))) + self.assertEqual(range(1, 101), [k for k, _ in gen_gs(100)]) + self.assertTrue(all(range(k) == list(vs) for k, vs in gen_gs(100))) + + for k, vs in gen_gs(50002, 10000): + self.assertEqual(k, len(vs)) + self.assertEqual(range(k), list(vs)) + + ser = PickleSerializer() + l = ser.loads(ser.dumps(list(gen_gs(50002, 30000)))) + for k, vs in l: + self.assertEqual(k, len(vs)) + self.assertEqual(range(k), list(vs)) + class SorterTests(unittest.TestCase): def test_in_memory_sort(self): @@ -702,6 +729,21 @@ def test_distinct(self): self.assertEquals(result.getNumPartitions(), 5) self.assertEquals(result.count(), 3) + def test_external_group_by_key(self): + self.sc._conf.set("spark.python.worker.memory", "5m") + N = 200001 + kv = self.sc.parallelize(range(N)).map(lambda x: (x % 3, x)) + gkv = kv.groupByKey().cache() + self.assertEqual(3, gkv.count()) + filtered = gkv.filter(lambda (k, vs): k == 1) + self.assertEqual(1, filtered.count()) + self.assertEqual([(1, N/3)], filtered.mapValues(len).collect()) + self.assertEqual([(N/3, N/3)], + filtered.values().map(lambda x: (len(x), len(list(x)))).collect()) + result = filtered.collect()[0][1] + self.assertEqual(N/3, len(result)) + self.assertTrue(isinstance(result.data, shuffle.ExternalList)) + def test_sort_on_empty_rdd(self): self.assertEqual([], self.sc.parallelize(zip([], [])).sortByKey().collect()) @@ -752,9 +794,9 @@ def test_narrow_dependency_in_join(self): self.assertEqual(rdd.getNumPartitions() + 2, parted.union(rdd).getNumPartitions()) self.assertEqual(rdd.getNumPartitions() + 2, rdd.union(parted).getNumPartitions()) - self.sc.setJobGroup("test1", "test", True) tracker = self.sc.statusTracker() + self.sc.setJobGroup("test1", "test", True) d = sorted(parted.join(parted).collect()) self.assertEqual(10, len(d)) self.assertEqual((0, (0, 0)), d[0]) From e2360810f50de77f79d372cc9b46db117d451cfc Mon Sep 17 00:00:00 2001 From: MechCoder Date: Thu, 9 Apr 2015 23:10:13 -0700 Subject: [PATCH 021/128] [SPARK-6577] [MLlib] [PySpark] SparseMatrix should be supported in PySpark Supporting of SparseMatrix in PySpark. Author: MechCoder Closes #5355 from MechCoder/spark-6577 and squashes the following commits: 7492190 [MechCoder] More readable code for densifying ea2c54b [MechCoder] Check bounds for indexing 454ef2c [MechCoder] Made the following changes 1. Used convert_to_array for array conversion. 2. Used F order for toArray 3. Minor improvements in speed. db76caf [MechCoder] Add support for CSR matrix 29653e7 [MechCoder] Renamed indices to rowIndices and indptr to colPtrs b6384fe [MechCoder] [SPARK-6577] SparseMatrix should be supported in PySpark --- python/pyspark/mllib/linalg.py | 110 +++++++++++++++++++++++++++++++-- python/pyspark/mllib/tests.py | 52 +++++++++++++++- 2 files changed, 154 insertions(+), 8 deletions(-) diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index 51c1490b1618d..a80320c52d1d0 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -640,6 +640,15 @@ def toArray(self): """ raise NotImplementedError + @staticmethod + def _convert_to_array(array_like, dtype): + """ + Convert Matrix attributes which are array-like or buffer to array. + """ + if isinstance(array_like, basestring): + return np.frombuffer(array_like, dtype=dtype) + return np.asarray(array_like, dtype=dtype) + class DenseMatrix(Matrix): """ @@ -647,13 +656,8 @@ 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) + values = self._convert_to_array(values, np.float64) assert len(values) == numRows * numCols - if values.dtype != np.float64: - values.astype(np.float64) self.values = values def __reduce__(self): @@ -670,6 +674,17 @@ def toArray(self): """ return self.values.reshape((self.numRows, self.numCols), order='F') + def toSparse(self): + """Convert to SparseMatrix""" + indices = np.nonzero(self.values)[0] + colCounts = np.bincount(indices / self.numRows) + colPtrs = np.cumsum(np.hstack( + (0, colCounts, np.zeros(self.numCols - colCounts.size)))) + values = self.values[indices] + rowIndices = indices % self.numRows + + return SparseMatrix(self.numRows, self.numCols, colPtrs, rowIndices, values) + def __getitem__(self, indices): i, j = indices if i < 0 or i >= self.numRows: @@ -687,6 +702,82 @@ def __eq__(self, other): all(self.values == other.values)) +class SparseMatrix(Matrix): + """Sparse Matrix stored in CSC format.""" + def __init__(self, numRows, numCols, colPtrs, rowIndices, values, + isTransposed=False): + Matrix.__init__(self, numRows, numCols) + self.isTransposed = isTransposed + self.colPtrs = self._convert_to_array(colPtrs, np.int32) + self.rowIndices = self._convert_to_array(rowIndices, np.int32) + self.values = self._convert_to_array(values, np.float64) + + if self.isTransposed: + if self.colPtrs.size != numRows + 1: + raise ValueError("Expected colPtrs of size %d, got %d." + % (numRows + 1, self.colPtrs.size)) + else: + if self.colPtrs.size != numCols + 1: + raise ValueError("Expected colPtrs of size %d, got %d." + % (numCols + 1, self.colPtrs.size)) + if self.rowIndices.size != self.values.size: + raise ValueError("Expected rowIndices of length %d, got %d." + % (self.rowIndices.size, self.values.size)) + + def __reduce__(self): + return SparseMatrix, ( + self.numRows, self.numCols, self.colPtrs.tostring(), + self.rowIndices.tostring(), self.values.tostring(), + self.isTransposed) + + def __getitem__(self, indices): + i, j = indices + if i < 0 or i >= self.numRows: + raise ValueError("Row index %d is out of range [0, %d)" + % (i, self.numRows)) + if j < 0 or j >= self.numCols: + raise ValueError("Column index %d is out of range [0, %d)" + % (j, self.numCols)) + + # If a CSR matrix is given, then the row index should be searched + # for in ColPtrs, and the column index should be searched for in the + # corresponding slice obtained from rowIndices. + if self.isTransposed: + j, i = i, j + + colStart = self.colPtrs[j] + colEnd = self.colPtrs[j + 1] + nz = self.rowIndices[colStart: colEnd] + ind = np.searchsorted(nz, i) + colStart + if ind < colEnd and self.rowIndices[ind] == i: + return self.values[ind] + else: + return 0.0 + + def toArray(self): + """ + Return an numpy.ndarray + """ + A = np.zeros((self.numRows, self.numCols), dtype=np.float64, order='F') + for k in xrange(self.colPtrs.size - 1): + startptr = self.colPtrs[k] + endptr = self.colPtrs[k + 1] + if self.isTransposed: + A[k, self.rowIndices[startptr:endptr]] = self.values[startptr:endptr] + else: + A[self.rowIndices[startptr:endptr], k] = self.values[startptr:endptr] + return A + + def toDense(self): + densevals = np.reshape( + self.toArray(), (self.numRows * self.numCols), order='F') + return DenseMatrix(self.numRows, self.numCols, densevals) + + # TODO: More efficient implementation: + def __eq__(self, other): + return np.all(self.toArray == other.toArray) + + class Matrices(object): @staticmethod def dense(numRows, numCols, values): @@ -695,6 +786,13 @@ def dense(numRows, numCols, values): """ return DenseMatrix(numRows, numCols, values) + @staticmethod + def sparse(numRows, numCols, colPtrs, rowIndices, values): + """ + Create a SparseMatrix + """ + return SparseMatrix(numRows, numCols, colPtrs, rowIndices, values) + def _test(): import doctest diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 61ef398487c0c..3b40158c12b74 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -24,7 +24,7 @@ import tempfile import array as pyarray -from numpy import array, array_equal +from numpy import array, array_equal, zeros from py4j.protocol import Py4JJavaError if sys.version_info[:2] <= (2, 6): @@ -38,7 +38,7 @@ from pyspark.mllib.common import _to_java_object_rdd from pyspark.mllib.linalg import Vector, SparseVector, DenseVector, VectorUDT, _convert_to_vector,\ - DenseMatrix, Vectors, Matrices + DenseMatrix, SparseMatrix, Vectors, Matrices from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.random import RandomRDDs from pyspark.mllib.stat import Statistics @@ -144,6 +144,54 @@ def test_matrix_indexing(self): for j in range(2): self.assertEquals(mat[i, j], expected[i][j]) + def test_sparse_matrix(self): + # Test sparse matrix creation. + sm1 = SparseMatrix( + 3, 4, [0, 2, 2, 4, 4], [1, 2, 1, 2], [1.0, 2.0, 4.0, 5.0]) + self.assertEquals(sm1.numRows, 3) + self.assertEquals(sm1.numCols, 4) + self.assertEquals(sm1.colPtrs.tolist(), [0, 2, 2, 4, 4]) + self.assertEquals(sm1.rowIndices.tolist(), [1, 2, 1, 2]) + self.assertEquals(sm1.values.tolist(), [1.0, 2.0, 4.0, 5.0]) + + # Test indexing + expected = [ + [0, 0, 0, 0], + [1, 0, 4, 0], + [2, 0, 5, 0]] + + for i in range(3): + for j in range(4): + self.assertEquals(expected[i][j], sm1[i, j]) + self.assertTrue(array_equal(sm1.toArray(), expected)) + + # Test conversion to dense and sparse. + smnew = sm1.toDense().toSparse() + self.assertEquals(sm1.numRows, smnew.numRows) + self.assertEquals(sm1.numCols, smnew.numCols) + self.assertTrue(array_equal(sm1.colPtrs, smnew.colPtrs)) + self.assertTrue(array_equal(sm1.rowIndices, smnew.rowIndices)) + self.assertTrue(array_equal(sm1.values, smnew.values)) + + sm1t = SparseMatrix( + 3, 4, [0, 2, 3, 5], [0, 1, 2, 0, 2], [3.0, 2.0, 4.0, 9.0, 8.0], + isTransposed=True) + self.assertEquals(sm1t.numRows, 3) + self.assertEquals(sm1t.numCols, 4) + self.assertEquals(sm1t.colPtrs.tolist(), [0, 2, 3, 5]) + self.assertEquals(sm1t.rowIndices.tolist(), [0, 1, 2, 0, 2]) + self.assertEquals(sm1t.values.tolist(), [3.0, 2.0, 4.0, 9.0, 8.0]) + + expected = [ + [3, 2, 0, 0], + [0, 0, 4, 0], + [9, 0, 8, 0]] + + for i in range(3): + for j in range(4): + self.assertEquals(expected[i][j], sm1t[i, j]) + self.assertTrue(array_equal(sm1t.toArray(), expected)) + class ListTests(PySparkTestCase): From 3290d2d13bb4bd875aec14425c8e3766f9cc644b Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 9 Apr 2015 23:14:24 -0700 Subject: [PATCH 022/128] [SPARK-6211][Streaming] Add Python Kafka API unit test Refactor the Kafka unit test and add Python API support. CC tdas davies please help to review, thanks a lot. Author: jerryshao Author: Saisai Shao Closes #4961 from jerryshao/SPARK-6211 and squashes the following commits: ee4b919 [jerryshao] Fixed newly merged issue 82c756e [jerryshao] Address the comments 92912d1 [jerryshao] Address the commits 0708bb1 [jerryshao] Fix rebase issue 40b47a3 [Saisai Shao] Style fix f889657 [Saisai Shao] Update the code according 8a2f3e2 [jerryshao] Address the issues 0f1b7ce [jerryshao] Still fix the bug 61a04f0 [jerryshao] Fix bugs and address the issues 64d9877 [jerryshao] Fix rebase bugs 8ad442f [jerryshao] Add kafka-assembly in run-tests 6020b00 [jerryshao] Add more debug info in Shell 8102d6e [jerryshao] Fix bug in Jenkins test fde1213 [jerryshao] Code style changes 5536f95 [jerryshao] Refactor the Kafka unit test and add Python Kafka unittest support --- dev/run-tests | 2 +- .../streaming/kafka/KafkaTestUtils.scala | 261 ++++++++++++++++++ .../kafka/JavaDirectKafkaStreamSuite.java | 28 +- .../streaming/kafka/JavaKafkaRDDSuite.java | 28 +- .../streaming/kafka/JavaKafkaStreamSuite.java | 34 ++- .../kafka/DirectKafkaStreamSuite.scala | 56 ++-- .../streaming/kafka/KafkaClusterSuite.scala | 29 +- .../spark/streaming/kafka/KafkaRDDSuite.scala | 40 +-- .../streaming/kafka/KafkaStreamSuite.scala | 211 ++------------ .../kafka/ReliableKafkaStreamSuite.scala | 62 +++-- python/pyspark/streaming/tests.py | 43 ++- python/run-tests | 19 +- 12 files changed, 502 insertions(+), 311 deletions(-) create mode 100644 external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala diff --git a/dev/run-tests b/dev/run-tests index 1b6cf78b5da01..bb21ab6c9aa04 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -173,7 +173,7 @@ CURRENT_BLOCK=$BLOCK_BUILD build/mvn $HIVE_BUILD_ARGS clean package -DskipTests else echo -e "q\n" \ - | build/sbt $HIVE_BUILD_ARGS package assembly/assembly \ + | build/sbt $HIVE_BUILD_ARGS package assembly/assembly streaming-kafka-assembly/assembly \ | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" fi } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala new file mode 100644 index 0000000000000..13e9475065979 --- /dev/null +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala @@ -0,0 +1,261 @@ +/* + * 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.streaming.kafka + +import java.io.File +import java.lang.{Integer => JInt} +import java.net.InetSocketAddress +import java.util.{Map => JMap} +import java.util.Properties +import java.util.concurrent.TimeoutException + +import scala.annotation.tailrec +import scala.language.postfixOps +import scala.util.control.NonFatal + +import kafka.admin.AdminUtils +import kafka.producer.{KeyedMessage, Producer, ProducerConfig} +import kafka.serializer.StringEncoder +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.ZKStringSerializer +import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} +import org.I0Itec.zkclient.ZkClient + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.streaming.Time +import org.apache.spark.util.Utils + +/** + * This is a helper class for Kafka test suites. This has the functionality to set up + * and tear down local Kafka servers, and to push data using Kafka producers. + * + * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. + */ +private class KafkaTestUtils extends Logging { + + // Zookeeper related configurations + private val zkHost = "localhost" + private var zkPort: Int = 0 + private val zkConnectionTimeout = 6000 + private val zkSessionTimeout = 6000 + + private var zookeeper: EmbeddedZookeeper = _ + + private var zkClient: ZkClient = _ + + // Kafka broker related configurations + private val brokerHost = "localhost" + private var brokerPort = 9092 + private var brokerConf: KafkaConfig = _ + + // Kafka broker server + private var server: KafkaServer = _ + + // Kafka producer + private var producer: Producer[String, String] = _ + + // Flag to test whether the system is correctly started + private var zkReady = false + private var brokerReady = false + + def zkAddress: String = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") + s"$zkHost:$zkPort" + } + + def brokerAddress: String = { + assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") + s"$brokerHost:$brokerPort" + } + + def zookeeperClient: ZkClient = { + assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") + Option(zkClient).getOrElse( + throw new IllegalStateException("Zookeeper client is not yet initialized")) + } + + // Set up the Embedded Zookeeper server and get the proper Zookeeper port + private def setupEmbeddedZookeeper(): Unit = { + // Zookeeper server startup + zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") + // Get the actual zookeeper binding port + zkPort = zookeeper.actualPort + zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, + ZKStringSerializer) + zkReady = true + } + + // Set up the Embedded Kafka server + private def setupEmbeddedKafkaServer(): Unit = { + assert(zkReady, "Zookeeper should be set up beforehand") + + // Kafka broker startup + Utils.startServiceOnPort(brokerPort, port => { + brokerPort = port + brokerConf = new KafkaConfig(brokerConfiguration) + server = new KafkaServer(brokerConf) + server.startup() + (server, port) + }, new SparkConf(), "KafkaBroker") + + brokerReady = true + } + + /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ + def setup(): Unit = { + setupEmbeddedZookeeper() + setupEmbeddedKafkaServer() + } + + /** Teardown the whole servers, including Kafka broker and Zookeeper */ + def teardown(): Unit = { + brokerReady = false + zkReady = false + + if (producer != null) { + producer.close() + producer = null + } + + if (server != null) { + server.shutdown() + server = null + } + + brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } + + if (zkClient != null) { + zkClient.close() + zkClient = null + } + + if (zookeeper != null) { + zookeeper.shutdown() + zookeeper = null + } + } + + /** Create a Kafka topic and wait until it propagated to the whole cluster */ + def createTopic(topic: String): Unit = { + AdminUtils.createTopic(zkClient, topic, 1, 1) + // wait until metadata is propagated + waitUntilMetadataIsPropagated(topic, 0) + } + + /** Java-friendly function for sending messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { + import scala.collection.JavaConversions._ + sendMessages(topic, Map(messageToFreq.mapValues(_.intValue()).toSeq: _*)) + } + + /** Send the messages to the Kafka broker */ + def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { + val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray + sendMessages(topic, messages) + } + + /** Send the array of messages to the Kafka broker */ + def sendMessages(topic: String, messages: Array[String]): Unit = { + producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) + producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) + producer.close() + producer = null + } + + private def brokerConfiguration: Properties = { + val props = new Properties() + props.put("broker.id", "0") + props.put("host.name", "localhost") + props.put("port", brokerPort.toString) + props.put("log.dir", Utils.createTempDir().getAbsolutePath) + props.put("zookeeper.connect", zkAddress) + props.put("log.flush.interval.messages", "1") + props.put("replica.socket.timeout.ms", "1500") + props + } + + private def producerConfiguration: Properties = { + val props = new Properties() + props.put("metadata.broker.list", brokerAddress) + props.put("serializer.class", classOf[StringEncoder].getName) + props + } + + // A simplified version of scalatest eventually, rewritten here to avoid adding extra test + // dependency + def eventually[T](timeout: Time, interval: Time)(func: => T): T = { + def makeAttempt(): Either[Throwable, T] = { + try { + Right(func) + } catch { + case e if NonFatal(e) => Left(e) + } + } + + val startTime = System.currentTimeMillis() + @tailrec + def tryAgain(attempt: Int): T = { + makeAttempt() match { + case Right(result) => result + case Left(e) => + val duration = System.currentTimeMillis() - startTime + if (duration < timeout.milliseconds) { + Thread.sleep(interval.milliseconds) + } else { + throw new TimeoutException(e.getMessage) + } + + tryAgain(attempt + 1) + } + } + + tryAgain(1) + } + + private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { + eventually(Time(10000), Time(100)) { + assert( + server.apis.metadataCache.containsTopicAndPartition(topic, partition), + s"Partition [$topic, $partition] metadata not propagated after timeout" + ) + } + } + + private class EmbeddedZookeeper(val zkConnect: String) { + val snapshotDir = Utils.createTempDir() + val logDir = Utils.createTempDir() + + val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) + val (ip, port) = { + val splits = zkConnect.split(":") + (splits(0), splits(1).toInt) + } + val factory = new NIOServerCnxnFactory() + factory.configure(new InetSocketAddress(ip, port), 16) + factory.startup(zookeeper) + + val actualPort = factory.getLocalPort + + def shutdown() { + factory.shutdown() + Utils.deleteRecursively(snapshotDir) + Utils.deleteRecursively(logDir) + } + } +} + diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java index d6ca6d58b5665..4c1d6a03eb2b8 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java @@ -41,24 +41,28 @@ public class JavaDirectKafkaStreamSuite implements Serializable { private transient JavaStreamingContext ssc = null; - private transient KafkaStreamSuiteBase suiteBase = null; + private transient KafkaTestUtils kafkaTestUtils = null; @Before public void setUp() { - suiteBase = new KafkaStreamSuiteBase() { }; - suiteBase.setupKafka(); - System.clearProperty("spark.driver.port"); - SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); - ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); + SparkConf sparkConf = new SparkConf() + .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); + ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); } @After public void tearDown() { + if (ssc != null) { ssc.stop(); ssc = null; - System.clearProperty("spark.driver.port"); - suiteBase.tearDownKafka(); + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } } @Test @@ -74,7 +78,7 @@ public void testKafkaStream() throws InterruptedException { sent.addAll(Arrays.asList(topic2data)); HashMap kafkaParams = new HashMap(); - kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress()); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); kafkaParams.put("auto.offset.reset", "smallest"); JavaDStream stream1 = KafkaUtils.createDirectStream( @@ -147,8 +151,8 @@ private HashMap topicOffsetToMap(String topic, Long off private String[] createTopicAndSendData(String topic) { String[] data = { topic + "-1", topic + "-2", topic + "-3"}; - suiteBase.createTopic(topic); - suiteBase.sendMessages(topic, data); + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); return data; } } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java index 4477b81827c70..a9dc6e50613ca 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java @@ -37,13 +37,12 @@ public class JavaKafkaRDDSuite implements Serializable { private transient JavaSparkContext sc = null; - private transient KafkaStreamSuiteBase suiteBase = null; + private transient KafkaTestUtils kafkaTestUtils = null; @Before public void setUp() { - suiteBase = new KafkaStreamSuiteBase() { }; - suiteBase.setupKafka(); - System.clearProperty("spark.driver.port"); + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); SparkConf sparkConf = new SparkConf() .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); sc = new JavaSparkContext(sparkConf); @@ -51,10 +50,15 @@ public void setUp() { @After public void tearDown() { - sc.stop(); - sc = null; - System.clearProperty("spark.driver.port"); - suiteBase.tearDownKafka(); + if (sc != null) { + sc.stop(); + sc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } } @Test @@ -66,7 +70,7 @@ public void testKafkaRDD() throws InterruptedException { String[] topic2data = createTopicAndSendData(topic2); HashMap kafkaParams = new HashMap(); - kafkaParams.put("metadata.broker.list", suiteBase.brokerAddress()); + kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); OffsetRange[] offsetRanges = { OffsetRange.create(topic1, 0, 0, 1), @@ -75,7 +79,7 @@ public void testKafkaRDD() throws InterruptedException { HashMap emptyLeaders = new HashMap(); HashMap leaders = new HashMap(); - String[] hostAndPort = suiteBase.brokerAddress().split(":"); + String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); leaders.put(new TopicAndPartition(topic1, 0), broker); leaders.put(new TopicAndPartition(topic2, 0), broker); @@ -144,8 +148,8 @@ public String call(MessageAndMetadata msgAndMd) throws Exception private String[] createTopicAndSendData(String topic) { String[] data = { topic + "-1", topic + "-2", topic + "-3"}; - suiteBase.createTopic(topic); - suiteBase.sendMessages(topic, data); + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, data); return data; } } diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index bad0a93eb2e84..540f4ceabab47 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -22,9 +22,7 @@ import java.util.List; import java.util.Random; -import scala.Predef; import scala.Tuple2; -import scala.collection.JavaConverters; import kafka.serializer.StringDecoder; import org.junit.After; @@ -44,13 +42,12 @@ public class JavaKafkaStreamSuite implements Serializable { private transient JavaStreamingContext ssc = null; private transient Random random = new Random(); - private transient KafkaStreamSuiteBase suiteBase = null; + private transient KafkaTestUtils kafkaTestUtils = null; @Before public void setUp() { - suiteBase = new KafkaStreamSuiteBase() { }; - suiteBase.setupKafka(); - System.clearProperty("spark.driver.port"); + kafkaTestUtils = new KafkaTestUtils(); + kafkaTestUtils.setup(); SparkConf sparkConf = new SparkConf() .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); ssc = new JavaStreamingContext(sparkConf, new Duration(500)); @@ -58,10 +55,15 @@ public void setUp() { @After public void tearDown() { - ssc.stop(); - ssc = null; - System.clearProperty("spark.driver.port"); - suiteBase.tearDownKafka(); + if (ssc != null) { + ssc.stop(); + ssc = null; + } + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown(); + kafkaTestUtils = null; + } } @Test @@ -75,15 +77,11 @@ public void testKafkaStream() throws InterruptedException { sent.put("b", 3); sent.put("c", 10); - suiteBase.createTopic(topic); - HashMap tmp = new HashMap(sent); - suiteBase.sendMessages(topic, - JavaConverters.mapAsScalaMapConverter(tmp).asScala().toMap( - Predef.>conforms()) - ); + kafkaTestUtils.createTopic(topic); + kafkaTestUtils.sendMessages(topic, sent); HashMap kafkaParams = new HashMap(); - kafkaParams.put("zookeeper.connect", suiteBase.zkAddress()); + kafkaParams.put("zookeeper.connect", kafkaTestUtils.zkAddress()); kafkaParams.put("group.id", "test-consumer-" + random.nextInt(10000)); kafkaParams.put("auto.offset.reset", "smallest"); @@ -126,6 +124,7 @@ public Void call(JavaPairRDD rdd) throws Exception { ); ssc.start(); + long startTime = System.currentTimeMillis(); boolean sizeMatches = false; while (!sizeMatches && System.currentTimeMillis() - startTime < 20000) { @@ -136,6 +135,5 @@ public Void call(JavaPairRDD rdd) throws Exception { for (String k : sent.keySet()) { Assert.assertEquals(sent.get(k).intValue(), result.get(k).intValue()); } - ssc.stop(); } } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala index 17ca9d145d665..415730f5559c5 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala @@ -27,31 +27,41 @@ import scala.language.postfixOps import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata import kafka.serializer.StringDecoder -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Eventually -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.rdd.RDD import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} import org.apache.spark.streaming.dstream.DStream import org.apache.spark.util.Utils -class DirectKafkaStreamSuite extends KafkaStreamSuiteBase - with BeforeAndAfter with BeforeAndAfterAll with Eventually { +class DirectKafkaStreamSuite + extends FunSuite + with BeforeAndAfter + with BeforeAndAfterAll + with Eventually + with Logging { val sparkConf = new SparkConf() .setMaster("local[4]") .setAppName(this.getClass.getSimpleName) - var sc: SparkContext = _ - var ssc: StreamingContext = _ - var testDir: File = _ + private var sc: SparkContext = _ + private var ssc: StreamingContext = _ + private var testDir: File = _ + + private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll { - setupKafka() + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() } override def afterAll { - tearDownKafka() + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } after { @@ -72,12 +82,12 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase val topics = Set("basic1", "basic2", "basic3") val data = Map("a" -> 7, "b" -> 9) topics.foreach { t => - createTopic(t) - sendMessages(t, data) + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) } val totalSent = data.values.sum * topics.size val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "smallest" ) @@ -121,9 +131,9 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase val topic = "largest" val topicPartition = TopicAndPartition(topic, 0) val data = Map("a" -> 10) - createTopic(topic) + kafkaTestUtils.createTopic(topic) val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "largest" ) val kc = new KafkaCluster(kafkaParams) @@ -132,7 +142,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } // Send some initial messages before starting context - sendMessages(topic, data) + kafkaTestUtils.sendMessages(topic, data) eventually(timeout(10 seconds), interval(20 milliseconds)) { assert(getLatestOffset() > 3) } @@ -154,7 +164,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase stream.map { _._2 }.foreachRDD { rdd => collectedData ++= rdd.collect() } ssc.start() val newData = Map("b" -> 10) - sendMessages(topic, newData) + kafkaTestUtils.sendMessages(topic, newData) eventually(timeout(10 seconds), interval(50 milliseconds)) { collectedData.contains("b") } @@ -166,9 +176,9 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase val topic = "offset" val topicPartition = TopicAndPartition(topic, 0) val data = Map("a" -> 10) - createTopic(topic) + kafkaTestUtils.createTopic(topic) val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "largest" ) val kc = new KafkaCluster(kafkaParams) @@ -177,7 +187,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase } // Send some initial messages before starting context - sendMessages(topic, data) + kafkaTestUtils.sendMessages(topic, data) eventually(timeout(10 seconds), interval(20 milliseconds)) { assert(getLatestOffset() >= 10) } @@ -200,7 +210,7 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase stream.foreachRDD { rdd => collectedData ++= rdd.collect() } ssc.start() val newData = Map("b" -> 10) - sendMessages(topic, newData) + kafkaTestUtils.sendMessages(topic, newData) eventually(timeout(10 seconds), interval(50 milliseconds)) { collectedData.contains("b") } @@ -210,18 +220,18 @@ class DirectKafkaStreamSuite extends KafkaStreamSuiteBase // Test to verify the offset ranges can be recovered from the checkpoints test("offset recovery") { val topic = "recovery" - createTopic(topic) + kafkaTestUtils.createTopic(topic) testDir = Utils.createTempDir() val kafkaParams = Map( - "metadata.broker.list" -> s"$brokerAddress", + "metadata.broker.list" -> kafkaTestUtils.brokerAddress, "auto.offset.reset" -> "smallest" ) // Send data to Kafka and wait for it to be received def sendDataAndWaitForReceive(data: Seq[Int]) { val strings = data.map { _.toString} - sendMessages(topic, strings.map { _ -> 1}.toMap) + kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) eventually(timeout(10 seconds), interval(50 milliseconds)) { assert(strings.forall { DirectKafkaStreamSuite.collectedData.contains }) } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala index fc9275b7207be..2b33d2a220b2b 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala @@ -20,28 +20,35 @@ package org.apache.spark.streaming.kafka import scala.util.Random import kafka.common.TopicAndPartition -import org.scalatest.BeforeAndAfterAll +import org.scalatest.{BeforeAndAfterAll, FunSuite} -class KafkaClusterSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { - val topic = "kcsuitetopic" + Random.nextInt(10000) - val topicAndPartition = TopicAndPartition(topic, 0) - var kc: KafkaCluster = null +class KafkaClusterSuite extends FunSuite with BeforeAndAfterAll { + private val topic = "kcsuitetopic" + Random.nextInt(10000) + private val topicAndPartition = TopicAndPartition(topic, 0) + private var kc: KafkaCluster = null + + private var kafkaTestUtils: KafkaTestUtils = _ override def beforeAll() { - setupKafka() - createTopic(topic) - sendMessages(topic, Map("a" -> 1)) - kc = new KafkaCluster(Map("metadata.broker.list" -> s"$brokerAddress")) + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() + + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, Map("a" -> 1)) + kc = new KafkaCluster(Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress)) } override def afterAll() { - tearDownKafka() + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } test("metadata apis") { val leader = kc.findLeaders(Set(topicAndPartition)).right.get(topicAndPartition) val leaderAddress = s"${leader._1}:${leader._2}" - assert(leaderAddress === brokerAddress, "didn't get leader") + assert(leaderAddress === kafkaTestUtils.brokerAddress, "didn't get leader") val parts = kc.getPartitions(Set(topic)).right.get assert(parts(topicAndPartition), "didn't get partitions") diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala index a223da70b043f..7d26ce50875b3 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala @@ -22,18 +22,22 @@ import scala.util.Random import kafka.serializer.StringDecoder import kafka.common.TopicAndPartition import kafka.message.MessageAndMetadata -import org.scalatest.BeforeAndAfterAll +import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.apache.spark._ -import org.apache.spark.SparkContext._ -class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { - val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) - var sc: SparkContext = _ +class KafkaRDDSuite extends FunSuite with BeforeAndAfterAll { + + private var kafkaTestUtils: KafkaTestUtils = _ + + private val sparkConf = new SparkConf().setMaster("local[4]") + .setAppName(this.getClass.getSimpleName) + private var sc: SparkContext = _ + override def beforeAll { sc = new SparkContext(sparkConf) - - setupKafka() + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() } override def afterAll { @@ -41,17 +45,21 @@ class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { sc.stop sc = null } - tearDownKafka() + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } test("basic usage") { val topic = "topicbasic" - createTopic(topic) + kafkaTestUtils.createTopic(topic) val messages = Set("the", "quick", "brown", "fox") - sendMessages(topic, messages.toArray) + kafkaTestUtils.sendMessages(topic, messages.toArray) - val kafkaParams = Map("metadata.broker.list" -> brokerAddress, + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, "group.id" -> s"test-consumer-${Random.nextInt(10000)}") val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) @@ -67,15 +75,15 @@ class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { // the idea is to find e.g. off-by-one errors between what kafka has available and the rdd val topic = "topic1" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) - createTopic(topic) + kafkaTestUtils.createTopic(topic) - val kafkaParams = Map("metadata.broker.list" -> brokerAddress, + val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, "group.id" -> s"test-consumer-${Random.nextInt(10000)}") val kc = new KafkaCluster(kafkaParams) // this is the "lots of messages" case - sendMessages(topic, sent) + kafkaTestUtils.sendMessages(topic, sent) // rdd defined from leaders after sending messages, should get the number sent val rdd = getRdd(kc, Set(topic)) @@ -92,14 +100,14 @@ class KafkaRDDSuite extends KafkaStreamSuiteBase with BeforeAndAfterAll { // shouldn't get anything, since message is sent after rdd was defined val sentOnlyOne = Map("d" -> 1) - sendMessages(topic, sentOnlyOne) + kafkaTestUtils.sendMessages(topic, sentOnlyOne) assert(rdd2.isDefined) assert(rdd2.get.count === 0, "got messages when there shouldn't be any") // this is the "exactly 1 message" case, namely the single message from sentOnlyOne above val rdd3 = getRdd(kc, Set(topic)) // send lots of messages after rdd was defined, they shouldn't show up - sendMessages(topic, Map("extra" -> 22)) + kafkaTestUtils.sendMessages(topic, Map("extra" -> 22)) assert(rdd3.isDefined) assert(rdd3.get.count === sentOnlyOne.values.sum, "didn't get exactly one message") diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index e4966eebb9b34..24699dfc33adb 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -17,209 +17,38 @@ package org.apache.spark.streaming.kafka -import java.io.File -import java.net.InetSocketAddress -import java.util.Properties - import scala.collection.mutable import scala.concurrent.duration._ import scala.language.postfixOps import scala.util.Random -import kafka.admin.AdminUtils -import kafka.common.{KafkaException, TopicAndPartition} -import kafka.producer.{KeyedMessage, Producer, ProducerConfig} -import kafka.serializer.{StringDecoder, StringEncoder} -import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.ZKStringSerializer -import org.I0Itec.zkclient.ZkClient -import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} -import org.scalatest.{BeforeAndAfter, FunSuite} +import kafka.serializer.StringDecoder +import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Eventually -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.SparkConf import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext} -import org.apache.spark.util.Utils - -/** - * This is an abstract base class for Kafka testsuites. This has the functionality to set up - * and tear down local Kafka servers, and to push data using Kafka producers. - */ -abstract class KafkaStreamSuiteBase extends FunSuite with Eventually with Logging { - - private val zkHost = "localhost" - private var zkPort: Int = 0 - private val zkConnectionTimeout = 6000 - private val zkSessionTimeout = 6000 - private var zookeeper: EmbeddedZookeeper = _ - private val brokerHost = "localhost" - private var brokerPort = 9092 - private var brokerConf: KafkaConfig = _ - private var server: KafkaServer = _ - private var producer: Producer[String, String] = _ - private var zkReady = false - private var brokerReady = false - - protected var zkClient: ZkClient = _ - - def zkAddress: String = { - assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") - s"$zkHost:$zkPort" - } - def brokerAddress: String = { - assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") - s"$brokerHost:$brokerPort" - } - - def setupKafka() { - // Zookeeper server startup - zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") - // Get the actual zookeeper binding port - zkPort = zookeeper.actualPort - zkReady = true - logInfo("==================== Zookeeper Started ====================") +class KafkaStreamSuite extends FunSuite with Eventually with BeforeAndAfterAll { + private var ssc: StreamingContext = _ + private var kafkaTestUtils: KafkaTestUtils = _ - zkClient = new ZkClient(zkAddress, zkSessionTimeout, zkConnectionTimeout, ZKStringSerializer) - logInfo("==================== Zookeeper Client Created ====================") - - // Kafka broker startup - var bindSuccess: Boolean = false - while(!bindSuccess) { - try { - val brokerProps = getBrokerConfig() - brokerConf = new KafkaConfig(brokerProps) - server = new KafkaServer(brokerConf) - server.startup() - logInfo("==================== Kafka Broker Started ====================") - bindSuccess = true - } catch { - case e: KafkaException => - if (e.getMessage != null && e.getMessage.contains("Socket server failed to bind to")) { - brokerPort += 1 - } - case e: Exception => throw new Exception("Kafka server create failed", e) - } - } - - Thread.sleep(2000) - logInfo("==================== Kafka + Zookeeper Ready ====================") - brokerReady = true + override def beforeAll(): Unit = { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() } - def tearDownKafka() { - brokerReady = false - zkReady = false - if (producer != null) { - producer.close() - producer = null - } - - if (server != null) { - server.shutdown() - server = null - } - - brokerConf.logDirs.foreach { f => Utils.deleteRecursively(new File(f)) } - - if (zkClient != null) { - zkClient.close() - zkClient = null - } - - if (zookeeper != null) { - zookeeper.shutdown() - zookeeper = null - } - } - - def createTopic(topic: String) { - AdminUtils.createTopic(zkClient, topic, 1, 1) - // wait until metadata is propagated - waitUntilMetadataIsPropagated(topic, 0) - logInfo(s"==================== Topic $topic Created ====================") - } - - def sendMessages(topic: String, messageToFreq: Map[String, Int]) { - val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray - sendMessages(topic, messages) - } - - def sendMessages(topic: String, messages: Array[String]) { - producer = new Producer[String, String](new ProducerConfig(getProducerConfig())) - producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) - producer.close() - logInfo(s"==================== Sent Messages: ${messages.mkString(", ")} ====================") - } - - private def getBrokerConfig(): Properties = { - val props = new Properties() - props.put("broker.id", "0") - props.put("host.name", "localhost") - props.put("port", brokerPort.toString) - props.put("log.dir", Utils.createTempDir().getAbsolutePath) - props.put("zookeeper.connect", zkAddress) - props.put("log.flush.interval.messages", "1") - props.put("replica.socket.timeout.ms", "1500") - props - } - - private def getProducerConfig(): Properties = { - val brokerAddr = brokerConf.hostName + ":" + brokerConf.port - val props = new Properties() - props.put("metadata.broker.list", brokerAddr) - props.put("serializer.class", classOf[StringEncoder].getName) - props - } - - private def waitUntilMetadataIsPropagated(topic: String, partition: Int) { - eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { - assert( - server.apis.metadataCache.containsTopicAndPartition(topic, partition), - s"Partition [$topic, $partition] metadata not propagated after timeout" - ) - } - } - - class EmbeddedZookeeper(val zkConnect: String) { - val random = new Random() - val snapshotDir = Utils.createTempDir() - val logDir = Utils.createTempDir() - - val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) - val (ip, port) = { - val splits = zkConnect.split(":") - (splits(0), splits(1).toInt) - } - val factory = new NIOServerCnxnFactory() - factory.configure(new InetSocketAddress(ip, port), 16) - factory.startup(zookeeper) - - val actualPort = factory.getLocalPort - - def shutdown() { - factory.shutdown() - Utils.deleteRecursively(snapshotDir) - Utils.deleteRecursively(logDir) - } - } -} - - -class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter { - var ssc: StreamingContext = _ - - before { - setupKafka() - } - - after { + override def afterAll(): Unit = { if (ssc != null) { ssc.stop() ssc = null } - tearDownKafka() + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } } test("Kafka input stream") { @@ -227,10 +56,10 @@ class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter { ssc = new StreamingContext(sparkConf, Milliseconds(500)) val topic = "topic1" val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) - createTopic(topic) - sendMessages(topic, sent) + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, sent) - val kafkaParams = Map("zookeeper.connect" -> zkAddress, + val kafkaParams = Map("zookeeper.connect" -> kafkaTestUtils.zkAddress, "group.id" -> s"test-consumer-${Random.nextInt(10000)}", "auto.offset.reset" -> "smallest") @@ -244,14 +73,14 @@ class KafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter { result.put(kv._1, count) } } + ssc.start() + eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { assert(sent.size === result.size) sent.keys.foreach { k => assert(sent(k) === result(k).toInt) } } - ssc.stop() } } - diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala index 3cd960d1fd1d4..38548dd73b82c 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.streaming.kafka - import java.io.File import scala.collection.mutable @@ -27,7 +26,7 @@ import scala.util.Random import kafka.serializer.StringDecoder import kafka.utils.{ZKGroupTopicDirs, ZkUtils} -import org.scalatest.BeforeAndAfter +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Eventually import org.apache.spark.SparkConf @@ -35,47 +34,61 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Milliseconds, StreamingContext} import org.apache.spark.util.Utils -class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter with Eventually { +class ReliableKafkaStreamSuite extends FunSuite + with BeforeAndAfterAll with BeforeAndAfter with Eventually { - val sparkConf = new SparkConf() + private val sparkConf = new SparkConf() .setMaster("local[4]") .setAppName(this.getClass.getSimpleName) .set("spark.streaming.receiver.writeAheadLog.enable", "true") - val data = Map("a" -> 10, "b" -> 10, "c" -> 10) + private val data = Map("a" -> 10, "b" -> 10, "c" -> 10) + private var kafkaTestUtils: KafkaTestUtils = _ - var groupId: String = _ - var kafkaParams: Map[String, String] = _ - var ssc: StreamingContext = _ - var tempDirectory: File = null + private var groupId: String = _ + private var kafkaParams: Map[String, String] = _ + private var ssc: StreamingContext = _ + private var tempDirectory: File = null + + override def beforeAll() : Unit = { + kafkaTestUtils = new KafkaTestUtils + kafkaTestUtils.setup() - before { - setupKafka() groupId = s"test-consumer-${Random.nextInt(10000)}" kafkaParams = Map( - "zookeeper.connect" -> zkAddress, + "zookeeper.connect" -> kafkaTestUtils.zkAddress, "group.id" -> groupId, "auto.offset.reset" -> "smallest" ) - ssc = new StreamingContext(sparkConf, Milliseconds(500)) tempDirectory = Utils.createTempDir() + } + + override def afterAll(): Unit = { + Utils.deleteRecursively(tempDirectory) + + if (kafkaTestUtils != null) { + kafkaTestUtils.teardown() + kafkaTestUtils = null + } + } + + before { + ssc = new StreamingContext(sparkConf, Milliseconds(500)) ssc.checkpoint(tempDirectory.getAbsolutePath) } after { if (ssc != null) { ssc.stop() + ssc = null } - Utils.deleteRecursively(tempDirectory) - tearDownKafka() } - test("Reliable Kafka input stream with single topic") { - var topic = "test-topic" - createTopic(topic) - sendMessages(topic, data) + val topic = "test-topic" + kafkaTestUtils.createTopic(topic) + kafkaTestUtils.sendMessages(topic, data) // Verify whether the offset of this group/topic/partition is 0 before starting. assert(getCommitOffset(groupId, topic, 0) === None) @@ -91,6 +104,7 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter } } ssc.start() + eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { // A basic process verification for ReliableKafkaReceiver. // Verify whether received message number is equal to the sent message number. @@ -100,14 +114,13 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter // Verify the offset number whether it is equal to the total message number. assert(getCommitOffset(groupId, topic, 0) === Some(29L)) } - ssc.stop() } test("Reliable Kafka input stream with multiple topics") { val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) topics.foreach { case (t, _) => - createTopic(t) - sendMessages(t, data) + kafkaTestUtils.createTopic(t) + kafkaTestUtils.sendMessages(t, data) } // Before started, verify all the group/topic/partition offsets are 0. @@ -118,19 +131,18 @@ class ReliableKafkaStreamSuite extends KafkaStreamSuiteBase with BeforeAndAfter ssc, kafkaParams, topics, StorageLevel.MEMORY_ONLY) stream.foreachRDD(_ => Unit) ssc.start() + eventually(timeout(20000 milliseconds), interval(100 milliseconds)) { // Verify the offset for each group/topic to see whether they are equal to the expected one. topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === Some(29L)) } } - ssc.stop() } /** Getting partition offset from Zookeeper. */ private def getCommitOffset(groupId: String, topic: String, partition: Int): Option[Long] = { - assert(zkClient != null, "Zookeeper client is not initialized") val topicDirs = new ZKGroupTopicDirs(groupId, topic) val zkPath = s"${topicDirs.consumerOffsetDir}/$partition" - ZkUtils.readDataMaybeNull(zkClient, zkPath)._1.map(_.toLong) + ZkUtils.readDataMaybeNull(kafkaTestUtils.zookeeperClient, zkPath)._1.map(_.toLong) } } diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 608f8e26473a6..9b4635e49020b 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -23,13 +23,16 @@ import tempfile import struct +from py4j.java_collections import MapConverter + from pyspark.context import SparkConf, SparkContext, RDD from pyspark.streaming.context import StreamingContext +from pyspark.streaming.kafka import KafkaUtils class PySparkStreamingTestCase(unittest.TestCase): - timeout = 10 # seconds + timeout = 20 # seconds duration = 1 def setUp(self): @@ -556,5 +559,43 @@ def check_output(n): check_output(3) +class KafkaStreamTests(PySparkStreamingTestCase): + + def setUp(self): + super(KafkaStreamTests, self).setUp() + + kafkaTestUtilsClz = self.ssc._jvm.java.lang.Thread.currentThread().getContextClassLoader()\ + .loadClass("org.apache.spark.streaming.kafka.KafkaTestUtils") + self._kafkaTestUtils = kafkaTestUtilsClz.newInstance() + self._kafkaTestUtils.setup() + + def tearDown(self): + if self._kafkaTestUtils is not None: + self._kafkaTestUtils.teardown() + self._kafkaTestUtils = None + + super(KafkaStreamTests, self).tearDown() + + def test_kafka_stream(self): + """Test the Python Kafka stream API.""" + topic = "topic1" + sendData = {"a": 3, "b": 5, "c": 10} + jSendData = MapConverter().convert(sendData, + self.ssc.sparkContext._gateway._gateway_client) + + self._kafkaTestUtils.createTopic(topic) + self._kafkaTestUtils.sendMessages(topic, jSendData) + + stream = KafkaUtils.createStream(self.ssc, self._kafkaTestUtils.zkAddress(), + "test-streaming-consumer", {topic: 1}, + {"auto.offset.reset": "smallest"}) + + result = {} + for i in chain.from_iterable(self._collect(stream.map(lambda x: x[1]), + sum(sendData.values()))): + result[i] = result.get(i, 0) + 1 + + self.assertEqual(sendData, result) + if __name__ == "__main__": unittest.main() diff --git a/python/run-tests b/python/run-tests index f569a56fb7a9a..f3a07d8aba562 100755 --- a/python/run-tests +++ b/python/run-tests @@ -21,6 +21,8 @@ # Figure out where the Spark framework is installed FWDIR="$(cd "`dirname "$0"`"; cd ../; pwd)" +. "$FWDIR"/bin/load-spark-env.sh + # CD into the python directory to find things on the right path cd "$FWDIR/python" @@ -57,7 +59,7 @@ function run_core_tests() { PYSPARK_DOC_TEST=1 run_test "pyspark/broadcast.py" PYSPARK_DOC_TEST=1 run_test "pyspark/accumulators.py" run_test "pyspark/serializers.py" - run_test "pyspark/profiler.py" + run_test "pyspark/profiler.py" run_test "pyspark/shuffle.py" run_test "pyspark/tests.py" } @@ -97,6 +99,21 @@ function run_ml_tests() { function run_streaming_tests() { echo "Run streaming tests ..." + + KAFKA_ASSEMBLY_DIR="$FWDIR"/external/kafka-assembly + JAR_PATH="${KAFKA_ASSEMBLY_DIR}/target/scala-${SPARK_SCALA_VERSION}" + for f in "${JAR_PATH}"/spark-streaming-kafka-assembly-*.jar; do + if [[ ! -e "$f" ]]; then + echo "Failed to find Spark Streaming Kafka assembly jar in $KAFKA_ASSEMBLY_DIR" 1>&2 + echo "You need to build Spark with " \ + "'build/sbt assembly/assembly streaming-kafka-assembly/assembly' or" \ + "'build/mvn package' before running this program" 1>&2 + exit 1 + fi + KAFKA_ASSEMBLY_JAR="$f" + done + + export PYSPARK_SUBMIT_ARGS="--jars ${KAFKA_ASSEMBLY_JAR} pyspark-shell" run_test "pyspark/streaming/util.py" run_test "pyspark/streaming/tests.py" } From 18ca089bed41ce3e87deeb14206317863518c12c Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 10 Apr 2015 01:51:42 -0700 Subject: [PATCH 023/128] [SPARK-6766][Streaming] Fix issue about StreamingListenerBatchSubmitted and StreamingListenerBatchStarted This PR includes: 1. Send `StreamingListenerBatchSubmitted` when `JobSet` is submitted 1. Fix `StreamingListenerBatchStarted.batchInfo.processingStartTime` 1. Fix a type: `completedaBatchInfos` -> `completedBatchInfos` Author: zsxwing Closes #5414 from zsxwing/SPARK-6766 and squashes the following commits: 2f85060 [zsxwing] Update tests ca0955b [zsxwing] Combine unit tests 79b4fed [zsxwing] Add StreamingJobProgressListenerSuite to test StreamingJobProgressListener fc3a2a1 [zsxwing] Add unit tests for SPARK-6766 74aed99 [zsxwing] Refactor as per TD's suggestion 493f978 [zsxwing] Send StreamingListenerBatchSubmitted when JobSet is submitted; fix StreamingListenerBatchStarted.batchInfo.processingStartTime; fix a typo --- .../streaming/scheduler/JobScheduler.scala | 8 +- .../ui/StreamingJobProgressListener.scala | 16 +-- .../streaming/StreamingListenerSuite.scala | 55 ++++++-- .../StreamingJobProgressListenerSuite.scala | 119 ++++++++++++++++++ 4 files changed, 180 insertions(+), 18 deletions(-) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index d6a93acbe711b..95f1857b4c377 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -105,6 +105,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { if (jobSet.jobs.isEmpty) { logInfo("No jobs added for time " + jobSet.time) } else { + listenerBus.post(StreamingListenerBatchSubmitted(jobSet.toBatchInfo)) jobSets.put(jobSet.time, jobSet) jobSet.jobs.foreach(job => jobExecutor.execute(new JobHandler(job))) logInfo("Added jobs for time " + jobSet.time) @@ -134,10 +135,13 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { private def handleJobStart(job: Job) { val jobSet = jobSets.get(job.time) - if (!jobSet.hasStarted) { + val isFirstJobOfJobSet = !jobSet.hasStarted + jobSet.handleJobStart(job) + if (isFirstJobOfJobSet) { + // "StreamingListenerBatchStarted" should be posted after calling "handleJobStart" to get the + // correct "jobSet.processingStartTime". listenerBus.post(StreamingListenerBatchStarted(jobSet.toBatchInfo)) } - jobSet.handleJobStart(job) logInfo("Starting job " + job.id + " from job set of time " + jobSet.time) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index e4bd067cacb77..84f80e638f638 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -33,7 +33,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) private val waitingBatchInfos = new HashMap[Time, BatchInfo] private val runningBatchInfos = new HashMap[Time, BatchInfo] - private val completedaBatchInfos = new Queue[BatchInfo] + private val completedBatchInfos = new Queue[BatchInfo] private val batchInfoLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) private var totalCompletedBatches = 0L private var totalReceivedRecords = 0L @@ -62,7 +62,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { synchronized { - runningBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo + waitingBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo } } @@ -79,8 +79,8 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) synchronized { waitingBatchInfos.remove(batchCompleted.batchInfo.batchTime) runningBatchInfos.remove(batchCompleted.batchInfo.batchTime) - completedaBatchInfos.enqueue(batchCompleted.batchInfo) - if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() + completedBatchInfos.enqueue(batchCompleted.batchInfo) + if (completedBatchInfos.size > batchInfoLimit) completedBatchInfos.dequeue() totalCompletedBatches += 1L batchCompleted.batchInfo.receivedBlockInfo.foreach { case (_, infos) => @@ -118,7 +118,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } def retainedCompletedBatches: Seq[BatchInfo] = synchronized { - completedaBatchInfos.toSeq + completedBatchInfos.toSeq } def processingDelayDistribution: Option[Distribution] = synchronized { @@ -165,7 +165,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } def lastCompletedBatch: Option[BatchInfo] = { - completedaBatchInfos.sortBy(_.batchTime)(Time.ordering).lastOption + completedBatchInfos.sortBy(_.batchTime)(Time.ordering).lastOption } def lastReceivedBatch: Option[BatchInfo] = { @@ -174,10 +174,10 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) private def retainedBatches: Seq[BatchInfo] = synchronized { (waitingBatchInfos.values.toSeq ++ - runningBatchInfos.values.toSeq ++ completedaBatchInfos).sortBy(_.batchTime)(Time.ordering) + runningBatchInfos.values.toSeq ++ completedBatchInfos).sortBy(_.batchTime)(Time.ordering) } private def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { - Distribution(completedaBatchInfos.flatMap(getMetric(_)).map(_.toDouble)) + Distribution(completedBatchInfos.flatMap(getMetric(_)).map(_.toDouble)) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 852e8bb71d4f6..7210439509541 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -46,10 +46,38 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { val collector = new BatchInfoCollector ssc.addStreamingListener(collector) runStreams(ssc, input.size, input.size) - val batchInfos = collector.batchInfos - batchInfos should have size 4 - batchInfos.foreach(info => { + // SPARK-6766: batch info should be submitted + val batchInfosSubmitted = collector.batchInfosSubmitted + batchInfosSubmitted should have size 4 + + batchInfosSubmitted.foreach(info => { + info.schedulingDelay should be (None) + info.processingDelay should be (None) + info.totalDelay should be (None) + }) + + isInIncreasingOrder(batchInfosSubmitted.map(_.submissionTime)) should be (true) + + // SPARK-6766: processingStartTime of batch info should not be None when starting + val batchInfosStarted = collector.batchInfosStarted + batchInfosStarted should have size 4 + + batchInfosStarted.foreach(info => { + info.schedulingDelay should not be None + info.schedulingDelay.get should be >= 0L + info.processingDelay should be (None) + info.totalDelay should be (None) + }) + + isInIncreasingOrder(batchInfosStarted.map(_.submissionTime)) should be (true) + isInIncreasingOrder(batchInfosStarted.map(_.processingStartTime.get)) should be (true) + + // test onBatchCompleted + val batchInfosCompleted = collector.batchInfosCompleted + batchInfosCompleted should have size 4 + + batchInfosCompleted.foreach(info => { info.schedulingDelay should not be None info.processingDelay should not be None info.totalDelay should not be None @@ -58,9 +86,9 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { info.totalDelay.get should be >= 0L }) - isInIncreasingOrder(batchInfos.map(_.submissionTime)) should be (true) - isInIncreasingOrder(batchInfos.map(_.processingStartTime.get)) should be (true) - isInIncreasingOrder(batchInfos.map(_.processingEndTime.get)) should be (true) + isInIncreasingOrder(batchInfosCompleted.map(_.submissionTime)) should be (true) + isInIncreasingOrder(batchInfosCompleted.map(_.processingStartTime.get)) should be (true) + isInIncreasingOrder(batchInfosCompleted.map(_.processingEndTime.get)) should be (true) } test("receiver info reporting") { @@ -99,9 +127,20 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { /** Listener that collects information on processed batches */ class BatchInfoCollector extends StreamingListener { - val batchInfos = new ArrayBuffer[BatchInfo] + val batchInfosCompleted = new ArrayBuffer[BatchInfo] + val batchInfosStarted = new ArrayBuffer[BatchInfo] + val batchInfosSubmitted = new ArrayBuffer[BatchInfo] + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { + batchInfosSubmitted += batchSubmitted.batchInfo + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { + batchInfosStarted += batchStarted.batchInfo + } + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { - batchInfos += batchCompleted.batchInfo + batchInfosCompleted += batchCompleted.batchInfo } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala new file mode 100644 index 0000000000000..2b9d164500b72 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala @@ -0,0 +1,119 @@ +/* + * 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.streaming.ui + +import org.scalatest.Matchers + +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.streaming.{Time, Milliseconds, TestSuiteBase} + +class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { + + val input = (1 to 4).map(Seq(_)).toSeq + val operation = (d: DStream[Int]) => d.map(x => x) + + override def batchDuration = Milliseconds(100) + + test("onBatchSubmitted, onBatchStarted, onBatchCompleted, " + + "onReceiverStarted, onReceiverError, onReceiverStopped") { + val ssc = setupStreams(input, operation) + val listener = new StreamingJobProgressListener(ssc) + + val receivedBlockInfo = Map( + 0 -> Array(ReceivedBlockInfo(0, 100, null), ReceivedBlockInfo(0, 200, null)), + 1 -> Array(ReceivedBlockInfo(1, 300, null)) + ) + + // onBatchSubmitted + val batchInfoSubmitted = BatchInfo(Time(1000), receivedBlockInfo, 1000, None, None) + listener.onBatchSubmitted(StreamingListenerBatchSubmitted(batchInfoSubmitted)) + listener.waitingBatches should be (List(batchInfoSubmitted)) + listener.runningBatches should be (Nil) + listener.retainedCompletedBatches should be (Nil) + listener.lastCompletedBatch should be (None) + listener.numUnprocessedBatches should be (1) + listener.numTotalCompletedBatches should be (0) + listener.numTotalProcessedRecords should be (0) + listener.numTotalReceivedRecords should be (0) + + // onBatchStarted + val batchInfoStarted = BatchInfo(Time(1000), receivedBlockInfo, 1000, Some(2000), None) + listener.onBatchStarted(StreamingListenerBatchStarted(batchInfoStarted)) + listener.waitingBatches should be (Nil) + listener.runningBatches should be (List(batchInfoStarted)) + listener.retainedCompletedBatches should be (Nil) + listener.lastCompletedBatch should be (None) + listener.numUnprocessedBatches should be (1) + listener.numTotalCompletedBatches should be (0) + listener.numTotalProcessedRecords should be (0) + listener.numTotalReceivedRecords should be (600) + + // onBatchCompleted + val batchInfoCompleted = BatchInfo(Time(1000), receivedBlockInfo, 1000, Some(2000), None) + listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) + listener.waitingBatches should be (Nil) + listener.runningBatches should be (Nil) + listener.retainedCompletedBatches should be (List(batchInfoCompleted)) + listener.lastCompletedBatch should be (Some(batchInfoCompleted)) + listener.numUnprocessedBatches should be (0) + listener.numTotalCompletedBatches should be (1) + listener.numTotalProcessedRecords should be (600) + listener.numTotalReceivedRecords should be (600) + + // onReceiverStarted + val receiverInfoStarted = ReceiverInfo(0, "test", null, true, "localhost") + listener.onReceiverStarted(StreamingListenerReceiverStarted(receiverInfoStarted)) + listener.receiverInfo(0) should be (Some(receiverInfoStarted)) + listener.receiverInfo(1) should be (None) + + // onReceiverError + val receiverInfoError = ReceiverInfo(1, "test", null, true, "localhost") + listener.onReceiverError(StreamingListenerReceiverError(receiverInfoError)) + listener.receiverInfo(0) should be (Some(receiverInfoStarted)) + listener.receiverInfo(1) should be (Some(receiverInfoError)) + listener.receiverInfo(2) should be (None) + + // onReceiverStopped + val receiverInfoStopped = ReceiverInfo(2, "test", null, true, "localhost") + listener.onReceiverStopped(StreamingListenerReceiverStopped(receiverInfoStopped)) + listener.receiverInfo(0) should be (Some(receiverInfoStarted)) + listener.receiverInfo(1) should be (Some(receiverInfoError)) + listener.receiverInfo(2) should be (Some(receiverInfoStopped)) + listener.receiverInfo(3) should be (None) + } + + test("Remove the old completed batches when exceeding the limit") { + val ssc = setupStreams(input, operation) + val limit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) + val listener = new StreamingJobProgressListener(ssc) + + val receivedBlockInfo = Map( + 0 -> Array(ReceivedBlockInfo(0, 100, null), ReceivedBlockInfo(0, 200, null)), + 1 -> Array(ReceivedBlockInfo(1, 300, null)) + ) + val batchInfoCompleted = BatchInfo(Time(1000), receivedBlockInfo, 1000, Some(2000), None) + + for(_ <- 0 until (limit + 10)) { + listener.onBatchCompleted(StreamingListenerBatchCompleted(batchInfoCompleted)) + } + + listener.retainedCompletedBatches.size should be (limit) + listener.numTotalCompletedBatches should be(limit + 10) + } +} From 9f5ed99d644949443d19c4895de6e0ece4be24d0 Mon Sep 17 00:00:00 2001 From: "June.He" Date: Fri, 10 Apr 2015 20:02:35 +0100 Subject: [PATCH 024/128] [SPARK-6773][Tests]Fix RAT checks still passed issue when download rat jar failed check -license will passed in next time when rat jar download failed. Add 2 step to fix this: 1. Clean the rat.jar if download failed. 2. Add a check logic after run rat checking. Author: June.He Closes #5421 from sisihj/rat-issue and squashes the following commits: 4958302 [June.He] delete redundant check and add quoted 66f7088 [June.He] Fix RAT checks still passed issue when download rat jar failed 7311e83 [June.He] Fix RAT checks still passed issue when download rat jar failed --- dev/check-license | 45 ++++++++++++++++++++++++--------------------- 1 file changed, 24 insertions(+), 21 deletions(-) diff --git a/dev/check-license b/dev/check-license index 39943f882b6ca..10740cfdc5242 100755 --- a/dev/check-license +++ b/dev/check-license @@ -24,29 +24,27 @@ acquire_rat_jar () { JAR="$rat_jar" - if [[ ! -f "$rat_jar" ]]; then - # Download rat launch jar if it hasn't been downloaded yet - if [ ! -f "$JAR" ]; then - # Download - printf "Attempting to fetch rat\n" - JAR_DL="${JAR}.part" - if [ $(command -v curl) ]; then - curl -L --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR" - elif [ $(command -v wget) ]; then - wget --quiet ${URL} -O "$JAR_DL" && mv "$JAR_DL" "$JAR" - else - printf "You do not have curl or wget installed, please install rat manually.\n" - exit -1 - fi - fi - - unzip -tq $JAR &> /dev/null - if [ $? -ne 0 ]; then - # We failed to download - printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" + # Download rat launch jar if it hasn't been downloaded yet + if [ ! -f "$JAR" ]; then + # Download + printf "Attempting to fetch rat\n" + JAR_DL="${JAR}.part" + if [ $(command -v curl) ]; then + curl -L --silent "${URL}" > "$JAR_DL" && mv "$JAR_DL" "$JAR" + elif [ $(command -v wget) ]; then + wget --quiet ${URL} -O "$JAR_DL" && mv "$JAR_DL" "$JAR" + else + printf "You do not have curl or wget installed, please install rat manually.\n" exit -1 fi - printf "Launching rat from ${JAR}\n" + fi + + unzip -tq "$JAR" &> /dev/null + if [ $? -ne 0 ]; then + # We failed to download + rm "$JAR" + printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" + exit -1 fi } @@ -71,6 +69,11 @@ mkdir -p "$FWDIR"/lib $java_cmd -jar "$rat_jar" -E "$FWDIR"/.rat-excludes -d "$FWDIR" > rat-results.txt +if [ $? -ne 0 ]; then + echo "RAT exited abnormally" + exit 1 +fi + ERRORS="$(cat rat-results.txt | grep -e "??")" if test ! -z "$ERRORS"; then From b9baa4cd9f6e9fc58161f79744b6b7729894d920 Mon Sep 17 00:00:00 2001 From: Volodymyr Lyubinets Date: Fri, 10 Apr 2015 12:09:54 -0700 Subject: [PATCH 025/128] [SQL] [SPARK-6794] Use kryo-based SparkSqlSerializer for GeneralHashedRelation Benchmarking results: http://pastie.org/private/1dneo1mta5zpsw6gmsoeq Author: Volodymyr Lyubinets Closes #5433 from vlyubin/joins and squashes the following commits: d70c829 [Volodymyr Lyubinets] Addressed review feedback 527eac6 [Volodymyr Lyubinets] Use kryo-based SparkSqlSerializer for GeneralHashedRelation --- .../sql/execution/SparkSqlSerializer.scala | 3 +- .../sql/execution/joins/HashedRelation.scala | 47 +++++++++++++++++-- 2 files changed, 44 insertions(+), 6 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 967bd76b302d8..347e2f4a1a1af 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -26,14 +26,13 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLog import com.esotericsoftware.kryo.io.{Input, Output} import com.esotericsoftware.kryo.{Serializer, Kryo} -import com.twitter.chill.{AllScalaRegistrar, ResourcePool} +import com.twitter.chill.ResourcePool import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.{SerializerInstance, KryoSerializer} import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.util.collection.OpenHashSet import org.apache.spark.util.MutablePair -import org.apache.spark.util.Utils import org.apache.spark.sql.catalyst.expressions.codegen.{IntegerHashSet, LongHashSet} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 2fa1cf5add3b5..ab84c123e0c0b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -17,9 +17,11 @@ package org.apache.spark.sql.execution.joins +import java.io.{ObjectInput, ObjectOutput, Externalizable} import java.util.{HashMap => JavaHashMap} import org.apache.spark.sql.catalyst.expressions.{Projection, Row} +import org.apache.spark.sql.execution.SparkSqlSerializer import org.apache.spark.util.collection.CompactBuffer @@ -29,16 +31,43 @@ import org.apache.spark.util.collection.CompactBuffer */ private[joins] sealed trait HashedRelation { def get(key: Row): CompactBuffer[Row] + + // This is a helper method to implement Externalizable, and is used by + // GeneralHashedRelation and UniqueKeyHashedRelation + protected def writeBytes(out: ObjectOutput, serialized: Array[Byte]): Unit = { + out.writeInt(serialized.length) // Write the length of serialized bytes first + out.write(serialized) + } + + // This is a helper method to implement Externalizable, and is used by + // GeneralHashedRelation and UniqueKeyHashedRelation + protected def readBytes(in: ObjectInput): Array[Byte] = { + val serializedSize = in.readInt() // Read the length of serialized bytes first + val bytes = new Array[Byte](serializedSize) + in.readFully(bytes) + bytes + } } /** * A general [[HashedRelation]] backed by a hash map that maps the key into a sequence of values. */ -private[joins] final class GeneralHashedRelation(hashTable: JavaHashMap[Row, CompactBuffer[Row]]) - extends HashedRelation with Serializable { +private[joins] final class GeneralHashedRelation( + private var hashTable: JavaHashMap[Row, CompactBuffer[Row]]) + extends HashedRelation with Externalizable { + + def this() = this(null) // Needed for serialization override def get(key: Row): CompactBuffer[Row] = hashTable.get(key) + + override def writeExternal(out: ObjectOutput): Unit = { + writeBytes(out, SparkSqlSerializer.serialize(hashTable)) + } + + override def readExternal(in: ObjectInput): Unit = { + hashTable = SparkSqlSerializer.deserialize(readBytes(in)) + } } @@ -46,8 +75,10 @@ private[joins] final class GeneralHashedRelation(hashTable: JavaHashMap[Row, Com * A specialized [[HashedRelation]] that maps key into a single value. This implementation * assumes the key is unique. */ -private[joins] final class UniqueKeyHashedRelation(hashTable: JavaHashMap[Row, Row]) - extends HashedRelation with Serializable { +private[joins] final class UniqueKeyHashedRelation(private var hashTable: JavaHashMap[Row, Row]) + extends HashedRelation with Externalizable { + + def this() = this(null) // Needed for serialization override def get(key: Row): CompactBuffer[Row] = { val v = hashTable.get(key) @@ -55,6 +86,14 @@ private[joins] final class UniqueKeyHashedRelation(hashTable: JavaHashMap[Row, R } def getValue(key: Row): Row = hashTable.get(key) + + override def writeExternal(out: ObjectOutput): Unit = { + writeBytes(out, SparkSqlSerializer.serialize(hashTable)) + } + + override def readExternal(in: ObjectInput): Unit = { + hashTable = SparkSqlSerializer.deserialize(readBytes(in)) + } } From 0375134f42197f2e29aa865a513cda381f0a1445 Mon Sep 17 00:00:00 2001 From: Milan Straka Date: Fri, 10 Apr 2015 13:50:32 -0700 Subject: [PATCH 026/128] [SPARK-5969][PySpark] Fix descending pyspark.rdd.sortByKey. The samples should always be sorted in ascending order, because bisect.bisect_left is used on it. The reverse order of the result is already achieved in rangePartitioner by reversing the found index. The current implementation also work, but always uses only two partitions -- the first one and the last one (because the bisect_left return returns either "beginning" or "end" for a descending sequence). Author: Milan Straka This patch had conflicts when merged, resolved by Committer: Josh Rosen Closes #4761 from foxik/fix-descending-sort and squashes the following commits: 95896b5 [Milan Straka] Add regression test for SPARK-5969. 5757490 [Milan Straka] Fix descending pyspark.rdd.sortByKey. --- python/pyspark/rdd.py | 2 +- python/pyspark/tests.py | 11 +++++++++++ 2 files changed, 12 insertions(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1b18789040360..c8e54ed5c6b2c 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -595,7 +595,7 @@ def sortPartition(iterator): maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner fraction = min(maxSampleSize / max(rddSize, 1), 1.0) samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect() - samples = sorted(samples, reverse=(not ascending), key=keyfunc) + samples = sorted(samples, key=keyfunc) # we have numPartitions many parts but one of the them has # an implicit boundary diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 0bd5d20f7877f..0e3721b55adf7 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -829,6 +829,17 @@ def test_take_on_jrdd(self): rdd = self.sc.parallelize(range(1 << 20)).map(lambda x: str(x)) rdd._jrdd.first() + def test_sortByKey_uses_all_partitions_not_only_first_and_last(self): + # Regression test for SPARK-5969 + seq = [(i * 59 % 101, i) for i in range(101)] # unsorted sequence + rdd = self.sc.parallelize(seq) + for ascending in [True, False]: + sort = rdd.sortByKey(ascending=ascending, numPartitions=5) + self.assertEqual(sort.collect(), sorted(seq, reverse=not ascending)) + sizes = sort.glom().map(len).collect() + for size in sizes: + self.assertGreater(size, 0) + class ProfilerTests(PySparkTestCase): From 4740d6a158cb4d35408a95265c5b950b9e9628a3 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 10 Apr 2015 14:04:53 -0700 Subject: [PATCH 027/128] [SPARK-6216] [PySpark] check the python version in worker Author: Davies Liu Closes #5404 from davies/check_version and squashes the following commits: e559248 [Davies Liu] add tests ec33b5f [Davies Liu] check the python version in worker --- python/pyspark/rdd.py | 2 +- python/pyspark/tests.py | 16 ++++++++++++++++ python/pyspark/worker.py | 6 +++++- 3 files changed, 22 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c8e54ed5c6b2c..c9ac95d117574 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -2233,7 +2233,7 @@ def toLocalIterator(self): def _prepare_for_python_RDD(sc, command, obj=None): # the serialized command will be compressed by broadcast ser = CloudPickleSerializer() - pickled_command = ser.dumps(command) + pickled_command = ser.dumps((command, sys.version_info[:2])) if len(pickled_command) > (1 << 20): # 1M broadcast = sc.broadcast(pickled_command) pickled_command = ser.dumps(broadcast) diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index 0e3721b55adf7..b938b9ce12395 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -35,6 +35,8 @@ import threading import hashlib +from py4j.protocol import Py4JJavaError + if sys.version_info[:2] <= (2, 6): try: import unittest2 as unittest @@ -1494,6 +1496,20 @@ def count(): self.assertTrue(not t.isAlive()) self.assertEqual(100000, rdd.count()) + def test_with_different_versions_of_python(self): + rdd = self.sc.parallelize(range(10)) + rdd.count() + version = sys.version_info + sys.version_info = (2, 0, 0) + log4j = self.sc._jvm.org.apache.log4j + old_level = log4j.LogManager.getRootLogger().getLevel() + log4j.LogManager.getRootLogger().setLevel(log4j.Level.FATAL) + try: + self.assertRaises(Py4JJavaError, lambda: rdd.count()) + finally: + sys.version_info = version + log4j.LogManager.getRootLogger().setLevel(old_level) + class SparkSubmitTests(unittest.TestCase): diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 8a93c320ec5d3..452d6fabdcc17 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -88,7 +88,11 @@ def main(infile, outfile): command = pickleSer._read_with_length(infile) if isinstance(command, Broadcast): command = pickleSer.loads(command.value) - (func, profiler, deserializer, serializer) = command + (func, profiler, deserializer, serializer), version = command + if version != sys.version_info[:2]: + raise Exception(("Python in worker has different version %s than that in " + + "driver %s, PySpark cannot run with different minor versions") % + (sys.version_info[:2], version)) init_time = time.time() def process(): From 68ecdb7f99ae30f7c04c33a47ab7f59a3836f2a4 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Fri, 10 Apr 2015 15:35:45 -0700 Subject: [PATCH 028/128] [SPARK-6850] [SparkR] use one partition when we need to compare the whole result Author: Davies Liu Closes #5460 from davies/r_test and squashes the following commits: 0a593ce [Davies Liu] use one partition when we need to compare the whole result --- R/pkg/inst/tests/test_binaryFile.R | 4 ++-- R/pkg/inst/tests/test_textFile.R | 4 ++-- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/R/pkg/inst/tests/test_binaryFile.R b/R/pkg/inst/tests/test_binaryFile.R index 4bb5f58d83dc9..ca4218f3819f8 100644 --- a/R/pkg/inst/tests/test_binaryFile.R +++ b/R/pkg/inst/tests/test_binaryFile.R @@ -27,7 +27,7 @@ test_that("saveAsObjectFile()/objectFile() following textFile() works", { fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") writeLines(mockFile, fileName1) - rdd <- textFile(sc, fileName1) + rdd <- textFile(sc, fileName1, 1) saveAsObjectFile(rdd, fileName2) rdd <- objectFile(sc, fileName2) expect_equal(collect(rdd), as.list(mockFile)) @@ -40,7 +40,7 @@ test_that("saveAsObjectFile()/objectFile() works on a parallelized list", { fileName <- tempfile(pattern="spark-test", fileext=".tmp") l <- list(1, 2, 3) - rdd <- parallelize(sc, l) + rdd <- parallelize(sc, l, 1) saveAsObjectFile(rdd, fileName) rdd <- objectFile(sc, fileName) expect_equal(collect(rdd), l) diff --git a/R/pkg/inst/tests/test_textFile.R b/R/pkg/inst/tests/test_textFile.R index 7bb3e8003131d..6b87b4b3e0b08 100644 --- a/R/pkg/inst/tests/test_textFile.R +++ b/R/pkg/inst/tests/test_textFile.R @@ -81,7 +81,7 @@ test_that("textFile() followed by a saveAsTextFile() returns the same content", fileName2 <- tempfile(pattern="spark-test", fileext=".tmp") writeLines(mockFile, fileName1) - rdd <- textFile(sc, fileName1) + rdd <- textFile(sc, fileName1, 1L) saveAsTextFile(rdd, fileName2) rdd <- textFile(sc, fileName2) expect_equal(collect(rdd), as.list(mockFile)) @@ -93,7 +93,7 @@ test_that("textFile() followed by a saveAsTextFile() returns the same content", test_that("saveAsTextFile() on a parallelized list works as expected", { fileName <- tempfile(pattern="spark-test", fileext=".tmp") l <- list(1, 2, 3) - rdd <- parallelize(sc, l) + rdd <- parallelize(sc, l, 1L) saveAsTextFile(rdd, fileName) rdd <- textFile(sc, fileName) expect_equal(collect(rdd), lapply(l, function(x) {toString(x)})) From 23d5f8864f7d665a74b1d38118700139854dbb1c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 10 Apr 2015 16:05:14 -0700 Subject: [PATCH 029/128] [SPARK-6851][SQL] Create new instance for each converted parquet relation Otherwise we end up rewriting predicates to be trivially equal (i.e. `a#1 = a#2` -> `a#3 = a#3`), at which point the query is no longer valid. Author: Michael Armbrust Closes #5458 from marmbrus/selfJoinParquet and squashes the following commits: 22df77c [Michael Armbrust] [SPARK-6851][SQL] Create new instance for each converted parquet relation --- .../spark/sql/hive/HiveMetastoreCatalog.scala | 4 +- .../sql/hive/execution/SQLQuerySuite.scala | 78 ++++++++++++++++++- 2 files changed, 80 insertions(+), 2 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 315fab673da5c..3ed5c5b031736 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 @@ -279,7 +279,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with } } - if (metastoreRelation.hiveQlTable.isPartitioned) { + val result = if (metastoreRelation.hiveQlTable.isPartitioned) { val partitionSchema = StructType.fromAttributes(metastoreRelation.partitionKeys) val partitionColumnDataTypes = partitionSchema.map(_.dataType) val partitions = metastoreRelation.hiveQlPartitions.map { p => @@ -314,6 +314,8 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with parquetRelation } + + result.newInstance() } override def getTables(databaseName: Option[String]): Seq[(String, Boolean)] = synchronized { 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 7811bd2e9effb..4c369c0634bda 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 @@ -34,6 +34,17 @@ case class Nested3(f3: Int) case class NestedArray2(b: Seq[Int]) case class NestedArray1(a: NestedArray2) +case class Order( + id: Int, + make: String, + `type`: String, + price: Int, + pdate: String, + customer: String, + city: String, + state: String, + month: Int) + /** * A collection of hive query tests where we generate the answers ourselves instead of depending on * Hive to generate them (in contrast to HiveQuerySuite). Often this is because the query is @@ -41,6 +52,72 @@ case class NestedArray1(a: NestedArray2) */ class SQLQuerySuite extends QueryTest { + test("SPARK-6851: Self-joined converted parquet tables") { + val orders = Seq( + Order(1, "Atlas", "MTB", 234, "2015-01-07", "John D", "Pacifica", "CA", 20151), + Order(3, "Swift", "MTB", 285, "2015-01-17", "John S", "Redwood City", "CA", 20151), + Order(4, "Atlas", "Hybrid", 303, "2015-01-23", "Jones S", "San Mateo", "CA", 20151), + Order(7, "Next", "MTB", 356, "2015-01-04", "Jane D", "Daly City", "CA", 20151), + Order(10, "Next", "YFlikr", 187, "2015-01-09", "John D", "Fremont", "CA", 20151), + Order(11, "Swift", "YFlikr", 187, "2015-01-23", "John D", "Hayward", "CA", 20151), + Order(2, "Next", "Hybrid", 324, "2015-02-03", "Jane D", "Daly City", "CA", 20152), + Order(5, "Next", "Street", 187, "2015-02-08", "John D", "Fremont", "CA", 20152), + Order(6, "Atlas", "Street", 154, "2015-02-09", "John D", "Pacifica", "CA", 20152), + Order(8, "Swift", "Hybrid", 485, "2015-02-19", "John S", "Redwood City", "CA", 20152), + Order(9, "Atlas", "Split", 303, "2015-02-28", "Jones S", "San Mateo", "CA", 20152)) + + val orderUpdates = Seq( + Order(1, "Atlas", "MTB", 434, "2015-01-07", "John D", "Pacifica", "CA", 20151), + Order(11, "Swift", "YFlikr", 137, "2015-01-23", "John D", "Hayward", "CA", 20151)) + + orders.toDF.registerTempTable("orders1") + orderUpdates.toDF.registerTempTable("orderupdates1") + + sql( + """CREATE TABLE orders( + | id INT, + | make String, + | type String, + | price INT, + | pdate String, + | customer String, + | city String) + |PARTITIONED BY (state STRING, month INT) + |STORED AS PARQUET + """.stripMargin) + + sql( + """CREATE TABLE orderupdates( + | id INT, + | make String, + | type String, + | price INT, + | pdate String, + | customer String, + | city String) + |PARTITIONED BY (state STRING, month INT) + |STORED AS PARQUET + """.stripMargin) + + sql("set hive.exec.dynamic.partition.mode=nonstrict") + sql("INSERT INTO TABLE orders PARTITION(state, month) SELECT * FROM orders1") + sql("INSERT INTO TABLE orderupdates PARTITION(state, month) SELECT * FROM orderupdates1") + + checkAnswer( + sql( + """ + |select orders.state, orders.month + |from orders + |join ( + | select distinct orders.state,orders.month + | from orders + | join orderupdates + | on orderupdates.id = orders.id) ao + | on ao.state = orders.state and ao.month = orders.month + """.stripMargin), + (1 to 6).map(_ => Row("CA", 20151))) + } + test("SPARK-5371: union with null and sum") { val df = Seq((1, 1)).toDF("c1", "c2") df.registerTempTable("table1") @@ -478,5 +555,4 @@ class SQLQuerySuite extends QueryTest { sql("select d from dn union all select d * 2 from dn") .queryExecution.analyzed } - } From 67d06880e47e0324409cf7e5b21db1dcb0107b82 Mon Sep 17 00:00:00 2001 From: Volodymyr Lyubinets Date: Fri, 10 Apr 2015 16:27:56 -0700 Subject: [PATCH 030/128] [SQL] [SPARK-6620] Speed up toDF() and rdd() functions by constructing converters in ScalaReflection cc marmbrus Author: Volodymyr Lyubinets Closes #5279 from vlyubin/speedup and squashes the following commits: e75a387 [Volodymyr Lyubinets] Changes to ScalaUDF 11a20ec [Volodymyr Lyubinets] Avoid creating a tuple c327bc9 [Volodymyr Lyubinets] Moved the only remaining function from DataTypeConversions to DateUtils dec6802 [Volodymyr Lyubinets] Addresed review feedback 74301fa [Volodymyr Lyubinets] Addressed review comments afa3aa5 [Volodymyr Lyubinets] Minor refactoring, added license, removed debug output 881dc60 [Volodymyr Lyubinets] Moved to a separate module; addressed review comments; one extra place of usage; changed behaviour for Java 8cad6e2 [Volodymyr Lyubinets] Addressed review commments 41b2aa9 [Volodymyr Lyubinets] Creating converters for ScalaReflection stuff, and more --- .../spark/ml/feature/TokenizerSuite.scala | 17 +- .../sql/catalyst/CatalystTypeConverters.scala | 295 +++++++ .../spark/sql/catalyst/ScalaReflection.scala | 55 -- .../sql/catalyst/expressions/ScalaUdf.scala | 819 ++++++++++++------ .../plans/logical/LocalRelation.scala | 7 +- .../spark/sql/types/DataTypeConversions.scala | 77 -- .../apache/spark/sql/types/DateUtils.scala | 29 + .../sql/catalyst/ScalaReflectionSuite.scala | 4 +- .../org/apache/spark/sql/DataFrame.scala | 11 +- .../org/apache/spark/sql/SQLContext.scala | 9 +- .../spark/sql/execution/ExistingRDD.scala | 14 +- .../spark/sql/execution/LocalTableScan.scala | 16 +- .../spark/sql/execution/SparkPlan.scala | 11 +- .../spark/sql/execution/basicOperators.scala | 9 +- .../org/apache/spark/sql/json/JsonRDD.scala | 4 +- .../apache/spark/sql/JavaDataFrameSuite.java | 10 +- .../org/apache/spark/sql/json/JsonSuite.scala | 3 +- 17 files changed, 929 insertions(+), 461 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala delete mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala index bf862b912d326..d186ead8f542f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala @@ -25,10 +25,7 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.sql.{DataFrame, Row, SQLContext} @BeanInfo -case class TokenizerTestData(rawText: String, wantedTokens: Seq[String]) { - /** Constructor used in [[org.apache.spark.ml.feature.JavaTokenizerSuite]] */ - def this(rawText: String, wantedTokens: Array[String]) = this(rawText, wantedTokens.toSeq) -} +case class TokenizerTestData(rawText: String, wantedTokens: Array[String]) class RegexTokenizerSuite extends FunSuite with MLlibTestSparkContext { import org.apache.spark.ml.feature.RegexTokenizerSuite._ @@ -46,14 +43,14 @@ class RegexTokenizerSuite extends FunSuite with MLlibTestSparkContext { .setOutputCol("tokens") val dataset0 = sqlContext.createDataFrame(Seq( - TokenizerTestData("Test for tokenization.", Seq("Test", "for", "tokenization", ".")), - TokenizerTestData("Te,st. punct", Seq("Te", ",", "st", ".", "punct")) + TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization", ".")), + TokenizerTestData("Te,st. punct", Array("Te", ",", "st", ".", "punct")) )) testRegexTokenizer(tokenizer, dataset0) val dataset1 = sqlContext.createDataFrame(Seq( - TokenizerTestData("Test for tokenization.", Seq("Test", "for", "tokenization")), - TokenizerTestData("Te,st. punct", Seq("punct")) + TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization")), + TokenizerTestData("Te,st. punct", Array("punct")) )) tokenizer.setMinTokenLength(3) @@ -64,8 +61,8 @@ class RegexTokenizerSuite extends FunSuite with MLlibTestSparkContext { .setGaps(true) .setMinTokenLength(0) val dataset2 = sqlContext.createDataFrame(Seq( - TokenizerTestData("Test for tokenization.", Seq("Test", "for", "tokenization.")), - TokenizerTestData("Te,st. punct", Seq("Te,st.", "", "punct")) + TokenizerTestData("Test for tokenization.", Array("Test", "for", "tokenization.")), + TokenizerTestData("Te,st. punct", Array("Te,st.", "", "punct")) )) testRegexTokenizer(tokenizer, dataset2) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala new file mode 100644 index 0000000000000..91976fef6dc0d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -0,0 +1,295 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst + +import java.util.{Map => JavaMap} + +import scala.collection.mutable.HashMap + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ + +/** + * Functions to convert Scala types to Catalyst types and vice versa. + */ +object CatalystTypeConverters { + // The Predef.Map is scala.collection.immutable.Map. + // Since the map values can be mutable, we explicitly import scala.collection.Map at here. + import scala.collection.Map + + /** + * Converts Scala objects to catalyst rows / types. This method is slow, and for batch + * conversion you should be using converter produced by createToCatalystConverter. + * Note: This is always called after schemaFor has been called. + * This ordering is important for UDT registration. + */ + def convertToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { + // Check UDT first since UDTs can override other types + case (obj, udt: UserDefinedType[_]) => + udt.serialize(obj) + + case (o: Option[_], _) => + o.map(convertToCatalyst(_, dataType)).orNull + + case (s: Seq[_], arrayType: ArrayType) => + s.map(convertToCatalyst(_, arrayType.elementType)) + + case (s: Array[_], arrayType: ArrayType) => + s.toSeq.map(convertToCatalyst(_, arrayType.elementType)) + + case (m: Map[_, _], mapType: MapType) => + m.map { case (k, v) => + convertToCatalyst(k, mapType.keyType) -> convertToCatalyst(v, mapType.valueType) + } + + case (jmap: JavaMap[_, _], mapType: MapType) => + val iter = jmap.entrySet.iterator + var listOfEntries: List[(Any, Any)] = List() + while (iter.hasNext) { + val entry = iter.next() + listOfEntries :+= (convertToCatalyst(entry.getKey, mapType.keyType), + convertToCatalyst(entry.getValue, mapType.valueType)) + } + listOfEntries.toMap + + case (p: Product, structType: StructType) => + val ar = new Array[Any](structType.size) + val iter = p.productIterator + var idx = 0 + while (idx < structType.size) { + ar(idx) = convertToCatalyst(iter.next(), structType.fields(idx).dataType) + idx += 1 + } + new GenericRowWithSchema(ar, structType) + + case (d: BigDecimal, _) => + Decimal(d) + + case (d: java.math.BigDecimal, _) => + Decimal(d) + + case (d: java.sql.Date, _) => + DateUtils.fromJavaDate(d) + + case (r: Row, structType: StructType) => + val converters = structType.fields.map { + f => (item: Any) => convertToCatalyst(item, f.dataType) + } + convertRowWithConverters(r, structType, converters) + + case (other, _) => + other + } + + /** + * Creates a converter function that will convert Scala objects to the specified catalyst type. + * Typical use case would be converting a collection of rows that have the same schema. You will + * call this function once to get a converter, and apply it to every row. + */ + private[sql] def createToCatalystConverter(dataType: DataType): Any => Any = { + def extractOption(item: Any): Any = item match { + case opt: Option[_] => opt.orNull + case other => other + } + + dataType match { + // Check UDT first since UDTs can override other types + case udt: UserDefinedType[_] => + (item) => extractOption(item) match { + case null => null + case other => udt.serialize(other) + } + + case arrayType: ArrayType => + val elementConverter = createToCatalystConverter(arrayType.elementType) + (item: Any) => { + extractOption(item) match { + case a: Array[_] => a.toSeq.map(elementConverter) + case s: Seq[_] => s.map(elementConverter) + case null => null + } + } + + case mapType: MapType => + val keyConverter = createToCatalystConverter(mapType.keyType) + val valueConverter = createToCatalystConverter(mapType.valueType) + (item: Any) => { + extractOption(item) match { + case m: Map[_, _] => + m.map { case (k, v) => + keyConverter(k) -> valueConverter(v) + } + + case jmap: JavaMap[_, _] => + val iter = jmap.entrySet.iterator + val convertedMap: HashMap[Any, Any] = HashMap() + while (iter.hasNext) { + val entry = iter.next() + convertedMap(keyConverter(entry.getKey)) = valueConverter(entry.getValue) + } + convertedMap + + case null => null + } + } + + case structType: StructType => + val converters = structType.fields.map(f => createToCatalystConverter(f.dataType)) + (item: Any) => { + extractOption(item) match { + case r: Row => + convertRowWithConverters(r, structType, converters) + + case p: Product => + val ar = new Array[Any](structType.size) + val iter = p.productIterator + var idx = 0 + while (idx < structType.size) { + ar(idx) = converters(idx)(iter.next()) + idx += 1 + } + new GenericRowWithSchema(ar, structType) + + case null => + null + } + } + + case dateType: DateType => (item: Any) => extractOption(item) match { + case d: java.sql.Date => DateUtils.fromJavaDate(d) + case other => other + } + + case _ => + (item: Any) => extractOption(item) match { + case d: BigDecimal => Decimal(d) + case d: java.math.BigDecimal => Decimal(d) + case other => other + } + } + } + + /** + * Converts Catalyst types used internally in rows to standard Scala types + * This method is slow, and for batch conversion you should be using converter + * produced by createToScalaConverter. + */ + def convertToScala(a: Any, dataType: DataType): Any = (a, dataType) match { + // Check UDT first since UDTs can override other types + case (d, udt: UserDefinedType[_]) => + udt.deserialize(d) + + case (s: Seq[_], arrayType: ArrayType) => + s.map(convertToScala(_, arrayType.elementType)) + + case (m: Map[_, _], mapType: MapType) => + m.map { case (k, v) => + convertToScala(k, mapType.keyType) -> convertToScala(v, mapType.valueType) + } + + case (r: Row, s: StructType) => + convertRowToScala(r, s) + + case (d: Decimal, _: DecimalType) => + d.toJavaBigDecimal + + case (i: Int, DateType) => + DateUtils.toJavaDate(i) + + case (other, _) => + other + } + + /** + * Creates a converter function that will convert Catalyst types to Scala type. + * Typical use case would be converting a collection of rows that have the same schema. You will + * call this function once to get a converter, and apply it to every row. + */ + private[sql] def createToScalaConverter(dataType: DataType): Any => Any = dataType match { + // Check UDT first since UDTs can override other types + case udt: UserDefinedType[_] => + (item: Any) => if (item == null) null else udt.deserialize(item) + + case arrayType: ArrayType => + val elementConverter = createToScalaConverter(arrayType.elementType) + (item: Any) => if (item == null) null else item.asInstanceOf[Seq[_]].map(elementConverter) + + case mapType: MapType => + val keyConverter = createToScalaConverter(mapType.keyType) + val valueConverter = createToScalaConverter(mapType.valueType) + (item: Any) => if (item == null) { + null + } else { + item.asInstanceOf[Map[_, _]].map { case (k, v) => + keyConverter(k) -> valueConverter(v) + } + } + + case s: StructType => + val converters = s.fields.map(f => createToScalaConverter(f.dataType)) + (item: Any) => { + if (item == null) { + null + } else { + convertRowWithConverters(item.asInstanceOf[Row], s, converters) + } + } + + case _: DecimalType => + (item: Any) => item match { + case d: Decimal => d.toJavaBigDecimal + case other => other + } + + case DateType => + (item: Any) => item match { + case i: Int => DateUtils.toJavaDate(i) + case other => other + } + + case other => + (item: Any) => item + } + + def convertRowToScala(r: Row, schema: StructType): Row = { + val ar = new Array[Any](r.size) + var idx = 0 + while (idx < r.size) { + ar(idx) = convertToScala(r(idx), schema.fields(idx).dataType) + idx += 1 + } + new GenericRowWithSchema(ar, schema) + } + + /** + * Converts a row by applying the provided set of converter functions. It is used for both + * toScala and toCatalyst conversions. + */ + private[sql] def convertRowWithConverters( + row: Row, + schema: StructType, + converters: Array[Any => Any]): Row = { + val ar = new Array[Any](row.size) + var idx = 0 + while (idx < row.size) { + ar(idx) = converters(idx)(row(idx)) + idx += 1 + } + new GenericRowWithSchema(ar, schema) + } +} 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 8bfd0471d9c7a..01d5c1512201a 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 @@ -46,61 +46,6 @@ trait ScalaReflection { case class Schema(dataType: DataType, nullable: Boolean) - /** - * Converts Scala objects to catalyst rows / types. - * Note: This is always called after schemaFor has been called. - * This ordering is important for UDT registration. - */ - def convertToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { - // Check UDT first since UDTs can override other types - case (obj, udt: UserDefinedType[_]) => udt.serialize(obj) - case (o: Option[_], _) => o.map(convertToCatalyst(_, dataType)).orNull - case (s: Seq[_], arrayType: ArrayType) => s.map(convertToCatalyst(_, arrayType.elementType)) - case (s: Array[_], arrayType: ArrayType) => if (arrayType.elementType.isPrimitive) { - s.toSeq - } else { - s.toSeq.map(convertToCatalyst(_, arrayType.elementType)) - } - case (m: Map[_, _], mapType: MapType) => m.map { case (k, v) => - convertToCatalyst(k, mapType.keyType) -> convertToCatalyst(v, mapType.valueType) - } - case (p: Product, structType: StructType) => - new GenericRow( - p.productIterator.toSeq.zip(structType.fields).map { case (elem, field) => - convertToCatalyst(elem, field.dataType) - }.toArray) - case (d: BigDecimal, _) => Decimal(d) - case (d: java.math.BigDecimal, _) => Decimal(d) - case (d: java.sql.Date, _) => DateUtils.fromJavaDate(d) - case (r: Row, structType: StructType) => - new GenericRow( - r.toSeq.zip(structType.fields).map { case (elem, field) => - convertToCatalyst(elem, field.dataType) - }.toArray) - case (other, _) => other - } - - /** Converts Catalyst types used internally in rows to standard Scala types */ - def convertToScala(a: Any, dataType: DataType): Any = (a, dataType) match { - // Check UDT first since UDTs can override other types - case (d, udt: UserDefinedType[_]) => udt.deserialize(d) - case (s: Seq[_], arrayType: ArrayType) => s.map(convertToScala(_, arrayType.elementType)) - case (m: Map[_, _], mapType: MapType) => m.map { case (k, v) => - convertToScala(k, mapType.keyType) -> convertToScala(v, mapType.valueType) - } - case (r: Row, s: StructType) => convertRowToScala(r, s) - case (d: Decimal, _: DecimalType) => d.toJavaBigDecimal - case (i: Int, DateType) => DateUtils.toJavaDate(i) - case (other, _) => other - } - - def convertRowToScala(r: Row, schema: StructType): Row = { - // TODO: This is very slow!!! - new GenericRowWithSchema( - r.toSeq.zip(schema.fields.map(_.dataType)) - .map(r_dt => convertToScala(r_dt._1, r_dt._2)).toArray, schema) - } - /** Returns a Sequence of attributes for the given case class type. */ def attributesFor[T: TypeTag]: Seq[Attribute] = schemaFor[T] match { case Schema(s: StructType, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala index 389dc4f745723..9a77ca624ebe2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.types.DataType /** @@ -39,12 +39,14 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi (1 to 22).map { x => val anys = (1 to x).map(x => "Any").reduce(_ + ", " + _) - val childs = (0 to x - 1).map(x => s"val child$x = children($x)").reduce(_ + "\n " + _) - val evals = (0 to x - 1).map(x => s"ScalaReflection.convertToScala(child$x.eval(input), child$x.dataType)").reduce(_ + ",\n " + _) + val childs = (0 to x - 1).map(x => s"val child$x = children($x)").reduce(_ + "\n " + _) + lazy val converters = (0 to x - 1).map(x => s"lazy val converter$x = CatalystTypeConverters.createToScalaConverter(child$x.dataType)").reduce(_ + "\n " + _) + val evals = (0 to x - 1).map(x => s"converter$x(child$x.eval(input))").reduce(_ + ",\n " + _) - s""" case $x => + s"""case $x => val func = function.asInstanceOf[($anys) => Any] $childs + $converters (input: Row) => { func( $evals) @@ -60,51 +62,61 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi (input: Row) => { func() } - + case 1 => val func = function.asInstanceOf[(Any) => Any] val child0 = children(0) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType)) + converter0(child0.eval(input))) } - + case 2 => val func = function.asInstanceOf[(Any, Any) => Any] val child0 = children(0) val child1 = children(1) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input))) } - + case 3 => val func = function.asInstanceOf[(Any, Any, Any) => Any] val child0 = children(0) val child1 = children(1) val child2 = children(2) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input))) } - + case 4 => val func = function.asInstanceOf[(Any, Any, Any, Any) => Any] val child0 = children(0) val child1 = children(1) val child2 = children(2) val child3 = children(3) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input))) } - + case 5 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -112,15 +124,20 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child2 = children(2) val child3 = children(3) val child4 = children(4) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input))) } - + case 6 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -129,16 +146,22 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child3 = children(3) val child4 = children(4) val child5 = children(5) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input))) } - + case 7 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -148,17 +171,24 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child4 = children(4) val child5 = children(5) val child6 = children(6) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input))) } - + case 8 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -169,18 +199,26 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child5 = children(5) val child6 = children(6) val child7 = children(7) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input))) } - + case 9 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -192,19 +230,28 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child6 = children(6) val child7 = children(7) val child8 = children(8) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input))) } - + case 10 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -217,20 +264,30 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child7 = children(7) val child8 = children(8) val child9 = children(9) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input))) } - + case 11 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -244,21 +301,32 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child8 = children(8) val child9 = children(9) val child10 = children(10) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input))) } - + case 12 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -273,22 +341,34 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child9 = children(9) val child10 = children(10) val child11 = children(11) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input))) } - + case 13 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -304,23 +384,36 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child10 = children(10) val child11 = children(11) val child12 = children(12) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input))) } - + case 14 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -337,24 +430,38 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child11 = children(11) val child12 = children(12) val child13 = children(13) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input))) } - + case 15 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -372,25 +479,40 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child12 = children(12) val child13 = children(13) val child14 = children(14) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input))) } - + case 16 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -409,26 +531,42 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child13 = children(13) val child14 = children(14) val child15 = children(15) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input))) } - + case 17 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -448,27 +586,44 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child14 = children(14) val child15 = children(15) val child16 = children(16) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input))) } - + case 18 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -489,28 +644,46 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child15 = children(15) val child16 = children(16) val child17 = children(17) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType), - ScalaReflection.convertToScala(child17.eval(input), child17.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input))) } - + case 19 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -532,29 +705,48 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child16 = children(16) val child17 = children(17) val child18 = children(18) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType), - ScalaReflection.convertToScala(child17.eval(input), child17.dataType), - ScalaReflection.convertToScala(child18.eval(input), child18.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input))) } - + case 20 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -577,30 +769,50 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child17 = children(17) val child18 = children(18) val child19 = children(19) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) + lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType), - ScalaReflection.convertToScala(child17.eval(input), child17.dataType), - ScalaReflection.convertToScala(child18.eval(input), child18.dataType), - ScalaReflection.convertToScala(child19.eval(input), child19.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input)), + converter19(child19.eval(input))) } - + case 21 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -624,31 +836,52 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child18 = children(18) val child19 = children(19) val child20 = children(20) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) + lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) + lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType), - ScalaReflection.convertToScala(child17.eval(input), child17.dataType), - ScalaReflection.convertToScala(child18.eval(input), child18.dataType), - ScalaReflection.convertToScala(child19.eval(input), child19.dataType), - ScalaReflection.convertToScala(child20.eval(input), child20.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input)), + converter19(child19.eval(input)), + converter20(child20.eval(input))) } - + case 22 => val func = function.asInstanceOf[(Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any, Any) => Any] val child0 = children(0) @@ -673,35 +906,57 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi val child19 = children(19) val child20 = children(20) val child21 = children(21) + lazy val converter0 = CatalystTypeConverters.createToScalaConverter(child0.dataType) + lazy val converter1 = CatalystTypeConverters.createToScalaConverter(child1.dataType) + lazy val converter2 = CatalystTypeConverters.createToScalaConverter(child2.dataType) + lazy val converter3 = CatalystTypeConverters.createToScalaConverter(child3.dataType) + lazy val converter4 = CatalystTypeConverters.createToScalaConverter(child4.dataType) + lazy val converter5 = CatalystTypeConverters.createToScalaConverter(child5.dataType) + lazy val converter6 = CatalystTypeConverters.createToScalaConverter(child6.dataType) + lazy val converter7 = CatalystTypeConverters.createToScalaConverter(child7.dataType) + lazy val converter8 = CatalystTypeConverters.createToScalaConverter(child8.dataType) + lazy val converter9 = CatalystTypeConverters.createToScalaConverter(child9.dataType) + lazy val converter10 = CatalystTypeConverters.createToScalaConverter(child10.dataType) + lazy val converter11 = CatalystTypeConverters.createToScalaConverter(child11.dataType) + lazy val converter12 = CatalystTypeConverters.createToScalaConverter(child12.dataType) + lazy val converter13 = CatalystTypeConverters.createToScalaConverter(child13.dataType) + lazy val converter14 = CatalystTypeConverters.createToScalaConverter(child14.dataType) + lazy val converter15 = CatalystTypeConverters.createToScalaConverter(child15.dataType) + lazy val converter16 = CatalystTypeConverters.createToScalaConverter(child16.dataType) + lazy val converter17 = CatalystTypeConverters.createToScalaConverter(child17.dataType) + lazy val converter18 = CatalystTypeConverters.createToScalaConverter(child18.dataType) + lazy val converter19 = CatalystTypeConverters.createToScalaConverter(child19.dataType) + lazy val converter20 = CatalystTypeConverters.createToScalaConverter(child20.dataType) + lazy val converter21 = CatalystTypeConverters.createToScalaConverter(child21.dataType) (input: Row) => { func( - ScalaReflection.convertToScala(child0.eval(input), child0.dataType), - ScalaReflection.convertToScala(child1.eval(input), child1.dataType), - ScalaReflection.convertToScala(child2.eval(input), child2.dataType), - ScalaReflection.convertToScala(child3.eval(input), child3.dataType), - ScalaReflection.convertToScala(child4.eval(input), child4.dataType), - ScalaReflection.convertToScala(child5.eval(input), child5.dataType), - ScalaReflection.convertToScala(child6.eval(input), child6.dataType), - ScalaReflection.convertToScala(child7.eval(input), child7.dataType), - ScalaReflection.convertToScala(child8.eval(input), child8.dataType), - ScalaReflection.convertToScala(child9.eval(input), child9.dataType), - ScalaReflection.convertToScala(child10.eval(input), child10.dataType), - ScalaReflection.convertToScala(child11.eval(input), child11.dataType), - ScalaReflection.convertToScala(child12.eval(input), child12.dataType), - ScalaReflection.convertToScala(child13.eval(input), child13.dataType), - ScalaReflection.convertToScala(child14.eval(input), child14.dataType), - ScalaReflection.convertToScala(child15.eval(input), child15.dataType), - ScalaReflection.convertToScala(child16.eval(input), child16.dataType), - ScalaReflection.convertToScala(child17.eval(input), child17.dataType), - ScalaReflection.convertToScala(child18.eval(input), child18.dataType), - ScalaReflection.convertToScala(child19.eval(input), child19.dataType), - ScalaReflection.convertToScala(child20.eval(input), child20.dataType), - ScalaReflection.convertToScala(child21.eval(input), child21.dataType)) + converter0(child0.eval(input)), + converter1(child1.eval(input)), + converter2(child2.eval(input)), + converter3(child3.eval(input)), + converter4(child4.eval(input)), + converter5(child5.eval(input)), + converter6(child6.eval(input)), + converter7(child7.eval(input)), + converter8(child8.eval(input)), + converter9(child9.eval(input)), + converter10(child10.eval(input)), + converter11(child11.eval(input)), + converter12(child12.eval(input)), + converter13(child13.eval(input)), + converter14(child14.eval(input)), + converter15(child15.eval(input)), + converter16(child16.eval(input)), + converter17(child17.eval(input)), + converter18(child18.eval(input)), + converter19(child19.eval(input)), + converter20(child20.eval(input)), + converter21(child21.eval(input))) } } - + // scalastyle:on - - override def eval(input: Row): Any = ScalaReflection.convertToCatalyst(f(input), dataType) + + override def eval(input: Row): Any = CatalystTypeConverters.convertToCatalyst(f(input), dataType) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala index bb79dc340553b..e3e070f0ff307 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala @@ -18,9 +18,9 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, analysis} import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.types.{DataTypeConversions, StructType, StructField} +import org.apache.spark.sql.types.{StructType, StructField} object LocalRelation { def apply(output: Attribute*): LocalRelation = new LocalRelation(output) @@ -31,7 +31,8 @@ object LocalRelation { def fromProduct(output: Seq[Attribute], data: Seq[Product]): LocalRelation = { val schema = StructType.fromAttributes(output) - LocalRelation(output, data.map(row => DataTypeConversions.productToRow(row, schema))) + val converter = CatalystTypeConverters.createToCatalystConverter(schema) + LocalRelation(output, data.map(converter(_).asInstanceOf[Row])) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala deleted file mode 100644 index a9d63e784963d..0000000000000 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeConversions.scala +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.types - -import java.text.SimpleDateFormat - -import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow - - -private[sql] object DataTypeConversions { - - def productToRow(product: Product, schema: StructType): Row = { - val mutableRow = new GenericMutableRow(product.productArity) - val schemaFields = schema.fields.toArray - - var i = 0 - while (i < mutableRow.length) { - mutableRow(i) = - ScalaReflection.convertToCatalyst(product.productElement(i), schemaFields(i).dataType) - i += 1 - } - - mutableRow - } - - def stringToTime(s: String): java.util.Date = { - if (!s.contains('T')) { - // JDBC escape string - if (s.contains(' ')) { - java.sql.Timestamp.valueOf(s) - } else { - java.sql.Date.valueOf(s) - } - } else if (s.endsWith("Z")) { - // this is zero timezone of ISO8601 - stringToTime(s.substring(0, s.length - 1) + "GMT-00:00") - } else if (s.indexOf("GMT") == -1) { - // timezone with ISO8601 - val inset = "+00.00".length - val s0 = s.substring(0, s.length - inset) - val s1 = s.substring(s.length - inset, s.length) - if (s0.substring(s0.lastIndexOf(':')).contains('.')) { - stringToTime(s0 + "GMT" + s1) - } else { - stringToTime(s0 + ".0GMT" + s1) - } - } else { - // ISO8601 with GMT insert - val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" ) - ISO8601GMT.parse(s) - } - } - - /** Converts Java objects to catalyst rows / types */ - def convertJavaToCatalyst(a: Any, dataType: DataType): Any = (a, dataType) match { - case (obj, udt: UserDefinedType[_]) => ScalaReflection.convertToCatalyst(obj, udt) // Scala type - case (d: java.math.BigDecimal, _) => Decimal(d) - case (other, _) => other - } -} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala index 8a1a3b81b3d2c..504fb05842505 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.types import java.sql.Date +import java.text.SimpleDateFormat import java.util.{Calendar, TimeZone} import org.apache.spark.sql.catalyst.expressions.Cast @@ -57,4 +58,32 @@ object DateUtils { } def toString(days: Int): String = Cast.threadLocalDateFormat.get.format(toJavaDate(days)) + + def stringToTime(s: String): java.util.Date = { + if (!s.contains('T')) { + // JDBC escape string + if (s.contains(' ')) { + java.sql.Timestamp.valueOf(s) + } else { + java.sql.Date.valueOf(s) + } + } else if (s.endsWith("Z")) { + // this is zero timezone of ISO8601 + stringToTime(s.substring(0, s.length - 1) + "GMT-00:00") + } else if (s.indexOf("GMT") == -1) { + // timezone with ISO8601 + val inset = "+00.00".length + val s0 = s.substring(0, s.length - inset) + val s1 = s.substring(s.length - inset, s.length) + if (s0.substring(s0.lastIndexOf(':')).contains('.')) { + stringToTime(s0 + "GMT" + s1) + } else { + stringToTime(s0 + ".0GMT" + s1) + } + } else { + // ISO8601 with GMT insert + val ISO8601GMT: SimpleDateFormat = new SimpleDateFormat( "yyyy-MM-dd'T'HH:mm:ss.SSSz" ) + ISO8601GMT.parse(s) + } + } } 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 eee00e3f7ea76..bbc0b661a0c0c 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 @@ -260,7 +260,7 @@ class ScalaReflectionSuite extends FunSuite { val data = PrimitiveData(1, 1, 1, 1, 1, 1, true) val convertedData = Row(1, 1.toLong, 1.toDouble, 1.toFloat, 1.toShort, 1.toByte, true) val dataType = schemaFor[PrimitiveData].dataType - assert(convertToCatalyst(data, dataType) === convertedData) + assert(CatalystTypeConverters.convertToCatalyst(data, dataType) === convertedData) } test("convert Option[Product] to catalyst") { @@ -270,7 +270,7 @@ class ScalaReflectionSuite extends FunSuite { val dataType = schemaFor[OptionalData].dataType val convertedData = Row(2, 2.toLong, 2.toDouble, 2.toFloat, 2.toShort, 2.toByte, true, Row(1, 1, 1, 1, 1, 1, true)) - assert(convertToCatalyst(data, dataType) === convertedData) + assert(CatalystTypeConverters.convertToCatalyst(data, dataType) === convertedData) } test("infer schema from case class with multiple constructors") { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 5c6016a4a2ce2..9b9adf855077a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -33,7 +33,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.python.SerDeUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.sql.catalyst.{ScalaReflection, SqlParser} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, SqlParser} import org.apache.spark.sql.catalyst.analysis.{UnresolvedRelation, ResolvedStar} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{JoinType, Inner} @@ -713,7 +713,7 @@ class DataFrame private[sql]( val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] val attributes = schema.toAttributes val rowFunction = - f.andThen(_.map(ScalaReflection.convertToCatalyst(_, schema).asInstanceOf[Row])) + f.andThen(_.map(CatalystTypeConverters.convertToCatalyst(_, schema).asInstanceOf[Row])) val generator = UserDefinedGenerator(attributes, rowFunction, input.map(_.expr)) Generate(generator, join = true, outer = false, None, logicalPlan) @@ -734,7 +734,7 @@ class DataFrame private[sql]( val dataType = ScalaReflection.schemaFor[B].dataType val attributes = AttributeReference(outputColumn, dataType)() :: Nil def rowFunction(row: Row): TraversableOnce[Row] = { - f(row(0).asInstanceOf[A]).map(o => Row(ScalaReflection.convertToCatalyst(o, dataType))) + f(row(0).asInstanceOf[A]).map(o => Row(CatalystTypeConverters.convertToCatalyst(o, dataType))) } val generator = UserDefinedGenerator(attributes, rowFunction, apply(inputColumn).expr :: Nil) @@ -961,7 +961,10 @@ class DataFrame private[sql]( lazy val rdd: RDD[Row] = { // use a local variable to make sure the map closure doesn't capture the whole DataFrame val schema = this.schema - queryExecution.executedPlan.execute().map(ScalaReflection.convertRowToScala(_, schema)) + queryExecution.executedPlan.execute().mapPartitions { rows => + val converter = CatalystTypeConverters.createToScalaConverter(schema) + rows.map(converter(_).asInstanceOf[Row]) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 39dd14e796f06..c25ef58e6f62a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -31,9 +31,9 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer} -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, OneRowRelation} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.{ScalaReflection, expressions} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection, expressions} import org.apache.spark.sql.execution.{Filter, _} import org.apache.spark.sql.jdbc.{JDBCPartition, JDBCPartitioningInfo, JDBCRelation} import org.apache.spark.sql.json._ @@ -404,7 +404,8 @@ class SQLContext(@transient val sparkContext: SparkContext) // TODO: use MutableProjection when rowRDD is another DataFrame and the applied // schema differs from the existing schema on any field data type. val catalystRows = if (needsConversion) { - rowRDD.map(ScalaReflection.convertToCatalyst(_, schema).asInstanceOf[Row]) + val converter = CatalystTypeConverters.createToCatalystConverter(schema) + rowRDD.map(converter(_).asInstanceOf[Row]) } else { rowRDD } @@ -459,7 +460,7 @@ class SQLContext(@transient val sparkContext: SparkContext) iter.map { row => new GenericRow( extractors.zip(attributeSeq).map { case (e, attr) => - DataTypeConversions.convertJavaToCatalyst(e.invoke(row), attr.dataType) + CatalystTypeConverters.convertToCatalyst(e.invoke(row), attr.dataType) }.toArray[Any] ) : Row } 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 d8955725e59b1..656bdd7212f56 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 @@ -20,14 +20,12 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SQLContext} -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow} +import org.apache.spark.sql.catalyst.expressions.{SpecificMutableRow, Attribute} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.types.StructType -import scala.collection.immutable - /** * :: DeveloperApi :: */ @@ -39,13 +37,15 @@ object RDDConversions { Iterator.empty } else { val bufferedIterator = iterator.buffered - val mutableRow = new GenericMutableRow(bufferedIterator.head.productArity) + val mutableRow = new SpecificMutableRow(schema.fields.map(_.dataType)) val schemaFields = schema.fields.toArray + val converters = schemaFields.map { + f => CatalystTypeConverters.createToCatalystConverter(f.dataType) + } bufferedIterator.map { r => var i = 0 while (i < mutableRow.length) { - mutableRow(i) = - ScalaReflection.convertToCatalyst(r.productElement(i), schemaFields(i).dataType) + mutableRow(i) = converters(i)(r.productElement(i)) i += 1 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala index 5bd699a2fa949..8a8c3a404323a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScan.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.Attribute @@ -32,9 +32,15 @@ case class LocalTableScan(output: Seq[Attribute], rows: Seq[Row]) extends LeafNo override def execute(): RDD[Row] = rdd - override def executeCollect(): Array[Row] = - rows.map(ScalaReflection.convertRowToScala(_, schema)).toArray - override def executeTake(limit: Int): Array[Row] = - rows.map(ScalaReflection.convertRowToScala(_, schema)).take(limit).toArray + override def executeCollect(): Array[Row] = { + val converter = CatalystTypeConverters.createToScalaConverter(schema) + rows.map(converter(_).asInstanceOf[Row]).toArray + } + + + override def executeTake(limit: Int): Array[Row] = { + val converter = CatalystTypeConverters.createToScalaConverter(schema) + rows.map(converter(_).asInstanceOf[Row]).take(limit).toArray + } } 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 d239637cd4b4e..fabcf6b4a0570 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 @@ -21,7 +21,7 @@ import org.apache.spark.annotation.DeveloperApi 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.{CatalystTypeConverters, trees} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.QueryPlan @@ -80,8 +80,12 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ /** * Runs this query returning the result as an array. */ + def executeCollect(): Array[Row] = { - execute().map(ScalaReflection.convertRowToScala(_, schema)).collect() + execute().mapPartitions { iter => + val converter = CatalystTypeConverters.createToScalaConverter(schema) + iter.map(converter(_).asInstanceOf[Row]) + }.collect() } /** @@ -125,7 +129,8 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ partsScanned += numPartsToTry } - buf.toArray.map(ScalaReflection.convertRowToScala(_, this.schema)) + val converter = CatalystTypeConverters.createToScalaConverter(schema) + buf.toArray.map(converter(_).asInstanceOf[Row]) } protected def newProjection( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 1f5251a20376f..6eec520abff53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -21,7 +21,7 @@ import org.apache.spark.{SparkEnv, HashPartitioner, SparkConf} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ @@ -139,9 +139,10 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) private def collectData(): Array[Row] = child.execute().map(_.copy()).takeOrdered(limit)(ord) - // TODO: Is this copying for no reason? - override def executeCollect(): Array[Row] = - collectData().map(ScalaReflection.convertRowToScala(_, this.schema)) + override def executeCollect(): Array[Row] = { + val converter = CatalystTypeConverters.createToScalaConverter(schema) + collectData().map(converter(_).asInstanceOf[Row]) + } // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 0b770f2251943..b1e8521383756 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -391,7 +391,7 @@ private[sql] object JsonRDD extends Logging { value match { // only support string as date case value: java.lang.String => - DateUtils.millisToDays(DataTypeConversions.stringToTime(value).getTime) + DateUtils.millisToDays(DateUtils.stringToTime(value).getTime) case value: java.sql.Date => DateUtils.fromJavaDate(value) } } @@ -400,7 +400,7 @@ private[sql] object JsonRDD extends Logging { value match { case value: java.lang.Integer => new Timestamp(value.asInstanceOf[Int].toLong) case value: java.lang.Long => new Timestamp(value) - case value: java.lang.String => toTimestamp(DataTypeConversions.stringToTime(value).getTime) + case value: java.lang.String => toTimestamp(DateUtils.stringToTime(value).getTime) } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java index 1ff2d5a190521..6d0fbe83c2f36 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDataFrameSuite.java @@ -20,6 +20,8 @@ import java.io.Serializable; import java.util.Arrays; +import scala.collection.Seq; + import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -127,6 +129,12 @@ public void testCreateDataFrameFromJavaBeans() { schema.apply("b")); Row first = df.select("a", "b").first(); Assert.assertEquals(bean.getA(), first.getDouble(0), 0.0); - Assert.assertArrayEquals(bean.getB(), first.getAs(1)); + // Now Java lists and maps are converetd to Scala Seq's and Map's. Once we get a Seq below, + // verify that it has the expected length, and contains expected elements. + Seq result = first.getAs(1); + Assert.assertEquals(bean.getB().length, result.length()); + for (int i = 0; i < result.length(); i++) { + Assert.assertEquals(bean.getB()[i], result.apply(i)); + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 1fe0b76c00be3..fd0e2746dc045 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -895,8 +895,7 @@ class JsonSuite extends QueryTest { ) } - test("SPARK-4228 DataFrame to JSON") - { + test("SPARK-4228 DataFrame to JSON") { val schema1 = StructType( StructField("f1", IntegerType, false) :: StructField("f2", StringType, false) :: From 95a07591b3e23782a7021ef2bbf07a67a1e4a83a Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 11 Apr 2015 13:07:41 +0100 Subject: [PATCH 031/128] [Minor][Core] Fix typo Author: Liang-Chi Hsieh Closes #5466 from viirya/fix_ShuffleMapTask_typo and squashes the following commits: 2789fd5 [Liang-Chi Hsieh] fix typo. --- .../main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index fd0d484b45460..6c7d00069acb2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -33,7 +33,7 @@ import org.apache.spark.shuffle.ShuffleWriter * See [[org.apache.spark.scheduler.Task]] for more information. * * @param stageId id of the stage this task belongs to - * @param taskBinary broadcast version of of the RDD and the ShuffleDependency. Once deserialized, + * @param taskBinary broadcast version of the RDD and the ShuffleDependency. Once deserialized, * the type should be (RDD[_], ShuffleDependency[_, _, _]). * @param partition partition of the RDD this task is associated with * @param locs preferred task execution locations for locality scheduling From 694aef0d71d2683eaf63cbd1d8e95c2da423b72e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Sat, 11 Apr 2015 13:10:01 +0100 Subject: [PATCH 032/128] [hotfix] [build] Make sure JAVA_HOME is set for tests. This is needed at least for YARN integration tests, since `$JAVA_HOME` is used to launch the executors. Author: Marcelo Vanzin Closes #5441 from vanzin/yarn-test-test and squashes the following commits: 3eeec30 [Marcelo Vanzin] Use JAVA_HOME when available, java.home otherwise. d71f1bb [Marcelo Vanzin] And sbt too. 6bda399 [Marcelo Vanzin] WIP: Testing to see whether this fixes the yarn test issue. --- pom.xml | 14 ++++++++++++++ project/SparkBuild.scala | 10 +++++++--- 2 files changed, 21 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 70e297c4f082a..d8881c213bf07 100644 --- a/pom.xml +++ b/pom.xml @@ -159,6 +159,8 @@ 1.1.1.6 1.1.2 + ${java.home} + ${test_classpath} + ${test.java.home} true @@ -1224,6 +1227,7 @@ launched by the tests have access to the correct test-time classpath. --> ${test_classpath} + ${test.java.home} true @@ -1716,6 +1720,16 @@ + + test-java-home + + env.JAVA_HOME + + + ${env.JAVA_HOME} + + + scala-2.11 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d3faa551a4b14..5f51f4b58f97a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -119,7 +119,9 @@ object SparkBuild extends PomBuild { lazy val publishLocalBoth = TaskKey[Unit]("publish-local", "publish local for m2 and ivy") lazy val sharedSettings = graphSettings ++ genjavadocSettings ++ Seq ( - javaHome := Properties.envOrNone("JAVA_HOME").map(file), + javaHome := sys.env.get("JAVA_HOME") + .orElse(sys.props.get("java.home").map { p => new File(p).getParentFile().getAbsolutePath() }) + .map(file), incOptions := incOptions.value.withNameHashing(true), retrieveManaged := true, retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", @@ -426,8 +428,10 @@ object TestSettings { fork := true, // Setting SPARK_DIST_CLASSPATH is a simple way to make sure any child processes // launched by the tests have access to the correct test-time classpath. - envVars in Test += ("SPARK_DIST_CLASSPATH" -> - (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":")), + envVars in Test ++= Map( + "SPARK_DIST_CLASSPATH" -> + (fullClasspath in Test).value.files.map(_.getAbsolutePath).mkString(":").stripSuffix(":"), + "JAVA_HOME" -> sys.env.get("JAVA_HOME").getOrElse(sys.props("java.home"))), javaOptions in Test += "-Dspark.test.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", javaOptions in Test += "-Dspark.port.maxRetries=100", From 3ceb810aa8e69bc4abb69cbe713a624cb351cb35 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Sat, 11 Apr 2015 22:11:03 +0800 Subject: [PATCH 033/128] [SPARK-6835] [SQL] Fix bug of Hive UDTF in Lateral View (ClassNotFound) ```SQL select key, v from src lateral view stack(3, 1+1, 2+2, 3) d as v; ``` Will cause exception ``` java.lang.ClassNotFoundException: stack at java.net.URLClassLoader$1.run(URLClassLoader.java:366) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) at org.apache.spark.sql.hive.HiveFunctionWrapper.createFunction(Shim13.scala:148) at org.apache.spark.sql.hive.HiveGenericUdtf.function$lzycompute(hiveUdfs.scala:274) at org.apache.spark.sql.hive.HiveGenericUdtf.function(hiveUdfs.scala:274) at org.apache.spark.sql.hive.HiveGenericUdtf.outputInspector$lzycompute(hiveUdfs.scala:280) at org.apache.spark.sql.hive.HiveGenericUdtf.outputInspector(hiveUdfs.scala:280) at org.apache.spark.sql.hive.HiveGenericUdtf.outputDataTypes$lzycompute(hiveUdfs.scala:285) at org.apache.spark.sql.hive.HiveGenericUdtf.outputDataTypes(hiveUdfs.scala:285) at org.apache.spark.sql.hive.HiveGenericUdtf.makeOutput(hiveUdfs.scala:291) at org.apache.spark.sql.catalyst.expressions.Generator.output(generators.scala:60) at org.apache.spark.sql.catalyst.plans.logical.Generate$$anonfun$2.apply(basicOperators.scala:60) at org.apache.spark.sql.catalyst.plans.logical.Generate$$anonfun$2.apply(basicOperators.scala:60) at scala.Option.map(Option.scala:145) at org.apache.spark.sql.catalyst.plans.logical.Generate.generatorOutput(basicOperators.scala:60) at org.apache.spark.sql.catalyst.plans.logical.Generate.output(basicOperators.scala:70) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveChildren$1.apply(LogicalPlan.scala:117) at org.apache.spark.sql.catalyst.plans.logical.LogicalPlan$$anonfun$resolveChildren$1.apply(LogicalPlan.scala:117) ``` Author: Cheng Hao Closes #5444 from chenghao-intel/hive_udtf and squashes the following commits: 065a98c [Cheng Hao] fix bug of Hive UDTF in Lateral View (ClassNotFound) --- .../main/scala/org/apache/spark/sql/hive/HiveQl.scala | 9 ++++++++- .../apache/spark/sql/hive/execution/SQLQuerySuite.scala | 6 ++++++ 2 files changed, 14 insertions(+), 1 deletion(-) 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 077e64133faad..0bdaf5f7ef8ef 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 @@ -20,6 +20,8 @@ package org.apache.spark.sql.hive import java.sql.Date +import org.apache.hadoop.hive.ql.exec.{FunctionRegistry, FunctionInfo} + import scala.collection.mutable.ArrayBuffer import org.apache.hadoop.hive.conf.HiveConf @@ -1284,8 +1286,13 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C Explode(attributes, nodeToExpr(child)) case Token("TOK_FUNCTION", Token(functionName, Nil) :: children) => + val functionInfo: FunctionInfo = + Option(FunctionRegistry.getFunctionInfo(functionName.toLowerCase)).getOrElse( + sys.error(s"Couldn't find function $functionName")) + val functionClassName = functionInfo.getFunctionClass.getName + HiveGenericUdtf( - new HiveFunctionWrapper(functionName), + new HiveFunctionWrapper(functionClassName), attributes, children.map(nodeToExpr)) 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 4c369c0634bda..47b4cb9ca61ff 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 @@ -51,6 +51,12 @@ case class Order( * valid, but Hive currently cannot execute it. */ class SQLQuerySuite extends QueryTest { + test("SPARK-6835: udtf in lateral view") { + val df = Seq((1, 1)).toDF("c1", "c2") + df.registerTempTable("table1") + val query = sql("SELECT c1, v FROM table1 LATERAL VIEW stack(3, 1, c1 + 1, c1 + 2) d AS v") + checkAnswer(query, Row(1, 1) :: Row(1, 2) :: Row(1, 3) :: Nil) + } test("SPARK-6851: Self-joined converted parquet tables") { val orders = Seq( From 198cf2a3fa9babb5e8b7b44da0471b63b9f3ec04 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Sat, 11 Apr 2015 14:50:50 -0700 Subject: [PATCH 034/128] [SPARK-6858][SQL] Register Java HashMap for SparkSqlSerializer Since now kyro serializer is used for `GeneralHashedRelation` whether kyro is enabled or not, it is better to register Java `HashMap` in `SparkSqlSerializer`. Author: Liang-Chi Hsieh Closes #5465 from viirya/register_hashmap and squashes the following commits: 9062601 [Liang-Chi Hsieh] Register Java HashMap for SparkSqlSerializer. --- .../org/apache/spark/sql/execution/SparkSqlSerializer.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 347e2f4a1a1af..914f387dec78f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution import java.nio.ByteBuffer +import java.util.{HashMap => JavaHashMap} import org.apache.spark.sql.types.Decimal @@ -54,6 +55,7 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co kryo.register(classOf[org.apache.spark.util.collection.OpenHashSet[_]], new OpenHashSetSerializer) kryo.register(classOf[Decimal]) + kryo.register(classOf[JavaHashMap[_, _]]) kryo.setReferences(false) kryo From 5f7b7cdab41002ad291f0a1ba7777cfff21d0019 Mon Sep 17 00:00:00 2001 From: "Santiago M. Mola" Date: Sat, 11 Apr 2015 14:52:49 -0700 Subject: [PATCH 035/128] [SPARK-6611][SQL] Add support for INTEGER as synonym of INT. https://issues.apache.org/jira/browse/SPARK-6611 Author: Santiago M. Mola Closes #5271 from smola/features/integer-parse and squashes the following commits: f5c1c64 [Santiago M. Mola] [SPARK-6611] Add support for INTEGER as synonym of INT. --- .../main/scala/org/apache/spark/sql/types/DataTypeParser.scala | 2 +- .../scala/org/apache/spark/sql/types/DataTypeParserSuite.scala | 2 ++ 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala index 34270d0ca7cd7..5163f05879e42 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataTypeParser.scala @@ -40,7 +40,7 @@ private[sql] trait DataTypeParser extends StandardTokenParsers { protected lazy val primitiveType: Parser[DataType] = "(?i)string".r ^^^ StringType | "(?i)float".r ^^^ FloatType | - "(?i)int".r ^^^ IntegerType | + "(?i)(?:int|integer)".r ^^^ IntegerType | "(?i)tinyint".r ^^^ ByteType | "(?i)smallint".r ^^^ ShortType | "(?i)double".r ^^^ DoubleType | diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala index 1ba21b64603ac..169125264a803 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeParserSuite.scala @@ -34,10 +34,12 @@ class DataTypeParserSuite extends FunSuite { } checkDataType("int", IntegerType) + checkDataType("integer", IntegerType) checkDataType("BooLean", BooleanType) checkDataType("tinYint", ByteType) checkDataType("smallINT", ShortType) checkDataType("INT", IntegerType) + checkDataType("INTEGER", IntegerType) checkDataType("bigint", LongType) checkDataType("float", FloatType) checkDataType("dOUBle", DoubleType) From 6437e7cc3bd405ebd4ad7d8f9c7a5e703652ad36 Mon Sep 17 00:00:00 2001 From: "Santiago M. Mola" Date: Sat, 11 Apr 2015 15:42:03 -0700 Subject: [PATCH 036/128] [SPARK-6863] Fix formatting on SQL programming guide. https://issues.apache.org/jira/browse/SPARK-6863 Author: Santiago M. Mola Closes #5472 from smola/fix/sql-docs and squashes the following commits: 42503d4 [Santiago M. Mola] [SPARK-6863] Fix formatting on SQL programming guide. --- docs/sql-programming-guide.md | 1 + 1 file changed, 1 insertion(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 663f656883721..332618edf0c55 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1784,6 +1784,7 @@ in Hive deployments. **Esoteric Hive Features** + * `UNION` type * Unique join * Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at From 7dbd37160ff57f80cc7abdcaef95f8c6df20a0f0 Mon Sep 17 00:00:00 2001 From: "Guancheng (G.C.) Chen" Date: Sat, 11 Apr 2015 15:43:12 -0700 Subject: [PATCH 037/128] [Minor][SQL] Fix typo in sql In this PR, "analyser" is changed to "analyzer" to keep a consistent naming. Some other typos are also fixed. Author: Guancheng (G.C.) Chen Closes #5474 from gchen/sql-typo and squashes the following commits: 70e6e76 [Guancheng (G.C.) Chen] Merge branch 'sql-typo' of github.com:gchen/spark into sql-typo fb7a6e2 [Guancheng (G.C.) Chen] fix typo in sql 37e3da1 [Guancheng (G.C.) Chen] fix type in sql --- .../org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../org/apache/spark/sql/catalyst/analysis/Catalog.scala | 4 ++-- .../apache/spark/sql/catalyst/analysis/FunctionRegistry.scala | 4 ++-- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index b3aba4f68ddf9..524c73c31bbe1 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 @@ -27,7 +27,7 @@ import org.apache.spark.sql.types._ /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing - * when all relations are already filled in and the analyser needs only to resolve attribute + * when all relations are already filled in and the analyzer needs only to resolve attribute * references. */ object SimpleAnalyzer extends Analyzer(EmptyCatalog, EmptyFunctionRegistry, true) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala index 5eb7dff0cede8..b2f8157a1a61f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} /** - * Thrown by a catalog when a table cannot be found. The analzyer will rethrow the exception + * Thrown by a catalog when a table cannot be found. The analyzer will rethrow the exception * as an AnalysisException with the correct position information. */ class NoSuchTableException extends Exception @@ -201,7 +201,7 @@ trait OverrideCatalog extends Catalog { /** * A trivial catalog that returns an error when a relation is requested. Used for testing when all - * relations are already filled in and the analyser needs only to resolve attribute references. + * relations are already filled in and the analyzer needs only to resolve attribute references. */ object EmptyCatalog extends Catalog { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala index c43ea55899695..16ca5bcd57a72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala @@ -57,8 +57,8 @@ class SimpleFunctionRegistry(val caseSensitive: Boolean) extends FunctionRegistr } /** - * A trivial catalog that returns an error when a function is requested. Used for testing when all - * functions are already filled in and the analyser needs only to resolve attribute references. + * A trivial catalog that returns an error when a function is requested. Used for testing when all + * functions are already filled in and the analyzer needs only to resolve attribute references. */ object EmptyFunctionRegistry extends FunctionRegistry { override def registerFunction(name: String, builder: FunctionBuilder): Unit = { From 2f53588738e95a2191f9844818e47f0d2ebbfd54 Mon Sep 17 00:00:00 2001 From: haiyang Date: Sat, 11 Apr 2015 18:30:17 -0700 Subject: [PATCH 038/128] [SPARK-6199] [SQL] Support CTE in HiveContext and SQLContext Author: haiyang Closes #4929 from haiyangsea/cte and squashes the following commits: 220b67d [haiyang] add golden files for cte test d3c7681 [haiyang] Merge branch 'master' into cte-repair 0ba2070 [haiyang] modify code style 9ce6b58 [haiyang] fix conflict ff74741 [haiyang] add comment for With plan 0d56af4 [haiyang] code indention 776a440 [haiyang] add comments for resolve relation strategy 2fccd7e [haiyang] add comments for resolve relation strategy 241bbe2 [haiyang] fix cte problem of view e9e1237 [haiyang] fix test case problem 614182f [haiyang] add test cases for CTE feature 32e415b [haiyang] add comment 1cc8c15 [haiyang] support with 03f1097 [haiyang] support with e960099 [haiyang] support with 9aaa874 [haiyang] support with 0566978 [haiyang] support with a99ecd2 [haiyang] support with c3fa4c2 [haiyang] support with 3b6077f [haiyang] support with 5f8abe3 [haiyang] support with 4572b05 [haiyang] support with f801f54 [haiyang] support with --- .../apache/spark/sql/catalyst/SqlParser.scala | 7 +++++ .../sql/catalyst/analysis/Analyzer.scala | 31 ++++++++++++++----- .../plans/logical/basicOperators.scala | 12 +++++++ .../org/apache/spark/sql/SQLQuerySuite.scala | 14 +++++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 27 ++++++++++++---- ...ture #1-0-eedabbfe6ba8799f7b7782fb47a82768 | 3 ++ ...ture #2-0-aa03d104251f97e36bc52279cb9931c9 | 4 +++ ...ture #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 | 1 + .../sql/hive/execution/HiveQuerySuite.scala | 15 +++++++++ 9 files changed, 100 insertions(+), 14 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/CTE feature #1-0-eedabbfe6ba8799f7b7782fb47a82768 create mode 100644 sql/hive/src/test/resources/golden/CTE feature #2-0-aa03d104251f97e36bc52279cb9931c9 create mode 100644 sql/hive/src/test/resources/golden/CTE feature #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 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 89f4a19add1c6..ee04cb579deb6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -111,6 +111,7 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { protected val UPPER = Keyword("UPPER") protected val WHEN = Keyword("WHEN") protected val WHERE = Keyword("WHERE") + protected val WITH = Keyword("WITH") protected def assignAliases(exprs: Seq[Expression]): Seq[NamedExpression] = { exprs.zipWithIndex.map { @@ -127,6 +128,7 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { | UNION ~ DISTINCT.? ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) | insert + | cte ) protected lazy val select: Parser[LogicalPlan] = @@ -156,6 +158,11 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o) } + protected lazy val cte: Parser[LogicalPlan] = + WITH ~> rep1sep(ident ~ ( AS ~ "(" ~> start <~ ")"), ",") ~ start ^^ { + case r ~ s => With(s, r.map({case n ~ s => (n, Subquery(n, s))}).toMap) + } + protected lazy val projection: Parser[Expression] = expression ~ (AS.? ~> ident.?) ^^ { case e ~ a => a.fold(e)(Alias(e, _)()) 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 524c73c31bbe1..b83f18abdd239 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 @@ -169,21 +169,36 @@ class Analyzer( * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { - def getTable(u: UnresolvedRelation): LogicalPlan = { + def getTable(u: UnresolvedRelation, cteRelations: Map[String, LogicalPlan]) = { try { - catalog.lookupRelation(u.tableIdentifier, u.alias) + // In hive, if there is same table name in database and CTE definition, + // hive will use the table in database, not the CTE one. + // Taking into account the reasonableness and the implementation complexity, + // here use the CTE definition first, check table name only and ignore database name + cteRelations.get(u.tableIdentifier.last) + .map(relation => u.alias.map(Subquery(_, relation)).getOrElse(relation)) + .getOrElse(catalog.lookupRelation(u.tableIdentifier, u.alias)) } catch { case _: NoSuchTableException => u.failAnalysis(s"no such table ${u.tableName}") } } - def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case i @ InsertIntoTable(u: UnresolvedRelation, _, _, _) => - i.copy( - table = EliminateSubQueries(getTable(u))) - case u: UnresolvedRelation => - getTable(u) + def apply(plan: LogicalPlan): LogicalPlan = { + val (realPlan, cteRelations) = plan match { + // TODO allow subquery to define CTE + // Add cte table to a temp relation map,drop `with` plan and keep its child + case With(child, relations) => (child, relations) + case other => (other, Map.empty[String, LogicalPlan]) + } + + realPlan transform { + case i@InsertIntoTable(u: UnresolvedRelation, _, _, _) => + i.copy( + table = EliminateSubQueries(getTable(u, cteRelations))) + case u: UnresolvedRelation => + getTable(u, cteRelations) + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 8633e06093cf3..3bd5aa5964221 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -147,6 +147,18 @@ case class CreateTableAsSelect[T]( override lazy val resolved: Boolean = databaseName != None && childrenResolved } +/** + * A container for holding named common table expressions (CTEs) and a query plan. + * This operator will be removed during analysis and the relations will be substituted into child. + * @param child The final query of this CTE. + * @param cteRelations Queries that this CTE defined, + * key is the alias of the CTE definition, + * value is the CTE definition. + */ +case class With(child: LogicalPlan, cteRelations: Map[String, Subquery]) extends UnaryNode { + override def output = child.output +} + case class WriteToFile( path: String, child: LogicalPlan) extends UnaryNode { 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 1392b4819131b..fb8fc6dbd1e1e 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 @@ -407,6 +407,20 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { mapData.collect().take(1).map(Row.fromTuple).toSeq) } + test("CTE feature") { + checkAnswer( + sql("with q1 as (select * from testData limit 10) select * from q1"), + testData.take(10).toSeq) + + checkAnswer( + sql(""" + |with q1 as (select * from testData where key= '5'), + |q2 as (select * from testData where key = '4') + |select * from q1 union all select * from q2""".stripMargin), + Row(5, "5") :: Row(4, "4") :: Nil) + + } + test("date row") { checkAnswer(sql( """select cast("2015-01-28" as date) from testData limit 1"""), 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 0bdaf5f7ef8ef..2fb2e7c4a5370 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 @@ -576,11 +576,23 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token("TOK_QUERY", queryArgs) if Seq("TOK_FROM", "TOK_INSERT").contains(queryArgs.head.getText) => - val (fromClause: Option[ASTNode], insertClauses) = queryArgs match { - case Token("TOK_FROM", args: Seq[ASTNode]) :: insertClauses => - (Some(args.head), insertClauses) - case Token("TOK_INSERT", _) :: Nil => (None, queryArgs) - } + val (fromClause: Option[ASTNode], insertClauses, cteRelations) = + queryArgs match { + case Token("TOK_FROM", args: Seq[ASTNode]) :: insertClauses => + // check if has CTE + insertClauses.last match { + case Token("TOK_CTE", cteClauses) => + val cteRelations = cteClauses.map(node => { + val relation = nodeToRelation(node).asInstanceOf[Subquery] + (relation.alias, relation) + }).toMap + (Some(args.head), insertClauses.init, Some(cteRelations)) + + case _ => (Some(args.head), insertClauses, None) + } + + case Token("TOK_INSERT", _) :: Nil => (None, queryArgs, None) + } // Return one query for each insert clause. val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) => @@ -794,7 +806,10 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C } // If there are multiple INSERTS just UNION them together into on query. - queries.reduceLeft(Union) + val query = queries.reduceLeft(Union) + + // return With plan if there is CTE + cteRelations.map(With(query, _)).getOrElse(query) case Token("TOK_UNION", left :: right :: Nil) => Union(nodeToPlan(left), nodeToPlan(right)) diff --git a/sql/hive/src/test/resources/golden/CTE feature #1-0-eedabbfe6ba8799f7b7782fb47a82768 b/sql/hive/src/test/resources/golden/CTE feature #1-0-eedabbfe6ba8799f7b7782fb47a82768 new file mode 100644 index 0000000000000..f6ba75da254ca --- /dev/null +++ b/sql/hive/src/test/resources/golden/CTE feature #1-0-eedabbfe6ba8799f7b7782fb47a82768 @@ -0,0 +1,3 @@ +5 +5 +5 diff --git a/sql/hive/src/test/resources/golden/CTE feature #2-0-aa03d104251f97e36bc52279cb9931c9 b/sql/hive/src/test/resources/golden/CTE feature #2-0-aa03d104251f97e36bc52279cb9931c9 new file mode 100644 index 0000000000000..ca7b591095e28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/CTE feature #2-0-aa03d104251f97e36bc52279cb9931c9 @@ -0,0 +1,4 @@ +val_4 +val_5 +val_5 +val_5 diff --git a/sql/hive/src/test/resources/golden/CTE feature #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 b/sql/hive/src/test/resources/golden/CTE feature #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 new file mode 100644 index 0000000000000..b8626c4cff284 --- /dev/null +++ b/sql/hive/src/test/resources/golden/CTE feature #3-0-b5d4bf3c0ee92b2fda0ca24f422383f2 @@ -0,0 +1 @@ +4 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index af781a502e9f3..1222fbabd8b33 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -542,6 +542,21 @@ class HiveQuerySuite extends HiveComparisonTest with BeforeAndAfter { createQueryTest("select null from table", "SELECT null FROM src LIMIT 1") + createQueryTest("CTE feature #1", + "with q1 as (select key from src) select * from q1 where key = 5") + + createQueryTest("CTE feature #2", + """with q1 as (select * from src where key= 5), + |q2 as (select * from src s2 where key = 4) + |select value from q1 union all select value from q2 + """.stripMargin) + + createQueryTest("CTE feature #3", + """with q1 as (select key from src) + |from q1 + |select * where key = 4 + """.stripMargin) + test("predicates contains an empty AttributeSet() references") { sql( """ From 1f39a61118184e136f38381a9f3ba0b2d5d589d9 Mon Sep 17 00:00:00 2001 From: lazymam500 Date: Sat, 11 Apr 2015 18:33:14 -0700 Subject: [PATCH 039/128] [Spark-5068][SQL]Fix bug query data when path doesn't exist for HiveContext This PR follow up PR #3907 & #3891 & #4356. According to marmbrus liancheng 's comments, I try to use fs.globStatus to retrieve all FileStatus objects under path(s), and then do the filtering locally. [1]. get pathPattern by path, and put it into pathPatternSet. (hdfs://cluster/user/demo/2016/08/12 -> hdfs://cluster/user/demo/*/*/*) [2]. retrieve all FileStatus objects ,and cache them by undating existPathSet. [3]. do the filtering locally [4]. if we have new pathPattern,do 1,2 step again. (external table maybe have more than one partition pathPattern) chenghao-intel jeanlyn Author: lazymam500 Author: lazyman Closes #5059 from lazyman500/SPARK-5068 and squashes the following commits: 5bfcbfd [lazyman] move spark.sql.hive.verifyPartitionPath to SQLConf,fix scala style e1d6386 [lazymam500] fix scala style f23133f [lazymam500] bug fix 47e0023 [lazymam500] fix scala style,add config flag,break the chaining 04c443c [lazyman] SPARK-5068: fix bug when partition path doesn't exists #2 41f60ce [lazymam500] Merge pull request #1 from apache/master --- .../scala/org/apache/spark/sql/SQLConf.scala | 6 ++ .../apache/spark/sql/hive/TableReader.scala | 41 +++++++++++- .../spark/sql/hive/QueryPartitionSuite.scala | 64 +++++++++++++++++++ 3 files changed, 110 insertions(+), 1 deletion(-) create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 4815620c6fe57..ee641bdfeb2d7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -39,6 +39,8 @@ private[spark] object SQLConf { val PARQUET_FILTER_PUSHDOWN_ENABLED = "spark.sql.parquet.filterPushdown" val PARQUET_USE_DATA_SOURCE_API = "spark.sql.parquet.useDataSourceApi" + val HIVE_VERIFY_PARTITIONPATH = "spark.sql.hive.verifyPartitionPath" + val COLUMN_NAME_OF_CORRUPT_RECORD = "spark.sql.columnNameOfCorruptRecord" val BROADCAST_TIMEOUT = "spark.sql.broadcastTimeout" @@ -119,6 +121,10 @@ private[sql] class SQLConf extends Serializable { private[spark] def parquetUseDataSourceApi = getConf(PARQUET_USE_DATA_SOURCE_API, "true").toBoolean + /** When true uses verifyPartitionPath to prune the path which is not exists. */ + private[spark] def verifyPartitionPath = + getConf(HIVE_VERIFY_PARTITIONPATH, "true").toBoolean + /** When true the planner will use the external sort, which may spill to disk. */ private[spark] def externalSortEnabled: Boolean = getConf(EXTERNAL_SORT, "false").toBoolean diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 3563472c7ae81..d35291543c9f9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -142,7 +142,46 @@ class HadoopTableReader( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], filterOpt: Option[PathFilter]): RDD[Row] = { - val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => + + // SPARK-5068:get FileStatus and do the filtering locally when the path is not exists + def verifyPartitionPath( + partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]]): + Map[HivePartition, Class[_ <: Deserializer]] = { + if (!sc.conf.verifyPartitionPath) { + partitionToDeserializer + } else { + var existPathSet = collection.mutable.Set[String]() + var pathPatternSet = collection.mutable.Set[String]() + partitionToDeserializer.filter { + case (partition, partDeserializer) => + def updateExistPathSetByPathPattern(pathPatternStr: String) { + val pathPattern = new Path(pathPatternStr) + val fs = pathPattern.getFileSystem(sc.hiveconf) + val matches = fs.globStatus(pathPattern) + matches.foreach(fileStatus => existPathSet += fileStatus.getPath.toString) + } + // convert /demo/data/year/month/day to /demo/data/*/*/*/ + def getPathPatternByPath(parNum: Int, tempPath: Path): String = { + var path = tempPath + for (i <- (1 to parNum)) path = path.getParent + val tails = (1 to parNum).map(_ => "*").mkString("/", "/", "/") + path.toString + tails + } + + val partPath = HiveShim.getDataLocationPath(partition) + val partNum = Utilities.getPartitionDesc(partition).getPartSpec.size(); + var pathPatternStr = getPathPatternByPath(partNum, partPath) + if (!pathPatternSet.contains(pathPatternStr)) { + pathPatternSet += pathPatternStr + updateExistPathSetByPathPattern(pathPatternStr) + } + existPathSet.contains(partPath.toString) + } + } + } + + val hivePartitionRDDs = verifyPartitionPath(partitionToDeserializer) + .map { case (partition, partDeserializer) => val partDesc = Utilities.getPartitionDesc(partition) val partPath = HiveShim.getDataLocationPath(partition) val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala new file mode 100644 index 0000000000000..83f97128c5e83 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import java.io.File + +import com.google.common.io.Files +import org.apache.spark.sql.{QueryTest, _} +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.util.Utils +/* Implicits */ +import org.apache.spark.sql.hive.test.TestHive._ + + + +class QueryPartitionSuite extends QueryTest { + import org.apache.spark.sql.hive.test.TestHive.implicits._ + + test("SPARK-5068: query data when path doesn't exists"){ + val testData = TestHive.sparkContext.parallelize( + (1 to 10).map(i => TestData(i, i.toString))).toDF() + testData.registerTempTable("testData") + + val tmpDir = Files.createTempDir() + //create the table for test + sql(s"CREATE TABLE table_with_partition(key int,value string) PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='2') SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='3') SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='4') SELECT key,value FROM testData") + + //test for the exist path + checkAnswer(sql("select key,value from table_with_partition"), + testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect + ++ testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect) + + //delect the path of one partition + val folders = tmpDir.listFiles.filter(_.isDirectory) + Utils.deleteRecursively(folders(0)) + + //test for affter delete the path + checkAnswer(sql("select key,value from table_with_partition"), + testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect + ++ testData.toSchemaRDD.collect) + + sql("DROP TABLE table_with_partition") + sql("DROP TABLE createAndInsertTest") + } +} From 48cc840021c43fcb4c5bb365d2c80512678cf120 Mon Sep 17 00:00:00 2001 From: DoingDone9 <799203320@qq.com> Date: Sat, 11 Apr 2015 18:34:17 -0700 Subject: [PATCH 040/128] [SPARK-6179][SQL] Add token for "SHOW PRINCIPALS role_name" and "SHOW TRANSACTIONS" and "SHOW COMPACTIONS" [SHOW PRINCIPALS role_name] Lists all roles and users who belong to this role. Only the admin role has privilege for this. [SHOW COMPACTIONS] It returns a list of all tables and partitions currently being compacted or scheduled for compaction when Hive transactions are being used. [SHOW TRANSACTIONS] It is for use by administrators when Hive transactions are being used. It returns a list of all currently open and aborted transactions in the system. Author: DoingDone9 <799203320@qq.com> Author: Zhongshuai Pei <799203320@qq.com> Author: Xu Tingjun Closes #4902 from DoingDone9/SHOW_PRINCIPALS and squashes the following commits: 4add42f [Zhongshuai Pei] for test 311f806 [Zhongshuai Pei] for test 0c7550a [DoingDone9] Update HiveQl.scala c8aeb1c [Xu Tingjun] aa 802261c [DoingDone9] Merge pull request #7 from apache/master d00303b [DoingDone9] Merge pull request #6 from apache/master 98b134f [DoingDone9] Merge pull request #5 from apache/master 161cae3 [DoingDone9] Merge pull request #4 from apache/master c87e8b6 [DoingDone9] Merge pull request #3 from apache/master cb1852d [DoingDone9] Merge pull request #2 from apache/master c3f046f [DoingDone9] Merge pull request #1 from apache/master --- sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala | 3 +++ 1 file changed, 3 insertions(+) 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 2fb2e7c4a5370..b2ae74efeb097 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 @@ -113,13 +113,16 @@ private[hive] object HiveQl { "TOK_REVOKE", + "TOK_SHOW_COMPACTIONS", "TOK_SHOW_CREATETABLE", "TOK_SHOW_GRANT", "TOK_SHOW_ROLE_GRANT", + "TOK_SHOW_ROLE_PRINCIPALS", "TOK_SHOW_ROLES", "TOK_SHOW_SET_ROLE", "TOK_SHOW_TABLESTATUS", "TOK_SHOW_TBLPROPERTIES", + "TOK_SHOW_TRANSACTIONS", "TOK_SHOWCOLUMNS", "TOK_SHOWDATABASES", "TOK_SHOWFUNCTIONS", From 352a5da421d61379f2a8bcd7548ccc5d2647120a Mon Sep 17 00:00:00 2001 From: Takeshi YAMAMURO Date: Sat, 11 Apr 2015 18:41:12 -0700 Subject: [PATCH 041/128] [SPARK-6379][SQL] Support a functon to call user-defined functions registered in SQLContext This is useful for using pre-defined UDFs in SQLContext; val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") val sqlctx = df.sqlContext sqlctx.udf.register("simpleUdf", (v: Int) => v * v) df.select($"id", sqlctx.callUdf("simpleUdf", $"value")) Author: Takeshi YAMAMURO Closes #5061 from maropu/SupportUDFConversionInSparkContext and squashes the following commits: f858aff [Takeshi YAMAMURO] Move the function into functions.scala afd0380 [Takeshi YAMAMURO] Add a return type of callUDF 599b76c [Takeshi YAMAMURO] Remove the implicit conversion and add SqlContext#callUdf 8b56f10 [Takeshi YAMAMURO] Support an implicit conversion from udf"name" to an UDF defined in SQLContext --- .../org/apache/spark/sql/functions.scala | 21 ++++++++++++++++++- .../org/apache/spark/sql/DataFrameSuite.scala | 9 ++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala index 111e751588a8b..ff91e1d74bc2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/functions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/functions.scala @@ -22,7 +22,7 @@ import scala.reflect.runtime.universe.{TypeTag, typeTag} import org.apache.spark.annotation.Experimental import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.analysis.Star +import org.apache.spark.sql.catalyst.analysis.{UnresolvedFunction, Star} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ @@ -605,4 +605,23 @@ object functions { } // scalastyle:on + + /** + * Call an user-defined function. + * Example: + * {{{ + * import org.apache.spark.sql._ + * + * val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") + * val sqlContext = df.sqlContext + * sqlContext.udf.register("simpleUdf", (v: Int) => v * v) + * df.select($"id", callUdf("simpleUdf", $"value")) + * }}} + * + * @group udf_funcs + */ + def callUdf(udfName: String, cols: Column*): Column = { + UnresolvedFunction(udfName, cols.map(_.expr)) + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index f5df8c6a59f10..b26e22f6229fe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -440,6 +440,15 @@ class DataFrameSuite extends QueryTest { ) } + test("call udf in SQLContext") { + val df = Seq(("id1", 1), ("id2", 4), ("id3", 5)).toDF("id", "value") + val sqlctx = df.sqlContext + sqlctx.udf.register("simpleUdf", (v: Int) => v * v) + checkAnswer( + df.select($"id", callUdf("simpleUdf", $"value")), + Row("id1", 1) :: Row("id2", 16) :: Row("id3", 25) :: Nil) + } + test("withColumn") { val df = testData.toDF().withColumn("newCol", col("key") + 1) checkAnswer( From d2383fb5ffafd6b3a56b1ee6e0e035594473e2c8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 11 Apr 2015 18:44:54 -0700 Subject: [PATCH 042/128] [SQL] Handle special characters in the authority of a Path's URI. Author: Yin Huai Closes #5381 from yhuai/parquetPath2 and squashes the following commits: fe296b4 [Yin Huai] Create new Path to take care special characters in the authority of a Path's URI. --- .../apache/spark/sql/parquet/newParquet.scala | 30 +++++++++++++++++-- 1 file changed, 27 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 0dce3623a66df..20fdf5e58ef82 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 @@ -432,7 +432,10 @@ private[sql] case class ParquetRelation2( // FileInputFormat cannot handle empty lists. if (selectedFiles.nonEmpty) { - FileInputFormat.setInputPaths(job, selectedFiles.map(_.getPath): _*) + // In order to encode the authority of a Path containning special characters such as /, + // we need to use the string retruned by the URI of the path to create a new Path. + val selectedPaths = selectedFiles.map(status => new Path(status.getPath.toUri.toString)) + FileInputFormat.setInputPaths(job, selectedPaths: _*) } // Try to push down filters when filter push-down is enabled. @@ -484,10 +487,31 @@ private[sql] case class ParquetRelation2( val cacheMetadata = useCache @transient - val cachedStatus = selectedFiles + val cachedStatus = selectedFiles.map { st => + // In order to encode the authority of a Path containning special characters such as /, + // we need to use the string retruned by the URI of the path to create a new Path. + val newPath = new Path(st.getPath.toUri.toString) + + new FileStatus( + st.getLen, + st.isDir, + st.getReplication, + st.getBlockSize, + st.getModificationTime, + st.getAccessTime, + st.getPermission, + st.getOwner, + st.getGroup, + newPath) + } @transient - val cachedFooters = selectedFooters + val cachedFooters = selectedFooters.map { f => + // In order to encode the authority of a Path containning special characters such as /, + // we need to use the string retruned by the URI of the path to create a new Path. + new Footer(new Path(f.getFile.toUri.toString), f.getParquetMetadata) + } + // Overridden so we can inject our own cached files statuses. override def getPartitions: Array[SparkPartition] = { From 6d4e854ffbd7dee9a3cd7b44a00fd9c0e551f5b8 Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Sat, 11 Apr 2015 19:26:15 -0700 Subject: [PATCH 043/128] [SPARK-6367][SQL] Use the proper data type for those expressions that are hijacking existing data types. This PR adds internal UDTs for expressions that are hijacking existing data types. The following UDTs are added: * `HyperLogLogUDT` (`BinaryType` as the SQL type) for `ApproxCountDistinctPartition` * `OpenHashSetUDT` (`ArrayType` as the SQL type) for `CollectHashSet`, `NewSet`, `AddItemToSet`, and `CombineSets`. I am also adding more unit tests for aggregation with code gen enabled. JIRA: https://issues.apache.org/jira/browse/SPARK-6367 Author: Yin Huai Closes #5094 from yhuai/expressionType and squashes the following commits: 8bcd11a [Yin Huai] Return types. 61a1d66 [Yin Huai] Merge remote-tracking branch 'upstream/master' into expressionType e8b4599 [Yin Huai] Merge remote-tracking branch 'upstream/master' into expressionType 2753156 [Yin Huai] Ignore aggregations having sum functions for now. b5eb259 [Yin Huai] Case object for HyperLogLog type. 00ebdbd [Yin Huai] deserialize/serialize. 54b87ae [Yin Huai] Add UDTs for expressions that return HyperLogLog and OpenHashSet. --- .../sql/catalyst/expressions/aggregates.scala | 24 +++++++++++-- .../expressions/codegen/CodeGenerator.scala | 4 +-- .../spark/sql/catalyst/expressions/sets.scala | 35 ++++++++++++++++--- .../sql/execution/GeneratedAggregate.scala | 12 ++++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 12 ++++--- .../spark/sql/UserDefinedTypeSuite.scala | 24 ++++++++++++- 6 files changed, 91 insertions(+), 20 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 406de38d1c483..14a855054b94d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -189,9 +189,10 @@ case class CollectHashSet(expressions: Seq[Expression]) extends AggregateExpress override def children: Seq[Expression] = expressions override def nullable: Boolean = false - override def dataType: ArrayType = ArrayType(expressions.head.dataType) + override def dataType: OpenHashSetUDT = new OpenHashSetUDT(expressions.head.dataType) override def toString: String = s"AddToHashSet(${expressions.mkString(",")})" - override def newInstance(): CollectHashSetFunction = new CollectHashSetFunction(expressions, this) + override def newInstance(): CollectHashSetFunction = + new CollectHashSetFunction(expressions, this) } case class CollectHashSetFunction( @@ -250,11 +251,28 @@ case class CombineSetsAndCountFunction( override def eval(input: Row): Any = seen.size.toLong } +/** The data type of ApproxCountDistinctPartition since its output is a HyperLogLog object. */ +private[sql] case object HyperLogLogUDT extends UserDefinedType[HyperLogLog] { + + override def sqlType: DataType = BinaryType + + /** Since we are using HyperLogLog internally, usually it will not be called. */ + override def serialize(obj: Any): Array[Byte] = + obj.asInstanceOf[HyperLogLog].getBytes + + + /** Since we are using HyperLogLog internally, usually it will not be called. */ + override def deserialize(datum: Any): HyperLogLog = + HyperLogLog.Builder.build(datum.asInstanceOf[Array[Byte]]) + + override def userClass: Class[HyperLogLog] = classOf[HyperLogLog] +} + case class ApproxCountDistinctPartition(child: Expression, relativeSD: Double) extends AggregateExpression with trees.UnaryNode[Expression] { override def nullable: Boolean = false - override def dataType: DataType = child.dataType + override def dataType: DataType = HyperLogLogUDT override def toString: String = s"APPROXIMATE COUNT(DISTINCT $child)" override def newInstance(): ApproxCountDistinctPartitionFunction = { new ApproxCountDistinctPartitionFunction(child, this, relativeSD) 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 d1abf3c0b64a5..aac56e1568332 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 @@ -464,7 +464,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val itemEval = expressionEvaluator(item) val setEval = expressionEvaluator(set) - val ArrayType(elementType, _) = set.dataType + val elementType = set.dataType.asInstanceOf[OpenHashSetUDT].elementType itemEval.code ++ setEval.code ++ q""" @@ -482,7 +482,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val leftEval = expressionEvaluator(left) val rightEval = expressionEvaluator(right) - val ArrayType(elementType, _) = left.dataType + val elementType = left.dataType.asInstanceOf[OpenHashSetUDT].elementType leftEval.code ++ rightEval.code ++ q""" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala index 35faa00782e80..4c44182278207 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/sets.scala @@ -20,6 +20,33 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.types._ import org.apache.spark.util.collection.OpenHashSet +/** The data type for expressions returning an OpenHashSet as the result. */ +private[sql] class OpenHashSetUDT( + val elementType: DataType) extends UserDefinedType[OpenHashSet[Any]] { + + override def sqlType: DataType = ArrayType(elementType) + + /** Since we are using OpenHashSet internally, usually it will not be called. */ + override def serialize(obj: Any): Seq[Any] = { + obj.asInstanceOf[OpenHashSet[Any]].iterator.toSeq + } + + /** Since we are using OpenHashSet internally, usually it will not be called. */ + override def deserialize(datum: Any): OpenHashSet[Any] = { + val iterator = datum.asInstanceOf[Seq[Any]].iterator + val set = new OpenHashSet[Any] + while(iterator.hasNext) { + set.add(iterator.next()) + } + + set + } + + override def userClass: Class[OpenHashSet[Any]] = classOf[OpenHashSet[Any]] + + private[spark] override def asNullable: OpenHashSetUDT = this +} + /** * Creates a new set of the specified type */ @@ -28,9 +55,7 @@ case class NewSet(elementType: DataType) extends LeafExpression { override def nullable: Boolean = false - // We are currently only using these Expressions internally for aggregation. However, if we ever - // expose these to users we'll want to create a proper type instead of hijacking ArrayType. - override def dataType: DataType = ArrayType(elementType) + override def dataType: OpenHashSetUDT = new OpenHashSetUDT(elementType) override def eval(input: Row): Any = { new OpenHashSet[Any]() @@ -50,7 +75,7 @@ case class AddItemToSet(item: Expression, set: Expression) extends Expression { override def nullable: Boolean = set.nullable - override def dataType: DataType = set.dataType + override def dataType: OpenHashSetUDT = set.dataType.asInstanceOf[OpenHashSetUDT] override def eval(input: Row): Any = { val itemEval = item.eval(input) @@ -80,7 +105,7 @@ case class CombineSets(left: Expression, right: Expression) extends BinaryExpres override def nullable: Boolean = left.nullable || right.nullable - override def dataType: DataType = left.dataType + override def dataType: OpenHashSetUDT = left.dataType.asInstanceOf[OpenHashSetUDT] override def symbol: String = "++=" 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 861a2c21ad9a0..3c58e93b45e9d 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 @@ -68,6 +68,8 @@ case class GeneratedAggregate( a.collect { case agg: AggregateExpression => agg} } + // If you add any new function support, please add tests in org.apache.spark.sql.SQLQuerySuite + // (in test "aggregation with codegen"). val computeFunctions = aggregatesToCompute.map { case c @ Count(expr) => // If we're evaluating UnscaledValue(x), we can do Count on x directly, since its @@ -208,7 +210,8 @@ case class GeneratedAggregate( currentMax) case CollectHashSet(Seq(expr)) => - val set = AttributeReference("hashSet", ArrayType(expr.dataType), nullable = false)() + val set = + AttributeReference("hashSet", new OpenHashSetUDT(expr.dataType), nullable = false)() val initialValue = NewSet(expr.dataType) val addToSet = AddItemToSet(expr, set) @@ -219,9 +222,10 @@ case class GeneratedAggregate( set) case CombineSetsAndCount(inputSet) => - val ArrayType(inputType, _) = inputSet.dataType - val set = AttributeReference("hashSet", inputSet.dataType, nullable = false)() - val initialValue = NewSet(inputType) + val elementType = inputSet.dataType.asInstanceOf[OpenHashSetUDT].elementType + val set = + AttributeReference("hashSet", new OpenHashSetUDT(elementType), nullable = false)() + val initialValue = NewSet(elementType) val collectSets = CombineSets(set, inputSet) AggregateEvaluation( 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 fb8fc6dbd1e1e..5e453e05e2ac7 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.test.TestSQLContext import org.scalatest.BeforeAndAfterAll @@ -151,10 +152,10 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { "SELECT count(distinct key) FROM testData3x", Row(100) :: Nil) // SUM - testCodeGen( - "SELECT value, sum(key) FROM testData3x GROUP BY value", - (1 to 100).map(i => Row(i.toString, 3 * i))) - testCodeGen( + testCodeGen( + "SELECT value, sum(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, 3 * i))) + testCodeGen( "SELECT sum(key), SUM(CAST(key as Double)) FROM testData3x", Row(5050 * 3, 5050 * 3.0) :: Nil) // AVERAGE @@ -192,10 +193,11 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { testCodeGen( "SELECT sum('a'), avg('a'), count(null) FROM testData", Row(0, null, 0) :: Nil) - + dropTempTable("testData3x") setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString) } + test("Add Parser of SQL COALESCE()") { checkAnswer( sql("""SELECT COALESCE(1, 2)"""), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index 902da5c3ba6d6..2672e20deadc5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -23,13 +23,16 @@ import org.apache.spark.util.Utils import scala.beans.{BeanInfo, BeanProperty} +import com.clearspring.analytics.stream.cardinality.HyperLogLog + import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{OpenHashSetUDT, HyperLogLogUDT} import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{sparkContext, sql} import org.apache.spark.sql.test.TestSQLContext.implicits._ import org.apache.spark.sql.types._ - +import org.apache.spark.util.collection.OpenHashSet @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { @@ -119,4 +122,23 @@ class UserDefinedTypeSuite extends QueryTest { df.limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) df.orderBy('int).limit(1).groupBy('int).agg(first('vec)).collect()(0).getAs[MyDenseVector](0) } + + test("HyperLogLogUDT") { + val hyperLogLogUDT = HyperLogLogUDT + val hyperLogLog = new HyperLogLog(0.4) + (1 to 10).foreach(i => hyperLogLog.offer(Row(i))) + + val actual = hyperLogLogUDT.deserialize(hyperLogLogUDT.serialize(hyperLogLog)) + assert(actual.cardinality() === hyperLogLog.cardinality()) + assert(java.util.Arrays.equals(actual.getBytes, hyperLogLog.getBytes)) + } + + test("OpenHashSetUDT") { + val openHashSetUDT = new OpenHashSetUDT(IntegerType) + val set = new OpenHashSet[Int] + (1 to 10).foreach(i => set.add(i)) + + val actual = openHashSetUDT.deserialize(openHashSetUDT.serialize(set)) + assert(actual.iterator.toSet === set.iterator.toSet) + } } From 5c2844c51aca6a0da9251a3fd346a6f872cf17f8 Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Sat, 11 Apr 2015 19:35:56 -0700 Subject: [PATCH 044/128] [SQL][minor] move `resolveGetField` into a object The method `resolveGetField` isn't belong to `LogicalPlan` logically and didn't access any members of it. Author: Wenchen Fan Closes #5435 from cloud-fan/tmp and squashes the following commits: 9a66c83 [Wenchen Fan] code clean up --- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../catalyst/expressions/complexTypes.scala | 37 +++++++++++++++++++ .../catalyst/plans/logical/LogicalPlan.scala | 35 +----------------- 3 files changed, 39 insertions(+), 35 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index b83f18abdd239..fd1ceb1f77931 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 @@ -308,7 +308,7 @@ class Analyzer( logDebug(s"Resolving $u to $result") result case UnresolvedGetField(child, fieldName) if child.resolved => - q.resolveGetField(child, fieldName, resolver) + GetField(child, fieldName, resolver) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala index 3b2b9211268a9..fc1f69655963d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.Map +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.types._ /** @@ -81,6 +83,41 @@ trait GetField extends UnaryExpression { def field: StructField } +object GetField { + /** + * Returns the resolved `GetField`, and report error if no desired field or over one + * desired fields are found. + */ + def apply( + expr: Expression, + fieldName: String, + resolver: Resolver): GetField = { + def findField(fields: Array[StructField]): Int = { + val checkField = (f: StructField) => resolver(f.name, fieldName) + val ordinal = fields.indexWhere(checkField) + if (ordinal == -1) { + throw new AnalysisException( + s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") + } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { + throw new AnalysisException( + s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") + } else { + ordinal + } + } + expr.dataType match { + case StructType(fields) => + val ordinal = findField(fields) + StructGetField(expr, fields(ordinal), ordinal) + case ArrayType(StructType(fields), containsNull) => + val ordinal = findField(fields) + ArrayGetField(expr, fields(ordinal), ordinal, containsNull) + case otherType => + throw new AnalysisException(s"GetField is not valid on fields of type $otherType") + } + } +} + /** * Returns the value of fields in the Struct `child`. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index d8f5858f5033e..579a0fb8d3f93 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -210,7 +210,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // For example, consider "a.b.c", where "a" is resolved to an existing attribute. // Then this will add GetField("c", GetField("b", a)), and alias // the final expression as "c". - val fieldExprs = nestedFields.foldLeft(a: Expression)(resolveGetField(_, _, resolver)) + val fieldExprs = nestedFields.foldLeft(a: Expression)(GetField(_, _, resolver)) val aliasName = nestedFields.last Some(Alias(fieldExprs, aliasName)()) } catch { @@ -229,39 +229,6 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { s"Reference '$name' is ambiguous, could be: $referenceNames.") } } - - /** - * Returns the resolved `GetField`, and report error if no desired field or over one - * desired fields are found. - */ - def resolveGetField( - expr: Expression, - fieldName: String, - resolver: Resolver): Expression = { - def findField(fields: Array[StructField]): Int = { - val checkField = (f: StructField) => resolver(f.name, fieldName) - val ordinal = fields.indexWhere(checkField) - if (ordinal == -1) { - throw new AnalysisException( - s"No such struct field $fieldName in ${fields.map(_.name).mkString(", ")}") - } else if (fields.indexWhere(checkField, ordinal + 1) != -1) { - throw new AnalysisException( - s"Ambiguous reference to fields ${fields.filter(checkField).mkString(", ")}") - } else { - ordinal - } - } - expr.dataType match { - case StructType(fields) => - val ordinal = findField(fields) - StructGetField(expr, fields(ordinal), ordinal) - case ArrayType(StructType(fields), containsNull) => - val ordinal = findField(fields) - ArrayGetField(expr, fields(ordinal), ordinal, containsNull) - case otherType => - throw new AnalysisException(s"GetField is not valid on fields of type $otherType") - } - } } /** From dea5dacc5d701c3dfe433360b2e17e50edec31e8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Sat, 11 Apr 2015 20:12:40 -0700 Subject: [PATCH 045/128] [HOTFIX] Add explicit return types to fix lint errors --- .../scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala | 2 +- .../spark/sql/catalyst/plans/logical/basicOperators.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala index fd1ceb1f77931..50702ac6832ec 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 @@ -169,7 +169,7 @@ class Analyzer( * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ object ResolveRelations extends Rule[LogicalPlan] { - def getTable(u: UnresolvedRelation, cteRelations: Map[String, LogicalPlan]) = { + def getTable(u: UnresolvedRelation, cteRelations: Map[String, LogicalPlan]): LogicalPlan = { try { // In hive, if there is same table name in database and CTE definition, // hive will use the table in database, not the CTE one. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 3bd5aa5964221..5d31a6eecfce2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -156,7 +156,7 @@ case class CreateTableAsSelect[T]( * value is the CTE definition. */ case class With(child: LogicalPlan, cteRelations: Map[String, Subquery]) extends UnaryNode { - override def output = child.output + override def output: Seq[Attribute] = child.output } case class WriteToFile( From 1205f7ea6165089985edf46ea4d2d53975d0f1f4 Mon Sep 17 00:00:00 2001 From: Michael Malak Date: Sat, 11 Apr 2015 21:01:23 -0700 Subject: [PATCH 046/128] SPARK-6710 GraphX Fixed Wrong initial bias in GraphX SVDPlusPlus Author: Michael Malak Closes #5464 from michaelmalak/master and squashes the following commits: 9d942ba [Michael Malak] SPARK-6710 GraphX Fixed Wrong initial bias in GraphX SVDPlusPlus --- .../main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index 1a7178b82e3af..3b0e1628d86b5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -93,7 +93,7 @@ object SVDPlusPlus { val gJoinT0 = g.outerJoinVertices(t0) { (vid: VertexId, vd: (Array[Double], Array[Double], Double, Double), msg: Option[(Long, Double)]) => - (vd._1, vd._2, msg.get._2 / msg.get._1, 1.0 / scala.math.sqrt(msg.get._1)) + (vd._1, vd._2, msg.get._2 / msg.get._1 - u, 1.0 / scala.math.sqrt(msg.get._1)) }.cache() materialize(gJoinT0) g.unpersist() From 0cc8fcb4cd20cb90a1fac50b1a3ffed833ce5eac Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 11 Apr 2015 22:12:56 -0700 Subject: [PATCH 047/128] MAINTENANCE: Automated closing of pull requests. This commit exists to close the following pull requests on Github: Closes #4994 (close requested by 'marmbrus') Closes #4995 (close requested by 'marmbrus') Closes #4491 (close requested by 'srowen') Closes #3597 (close requested by 'srowen') Closes #4693 (close requested by 'marmbrus') Closes #3855 (close requested by 'marmbrus') Closes #4398 (close requested by 'marmbrus') Closes #4246 (close requested by 'marmbrus') Closes #5153 (close requested by 'srowen') Closes #3626 (close requested by 'srowen') Closes #5166 (close requested by 'marmbrus') Closes #5040 (close requested by 'marmbrus') Closes #5044 (close requested by 'marmbrus') Closes #5440 (close requested by 'JoshRosen') Closes #4039 (close requested by 'marmbrus') Closes #1237 (close requested by 'srowen') Closes #216 (close requested by 'mengxr') Closes #5092 (close requested by 'srowen') Closes #5100 (close requested by 'marmbrus') Closes #4469 (close requested by 'marmbrus') Closes #5246 (close requested by 'srowen') Closes #5013 (close requested by 'marmbrus') From 5d8f7b9e87e8066d54717a1a78b06e8531d8b0d4 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Sat, 11 Apr 2015 22:33:23 -0700 Subject: [PATCH 048/128] [SPARK-6677] [SQL] [PySpark] fix cached classes It's possible to have two DataType object with same id (memory address) at different time, we should check the cached classes to verify that it's generated by given datatype. This PR also change `__FIELDS__` and `__DATATYPE__` to lower case to match Python code style. Author: Davies Liu Closes #5445 from davies/fix_type_cache and squashes the following commits: 63b3238 [Davies Liu] typo 47bdede [Davies Liu] fix cached classes --- python/pyspark/sql/types.py | 39 +++++++++++++++++++------------------ 1 file changed, 20 insertions(+), 19 deletions(-) diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/types.py index 7e0124b13671b..ef76d84c00481 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/types.py @@ -567,8 +567,8 @@ def _infer_schema(row): elif isinstance(row, (tuple, list)): if hasattr(row, "_fields"): # namedtuple items = zip(row._fields, tuple(row)) - elif hasattr(row, "__FIELDS__"): # Row - items = zip(row.__FIELDS__, tuple(row)) + elif hasattr(row, "__fields__"): # Row + items = zip(row.__fields__, tuple(row)) else: names = ['_%d' % i for i in range(1, len(row) + 1)] items = zip(names, row) @@ -647,7 +647,7 @@ def converter(obj): if isinstance(obj, dict): return tuple(c(obj.get(n)) for n, c in zip(names, converters)) elif isinstance(obj, tuple): - if hasattr(obj, "_fields") or hasattr(obj, "__FIELDS__"): + if hasattr(obj, "_fields") or hasattr(obj, "__fields__"): return tuple(c(v) for c, v in zip(converters, obj)) elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): # k-v pairs d = dict(obj) @@ -997,12 +997,13 @@ def _restore_object(dataType, obj): # same object in most cases. k = id(dataType) cls = _cached_cls.get(k) - if cls is None: + if cls is None or cls.__datatype is not dataType: # use dataType as key to avoid create multiple class cls = _cached_cls.get(dataType) if cls is None: cls = _create_cls(dataType) _cached_cls[dataType] = cls + cls.__datatype = dataType _cached_cls[k] = cls return cls(obj) @@ -1119,8 +1120,8 @@ def Dict(d): class Row(tuple): """ Row in DataFrame """ - __DATATYPE__ = dataType - __FIELDS__ = tuple(f.name for f in dataType.fields) + __datatype = dataType + __fields__ = tuple(f.name for f in dataType.fields) __slots__ = () # create property for fast access @@ -1128,22 +1129,22 @@ class Row(tuple): def asDict(self): """ Return as a dict """ - return dict((n, getattr(self, n)) for n in self.__FIELDS__) + return dict((n, getattr(self, n)) for n in self.__fields__) def __repr__(self): # call collect __repr__ for nested objects return ("Row(%s)" % ", ".join("%s=%r" % (n, getattr(self, n)) - for n in self.__FIELDS__)) + for n in self.__fields__)) def __reduce__(self): - return (_restore_object, (self.__DATATYPE__, tuple(self))) + return (_restore_object, (self.__datatype, tuple(self))) return Row def _create_row(fields, values): row = Row(*values) - row.__FIELDS__ = fields + row.__fields__ = fields return row @@ -1183,7 +1184,7 @@ def __new__(self, *args, **kwargs): # create row objects names = sorted(kwargs.keys()) row = tuple.__new__(self, [kwargs[n] for n in names]) - row.__FIELDS__ = names + row.__fields__ = names return row else: @@ -1193,11 +1194,11 @@ def asDict(self): """ Return as an dict """ - if not hasattr(self, "__FIELDS__"): + if not hasattr(self, "__fields__"): raise TypeError("Cannot convert a Row class into dict") - return dict(zip(self.__FIELDS__, self)) + return dict(zip(self.__fields__, self)) - # let obect acs like class + # let object acts like class def __call__(self, *args): """create new Row object""" return _create_row(self, args) @@ -1208,21 +1209,21 @@ def __getattr__(self, item): try: # it will be slow when it has many fields, # but this will not be used in normal cases - idx = self.__FIELDS__.index(item) + idx = self.__fields__.index(item) return self[idx] except IndexError: raise AttributeError(item) def __reduce__(self): - if hasattr(self, "__FIELDS__"): - return (_create_row, (self.__FIELDS__, tuple(self))) + if hasattr(self, "__fields__"): + return (_create_row, (self.__fields__, tuple(self))) else: return tuple.__reduce__(self) def __repr__(self): - if hasattr(self, "__FIELDS__"): + if hasattr(self, "__fields__"): return "Row(%s)" % ", ".join("%s=%r" % (k, v) - for k, v in zip(self.__FIELDS__, self)) + for k, v in zip(self.__fields__, tuple(self))) else: return "" % ", ".join(self) From e9445b187e8f5c3703771b775e60164166309570 Mon Sep 17 00:00:00 2001 From: "Guancheng (G.C.) Chen" Date: Sun, 12 Apr 2015 11:36:41 +0100 Subject: [PATCH 049/128] [SPARK-6866][Build] Remove duplicated dependency in launcher/pom.xml JIRA: https://issues.apache.org/jira/browse/SPARK-6866 Remove duplicated dependency of scalatest in launcher/pom.xml since it already inherited the dependency from the parent pom.xml. Author: Guancheng (G.C.) Chen Closes #5476 from gchen/SPARK-6866 and squashes the following commits: 1ab484b [Guancheng (G.C.) Chen] remove duplicated dependency in launcher/pom.xml --- launcher/pom.xml | 5 ----- 1 file changed, 5 deletions(-) diff --git a/launcher/pom.xml b/launcher/pom.xml index 0fe2814135d88..182e5f60218db 100644 --- a/launcher/pom.xml +++ b/launcher/pom.xml @@ -52,11 +52,6 @@ mockito-all test - - org.scalatest - scalatest_${scala.binary.version} - test - org.slf4j slf4j-api From ddc17431a4108ab6efe0cd329d69e1f2fca5ac12 Mon Sep 17 00:00:00 2001 From: lisurprise Date: Sun, 12 Apr 2015 13:41:44 +0100 Subject: [PATCH 050/128] [SPARK-6843][core]Add volatile for the "state" Fix potential visibility problem for the "state" of Executor The field of "state" is shared and modified by multiple threads. i.e: ```scala Within ExecutorRunner.scala (1) workerThread = new Thread("ExecutorRunner for " + fullId) { override def run() { fetchAndRunExecutor() } } workerThread.start() // Shutdown hook that kills actors on shutdown. (2)shutdownHook = new Thread() { override def run() { killProcess(Some("Worker shutting down")) } } (3)and also the "Actor thread" for worker. ``` I think we should at lease add volatile to ensure the visibility among threads otherwise the worker might send an out-of-date status to the master. https://issues.apache.org/jira/browse/SPARK-6843 Author: lisurprise Closes #5448 from zhichao-li/state and squashes the following commits: a2386e7 [lisurprise] add volatile for state field --- .../scala/org/apache/spark/deploy/worker/ExecutorRunner.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 83e24a7a1f80c..7d5acabb95a48 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 @@ -50,7 +50,7 @@ private[deploy] class ExecutorRunner( val workerUrl: String, conf: SparkConf, val appLocalDirs: Seq[String], - var state: ExecutorState.Value) + @volatile var state: ExecutorState.Value) extends Logging { private val fullId = appId + "/" + execId From 6ac8eea2fc6b782015236e4e7106e59d0d9e1b38 Mon Sep 17 00:00:00 2001 From: cody koeninger Date: Sun, 12 Apr 2015 17:37:30 +0100 Subject: [PATCH 051/128] [SPARK-6431][Streaming][Kafka] Error message for partition metadata requ... ...ests The original reported problem was misdiagnosed; the topic just didn't exist yet. Agreed upon solution was to improve error handling / message Author: cody koeninger Closes #5454 from koeninger/spark-6431-master and squashes the following commits: 44300f8 [cody koeninger] [SPARK-6431][Streaming][Kafka] Error message for partition metadata requests --- .../spark/streaming/kafka/KafkaCluster.scala | 14 +++++++++++--- .../spark/streaming/kafka/KafkaClusterSuite.scala | 3 +++ 2 files changed, 14 insertions(+), 3 deletions(-) diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala index 2f7e0ab39fefd..bd767031c1849 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala @@ -123,9 +123,17 @@ class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { val errs = new Err withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => val resp: TopicMetadataResponse = consumer.send(req) - // error codes here indicate missing / just created topic, - // repeating on a different broker wont be useful - return Right(resp.topicsMetadata.toSet) + val respErrs = resp.topicsMetadata.filter(m => m.errorCode != ErrorMapping.NoError) + + if (respErrs.isEmpty) { + return Right(resp.topicsMetadata.toSet) + } else { + respErrs.foreach { m => + val cause = ErrorMapping.exceptionFor(m.errorCode) + val msg = s"Error getting partition metadata for '${m.topic}'. Does the topic exist?" + errs.append(new SparkException(msg, cause)) + } + } } Left(errs) } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala index 2b33d2a220b2b..7fb841b79cb65 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala @@ -52,6 +52,9 @@ class KafkaClusterSuite extends FunSuite with BeforeAndAfterAll { val parts = kc.getPartitions(Set(topic)).right.get assert(parts(topicAndPartition), "didn't get partitions") + + val err = kc.getPartitions(Set(topic + "BAD")) + assert(err.isLeft, "getPartitions for a nonexistant topic should be an error") } test("leader offset apis") { From 04bcd67cfc50f847559a9ff59a31aa93028b3628 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Sun, 12 Apr 2015 18:58:53 +0100 Subject: [PATCH 052/128] [MINOR] a typo: coalesce Author: Daoyuan Wang Closes #5482 from adrian-wang/typo and squashes the following commits: e65ef6f [Daoyuan Wang] typo --- .../org/apache/spark/sql/execution/GeneratedAggregate.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 3c58e93b45e9d..95176e425132d 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 @@ -97,7 +97,7 @@ case class GeneratedAggregate( val currentSum = AttributeReference("currentSum", calcType, nullable = true)() val initialValue = Literal.create(null, calcType) - // Coalasce avoids double calculation... + // Coalesce avoids double calculation... // but really, common sub expression elimination would be better.... val zero = Cast(Literal(0), calcType) val updateFunction = Coalesce( From a1fe59dae50f551d02dd18676308eca054ff6b07 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 12 Apr 2015 20:50:49 -0700 Subject: [PATCH 053/128] [SPARK-6765] Fix test code style for core. Author: Reynold Xin Closes #5484 from rxin/test-style-core and squashes the following commits: e0b0100 [Reynold Xin] [SPARK-6765] Fix test code style for core. --- .../org/apache/spark/AccumulatorSuite.scala | 30 ++--- .../org/apache/spark/CacheManagerSuite.scala | 7 +- .../org/apache/spark/CheckpointSuite.scala | 15 ++- .../apache/spark/ContextCleanerSuite.scala | 4 +- .../scala/org/apache/spark/FileSuite.scala | 26 +++-- .../apache/spark/ImplicitOrderingSuite.scala | 6 +- .../apache/spark/JobCancellationSuite.scala | 2 +- .../org/apache/spark/LocalSparkContext.scala | 4 +- .../org/apache/spark/PartitioningSuite.scala | 30 ++--- .../org/apache/spark/SSLOptionsSuite.scala | 15 ++- .../org/apache/spark/SSLSampleConfigs.scala | 7 +- .../scala/org/apache/spark/ShuffleSuite.scala | 8 +- .../org/apache/spark/SparkContextSuite.scala | 6 +- .../org/apache/spark/StatusTrackerSuite.scala | 5 +- .../spark/broadcast/BroadcastSuite.scala | 2 +- .../spark/deploy/JsonProtocolSuite.scala | 6 +- .../spark/deploy/LogUrlsStandaloneSuite.scala | 2 +- .../deploy/history/HistoryServerSuite.scala | 4 +- .../rest/StandaloneRestSubmitSuite.scala | 4 +- .../deploy/rest/SubmitRestProtocolSuite.scala | 3 +- .../deploy/worker/ExecutorRunnerTest.scala | 3 +- .../deploy/worker/WorkerArgumentsTest.scala | 4 +- .../spark/deploy/worker/WorkerSuite.scala | 6 +- .../metrics/InputOutputMetricsSuite.scala | 2 +- .../spark/metrics/MetricsConfigSuite.scala | 15 ++- .../org/apache/spark/rdd/JdbcRDDSuite.scala | 4 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 41 +++---- .../rdd/ParallelCollectionSplitSuite.scala | 20 ++-- .../spark/rdd/PartitionPruningRDDSuite.scala | 7 +- .../rdd/PartitionwiseSampledRDDSuite.scala | 2 +- .../scala/org/apache/spark/rdd/RDDSuite.scala | 22 ++-- .../org/apache/spark/rdd/RDDSuiteUtils.scala | 4 +- .../org/apache/spark/rpc/RpcEnvSuite.scala | 40 +++---- .../spark/scheduler/DAGSchedulerSuite.scala | 103 ++++++++++-------- .../scheduler/EventLoggingListenerSuite.scala | 4 +- .../scheduler/NotSerializableFakeTask.scala | 4 +- .../spark/scheduler/SparkListenerSuite.scala | 21 ++-- .../scheduler/TaskSchedulerImplSuite.scala | 10 +- .../spark/scheduler/TaskSetManagerSuite.scala | 26 +++-- .../mesos/MesosSchedulerBackendSuite.scala | 14 ++- .../serializer/KryoSerializerSuite.scala | 15 ++- .../ProactiveClosureSerializationSuite.scala | 8 +- .../spark/serializer/TestSerializer.scala | 5 +- .../hash/HashShuffleManagerSuite.scala | 4 +- .../spark/storage/BlockManagerSuite.scala | 32 +++--- .../apache/spark/storage/LocalDirsSuite.scala | 2 +- .../org/apache/spark/ui/UISeleniumSuite.scala | 2 +- .../ui/jobs/JobProgressListenerSuite.scala | 4 +- .../spark/ui/storage/StorageTabSuite.scala | 3 +- .../spark/util/ClosureCleanerSuite.scala | 6 +- .../apache/spark/util/FileAppenderSuite.scala | 13 ++- .../apache/spark/util/NextIteratorSuite.scala | 2 +- .../spark/util/SizeEstimatorSuite.scala | 6 +- .../spark/util/TimeStampedHashMapSuite.scala | 4 +- .../org/apache/spark/util/UtilsSuite.scala | 5 +- .../org/apache/spark/util/VectorSuite.scala | 2 +- .../ExternalAppendOnlyMapSuite.scala | 2 +- .../util/collection/ExternalSorterSuite.scala | 35 +++--- .../util/random/XORShiftRandomSuite.scala | 2 +- 59 files changed, 386 insertions(+), 304 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index bd0f8bdefa171..75399461f2a5f 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -27,19 +27,20 @@ import org.scalatest.Matchers class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { - implicit def setAccum[A] = new AccumulableParam[mutable.Set[A], A] { - def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = { - t1 ++= t2 - t1 - } - def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = { - t1 += t2 - t1 - } - def zero(t: mutable.Set[A]) : mutable.Set[A] = { - new mutable.HashSet[A]() + implicit def setAccum[A]: AccumulableParam[mutable.Set[A], A] = + new AccumulableParam[mutable.Set[A], A] { + def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = { + t1 ++= t2 + t1 + } + def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = { + t1 += t2 + t1 + } + def zero(t: mutable.Set[A]) : mutable.Set[A] = { + new mutable.HashSet[A]() + } } - } test ("basic accumulation"){ sc = new SparkContext("local", "test") @@ -49,11 +50,10 @@ class AccumulatorSuite extends FunSuite with Matchers with LocalSparkContext { d.foreach{x => acc += x} acc.value should be (210) - - val longAcc = sc.accumulator(0l) + val longAcc = sc.accumulator(0L) val maxInt = Integer.MAX_VALUE.toLong d.foreach{x => longAcc += maxInt + x} - longAcc.value should be (210l + maxInt * 20) + longAcc.value should be (210L + maxInt * 20) } test ("value not assignable from tasks") { diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index 4b25c200a695a..70529d9216591 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -45,16 +45,17 @@ class CacheManagerSuite extends FunSuite with LocalSparkContext with BeforeAndAf rdd = new RDD[Int](sc, Nil) { override def getPartitions: Array[Partition] = Array(split) override val getDependencies = List[Dependency[_]]() - override def compute(split: Partition, context: TaskContext) = Array(1, 2, 3, 4).iterator + override def compute(split: Partition, context: TaskContext): Iterator[Int] = + Array(1, 2, 3, 4).iterator } rdd2 = new RDD[Int](sc, List(new OneToOneDependency(rdd))) { override def getPartitions: Array[Partition] = firstParent[Int].partitions - override def compute(split: Partition, context: TaskContext) = + override def compute(split: Partition, context: TaskContext): Iterator[Int] = firstParent[Int].iterator(split, context) }.cache() rdd3 = new RDD[Int](sc, List(new OneToOneDependency(rdd2))) { override def getPartitions: Array[Partition] = firstParent[Int].partitions - override def compute(split: Partition, context: TaskContext) = + override def compute(split: Partition, context: TaskContext): Iterator[Int] = firstParent[Int].iterator(split, context) }.cache() } diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 32abc65385267..e1faddeabec79 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -75,7 +75,8 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { assert(sc.checkpointFile[Int](parCollection.getCheckpointFile.get).collect() === result) assert(parCollection.dependencies != Nil) assert(parCollection.partitions.length === numPartitions) - assert(parCollection.partitions.toList === parCollection.checkpointData.get.getPartitions.toList) + assert(parCollection.partitions.toList === + parCollection.checkpointData.get.getPartitions.toList) assert(parCollection.collect() === result) } @@ -102,13 +103,13 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { } test("UnionRDD") { - def otherRDD = sc.makeRDD(1 to 10, 1) + def otherRDD: RDD[Int] = sc.makeRDD(1 to 10, 1) testRDD(_.union(otherRDD)) testRDDPartitions(_.union(otherRDD)) } test("CartesianRDD") { - def otherRDD = sc.makeRDD(1 to 10, 1) + def otherRDD: RDD[Int] = sc.makeRDD(1 to 10, 1) testRDD(new CartesianRDD(sc, _, otherRDD)) testRDDPartitions(new CartesianRDD(sc, _, otherRDD)) @@ -223,7 +224,8 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { val partitionAfterCheckpoint = serializeDeserialize( unionRDD.partitions.head.asInstanceOf[PartitionerAwareUnionRDDPartition]) assert( - partitionBeforeCheckpoint.parents.head.getClass != partitionAfterCheckpoint.parents.head.getClass, + partitionBeforeCheckpoint.parents.head.getClass != + partitionAfterCheckpoint.parents.head.getClass, "PartitionerAwareUnionRDDPartition.parents not updated after parent RDD is checkpointed" ) } @@ -358,7 +360,7 @@ class CheckpointSuite extends FunSuite with LocalSparkContext with Logging { * Generate an pair RDD (with partitioner) such that both the RDD and its partitions * have large size. */ - def generateFatPairRDD() = { + def generateFatPairRDD(): RDD[(Int, Int)] = { new FatPairRDD(sc.makeRDD(1 to 100, 4), partitioner).mapValues(x => x) } @@ -445,7 +447,8 @@ class FatPairRDD(parent: RDD[Int], _partitioner: Partitioner) extends RDD[(Int, object CheckpointSuite { // This is a custom cogroup function that does not use mapValues like // the PairRDDFunctions.cogroup() - def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) = { + def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) + : RDD[(K, Array[Iterable[V]])] = { new CoGroupedRDD[K]( Seq(first.asInstanceOf[RDD[(K, _)]], second.asInstanceOf[RDD[(K, _)]]), part diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index cdfaacee7da40..1de169d964d23 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -64,7 +64,7 @@ abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[Ha } } - //------ Helper functions ------ + // ------ Helper functions ------ protected def newRDD() = sc.makeRDD(1 to 10) protected def newPairRDD() = newRDD().map(_ -> 1) @@ -370,7 +370,7 @@ class CleanerTester( val cleanerListener = new CleanerListener { def rddCleaned(rddId: Int): Unit = { toBeCleanedRDDIds -= rddId - logInfo("RDD "+ rddId + " cleaned") + logInfo("RDD " + rddId + " cleaned") } def shuffleCleaned(shuffleId: Int): Unit = { diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 7acd27c735727..c8f08eed47c76 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -222,7 +222,7 @@ class FileSuite extends FunSuite with LocalSparkContext { val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) nums.saveAsSequenceFile(outputDir) val output = - sc.newAPIHadoopFile[IntWritable, Text, SequenceFileInputFormat[IntWritable, Text]](outputDir) + sc.newAPIHadoopFile[IntWritable, Text, SequenceFileInputFormat[IntWritable, Text]](outputDir) assert(output.map(_.toString).collect().toList === List("(1,a)", "(2,aa)", "(3,aaa)")) } @@ -451,7 +451,8 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("prevent user from overwriting the empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) intercept[FileAlreadyExistsException] { randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } @@ -459,8 +460,10 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("prevent user from overwriting the non-empty directory (new Hadoop API)") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( + tempDir.getPath + "/output") assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) intercept[FileAlreadyExistsException] { randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) @@ -471,16 +474,20 @@ class FileSuite extends FunSuite with LocalSparkContext { val sf = new SparkConf() sf.setAppName("test").setMaster("local").set("spark.hadoop.validateOutputSpecs", "false") sc = new SparkContext(sf) - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( + tempDir.getPath + "/output") assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) - randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath + "/output") + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( + tempDir.getPath + "/output") assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) } test ("save Hadoop Dataset through old Hadoop API") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new JobConf() job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) @@ -492,7 +499,8 @@ class FileSuite extends FunSuite with LocalSparkContext { test ("save Hadoop Dataset through new Hadoop API") { sc = new SparkContext("local", "test") - val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + val randomRDD = sc.parallelize( + Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) val job = new Job(sc.hadoopConfiguration) job.setOutputKeyClass(classOf[String]) job.setOutputValueClass(classOf[String]) diff --git a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala index d895230ecf330..51348c039b5c9 100644 --- a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala @@ -51,7 +51,7 @@ private object ImplicitOrderingSuite { override def compare(o: OrderedClass): Int = ??? } - def basicMapExpectations(rdd: RDD[Int]) = { + def basicMapExpectations(rdd: RDD[Int]): List[(Boolean, String)] = { List((rdd.map(x => (x, x)).keyOrdering.isDefined, "rdd.map(x => (x, x)).keyOrdering.isDefined"), (rdd.map(x => (1, x)).keyOrdering.isDefined, @@ -68,7 +68,7 @@ private object ImplicitOrderingSuite { "rdd.map(x => (new OrderedClass, x)).keyOrdering.isDefined")) } - def otherRDDMethodExpectations(rdd: RDD[Int]) = { + def otherRDDMethodExpectations(rdd: RDD[Int]): List[(Boolean, String)] = { List((rdd.groupBy(x => x).keyOrdering.isDefined, "rdd.groupBy(x => x).keyOrdering.isDefined"), (rdd.groupBy(x => new NonOrderedClass).keyOrdering.isEmpty, @@ -82,4 +82,4 @@ private object ImplicitOrderingSuite { (rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined, "rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined")) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 21487bc24d58a..4d3e09793faff 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -188,7 +188,7 @@ class JobCancellationSuite extends FunSuite with Matchers with BeforeAndAfter val rdd = sc.parallelize(1 to 10, 2).map { i => JobCancellationSuite.twoJobsSharingStageSemaphore.acquire() (i, i) - }.reduceByKey(_+_) + }.reduceByKey(_ + _) val f1 = rdd.collectAsync() val f2 = rdd.countAsync() diff --git a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala index 53e367a61715b..8bf2e55defd02 100644 --- a/core/src/test/scala/org/apache/spark/LocalSparkContext.scala +++ b/core/src/test/scala/org/apache/spark/LocalSparkContext.scala @@ -37,7 +37,7 @@ trait LocalSparkContext extends BeforeAndAfterEach with BeforeAndAfterAll { self super.afterEach() } - def resetSparkContext() = { + def resetSparkContext(): Unit = { LocalSparkContext.stop(sc) sc = null } @@ -54,7 +54,7 @@ object LocalSparkContext { } /** Runs `f` by passing in `sc` and ensures that `sc` is stopped. */ - def withSpark[T](sc: SparkContext)(f: SparkContext => T) = { + def withSpark[T](sc: SparkContext)(f: SparkContext => T): T = { try { f(sc) } finally { diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index b7532314ada01..47e3bf6e1ac41 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -92,7 +92,7 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet test("RangePartitioner for keys that are not Comparable (but with Ordering)") { // Row does not extend Comparable, but has an implicit Ordering defined. implicit object RowOrdering extends Ordering[Row] { - override def compare(x: Row, y: Row) = x.value - y.value + override def compare(x: Row, y: Row): Int = x.value - y.value } val rdd = sc.parallelize(1 to 4500).map(x => (Row(x), Row(x))) @@ -212,20 +212,24 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet val arrPairs: RDD[(Array[Int], Int)] = sc.parallelize(Array(1, 2, 3, 4), 2).map(x => (Array(x), x)) - assert(intercept[SparkException]{ arrs.distinct() }.getMessage.contains("array")) + def verify(testFun: => Unit): Unit = { + intercept[SparkException](testFun).getMessage.contains("array") + } + + verify(arrs.distinct()) // We can't catch all usages of arrays, since they might occur inside other collections: // assert(fails { arrPairs.distinct() }) - assert(intercept[SparkException]{ arrPairs.partitionBy(new HashPartitioner(2)) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.join(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.leftOuterJoin(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.rightOuterJoin(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.fullOuterJoin(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.groupByKey() }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.countByKey() }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.countByKeyApprox(1) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.cogroup(arrPairs) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.reduceByKeyLocally(_ + _) }.getMessage.contains("array")) - assert(intercept[SparkException]{ arrPairs.reduceByKey(_ + _) }.getMessage.contains("array")) + verify(arrPairs.partitionBy(new HashPartitioner(2))) + verify(arrPairs.join(arrPairs)) + verify(arrPairs.leftOuterJoin(arrPairs)) + verify(arrPairs.rightOuterJoin(arrPairs)) + verify(arrPairs.fullOuterJoin(arrPairs)) + verify(arrPairs.groupByKey()) + verify(arrPairs.countByKey()) + verify(arrPairs.countByKeyApprox(1)) + verify(arrPairs.cogroup(arrPairs)) + verify(arrPairs.reduceByKeyLocally(_ + _)) + verify(arrPairs.reduceByKey(_ + _)) } test("zero-length partitions should be correctly handled") { diff --git a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala index 444a33371bd71..93f46ef11c0e2 100644 --- a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala @@ -36,7 +36,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { conf.set("spark.ssl.keyPassword", "password") conf.set("spark.ssl.trustStore", trustStorePath) conf.set("spark.ssl.trustStorePassword", "password") - conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") conf.set("spark.ssl.protocol", "SSLv3") val opts = SSLOptions.parse(conf, "spark.ssl") @@ -52,7 +53,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { assert(opts.keyStorePassword === Some("password")) assert(opts.keyPassword === Some("password")) assert(opts.protocol === Some("SSLv3")) - assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) + assert(opts.enabledAlgorithms === + Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) } test("test resolving property with defaults specified ") { @@ -66,7 +68,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { conf.set("spark.ssl.keyPassword", "password") conf.set("spark.ssl.trustStore", trustStorePath) conf.set("spark.ssl.trustStorePassword", "password") - conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") conf.set("spark.ssl.protocol", "SSLv3") val defaultOpts = SSLOptions.parse(conf, "spark.ssl", defaults = None) @@ -83,7 +86,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { assert(opts.keyStorePassword === Some("password")) assert(opts.keyPassword === Some("password")) assert(opts.protocol === Some("SSLv3")) - assert(opts.enabledAlgorithms === Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) + assert(opts.enabledAlgorithms === + Set("TLS_RSA_WITH_AES_128_CBC_SHA", "TLS_RSA_WITH_AES_256_CBC_SHA")) } test("test whether defaults can be overridden ") { @@ -99,7 +103,8 @@ class SSLOptionsSuite extends FunSuite with BeforeAndAfterAll { conf.set("spark.ssl.keyPassword", "password") conf.set("spark.ssl.trustStore", trustStorePath) conf.set("spark.ssl.trustStorePassword", "password") - conf.set("spark.ssl.enabledAlgorithms", "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") + conf.set("spark.ssl.enabledAlgorithms", + "TLS_RSA_WITH_AES_128_CBC_SHA, TLS_RSA_WITH_AES_256_CBC_SHA") conf.set("spark.ui.ssl.enabledAlgorithms", "ABC, DEF") conf.set("spark.ssl.protocol", "SSLv3") diff --git a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala index ace8123a8961f..308b9ea17708d 100644 --- a/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala +++ b/core/src/test/scala/org/apache/spark/SSLSampleConfigs.scala @@ -21,10 +21,11 @@ import java.io.File object SSLSampleConfigs { val keyStorePath = new File(this.getClass.getResource("/keystore").toURI).getAbsolutePath - val untrustedKeyStorePath = new File(this.getClass.getResource("/untrusted-keystore").toURI).getAbsolutePath + val untrustedKeyStorePath = new File( + this.getClass.getResource("/untrusted-keystore").toURI).getAbsolutePath val trustStorePath = new File(this.getClass.getResource("/truststore").toURI).getAbsolutePath - def sparkSSLConfig() = { + def sparkSSLConfig(): SparkConf = { val conf = new SparkConf(loadDefaults = false) conf.set("spark.ssl.enabled", "true") conf.set("spark.ssl.keyStore", keyStorePath) @@ -38,7 +39,7 @@ object SSLSampleConfigs { conf } - def sparkSSLConfigUntrusted() = { + def sparkSSLConfigUntrusted(): SparkConf = { val conf = new SparkConf(loadDefaults = false) conf.set("spark.ssl.enabled", "true") conf.set("spark.ssl.keyStore", untrustedKeyStorePath) diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index 30b6184c77839..d7180516029d5 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -142,7 +142,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("shuffle on mutable pairs") { // 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) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data = Array(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) val results = new ShuffledRDD[Int, Int, Int](pairs, @@ -155,7 +155,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex // This is not in SortingSuite because of the local cluster setup. // 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) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data = Array(p(1, 11), p(3, 33), p(100, 100), p(2, 22)) val pairs: RDD[MutablePair[Int, Int]] = sc.parallelize(data, 2) val results = new OrderedRDDFunctions[Int, Int, MutablePair[Int, Int]](pairs) @@ -169,7 +169,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("cogroup using mutable pairs") { // 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) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1)) val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22"), p(3, "3")) val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2) @@ -196,7 +196,7 @@ abstract class ShuffleSuite extends FunSuite with Matchers with LocalSparkContex test("subtract mutable pairs") { // 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) - def p[T1, T2](_1: T1, _2: T2) = MutablePair(_1, _2) + def p[T1, T2](_1: T1, _2: T2): MutablePair[T1, T2] = MutablePair(_1, _2) val data1 = Seq(p(1, 1), p(1, 2), p(1, 3), p(2, 1), p(3, 33)) val data2 = Seq(p(1, "11"), p(1, "12"), p(2, "22")) val pairs1: RDD[MutablePair[Int, Int]] = sc.parallelize(data1, 2) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index c7301a30d8b11..94be1c6d6397c 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -114,11 +114,13 @@ class SparkContextSuite extends FunSuite with LocalSparkContext { if (length1 != gotten1.length()) { throw new SparkException( - s"file has different length $length1 than added file ${gotten1.length()} : " + absolutePath1) + s"file has different length $length1 than added file ${gotten1.length()} : " + + absolutePath1) } if (length2 != gotten2.length()) { throw new SparkException( - s"file has different length $length2 than added file ${gotten2.length()} : " + absolutePath2) + s"file has different length $length2 than added file ${gotten2.length()} : " + + absolutePath2) } if (absolutePath1 == gotten1.getAbsolutePath) { diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala index 41d6ea29d5b06..084eb237d70d1 100644 --- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala @@ -82,7 +82,8 @@ class StatusTrackerSuite extends FunSuite with Matchers with LocalSparkContext { secondJobFuture.jobIds.head } eventually(timeout(10 seconds)) { - sc.statusTracker.getJobIdsForGroup("my-job-group").toSet should be (Set(firstJobId, secondJobId)) + sc.statusTracker.getJobIdsForGroup("my-job-group").toSet should be ( + Set(firstJobId, secondJobId)) } } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index af3272692d7a1..c8fdfa693912e 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -33,7 +33,7 @@ class DummyBroadcastClass(rdd: RDD[Int]) extends Serializable { val broadcast = rdd.context.broadcast(list) val bid = broadcast.id - def doSomething() = { + def doSomething(): Set[(Int, Boolean)] = { rdd.map { x => val bm = SparkEnv.get.blockManager // Check if broadcast block was fetched diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 68b5776fc6515..2071701b313db 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -100,13 +100,13 @@ class JsonProtocolSuite extends FunSuite { appInfo } - def createDriverCommand() = new Command( + def createDriverCommand(): Command = new Command( "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Seq("-Dfoo") ) - def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, - false, createDriverCommand()) + def createDriverDesc(): DriverDescription = + new DriverDescription("hdfs://some-dir/some.jar", 100, 3, false, createDriverCommand()) def createDriverInfo(): DriverInfo = new DriverInfo(3, "driver-3", createDriverDesc(), new Date()) diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala index 54dd7c9c45c61..9cdb42814ca32 100644 --- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -56,7 +56,7 @@ class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { test("verify that log urls reflect SPARK_PUBLIC_DNS (SPARK-6175)") { val SPARK_PUBLIC_DNS = "public_dns" class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_PUBLIC_DNS") SPARK_PUBLIC_DNS else super.getenv(name) } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 3a9963a5ce7b7..20de46fdab909 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -42,10 +42,10 @@ class HistoryServerSuite extends FunSuite with Matchers with MockitoSugar { when(historyServer.getProviderConfig()).thenReturn(Map[String, String]()) val page = new HistoryPage(historyServer) - //when + // when val response = page.render(request) - //then + // then val links = response \\ "a" val justHrefs = for { l <- links diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 2fa90e3bd1c63..8e09976636386 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -508,7 +508,7 @@ private class DummyMaster( exception: Option[Exception] = None) extends Actor { - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case RequestSubmitDriver(driverDesc) => sender ! SubmitDriverResponse(success = true, Some(submitId), submitMessage) case RequestKillDriver(driverId) => @@ -531,7 +531,7 @@ private class SmarterMaster extends Actor { private var counter: Int = 0 private val submittedDrivers = new mutable.HashMap[String, DriverState] - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case RequestSubmitDriver(driverDesc) => val driverId = s"driver-$counter" submittedDrivers(driverId) = RUNNING diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala index 1d64ec201e647..61071ee17256c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala @@ -129,7 +129,8 @@ class SubmitRestProtocolSuite extends FunSuite { assert(newMessage.sparkProperties("spark.files") === "fireball.png") assert(newMessage.sparkProperties("spark.driver.memory") === "512m") assert(newMessage.sparkProperties("spark.driver.cores") === "180") - assert(newMessage.sparkProperties("spark.driver.extraJavaOptions") === " -Dslices=5 -Dcolor=mostly_red") + assert(newMessage.sparkProperties("spark.driver.extraJavaOptions") === + " -Dslices=5 -Dcolor=mostly_red") assert(newMessage.sparkProperties("spark.driver.extraClassPath") === "food-coloring.jar") assert(newMessage.sparkProperties("spark.driver.extraLibraryPath") === "pickle.jar") assert(newMessage.sparkProperties("spark.driver.supervise") === "false") diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 6fca6321e5a1b..a8b9df227c996 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -35,7 +35,8 @@ class ExecutorRunnerTest extends FunSuite { val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", 123, "publicAddr", new File(sparkHome), new File("ooga"), "blah", new SparkConf, Seq("localDir"), ExecutorState.RUNNING) - val builder = CommandUtils.buildProcessBuilder(appDesc.command, 512, sparkHome, er.substituteVariables) + val builder = CommandUtils.buildProcessBuilder( + appDesc.command, 512, sparkHome, er.substituteVariables) assert(builder.command().last === appId) } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala index 372d7aa453008..7cc2104281464 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerArgumentsTest.scala @@ -37,7 +37,7 @@ class WorkerArgumentsTest extends FunSuite { val args = Array("spark://localhost:0000 ") class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_WORKER_MEMORY") "50000" else super.getenv(name) } @@ -56,7 +56,7 @@ class WorkerArgumentsTest extends FunSuite { val args = Array("spark://localhost:0000 ") class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_WORKER_MEMORY") "5G" else super.getenv(name) } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala index 84e2fd7ad936d..450fba21f4b5c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -24,8 +24,10 @@ import org.scalatest.{Matchers, FunSuite} class WorkerSuite extends FunSuite with Matchers { - def cmd(javaOpts: String*) = Command("", Seq.empty, Map.empty, Seq.empty, Seq.empty, Seq(javaOpts:_*)) - def conf(opts: (String, String)*) = new SparkConf(loadDefaults = false).setAll(opts) + def cmd(javaOpts: String*): Command = { + Command("", Seq.empty, Map.empty, Seq.empty, Seq.empty, Seq(javaOpts:_*)) + } + def conf(opts: (String, String)*): SparkConf = new SparkConf(loadDefaults = false).setAll(opts) test("test isUseLocalNodeSSLConfig") { Worker.isUseLocalNodeSSLConfig(cmd("-Dasdf=dfgh")) shouldBe false diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index 78fa98a3b9065..190b08d950a02 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -238,7 +238,7 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext sc.textFile(tmpFilePath, 4) .map(key => (key, 1)) - .reduceByKey(_+_) + .reduceByKey(_ + _) .saveAsTextFile("file://" + tmpFile.getAbsolutePath) sc.listenerBus.waitUntilEmpty(500) diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala index 37e528435aa5d..100ac77dec1f7 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala @@ -35,7 +35,8 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { val property = conf.getInstance("random") assert(property.size() === 2) - assert(property.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") + assert(property.getProperty("sink.servlet.class") === + "org.apache.spark.metrics.sink.MetricsServlet") assert(property.getProperty("sink.servlet.path") === "/metrics/json") } @@ -47,16 +48,20 @@ class MetricsConfigSuite extends FunSuite with BeforeAndAfter { assert(masterProp.size() === 5) assert(masterProp.getProperty("sink.console.period") === "20") assert(masterProp.getProperty("sink.console.unit") === "minutes") - assert(masterProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource") - assert(masterProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") + assert(masterProp.getProperty("source.jvm.class") === + "org.apache.spark.metrics.source.JvmSource") + assert(masterProp.getProperty("sink.servlet.class") === + "org.apache.spark.metrics.sink.MetricsServlet") assert(masterProp.getProperty("sink.servlet.path") === "/metrics/master/json") val workerProp = conf.getInstance("worker") assert(workerProp.size() === 5) assert(workerProp.getProperty("sink.console.period") === "10") assert(workerProp.getProperty("sink.console.unit") === "seconds") - assert(workerProp.getProperty("source.jvm.class") === "org.apache.spark.metrics.source.JvmSource") - assert(workerProp.getProperty("sink.servlet.class") === "org.apache.spark.metrics.sink.MetricsServlet") + assert(workerProp.getProperty("source.jvm.class") === + "org.apache.spark.metrics.source.JvmSource") + assert(workerProp.getProperty("sink.servlet.class") === + "org.apache.spark.metrics.sink.MetricsServlet") assert(workerProp.getProperty("sink.servlet.path") === "/metrics/json") } 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 0dc59888f7304..be8467354b222 100644 --- a/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/JdbcRDDSuite.scala @@ -80,7 +80,7 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { (r: ResultSet) => { r.getInt(1) } ).cache() assert(rdd.count === 100) - assert(rdd.reduce(_+_) === 10100) + assert(rdd.reduce(_ + _) === 10100) } test("large id overflow") { @@ -92,7 +92,7 @@ class JdbcRDDSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { 1131544775L, 567279358897692673L, 20, (r: ResultSet) => { r.getInt(1) } ).cache() assert(rdd.count === 100) - assert(rdd.reduce(_+_) === 5050) + assert(rdd.reduce(_ + _) === 5050) } after { 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 108f70af43f37..ca0d953d306d8 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -168,13 +168,13 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("reduceByKey") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collect() + val sums = pairs.reduceByKey(_ + _).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } test("reduceByKey with collectAsMap") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_).collectAsMap() + val sums = pairs.reduceByKey(_ + _).collectAsMap() assert(sums.size === 2) assert(sums(1) === 7) assert(sums(2) === 1) @@ -182,7 +182,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("reduceByKey with many output partitons") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.reduceByKey(_+_, 10).collect() + val sums = pairs.reduceByKey(_ + _, 10).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } @@ -192,7 +192,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { def getPartition(key: Any) = key.asInstanceOf[Int] } val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 1), (0, 1))).partitionBy(p) - val sums = pairs.reduceByKey(_+_) + val sums = pairs.reduceByKey(_ + _) assert(sums.collect().toSet === Set((1, 4), (0, 1))) assert(sums.partitioner === Some(p)) // count the dependencies to make sure there is only 1 ShuffledRDD @@ -208,7 +208,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } test("countApproxDistinctByKey") { - def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble + def error(est: Long, size: Long): Double = math.abs(est - size) / size.toDouble /* Since HyperLogLog unique counting is approximate, and the relative standard deviation is * only a statistical bound, the tests can fail for large values of relativeSD. We will be using @@ -465,7 +465,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { test("foldByKey") { val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (1, 1), (2, 1))) - val sums = pairs.foldByKey(0)(_+_).collect() + val sums = pairs.foldByKey(0)(_ + _).collect() assert(sums.toSet === Set((1, 7), (2, 1))) } @@ -505,7 +505,8 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { conf.setOutputCommitter(classOf[FakeOutputCommitter]) FakeOutputCommitter.ran = false - pairs.saveAsHadoopFile("ignored", pairs.keyClass, pairs.valueClass, classOf[FakeOutputFormat], conf) + pairs.saveAsHadoopFile( + "ignored", pairs.keyClass, pairs.valueClass, classOf[FakeOutputFormat], conf) assert(FakeOutputCommitter.ran, "OutputCommitter was never called") } @@ -552,7 +553,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { } private object StratifiedAuxiliary { - def stratifier (fractionPositive: Double) = { + def stratifier (fractionPositive: Double): (Int) => String = { (x: Int) => if (x % 10 < (10 * fractionPositive).toInt) "1" else "0" } @@ -572,7 +573,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { def testSampleExact(stratifiedData: RDD[(String, Int)], samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { testBernoulli(stratifiedData, true, samplingRate, seed, n) testPoisson(stratifiedData, true, samplingRate, seed, n) } @@ -580,7 +581,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { def testSample(stratifiedData: RDD[(String, Int)], samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { testBernoulli(stratifiedData, false, samplingRate, seed, n) testPoisson(stratifiedData, false, samplingRate, seed, n) } @@ -590,7 +591,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { exact: Boolean, samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { val expectedSampleSize = stratifiedData.countByKey() .mapValues(count => math.ceil(count * samplingRate).toInt) val fractions = Map("1" -> samplingRate, "0" -> samplingRate) @@ -612,7 +613,7 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { exact: Boolean, samplingRate: Double, seed: Long, - n: Long) = { + n: Long): Unit = { val expectedSampleSize = stratifiedData.countByKey().mapValues(count => math.ceil(count * samplingRate).toInt) val fractions = Map("1" -> samplingRate, "0" -> samplingRate) @@ -701,27 +702,27 @@ class FakeOutputFormat() extends OutputFormat[Integer, Integer]() { */ class NewFakeWriter extends NewRecordWriter[Integer, Integer] { - def close(p1: NewTaskAttempContext) = () + def close(p1: NewTaskAttempContext): Unit = () - def write(p1: Integer, p2: Integer) = () + def write(p1: Integer, p2: Integer): Unit = () } class NewFakeCommitter extends NewOutputCommitter { - def setupJob(p1: NewJobContext) = () + def setupJob(p1: NewJobContext): Unit = () def needsTaskCommit(p1: NewTaskAttempContext): Boolean = false - def setupTask(p1: NewTaskAttempContext) = () + def setupTask(p1: NewTaskAttempContext): Unit = () - def commitTask(p1: NewTaskAttempContext) = () + def commitTask(p1: NewTaskAttempContext): Unit = () - def abortTask(p1: NewTaskAttempContext) = () + def abortTask(p1: NewTaskAttempContext): Unit = () } class NewFakeFormat() extends NewOutputFormat[Integer, Integer]() { - def checkOutputSpecs(p1: NewJobContext) = () + def checkOutputSpecs(p1: NewJobContext): Unit = () def getRecordWriter(p1: NewTaskAttempContext): NewRecordWriter[Integer, Integer] = { new NewFakeWriter() @@ -735,7 +736,7 @@ class NewFakeFormat() extends NewOutputFormat[Integer, Integer]() { class ConfigTestFormat() extends NewFakeFormat() with Configurable { var setConfCalled = false - def setConf(p1: Configuration) = { + def setConf(p1: Configuration): Unit = { setConfCalled = true () } diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index cd193ae4f5238..1880364581c1a 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -100,7 +100,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1 until 100 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 99) + assert(slices.map(_.size).reduceLeft(_ + _) === 99) assert(slices.forall(_.isInstanceOf[Range])) } @@ -108,7 +108,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1 to 100 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 100) + assert(slices.map(_.size).reduceLeft(_ + _) === 100) assert(slices.forall(_.isInstanceOf[Range])) } @@ -139,7 +139,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(i).isInstanceOf[Range]) val range = slices(i).asInstanceOf[Range] assert(range.start === i * (N / 40), "slice " + i + " start") - assert(range.end === (i+1) * (N / 40), "slice " + i + " end") + assert(range.end === (i + 1) * (N / 40), "slice " + i + " end") assert(range.step === 1, "slice " + i + " step") } } @@ -156,7 +156,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val slices = ParallelCollectionRDD.slice(d, n) ("n slices" |: slices.size == n) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && - ("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1)) + ("equal sizes" |: slices.map(_.size).forall(x => x == d.size / n || x == d.size /n + 1)) } check(prop) } @@ -174,7 +174,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { ("n slices" |: slices.size == n) && ("all ranges" |: slices.forall(_.isInstanceOf[Range])) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && - ("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1)) + ("equal sizes" |: slices.map(_.size).forall(x => x == d.size / n || x == d.size / n + 1)) } check(prop) } @@ -192,7 +192,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { ("n slices" |: slices.size == n) && ("all ranges" |: slices.forall(_.isInstanceOf[Range])) && ("concat to d" |: Seq.concat(slices: _*).mkString(",") == d.mkString(",")) && - ("equal sizes" |: slices.map(_.size).forall(x => x==d.size/n || x==d.size/n+1)) + ("equal sizes" |: slices.map(_.size).forall(x => x == d.size / n || x == d.size / n + 1)) } check(prop) } @@ -201,7 +201,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1L until 100L val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 99) + assert(slices.map(_.size).reduceLeft(_ + _) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -209,7 +209,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1L to 100L val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 100) + assert(slices.map(_.size).reduceLeft(_ + _) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -217,7 +217,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1.0 until 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 99) + assert(slices.map(_.size).reduceLeft(_ + _) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } @@ -225,7 +225,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = 1.0 to 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) - assert(slices.map(_.size).reduceLeft(_+_) === 100) + assert(slices.map(_.size).reduceLeft(_ + _) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala index 8408d7e785c65..465068c6cbb16 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionPruningRDDSuite.scala @@ -23,7 +23,6 @@ import org.apache.spark.{Partition, SharedSparkContext, TaskContext} class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { - test("Pruned Partitions inherit locality prefs correctly") { val rdd = new RDD[Int](sc, Nil) { @@ -74,8 +73,6 @@ class PartitionPruningRDDSuite extends FunSuite with SharedSparkContext { } class TestPartition(i: Int, value: Int) extends Partition with Serializable { - def index = i - - def testValue = this.value - + def index: Int = i + def testValue: Int = this.value } diff --git a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala index a0483886f8db3..0d1369c19c69e 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PartitionwiseSampledRDDSuite.scala @@ -35,7 +35,7 @@ class MockSampler extends RandomSampler[Long, Long] { Iterator(s) } - override def clone = new MockSampler + override def clone: MockSampler = new MockSampler } class PartitionwiseSampledRDDSuite extends FunSuite with SharedSparkContext { 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 bede1ffb3e2d0..df42faab64505 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -82,7 +82,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("countApproxDistinct") { - def error(est: Long, size: Long) = math.abs(est - size) / size.toDouble + def error(est: Long, size: Long): Double = math.abs(est - size) / size.toDouble val size = 1000 val uniformDistro = for (i <- 1 to 5000) yield i % size @@ -100,7 +100,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { } test("partitioner aware union") { - def makeRDDWithPartitioner(seq: Seq[Int]) = { + def makeRDDWithPartitioner(seq: Seq[Int]): RDD[Int] = { sc.makeRDD(seq, 1) .map(x => (x, null)) .partitionBy(new HashPartitioner(2)) @@ -159,8 +159,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("treeAggregate") { val rdd = sc.makeRDD(-1000 until 1000, 10) - def seqOp = (c: Long, x: Int) => c + x - def combOp = (c1: Long, c2: Long) => c1 + c2 + def seqOp: (Long, Int) => Long = (c: Long, x: Int) => c + x + def combOp: (Long, Long) => Long = (c1: Long, c2: Long) => c1 + c2 for (depth <- 1 until 10) { val sum = rdd.treeAggregate(0L)(seqOp, combOp, depth) assert(sum === -1000L) @@ -204,7 +204,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(empty.collect().size === 0) val thrown = intercept[UnsupportedOperationException]{ - empty.reduce(_+_) + empty.reduce(_ + _) } assert(thrown.getMessage.contains("empty")) @@ -321,7 +321,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(list3.sorted === Array("a","b","c"), "Locality preferences are dropped") // RDD with locality preferences spread (non-randomly) over 6 machines, m0 through m5 - val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i+2)).map{ j => "m" + (j%6)}))) + val data = sc.makeRDD((1 to 9).map(i => (i, (i to (i + 2)).map{ j => "m" + (j%6)}))) val coalesced1 = data.coalesce(3) assert(coalesced1.collect().toList.sorted === (1 to 9).toList, "Data got *lost* in coalescing") @@ -921,15 +921,17 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("task serialization exception should not hang scheduler") { class BadSerializable extends Serializable { @throws(classOf[IOException]) - private def writeObject(out: ObjectOutputStream): Unit = throw new KryoException("Bad serialization") + private def writeObject(out: ObjectOutputStream): Unit = + throw new KryoException("Bad serialization") @throws(classOf[IOException]) private def readObject(in: ObjectInputStream): Unit = {} } - // Note that in the original bug, SPARK-4349, that this verifies, the job would only hang if there were - // more threads in the Spark Context than there were number of objects in this sequence. + // Note that in the original bug, SPARK-4349, that this verifies, the job would only hang if + // there were more threads in the Spark Context than there were number of objects in this + // sequence. intercept[Throwable] { - sc.parallelize(Seq(new BadSerializable, new BadSerializable)).collect + sc.parallelize(Seq(new BadSerializable, new BadSerializable)).collect() } // Check that the context has not crashed sc.parallelize(1 to 100).map(x => x*2).collect diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala index 4762fc17855ce..fe695d85e29dd 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuiteUtils.scala @@ -21,11 +21,11 @@ object RDDSuiteUtils { case class Person(first: String, last: String, age: Int) object AgeOrdering extends Ordering[Person] { - def compare(a:Person, b:Person) = a.age compare b.age + def compare(a:Person, b:Person): Int = a.age.compare(b.age) } object NameOrdering extends Ordering[Person] { - def compare(a:Person, b:Person) = + def compare(a:Person, b:Person): Int = implicitly[Ordering[Tuple2[String,String]]].compare((a.last, a.first), (b.last, b.first)) } } diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index 5a734ec5ba5ec..ada07ef11cd7a 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -70,7 +70,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("send-remotely", new RpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case msg: String => message = msg } }) @@ -109,7 +109,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val rpcEndpointRef = env.setupEndpoint("ask-locally", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case msg: String => { context.reply(msg) } @@ -123,7 +123,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("ask-remotely", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case msg: String => { context.reply(msg) } @@ -146,7 +146,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("ask-timeout", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case msg: String => { Thread.sleep(100) context.reply(msg) @@ -182,7 +182,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { calledMethods += "start" } - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case msg: String => } @@ -206,7 +206,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { throw new RuntimeException("Oops!") } - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => } @@ -225,7 +225,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("onError-onStop", new RpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => } @@ -250,8 +250,8 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("onError-receive", new RpcEndpoint { override val rpcEnv = env - override def receive = { - case m => throw new RuntimeException("Oops!") + override def receive: PartialFunction[Any, Unit] = { + case m => throw new RuntimeException("Oops!") } override def onError(cause: Throwable): Unit = { @@ -277,7 +277,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { callSelfSuccessfully = true } - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => } }) @@ -294,7 +294,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("self-receive", new RpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => { self callSelfSuccessfully = true @@ -316,7 +316,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("self-onStop", new RpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => } @@ -343,7 +343,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint(s"receive-in-sequence-$i", new ThreadSafeRpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => result += 1 } @@ -372,7 +372,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("stop-reentrant", new RpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case m => } @@ -394,7 +394,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("sendWithReply", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case m => context.reply("ack") } }) @@ -410,7 +410,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("sendWithReply-remotely", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case m => context.reply("ack") } }) @@ -432,7 +432,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { val endpointRef = env.setupEndpoint("sendWithReply-error", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case m => context.sendFailure(new SparkException("Oops")) } }) @@ -450,7 +450,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("sendWithReply-remotely-error", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case msg: String => context.sendFailure(new SparkException("Oops")) } }) @@ -476,7 +476,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("network-events", new ThreadSafeRpcEndpoint { override val rpcEnv = env - override def receive = { + override def receive: PartialFunction[Any, Unit] = { case "hello" => case m => events += "receive" -> m } @@ -519,7 +519,7 @@ abstract class RpcEnvSuite extends FunSuite with BeforeAndAfterAll { env.setupEndpoint("sendWithReply-unserializable-error", new RpcEndpoint { override val rpcEnv = env - override def receiveAndReply(context: RpcCallContext) = { + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case msg: String => context.sendFailure(new UnserializableException) } }) 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 eb759f0807a17..3c52a8c4460c6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -57,20 +57,18 @@ class MyRDD( locations: Seq[Seq[String]] = Nil) extends RDD[(Int, Int)](sc, dependencies) with Serializable { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = throw new RuntimeException("should not be reached") - override def getPartitions = (0 until numPartitions).map(i => new Partition { - override def index = i + override def getPartitions: Array[Partition] = (0 until numPartitions).map(i => new Partition { + override def index: Int = i }).toArray override def getPreferredLocations(split: Partition): Seq[String] = - if (locations.isDefinedAt(split.index)) - locations(split.index) - else - Nil + if (locations.isDefinedAt(split.index)) locations(split.index) else Nil override def toString: String = "DAGSchedulerSuiteRDD " + id } class DAGSchedulerSuiteDummyException extends Exception -class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSparkContext with Timeouts { +class DAGSchedulerSuite + extends FunSuiteLike with BeforeAndAfter with LocalSparkContext with Timeouts { val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ @@ -209,7 +207,8 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar 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, null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(i), result._1, result._2, null, createFakeTaskInfo(), null)) } } } @@ -269,21 +268,23 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar submit(new MyRDD(sc, 1, Nil), Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("local job") { val rdd = new PairOfIntsRDD(sc, Nil) { override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = Array(42 -> 0).iterator - override def getPartitions = Array( new Partition { override def index = 0 } ) - override def getPreferredLocations(split: Partition) = Nil - override def toString = "DAGSchedulerSuite Local RDD" + override def getPartitions: Array[Partition] = + Array( new Partition { override def index: Int = 0 } ) + override def getPreferredLocations(split: Partition): List[String] = Nil + override def toString: String = "DAGSchedulerSuite Local RDD" } val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) + runEvent( + JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("local job oom") { @@ -295,9 +296,10 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar override def toString = "DAGSchedulerSuite Local RDD" } val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) + runEvent( + JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, CallSite("", ""), jobListener)) assert(results.size == 0) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run trivial job w/ dependency") { @@ -306,7 +308,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar submit(finalRdd, Array(0)) complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("cache location preferences w/ dependency") { @@ -319,7 +321,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assertLocations(taskSet, Seq(Seq("hostA", "hostB"))) complete(taskSet, Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("regression test for getCacheLocs") { @@ -335,7 +337,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar } test("avoid exponential blowup when getting preferred locs list") { - // Build up a complex dependency graph with repeated zip operations, without preferred locations. + // Build up a complex dependency graph with repeated zip operations, without preferred locations var rdd: RDD[_] = new MyRDD(sc, 1, Nil) (1 to 30).foreach(_ => rdd = rdd.zip(rdd)) // getPreferredLocs runs quickly, indicating that exponential graph traversal is avoided. @@ -357,7 +359,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("trivial job failure") { @@ -367,7 +369,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("trivial job cancellation") { @@ -378,7 +380,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("job cancellation no-kill backend") { @@ -387,18 +389,20 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar val noKillTaskScheduler = new TaskScheduler() { override def rootPool: Pool = null override def schedulingMode: SchedulingMode = SchedulingMode.NONE - override def start() = {} - override def stop() = {} - override def submitTasks(taskSet: TaskSet) = { + override def start(): Unit = {} + override def stop(): Unit = {} + override def submitTasks(taskSet: TaskSet): Unit = { taskSets += taskSet } override def cancelTasks(stageId: Int, interruptThread: Boolean) { throw new UnsupportedOperationException } - override def setDAGScheduler(dagScheduler: DAGScheduler) = {} - override def defaultParallelism() = 2 - override def executorHeartbeatReceived(execId: String, taskMetrics: Array[(Long, TaskMetrics)], - blockManagerId: BlockManagerId): Boolean = true + override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} + override def defaultParallelism(): Int = 2 + override def executorHeartbeatReceived( + execId: String, + taskMetrics: Array[(Long, TaskMetrics)], + blockManagerId: BlockManagerId): Boolean = true override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} } val noKillScheduler = new DAGScheduler( @@ -422,7 +426,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar // When the task set completes normally, state should be correctly updated. complete(taskSets(0), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.isEmpty) @@ -442,7 +446,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar Array(makeBlockManagerId("hostA"), makeBlockManagerId("hostB"))) complete(taskSets(1), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run trivial shuffle with fetch failure") { @@ -465,10 +469,11 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar // have the 2nd attempt pass complete(taskSets(2), Seq((Success, makeMapStatus("hostA", 1)))) // we can see both result blocks now - assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === Array("hostA", "hostB")) + assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1.host) === + Array("hostA", "hostB")) complete(taskSets(3), Seq((Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("trivial shuffle with multiple fetch failures") { @@ -521,19 +526,23 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(newEpoch > oldEpoch) val taskSet = taskSets(0) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) // should work because it's a non-failed host - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(0), Success, makeMapStatus("hostB", 1), null, createFakeTaskInfo(), null)) // should be ignored for being too old - runEvent(CompletionEvent(taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(0), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) // should work because it's a new epoch taskSet.tasks(1).epoch = newEpoch - runEvent(CompletionEvent(taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) + runEvent(CompletionEvent( + taskSet.tasks(1), Success, makeMapStatus("hostA", 1), null, createFakeTaskInfo(), null)) assert(mapOutputTracker.getServerStatuses(shuffleId, 0).map(_._1) === Array(makeBlockManagerId("hostB"), makeBlockManagerId("hostA"))) complete(taskSets(1), Seq((Success, 42), (Success, 43))) assert(results === Map(0 -> 42, 1 -> 43)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run shuffle with map stage failure") { @@ -552,7 +561,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.toSet === Set(0)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } /** @@ -586,7 +595,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar class FailureRecordingJobListener() extends JobListener { var failureMessage: String = _ override def taskSucceeded(index: Int, result: Any) {} - override def jobFailed(exception: Exception) = { failureMessage = exception.getMessage } + override def jobFailed(exception: Exception): Unit = { failureMessage = exception.getMessage } } val listener1 = new FailureRecordingJobListener() val listener2 = new FailureRecordingJobListener() @@ -606,7 +615,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("run trivial shuffle with out-of-band failure and retry") { @@ -629,7 +638,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar Array(makeBlockManagerId("hostC"), makeBlockManagerId("hostB"))) complete(taskSets(2), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("recursive shuffle failures") { @@ -658,7 +667,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar complete(taskSets(4), Seq((Success, makeMapStatus("hostA", 1)))) complete(taskSets(5), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("cached post-shuffle") { @@ -690,7 +699,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar complete(taskSets(3), Seq((Success, makeMapStatus("hostD", 1)))) complete(taskSets(4), Seq((Success, 42))) assert(results === Map(0 -> 42)) - assertDataStructuresEmpty + assertDataStructuresEmpty() } test("misbehaved accumulator should not crash DAGScheduler and SparkContext") { @@ -742,7 +751,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar } test("accumulator not calculated for resubmitted result stage") { - //just for register + // just for register val accum = new Accumulator[Int](0, AccumulatorParam.IntAccumulatorParam) val finalRdd = new MyRDD(sc, 1, Nil) submit(finalRdd, Array(0)) @@ -754,7 +763,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar assert(accVal === 1) - assertDataStructuresEmpty + assertDataStructuresEmpty() } /** @@ -774,7 +783,7 @@ class DAGSchedulerSuite extends FunSuiteLike with BeforeAndAfter with LocalSpar private def makeBlockManagerId(host: String): BlockManagerId = BlockManagerId("exec-" + host, host, 12345) - private def assertDataStructuresEmpty = { + private def assertDataStructuresEmpty(): Unit = { assert(scheduler.activeJobs.isEmpty) assert(scheduler.failedStages.isEmpty) assert(scheduler.jobIdToActiveJob.isEmpty) diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index 30ee63e78d9d8..6d25edb7d20dc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -268,7 +268,7 @@ class EventLoggingListenerSuite extends FunSuite with LocalSparkContext with Bef object EventLoggingListenerSuite { /** Get a SparkConf with event logging enabled. */ - def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None) = { + def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None): SparkConf = { val conf = new SparkConf conf.set("spark.eventLog.enabled", "true") conf.set("spark.eventLog.testing", "true") @@ -280,5 +280,5 @@ object EventLoggingListenerSuite { conf } - def getUniqueApplicationId = "test-" + System.currentTimeMillis + def getUniqueApplicationId: String = "test-" + System.currentTimeMillis } diff --git a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala index 6b75c98839e03..9b92f8de56759 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/NotSerializableFakeTask.scala @@ -24,7 +24,9 @@ import org.apache.spark.TaskContext /** * A Task implementation that fails to serialize. */ -private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int) extends Task[Array[Byte]](stageId, 0) { +private[spark] class NotSerializableFakeTask(myId: Int, stageId: Int) + extends Task[Array[Byte]](stageId, 0) { + override def runTask(context: TaskContext): Array[Byte] = Array.empty[Byte] override def preferredLocations: Seq[TaskLocation] = Seq[TaskLocation]() 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 627c9a4ddfffc..825c616c0c3e0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -85,7 +85,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers val stopperReturned = new Semaphore(0) class BlockingListener extends SparkListener { - override def onJobEnd(jobEnd: SparkListenerJobEnd) = { + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { listenerStarted.release() listenerWait.acquire() drained = true @@ -206,8 +206,9 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers sc.addSparkListener(new StatsReportListener) // just to make sure some of the tasks take a noticeable amount of time val w = { i: Int => - if (i == 0) + if (i == 0) { Thread.sleep(100) + } i } @@ -247,12 +248,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers */ taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => - taskMetrics.resultSize should be > (0l) + taskMetrics.resultSize should be > (0L) if (stageInfo.rddInfos.exists(info => info.name == d2.name || info.name == d3.name)) { taskMetrics.inputMetrics should not be ('defined) taskMetrics.outputMetrics should not be ('defined) taskMetrics.shuffleWriteMetrics should be ('defined) - taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) + taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0L) } if (stageInfo.rddInfos.exists(_.name == d4.name)) { taskMetrics.shuffleReadMetrics should be ('defined) @@ -260,7 +261,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers sm.totalBlocksFetched should be (128) sm.localBlocksFetched should be (128) sm.remoteBlocksFetched should be (0) - sm.remoteBytesRead should be (0l) + sm.remoteBytesRead should be (0L) } } } @@ -406,12 +407,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers val startedGettingResultTasks = new mutable.HashSet[Int]() val endedTasks = new mutable.HashSet[Int]() - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = synchronized { startedTasks += taskStart.taskInfo.index notify() } - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = synchronized { endedTasks += taskEnd.taskInfo.index notify() } @@ -425,7 +426,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers * A simple listener that throws an exception on job end. */ private class BadListener extends SparkListener { - override def onJobEnd(jobEnd: SparkListenerJobEnd) = { throw new Exception } + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { throw new Exception } } } @@ -438,10 +439,10 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with Matchers */ private class BasicJobCounter extends SparkListener { var count = 0 - override def onJobEnd(job: SparkListenerJobEnd) = count += 1 + override def onJobEnd(job: SparkListenerJobEnd): Unit = count += 1 } private class ListenerThatAcceptsSparkConf(conf: SparkConf) extends SparkListener { var count = 0 - override def onJobEnd(job: SparkListenerJobEnd) = count += 1 + override def onJobEnd(job: SparkListenerJobEnd): Unit = count += 1 } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index add13f5b21765..ffa4381969b68 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.scheduler -import java.util.Properties - import org.scalatest.FunSuite import org.apache.spark._ @@ -27,7 +25,7 @@ class FakeSchedulerBackend extends SchedulerBackend { def start() {} def stop() {} def reviveOffers() {} - def defaultParallelism() = 1 + def defaultParallelism(): Int = 1 } class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Logging { @@ -115,7 +113,8 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin } val numFreeCores = 1 taskScheduler.setDAGScheduler(dagScheduler) - var taskSet = new TaskSet(Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + val taskSet = new TaskSet( + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) val multiCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", taskCpus), new WorkerOffer("executor1", "host1", numFreeCores)) taskScheduler.submitTasks(taskSet) @@ -123,7 +122,8 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin assert(0 === taskDescriptions.length) // Now check that we can still submit tasks - // Even if one of the tasks has not-serializable tasks, the other task set should still be processed without error + // Even if one of the tasks has not-serializable tasks, the other task set should + // still be processed without error taskScheduler.submitTasks(taskSet) taskScheduler.submitTasks(FakeTask.createTaskSet(1)) taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 12330d8f63c40..716d12c0762cf 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -67,7 +67,7 @@ object FakeRackUtil { hostToRack(host) = rack } - def getRackForHost(host: String) = { + def getRackForHost(host: String): Option[String] = { hostToRack.get(host) } } @@ -327,8 +327,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // First offer host1: first task should be chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) - // After this, nothing should get chosen, because we have separated tasks with unavailable preference - // from the noPrefPendingTasks + // After this, nothing should get chosen, because we have separated tasks with unavailable + // preference from the noPrefPendingTasks assert(manager.resourceOffer("exec1", "host1", ANY) === None) // Now mark host2 as dead @@ -499,7 +499,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sched.addExecutor("execC", "host2") manager.executorAdded() // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL and ANY - assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY))) + assert(manager.myLocalityLevels.sameElements( + Array(PROCESS_LOCAL, NODE_LOCAL, NO_PREF, RACK_LOCAL, ANY))) // test if the valid locality is recomputed when the executor is lost sched.removeExecutor("execC") manager.executorLost("execC", "host2") @@ -569,7 +570,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) - val taskSet = new TaskSet(Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) + val taskSet = new TaskSet( + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 0, 0, 0, null) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) intercept[TaskNotSerializableException] { @@ -582,7 +584,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val conf = new SparkConf().set("spark.driver.maxResultSize", "2m") sc = new SparkContext("local", "test", conf) - def genBytes(size: Int) = { (x: Int) => + def genBytes(size: Int): (Int) => Array[Byte] = { (x: Int) => val bytes = Array.ofDim[Byte](size) scala.util.Random.nextBytes(bytes) bytes @@ -605,7 +607,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("speculative and noPref task should be scheduled after node-local") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + val sched = new FakeTaskScheduler( + sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1", "execA")), Seq(TaskLocation("host2"), TaskLocation("host1")), @@ -629,9 +632,11 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) } - test("node-local tasks should be scheduled right away when there are only node-local and no-preference tasks") { + test("node-local tasks should be scheduled right away " + + "when there are only node-local and no-preference tasks") { sc = new SparkContext("local", "test") - val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) + val sched = new FakeTaskScheduler( + sc, ("execA", "host1"), ("execB", "host2"), ("execC", "host3")) val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1")), Seq(TaskLocation("host2")), @@ -650,7 +655,8 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execA", "host3", NO_PREF).get.index === 2) } - test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") { + test("SPARK-4939: node-local tasks should be scheduled right after process-local tasks finished") + { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) val taskSet = FakeTask.createTaskSet(4, diff --git a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala index f1a4380d349b3..a311512e82c5e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendSuite.scala @@ -63,16 +63,18 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Mo // uri is null. val executorInfo = mesosSchedulerBackend.createExecutorInfo("test-id") - assert(executorInfo.getCommand.getValue === s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}") + assert(executorInfo.getCommand.getValue === + s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}") // uri exists. conf.set("spark.executor.uri", "hdfs:///test-app-1.0.0.tgz") val executorInfo1 = mesosSchedulerBackend.createExecutorInfo("test-id") - assert(executorInfo1.getCommand.getValue === s"cd test-app-1*; ./bin/spark-class ${classOf[MesosExecutorBackend].getName}") + assert(executorInfo1.getCommand.getValue === + s"cd test-app-1*; ./bin/spark-class ${classOf[MesosExecutorBackend].getName}") } test("mesos resource offers result in launching tasks") { - def createOffer(id: Int, mem: Int, cpu: Int) = { + def createOffer(id: Int, mem: Int, cpu: Int): Offer = { val builder = Offer.newBuilder() builder.addResourcesBuilder() .setName("mem") @@ -82,8 +84,10 @@ class MesosSchedulerBackendSuite extends FunSuite with LocalSparkContext with Mo .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(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 = mock[SchedulerDriver] diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index 6198df84fab3d..b070a54aa989b 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -106,7 +106,9 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { check(mutable.HashMap(1 -> "one", 2 -> "two")) check(mutable.HashMap("one" -> 1, "two" -> 2)) check(List(Some(mutable.HashMap(1->1, 2->2)), None, Some(mutable.HashMap(3->4)))) - check(List(mutable.HashMap("one" -> 1, "two" -> 2),mutable.HashMap(1->"one",2->"two",3->"three"))) + check(List( + mutable.HashMap("one" -> 1, "two" -> 2), + mutable.HashMap(1->"one",2->"two",3->"three"))) } test("ranges") { @@ -169,7 +171,10 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { test("kryo with collect") { val control = 1 :: 2 :: Nil - val result = sc.parallelize(control, 2).map(new ClassWithoutNoArgConstructor(_)).collect().map(_.x) + val result = sc.parallelize(control, 2) + .map(new ClassWithoutNoArgConstructor(_)) + .collect() + .map(_.x) assert(control === result.toSeq) } @@ -237,7 +242,7 @@ class KryoSerializerSuite extends FunSuite with SharedSparkContext { // Set a special, broken ClassLoader and make sure we get an exception on deserialization ser.setDefaultClassLoader(new ClassLoader() { - override def loadClass(name: String) = throw new UnsupportedOperationException + override def loadClass(name: String): Class[_] = throw new UnsupportedOperationException }) intercept[UnsupportedOperationException] { ser.newInstance().deserialize[ClassLoaderTestingObject](bytes) @@ -287,14 +292,14 @@ object KryoTest { class ClassWithNoArgConstructor { var x: Int = 0 - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case c: ClassWithNoArgConstructor => x == c.x case _ => false } } class ClassWithoutNoArgConstructor(val x: Int) { - override def equals(other: Any) = other match { + override def equals(other: Any): Boolean = other match { case c: ClassWithoutNoArgConstructor => x == c.x case _ => false } diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala index d037e2c19a64d..433fd6bb4a11d 100644 --- a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala @@ -24,14 +24,16 @@ import org.apache.spark.rdd.RDD /* A trivial (but unserializable) container for trivial functions */ class UnserializableClass { - def op[T](x: T) = x.toString + def op[T](x: T): String = x.toString - def pred[T](x: T) = x.toString.length % 2 == 0 + def pred[T](x: T): Boolean = x.toString.length % 2 == 0 } class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContext { - def fixture = (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) + def fixture: (RDD[String], UnserializableClass) = { + (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) + } test("throws expected serialization exceptions on actions") { val (data, uc) = fixture diff --git a/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala index 0ade1bab18d7e..963264cef3a71 100644 --- a/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala +++ b/core/src/test/scala/org/apache/spark/serializer/TestSerializer.scala @@ -27,7 +27,7 @@ import scala.reflect.ClassTag * A serializer implementation that always return a single element in a deserialization stream. */ class TestSerializer extends Serializer { - override def newInstance() = new TestSerializerInstance + override def newInstance(): TestSerializerInstance = new TestSerializerInstance } @@ -36,7 +36,8 @@ class TestSerializerInstance extends SerializerInstance { override def serializeStream(s: OutputStream): SerializationStream = ??? - override def deserializeStream(s: InputStream) = new TestDeserializationStream + override def deserializeStream(s: InputStream): TestDeserializationStream = + new TestDeserializationStream override def deserialize[T: ClassTag](bytes: ByteBuffer): T = ??? diff --git a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala index b834dc0e735eb..7d76435cd75e7 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/hash/HashShuffleManagerSuite.scala @@ -85,8 +85,8 @@ class HashShuffleManagerSuite extends FunSuite with LocalSparkContext { // Now comes the test : // Write to shuffle 3; and close it, but before registering it, check if the file lengths for // previous task (forof shuffle1) is the same as 'segments'. Earlier, we were inferring length - // of block based on remaining data in file : which could mess things up when there is concurrent read - // and writes happening to the same shuffle group. + // of block based on remaining data in file : which could mess things up when there is + // concurrent read and writes happening to the same shuffle group. val shuffle3 = shuffleBlockManager.forMapTask(1, 3, 1, new JavaSerializer(testConf), new ShuffleWriteMetrics) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 6dc5bc4cb08c4..545722b050ee8 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -60,7 +60,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach // Implicitly convert strings to BlockIds for test clarity. implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) - def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) + def rdd(rddId: Int, splitId: Int): RDDBlockId = RDDBlockId(rddId, splitId) private def makeBlockManager( maxMem: Long, @@ -107,8 +107,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach test("StorageLevel object caching") { val level1 = StorageLevel(false, false, false, false, 3) - val level2 = StorageLevel(false, false, false, false, 3) // this should return the same object as level1 - val level3 = StorageLevel(false, false, false, false, 2) // this should return a different object + // this should return the same object as level1 + val level2 = StorageLevel(false, false, false, false, 3) + // this should return a different object + val level3 = StorageLevel(false, false, false, false, 2) assert(level2 === level1, "level2 is not same as level1") assert(level2.eq(level1), "level2 is not the same object as level1") assert(level3 != level1, "level3 is same as level1") @@ -802,7 +804,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach // Create a non-trivial (not all zeros) byte array var counter = 0.toByte - def incr = {counter = (counter + 1).toByte; counter;} + def incr: Byte = {counter = (counter + 1).toByte; counter;} val bytes = Array.fill[Byte](1000)(incr) val byteBuffer = ByteBuffer.wrap(bytes) @@ -956,8 +958,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach store.putIterator("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) // getLocations and getBlockStatus should yield the same locations - assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size === 3) - assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size === 1) + assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size + === 3) + assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size + === 1) // insert some more blocks store.putIterator("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) @@ -965,8 +969,10 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach store.putIterator("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // getLocations and getBlockStatus should yield the same locations - assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size === 1) - assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = true).size === 3) + assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size + === 1) + assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = true).size + === 3) val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) blockIds.foreach { blockId => @@ -1090,8 +1096,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val memoryStore = store.memoryStore val smallList = List.fill(40)(new Array[Byte](100)) val bigList = List.fill(40)(new Array[Byte](1000)) - def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] - def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] assert(memoryStore.currentUnrollMemoryForThisThread === 0) // Unroll with plenty of space. This should succeed and cache both blocks. @@ -1144,8 +1150,8 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val diskStore = store.diskStore val smallList = List.fill(40)(new Array[Byte](100)) val bigList = List.fill(40)(new Array[Byte](1000)) - def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] - def bigIterator = bigList.iterator.asInstanceOf[Iterator[Any]] + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] + def bigIterator: Iterator[Any] = bigList.iterator.asInstanceOf[Iterator[Any]] assert(memoryStore.currentUnrollMemoryForThisThread === 0) store.putIterator("b1", smallIterator, memAndDisk) @@ -1187,7 +1193,7 @@ class BlockManagerSuite extends FunSuite with Matchers with BeforeAndAfterEach val memOnly = StorageLevel.MEMORY_ONLY val memoryStore = store.memoryStore val smallList = List.fill(40)(new Array[Byte](100)) - def smallIterator = smallList.iterator.asInstanceOf[Iterator[Any]] + def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] assert(memoryStore.currentUnrollMemoryForThisThread === 0) // All unroll memory used is released because unrollSafely returned an array diff --git a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala index 82a82e23eecf2..b47157f8331cc 100644 --- a/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/LocalDirsSuite.scala @@ -47,7 +47,7 @@ class LocalDirsSuite extends FunSuite with BeforeAndAfter { assert(!new File("/NONEXISTENT_DIR").exists()) // SPARK_LOCAL_DIRS is a valid directory: class MySparkConf extends SparkConf(false) { - override def getenv(name: String) = { + override def getenv(name: String): String = { if (name == "SPARK_LOCAL_DIRS") System.getProperty("java.io.tmpdir") else super.getenv(name) } 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 0d155982a8c54..1cb594633f331 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISeleniumSuite.scala @@ -137,7 +137,7 @@ class UISeleniumSuite extends FunSuite with WebBrowser with Matchers with Before new SparkContext(conf) } - def hasKillLink = find(className("kill-link")).isDefined + def hasKillLink: Boolean = find(className("kill-link")).isDefined def runSlowJob(sc: SparkContext) { sc.parallelize(1 to 10).map{x => Thread.sleep(10000); x}.countAsync() } 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 c0c28cb60e21d..21d8267114133 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 @@ -269,7 +269,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc val taskType = Utils.getFormattedClassName(new ShuffleMapTask(0)) val execId = "exe-1" - def makeTaskMetrics(base: Int) = { + def makeTaskMetrics(base: Int): TaskMetrics = { val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() val shuffleWriteMetrics = new ShuffleWriteMetrics() @@ -291,7 +291,7 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Matc taskMetrics } - def makeTaskInfo(taskId: Long, finishTime: Int = 0) = { + def makeTaskInfo(taskId: Long, finishTime: Int = 0): TaskInfo = { val taskInfo = new TaskInfo(taskId, 0, 1, 0L, execId, "host1", TaskLocality.NODE_LOCAL, false) taskInfo.finishTime = finishTime diff --git a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala index e1bc1379b5d80..3744e479d2f05 100644 --- a/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/storage/StorageTabSuite.scala @@ -107,7 +107,8 @@ class StorageTabSuite extends FunSuite with BeforeAndAfter { val myRddInfo0 = rddInfo0 val myRddInfo1 = rddInfo1 val myRddInfo2 = rddInfo2 - val stageInfo0 = new StageInfo(0, 0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") + val stageInfo0 = new StageInfo( + 0, 0, "0", 100, Seq(myRddInfo0, myRddInfo1, myRddInfo2), "details") bus.postToAll(SparkListenerBlockManagerAdded(1L, bm1, 1000L)) bus.postToAll(SparkListenerStageSubmitted(stageInfo0)) assert(storageListener._rddInfoMap.size === 3) diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index 054ef54e746a5..c47162779bbba 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -83,7 +83,7 @@ object TestObject { class TestClass extends Serializable { var x = 5 - def getX = x + def getX: Int = x def run(): Int = { var nonSer = new NonSerializable @@ -95,7 +95,7 @@ class TestClass extends Serializable { } class TestClassWithoutDefaultConstructor(x: Int) extends Serializable { - def getX = x + def getX: Int = x def run(): Int = { var nonSer = new NonSerializable @@ -164,7 +164,7 @@ object TestObjectWithNesting { } class TestClassWithNesting(val y: Int) extends Serializable { - def getY = y + def getY: Int = y def run(): Int = { var nonSer = new NonSerializable diff --git a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala index 43b6a405cb68c..c05317534cddf 100644 --- a/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/FileAppenderSuite.scala @@ -109,7 +109,8 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { // verify whether the earliest file has been deleted val rolledOverFiles = allGeneratedFiles.filter { _ != testFile.toString }.toArray.sorted - logInfo(s"All rolled over files generated:${rolledOverFiles.size}\n" + rolledOverFiles.mkString("\n")) + logInfo(s"All rolled over files generated:${rolledOverFiles.size}\n" + + rolledOverFiles.mkString("\n")) assert(rolledOverFiles.size > 2) val earliestRolledOverFile = rolledOverFiles.head val existingRolledOverFiles = RollingFileAppender.getSortedRolledOverFiles( @@ -135,7 +136,7 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { val testOutputStream = new PipedOutputStream() val testInputStream = new PipedInputStream(testOutputStream) val appender = FileAppender(testInputStream, testFile, conf) - //assert(appender.getClass === classTag[ExpectedAppender].getClass) + // assert(appender.getClass === classTag[ExpectedAppender].getClass) assert(appender.getClass.getSimpleName === classTag[ExpectedAppender].runtimeClass.getSimpleName) if (appender.isInstanceOf[RollingFileAppender]) { @@ -153,9 +154,11 @@ class FileAppenderSuite extends FunSuite with BeforeAndAfter with Logging { import RollingFileAppender._ - def rollingStrategy(strategy: String) = Seq(STRATEGY_PROPERTY -> strategy) - def rollingSize(size: String) = Seq(SIZE_PROPERTY -> size) - def rollingInterval(interval: String) = Seq(INTERVAL_PROPERTY -> interval) + def rollingStrategy(strategy: String): Seq[(String, String)] = + Seq(STRATEGY_PROPERTY -> strategy) + def rollingSize(size: String): Seq[(String, String)] = Seq(SIZE_PROPERTY -> size) + def rollingInterval(interval: String): Seq[(String, String)] = + Seq(INTERVAL_PROPERTY -> interval) val msInDay = 24 * 60 * 60 * 1000L val msInHour = 60 * 60 * 1000L diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index 72e81f3f1a884..403dcb03bd6e5 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -71,7 +71,7 @@ class NextIteratorSuite extends FunSuite with Matchers { class StubIterator(ints: Buffer[Int]) extends NextIterator[Int] { var closeCalled = 0 - override def getNext() = { + override def getNext(): Int = { if (ints.size == 0) { finished = true 0 diff --git a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala index 7424c2e91d4f2..67a9f75ff2187 100644 --- a/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/SizeEstimatorSuite.scala @@ -98,8 +98,10 @@ class SizeEstimatorSuite // If an array contains the *same* element many times, we should only count it once. val d1 = new DummyClass1 - assertResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) // 10 pointers plus 8-byte object - assertResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // 100 pointers plus 8-byte object + // 10 pointers plus 8-byte object + assertResult(72)(SizeEstimator.estimate(Array.fill(10)(d1))) + // 100 pointers plus 8-byte object + assertResult(432)(SizeEstimator.estimate(Array.fill(100)(d1))) // Same thing with huge array containing the same element many times. Note that this won't // return exactly 4032 because it can't tell that *all* the elements will equal the first diff --git a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala index c1c605cdb487b..8b72fe665c214 100644 --- a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala @@ -63,7 +63,7 @@ class TimeStampedHashMapSuite extends FunSuite { assert(map1.getTimestamp("k1").get < threshTime1) assert(map1.getTimestamp("k2").isDefined) assert(map1.getTimestamp("k2").get >= threshTime1) - map1.clearOldValues(threshTime1) //should only clear k1 + map1.clearOldValues(threshTime1) // should only clear k1 assert(map1.get("k1") === None) assert(map1.get("k2").isDefined) } @@ -93,7 +93,7 @@ class TimeStampedHashMapSuite extends FunSuite { assert(map1.getTimestamp("k1").get < threshTime1) assert(map1.getTimestamp("k2").isDefined) assert(map1.getTimestamp("k2").get >= threshTime1) - map1.clearOldValues(threshTime1) //should only clear k1 + map1.clearOldValues(threshTime1) // should only clear k1 assert(map1.get("k1") === None) assert(map1.get("k2").isDefined) } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 5d93086082189..449fb87f111c4 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -106,7 +106,7 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { val second = 1000 val minute = second * 60 val hour = minute * 60 - def str = Utils.msDurationToString(_) + def str: (Long) => String = Utils.msDurationToString(_) val sep = new DecimalFormatSymbols(Locale.getDefault()).getDecimalSeparator() @@ -199,7 +199,8 @@ class UtilsSuite extends FunSuite with ResetSystemProperties { test("doesDirectoryContainFilesNewerThan") { // create some temporary directories and files val parent: File = Utils.createTempDir() - val child1: File = Utils.createTempDir(parent.getCanonicalPath) // The parent directory has two child directories + // The parent directory has two child directories + val child1: File = Utils.createTempDir(parent.getCanonicalPath) val child2: File = Utils.createTempDir(parent.getCanonicalPath) val child3: File = Utils.createTempDir(child1.getCanonicalPath) // set the last modified time of child1 to 30 secs old diff --git a/core/src/test/scala/org/apache/spark/util/VectorSuite.scala b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala index 794a55d61750b..ce2968728a996 100644 --- a/core/src/test/scala/org/apache/spark/util/VectorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/VectorSuite.scala @@ -27,7 +27,7 @@ import org.scalatest.FunSuite @deprecated("suppress compile time deprecation warning", "1.0.0") class VectorSuite extends FunSuite { - def verifyVector(vector: Vector, expectedLength: Int) = { + def verifyVector(vector: Vector, expectedLength: Int): Unit = { assert(vector.length == expectedLength) assert(vector.elements.min > 0.0) assert(vector.elements.max < 1.0) 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 48f79ea651018..dff8f3ddc816f 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 @@ -185,7 +185,7 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { // reduceByKey val rdd = sc.parallelize(1 to 10).map(i => (i%2, 1)) - val result1 = rdd.reduceByKey(_+_).collect() + val result1 = rdd.reduceByKey(_ + _).collect() assert(result1.toSet === Set[(Int, Int)]((0, 5), (1, 5))) // groupByKey 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 72d96798b1141..9ff067f86af44 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 @@ -553,10 +553,10 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - def createCombiner(i: String) = ArrayBuffer[String](i) - def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i - def mergeCombiners(buffer1: ArrayBuffer[String], buffer2: ArrayBuffer[String]) = - buffer1 ++= buffer2 + def createCombiner(i: String): ArrayBuffer[String] = ArrayBuffer[String](i) + def mergeValue(buffer: ArrayBuffer[String], i: String): ArrayBuffer[String] = buffer += i + def mergeCombiners(buffer1: ArrayBuffer[String], buffer2: ArrayBuffer[String]) + : ArrayBuffer[String] = buffer1 ++= buffer2 val agg = new Aggregator[String, String, ArrayBuffer[String]]( createCombiner _, mergeValue _, mergeCombiners _) @@ -633,14 +633,17 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - def createCombiner(i: Int) = ArrayBuffer[Int](i) - def mergeValue(buffer: ArrayBuffer[Int], i: Int) = buffer += i - def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]) = buf1 ++= buf2 + def createCombiner(i: Int): ArrayBuffer[Int] = ArrayBuffer[Int](i) + def mergeValue(buffer: ArrayBuffer[Int], i: Int): ArrayBuffer[Int] = buffer += i + def mergeCombiners(buf1: ArrayBuffer[Int], buf2: ArrayBuffer[Int]): ArrayBuffer[Int] = { + buf1 ++= buf2 + } val agg = new Aggregator[Int, Int, ArrayBuffer[Int]](createCombiner, mergeValue, mergeCombiners) val sorter = new ExternalSorter[Int, Int, ArrayBuffer[Int]](Some(agg), None, None, None) - sorter.insertAll((1 to 100000).iterator.map(i => (i, i)) ++ Iterator((Int.MaxValue, Int.MaxValue))) + sorter.insertAll( + (1 to 100000).iterator.map(i => (i, i)) ++ Iterator((Int.MaxValue, Int.MaxValue))) val it = sorter.iterator while (it.hasNext) { @@ -654,9 +657,10 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe conf.set("spark.shuffle.memoryFraction", "0.001") sc = new SparkContext("local-cluster[1,1,512]", "test", conf) - def createCombiner(i: String) = ArrayBuffer[String](i) - def mergeValue(buffer: ArrayBuffer[String], i: String) = buffer += i - def mergeCombiners(buf1: ArrayBuffer[String], buf2: ArrayBuffer[String]) = buf1 ++= buf2 + def createCombiner(i: String): ArrayBuffer[String] = ArrayBuffer[String](i) + def mergeValue(buffer: ArrayBuffer[String], i: String): ArrayBuffer[String] = buffer += i + def mergeCombiners(buf1: ArrayBuffer[String], buf2: ArrayBuffer[String]): ArrayBuffer[String] = + buf1 ++= buf2 val agg = new Aggregator[String, String, ArrayBuffer[String]]( createCombiner, mergeValue, mergeCombiners) @@ -720,7 +724,7 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe // Using wrongOrdering to show integer overflow introduced exception. val rand = new Random(100L) val wrongOrdering = new Ordering[String] { - override def compare(a: String, b: String) = { + override def compare(a: String, b: String): Int = { val h1 = if (a == null) 0 else a.hashCode() val h2 = if (b == null) 0 else b.hashCode() h1 - h2 @@ -742,9 +746,10 @@ class ExternalSorterSuite extends FunSuite with LocalSparkContext with PrivateMe // Using aggregation and external spill to make sure ExternalSorter using // partitionKeyComparator. - def createCombiner(i: String) = ArrayBuffer(i) - def mergeValue(c: ArrayBuffer[String], i: String) = c += i - def mergeCombiners(c1: ArrayBuffer[String], c2: ArrayBuffer[String]) = c1 ++= c2 + def createCombiner(i: String): ArrayBuffer[String] = ArrayBuffer(i) + def mergeValue(c: ArrayBuffer[String], i: String): ArrayBuffer[String] = c += i + def mergeCombiners(c1: ArrayBuffer[String], c2: ArrayBuffer[String]): ArrayBuffer[String] = + c1 ++= c2 val agg = new Aggregator[String, String, ArrayBuffer[String]]( createCombiner, mergeValue, mergeCombiners) diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index ef7178bcdf5c2..03f5f2d1b8528 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -28,7 +28,7 @@ import scala.language.reflectiveCalls class XORShiftRandomSuite extends FunSuite with Matchers { - def fixture = new { + def fixture: Object {val seed: Long; val hundMil: Int; val xorRand: XORShiftRandom} = new { val seed = 1L val xorRand = new XORShiftRandom(seed) val hundMil = 1e8.toInt From fc17661475443d9f0a8d28e3439feeb7a7bca67b Mon Sep 17 00:00:00 2001 From: lewuathe Date: Sun, 12 Apr 2015 22:17:16 -0700 Subject: [PATCH 054/128] [SPARK-6643][MLLIB] Implement StandardScalerModel missing methods This is the sub-task of SPARK-6254. Wrap missing method for `StandardScalerModel`. Author: lewuathe Closes #5310 from Lewuathe/SPARK-6643 and squashes the following commits: fafd690 [lewuathe] Fix for lint-python bd31a64 [lewuathe] Merge branch 'master' into SPARK-6643 578f5ee [lewuathe] Remove unnecessary class a38f155 [lewuathe] Merge master 66bb2ab [lewuathe] Fix typos 82683a0 [lewuathe] [SPARK-6643] Implement StandardScalerModel missing methods --- .../mllib/api/python/PythonMLLibAPI.scala | 4 ++-- python/pyspark/mllib/feature.py | 16 +++++++++++++ python/pyspark/mllib/tests.py | 24 +++++++++++++++++++ 3 files changed, 42 insertions(+), 2 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 1faa3def0e042..ab15f0f36a14b 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 @@ -450,9 +450,9 @@ private[python] class PythonMLLibAPI extends Serializable { def normalizeVector(p: Double, rdd: JavaRDD[Vector]): JavaRDD[Vector] = { new Normalizer(p).transform(rdd) } - + /** - * Java stub for IDF.fit(). This stub returns a + * Java stub for StandardScaler.fit(). This stub returns a * handle to the Java object instead of the content of the Java object. * Extra care needs to be taken in the Python code to ensure it gets freed on * exit; see the Py4J documentation. diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 3cda1205e1391..8be819aceec24 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -132,6 +132,22 @@ def transform(self, vector): """ return JavaVectorTransformer.transform(self, vector) + def setWithMean(self, withMean): + """ + Setter of the boolean which decides + whether it uses mean or not + """ + self.call("setWithMean", withMean) + return self + + def setWithStd(self, withStd): + """ + Setter of the boolean which decides + whether it uses std or not + """ + self.call("setWithStd", withStd) + return self + class StandardScaler(object): """ diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 3b40158c12b74..8eaddcf8b9b5e 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -44,6 +44,7 @@ from pyspark.mllib.stat import Statistics from pyspark.mllib.feature import Word2Vec from pyspark.mllib.feature import IDF +from pyspark.mllib.feature import StandardScaler from pyspark.serializers import PickleSerializer from pyspark.sql import SQLContext from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase @@ -745,6 +746,29 @@ def test_word2vec_get_vectors(self): model = Word2Vec().fit(self.sc.parallelize(data)) self.assertEquals(len(model.getVectors()), 3) + +class StandardScalerTests(PySparkTestCase): + def test_model_setters(self): + data = [ + [1.0, 2.0, 3.0], + [2.0, 3.0, 4.0], + [3.0, 4.0, 5.0] + ] + model = StandardScaler().fit(self.sc.parallelize(data)) + self.assertIsNotNone(model.setWithMean(True)) + self.assertIsNotNone(model.setWithStd(True)) + self.assertEqual(model.transform([1.0, 2.0, 3.0]), DenseVector([-1.0, -1.0, -1.0])) + + def test_model_transform(self): + data = [ + [1.0, 2.0, 3.0], + [2.0, 3.0, 4.0], + [3.0, 4.0, 5.0] + ] + model = StandardScaler().fit(self.sc.parallelize(data)) + self.assertEqual(model.transform([1.0, 2.0, 3.0]), DenseVector([1.0, 2.0, 3.0])) + + if __name__ == "__main__": if not _have_scipy: print "NOTE: Skipping SciPy tests as it does not seem to be installed" From d3792f54974e16cbe8f10b3091d248e0bdd48986 Mon Sep 17 00:00:00 2001 From: "Joseph K. Bradley" Date: Sun, 12 Apr 2015 22:38:27 -0700 Subject: [PATCH 055/128] [SPARK-4081] [mllib] VectorIndexer **Ready for review!** Since the original PR, I moved the code to the spark.ml API and renamed this to VectorIndexer. This introduces a VectorIndexer class which does the following: * VectorIndexer.fit(): collect statistics about how many values each feature in a dataset (RDD[Vector]) can take (limited by maxCategories) * Feature which exceed maxCategories are declared continuous, and the Model will treat them as such. * VectorIndexerModel.transform(): Convert categorical feature values to corresponding 0-based indices Design notes: * This maintains sparsity in vectors by ensuring that categorical feature value 0.0 gets index 0. * This does not yet support transforming data with new (unknown) categorical feature values. That can be added later. * This is necessary for DecisionTree and tree ensembles. Reviewers: Please check my use of metadata and my unit tests for it; I'm not sure if I covered everything in the tests. Other notes: * This also adds a public toMetadata method to AttributeGroup (for simpler construction of metadata). CC: mengxr Author: Joseph K. Bradley Closes #3000 from jkbradley/indexer and squashes the following commits: 5956d91 [Joseph K. Bradley] minor cleanups f5c57a8 [Joseph K. Bradley] added Java test suite 643b444 [Joseph K. Bradley] removed FeatureTests 02236c3 [Joseph K. Bradley] Updated VectorIndexer, ready for PR 286d221 [Joseph K. Bradley] Reworked DatasetIndexer for spark.ml API, and renamed it to VectorIndexer 12e6cf2 [Joseph K. Bradley] Merge remote-tracking branch 'upstream/master' into indexer 6d8f3f1 [Joseph K. Bradley] Added partly done DatasetIndexer to spark.ml 6a2f553 [Joseph K. Bradley] Updated TODO for allowUnknownCategories 3f041f8 [Joseph K. Bradley] Final cleanups for DatasetIndexer 038b9e3 [Joseph K. Bradley] DatasetIndexer now maintains sparsity in SparseVector 3a4a0bd [Joseph K. Bradley] Added another test for DatasetIndexer 2006923 [Joseph K. Bradley] DatasetIndexer now passes tests f409987 [Joseph K. Bradley] partly done with DatasetIndexerSuite 5e7c874 [Joseph K. Bradley] working on DatasetIndexer --- .../scala/org/apache/spark/ml/Pipeline.scala | 3 + .../spark/ml/attribute/AttributeGroup.scala | 21 +- .../spark/ml/feature/VectorIndexer.scala | 393 ++++++++++++++++++ .../org/apache/spark/ml/param/params.scala | 20 +- .../ml/feature/JavaVectorIndexerSuite.java | 70 ++++ .../ml/attribute/AttributeGroupSuite.scala | 8 +- .../spark/ml/feature/NormalizerSuite.scala | 7 +- .../spark/ml/feature/VectorIndexerSuite.scala | 255 ++++++++++++ .../apache/spark/ml/util/TestingUtils.scala | 60 +++ 9 files changed, 818 insertions(+), 19 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala create mode 100644 mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java create mode 100644 mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala 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 c4a36103303a2..a455341a1f723 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -47,6 +47,9 @@ abstract class PipelineStage extends Serializable with Logging { /** * Derives the output schema from the input schema and parameters, optionally with logging. + * + * This should be optimistic. If it is unclear whether the schema will be valid, then it should + * be assumed valid until proven otherwise. */ protected def transformSchema( schema: StructType, diff --git a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala index 970e6ad5514d1..aa27a668f1695 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/attribute/AttributeGroup.scala @@ -106,7 +106,7 @@ class AttributeGroup private ( def getAttr(attrIndex: Int): Attribute = this(attrIndex) /** Converts to metadata without name. */ - private[attribute] def toMetadata: Metadata = { + private[attribute] def toMetadataImpl: Metadata = { import AttributeKeys._ val bldr = new MetadataBuilder() if (attributes.isDefined) { @@ -142,17 +142,24 @@ class AttributeGroup private ( bldr.build() } - /** Converts to a StructField with some existing metadata. */ - def toStructField(existingMetadata: Metadata): StructField = { - val newMetadata = new MetadataBuilder() + /** Converts to ML metadata with some existing metadata. */ + def toMetadata(existingMetadata: Metadata): Metadata = { + new MetadataBuilder() .withMetadata(existingMetadata) - .putMetadata(AttributeKeys.ML_ATTR, toMetadata) + .putMetadata(AttributeKeys.ML_ATTR, toMetadataImpl) .build() - StructField(name, new VectorUDT, nullable = false, newMetadata) + } + + /** Converts to ML metadata */ + def toMetadata: Metadata = toMetadata(Metadata.empty) + + /** Converts to a StructField with some existing metadata. */ + def toStructField(existingMetadata: Metadata): StructField = { + StructField(name, new VectorUDT, nullable = false, toMetadata(existingMetadata)) } /** Converts to a StructField. */ - def toStructField(): StructField = toStructField(Metadata.empty) + def toStructField: StructField = toStructField(Metadata.empty) override def equals(other: Any): Boolean = { other match { diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala new file mode 100644 index 0000000000000..8760960e19272 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -0,0 +1,393 @@ +/* + * 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.ml.feature + +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.attribute.{BinaryAttribute, NumericAttribute, NominalAttribute, + Attribute, AttributeGroup} +import org.apache.spark.ml.param.{HasInputCol, HasOutputCol, IntParam, ParamMap, Params} +import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, VectorUDT} +import org.apache.spark.sql.{Row, DataFrame} +import org.apache.spark.sql.functions.callUDF +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.util.collection.OpenHashSet + + +/** Private trait for params for VectorIndexer and VectorIndexerModel */ +private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOutputCol { + + /** + * Threshold for the number of values a categorical feature can take. + * If a feature is found to have > maxCategories values, then it is declared continuous. + * + * (default = 20) + */ + val maxCategories = new IntParam(this, "maxCategories", + "Threshold for the number of values a categorical feature can take." + + " If a feature is found to have > maxCategories values, then it is declared continuous.", + Some(20)) + + /** @group getParam */ + def getMaxCategories: Int = get(maxCategories) +} + +/** + * :: AlphaComponent :: + * + * Class for indexing categorical feature columns in a dataset of [[Vector]]. + * + * This has 2 usage modes: + * - Automatically identify categorical features (default behavior) + * - This helps process a dataset of unknown vectors into a dataset with some continuous + * features and some categorical features. The choice between continuous and categorical + * is based upon a maxCategories parameter. + * - Set maxCategories to the maximum number of categorical any categorical feature should have. + * - E.g.: Feature 0 has unique values {-1.0, 0.0}, and feature 1 values {1.0, 3.0, 5.0}. + * If maxCategories = 2, then feature 0 will be declared categorical and use indices {0, 1}, + * and feature 1 will be declared continuous. + * - Index all features, if all features are categorical + * - If maxCategories is set to be very large, then this will build an index of unique + * values for all features. + * - Warning: This can cause problems if features are continuous since this will collect ALL + * unique values to the driver. + * - E.g.: Feature 0 has unique values {-1.0, 0.0}, and feature 1 values {1.0, 3.0, 5.0}. + * If maxCategories >= 3, then both features will be declared categorical. + * + * This returns a model which can transform categorical features to use 0-based indices. + * + * Index stability: + * - This is not guaranteed to choose the same category index across multiple runs. + * - If a categorical feature includes value 0, then this is guaranteed to map value 0 to index 0. + * This maintains vector sparsity. + * - More stability may be added in the future. + * + * TODO: Future extensions: The following functionality is planned for the future: + * - Preserve metadata in transform; if a feature's metadata is already present, do not recompute. + * - Specify certain features to not index, either via a parameter or via existing metadata. + * - Add warning if a categorical feature has only 1 category. + * - Add option for allowing unknown categories. + */ +@AlphaComponent +class VectorIndexer extends Estimator[VectorIndexerModel] with VectorIndexerParams { + + /** @group setParam */ + def setMaxCategories(value: Int): this.type = { + require(value > 1, + s"DatasetIndexer given maxCategories = value, but requires maxCategories > 1.") + set(maxCategories, value) + } + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def fit(dataset: DataFrame, paramMap: ParamMap): VectorIndexerModel = { + transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap + val firstRow = dataset.select(map(inputCol)).take(1) + require(firstRow.length == 1, s"VectorIndexer cannot be fit on an empty dataset.") + val numFeatures = firstRow(0).getAs[Vector](0).size + val vectorDataset = dataset.select(map(inputCol)).map { case Row(v: Vector) => v } + val maxCats = map(maxCategories) + val categoryStats: VectorIndexer.CategoryStats = vectorDataset.mapPartitions { iter => + val localCatStats = new VectorIndexer.CategoryStats(numFeatures, maxCats) + iter.foreach(localCatStats.addVector) + Iterator(localCatStats) + }.reduce((stats1, stats2) => stats1.merge(stats2)) + val model = new VectorIndexerModel(this, map, numFeatures, categoryStats.getCategoryMaps) + Params.inheritValues(map, this, model) + model + } + + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + // We do not transfer feature metadata since we do not know what types of features we will + // produce in transform(). + val map = this.paramMap ++ paramMap + val dataType = new VectorUDT + require(map.contains(inputCol), s"VectorIndexer requires input column parameter: $inputCol") + require(map.contains(outputCol), s"VectorIndexer requires output column parameter: $outputCol") + checkInputColumn(schema, map(inputCol), dataType) + addOutputColumn(schema, map(outputCol), dataType) + } +} + +private object VectorIndexer { + + /** + * Helper class for tracking unique values for each feature. + * + * TODO: Track which features are known to be continuous already; do not update counts for them. + * + * @param numFeatures This class fails if it encounters a Vector whose length is not numFeatures. + * @param maxCategories This class caps the number of unique values collected at maxCategories. + */ + class CategoryStats(private val numFeatures: Int, private val maxCategories: Int) + extends Serializable { + + /** featureValueSets[feature index] = set of unique values */ + private val featureValueSets = + Array.fill[OpenHashSet[Double]](numFeatures)(new OpenHashSet[Double]()) + + /** Merge with another instance, modifying this instance. */ + def merge(other: CategoryStats): CategoryStats = { + featureValueSets.zip(other.featureValueSets).foreach { case (thisValSet, otherValSet) => + otherValSet.iterator.foreach { x => + // Once we have found > maxCategories values, we know the feature is continuous + // and do not need to collect more values for it. + if (thisValSet.size <= maxCategories) thisValSet.add(x) + } + } + this + } + + /** Add a new vector to this index, updating sets of unique feature values */ + def addVector(v: Vector): Unit = { + require(v.size == numFeatures, s"VectorIndexer expected $numFeatures features but" + + s" found vector of size ${v.size}.") + v match { + case dv: DenseVector => addDenseVector(dv) + case sv: SparseVector => addSparseVector(sv) + } + } + + /** + * Based on stats collected, decide which features are categorical, + * and choose indices for categories. + * + * Sparsity: This tries to maintain sparsity by treating value 0.0 specially. + * If a categorical feature takes value 0.0, then value 0.0 is given index 0. + * + * @return Feature value index. Keys are categorical feature indices (column indices). + * Values are mappings from original features values to 0-based category indices. + */ + def getCategoryMaps: Map[Int, Map[Double, Int]] = { + // Filter out features which are declared continuous. + featureValueSets.zipWithIndex.filter(_._1.size <= maxCategories).map { + case (featureValues: OpenHashSet[Double], featureIndex: Int) => + var sortedFeatureValues = featureValues.iterator.filter(_ != 0.0).toArray.sorted + val zeroExists = sortedFeatureValues.length + 1 == featureValues.size + if (zeroExists) { + sortedFeatureValues = 0.0 +: sortedFeatureValues + } + val categoryMap: Map[Double, Int] = sortedFeatureValues.zipWithIndex.toMap + (featureIndex, categoryMap) + }.toMap + } + + private def addDenseVector(dv: DenseVector): Unit = { + var i = 0 + while (i < dv.size) { + if (featureValueSets(i).size <= maxCategories) { + featureValueSets(i).add(dv(i)) + } + i += 1 + } + } + + private def addSparseVector(sv: SparseVector): Unit = { + // TODO: This might be able to handle 0's more efficiently. + var vecIndex = 0 // index into vector + var k = 0 // index into non-zero elements + while (vecIndex < sv.size) { + val featureValue = if (k < sv.indices.length && vecIndex == sv.indices(k)) { + k += 1 + sv.values(k - 1) + } else { + 0.0 + } + if (featureValueSets(vecIndex).size <= maxCategories) { + featureValueSets(vecIndex).add(featureValue) + } + vecIndex += 1 + } + } + } +} + +/** + * :: AlphaComponent :: + * + * Transform categorical features to use 0-based indices instead of their original values. + * - Categorical features are mapped to indices. + * - Continuous features (columns) are left unchanged. + * This also appends metadata to the output column, marking features as Numeric (continuous), + * Nominal (categorical), or Binary (either continuous or categorical). + * + * This maintains vector sparsity. + * + * @param numFeatures Number of features, i.e., length of Vectors which this transforms + * @param categoryMaps Feature value index. Keys are categorical feature indices (column indices). + * Values are maps from original features values to 0-based category indices. + * If a feature is not in this map, it is treated as continuous. + */ +@AlphaComponent +class VectorIndexerModel private[ml] ( + override val parent: VectorIndexer, + override val fittingParamMap: ParamMap, + val numFeatures: Int, + val categoryMaps: Map[Int, Map[Double, Int]]) + extends Model[VectorIndexerModel] with VectorIndexerParams { + + /** + * Pre-computed feature attributes, with some missing info. + * In transform(), set attribute name and other info, if available. + */ + private val partialFeatureAttributes: Array[Attribute] = { + val attrs = new Array[Attribute](numFeatures) + var categoricalFeatureCount = 0 // validity check for numFeatures, categoryMaps + var featureIndex = 0 + while (featureIndex < numFeatures) { + if (categoryMaps.contains(featureIndex)) { + // categorical feature + val featureValues: Array[String] = + categoryMaps(featureIndex).toArray.sortBy(_._1).map(_._1).map(_.toString) + if (featureValues.length == 2) { + attrs(featureIndex) = new BinaryAttribute(index = Some(featureIndex), + values = Some(featureValues)) + } else { + attrs(featureIndex) = new NominalAttribute(index = Some(featureIndex), + isOrdinal = Some(false), values = Some(featureValues)) + } + categoricalFeatureCount += 1 + } else { + // continuous feature + attrs(featureIndex) = new NumericAttribute(index = Some(featureIndex)) + } + featureIndex += 1 + } + require(categoricalFeatureCount == categoryMaps.size, "VectorIndexerModel given categoryMaps" + + s" with keys outside expected range [0,...,numFeatures), where numFeatures=$numFeatures") + attrs + } + + // TODO: Check more carefully about whether this whole class will be included in a closure. + + private val transformFunc: Vector => Vector = { + val sortedCategoricalFeatureIndices = categoryMaps.keys.toArray.sorted + val localVectorMap = categoryMaps + val f: Vector => Vector = { + case dv: DenseVector => + val tmpv = dv.copy + localVectorMap.foreach { case (featureIndex: Int, categoryMap: Map[Double, Int]) => + tmpv.values(featureIndex) = categoryMap(tmpv(featureIndex)) + } + tmpv + case sv: SparseVector => + // We use the fact that categorical value 0 is always mapped to index 0. + val tmpv = sv.copy + var catFeatureIdx = 0 // index into sortedCategoricalFeatureIndices + var k = 0 // index into non-zero elements of sparse vector + while (catFeatureIdx < sortedCategoricalFeatureIndices.length && k < tmpv.indices.length) { + val featureIndex = sortedCategoricalFeatureIndices(catFeatureIdx) + if (featureIndex < tmpv.indices(k)) { + catFeatureIdx += 1 + } else if (featureIndex > tmpv.indices(k)) { + k += 1 + } else { + tmpv.values(k) = localVectorMap(featureIndex)(tmpv.values(k)) + catFeatureIdx += 1 + k += 1 + } + } + tmpv + } + f + } + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + transformSchema(dataset.schema, paramMap, logging = true) + val map = this.paramMap ++ paramMap + val newField = prepOutputField(dataset.schema, map) + val newCol = callUDF(transformFunc, new VectorUDT, dataset(map(inputCol))) + // For now, just check the first row of inputCol for vector length. + val firstRow = dataset.select(map(inputCol)).take(1) + if (firstRow.length != 0) { + val actualNumFeatures = firstRow(0).getAs[Vector](0).size + require(numFeatures == actualNumFeatures, "VectorIndexerModel expected vector of length" + + s" $numFeatures but found length $actualNumFeatures") + } + dataset.withColumn(map(outputCol), newCol.as(map(outputCol), newField.metadata)) + } + + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + val map = this.paramMap ++ paramMap + val dataType = new VectorUDT + require(map.contains(inputCol), + s"VectorIndexerModel requires input column parameter: $inputCol") + require(map.contains(outputCol), + s"VectorIndexerModel requires output column parameter: $outputCol") + checkInputColumn(schema, map(inputCol), dataType) + + val origAttrGroup = AttributeGroup.fromStructField(schema(map(inputCol))) + val origNumFeatures: Option[Int] = if (origAttrGroup.attributes.nonEmpty) { + Some(origAttrGroup.attributes.get.length) + } else { + origAttrGroup.numAttributes + } + require(origNumFeatures.forall(_ == numFeatures), "VectorIndexerModel expected" + + s" $numFeatures features, but input column ${map(inputCol)} had metadata specifying" + + s" ${origAttrGroup.numAttributes.get} features.") + + val newField = prepOutputField(schema, map) + val outputFields = schema.fields :+ newField + StructType(outputFields) + } + + /** + * Prepare the output column field, including per-feature metadata. + * @param schema Input schema + * @param map Parameter map (with this class' embedded parameter map folded in) + * @return Output column field + */ + private def prepOutputField(schema: StructType, map: ParamMap): StructField = { + val origAttrGroup = AttributeGroup.fromStructField(schema(map(inputCol))) + val featureAttributes: Array[Attribute] = if (origAttrGroup.attributes.nonEmpty) { + // Convert original attributes to modified attributes + val origAttrs: Array[Attribute] = origAttrGroup.attributes.get + origAttrs.zip(partialFeatureAttributes).map { + case (origAttr: Attribute, featAttr: BinaryAttribute) => + if (origAttr.name.nonEmpty) { + featAttr.withName(origAttr.name.get) + } else { + featAttr + } + case (origAttr: Attribute, featAttr: NominalAttribute) => + if (origAttr.name.nonEmpty) { + featAttr.withName(origAttr.name.get) + } else { + featAttr + } + case (origAttr: Attribute, featAttr: NumericAttribute) => + origAttr.withIndex(featAttr.index.get) + } + } else { + partialFeatureAttributes + } + val newAttributeGroup = new AttributeGroup(map(outputCol), featureAttributes) + newAttributeGroup.toStructField(schema(map(inputCol)).metadata) + } +} 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 17ece897a6c55..7d5178d0abb2d 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 @@ -198,23 +198,31 @@ trait Params extends Identifiable with Serializable { /** * Check whether the given schema contains an input column. - * @param colName Parameter name for the input column. - * @param dataType SQL DataType of the input column. + * @param colName Input column name + * @param dataType Input column DataType */ protected def checkInputColumn(schema: StructType, colName: String, dataType: DataType): Unit = { val actualDataType = schema(colName).dataType - require(actualDataType.equals(dataType), - s"Input column $colName must be of type $dataType" + - s" but was actually $actualDataType. Column param description: ${getParam(colName)}") + require(actualDataType.equals(dataType), s"Input column $colName must be of type $dataType" + + s" but was actually $actualDataType. Column param description: ${getParam(colName)}") } + /** + * Add an output column to the given schema. + * This fails if the given output column already exists. + * @param schema Initial schema (not modified) + * @param colName Output column name. If this column name is an empy String "", this method + * returns the initial schema, unchanged. This allows users to disable output + * columns. + * @param dataType Output column DataType + */ protected def addOutputColumn( schema: StructType, colName: String, dataType: DataType): StructType = { if (colName.length == 0) return schema val fieldNames = schema.fieldNames - require(!fieldNames.contains(colName), s"Prediction column $colName already exists.") + require(!fieldNames.contains(colName), s"Output column $colName already exists.") val outputFields = schema.fields ++ Seq(StructField(colName, dataType, nullable = false)) StructType(outputFields) } diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java new file mode 100644 index 0000000000000..161100134c92d --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaVectorIndexerSuite.java @@ -0,0 +1,70 @@ +/* + * 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.ml.feature; + +import java.io.Serializable; +import java.util.List; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import com.google.common.collect.Lists; + +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.ml.feature.VectorIndexerSuite.FeatureData; +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.sql.DataFrame; +import org.apache.spark.sql.SQLContext; + + +public class JavaVectorIndexerSuite implements Serializable { + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaVectorIndexerSuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + } + + @Test + public void vectorIndexerAPI() { + // The tests are to check Java compatibility. + List points = Lists.newArrayList( + new FeatureData(Vectors.dense(0.0, -2.0)), + new FeatureData(Vectors.dense(1.0, 3.0)), + new FeatureData(Vectors.dense(1.0, 4.0)) + ); + SQLContext sqlContext = new SQLContext(sc); + DataFrame data = sqlContext.createDataFrame(sc.parallelize(points, 2), FeatureData.class); + VectorIndexer indexer = new VectorIndexer() + .setInputCol("features") + .setOutputCol("indexed") + .setMaxCategories(2); + VectorIndexerModel model = indexer.fit(data); + Assert.assertEquals(model.numFeatures(), 2); + Assert.assertEquals(model.categoryMaps().size(), 1); + DataFrame indexedData = model.transform(data); + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala index 3fb6e2ec46468..0dcfe5a2002dc 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeGroupSuite.scala @@ -43,8 +43,8 @@ class AttributeGroupSuite extends FunSuite { intercept[NoSuchElementException] { group("abc") } - assert(group === AttributeGroup.fromMetadata(group.toMetadata, group.name)) - assert(group === AttributeGroup.fromStructField(group.toStructField())) + assert(group === AttributeGroup.fromMetadata(group.toMetadataImpl, group.name)) + assert(group === AttributeGroup.fromStructField(group.toStructField)) } test("attribute group without attributes") { @@ -53,8 +53,8 @@ class AttributeGroupSuite extends FunSuite { assert(group0.numAttributes === Some(10)) assert(group0.size === 10) assert(group0.attributes.isEmpty) - assert(group0 === AttributeGroup.fromMetadata(group0.toMetadata, group0.name)) - assert(group0 === AttributeGroup.fromStructField(group0.toStructField())) + assert(group0 === AttributeGroup.fromMetadata(group0.toMetadataImpl, group0.name)) + assert(group0 === AttributeGroup.fromStructField(group0.toStructField)) val group1 = new AttributeGroup("item") assert(group1.name === "item") diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala index a18c335952b96..9d09f24709e23 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NormalizerSuite.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.sql.{DataFrame, Row, SQLContext} -private case class DataSet(features: Vector) class NormalizerSuite extends FunSuite with MLlibTestSparkContext { @@ -63,7 +62,7 @@ class NormalizerSuite extends FunSuite with MLlibTestSparkContext { ) val sqlContext = new SQLContext(sc) - dataFrame = sqlContext.createDataFrame(sc.parallelize(data, 2).map(DataSet)) + dataFrame = sqlContext.createDataFrame(sc.parallelize(data, 2).map(NormalizerSuite.FeatureData)) normalizer = new Normalizer() .setInputCol("features") .setOutputCol("normalized_features") @@ -107,3 +106,7 @@ class NormalizerSuite extends FunSuite with MLlibTestSparkContext { assertValues(result, l1Normalized) } } + +private object NormalizerSuite { + case class FeatureData(features: Vector) +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala new file mode 100644 index 0000000000000..61c46c85a78b5 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -0,0 +1,255 @@ +/* + * 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.ml.feature + +import scala.beans.{BeanInfo, BeanProperty} + +import org.scalatest.FunSuite + +import org.apache.spark.SparkException +import org.apache.spark.ml.attribute._ +import org.apache.spark.ml.util.TestingUtils +import org.apache.spark.mllib.linalg.{SparseVector, Vector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{DataFrame, SQLContext} + + +class VectorIndexerSuite extends FunSuite with MLlibTestSparkContext { + + import VectorIndexerSuite.FeatureData + + @transient var sqlContext: SQLContext = _ + + // identical, of length 3 + @transient var densePoints1: DataFrame = _ + @transient var sparsePoints1: DataFrame = _ + @transient var point1maxes: Array[Double] = _ + + // identical, of length 2 + @transient var densePoints2: DataFrame = _ + @transient var sparsePoints2: DataFrame = _ + + // different lengths + @transient var badPoints: DataFrame = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + val densePoints1Seq = Seq( + Vectors.dense(1.0, 2.0, 0.0), + Vectors.dense(0.0, 1.0, 2.0), + Vectors.dense(0.0, 0.0, -1.0), + Vectors.dense(1.0, 3.0, 2.0)) + val sparsePoints1Seq = Seq( + Vectors.sparse(3, Array(0, 1), Array(1.0, 2.0)), + Vectors.sparse(3, Array(1, 2), Array(1.0, 2.0)), + Vectors.sparse(3, Array(2), Array(-1.0)), + Vectors.sparse(3, Array(0, 1, 2), Array(1.0, 3.0, 2.0))) + point1maxes = Array(1.0, 3.0, 2.0) + + val densePoints2Seq = Seq( + Vectors.dense(1.0, 1.0, 0.0, 1.0), + Vectors.dense(0.0, 1.0, 1.0, 1.0), + Vectors.dense(-1.0, 1.0, 2.0, 0.0)) + val sparsePoints2Seq = Seq( + Vectors.sparse(4, Array(0, 1, 3), Array(1.0, 1.0, 1.0)), + Vectors.sparse(4, Array(1, 2, 3), Array(1.0, 1.0, 1.0)), + Vectors.sparse(4, Array(0, 1, 2), Array(-1.0, 1.0, 2.0))) + + val badPointsSeq = Seq( + Vectors.sparse(2, Array(0, 1), Array(1.0, 1.0)), + Vectors.sparse(3, Array(2), Array(-1.0))) + + // Sanity checks for assumptions made in tests + assert(densePoints1Seq.head.size == sparsePoints1Seq.head.size) + assert(densePoints2Seq.head.size == sparsePoints2Seq.head.size) + assert(densePoints1Seq.head.size != densePoints2Seq.head.size) + def checkPair(dvSeq: Seq[Vector], svSeq: Seq[Vector]): Unit = { + assert(dvSeq.zip(svSeq).forall { case (dv, sv) => dv.toArray === sv.toArray }, + "typo in unit test") + } + checkPair(densePoints1Seq, sparsePoints1Seq) + checkPair(densePoints2Seq, sparsePoints2Seq) + + sqlContext = new SQLContext(sc) + densePoints1 = sqlContext.createDataFrame(sc.parallelize(densePoints1Seq, 2).map(FeatureData)) + sparsePoints1 = sqlContext.createDataFrame(sc.parallelize(sparsePoints1Seq, 2).map(FeatureData)) + densePoints2 = sqlContext.createDataFrame(sc.parallelize(densePoints2Seq, 2).map(FeatureData)) + sparsePoints2 = sqlContext.createDataFrame(sc.parallelize(sparsePoints2Seq, 2).map(FeatureData)) + badPoints = sqlContext.createDataFrame(sc.parallelize(badPointsSeq, 2).map(FeatureData)) + } + + private def getIndexer: VectorIndexer = + new VectorIndexer().setInputCol("features").setOutputCol("indexed") + + test("Cannot fit an empty DataFrame") { + val rdd = sqlContext.createDataFrame(sc.parallelize(Array.empty[Vector], 2).map(FeatureData)) + val vectorIndexer = getIndexer + intercept[IllegalArgumentException] { + vectorIndexer.fit(rdd) + } + } + + test("Throws error when given RDDs with different size vectors") { + val vectorIndexer = getIndexer + val model = vectorIndexer.fit(densePoints1) // vectors of length 3 + model.transform(densePoints1) // should work + model.transform(sparsePoints1) // should work + intercept[IllegalArgumentException] { + model.transform(densePoints2) + println("Did not throw error when fit, transform were called on vectors of different lengths") + } + intercept[SparkException] { + vectorIndexer.fit(badPoints) + println("Did not throw error when fitting vectors of different lengths in same RDD.") + } + } + + test("Same result with dense and sparse vectors") { + def testDenseSparse(densePoints: DataFrame, sparsePoints: DataFrame): Unit = { + val denseVectorIndexer = getIndexer.setMaxCategories(2) + val sparseVectorIndexer = getIndexer.setMaxCategories(2) + val denseModel = denseVectorIndexer.fit(densePoints) + val sparseModel = sparseVectorIndexer.fit(sparsePoints) + val denseMap = denseModel.categoryMaps + val sparseMap = sparseModel.categoryMaps + assert(denseMap.keys.toSet == sparseMap.keys.toSet, + "Categorical features chosen from dense vs. sparse vectors did not match.") + assert(denseMap == sparseMap, + "Categorical feature value indexes chosen from dense vs. sparse vectors did not match.") + } + testDenseSparse(densePoints1, sparsePoints1) + testDenseSparse(densePoints2, sparsePoints2) + } + + test("Builds valid categorical feature value index, transform correctly, check metadata") { + def checkCategoryMaps( + data: DataFrame, + maxCategories: Int, + categoricalFeatures: Set[Int]): Unit = { + val collectedData = data.collect().map(_.getAs[Vector](0)) + val errMsg = s"checkCategoryMaps failed for input with maxCategories=$maxCategories," + + s" categoricalFeatures=${categoricalFeatures.mkString(", ")}" + try { + val vectorIndexer = getIndexer.setMaxCategories(maxCategories) + val model = vectorIndexer.fit(data) + val categoryMaps = model.categoryMaps + assert(categoryMaps.keys.toSet === categoricalFeatures) // Chose correct categorical features + val transformed = model.transform(data).select("indexed") + val indexedRDD: RDD[Vector] = transformed.map(_.getAs[Vector](0)) + val featureAttrs = AttributeGroup.fromStructField(transformed.schema("indexed")) + assert(featureAttrs.name === "indexed") + assert(featureAttrs.attributes.get.length === model.numFeatures) + categoricalFeatures.foreach { feature: Int => + val origValueSet = collectedData.map(_(feature)).toSet + val targetValueIndexSet = Range(0, origValueSet.size).toSet + val catMap = categoryMaps(feature) + assert(catMap.keys.toSet === origValueSet) // Correct categories + assert(catMap.values.toSet === targetValueIndexSet) // Correct category indices + if (origValueSet.contains(0.0)) { + assert(catMap(0.0) === 0) // value 0 gets index 0 + } + // Check transformed data + assert(indexedRDD.map(_(feature)).collect().toSet === targetValueIndexSet) + // Check metadata + val featureAttr = featureAttrs(feature) + assert(featureAttr.index.get === feature) + featureAttr match { + case attr: BinaryAttribute => + assert(attr.values.get === origValueSet.toArray.sorted.map(_.toString)) + case attr: NominalAttribute => + assert(attr.values.get === origValueSet.toArray.sorted.map(_.toString)) + assert(attr.isOrdinal.get === false) + case _ => + throw new RuntimeException(errMsg + s". Categorical feature $feature failed" + + s" metadata check. Found feature attribute: $featureAttr.") + } + } + // Check numerical feature metadata. + Range(0, model.numFeatures).filter(feature => !categoricalFeatures.contains(feature)) + .foreach { feature: Int => + val featureAttr = featureAttrs(feature) + featureAttr match { + case attr: NumericAttribute => + assert(featureAttr.index.get === feature) + case _ => + throw new RuntimeException(errMsg + s". Numerical feature $feature failed" + + s" metadata check. Found feature attribute: $featureAttr.") + } + } + } catch { + case e: org.scalatest.exceptions.TestFailedException => + println(errMsg) + throw e + } + } + checkCategoryMaps(densePoints1, maxCategories = 2, categoricalFeatures = Set(0)) + checkCategoryMaps(densePoints1, maxCategories = 3, categoricalFeatures = Set(0, 2)) + checkCategoryMaps(densePoints2, maxCategories = 2, categoricalFeatures = Set(1, 3)) + } + + test("Maintain sparsity for sparse vectors") { + def checkSparsity(data: DataFrame, maxCategories: Int): Unit = { + val points = data.collect().map(_.getAs[Vector](0)) + val vectorIndexer = getIndexer.setMaxCategories(maxCategories) + val model = vectorIndexer.fit(data) + val indexedPoints = model.transform(data).select("indexed").map(_.getAs[Vector](0)).collect() + points.zip(indexedPoints).foreach { + case (orig: SparseVector, indexed: SparseVector) => + assert(orig.indices.length == indexed.indices.length) + case _ => throw new UnknownError("Unit test has a bug in it.") // should never happen + } + } + checkSparsity(sparsePoints1, maxCategories = 2) + checkSparsity(sparsePoints2, maxCategories = 2) + } + + test("Preserve metadata") { + // For continuous features, preserve name and stats. + val featureAttributes: Array[Attribute] = point1maxes.zipWithIndex.map { case (maxVal, i) => + NumericAttribute.defaultAttr.withName(i.toString).withMax(maxVal) + } + val attrGroup = new AttributeGroup("features", featureAttributes) + val densePoints1WithMeta = + densePoints1.select(densePoints1("features").as("features", attrGroup.toMetadata)) + val vectorIndexer = getIndexer.setMaxCategories(2) + val model = vectorIndexer.fit(densePoints1WithMeta) + // Check that ML metadata are preserved. + val indexedPoints = model.transform(densePoints1WithMeta) + val transAttributes: Array[Attribute] = + AttributeGroup.fromStructField(indexedPoints.schema("indexed")).attributes.get + featureAttributes.zip(transAttributes).foreach { case (orig, trans) => + assert(orig.name === trans.name) + (orig, trans) match { + case (orig: NumericAttribute, trans: NumericAttribute) => + assert(orig.max.nonEmpty && orig.max === trans.max) + case _ => + // do nothing + // TODO: Once input features marked as categorical are handled correctly, check that here. + } + } + // Check that non-ML metadata are preserved. + TestingUtils.testPreserveMetadata(densePoints1WithMeta, model, "features", "indexed") + } +} + +private[feature] object VectorIndexerSuite { + @BeanInfo + case class FeatureData(@BeanProperty features: Vector) +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala new file mode 100644 index 0000000000000..c44cb61b34171 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/util/TestingUtils.scala @@ -0,0 +1,60 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.util + +import org.apache.spark.ml.Transformer +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.types.MetadataBuilder +import org.scalatest.FunSuite + +private[ml] object TestingUtils extends FunSuite { + + /** + * Test whether unrelated metadata are preserved for this transformer. + * This attaches extra metadata to a column, transforms the column, and check to ensure the + * extra metadata have not changed. + * @param data Input dataset + * @param transformer Transformer to test + * @param inputCol Unique input column for Transformer. This must be the ONLY input column. + * @param outputCol Output column to test for metadata presence. + */ + def testPreserveMetadata( + data: DataFrame, + transformer: Transformer, + inputCol: String, + outputCol: String): Unit = { + // Create some fake metadata + val origMetadata = data.schema(inputCol).metadata + val metaKey = "__testPreserveMetadata__fake_key" + val metaValue = 12345 + assert(!origMetadata.contains(metaKey), + s"Unit test with testPreserveMetadata will fail since metadata key was present: $metaKey") + val newMetadata = + new MetadataBuilder().withMetadata(origMetadata).putLong(metaKey, metaValue).build() + // Add metadata to the inputCol + val withMetadata = data.select(data(inputCol).as(inputCol, newMetadata)) + // Transform, and ensure extra metadata was not affected + val transformed = transformer.transform(withMetadata) + val transMetadata = transformed.schema(outputCol).metadata + assert(transMetadata.contains(metaKey), + "Unit test with testPreserveMetadata failed; extra metadata key was not present.") + assert(transMetadata.getLong(metaKey) === metaValue, + "Unit test with testPreserveMetadata failed; extra metadata value was wrong." + + s" Expected $metaValue but found ${transMetadata.getLong(metaKey)}") + } +} From 685ddcf5253c0ecb39853802431e22b0c7b61dee Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 12 Apr 2015 22:41:05 -0700 Subject: [PATCH 056/128] [SPARK-5886][ML] Add StringIndexer as a feature transformer This PR adds string indexer, which takes a column of string labels and outputs a double column with labels indexed by their frequency. TODOs: - [x] store feature to index map in output metadata Author: Xiangrui Meng Closes #4735 from mengxr/SPARK-5886 and squashes the following commits: d82575f [Xiangrui Meng] fix test 700e70f [Xiangrui Meng] rename LabelIndexer to StringIndexer 16a6f8c [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5886 457166e [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5886 f8b30f4 [Xiangrui Meng] update label indexer to output metadata e81ec28 [Xiangrui Meng] Merge branch 'openhashmap-contains' into SPARK-5886-2 d6e6f1f [Xiangrui Meng] add contains to primitivekeyopenhashmap 748a69b [Xiangrui Meng] add contains to OpenHashMap def3c5c [Xiangrui Meng] add LabelIndexer --- .../spark/ml/feature/StringIndexer.scala | 126 ++++++++++++++++++ .../spark/ml/feature/StringIndexerSuite.scala | 52 ++++++++ 2 files changed, 178 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala new file mode 100644 index 0000000000000..61e6742e880d8 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.apache.spark.SparkException +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.{Estimator, Model} +import org.apache.spark.ml.attribute.NominalAttribute +import org.apache.spark.ml.param._ +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.util.collection.OpenHashMap + +/** + * Base trait for [[StringIndexer]] and [[StringIndexerModel]]. + */ +private[feature] trait StringIndexerBase extends Params with HasInputCol with HasOutputCol { + + /** Validates and transforms the input schema. */ + protected def validateAndTransformSchema(schema: StructType, paramMap: ParamMap): StructType = { + val map = this.paramMap ++ paramMap + checkInputColumn(schema, map(inputCol), StringType) + val inputFields = schema.fields + val outputColName = map(outputCol) + require(inputFields.forall(_.name != outputColName), + s"Output column $outputColName already exists.") + val attr = NominalAttribute.defaultAttr.withName(map(outputCol)) + val outputFields = inputFields :+ attr.toStructField() + StructType(outputFields) + } +} + +/** + * :: AlphaComponent :: + * A label indexer that maps a string column of labels to an ML column of label indices. + * The indices are in [0, numLabels), ordered by label frequencies. + * So the most frequent label gets index 0. + */ +@AlphaComponent +class StringIndexer extends Estimator[StringIndexerModel] with StringIndexerBase { + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + // TODO: handle unseen labels + + override def fit(dataset: DataFrame, paramMap: ParamMap): StringIndexerModel = { + val map = this.paramMap ++ paramMap + val counts = dataset.select(map(inputCol)).map(_.getString(0)).countByValue() + val labels = counts.toSeq.sortBy(-_._2).map(_._1).toArray + val model = new StringIndexerModel(this, map, labels) + Params.inheritValues(map, this, model) + model + } + + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + validateAndTransformSchema(schema, paramMap) + } +} + +/** + * :: AlphaComponent :: + * Model fitted by [[StringIndexer]]. + */ +@AlphaComponent +class StringIndexerModel private[ml] ( + override val parent: StringIndexer, + override val fittingParamMap: ParamMap, + labels: Array[String]) extends Model[StringIndexerModel] with StringIndexerBase { + + private val labelToIndex: OpenHashMap[String, Double] = { + val n = labels.length + val map = new OpenHashMap[String, Double](n) + var i = 0 + while (i < n) { + map.update(labels(i), i) + i += 1 + } + map + } + + /** @group setParam */ + def setInputCol(value: String): this.type = set(inputCol, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + val map = this.paramMap ++ paramMap + val indexer = udf { label: String => + if (labelToIndex.contains(label)) { + labelToIndex(label) + } else { + // TODO: handle unseen labels + throw new SparkException(s"Unseen label: $label.") + } + } + val outputColName = map(outputCol) + val metadata = NominalAttribute.defaultAttr + .withName(outputColName).withValues(labels).toStructField().metadata + dataset.select(col("*"), indexer(dataset(map(inputCol))).as(outputColName, metadata)) + } + + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + validateAndTransformSchema(schema, paramMap) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala new file mode 100644 index 0000000000000..00b5d094d82f1 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ml.feature + +import org.scalatest.FunSuite + +import org.apache.spark.ml.attribute.{Attribute, NominalAttribute} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.SQLContext + +class StringIndexerSuite extends FunSuite with MLlibTestSparkContext { + private var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + test("StringIndexer") { + val data = sc.parallelize(Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")), 2) + val df = sqlContext.createDataFrame(data).toDF("id", "label") + val indexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("labelIndex") + .fit(df) + val transformed = indexer.transform(df) + val attr = Attribute.fromStructField(transformed.schema("labelIndex")) + .asInstanceOf[NominalAttribute] + assert(attr.values.get === Array("a", "c", "b")) + val output = transformed.select("id", "labelIndex").map { r => + (r.getInt(0), r.getDouble(1)) + }.collect().toSet + // a -> 0, b -> 2, c -> 1 + val expected = Set((0, 0.0), (1, 2.0), (2, 1.0), (3, 0.0), (4, 0.0), (5, 1.0)) + assert(output === expected) + } +} From 929404498506c34180e2eaaa1a4d4a3c4ed51daa Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 12 Apr 2015 22:42:01 -0700 Subject: [PATCH 057/128] [SPARK-5885][MLLIB] Add VectorAssembler as a feature transformer VectorAssembler merges multiple columns into a vector column. This PR contains content from #5195. ~~carry ML attributes~~ (moved to a follow-up PR) Author: Xiangrui Meng Closes #5196 from mengxr/SPARK-5885 and squashes the following commits: a52b101 [Xiangrui Meng] recognize more types 35daac2 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5885 bb5e64b [Xiangrui Meng] add TODO for null 976a3d6 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5885 0859311 [Xiangrui Meng] Revert "add CreateStruct" 29fb6ac [Xiangrui Meng] use CreateStruct adb71c4 [Xiangrui Meng] Merge branch 'SPARK-6542' into SPARK-5885 85f3106 [Xiangrui Meng] add CreateStruct 4ff16ce [Xiangrui Meng] add VectorAssembler --- .../org/apache/spark/ml/Identifiable.scala | 2 +- .../spark/ml/feature/VectorAssembler.scala | 111 ++++++++++++++++++ .../apache/spark/ml/param/sharedParams.scala | 10 ++ .../ml/feature/VectorAssemblerSuite.scala | 63 ++++++++++ 4 files changed, 185 insertions(+), 1 deletion(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala create mode 100644 mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala b/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala index cd84b05bfb496..a50090671ae48 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala @@ -29,5 +29,5 @@ private[ml] trait Identifiable extends Serializable { * random hex chars. */ private[ml] val uid: String = - this.getClass.getSimpleName + "-" + UUID.randomUUID().toString.take(8) + this.getClass.getSimpleName + "_" + UUID.randomUUID().toString.take(8) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala new file mode 100644 index 0000000000000..d1b8f7e6e9295 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -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.ml.feature + +import scala.collection.mutable.ArrayBuilder + +import org.apache.spark.SparkException +import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.Transformer +import org.apache.spark.ml.param.{HasInputCols, HasOutputCol, ParamMap} +import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} +import org.apache.spark.sql.{Column, DataFrame, Row} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Alias, Cast, CreateStruct} +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +/** + * :: AlphaComponent :: + * A feature transformer than merge multiple columns into a vector column. + */ +@AlphaComponent +class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { + + /** @group setParam */ + def setInputCols(value: Array[String]): this.type = set(inputCols, value) + + /** @group setParam */ + def setOutputCol(value: String): this.type = set(outputCol, value) + + override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { + val map = this.paramMap ++ paramMap + val assembleFunc = udf { r: Row => + VectorAssembler.assemble(r.toSeq: _*) + } + val schema = dataset.schema + val inputColNames = map(inputCols) + val args = inputColNames.map { c => + schema(c).dataType match { + case DoubleType => UnresolvedAttribute(c) + case t if t.isInstanceOf[VectorUDT] => UnresolvedAttribute(c) + case _: NativeType => Alias(Cast(UnresolvedAttribute(c), DoubleType), s"${c}_double_$uid")() + } + } + dataset.select(col("*"), assembleFunc(new Column(CreateStruct(args))).as(map(outputCol))) + } + + override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { + val map = this.paramMap ++ paramMap + val inputColNames = map(inputCols) + val outputColName = map(outputCol) + val inputDataTypes = inputColNames.map(name => schema(name).dataType) + inputDataTypes.foreach { + case _: NativeType => + case t if t.isInstanceOf[VectorUDT] => + case other => + throw new IllegalArgumentException(s"Data type $other is not supported.") + } + if (schema.fieldNames.contains(outputColName)) { + throw new IllegalArgumentException(s"Output column $outputColName already exists.") + } + StructType(schema.fields :+ new StructField(outputColName, new VectorUDT, false)) + } +} + +@AlphaComponent +object VectorAssembler { + + private[feature] def assemble(vv: Any*): Vector = { + val indices = ArrayBuilder.make[Int] + val values = ArrayBuilder.make[Double] + var cur = 0 + vv.foreach { + case v: Double => + if (v != 0.0) { + indices += cur + values += v + } + cur += 1 + case vec: Vector => + vec.foreachActive { case (i, v) => + if (v != 0.0) { + indices += cur + i + values += v + } + } + cur += vec.size + case null => + // TODO: output Double.NaN? + throw new SparkException("Values to assemble cannot be null.") + case o => + throw new SparkException(s"$o of type ${o.getClass.getName} is not supported.") + } + Vectors.sparse(cur, indices.result(), values.result()) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala index 0739fdbfcbaae..07e6eb417763d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala @@ -140,6 +140,16 @@ private[ml] trait HasInputCol extends Params { def getInputCol: String = get(inputCol) } +private[ml] trait HasInputCols extends Params { + /** + * Param for input column names. + */ + val inputCols: Param[Array[String]] = new Param(this, "inputCols", "input column names") + + /** @group getParam */ + def getInputCols: Array[String] = get(inputCols) +} + private[ml] trait HasOutputCol extends Params { /** * param for output column name diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala new file mode 100644 index 0000000000000..57d0278e03639 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorAssemblerSuite.scala @@ -0,0 +1,63 @@ +/* + * 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.ml.feature + +import org.scalatest.FunSuite + +import org.apache.spark.SparkException +import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.{Row, SQLContext} + +class VectorAssemblerSuite extends FunSuite with MLlibTestSparkContext { + + @transient var sqlContext: SQLContext = _ + + override def beforeAll(): Unit = { + super.beforeAll() + sqlContext = new SQLContext(sc) + } + + test("assemble") { + import org.apache.spark.ml.feature.VectorAssembler.assemble + assert(assemble(0.0) === Vectors.sparse(1, Array.empty, Array.empty)) + assert(assemble(0.0, 1.0) === Vectors.sparse(2, Array(1), Array(1.0))) + val dv = Vectors.dense(2.0, 0.0) + assert(assemble(0.0, dv, 1.0) === Vectors.sparse(4, Array(1, 3), Array(2.0, 1.0))) + val sv = Vectors.sparse(2, Array(0, 1), Array(3.0, 4.0)) + assert(assemble(0.0, dv, 1.0, sv) === + Vectors.sparse(6, Array(1, 3, 4, 5), Array(2.0, 1.0, 3.0, 4.0))) + for (v <- Seq(1, "a", null)) { + intercept[SparkException](assemble(v)) + intercept[SparkException](assemble(1.0, v)) + } + } + + test("VectorAssembler") { + val df = sqlContext.createDataFrame(Seq( + (0, 0.0, Vectors.dense(1.0, 2.0), "a", Vectors.sparse(2, Array(1), Array(3.0)), 10L) + )).toDF("id", "x", "y", "name", "z", "n") + val assembler = new VectorAssembler() + .setInputCols(Array("x", "y", "z", "n")) + .setOutputCol("features") + assembler.transform(df).select("features").collect().foreach { + case Row(v: Vector) => + assert(v === Vectors.sparse(6, Array(1, 2, 4, 5), Array(1.0, 2.0, 3.0, 10.0))) + } + } +} From 68d1faa3c04e9412bbc2b60421dc12bd19c396b2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 12 Apr 2015 22:56:12 -0700 Subject: [PATCH 058/128] [SPARK-6562][SQL] DataFrame.replace Supports replacing values with other values in DataFrames. Python support should be in a separate pull request. Author: Reynold Xin Closes #5282 from rxin/df-na-replace and squashes the following commits: 4b72434 [Reynold Xin] Removed println. c8d9946 [Reynold Xin] col -> cols fbb3c21 [Reynold Xin] [SPARK-6562][SQL] DataFrame.replace --- .../spark/sql/DataFrameNaFunctions.scala | 144 ++++++++++++++++++ .../spark/sql/DataFrameNaFunctionsSuite.scala | 34 +++++ 2 files changed, 178 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index bf3c3fe876873..481ed4924857e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -192,6 +192,127 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { */ def fill(valueMap: Map[String, Any]): DataFrame = fill0(valueMap.toSeq) + /** + * Replaces values matching keys in `replacement` map with the corresponding values. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * If `col` is "*", then the replacement is applied on all string columns or numeric columns. + * + * {{{ + * import com.google.common.collect.ImmutableMap; + * + * // Replaces all occurrences of 1.0 with 2.0 in column "height". + * df.replace("height", ImmutableMap.of(1.0, 2.0)); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "name". + * df.replace("name", ImmutableMap.of("UNKNOWN", "unnamed")); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in all string columns. + * df.replace("*", ImmutableMap.of("UNKNOWN", "unnamed")); + * }}} + * + * @param col name of the column to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](col: String, replacement: java.util.Map[T, T]): DataFrame = { + replace[T](col, replacement.toMap : Map[T, T]) + } + + /** + * Replaces values matching keys in `replacement` map with the corresponding values. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * + * {{{ + * import com.google.common.collect.ImmutableMap; + * + * // Replaces all occurrences of 1.0 with 2.0 in column "height" and "weight". + * df.replace(new String[] {"height", "weight"}, ImmutableMap.of(1.0, 2.0)); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "firstname" and "lastname". + * df.replace(new String[] {"firstname", "lastname"}, ImmutableMap.of("UNKNOWN", "unnamed")); + * }}} + * + * @param cols list of columns to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](cols: Array[String], replacement: java.util.Map[T, T]): DataFrame = { + replace(cols.toSeq, replacement.toMap) + } + + /** + * (Scala-specific) Replaces values matching keys in `replacement` map. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * If `col` is "*", then the replacement is applied on all string columns or numeric columns. + * + * {{{ + * // Replaces all occurrences of 1.0 with 2.0 in column "height". + * df.replace("height", Map(1.0 -> 2.0)) + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "name". + * df.replace("name", Map("UNKNOWN" -> "unnamed") + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in all string columns. + * df.replace("*", Map("UNKNOWN" -> "unnamed") + * }}} + * + * @param col name of the column to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](col: String, replacement: Map[T, T]): DataFrame = { + if (col == "*") { + replace0(df.columns, replacement) + } else { + replace0(Seq(col), replacement) + } + } + + /** + * (Scala-specific) Replaces values matching keys in `replacement` map. + * Key and value of `replacement` map must have the same type, and can only be doubles or strings. + * + * {{{ + * // Replaces all occurrences of 1.0 with 2.0 in column "height" and "weight". + * df.replace("height" :: "weight" :: Nil, Map(1.0 -> 2.0)); + * + * // Replaces all occurrences of "UNKNOWN" with "unnamed" in column "firstname" and "lastname". + * df.replace("firstname" :: "lastname" :: Nil, Map("UNKNOWN" -> "unnamed"); + * }}} + * + * @param cols list of columns to apply the value replacement + * @param replacement value replacement map, as explained above + */ + def replace[T](cols: Seq[String], replacement: Map[T, T]): DataFrame = replace0(cols, replacement) + + private def replace0[T](cols: Seq[String], replacement: Map[T, T]): DataFrame = { + if (replacement.isEmpty || cols.isEmpty) { + return df + } + + // replacementMap is either Map[String, String] or Map[Double, Double] + val replacementMap: Map[_, _] = replacement.head._2 match { + case v: String => replacement + case _ => replacement.map { case (k, v) => (convertToDouble(k), convertToDouble(v)) } + } + + // targetColumnType is either DoubleType or StringType + val targetColumnType = replacement.head._1 match { + case _: jl.Double | _: jl.Float | _: jl.Integer | _: jl.Long => DoubleType + case _: String => StringType + } + + val columnEquals = df.sqlContext.analyzer.resolver + val projections = df.schema.fields.map { f => + val shouldReplace = cols.exists(colName => columnEquals(colName, f.name)) + if (f.dataType.isInstanceOf[NumericType] && targetColumnType == DoubleType && shouldReplace) { + replaceCol(f, replacementMap) + } else if (f.dataType == targetColumnType && shouldReplace) { + replaceCol(f, replacementMap) + } else { + df.col(f.name) + } + } + df.select(projections : _*) + } + private def fill0(values: Seq[(String, Any)]): DataFrame = { // Error handling values.foreach { case (colName, replaceValue) => @@ -228,4 +349,27 @@ final class DataFrameNaFunctions private[sql](df: DataFrame) { private def fillCol[T](col: StructField, replacement: T): Column = { coalesce(df.col(col.name), lit(replacement).cast(col.dataType)).as(col.name) } + + /** + * Returns a [[Column]] expression that replaces value matching key in `replacementMap` with + * value in `replacementMap`, using [[CaseWhen]]. + * + * TODO: This can be optimized to use broadcast join when replacementMap is large. + */ + private def replaceCol(col: StructField, replacementMap: Map[_, _]): Column = { + val branches: Seq[Expression] = replacementMap.flatMap { case (source, target) => + df.col(col.name).equalTo(lit(source).cast(col.dataType)).expr :: + lit(target).cast(col.dataType).expr :: Nil + }.toSeq + new Column(CaseWhen(branches ++ Seq(df.col(col.name).expr))).as(col.name) + } + + private def convertToDouble(v: Any): Double = v match { + case v: Float => v.toDouble + case v: Double => v + case v: Long => v.toDouble + case v: Int => v.toDouble + case v => throw new IllegalArgumentException( + s"Unsupported value type ${v.getClass.getName} ($v).") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala index 0896f175c056f..41b4f02e6a294 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameNaFunctionsSuite.scala @@ -154,4 +154,38 @@ class DataFrameNaFunctionsSuite extends QueryTest { ))), Row("test", null, 1, 2.2)) } + + test("replace") { + val input = createDF() + + // Replace two numeric columns: age and height + val out = input.na.replace(Seq("age", "height"), Map( + 16 -> 61, + 60 -> 6, + 164.3 -> 461.3 // Alice is really tall + )) + + checkAnswer( + out, + Row("Bob", 61, 176.5) :: + Row("Alice", null, 461.3) :: + Row("David", 6, null) :: + Row("Amy", null, null) :: + Row(null, null, null) :: Nil) + + // Replace only the age column + val out1 = input.na.replace("age", Map( + 16 -> 61, + 60 -> 6, + 164.3 -> 461.3 // Alice is really tall + )) + + checkAnswer( + out1, + Row("Bob", 61, 176.5) :: + Row("Alice", null, 164.3) :: + Row("David", 6, null) :: + Row("Amy", null, null) :: + Row(null, null, null) :: Nil) + } } From 950645d597dbc5a8c5010bcb1a9b51c6abad86ea Mon Sep 17 00:00:00 2001 From: Dean Chen Date: Mon, 13 Apr 2015 12:08:55 +0100 Subject: [PATCH 059/128] [SPARK-6868][YARN] Fix broken container log link on executor page when HTTPS_ONLY. Correct http schema in YARN container log link in Spark UI when container logs when YARN is configured to be HTTPS_ONLY. Uses the same logic as the YARN jobtracker webapp. Entry point is [JobBlock](https://github.com/apache/hadoop/blob/e1109fb65608a668cd53dc324dadc6f63a74eeb9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-app/src/main/java/org/apache/hadoop/mapreduce/v2/app/webapp/JobBlock.java#L108) and logic is in [MRWebAppUtil](https://github.com/apache/hadoop/blob/e1109fb65608a668cd53dc324dadc6f63a74eeb9/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-common/src/main/java/org/apache/hadoop/mapreduce/v2/util/MRWebAppUtil.java#L75). I chose to migrate the logic over instead of importing MRWebAppUtil(but can update the PR to do so) since the class is designated as private and the logic was straightforward. Author: Dean Chen Closes #5477 from deanchen/container-url and squashes the following commits: 91d3090 [Dean Chen] Correct http schema in YARN container log link in Spark UI when container logs when YARN is configured to be HTTPS_ONLY. --- .../apache/spark/deploy/yarn/ExecutorRunnable.scala | 13 +++++++++++-- 1 file changed, 11 insertions(+), 2 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 1ce10d906ab23..b06069c07f451 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -290,10 +290,19 @@ class ExecutorRunnable( YarnSparkHadoopUtil.setEnvFromInputString(env, userEnvs) } + // lookup appropriate http scheme for container log urls + val yarnHttpPolicy = yarnConf.get( + YarnConfiguration.YARN_HTTP_POLICY_KEY, + YarnConfiguration.YARN_HTTP_POLICY_DEFAULT + ) + val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" + // Add log urls sys.env.get("SPARK_USER").foreach { user => - val baseUrl = "http://%s/node/containerlogs/%s/%s" - .format(container.getNodeHttpAddress, ConverterUtils.toString(container.getId), user) + val containerId = ConverterUtils.toString(container.getId) + val address = container.getNodeHttpAddress + val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user" + env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=0" env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=0" } From cadd7d72c52ccc8d2def405a77dcf807fb5c17c2 Mon Sep 17 00:00:00 2001 From: lisurprise Date: Mon, 13 Apr 2015 12:18:05 +0100 Subject: [PATCH 060/128] [SPARK-6762]Fix potential resource leaks in CheckPoint CheckpointWriter and CheckpointReader The close action should be placed within finally block to avoid the potential resource leaks Author: lisurprise Closes #5407 from zhichao-li/master and squashes the following commits: 065999f [lisurprise] add guard for null ef862d6 [lisurprise] remove fs.close a754adc [lisurprise] refactor with tryWithSafeFinally 824adb3 [lisurprise] close before validation c877da7 [lisurprise] Fix potential resource leaks --- .../apache/spark/streaming/Checkpoint.scala | 47 ++++++++++++------- .../spark/streaming/util/RawTextSender.scala | 3 +- 2 files changed, 31 insertions(+), 19 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 28703ef8129b3..0a50485118588 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -26,7 +26,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.spark.{SparkException, SparkConf, Logging} import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.MetadataCleaner +import org.apache.spark.util.{MetadataCleaner, Utils} import org.apache.spark.streaming.scheduler.JobGenerator @@ -139,8 +139,11 @@ class CheckpointWriter( // Write checkpoint to temp file fs.delete(tempFile, true) // just in case it exists val fos = fs.create(tempFile) - fos.write(bytes) - fos.close() + Utils.tryWithSafeFinally { + fos.write(bytes) + } { + fos.close() + } // If the checkpoint file exists, back it up // If the backup exists as well, just delete it, otherwise rename will fail @@ -187,9 +190,11 @@ class CheckpointWriter( val bos = new ByteArrayOutputStream() val zos = compressionCodec.compressedOutputStream(bos) val oos = new ObjectOutputStream(zos) - oos.writeObject(checkpoint) - oos.close() - bos.close() + Utils.tryWithSafeFinally { + oos.writeObject(checkpoint) + } { + oos.close() + } try { executor.execute(new CheckpointWriteHandler( checkpoint.checkpointTime, bos.toByteArray, clearCheckpointDataLater)) @@ -248,18 +253,24 @@ object CheckpointReader extends Logging { checkpointFiles.foreach(file => { logInfo("Attempting to load checkpoint from file " + file) try { - val fis = fs.open(file) - // ObjectInputStream uses the last defined user-defined class loader in the stack - // to find classes, which maybe the wrong class loader. Hence, a inherited version - // of ObjectInputStream is used to explicitly use the current thread's default class - // loader to find and load classes. This is a well know Java issue and has popped up - // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) - val zis = compressionCodec.compressedInputStream(fis) - val ois = new ObjectInputStreamWithLoader(zis, - Thread.currentThread().getContextClassLoader) - val cp = ois.readObject.asInstanceOf[Checkpoint] - ois.close() - fs.close() + var ois: ObjectInputStreamWithLoader = null + var cp: Checkpoint = null + Utils.tryWithSafeFinally { + val fis = fs.open(file) + // ObjectInputStream uses the last defined user-defined class loader in the stack + // to find classes, which maybe the wrong class loader. Hence, a inherited version + // of ObjectInputStream is used to explicitly use the current thread's default class + // loader to find and load classes. This is a well know Java issue and has popped up + // in other places (e.g., http://jira.codehaus.org/browse/GROOVY-1627) + val zis = compressionCodec.compressedInputStream(fis) + ois = new ObjectInputStreamWithLoader(zis, + Thread.currentThread().getContextClassLoader) + cp = ois.readObject.asInstanceOf[Checkpoint] + } { + if (ois != null) { + ois.close() + } + } cp.validate() logInfo("Checkpoint successfully loaded from file " + file) logInfo("Checkpoint was generated at time " + cp.checkpointTime) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index a7850812bd612..ca2f319f174a2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -72,7 +72,8 @@ object RawTextSender extends Logging { } catch { case e: IOException => logError("Client disconnected") - socket.close() + } finally { + socket.close() } } } From 14ce3ea2c9546c58203af85aceb76b1bfc1f650a Mon Sep 17 00:00:00 2001 From: zsxwing Date: Mon, 13 Apr 2015 12:21:29 +0100 Subject: [PATCH 061/128] [SPARK-6860][Streaming][WebUI] Fix the possible inconsistency of StreamingPage Because `StreamingPage.render` doesn't hold the `listener` lock when generating the content, the different parts of content may have some inconsistent values if `listener` updates its status at the same time. And it will confuse people. This PR added `listener.synchronized` to make sure we have a consistent view of StreamingJobProgressListener when creating the content. Author: zsxwing Closes #5470 from zsxwing/SPARK-6860 and squashes the following commits: cec6f92 [zsxwing] Add missing 'synchronized' in StreamingJobProgressListener 7182498 [zsxwing] Add synchronized to make sure we have a consistent view of StreamingJobProgressListener when creating the content --- .../streaming/ui/StreamingJobProgressListener.scala | 10 +++++----- .../org/apache/spark/streaming/ui/StreamingPage.scala | 3 ++- 2 files changed, 7 insertions(+), 6 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 84f80e638f638..be1e8686cf9fa 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -149,7 +149,7 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) }.toMap } - def lastReceivedBatchRecords: Map[Int, Long] = { + def lastReceivedBatchRecords: Map[Int, Long] = synchronized { val lastReceivedBlockInfoOption = lastReceivedBatch.map(_.receivedBlockInfo) lastReceivedBlockInfoOption.map { lastReceivedBlockInfo => (0 until numReceivers).map { receiverId => @@ -160,19 +160,19 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) } } - def receiverInfo(receiverId: Int): Option[ReceiverInfo] = { + def receiverInfo(receiverId: Int): Option[ReceiverInfo] = synchronized { receiverInfos.get(receiverId) } - def lastCompletedBatch: Option[BatchInfo] = { + def lastCompletedBatch: Option[BatchInfo] = synchronized { completedBatchInfos.sortBy(_.batchTime)(Time.ordering).lastOption } - def lastReceivedBatch: Option[BatchInfo] = { + def lastReceivedBatch: Option[BatchInfo] = synchronized { retainedBatches.lastOption } - private def retainedBatches: Seq[BatchInfo] = synchronized { + private def retainedBatches: Seq[BatchInfo] = { (waitingBatchInfos.values.toSeq ++ runningBatchInfos.values.toSeq ++ completedBatchInfos).sortBy(_.batchTime)(Time.ordering) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index bfe8086fcf8fe..b6dcb62bfeec8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -37,11 +37,12 @@ private[ui] class StreamingPage(parent: StreamingTab) /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { - val content = + val content = listener.synchronized { generateBasicStats() ++

    ++

    Statistics over last {listener.retainedCompletedBatches.size} processed batches

    ++ generateReceiverStats() ++ generateBatchStatsTable() + } UIUtils.headerSparkPage("Streaming", content, parent, Some(5000)) } From 9d117cee0be2c73a25702d98f78211055d50babe Mon Sep 17 00:00:00 2001 From: nyaapa Date: Mon, 13 Apr 2015 12:55:25 +0100 Subject: [PATCH 062/128] [SPARK-6440][CORE]Handle IPv6 addresses properly when constructing URI Author: nyaapa Closes #5424 from nyaapa/master and squashes the following commits: 6b717aa [nyaapa] [SPARK-6440][CORE] Remove Utils.localIpAddressHostname, Utils.localIpAddressURI and Utils.getAddressHostName; make Utils.localIpAddress private; rename Utils.localHostURI into Utils.localHostNameForURI; use Utils.localHostName in org.apache.spark.streaming.kinesis.KinesisReceiver and org.apache.spark.sql.hive.thriftserver.SparkSQLEnv 2098081 [nyaapa] [SPARK-6440][CORE] style fixes and use getHostAddress instead of getHostName 84763d7 [nyaapa] [SPARK-6440][CORE]Handle IPv6 addresses properly when constructing URI --- .../scala/org/apache/spark/HttpServer.scala | 2 +- .../spark/deploy/LocalSparkCluster.scala | 2 +- .../spark/deploy/client/TestClient.scala | 2 +- .../scala/org/apache/spark/ui/WebUI.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 34 ++++++++++++------- .../streaming/kinesis/KinesisReceiver.scala | 3 +- .../sql/hive/thriftserver/SparkSQLEnv.scala | 3 +- 7 files changed, 29 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index 09a9ccc226721..8de3a6c04df34 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -160,7 +160,7 @@ private[spark] class HttpServer( throw new ServerStateException("Server is not started") } else { val scheme = if (securityManager.fileServerSSLOptions.enabled) "https" else "http" - s"$scheme://${Utils.localIpAddress}:$port" + s"$scheme://${Utils.localHostNameForURI()}:$port" } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index 3ab425aab84c8..f0e77c2ba982b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -53,7 +53,7 @@ class LocalSparkCluster( /* Start the Master */ val (masterSystem, masterPort, _, _) = Master.startSystemAndActor(localHostname, 0, 0, _conf) masterActorSystems += masterSystem - val masterUrl = "spark://" + localHostname + ":" + masterPort + val masterUrl = "spark://" + Utils.localHostNameForURI() + ":" + masterPort val masters = Array(masterUrl) /* Start the Workers */ diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index c1c4812f17fbe..40835b9550586 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -46,7 +46,7 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) val conf = new SparkConf - val (actorSystem, _) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, + val (actorSystem, _) = AkkaUtils.createActorSystem("spark", Utils.localHostName(), 0, conf = conf, securityManager = new SecurityManager(conf)) val desc = new ApplicationDescription("TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), Seq(), Seq()), "ignored") diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index ea548f23120d9..f9860d1a5ce76 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -48,7 +48,7 @@ private[spark] abstract class WebUI( protected val handlers = ArrayBuffer[ServletContextHandler]() protected val pageToHandlers = new HashMap[WebUIPage, ArrayBuffer[ServletContextHandler]] protected var serverInfo: Option[ServerInfo] = None - protected val localHostName = Utils.localHostName() + protected val localHostName = Utils.localHostNameForURI() protected val publicHostName = Option(conf.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) private val className = Utils.getFormattedClassName(this) 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 0fdfaf300e95d..a541d660cd5c6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -34,6 +34,7 @@ import scala.util.Try import scala.util.control.{ControlThrowable, NonFatal} import com.google.common.io.{ByteStreams, Files} +import com.google.common.net.InetAddresses import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.commons.lang3.SystemUtils import org.apache.hadoop.conf.Configuration @@ -789,13 +790,12 @@ private[spark] object Utils extends Logging { * Get the local host's IP address in dotted-quad format (e.g. 1.2.3.4). * Note, this is typically not used from within core spark. */ - lazy val localIpAddress: String = findLocalIpAddress() - lazy val localIpAddressHostname: String = getAddressHostName(localIpAddress) + private lazy val localIpAddress: InetAddress = findLocalInetAddress() - private def findLocalIpAddress(): String = { + private def findLocalInetAddress(): InetAddress = { val defaultIpOverride = System.getenv("SPARK_LOCAL_IP") if (defaultIpOverride != null) { - defaultIpOverride + InetAddress.getByName(defaultIpOverride) } else { val address = InetAddress.getLocalHost if (address.isLoopbackAddress) { @@ -806,15 +806,20 @@ private[spark] object Utils extends Logging { // It's more proper to pick ip address following system output order. val activeNetworkIFs = NetworkInterface.getNetworkInterfaces.toList val reOrderedNetworkIFs = if (isWindows) activeNetworkIFs else activeNetworkIFs.reverse + for (ni <- reOrderedNetworkIFs) { - for (addr <- ni.getInetAddresses if !addr.isLinkLocalAddress && - !addr.isLoopbackAddress && addr.isInstanceOf[Inet4Address]) { + val addresses = ni.getInetAddresses.toList + .filterNot(addr => addr.isLinkLocalAddress || addr.isLoopbackAddress) + if (addresses.nonEmpty) { + val addr = addresses.find(_.isInstanceOf[Inet4Address]).getOrElse(addresses.head) + // because of Inet6Address.toHostName may add interface at the end if it knows about it + val strippedAddress = InetAddress.getByAddress(addr.getAddress) // We've found an address that looks reasonable! logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + - " a loopback address: " + address.getHostAddress + "; using " + addr.getHostAddress + - " instead (on interface " + ni.getName + ")") + " a loopback address: " + address.getHostAddress + "; using " + + strippedAddress.getHostAddress + " instead (on interface " + ni.getName + ")") logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") - return addr.getHostAddress + return strippedAddress } } logWarning("Your hostname, " + InetAddress.getLocalHost.getHostName + " resolves to" + @@ -822,7 +827,7 @@ private[spark] object Utils extends Logging { " external IP address!") logWarning("Set SPARK_LOCAL_IP if you need to bind to another address") } - address.getHostAddress + address } } @@ -842,11 +847,14 @@ private[spark] object Utils extends Logging { * Get the local machine's hostname. */ def localHostName(): String = { - customHostname.getOrElse(localIpAddressHostname) + customHostname.getOrElse(localIpAddress.getHostAddress) } - def getAddressHostName(address: String): String = { - InetAddress.getByName(address).getHostName + /** + * Get the local machine's URI. + */ + def localHostNameForURI(): String = { + customHostname.getOrElse(InetAddresses.toUriString(localIpAddress)) } def checkHost(host: String, message: String = "") { diff --git a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala index 1bd1f324298e7..a7fe4476cacb8 100644 --- a/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala +++ b/extras/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisReceiver.scala @@ -23,6 +23,7 @@ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.Duration import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.util.Utils import com.amazonaws.auth.AWSCredentialsProvider import com.amazonaws.auth.DefaultAWSCredentialsProviderChain @@ -118,7 +119,7 @@ private[kinesis] class KinesisReceiver( * method. */ override def onStart() { - workerId = InetAddress.getLocalHost.getHostAddress() + ":" + UUID.randomUUID() + workerId = Utils.localHostName() + ":" + UUID.randomUUID() credentialsProvider = new DefaultAWSCredentialsProviderChain() kinesisClientLibConfiguration = new KinesisClientLibConfiguration(appName, streamName, credentialsProvider, workerId).withKinesisEndpoint(endpointUrl) 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 158c225159720..97b46a01ba5b4 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 @@ -22,6 +22,7 @@ import scala.collection.JavaConversions._ import org.apache.spark.scheduler.StatsReportListener import org.apache.spark.sql.hive.{HiveShim, HiveContext} import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.util.Utils /** A singleton object for the master program. The slaves should not access this. */ private[hive] object SparkSQLEnv extends Logging { @@ -37,7 +38,7 @@ private[hive] object SparkSQLEnv extends Logging { val maybeKryoReferenceTracking = sparkConf.getOption("spark.kryo.referenceTracking") sparkConf - .setAppName(s"SparkSQL::${java.net.InetAddress.getLocalHost.getHostName}") + .setAppName(s"SparkSQL::${Utils.localHostName()}") .set("spark.sql.hive.version", HiveShim.version) .set( "spark.serializer", From 240ea03faea005306e73c10253716b95487325ff Mon Sep 17 00:00:00 2001 From: Pradeep Chanumolu Date: Mon, 13 Apr 2015 13:02:55 +0100 Subject: [PATCH 063/128] [SPARK-6671] Add status command for spark daemons SPARK-6671 Currently using the spark-daemon.sh script we can start and stop the spark demons. But we cannot get the status of the daemons. It will be nice to include the status command in the spark-daemon.sh script, through which we can know if the spark demon is alive or not. Author: Pradeep Chanumolu Closes #5327 from pchanumolu/master and squashes the following commits: d3a1f05 [Pradeep Chanumolu] Make status command check consistent with Stop command 5062926 [Pradeep Chanumolu] Fix indentation in spark-daemon.sh 3e66bc8 [Pradeep Chanumolu] SPARK-6671 : Add status command to spark daemons 1ac3918 [Pradeep Chanumolu] Add status command to spark-daemon --- sbin/spark-daemon.sh | 19 ++++++++++++++++++- 1 file changed, 18 insertions(+), 1 deletion(-) diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 92e76a3fe6ca2..d8e0facb81169 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -29,7 +29,7 @@ # SPARK_NICENESS The scheduling priority for daemons. Defaults to 0. ## -usage="Usage: spark-daemon.sh [--config ] (start|stop) " +usage="Usage: spark-daemon.sh [--config ] (start|stop|status) " # if no args specified, show usage if [ $# -le 1 ]; then @@ -195,6 +195,23 @@ case $option in fi ;; + (status) + + if [ -f $pid ]; then + TARGET_ID="$(cat "$pid")" + if [[ $(ps -p "$TARGET_ID" -o comm=) =~ "java" ]]; then + echo $command is running. + exit 0 + else + echo $pid file is present but $command not running + exit 1 + fi + else + echo $command not running. + exit 2 + fi + ;; + (*) echo $usage exit 1 From 202ebf06e0f2d5df8b712e604fd95fa58e34ea20 Mon Sep 17 00:00:00 2001 From: linweizhong Date: Mon, 13 Apr 2015 13:06:54 +0100 Subject: [PATCH 064/128] [SPARK-6870][Yarn] Catch InterruptedException when yarn application state monitor thread been interrupted On PR #5305 we interrupt the monitor thread but forget to catch the InterruptedException, then in the log will print the stack info, so we need to catch it. Author: linweizhong Closes #5479 from Sephiroth-Lin/SPARK-6870 and squashes the following commits: f775f93 [linweizhong] Update, don't need to call Thread.currentThread() on monitor thread 0e2ef1f [linweizhong] Update 0d8958a [linweizhong] Update 3513fdb [linweizhong] Catch InterruptedException --- .../cluster/YarnClientSchedulerBackend.scala | 11 +++++++---- 1 file changed, 7 insertions(+), 4 deletions(-) diff --git a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 407dc1ac4d37d..99c05329b4d73 100644 --- a/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -128,10 +128,13 @@ private[spark] class YarnClientSchedulerBackend( assert(client != null && appId != null, "Application has not been submitted yet!") val t = new Thread { override def run() { - val (state, _) = client.monitorApplication(appId, logApplicationReport = false) - logError(s"Yarn application has already exited with state $state!") - sc.stop() - Thread.currentThread().interrupt() + try { + val (state, _) = client.monitorApplication(appId, logApplicationReport = false) + logError(s"Yarn application has already exited with state $state!") + sc.stop() + } catch { + case e: InterruptedException => logInfo("Interrupting monitor thread") + } } } t.setName("Yarn application state monitor") From b29663eeea440b1d1a288d41b5ddf67e77c5bd54 Mon Sep 17 00:00:00 2001 From: Pei-Lun Lee Date: Mon, 13 Apr 2015 21:52:00 +0800 Subject: [PATCH 065/128] [SPARK-6352] [SQL] Add DirectParquetOutputCommitter Add a DirectParquetOutputCommitter class that skips _temporary directory when saving to s3. Add new config value "spark.sql.parquet.useDirectParquetOutputCommitter" (default false) to choose between the default output committer. Author: Pei-Lun Lee Closes #5042 from ypcat/spark-6352 and squashes the following commits: e17bf47 [Pei-Lun Lee] Merge branch 'master' of https://github.com/apache/spark into spark-6352 9ae7545 [Pei-Lun Lee] [SPARL-6352] [SQL] Change to allow custom parquet output committer. 0d540b9 [Pei-Lun Lee] [SPARK-6352] [SQL] add license c42468c [Pei-Lun Lee] [SPARK-6352] [SQL] add test case 0fc03ca [Pei-Lun Lee] [SPARK-6532] [SQL] hide class DirectParquetOutputCommitter 769bd67 [Pei-Lun Lee] DirectParquetOutputCommitter f75e261 [Pei-Lun Lee] DirectParquetOutputCommitter --- .../DirectParquetOutputCommitter.scala | 66 +++++++++++++++++++ .../sql/parquet/ParquetTableOperations.scala | 22 +++++++ .../spark/sql/parquet/ParquetIOSuite.scala | 21 ++++++ 3 files changed, 109 insertions(+) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala new file mode 100644 index 0000000000000..25a66cb488103 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.parquet + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} +import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter + +import parquet.Log +import parquet.hadoop.util.ContextUtil +import parquet.hadoop.{ParquetFileReader, ParquetFileWriter, ParquetOutputCommitter} + +private[parquet] class DirectParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) + extends ParquetOutputCommitter(outputPath, context) { + val LOG = Log.getLog(classOf[ParquetOutputCommitter]) + + override def getWorkPath(): Path = outputPath + override def abortTask(taskContext: TaskAttemptContext): Unit = {} + override def commitTask(taskContext: TaskAttemptContext): Unit = {} + override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = true + override def setupJob(jobContext: JobContext): Unit = {} + override def setupTask(taskContext: TaskAttemptContext): Unit = {} + + override def commitJob(jobContext: JobContext) { + try { + val configuration = ContextUtil.getConfiguration(jobContext) + val fileSystem = outputPath.getFileSystem(configuration) + val outputStatus = fileSystem.getFileStatus(outputPath) + val footers = ParquetFileReader.readAllFootersInParallel(configuration, outputStatus) + try { + ParquetFileWriter.writeMetadataFile(configuration, outputPath, footers) + if (configuration.getBoolean("mapreduce.fileoutputcommitter.marksuccessfuljobs", true)) { + val successPath = new Path(outputPath, FileOutputCommitter.SUCCEEDED_FILE_NAME) + fileSystem.create(successPath).close() + } + } catch { + case e: Exception => { + LOG.warn("could not write summary file for " + outputPath, e) + val metadataPath = new Path(outputPath, ParquetFileWriter.PARQUET_METADATA_FILE) + if (fileSystem.exists(metadataPath)) { + fileSystem.delete(metadataPath, true) + } + } + } + } catch { + case e: Exception => LOG.warn("could not write summary file for " + outputPath, e) + } + } + +} + 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 1c868da23e060..3724bda829d30 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 @@ -379,6 +379,8 @@ private[sql] case class InsertIntoParquetTable( */ private[parquet] class AppendingParquetOutputFormat(offset: Int) extends parquet.hadoop.ParquetOutputFormat[Row] { + var committer: OutputCommitter = null + // override to accept existing directories as valid output directory override def checkOutputSpecs(job: JobContext): Unit = {} @@ -403,6 +405,26 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) private def getTaskAttemptID(context: TaskAttemptContext): TaskAttemptID = { context.getClass.getMethod("getTaskAttemptID").invoke(context).asInstanceOf[TaskAttemptID] } + + // override to create output committer from configuration + override def getOutputCommitter(context: TaskAttemptContext): OutputCommitter = { + if (committer == null) { + val output = getOutputPath(context) + val cls = context.getConfiguration.getClass("spark.sql.parquet.output.committer.class", + classOf[ParquetOutputCommitter], classOf[ParquetOutputCommitter]) + val ctor = cls.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) + committer = ctor.newInstance(output, context).asInstanceOf[ParquetOutputCommitter] + } + committer + } + + // FileOutputFormat.getOutputPath takes JobConf in hadoop-1 but JobContext in hadoop-2 + private def getOutputPath(context: TaskAttemptContext): Path = { + context.getConfiguration().get("mapred.output.dir") match { + case null => null + case name => new Path(name) + } + } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 97c0f439acf13..4d0bf7cf99cdf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -381,6 +381,27 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } } } + + test("SPARK-6352 DirectParquetOutputCommitter") { + // Write to a parquet file and let it fail. + // _temporary should be missing if direct output committer works. + try { + configuration.set("spark.sql.parquet.output.committer.class", + "org.apache.spark.sql.parquet.DirectParquetOutputCommitter") + sqlContext.udf.register("div0", (x: Int) => x / 0) + withTempPath { dir => + intercept[org.apache.spark.SparkException] { + sqlContext.sql("select div0(1)").saveAsParquetFile(dir.getCanonicalPath) + } + val path = new Path(dir.getCanonicalPath, "_temporary") + val fs = path.getFileSystem(configuration) + assert(!fs.exists(path)) + } + } + finally { + configuration.unset("spark.sql.parquet.output.committer.class") + } + } } class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { From 77620be76e82b6cdaae406cd752d3272656f5fe0 Mon Sep 17 00:00:00 2001 From: Doug Balog Date: Mon, 13 Apr 2015 09:49:58 -0500 Subject: [PATCH 066/128] [SPARK-6207] [YARN] [SQL] Adds delegation tokens for metastore to conf. Adds hive2-metastore delegation token to conf when running in secure mode. Without this change, running on YARN in cluster mode fails with a GSS exception. This is a rough patch that adds a dependency to spark/yarn on hive-exec. I'm looking for suggestions on how to make this patch better. This contribution is my original work and that I licenses the work to the Apache Spark project under the project's open source licenses. Author: Doug Balog Author: Doug Balog Closes #5031 from dougb/SPARK-6207 and squashes the following commits: 3e9ac16 [Doug Balog] [SPARK-6207] Fixes minor code spacing issues. e260765 [Doug Balog] [SPARK-6207] Second pass at adding Hive delegation token to conf. - Use reflection instead of adding dependency on hive. - Tested on Hive 0.13 and Hadoop 2.4.1 1ab1729 [Doug Balog] Merge branch 'master' of git://github.com/apache/spark into SPARK-6207 bf356d2 [Doug Balog] [SPARK-6207] [YARN] [SQL] Adds delegation tokens for metastore to conf. Adds hive2-metastore delagations token to conf when running in securemode. Without this change, runing on YARN in cluster mode fails with a GSS exception. --- .../org/apache/spark/deploy/yarn/Client.scala | 63 +++++++++++++++++++ 1 file changed, 63 insertions(+) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index c1effd3c8a718..1091ff54b0463 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -22,17 +22,21 @@ import java.nio.ByteBuffer import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap, ListBuffer, Map} +import scala.reflect.runtime.universe import scala.util.{Try, Success, Failure} import com.google.common.base.Objects import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission +import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.Master import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.Token import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment @@ -220,6 +224,7 @@ private[spark] class Client( val dst = new Path(fs.getHomeDirectory(), appStagingDir) val nns = getNameNodesToAccess(sparkConf) + dst obtainTokensForNamenodes(nns, hadoopConf, credentials) + obtainTokenForHiveMetastore(hadoopConf, credentials) val replication = sparkConf.getInt("spark.yarn.submit.file.replication", fs.getDefaultReplication(dst)).toShort @@ -936,6 +941,64 @@ object Client extends Logging { } } + /** + * Obtains token for the Hive metastore and adds them to the credentials. + */ + private def obtainTokenForHiveMetastore(conf: Configuration, credentials: Credentials) { + if (UserGroupInformation.isSecurityEnabled) { + val mirror = universe.runtimeMirror(getClass.getClassLoader) + + try { + val hiveClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.ql.metadata.Hive") + val hive = hiveClass.getMethod("get").invoke(null) + + val hiveConf = hiveClass.getMethod("getConf").invoke(hive) + val hiveConfClass = mirror.classLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf") + + val hiveConfGet = (param:String) => Option(hiveConfClass + .getMethod("get", classOf[java.lang.String]) + .invoke(hiveConf, param)) + + val metastore_uri = hiveConfGet("hive.metastore.uris") + + // Check for local metastore + if (metastore_uri != None && metastore_uri.get.toString.size > 0) { + val metastore_kerberos_principal_conf_var = mirror.classLoader + .loadClass("org.apache.hadoop.hive.conf.HiveConf$ConfVars") + .getField("METASTORE_KERBEROS_PRINCIPAL").get("varname").toString + + val principal = hiveConfGet(metastore_kerberos_principal_conf_var) + + val username = Option(UserGroupInformation.getCurrentUser().getUserName) + if (principal != None && username != None) { + val tokenStr = hiveClass.getMethod("getDelegationToken", + classOf[java.lang.String], classOf[java.lang.String]) + .invoke(hive, username.get, principal.get).asInstanceOf[java.lang.String] + + val hive2Token = new Token[DelegationTokenIdentifier]() + hive2Token.decodeFromUrlString(tokenStr) + credentials.addToken(new Text("hive.server2.delegation.token"),hive2Token) + logDebug("Added hive.Server2.delegation.token to conf.") + hiveClass.getMethod("closeCurrent").invoke(null) + } else { + logError("Username or principal == NULL") + logError(s"""username=${username.getOrElse("(NULL)")}""") + logError(s"""principal=${principal.getOrElse("(NULL)")}""") + throw new IllegalArgumentException("username and/or principal is equal to null!") + } + } else { + logDebug("HiveMetaStore configured in localmode") + } + } catch { + case e:java.lang.NoSuchMethodException => { logInfo("Hive Method not found " + e); return } + case e:java.lang.ClassNotFoundException => { logInfo("Hive Class not found " + e); return } + case e:Exception => { logError("Unexpected Exception " + e) + throw new RuntimeException("Unexpected exception", e) + } + } + } + } + /** * Return whether the two file systems are the same. */ From c5b0b296b842926b5c07531a5affe8984bc799c5 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 13 Apr 2015 09:29:04 -0700 Subject: [PATCH 067/128] [SPARK-6765] Enable scalastyle on test code. Turn scalastyle on for all test code. Most of the violations have been resolved in my previous pull requests: Core: https://github.com/apache/spark/pull/5484 SQL: https://github.com/apache/spark/pull/5412 MLlib: https://github.com/apache/spark/pull/5411 GraphX: https://github.com/apache/spark/pull/5410 Streaming: https://github.com/apache/spark/pull/5409 Author: Reynold Xin Closes #5486 from rxin/test-style-enable and squashes the following commits: 01683de [Reynold Xin] Fixed new code. a4ab46e [Reynold Xin] Fixed tests. 20adbc8 [Reynold Xin] Missed one violation. 5e36521 [Reynold Xin] [SPARK-6765] Enable scalastyle on test code. --- .../org/apache/spark/deploy/ClientSuite.scala | 1 - dev/scalastyle | 5 ++-- .../spark/ml/feature/VectorIndexerSuite.scala | 3 +- .../sql/hive/thriftserver/CliSuite.scala | 15 +++++----- .../HiveThriftServer2Suites.scala | 7 +++-- .../spark/sql/hive/QueryPartitionSuite.scala | 30 ++++++++++--------- .../StreamingJobProgressListenerSuite.scala | 4 +-- .../spark/deploy/yarn/ClientSuite.scala | 19 ++++++++---- .../deploy/yarn/YarnAllocatorSuite.scala | 6 ++-- .../yarn/YarnSparkHadoopUtilSuite.scala | 2 +- 10 files changed, 52 insertions(+), 40 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala index 518073dcbb64e..745f9eeee7536 100644 --- a/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/ClientSuite.scala @@ -46,5 +46,4 @@ class ClientSuite extends FunSuite with Matchers { // Invalid syntax. ClientArguments.isValidJarUrl("hdfs:") should be (false) } - } diff --git a/dev/scalastyle b/dev/scalastyle index 86919227ed1ab..4e03f89ed5d5d 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -18,9 +18,10 @@ # echo -e "q\n" | build/sbt -Phive -Phive-thriftserver scalastyle > scalastyle.txt +echo -e "q\n" | build/sbt -Phive -Phive-thriftserver test:scalastyle >> scalastyle.txt # Check style with YARN built too -echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle \ - >> scalastyle.txt +echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 scalastyle >> scalastyle.txt +echo -e "q\n" | build/sbt -Pyarn -Phadoop-2.2 -Dhadoop.version=2.2.0 test:scalastyle >> scalastyle.txt ERRORS=$(cat scalastyle.txt | awk '{if($1~/error/)print}') rm scalastyle.txt diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala index 61c46c85a78b5..81ef831c42e55 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -150,7 +150,8 @@ class VectorIndexerSuite extends FunSuite with MLlibTestSparkContext { val vectorIndexer = getIndexer.setMaxCategories(maxCategories) val model = vectorIndexer.fit(data) val categoryMaps = model.categoryMaps - assert(categoryMaps.keys.toSet === categoricalFeatures) // Chose correct categorical features + // Chose correct categorical features + assert(categoryMaps.keys.toSet === categoricalFeatures) val transformed = model.transform(data).select("indexed") val indexedRDD: RDD[Vector] = transformed.map(_.getAs[Vector](0)) val featureAttrs = AttributeGroup.fromStructField(transformed.schema("indexed")) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 75738fa22b572..6d1d7c3a4e698 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -1,13 +1,12 @@ /* - * 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 + * 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 + * 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, diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index bf20acecb1f32..4cf95e7bdfb2b 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.thriftserver import java.io.File +import java.net.URL import java.sql.{Date, DriverManager, Statement} import scala.collection.mutable.ArrayBuffer @@ -41,7 +42,7 @@ import org.apache.spark.sql.hive.HiveShim import org.apache.spark.util.Utils object TestData { - def getTestDataFilePath(name: String) = { + def getTestDataFilePath(name: String): URL = { Thread.currentThread().getContextClassLoader.getResource(s"data/files/$name") } @@ -50,7 +51,7 @@ object TestData { } class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { - override def mode = ServerMode.binary + override def mode: ServerMode.Value = ServerMode.binary private def withCLIServiceClient(f: ThriftCLIServiceClient => Unit): Unit = { // Transport creation logics below mimics HiveConnection.createBinaryTransport @@ -337,7 +338,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { } class HiveThriftHttpServerSuite extends HiveThriftJdbcTest { - override def mode = ServerMode.http + override def mode: ServerMode.Value = ServerMode.http test("JDBC query execution") { withJdbcStatement { statement => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala index 83f97128c5e83..a787fa5546e76 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/QueryPartitionSuite.scala @@ -17,15 +17,12 @@ package org.apache.spark.sql.hive -import java.io.File - import com.google.common.io.Files + import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.hive.test.TestHive -import org.apache.spark.util.Utils -/* Implicits */ import org.apache.spark.sql.hive.test.TestHive._ - +import org.apache.spark.util.Utils class QueryPartitionSuite extends QueryTest { @@ -37,23 +34,28 @@ class QueryPartitionSuite extends QueryTest { testData.registerTempTable("testData") val tmpDir = Files.createTempDir() - //create the table for test - sql(s"CREATE TABLE table_with_partition(key int,value string) PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') SELECT key,value FROM testData") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='2') SELECT key,value FROM testData") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='3') SELECT key,value FROM testData") - sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='4') SELECT key,value FROM testData") + // create the table for test + sql(s"CREATE TABLE table_with_partition(key int,value string) " + + s"PARTITIONED by (ds string) location '${tmpDir.toURI.toString}' ") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='1') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='2') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='3') " + + "SELECT key,value FROM testData") + sql("INSERT OVERWRITE TABLE table_with_partition partition (ds='4') " + + "SELECT key,value FROM testData") - //test for the exist path + // test for the exist path checkAnswer(sql("select key,value from table_with_partition"), testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect) - //delect the path of one partition + // delete the path of one partition val folders = tmpDir.listFiles.filter(_.isDirectory) Utils.deleteRecursively(folders(0)) - //test for affter delete the path + // test for after delete the path checkAnswer(sql("select key,value from table_with_partition"), testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect ++ testData.toSchemaRDD.collect) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala index 2b9d164500b72..94b1985116feb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ui/StreamingJobProgressListenerSuite.scala @@ -21,14 +21,14 @@ import org.scalatest.Matchers import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.scheduler._ -import org.apache.spark.streaming.{Time, Milliseconds, TestSuiteBase} +import org.apache.spark.streaming.{Duration, Time, Milliseconds, TestSuiteBase} class StreamingJobProgressListenerSuite extends TestSuiteBase with Matchers { val input = (1 to 4).map(Seq(_)).toSeq val operation = (d: DStream[Int]) => d.map(x => x) - override def batchDuration = Milliseconds(100) + override def batchDuration: Duration = Milliseconds(100) test("onBatchSubmitted, onBatchStarted, onBatchCompleted, " + "onReceiverStarted, onReceiverError, onReceiverStopped") { diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 92f04b4b859b3..c1b94ac9c5bdd 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -232,19 +232,26 @@ class ClientSuite extends FunSuite with Matchers with BeforeAndAfterAll { testCode(conf) } - def newEnv = MutableHashMap[String, String]() + def newEnv: MutableHashMap[String, String] = MutableHashMap[String, String]() - def classpath(env: MutableHashMap[String, String]) = env(Environment.CLASSPATH.name).split(":|;|") + def classpath(env: MutableHashMap[String, String]): Array[String] = + env(Environment.CLASSPATH.name).split(":|;|") - def flatten(a: Option[Seq[String]], b: Option[Seq[String]]) = (a ++ b).flatten.toArray + def flatten(a: Option[Seq[String]], b: Option[Seq[String]]): Array[String] = + (a ++ b).flatten.toArray - def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = - Try(clazz.getField(field)).map(_.get(null).asInstanceOf[A]).toOption.map(mapTo).getOrElse(defaults) + def getFieldValue[A, B](clazz: Class[_], field: String, defaults: => B)(mapTo: A => B): B = { + Try(clazz.getField(field)) + .map(_.get(null).asInstanceOf[A]) + .toOption + .map(mapTo) + .getOrElse(defaults) + } def getFieldValue2[A: ClassTag, A1: ClassTag, B]( clazz: Class[_], field: String, - defaults: => B)(mapTo: A => B)(mapTo1: A1 => B) : B = { + defaults: => B)(mapTo: A => B)(mapTo1: A1 => B): B = { Try(clazz.getField(field)).map(_.get(null)).map { case v: A => mapTo(v) case v1: A1 => mapTo1(v1) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index c09b01bafce37..455f1019d86dd 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -79,7 +79,7 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach } class MockSplitInfo(host: String) extends SplitInfo(null, host, null, 1, null) { - override def equals(other: Any) = false + override def equals(other: Any): Boolean = false } def createAllocator(maxExecutors: Int = 5): YarnAllocator = { @@ -118,7 +118,9 @@ class YarnAllocatorSuite extends FunSuite with Matchers with BeforeAndAfterEach handler.getNumExecutorsRunning should be (1) handler.allocatedContainerToHostMap.get(container.getId).get should be ("host1") handler.allocatedHostToContainersMap.get("host1").get should contain (container.getId) - rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size should be (0) + + val size = rmClient.getMatchingRequests(container.getPriority, "host1", containerResource).size + size should be (0) } test("some containers allocated") { diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index 4194f36499e66..9395316b71ff4 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -46,7 +46,7 @@ class YarnSparkHadoopUtilSuite extends FunSuite with Matchers with Logging { logWarning("Cannot execute bash, skipping bash tests.") } - def bashTest(name: String)(fn: => Unit) = + def bashTest(name: String)(fn: => Unit): Unit = if (hasBash) test(name)(fn) else ignore(name)(fn) bashTest("shell script escaping") { From 6cc5b3ed3c0c729f97956fa017d8eb7d6b43f90f Mon Sep 17 00:00:00 2001 From: Cheolsoo Park Date: Mon, 13 Apr 2015 13:45:10 -0500 Subject: [PATCH 068/128] [SPARK-6662][YARN] Allow variable substitution in spark.yarn.historyServer.address In Spark on YARN, explicit hostname and port number need to be set for "spark.yarn.historyServer.address" in SparkConf to make the HISTORY link. If the history server address is known and static, this is usually not a problem. But in cloud, that is usually not true. Particularly in EMR, the history server always runs on the same node as with RM. So I could simply set it to ${yarn.resourcemanager.hostname}:18080 if variable substitution is allowed. In fact, Hadoop configuration already implements variable substitution, so if this property is read via YarnConf, this can be easily achievable. Author: Cheolsoo Park Closes #5321 from piaozhexiu/SPARK-6662 and squashes the following commits: e37de75 [Cheolsoo Park] Preserve the space between the Hadoop and Spark imports 79757c6 [Cheolsoo Park] Incorporate review comments 10e2917 [Cheolsoo Park] Add helper function that substitutes hadoop vars to SparkHadoopUtil 589b52c [Cheolsoo Park] Revert "Allow variable substitution for spark.yarn. properties" ff9c35d [Cheolsoo Park] Allow variable substitution for spark.yarn. properties --- .../apache/spark/deploy/SparkHadoopUtil.scala | 38 +++++++++++++++++-- docs/running-on-yarn.md | 3 +- .../spark/deploy/yarn/ApplicationMaster.scala | 1 + 3 files changed, 37 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index c2568eb4b60ac..cfaebf9ea5050 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -24,11 +24,10 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.hadoop.fs.FileSystem.Statistics import org.apache.hadoop.mapred.JobConf -import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} -import org.apache.hadoop.security.Credentials -import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.security.{Credentials, UserGroupInformation} -import org.apache.spark.{Logging, SparkContext, SparkConf, SparkException} +import org.apache.spark.{Logging, SparkConf, SparkException} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils @@ -201,6 +200,37 @@ class SparkHadoopUtil extends Logging { val baseStatus = fs.getFileStatus(basePath) if (baseStatus.isDir) recurse(basePath) else Array(baseStatus) } + + private val HADOOP_CONF_PATTERN = "(\\$\\{hadoopconf-[^\\}\\$\\s]+\\})".r.unanchored + + /** + * Substitute variables by looking them up in Hadoop configs. Only variables that match the + * ${hadoopconf- .. } pattern are substituted. + */ + def substituteHadoopVariables(text: String, hadoopConf: Configuration): String = { + text match { + case HADOOP_CONF_PATTERN(matched) => { + logDebug(text + " matched " + HADOOP_CONF_PATTERN) + val key = matched.substring(13, matched.length() - 1) // remove ${hadoopconf- .. } + val eval = Option[String](hadoopConf.get(key)) + .map { value => + logDebug("Substituted " + matched + " with " + value) + text.replace(matched, value) + } + if (eval.isEmpty) { + // The variable was not found in Hadoop configs, so return text as is. + text + } else { + // Continue to substitute more variables. + substituteHadoopVariables(eval.get, hadoopConf) + } + } + case _ => { + logDebug(text + " didn't match " + HADOOP_CONF_PATTERN) + text + } + } + } } object SparkHadoopUtil { diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index b7e68d4f71714..ed5bb263a5809 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -87,7 +87,8 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.historyServer.address (none) - The address of the Spark history server (i.e. host.com:18080). The address should not contain a scheme (http://). Defaults to not being set since the history server is an optional service. This address is given to the YARN ResourceManager when the Spark application finishes to link the application from the ResourceManager UI to the Spark history server UI. + The address of the Spark history server (i.e. host.com:18080). The address should not contain a scheme (http://). Defaults to not being set since the history server is an optional service. This address is given to the YARN ResourceManager when the Spark application finishes to link the application from the ResourceManager UI to the Spark history server UI. + For this property, YARN properties can be used as variables, and these are substituted by Spark at runtime. For eg, if the Spark history server runs on the same node as the YARN ResourceManager, it can be set to `${hadoopconf-yarn.resourcemanager.hostname}:18080`. diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 32bc4e5663062..26259cee77151 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -223,6 +223,7 @@ private[spark] class ApplicationMaster( val appId = client.getAttemptId().getApplicationId().toString() val historyAddress = sparkConf.getOption("spark.yarn.historyServer.address") + .map { text => SparkHadoopUtil.get.substituteHadoopVariables(text, yarnConf) } .map { address => s"${address}${HistoryServer.UI_PATH_PREFIX}/${appId}" } .getOrElse("") From 1e340c3ae4d5361d048a3d6990f144cfc923666f Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Mon, 13 Apr 2015 11:53:17 -0700 Subject: [PATCH 069/128] [SPARK-5988][MLlib] add save/load for PowerIterationClusteringModel See JIRA issue [SPARK-5988](https://issues.apache.org/jira/browse/SPARK-5988). Author: Xusen Yin Closes #5450 from yinxusen/SPARK-5988 and squashes the following commits: cb1ecfa [Xusen Yin] change Assignment into case class b1dd24c [Xusen Yin] add test suite 63c3923 [Xusen Yin] add save load for power iteration clustering --- .../clustering/PowerIterationClustering.scala | 68 +++++++++++++++++-- .../PowerIterationClusteringSuite.scala | 34 ++++++++++ 2 files changed, 97 insertions(+), 5 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index 180023922a9b0..aa53e88d59856 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -17,15 +17,20 @@ package org.apache.spark.mllib.clustering -import org.apache.spark.{Logging, SparkException} +import org.json4s.JsonDSL._ +import org.json4s._ +import org.json4s.jackson.JsonMethods._ + import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaRDD import org.apache.spark.graphx._ import org.apache.spark.graphx.impl.GraphImpl import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.mllib.util.{Loader, MLUtils, Saveable} import org.apache.spark.rdd.RDD +import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.{Logging, SparkContext, SparkException} /** * :: Experimental :: @@ -38,7 +43,60 @@ import org.apache.spark.util.random.XORShiftRandom @Experimental class PowerIterationClusteringModel( val k: Int, - val assignments: RDD[PowerIterationClustering.Assignment]) extends Serializable + val assignments: RDD[PowerIterationClustering.Assignment]) extends Saveable with Serializable { + + override def save(sc: SparkContext, path: String): Unit = { + PowerIterationClusteringModel.SaveLoadV1_0.save(sc, this, path) + } + + override protected def formatVersion: String = "1.0" +} + +object PowerIterationClusteringModel extends Loader[PowerIterationClusteringModel] { + override def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { + PowerIterationClusteringModel.SaveLoadV1_0.load(sc, path) + } + + private[clustering] + object SaveLoadV1_0 { + + private val thisFormatVersion = "1.0" + + private[clustering] + val thisClassName = "org.apache.spark.mllib.clustering.PowerIterationClusteringModel" + + def save(sc: SparkContext, model: PowerIterationClusteringModel, path: String): Unit = { + val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ + + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) ~ ("k" -> model.k))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + val dataRDD = model.assignments.toDF() + dataRDD.saveAsParquetFile(Loader.dataPath(path)) + } + + def load(sc: SparkContext, path: String): PowerIterationClusteringModel = { + implicit val formats = DefaultFormats + val sqlContext = new SQLContext(sc) + + val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) + assert(className == thisClassName) + assert(formatVersion == thisFormatVersion) + + val k = (metadata \ "k").extract[Int] + val assignments = sqlContext.parquetFile(Loader.dataPath(path)) + Loader.checkSchema[PowerIterationClustering.Assignment](assignments.schema) + + val assignmentsRDD = assignments.map { + case Row(id: Long, cluster: Int) => PowerIterationClustering.Assignment(id, cluster) + } + + new PowerIterationClusteringModel(k, assignmentsRDD) + } + } +} /** * :: Experimental :: @@ -135,7 +193,7 @@ class PowerIterationClustering private[clustering] ( val v = powerIter(w, maxIterations) val assignments = kMeans(v, k).mapPartitions({ iter => iter.map { case (id, cluster) => - new Assignment(id, cluster) + Assignment(id, cluster) } }, preservesPartitioning = true) new PowerIterationClusteringModel(k, assignments) @@ -152,7 +210,7 @@ object PowerIterationClustering extends Logging { * @param cluster assigned cluster id */ @Experimental - class Assignment(val id: Long, val cluster: Int) extends Serializable + case class Assignment(id: Long, cluster: Int) /** * Normalizes the affinity matrix (A) by row sums and returns the normalized affinity matrix (W). diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala index 6315c03a700f1..6d6fe6fe46bab 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala @@ -18,12 +18,15 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable +import scala.util.Random import org.scalatest.FunSuite +import org.apache.spark.SparkContext import org.apache.spark.graphx.{Edge, Graph} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.util.Utils class PowerIterationClusteringSuite extends FunSuite with MLlibTestSparkContext { @@ -110,4 +113,35 @@ class PowerIterationClusteringSuite extends FunSuite with MLlibTestSparkContext assert(x ~== u1(i.toInt) absTol 1e-14) } } + + test("model save/load") { + val tempDir = Utils.createTempDir() + val path = tempDir.toURI.toString + val model = PowerIterationClusteringSuite.createModel(sc, 3, 10) + try { + model.save(sc, path) + val sameModel = PowerIterationClusteringModel.load(sc, path) + PowerIterationClusteringSuite.checkEqual(model, sameModel) + } finally { + Utils.deleteRecursively(tempDir) + } + } +} + +object PowerIterationClusteringSuite extends FunSuite { + def createModel(sc: SparkContext, k: Int, nPoints: Int): PowerIterationClusteringModel = { + val assignments = sc.parallelize( + (0 until nPoints).map(p => PowerIterationClustering.Assignment(p, Random.nextInt(k)))) + new PowerIterationClusteringModel(k, assignments) + } + + def checkEqual(a: PowerIterationClusteringModel, b: PowerIterationClusteringModel): Unit = { + assert(a.k === b.k) + + val aAssignments = a.assignments.map(x => (x.id, x.cluster)) + val bAssignments = b.assignments.map(x => (x.id, x.cluster)) + val unequalElements = aAssignments.join(bAssignments).filter { + case (id, (c1, c2)) => c1 != c2 }.count() + assert(unequalElements === 0L) + } } From 85ee0cabe87a27b6947c2d3e8525f04c77f80f6f Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Apr 2015 14:29:07 -0700 Subject: [PATCH 070/128] [SPARK-6130] [SQL] support if not exists for insert overwrite into partition in hiveQl MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Standard syntax: INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1 FROM from_statement; INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1 FROM from_statement;   Hive extension (multiple inserts): FROM from_statement INSERT OVERWRITE TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...) [IF NOT EXISTS]] select_statement1 [INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2] [INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2] ...; FROM from_statement INSERT INTO TABLE tablename1 [PARTITION (partcol1=val1, partcol2=val2 ...)] select_statement1 [INSERT INTO TABLE tablename2 [PARTITION ...] select_statement2] [INSERT OVERWRITE TABLE tablename2 [PARTITION ... [IF NOT EXISTS]] select_statement2] ...;   Hive extension (dynamic partition inserts): INSERT OVERWRITE TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement; INSERT INTO TABLE tablename PARTITION (partcol1[=val1], partcol2[=val2] ...) select_statement FROM from_statement; Author: Daoyuan Wang Closes #4865 from adrian-wang/insertoverwrite and squashes the following commits: 2fce94f [Daoyuan Wang] add assert 10ea6f3 [Daoyuan Wang] add name for boolean parameter 0bbe9b9 [Daoyuan Wang] fix failure 4391154 [Daoyuan Wang] support if not exists for insert overwrite into partition in hiveQl --- .../apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../sql/catalyst/analysis/Analyzer.scala | 2 +- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../plans/logical/basicOperators.scala | 4 ++- .../org/apache/spark/sql/DataFrame.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 3 +- .../sql/sources/DataSourceStrategy.scala | 2 +- .../org/apache/spark/sql/sources/rules.scala | 7 +++-- .../execution/HiveCompatibilitySuite.scala | 1 + .../spark/sql/hive/HiveMetastoreCatalog.scala | 19 ++++++------ .../org/apache/spark/sql/hive/HiveQl.scala | 22 ++++++++++++- .../spark/sql/hive/HiveStrategies.scala | 10 +++--- .../hive/execution/CreateTableAsSelect.scala | 2 +- .../hive/execution/InsertIntoHiveTable.scala | 31 +++++++++++++------ ...titions-0-d5edc0daa94b33915df794df3b710774 | 0 ...titions-1-9eb9372f4855928fae16f5fa554b3a62 | 0 ...itions-10-ec2cef3d37146c450c60202a572f5cab | 0 ...itions-11-8854d6001200fc11529b2e2da755e5a2 | 0 ...itions-12-71ff68fda0aa7a36cb50d8fab0d70d25 | 0 ...titions-13-7e4e7d7003fc6ef17bc19c3461ad899 | 0 ...itions-14-ec2cef3d37146c450c60202a572f5cab | 0 ...itions-15-a3b2e230efde74e970ae8a3b55f383fc | 0 ...titions-2-8396c17a66e3d9a374d4361873b9bfe3 | 0 ...titions-3-3876bb356dd8af7e78d061093d555457 | 0 ...rtitions-4-528e23afb272c2e69004c86ddaa70ee | 0 ...titions-5-de5d56456c28d63775554e56355911d2 | 0 ...titions-6-3efdc331b3b4bdac3e60c757600fff53 | 5 +++ ...titions-7-92f6af82704504968de078c133f222f8 | 0 ...titions-8-316cad7c63ddd4fb043be2affa5b0a67 | 0 ...titions-9-3efdc331b3b4bdac3e60c757600fff53 | 5 +++ 30 files changed, 84 insertions(+), 35 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-0-d5edc0daa94b33915df794df3b710774 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-1-9eb9372f4855928fae16f5fa554b3a62 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-10-ec2cef3d37146c450c60202a572f5cab create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-11-8854d6001200fc11529b2e2da755e5a2 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-12-71ff68fda0aa7a36cb50d8fab0d70d25 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-13-7e4e7d7003fc6ef17bc19c3461ad899 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-14-ec2cef3d37146c450c60202a572f5cab create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-15-a3b2e230efde74e970ae8a3b55f383fc create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-2-8396c17a66e3d9a374d4361873b9bfe3 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-3-3876bb356dd8af7e78d061093d555457 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-4-528e23afb272c2e69004c86ddaa70ee create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-5-de5d56456c28d63775554e56355911d2 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-7-92f6af82704504968de078c133f222f8 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-8-316cad7c63ddd4fb043be2affa5b0a67 create mode 100644 sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 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 ee04cb579deb6..bc8d3751f6616 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -155,7 +155,7 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { protected lazy val insert: Parser[LogicalPlan] = INSERT ~> (OVERWRITE ^^^ true | INTO ^^^ false) ~ (TABLE ~> relation) ~ select ^^ { - case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o) + case o ~ r ~ s => InsertIntoTable(r, Map.empty[String, Option[String]], s, o, false) } protected lazy val cte: Parser[LogicalPlan] = 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 50702ac6832ec..8b68b0df35f48 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 @@ -193,7 +193,7 @@ class Analyzer( } realPlan transform { - case i@InsertIntoTable(u: UnresolvedRelation, _, _, _) => + case i@InsertIntoTable(u: UnresolvedRelation, _, _, _, _) => i.copy( table = EliminateSubQueries(getTable(u, cteRelations))) case u: UnresolvedRelation => 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 145f062dd6817..21c15ad14fd19 100644 --- 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 @@ -293,7 +293,7 @@ package object dsl { def insertInto(tableName: String, overwrite: Boolean = false): LogicalPlan = InsertIntoTable( - analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite) + analysis.UnresolvedRelation(Seq(tableName)), Map.empty, logicalPlan, overwrite, false) def analyze: LogicalPlan = EliminateSubQueries(analysis.SimpleAnalyzer(logicalPlan)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala index 5d31a6eecfce2..17522976dc2c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -125,12 +125,14 @@ case class InsertIntoTable( table: LogicalPlan, partition: Map[String, Option[String]], child: LogicalPlan, - overwrite: Boolean) + overwrite: Boolean, + ifNotExists: Boolean) extends LogicalPlan { override def children: Seq[LogicalPlan] = child :: Nil override def output: Seq[Attribute] = child.output + assert(overwrite || !ifNotExists) override lazy val resolved: Boolean = childrenResolved && child.output.zip(table.output).forall { case (childAttr, tableAttr) => DataType.equalsIgnoreCompatibleNullability(childAttr.dataType, tableAttr.dataType) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 9b9adf855077a..94ae2d65fd0e4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1209,7 +1209,7 @@ class DataFrame private[sql]( @Experimental def insertInto(tableName: String, overwrite: Boolean): Unit = { sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), - Map.empty, logicalPlan, overwrite)).toRdd + Map.empty, logicalPlan, overwrite, ifNotExists = false)).toRdd } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 23f7e5609414b..5268b7334051a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -211,7 +211,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, sqlContext) // Note: overwrite=false because otherwise the metadata we just created will be deleted InsertIntoParquetTable(relation, planLater(child), overwrite = false) :: Nil - case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => + case logical.InsertIntoTable( + table: ParquetRelation, partition, child, overwrite, ifNotExists) => InsertIntoParquetTable(table, planLater(child), overwrite) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => val prunePushedDownFilters = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index e13759b7feb7b..34d048e426d10 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -56,7 +56,7 @@ private[sql] object DataSourceStrategy extends Strategy { execution.PhysicalRDD(l.output, t.buildScan()) :: Nil case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation), part, query, overwrite) if part.isEmpty => + l @ LogicalRelation(t: InsertableRelation), part, query, overwrite, false) if part.isEmpty => execution.ExecutedCommand(InsertIntoDataSource(l, query, overwrite)) :: Nil case _ => Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala index 5a78001117d1b..6ed68d179edc9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/rules.scala @@ -37,7 +37,7 @@ private[sql] object PreInsertCastAndRename extends Rule[LogicalPlan] { // We are inserting into an InsertableRelation. case i @ InsertIntoTable( - l @ LogicalRelation(r: InsertableRelation), partition, child, overwrite) => { + l @ LogicalRelation(r: InsertableRelation), partition, child, overwrite, ifNotExists) => { // First, make sure the data to be inserted have the same number of fields with the // schema of the relation. if (l.output.size != child.output.size) { @@ -84,7 +84,7 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => def apply(plan: LogicalPlan): Unit = { plan.foreach { case i @ logical.InsertIntoTable( - l @ LogicalRelation(t: InsertableRelation), partition, query, overwrite) => + l @ LogicalRelation(t: InsertableRelation), partition, query, overwrite, ifNotExists) => // Right now, we do not support insert into a data source table with partition specs. if (partition.nonEmpty) { failAnalysis(s"Insert into a partition is not allowed because $l is not partitioned.") @@ -102,7 +102,8 @@ private[sql] case class PreWriteCheck(catalog: Catalog) extends (LogicalPlan => } case i @ logical.InsertIntoTable( - l: LogicalRelation, partition, query, overwrite) if !l.isInstanceOf[InsertableRelation] => + l: LogicalRelation, partition, query, overwrite, ifNotExists) + if !l.isInstanceOf[InsertableRelation] => // The relation in l is not an InsertableRelation. failAnalysis(s"$l does not allow insertion.") diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 2ae9d018e1b1b..81ee48ef4152f 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -532,6 +532,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { "inputddl7", "inputddl8", "insert1", + "insert1_overwrite_partitions", "insert2_overwrite_partitions", "insert_compressed", "join0", 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 3ed5c5b031736..f1c0bd92aa23d 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 @@ -527,7 +527,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Collects all `MetastoreRelation`s which should be replaced val toBeReplaced = plan.collect { // Write path - case InsertIntoTable(relation: MetastoreRelation, _, _, _) + case InsertIntoTable(relation: MetastoreRelation, _, _, _, _) // Inserting into partitioned table is not supported in Parquet data source (yet). if !relation.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && @@ -538,7 +538,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with (relation, parquetRelation, attributedRewrites) // Write path - case InsertIntoHiveTable(relation: MetastoreRelation, _, _, _) + case InsertIntoHiveTable(relation: MetastoreRelation, _, _, _, _) // Inserting into partitioned table is not supported in Parquet data source (yet). if !relation.hiveQlTable.isPartitioned && hive.convertMetastoreParquet && @@ -569,15 +569,15 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with val alias = r.alias.getOrElse(r.tableName) Subquery(alias, parquetRelation) - case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite) + case InsertIntoTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) if relationMap.contains(r) => val parquetRelation = relationMap(r) - InsertIntoTable(parquetRelation, partition, child, overwrite) + InsertIntoTable(parquetRelation, partition, child, overwrite, ifNotExists) - case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite) + case InsertIntoHiveTable(r: MetastoreRelation, partition, child, overwrite, ifNotExists) if relationMap.contains(r) => val parquetRelation = relationMap(r) - InsertIntoTable(parquetRelation, partition, child, overwrite) + InsertIntoTable(parquetRelation, partition, child, overwrite, ifNotExists) case other => other.transformExpressions { case a: Attribute if a.resolved => attributedRewrites.getOrElse(a, a) @@ -698,7 +698,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with // Wait until children are resolved. case p: LogicalPlan if !p.childrenResolved => p - case p @ InsertIntoTable(table: MetastoreRelation, _, child, _) => + case p @ InsertIntoTable(table: MetastoreRelation, _, child, _, _) => castChildOutput(p, table, child) } @@ -715,7 +715,7 @@ private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with .forall { case (left, right) => left.sameType(right) }) { // If both types ignoring nullability of ArrayType, MapType, StructType are the same, // use InsertIntoHiveTable instead of InsertIntoTable. - InsertIntoHiveTable(p.table, p.partition, p.child, p.overwrite) + InsertIntoHiveTable(p.table, p.partition, p.child, p.overwrite, p.ifNotExists) } else { // Only do the casting when child output data types differ from table output data types. val castedChildOutput = child.output.zip(table.output).map { @@ -753,7 +753,8 @@ private[hive] case class InsertIntoHiveTable( table: LogicalPlan, partition: Map[String, Option[String]], child: LogicalPlan, - overwrite: Boolean) + overwrite: Boolean, + ifNotExists: Boolean) extends LogicalPlan { override def children: Seq[LogicalPlan] = child :: Nil 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 b2ae74efeb097..53a204b8c2932 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 @@ -1002,7 +1002,27 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C cleanIdentifier(key.toLowerCase) -> None }.toMap).getOrElse(Map.empty) - InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite) + InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, false) + + case Token(destinationToken(), + Token("TOK_TAB", + tableArgs) :: + Token("TOK_IFNOTEXISTS", + ifNotExists) :: Nil) => + val Some(tableNameParts) :: partitionClause :: Nil = + getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) + + val tableIdent = extractTableIdent(tableNameParts) + + val partitionKeys = partitionClause.map(_.getChildren.map { + // Parse partitions. We also make keys case insensitive. + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> Some(PlanUtils.stripQuotes(value)) + case Token("TOK_PARTVAL", Token(key, Nil) :: Nil) => + cleanIdentifier(key.toLowerCase) -> None + }.toMap).getOrElse(Map.empty) + + InsertIntoTable(UnresolvedRelation(tableIdent, None), partitionKeys, query, overwrite, true) case a: ASTNode => throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") 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 5f7e897295117..1ccb0c279c60e 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 @@ -184,12 +184,14 @@ private[hive] trait HiveStrategies { object DataSinks extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.InsertIntoTable(table: MetastoreRelation, partition, child, overwrite) => + case logical.InsertIntoTable( + table: MetastoreRelation, partition, child, overwrite, ifNotExists) => execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite) :: Nil - case hive.InsertIntoHiveTable(table: MetastoreRelation, partition, child, overwrite) => + table, partition, planLater(child), overwrite, ifNotExists) :: Nil + case hive.InsertIntoHiveTable( + table: MetastoreRelation, partition, child, overwrite, ifNotExists) => execution.InsertIntoHiveTable( - table, partition, planLater(child), overwrite) :: Nil + table, partition, planLater(child), overwrite, ifNotExists) :: 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 fade9e5852eaa..76a1965f3cb25 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 @@ -67,7 +67,7 @@ case class CreateTableAsSelect( new org.apache.hadoop.hive.metastore.api.AlreadyExistsException(s"$database.$tableName") } } else { - hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true)).toRdd + hiveContext.executePlan(InsertIntoTable(metastoreRelation, Map(), query, true, false)).toRdd } Seq.empty[Row] diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index 6c96747439683..89995a91b1a92 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -45,7 +45,8 @@ case class InsertIntoHiveTable( table: MetastoreRelation, partition: Map[String, Option[String]], child: SparkPlan, - overwrite: Boolean) extends UnaryNode with HiveInspectors { + overwrite: Boolean, + ifNotExists: Boolean) extends UnaryNode with HiveInspectors { @transient val sc: HiveContext = sqlContext.asInstanceOf[HiveContext] @transient lazy val outputClass = newSerializer(table.tableDesc).getSerializedClass @@ -219,15 +220,25 @@ case class InsertIntoHiveTable( isSkewedStoreAsSubdir) } } else { - catalog.synchronized { - catalog.client.loadPartition( - outputPath, - qualifiedTableName, - orderedPartitionSpec, - overwrite, - holdDDLTime, - inheritTableSpecs, - isSkewedStoreAsSubdir) + // scalastyle:off + // ifNotExists is only valid with static partition, refer to + // https://cwiki.apache.org/confluence/display/Hive/LanguageManual+DML#LanguageManualDML-InsertingdataintoHiveTablesfromqueries + // scalastyle:on + val oldPart = catalog.synchronized { + catalog.client.getPartition( + catalog.client.getTable(qualifiedTableName), partitionSpec, false) + } + if (oldPart == null || !ifNotExists) { + catalog.synchronized { + catalog.client.loadPartition( + outputPath, + qualifiedTableName, + orderedPartitionSpec, + overwrite, + holdDDLTime, + inheritTableSpecs, + isSkewedStoreAsSubdir) + } } } } else { diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-0-d5edc0daa94b33915df794df3b710774 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-0-d5edc0daa94b33915df794df3b710774 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-1-9eb9372f4855928fae16f5fa554b3a62 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-1-9eb9372f4855928fae16f5fa554b3a62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-10-ec2cef3d37146c450c60202a572f5cab b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-10-ec2cef3d37146c450c60202a572f5cab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-11-8854d6001200fc11529b2e2da755e5a2 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-11-8854d6001200fc11529b2e2da755e5a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-12-71ff68fda0aa7a36cb50d8fab0d70d25 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-12-71ff68fda0aa7a36cb50d8fab0d70d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-13-7e4e7d7003fc6ef17bc19c3461ad899 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-13-7e4e7d7003fc6ef17bc19c3461ad899 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-14-ec2cef3d37146c450c60202a572f5cab b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-14-ec2cef3d37146c450c60202a572f5cab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-15-a3b2e230efde74e970ae8a3b55f383fc b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-15-a3b2e230efde74e970ae8a3b55f383fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-2-8396c17a66e3d9a374d4361873b9bfe3 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-2-8396c17a66e3d9a374d4361873b9bfe3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-3-3876bb356dd8af7e78d061093d555457 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-3-3876bb356dd8af7e78d061093d555457 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-4-528e23afb272c2e69004c86ddaa70ee b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-4-528e23afb272c2e69004c86ddaa70ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-5-de5d56456c28d63775554e56355911d2 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-5-de5d56456c28d63775554e56355911d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 new file mode 100644 index 0000000000000..185a91c110d6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-6-3efdc331b3b4bdac3e60c757600fff53 @@ -0,0 +1,5 @@ +98 val_98 +98 val_98 +97 val_97 +97 val_97 +96 val_96 diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-7-92f6af82704504968de078c133f222f8 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-7-92f6af82704504968de078c133f222f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-8-316cad7c63ddd4fb043be2affa5b0a67 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-8-316cad7c63ddd4fb043be2affa5b0a67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 new file mode 100644 index 0000000000000..185a91c110d6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert1_overwrite_partitions-9-3efdc331b3b4bdac3e60c757600fff53 @@ -0,0 +1,5 @@ +98 val_98 +98 val_98 +97 val_97 +97 val_97 +96 val_96 From 3a205bbd9e352668a020c3146391e1e4441467af Mon Sep 17 00:00:00 2001 From: Yash Datta Date: Mon, 13 Apr 2015 14:43:07 -0700 Subject: [PATCH 071/128] [SQL][SPARK-6742]: Don't push down predicates which reference partition column(s) cc liancheng Author: Yash Datta Closes #5390 from saucam/fpush and squashes the following commits: 3f026d6 [Yash Datta] SPARK-6742: Fix scalastyle ce3d702 [Yash Datta] SPARK-6742: Add test case, fix scalastyle 8592acc [Yash Datta] SPARK-6742: Don't push down predicates which reference partition column(s) --- .../spark/sql/execution/SparkStrategies.scala | 11 ++++++++- .../sql/parquet/ParquetFilterSuite.scala | 24 ++++++++++++++++++- 2 files changed, 33 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 5268b7334051a..f0d92ffffcda3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -215,6 +215,11 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { table: ParquetRelation, partition, child, overwrite, ifNotExists) => InsertIntoParquetTable(table, planLater(child), overwrite) :: Nil case PhysicalOperation(projectList, filters: Seq[Expression], relation: ParquetRelation) => + val partitionColNames = relation.partitioningAttributes.map(_.name).toSet + val filtersToPush = filters.filter { pred => + val referencedColNames = pred.references.map(_.name).toSet + referencedColNames.intersect(partitionColNames).isEmpty + } val prunePushedDownFilters = if (sqlContext.conf.parquetFilterPushDown) { (predicates: Seq[Expression]) => { @@ -226,6 +231,10 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // "A AND B" in the higher-level filter, not just "B". predicates.map(p => p -> ParquetFilters.createFilter(p)).collect { case (predicate, None) => predicate + // Filter needs to be applied above when it contains partitioning + // columns + case (predicate, _) if(!predicate.references.map(_.name).toSet + .intersect (partitionColNames).isEmpty) => predicate } } } else { @@ -238,7 +247,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { ParquetTableScan( _, relation, - if (sqlContext.conf.parquetFilterPushDown) filters else Nil)) :: Nil + if (sqlContext.conf.parquetFilterPushDown) filtersToPush else Nil)) :: Nil case _ => Nil } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala index 6a2c2a7c4080a..10d0ede4dc0dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetFilterSuite.scala @@ -22,7 +22,7 @@ import parquet.filter2.predicate.Operators._ import parquet.filter2.predicate.{FilterPredicate, Operators} import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal, Predicate, Row} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.sources.LogicalRelation import org.apache.spark.sql.test.TestSQLContext @@ -350,4 +350,26 @@ class ParquetDataSourceOffFilterSuite extends ParquetFilterSuiteBase with Before override protected def afterAll(): Unit = { sqlContext.setConf(SQLConf.PARQUET_USE_DATA_SOURCE_API, originalConf.toString) } + + test("SPARK-6742: don't push down predicates which reference partition columns") { + import sqlContext.implicits._ + + withSQLConf(SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED -> "true") { + withTempPath { dir => + val path = s"${dir.getCanonicalPath}/part=1" + (1 to 3).map(i => (i, i.toString)).toDF("a", "b").saveAsParquetFile(path) + + // If the "part = 1" filter gets pushed down, this query will throw an exception since + // "part" is not a valid column in the actual Parquet file + val df = DataFrame(sqlContext, org.apache.spark.sql.parquet.ParquetRelation( + path, + Some(sqlContext.sparkContext.hadoopConfiguration), sqlContext, + Seq(AttributeReference("part", IntegerType, false)()) )) + + checkAnswer( + df.filter("a = 1 or part = 1"), + (1 to 3).map(i => Row(1, i, i.toString))) + } + } + } } From 2a55cb41bf7da1786be2c76b8af398da8fedb44b Mon Sep 17 00:00:00 2001 From: MechCoder Date: Mon, 13 Apr 2015 15:36:33 -0700 Subject: [PATCH 072/128] [SPARK-5972] [MLlib] Cache residuals and gradient in GBT during training and validation The previous PR https://github.com/apache/spark/pull/4906 helped to extract the learning curve giving the error for each iteration. This continues the work refactoring some code and extending the same logic during training and validation. Author: MechCoder Closes #5330 from MechCoder/spark-5972 and squashes the following commits: 0b5d659 [MechCoder] minor 32d409d [MechCoder] EvaluateeachIteration and training cache should follow different paths d542bb0 [MechCoder] Remove unused imports and docs 58f4932 [MechCoder] Remove unpersist 70d3b4c [MechCoder] Broadcast for each tree 5869533 [MechCoder] Access broadcasted values locally and other minor changes 923dbf6 [MechCoder] [SPARK-5972] Cache residuals and gradient in GBT during training and validation --- .../mllib/tree/GradientBoostedTrees.scala | 42 ++++++---- .../spark/mllib/tree/loss/AbsoluteError.scala | 10 +-- .../spark/mllib/tree/loss/LogLoss.scala | 11 +-- .../apache/spark/mllib/tree/loss/Loss.scala | 8 +- .../spark/mllib/tree/loss/SquaredError.scala | 10 +-- .../mllib/tree/model/treeEnsembleModels.scala | 77 +++++++++++++++---- 6 files changed, 105 insertions(+), 53 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 a9c93e181e3ce..c02c79f094b66 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 @@ -157,7 +157,6 @@ object GradientBoostedTrees extends Logging { validationInput: RDD[LabeledPoint], boostingStrategy: BoostingStrategy, validate: Boolean): GradientBoostedTreesModel = { - val timer = new TimeTracker() timer.start("total") timer.start("init") @@ -192,20 +191,29 @@ object GradientBoostedTrees extends Logging { // Initialize tree timer.start("building tree 0") val firstTreeModel = new DecisionTree(treeStrategy).run(data) + val firstTreeWeight = 1.0 baseLearners(0) = firstTreeModel - baseLearnerWeights(0) = 1.0 - val startingModel = new GradientBoostedTreesModel(Regression, Array(firstTreeModel), Array(1.0)) - logDebug("error of gbt = " + loss.computeError(startingModel, input)) + baseLearnerWeights(0) = firstTreeWeight + val startingModel = new GradientBoostedTreesModel( + Regression, Array(firstTreeModel), baseLearnerWeights.slice(0, 1)) + + var predError: RDD[(Double, Double)] = GradientBoostedTreesModel. + computeInitialPredictionAndError(input, firstTreeWeight, firstTreeModel, loss) + logDebug("error of gbt = " + predError.values.mean()) // Note: A model of type regression is used since we require raw prediction timer.stop("building tree 0") - var bestValidateError = if (validate) loss.computeError(startingModel, validationInput) else 0.0 + var validatePredError: RDD[(Double, Double)] = GradientBoostedTreesModel. + computeInitialPredictionAndError(validationInput, firstTreeWeight, firstTreeModel, loss) + var bestValidateError = if (validate) validatePredError.values.mean() else 0.0 var bestM = 1 - // psuedo-residual for second iteration - data = input.map(point => LabeledPoint(loss.gradient(startingModel, point), - point.features)) + // pseudo-residual for second iteration + data = predError.zip(input).map { case ((pred, _), point) => + LabeledPoint(-loss.gradient(pred, point.label), point.features) + } + var m = 1 while (m < numIterations) { timer.start(s"building tree $m") @@ -222,15 +230,22 @@ object GradientBoostedTrees extends Logging { baseLearnerWeights(m) = learningRate // Note: A model of type regression is used since we require raw prediction val partialModel = new GradientBoostedTreesModel( - Regression, baseLearners.slice(0, m + 1), baseLearnerWeights.slice(0, m + 1)) - logDebug("error of gbt = " + loss.computeError(partialModel, input)) + Regression, baseLearners.slice(0, m + 1), + baseLearnerWeights.slice(0, m + 1)) + + predError = GradientBoostedTreesModel.updatePredictionError( + input, predError, baseLearnerWeights(m), baseLearners(m), loss) + logDebug("error of gbt = " + predError.values.mean()) if (validate) { // Stop training early if // 1. Reduction in error is less than the validationTol or // 2. If the error increases, that is if the model is overfit. // We want the model returned corresponding to the best validation error. - val currentValidateError = loss.computeError(partialModel, validationInput) + + validatePredError = GradientBoostedTreesModel.updatePredictionError( + validationInput, validatePredError, baseLearnerWeights(m), baseLearners(m), loss) + val currentValidateError = validatePredError.values.mean() if (bestValidateError - currentValidateError < validationTol) { return new GradientBoostedTreesModel( boostingStrategy.treeStrategy.algo, @@ -242,8 +257,9 @@ object GradientBoostedTrees extends Logging { } } // Update data with pseudo-residuals - data = input.map(point => LabeledPoint(-loss.gradient(partialModel, point), - point.features)) + data = predError.zip(input).map { case ((pred, _), point) => + LabeledPoint(-loss.gradient(pred, point.label), point.features) + } m += 1 } 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 793dd664c5d5a..6f570b4e09c79 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 @@ -37,14 +37,12 @@ object AbsoluteError extends Loss { * Method to calculate the gradients for the gradient boosting calculation for least * absolute error calculation. * The gradient with respect to F(x) is: sign(F(x) - y) - * @param model Ensemble model - * @param point Instance of the training dataset + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - if ((point.label - model.predict(point.features)) < 0) 1.0 else -1.0 + override def gradient(prediction: Double, label: Double): Double = { + if (label - prediction < 0) 1.0 else -1.0 } override def computeError(prediction: Double, label: Double): Double = { 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 51b1aed167b66..24ee9f3d51293 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 @@ -39,15 +39,12 @@ object LogLoss extends Loss { * Method to calculate the loss gradients for the gradient boosting calculation for binary * classification * 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 + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - val prediction = model.predict(point.features) - - 4.0 * point.label / (1.0 + math.exp(2.0 * point.label * prediction)) + override def gradient(prediction: Double, label: Double): Double = { + - 4.0 * label / (1.0 + math.exp(2.0 * label * prediction)) } override def computeError(prediction: Double, label: Double): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala index 357869ff6b333..d3b82b752fa0d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/loss/Loss.scala @@ -31,13 +31,11 @@ trait Loss extends Serializable { /** * Method to calculate the gradients for the gradient boosting calculation. - * @param model Model of the weak learner. - * @param point Instance of the training dataset. + * @param prediction Predicted feature + * @param label true label. * @return Loss gradient. */ - def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double + def gradient(prediction: Double, label: Double): Double /** * Method to calculate error of the base learner for the gradient boosting calculation. 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 b990707ca4525..58857ae15e93e 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 @@ -37,14 +37,12 @@ object SquaredError extends Loss { * Method to calculate the gradients for the gradient boosting calculation for least * squares error calculation. * The gradient with respect to F(x) is: - 2 (y - F(x)) - * @param model Ensemble model - * @param point Instance of the training dataset + * @param prediction Predicted label. + * @param label True label. * @return Loss gradient */ - override def gradient( - model: TreeEnsembleModel, - point: LabeledPoint): Double = { - 2.0 * (model.predict(point.features) - point.label) + override def gradient(prediction: Double, label: Double): Double = { + 2.0 * (prediction - label) } override def computeError(prediction: Double, label: Double): Double = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala index 1950254b2aa6d..fef3d2acb202a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/treeEnsembleModels.scala @@ -130,35 +130,28 @@ class GradientBoostedTreesModel( val numIterations = trees.length val evaluationArray = Array.fill(numIterations)(0.0) + val localTreeWeights = treeWeights + + var predictionAndError = GradientBoostedTreesModel.computeInitialPredictionAndError( + remappedData, localTreeWeights(0), trees(0), loss) - var predictionAndError: RDD[(Double, Double)] = remappedData.map { i => - val pred = treeWeights(0) * trees(0).predict(i.features) - val error = loss.computeError(pred, i.label) - (pred, error) - } evaluationArray(0) = predictionAndError.values.mean() - // Avoid the model being copied across numIterations. val broadcastTrees = sc.broadcast(trees) - val broadcastWeights = sc.broadcast(treeWeights) - (1 until numIterations).map { nTree => predictionAndError = remappedData.zip(predictionAndError).mapPartitions { iter => val currentTree = broadcastTrees.value(nTree) - val currentTreeWeight = broadcastWeights.value(nTree) - iter.map { - case (point, (pred, error)) => { - val newPred = pred + currentTree.predict(point.features) * currentTreeWeight - val newError = loss.computeError(newPred, point.label) - (newPred, newError) - } + val currentTreeWeight = localTreeWeights(nTree) + iter.map { case (point, (pred, error)) => + val newPred = pred + currentTree.predict(point.features) * currentTreeWeight + val newError = loss.computeError(newPred, point.label) + (newPred, newError) } } evaluationArray(nTree) = predictionAndError.values.mean() } broadcastTrees.unpersist() - broadcastWeights.unpersist() evaluationArray } @@ -166,6 +159,58 @@ class GradientBoostedTreesModel( object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { + /** + * Compute the initial predictions and errors for a dataset for the first + * iteration of gradient boosting. + * @param data: training data. + * @param initTreeWeight: learning rate assigned to the first tree. + * @param initTree: first DecisionTreeModel. + * @param loss: evaluation metric. + * @return a RDD with each element being a zip of the prediction and error + * corresponding to every sample. + */ + def computeInitialPredictionAndError( + data: RDD[LabeledPoint], + initTreeWeight: Double, + initTree: DecisionTreeModel, + loss: Loss): RDD[(Double, Double)] = { + data.map { lp => + val pred = initTreeWeight * initTree.predict(lp.features) + val error = loss.computeError(pred, lp.label) + (pred, error) + } + } + + /** + * Update a zipped predictionError RDD + * (as obtained with computeInitialPredictionAndError) + * @param data: training data. + * @param predictionAndError: predictionError RDD + * @param treeWeight: Learning rate. + * @param tree: Tree using which the prediction and error should be updated. + * @param loss: evaluation metric. + * @return a RDD with each element being a zip of the prediction and error + * corresponding to each sample. + */ + def updatePredictionError( + data: RDD[LabeledPoint], + predictionAndError: RDD[(Double, Double)], + treeWeight: Double, + tree: DecisionTreeModel, + loss: Loss): RDD[(Double, Double)] = { + + val newPredError = data.zip(predictionAndError).mapPartitions { iter => + iter.map { + case (lp, (pred, error)) => { + val newPred = pred + tree.predict(lp.features) * treeWeight + val newError = loss.computeError(newPred, lp.label) + (newPred, newError) + } + } + } + newPredError + } + override def load(sc: SparkContext, path: String): GradientBoostedTreesModel = { val (loadedClassName, version, jsonMetadata) = Loader.loadMetadata(sc, path) val classNameV1_0 = SaveLoadV1_0.thisClassName From e63a86abe2794332cdad71d87b72a7c56327a43d Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Apr 2015 16:00:58 -0700 Subject: [PATCH 073/128] [SPARK-6872] [SQL] add copy in external sort We need add copy before call externalsort. Author: Daoyuan Wang Closes #5481 from adrian-wang/extsort and squashes the following commits: 9611586 [Daoyuan Wang] fix bug in external sort --- .../scala/org/apache/spark/sql/execution/basicOperators.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 6eec520abff53..f8221f41bc6c3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -194,7 +194,7 @@ case class ExternalSort( child.execute().mapPartitions( { iterator => val ordering = newOrdering(sortOrder, child.output) val sorter = new ExternalSorter[Row, Null, Row](ordering = Some(ordering)) - sorter.insertAll(iterator.map(r => (r, null))) + sorter.insertAll(iterator.map(r => (r.copy, null))) val baseIterator = sorter.iterator.map(_._1) // TODO(marmbrus): The complex type signature below thwarts inference for no reason. CompletionIterator[Row, Iterator[Row]](baseIterator, sorter.stop()) From c5602bdc310cc8f82dc304500bebe40217cba785 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Mon, 13 Apr 2015 16:02:18 -0700 Subject: [PATCH 074/128] [SPARK-5941] [SQL] Unit Test loads the table `src` twice for leftsemijoin.q In `leftsemijoin.q`, there is a data loading command for table `sales` already, but in `TestHive`, it also created the table `sales`, which causes duplicated records inserted into the `sales`. Author: Cheng Hao Closes #4506 from chenghao-intel/df_table and squashes the following commits: 0be05f7 [Cheng Hao] Remove the table `sales` creating from TestHive --- .../sql/columnar/InMemoryColumnarQuerySuite.scala | 2 +- .../org/apache/spark/sql/hive/test/TestHive.scala | 6 ------ ...ftsemijoin-10-89737a8857b5b61cc909e0c797f86aea | 2 -- .../leftsemijoin-8-73cad58a10a1483ccb15e94a857013 | 2 -- .../spark/sql/hive/execution/HiveSerDeSuite.scala | 15 +++++++++++---- 5 files changed, 12 insertions(+), 15 deletions(-) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 27dfabca90217..479210d1c9c43 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -42,7 +42,7 @@ class InMemoryColumnarQuerySuite extends QueryTest { .toDF().registerTempTable("sizeTst") cacheTable("sizeTst") assert( - table("sizeTst").queryExecution.logical.statistics.sizeInBytes > + table("sizeTst").queryExecution.analyzed.statistics.sizeInBytes > conf.autoBroadcastJoinThreshold) } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala index a3497eadd67f6..6570fa1043900 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -262,12 +262,6 @@ class TestHiveContext(sc: SparkContext) extends HiveContext(sc) { |WITH SERDEPROPERTIES ('field.delim'='\\t') """.stripMargin.cmd, "INSERT OVERWRITE TABLE serdeins SELECT * FROM src".cmd), - TestTable("sales", - s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) - |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' - |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") - """.stripMargin.cmd, - s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt")}' INTO TABLE sales".cmd), TestTable("episodes", s"""CREATE TABLE episodes (title STRING, air_date STRING, doctor INT) |ROW FORMAT SERDE '${classOf[AvroSerDe].getCanonicalName}' diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea b/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea index 25ce912507d55..a1963ba81e0da 100644 --- a/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea +++ b/sql/hive/src/test/resources/golden/leftsemijoin-10-89737a8857b5b61cc909e0c797f86aea @@ -1,4 +1,2 @@ Hank 2 -Hank 2 -Joe 2 Joe 2 diff --git a/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 b/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 index 25ce912507d55..a1963ba81e0da 100644 --- a/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 +++ b/sql/hive/src/test/resources/golden/leftsemijoin-8-73cad58a10a1483ccb15e94a857013 @@ -1,4 +1,2 @@ Hank 2 -Hank 2 -Joe 2 Joe 2 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index d05e11fcf281b..5586a793618bd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -25,18 +25,25 @@ import org.apache.spark.sql.hive.test.TestHive * A set of tests that validates support for Hive SerDe. */ class HiveSerDeSuite extends HiveComparisonTest with BeforeAndAfterAll { - override def beforeAll(): Unit = { + import TestHive._ + import org.apache.hadoop.hive.serde2.RegexSerDe + super.beforeAll() TestHive.cacheTables = false - super.beforeAll() + sql(s"""CREATE TABLE IF NOT EXISTS sales (key STRING, value INT) + |ROW FORMAT SERDE '${classOf[RegexSerDe].getCanonicalName}' + |WITH SERDEPROPERTIES ("input.regex" = "([^ ]*)\t([^ ]*)") + """.stripMargin) + sql(s"LOAD DATA LOCAL INPATH '${getHiveFile("data/files/sales.txt")}' INTO TABLE sales") } + // table sales is not a cache table, and will be clear after reset + createQueryTest("Read with RegexSerDe", "SELECT * FROM sales", false) + createQueryTest( "Read and write with LazySimpleSerDe (tab separated)", "SELECT * from serdeins") - createQueryTest("Read with RegexSerDe", "SELECT * FROM sales") - createQueryTest("Read with AvroSerDe", "SELECT * FROM episodes") createQueryTest("Read Partitioned with AvroSerDe", "SELECT * FROM episodes_part") From c4ab255e94366ba9b9023d5431f9d2412e0d6dc7 Mon Sep 17 00:00:00 2001 From: Ilya Ganelin Date: Mon, 13 Apr 2015 16:28:07 -0700 Subject: [PATCH 075/128] [SPARK-5931][CORE] Use consistent naming for time properties I've added new utility methods to do the conversion from times specified as e.g. 120s, 240ms, 360us to convert to a consistent internal representation. I've updated usage of these constants throughout the code to be consistent. I believe I've captured all usages of time-based properties throughout the code. I've also updated variable names in a number of places to reflect their units for clarity and updated documentation where appropriate. Author: Ilya Ganelin Author: Ilya Ganelin Closes #5236 from ilganeli/SPARK-5931 and squashes the following commits: 4526c81 [Ilya Ganelin] Update configuration.md de3bff9 [Ilya Ganelin] Fixing style errors f5fafcd [Ilya Ganelin] Doc updates 951ca2d [Ilya Ganelin] Made the most recent round of changes bc04e05 [Ilya Ganelin] Minor fixes and doc updates 25d3f52 [Ilya Ganelin] Minor nit fixes 642a06d [Ilya Ganelin] Fixed logic for invalid suffixes and addid matching test 8927e66 [Ilya Ganelin] Fixed handling of -1 69fedcc [Ilya Ganelin] Added test for zero dc7bd08 [Ilya Ganelin] Fixed error in exception handling 7d19cdd [Ilya Ganelin] Added fix for possible NPE 6f651a8 [Ilya Ganelin] Now using regexes to simplify code in parseTimeString. Introduces getTimeAsSec and getTimeAsMs methods in SparkConf. Updated documentation cbd2ca6 [Ilya Ganelin] Formatting error 1a1122c [Ilya Ganelin] Formatting fixes and added m for use as minute formatter 4e48679 [Ilya Ganelin] Fixed priority order and mixed up conversions in a couple spots d4efd26 [Ilya Ganelin] Added time conversion for yarn.scheduler.heartbeat.interval-ms cbf41db [Ilya Ganelin] Got rid of thrown exceptions 1465390 [Ilya Ganelin] Nit 28187bf [Ilya Ganelin] Convert straight to seconds ff40bfe [Ilya Ganelin] Updated tests to fix small bugs 19c31af [Ilya Ganelin] Added cleaner computation of time conversions in tests 6387772 [Ilya Ganelin] Updated suffix handling to handle overlap of units more gracefully 5193d5f [Ilya Ganelin] Resolved merge conflicts 76cfa27 [Ilya Ganelin] [SPARK-5931] Minor nit fixes' bf779b0 [Ilya Ganelin] Special handling of overlapping usffixes for java dd0a680 [Ilya Ganelin] Updated scala code to call into java b2fc965 [Ilya Ganelin] replaced get or default since it's not present in this version of java 39164f9 [Ilya Ganelin] [SPARK-5931] Updated Java conversion to be similar to scala conversion. Updated conversions to clean up code a little using TimeUnit.convert. Added Unit tests 3b126e1 [Ilya Ganelin] Fixed conversion to US from seconds 1858197 [Ilya Ganelin] Fixed bug where all time was being converted to us instead of the appropriate units bac9edf [Ilya Ganelin] More whitespace 8613631 [Ilya Ganelin] Whitespace 1c0c07c [Ilya Ganelin] Updated Java code to add day, minutes, and hours 647b5ac [Ilya Ganelin] Udpated time conversion to use map iterator instead of if fall through 70ac213 [Ilya Ganelin] Fixed remaining usages to be consistent. Updated Java-side time conversion 68f4e93 [Ilya Ganelin] Updated more files to clean up usage of default time strings 3a12dd8 [Ilya Ganelin] Updated host revceiver 5232a36 [Ilya Ganelin] [SPARK-5931] Changed default behavior of time string conversion. 499bdf0 [Ilya Ganelin] Merge branch 'SPARK-5931' of github.com:ilganeli/spark into SPARK-5931 9e2547c [Ilya Ganelin] Reverting doc changes 8f741e1 [Ilya Ganelin] Update JavaUtils.java 34f87c2 [Ilya Ganelin] Update Utils.scala 9a29d8d [Ilya Ganelin] Fixed misuse of time in streaming context test 42477aa [Ilya Ganelin] Updated configuration doc with note on specifying time properties cde9bff [Ilya Ganelin] Updated spark.streaming.blockInterval c6a0095 [Ilya Ganelin] Updated spark.core.connection.auth.wait.timeout 5181597 [Ilya Ganelin] Updated spark.dynamicAllocation.schedulerBacklogTimeout 2fcc91c [Ilya Ganelin] Updated spark.dynamicAllocation.executorIdleTimeout 6d1518e [Ilya Ganelin] Upated spark.speculation.interval 3f1cfc8 [Ilya Ganelin] Updated spark.scheduler.revive.interval 3352d34 [Ilya Ganelin] Updated spark.scheduler.maxRegisteredResourcesWaitingTime 272c215 [Ilya Ganelin] Updated spark.locality.wait 7320c87 [Ilya Ganelin] updated spark.akka.heartbeat.interval 064ebd6 [Ilya Ganelin] Updated usage of spark.cleaner.ttl 21ef3dd [Ilya Ganelin] updated spark.shuffle.sasl.timeout c9f5cad [Ilya Ganelin] Updated spark.shuffle.io.retryWait 4933fda [Ilya Ganelin] Updated usage of spark.storage.blockManagerSlaveTimeout 7db6d2a [Ilya Ganelin] Updated usage of spark.akka.timeout 404f8c3 [Ilya Ganelin] Updated usage of spark.core.connection.ack.wait.timeout 59bf9e1 [Ilya Ganelin] [SPARK-5931] Updated Utils and JavaUtils classes to add helper methods to handle time strings. Updated time strings in a few places to properly parse time --- .../spark/ExecutorAllocationManager.scala | 34 ++++---- .../org/apache/spark/HeartbeatReceiver.scala | 15 ++-- .../scala/org/apache/spark/SparkConf.scala | 36 ++++++++ .../org/apache/spark/executor/Executor.scala | 6 +- .../spark/network/nio/ConnectionManager.scala | 3 +- .../spark/scheduler/TaskSchedulerImpl.scala | 10 +-- .../spark/scheduler/TaskSetManager.scala | 21 +++-- .../CoarseGrainedSchedulerBackend.scala | 13 +-- .../org/apache/spark/util/AkkaUtils.scala | 14 +-- .../apache/spark/util/MetadataCleaner.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 26 +++++- .../ExecutorAllocationManagerSuite.scala | 7 +- .../network/nio/ConnectionManagerSuite.scala | 8 +- .../spark/scheduler/TaskSetManagerSuite.scala | 25 +++--- .../BlockManagerReplicationSuite.scala | 2 +- .../org/apache/spark/util/UtilsSuite.scala | 44 ++++++++++ docs/configuration.md | 86 ++++++++++--------- docs/running-on-yarn.md | 4 +- .../apache/spark/network/util/JavaUtils.java | 66 ++++++++++++++ .../spark/network/util/TransportConf.java | 15 +++- .../streaming/receiver/BlockGenerator.scala | 8 +- .../streaming/scheduler/JobGenerator.scala | 12 ++- .../spark/streaming/ReceiverSuite.scala | 14 +-- .../streaming/StreamingContextSuite.scala | 22 ++--- .../spark/deploy/yarn/ApplicationMaster.scala | 9 +- 25 files changed, 345 insertions(+), 157 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index 9385f557c4614..4e7bf51fc0622 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -80,16 +80,16 @@ private[spark] class ExecutorAllocationManager( Integer.MAX_VALUE) // How long there must be backlogged tasks for before an addition is triggered (seconds) - private val schedulerBacklogTimeout = conf.getLong( - "spark.dynamicAllocation.schedulerBacklogTimeout", 5) + private val schedulerBacklogTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.schedulerBacklogTimeout", "5s") - // Same as above, but used only after `schedulerBacklogTimeout` is exceeded - private val sustainedSchedulerBacklogTimeout = conf.getLong( - "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", schedulerBacklogTimeout) + // Same as above, but used only after `schedulerBacklogTimeoutS` is exceeded + private val sustainedSchedulerBacklogTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", s"${schedulerBacklogTimeoutS}s") // How long an executor must be idle for before it is removed (seconds) - private val executorIdleTimeout = conf.getLong( - "spark.dynamicAllocation.executorIdleTimeout", 600) + private val executorIdleTimeoutS = conf.getTimeAsSeconds( + "spark.dynamicAllocation.executorIdleTimeout", "600s") // During testing, the methods to actually kill and add executors are mocked out private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) @@ -150,14 +150,14 @@ private[spark] class ExecutorAllocationManager( throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " + s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!") } - if (schedulerBacklogTimeout <= 0) { + if (schedulerBacklogTimeoutS <= 0) { throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeout must be > 0!") } - if (sustainedSchedulerBacklogTimeout <= 0) { + if (sustainedSchedulerBacklogTimeoutS <= 0) { throw new SparkException( "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout must be > 0!") } - if (executorIdleTimeout <= 0) { + if (executorIdleTimeoutS <= 0) { throw new SparkException("spark.dynamicAllocation.executorIdleTimeout must be > 0!") } // Require external shuffle service for dynamic allocation @@ -262,8 +262,8 @@ private[spark] class ExecutorAllocationManager( } else if (addTime != NOT_SET && now >= addTime) { val delta = addExecutors(maxNeeded) logDebug(s"Starting timer to add more executors (to " + - s"expire in $sustainedSchedulerBacklogTimeout seconds)") - addTime += sustainedSchedulerBacklogTimeout * 1000 + s"expire in $sustainedSchedulerBacklogTimeoutS seconds)") + addTime += sustainedSchedulerBacklogTimeoutS * 1000 delta } else { 0 @@ -351,7 +351,7 @@ private[spark] class ExecutorAllocationManager( val removeRequestAcknowledged = testing || client.killExecutor(executorId) if (removeRequestAcknowledged) { logInfo(s"Removing executor $executorId because it has been idle for " + - s"$executorIdleTimeout seconds (new desired total will be ${numExistingExecutors - 1})") + s"$executorIdleTimeoutS seconds (new desired total will be ${numExistingExecutors - 1})") executorsPendingToRemove.add(executorId) true } else { @@ -407,8 +407,8 @@ private[spark] class ExecutorAllocationManager( private def onSchedulerBacklogged(): Unit = synchronized { if (addTime == NOT_SET) { logDebug(s"Starting timer to add executors because pending tasks " + - s"are building up (to expire in $schedulerBacklogTimeout seconds)") - addTime = clock.getTimeMillis + schedulerBacklogTimeout * 1000 + s"are building up (to expire in $schedulerBacklogTimeoutS seconds)") + addTime = clock.getTimeMillis + schedulerBacklogTimeoutS * 1000 } } @@ -431,8 +431,8 @@ private[spark] class ExecutorAllocationManager( if (executorIds.contains(executorId)) { if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { logDebug(s"Starting idle timer for $executorId because there are no more tasks " + - s"scheduled to run on the executor (to expire in $executorIdleTimeout seconds)") - removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeout * 1000 + s"scheduled to run on the executor (to expire in $executorIdleTimeoutS seconds)") + removeTimes(executorId) = clock.getTimeMillis + executorIdleTimeoutS * 1000 } } else { logWarning(s"Attempted to mark unknown executor $executorId idle") diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index 5871b8c869f03..e3bd16f1cbf24 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -62,14 +62,17 @@ private[spark] class HeartbeatReceiver(sc: SparkContext) // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses // "milliseconds" - private val executorTimeoutMs = sc.conf.getOption("spark.network.timeout").map(_.toLong * 1000). - getOrElse(sc.conf.getLong("spark.storage.blockManagerSlaveTimeoutMs", 120000)) - + private val slaveTimeoutMs = + sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", "120s") + private val executorTimeoutMs = + sc.conf.getTimeAsSeconds("spark.network.timeout", s"${slaveTimeoutMs}ms") * 1000 + // "spark.network.timeoutInterval" uses "seconds", while // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" - private val checkTimeoutIntervalMs = - sc.conf.getOption("spark.network.timeoutInterval").map(_.toLong * 1000). - getOrElse(sc.conf.getLong("spark.storage.blockManagerTimeoutIntervalMs", 60000)) + private val timeoutIntervalMs = + sc.conf.getTimeAsMs("spark.storage.blockManagerTimeoutIntervalMs", "60s") + private val checkTimeoutIntervalMs = + sc.conf.getTimeAsSeconds("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms") * 1000 private var timeoutCheckingTask: ScheduledFuture[_] = null diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 0c123c96b8d7b..390e631647bd6 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -174,6 +174,42 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { getOption(key).getOrElse(defaultValue) } + /** + * Get a time parameter as seconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then seconds are assumed. + * @throws NoSuchElementException + */ + def getTimeAsSeconds(key: String): Long = { + Utils.timeStringAsSeconds(get(key)) + } + + /** + * Get a time parameter as seconds, falling back to a default if not set. If no + * suffix is provided then seconds are assumed. + * + */ + def getTimeAsSeconds(key: String, defaultValue: String): Long = { + Utils.timeStringAsSeconds(get(key, defaultValue)) + } + + /** + * Get a time parameter as milliseconds; throws a NoSuchElementException if it's not set. If no + * suffix is provided then milliseconds are assumed. + * @throws NoSuchElementException + */ + def getTimeAsMs(key: String): Long = { + Utils.timeStringAsMs(get(key)) + } + + /** + * Get a time parameter as milliseconds, falling back to a default if not set. If no + * suffix is provided then milliseconds are assumed. + */ + def getTimeAsMs(key: String, defaultValue: String): Long = { + Utils.timeStringAsMs(get(key, defaultValue)) + } + + /** Get a parameter as an Option */ def getOption(key: String): Option[String] = { Option(settings.get(key)) 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 14f99a464b6e9..516f619529c48 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -436,14 +436,14 @@ private[spark] class Executor( * This thread stops running when the executor is stopped. */ private def startDriverHeartbeater(): Unit = { - val interval = conf.getInt("spark.executor.heartbeatInterval", 10000) + val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s") val thread = new Thread() { override def run() { // Sleep a random interval so the heartbeats don't end up in sync - Thread.sleep(interval + (math.random * interval).asInstanceOf[Int]) + Thread.sleep(intervalMs + (math.random * intervalMs).asInstanceOf[Int]) while (!isStopped) { reportHeartBeat() - Thread.sleep(interval) + Thread.sleep(intervalMs) } } } diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 741fe3e1ea750..8e3c30fc3d781 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -82,7 +82,8 @@ private[nio] class ConnectionManager( new HashedWheelTimer(Utils.namedThreadFactory("AckTimeoutMonitor")) private val ackTimeout = - conf.getInt("spark.core.connection.ack.wait.timeout", conf.getInt("spark.network.timeout", 120)) + conf.getTimeAsSeconds("spark.core.connection.ack.wait.timeout", + conf.get("spark.network.timeout", "120s")) // Get the thread counts from the Spark Configuration. // diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 076b36e86c0ce..2362cc7240039 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -62,10 +62,10 @@ private[spark] class TaskSchedulerImpl( val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL = conf.getLong("spark.speculation.interval", 100) + val SPECULATION_INTERVAL_MS = conf.getTimeAsMs("spark.speculation.interval", "100ms") // Threshold above which we warn user initial TaskSet may be starved - val STARVATION_TIMEOUT = conf.getLong("spark.starvation.timeout", 15000) + val STARVATION_TIMEOUT_MS = conf.getTimeAsMs("spark.starvation.timeout", "15s") // CPUs to request per task val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) @@ -143,8 +143,8 @@ private[spark] class TaskSchedulerImpl( if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") import sc.env.actorSystem.dispatcher - sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL milliseconds, - SPECULATION_INTERVAL milliseconds) { + sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL_MS milliseconds, + SPECULATION_INTERVAL_MS milliseconds) { Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } } } @@ -173,7 +173,7 @@ private[spark] class TaskSchedulerImpl( this.cancel() } } - }, STARVATION_TIMEOUT, STARVATION_TIMEOUT) + }, STARVATION_TIMEOUT_MS, STARVATION_TIMEOUT_MS) } hasReceivedTask = true } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index d509881c74fef..7dc325283d961 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -848,15 +848,18 @@ private[spark] class TaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.get("spark.locality.wait", "3000") - level match { - case TaskLocality.PROCESS_LOCAL => - conf.get("spark.locality.wait.process", defaultWait).toLong - case TaskLocality.NODE_LOCAL => - conf.get("spark.locality.wait.node", defaultWait).toLong - case TaskLocality.RACK_LOCAL => - conf.get("spark.locality.wait.rack", defaultWait).toLong - case _ => 0L + val defaultWait = conf.get("spark.locality.wait", "3s") + val localityWaitKey = level match { + case TaskLocality.PROCESS_LOCAL => "spark.locality.wait.process" + case TaskLocality.NODE_LOCAL => "spark.locality.wait.node" + case TaskLocality.RACK_LOCAL => "spark.locality.wait.rack" + case _ => null + } + + if (localityWaitKey != null) { + conf.getTimeAsMs(localityWaitKey, defaultWait) + } else { + 0L } } 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 4c49da87af9dc..63987dfb32695 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 @@ -52,8 +52,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) // Submit tasks after maxRegisteredWaitingTime milliseconds // if minRegisteredRatio has not yet been reached - val maxRegisteredWaitingTime = - conf.getInt("spark.scheduler.maxRegisteredResourcesWaitingTime", 30000) + val maxRegisteredWaitingTimeMs = + conf.getTimeAsMs("spark.scheduler.maxRegisteredResourcesWaitingTime", "30s") val createTime = System.currentTimeMillis() private val executorDataMap = new HashMap[String, ExecutorData] @@ -77,12 +77,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def onStart() { // Periodically revive offers to allow delay scheduling to work - val reviveInterval = conf.getLong("spark.scheduler.revive.interval", 1000) + val reviveIntervalMs = conf.getTimeAsMs("spark.scheduler.revive.interval", "1s") + reviveThread.scheduleAtFixedRate(new Runnable { override def run(): Unit = Utils.tryLogNonFatalError { Option(self).foreach(_.send(ReviveOffers)) } - }, 0, reviveInterval, TimeUnit.MILLISECONDS) + }, 0, reviveIntervalMs, TimeUnit.MILLISECONDS) } override def receive: PartialFunction[Any, Unit] = { @@ -301,9 +302,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp s"reached minRegisteredResourcesRatio: $minRegisteredRatio") return true } - if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTime) { + if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTimeMs) { logInfo("SchedulerBackend is ready for scheduling beginning after waiting " + - s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTime(ms)") + s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTimeMs(ms)") return true } false 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 6c2c5261306e7..8e8cc7cc6389e 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -20,7 +20,6 @@ package org.apache.spark.util import scala.collection.JavaConversions.mapAsJavaMap import scala.concurrent.Await import scala.concurrent.duration.{Duration, FiniteDuration} -import scala.util.Try import akka.actor.{ActorRef, ActorSystem, ExtendedActorSystem} import akka.pattern.ask @@ -66,7 +65,8 @@ private[spark] object AkkaUtils extends Logging { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) - val akkaTimeout = conf.getInt("spark.akka.timeout", conf.getInt("spark.network.timeout", 120)) + val akkaTimeoutS = conf.getTimeAsSeconds("spark.akka.timeout", + conf.get("spark.network.timeout", "120s")) val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" @@ -78,8 +78,8 @@ private[spark] object AkkaUtils extends Logging { val logAkkaConfig = if (conf.getBoolean("spark.akka.logAkkaConfig", false)) "on" else "off" - val akkaHeartBeatPauses = conf.getInt("spark.akka.heartbeat.pauses", 6000) - val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000) + val akkaHeartBeatPausesS = conf.getTimeAsSeconds("spark.akka.heartbeat.pauses", "6000s") + val akkaHeartBeatIntervalS = conf.getTimeAsSeconds("spark.akka.heartbeat.interval", "1000s") val secretKey = securityManager.getSecretKey() val isAuthOn = securityManager.isAuthenticationEnabled() @@ -102,14 +102,14 @@ private[spark] object AkkaUtils extends Logging { |akka.jvm-exit-on-fatal-error = off |akka.remote.require-cookie = "$requireCookie" |akka.remote.secure-cookie = "$secureCookie" - |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s - |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s + |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatIntervalS s + |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPausesS s |akka.actor.provider = "akka.remote.RemoteActorRefProvider" |akka.remote.netty.tcp.transport-class = "akka.remote.transport.netty.NettyTransport" |akka.remote.netty.tcp.hostname = "$host" |akka.remote.netty.tcp.port = $port |akka.remote.netty.tcp.tcp-nodelay = on - |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s + |akka.remote.netty.tcp.connection-timeout = $akkaTimeoutS s |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}B |akka.remote.netty.tcp.execution-pool-size = $akkaThreads |akka.actor.default-dispatcher.throughput = $akkaBatchSize diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 375ed430bde45..2bbfc988a99a8 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -76,7 +76,7 @@ private[spark] object MetadataCleanerType extends Enumeration { // initialization of StreamingContext. It's okay for users trying to configure stuff themselves. private[spark] object MetadataCleaner { def getDelaySeconds(conf: SparkConf): Int = { - conf.getInt("spark.cleaner.ttl", -1) + conf.getTimeAsSeconds("spark.cleaner.ttl", "-1").toInt } def getDelaySeconds( 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 a541d660cd5c6..1029b0f9fce1e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -22,7 +22,7 @@ import java.lang.management.ManagementFactory import java.net._ import java.nio.ByteBuffer import java.util.{Properties, Locale, Random, UUID} -import java.util.concurrent.{ThreadFactory, ConcurrentHashMap, Executors, ThreadPoolExecutor} +import java.util.concurrent._ import javax.net.ssl.HttpsURLConnection import scala.collection.JavaConversions._ @@ -47,6 +47,7 @@ import tachyon.client.{TachyonFS, TachyonFile} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** CallSite represents a place in user code. It can have a short and a long form. */ @@ -612,9 +613,10 @@ private[spark] object Utils extends Logging { } Utils.setupSecureURLConnection(uc, securityMgr) - val timeout = conf.getInt("spark.files.fetchTimeout", 60) * 1000 - uc.setConnectTimeout(timeout) - uc.setReadTimeout(timeout) + val timeoutMs = + conf.getTimeAsSeconds("spark.files.fetchTimeout", "60s").toInt * 1000 + uc.setConnectTimeout(timeoutMs) + uc.setReadTimeout(timeoutMs) uc.connect() val in = uc.getInputStream() downloadFile(url, in, targetFile, fileOverwrite) @@ -1018,6 +1020,22 @@ private[spark] object Utils extends Logging { ) } + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. + */ + def timeStringAsMs(str: String): Long = { + JavaUtils.timeStringAsMs(str) + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. + */ + def timeStringAsSeconds(str: String): Long = { + JavaUtils.timeStringAsSec(str) + } + /** * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. */ diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 3ded1e4af8742..6b3049b28cd5e 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -684,10 +684,11 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext wit .set("spark.dynamicAllocation.enabled", "true") .set("spark.dynamicAllocation.minExecutors", minExecutors.toString) .set("spark.dynamicAllocation.maxExecutors", maxExecutors.toString) - .set("spark.dynamicAllocation.schedulerBacklogTimeout", schedulerBacklogTimeout.toString) + .set("spark.dynamicAllocation.schedulerBacklogTimeout", + s"${schedulerBacklogTimeout.toString}s") .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", - sustainedSchedulerBacklogTimeout.toString) - .set("spark.dynamicAllocation.executorIdleTimeout", executorIdleTimeout.toString) + s"${sustainedSchedulerBacklogTimeout.toString}s") + .set("spark.dynamicAllocation.executorIdleTimeout", s"${executorIdleTimeout.toString}s") .set("spark.dynamicAllocation.testing", "true") val sc = new SparkContext(conf) contexts += sc diff --git a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala index 716f875d30b8a..02424c59d6831 100644 --- a/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/network/nio/ConnectionManagerSuite.scala @@ -260,8 +260,8 @@ class ConnectionManagerSuite extends FunSuite { test("sendMessageReliably timeout") { val clientConf = new SparkConf clientConf.set("spark.authenticate", "false") - val ackTimeout = 30 - clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeout}") + val ackTimeoutS = 30 + clientConf.set("spark.core.connection.ack.wait.timeout", s"${ackTimeoutS}s") val clientSecurityManager = new SecurityManager(clientConf) val manager = new ConnectionManager(0, clientConf, clientSecurityManager) @@ -272,7 +272,7 @@ class ConnectionManagerSuite extends FunSuite { val managerServer = new ConnectionManager(0, serverConf, serverSecurityManager) managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { // sleep 60 sec > ack timeout for simulating server slow down or hang up - Thread.sleep(ackTimeout * 3 * 1000) + Thread.sleep(ackTimeoutS * 3 * 1000) None }) @@ -287,7 +287,7 @@ class ConnectionManagerSuite extends FunSuite { // Otherwise TimeoutExcepton is thrown from Await.result. // We expect TimeoutException is not thrown. intercept[IOException] { - Await.result(future, (ackTimeout * 2) second) + Await.result(future, (ackTimeoutS * 2) second) } manager.stop() diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 716d12c0762cf..6198cea46ddf8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.scheduler -import java.io.{ObjectInputStream, ObjectOutputStream, IOException} import java.util.Random import scala.collection.mutable.ArrayBuffer @@ -27,7 +26,7 @@ import org.scalatest.FunSuite import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{ManualClock, Utils} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -152,7 +151,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { private val conf = new SparkConf - val LOCALITY_WAIT = conf.getLong("spark.locality.wait", 3000) + val LOCALITY_WAIT_MS = conf.getTimeAsMs("spark.locality.wait", "3s") val MAX_TASK_FAILURES = 4 override def beforeEach() { @@ -240,7 +239,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) == None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // Offer host1, exec1 again, at NODE_LOCAL level: the node local (task 2) should // get chosen before the noPref task assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) @@ -251,7 +250,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Offer host2, exec3 again, at NODE_LOCAL level: we should get noPref task // after failing to find a node_Local task assert(manager.resourceOffer("exec2", "host2", NODE_LOCAL) == None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) assert(manager.resourceOffer("exec2", "host2", NO_PREF).get.index == 3) } @@ -292,7 +291,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Offer host1 again: nothing should get chosen assert(manager.resourceOffer("exec1", "host1", ANY) === None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // Offer host1 again: second task (on host2) should get chosen assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) @@ -306,7 +305,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Now that we've launched a local task, we should no longer launch the task for host3 assert(manager.resourceOffer("exec2", "host2", ANY) === None) - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // After another delay, we can go ahead and launch that task non-locally assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 3) @@ -338,7 +337,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // nothing should be chosen assert(manager.resourceOffer("exec1", "host1", ANY) === None) - clock.advance(LOCALITY_WAIT * 2) + clock.advance(LOCALITY_WAIT_MS * 2) // task 1 and 2 would be scheduled as nonLocal task assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) @@ -528,7 +527,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY))) // Set allowed locality to ANY - clock.advance(LOCALITY_WAIT * 3) + clock.advance(LOCALITY_WAIT_MS * 3) // Offer host3 // No task is scheduled if we restrict locality to RACK_LOCAL assert(manager.resourceOffer("execC", "host3", RACK_LOCAL) === None) @@ -622,12 +621,12 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index == 1) manager.speculatableTasks += 1 - clock.advance(LOCALITY_WAIT) + clock.advance(LOCALITY_WAIT_MS) // schedule the nonPref task assert(manager.resourceOffer("execA", "host1", NO_PREF).get.index === 2) // schedule the speculative task assert(manager.resourceOffer("execB", "host2", NO_PREF).get.index === 1) - clock.advance(LOCALITY_WAIT * 3) + clock.advance(LOCALITY_WAIT_MS * 3) // schedule non-local tasks assert(manager.resourceOffer("execB", "host2", ANY).get.index === 3) } @@ -716,13 +715,13 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Valid locality should contain PROCESS_LOCAL, NODE_LOCAL and ANY assert(manager.myLocalityLevels.sameElements(Array(PROCESS_LOCAL, NODE_LOCAL, ANY))) assert(manager.resourceOffer("execA", "host1", ANY) !== None) - clock.advance(LOCALITY_WAIT * 4) + clock.advance(LOCALITY_WAIT_MS * 4) assert(manager.resourceOffer("execB.2", "host2", ANY) !== None) sched.removeExecutor("execA") sched.removeExecutor("execB.2") manager.executorLost("execA", "host1") manager.executorLost("execB.2", "host2") - clock.advance(LOCALITY_WAIT * 4) + clock.advance(LOCALITY_WAIT_MS * 4) sched.addExecutor("execC", "host3") manager.executorAdded() // Prior to the fix, this line resulted in an ArrayIndexOutOfBoundsException: diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index b4de90b65d545..ffa5162a31841 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -76,7 +76,7 @@ class BlockManagerReplicationSuite extends FunSuite with Matchers with BeforeAnd conf.set("spark.storage.unrollMemoryThreshold", "512") // to make a replication attempt to inactive store fail fast - conf.set("spark.core.connection.ack.wait.timeout", "1") + conf.set("spark.core.connection.ack.wait.timeout", "1s") // to make cached peers refresh frequently conf.set("spark.storage.cachedPeersTtl", "10") diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 449fb87f111c4..fb97e650ff95c 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -23,6 +23,7 @@ import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStr import java.net.{BindException, ServerSocket, URI} import java.nio.{ByteBuffer, ByteOrder} import java.text.DecimalFormatSymbols +import java.util.concurrent.TimeUnit import java.util.Locale import com.google.common.base.Charsets.UTF_8 @@ -35,7 +36,50 @@ import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf class UtilsSuite extends FunSuite with ResetSystemProperties { + + test("timeConversion") { + // Test -1 + assert(Utils.timeStringAsSeconds("-1") === -1) + + // Test zero + assert(Utils.timeStringAsSeconds("0") === 0) + + assert(Utils.timeStringAsSeconds("1") === 1) + assert(Utils.timeStringAsSeconds("1s") === 1) + assert(Utils.timeStringAsSeconds("1000ms") === 1) + assert(Utils.timeStringAsSeconds("1000000us") === 1) + assert(Utils.timeStringAsSeconds("1m") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1min") === TimeUnit.MINUTES.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1h") === TimeUnit.HOURS.toSeconds(1)) + assert(Utils.timeStringAsSeconds("1d") === TimeUnit.DAYS.toSeconds(1)) + + assert(Utils.timeStringAsMs("1") === 1) + assert(Utils.timeStringAsMs("1ms") === 1) + assert(Utils.timeStringAsMs("1000us") === 1) + assert(Utils.timeStringAsMs("1s") === TimeUnit.SECONDS.toMillis(1)) + assert(Utils.timeStringAsMs("1m") === TimeUnit.MINUTES.toMillis(1)) + assert(Utils.timeStringAsMs("1min") === TimeUnit.MINUTES.toMillis(1)) + assert(Utils.timeStringAsMs("1h") === TimeUnit.HOURS.toMillis(1)) + assert(Utils.timeStringAsMs("1d") === TimeUnit.DAYS.toMillis(1)) + + // Test invalid strings + intercept[NumberFormatException] { + Utils.timeStringAsMs("This breaks 600s") + } + + intercept[NumberFormatException] { + Utils.timeStringAsMs("This breaks 600ds") + } + intercept[NumberFormatException] { + Utils.timeStringAsMs("600s This breaks") + } + + intercept[NumberFormatException] { + Utils.timeStringAsMs("This 123s breaks") + } + } + test("bytesToString") { assert(Utils.bytesToString(10) === "10.0 B") assert(Utils.bytesToString(1500) === "1500.0 B") diff --git a/docs/configuration.md b/docs/configuration.md index 7fe11475212b3..7169ec295ef7f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -35,9 +35,19 @@ val conf = new SparkConf() val sc = new SparkContext(conf) {% endhighlight %} -Note that we can have more than 1 thread in local mode, and in cases like spark streaming, we may actually -require one to prevent any sort of starvation issues. +Note that we can have more than 1 thread in local mode, and in cases like Spark Streaming, we may +actually require one to prevent any sort of starvation issues. +Properties that specify some time duration should be configured with a unit of time. +The following format is accepted: + + 25ms (milliseconds) + 5s (seconds) + 10m or 10min (minutes) + 3h (hours) + 5d (days) + 1y (years) + ## Dynamically Loading Spark Properties In some cases, you may want to avoid hard-coding certain configurations in a `SparkConf`. For instance, if you'd like to run the same application with different masters or different @@ -429,10 +439,10 @@ Apart from these, the following properties are also available, and may be useful spark.shuffle.io.retryWait - 5 + 5s - (Netty only) Seconds to wait between retries of fetches. The maximum delay caused by retrying - is simply maxRetries * retryWait, by default 15 seconds. + (Netty only) How long to wait between retries of fetches. The maximum delay caused by retrying + is 15 seconds by default, calculated as maxRetries * retryWait. @@ -732,17 +742,17 @@ Apart from these, the following properties are also available, and may be useful spark.executor.heartbeatInterval - 10000 - Interval (milliseconds) between each executor's heartbeats to the driver. Heartbeats let + 10s + Interval between each executor's heartbeats to the driver. Heartbeats let the driver know that the executor is still alive and update it with metrics for in-progress tasks. spark.files.fetchTimeout - 60 + 60s Communication timeout to use when fetching files added through SparkContext.addFile() from - the driver, in seconds. + the driver. @@ -853,11 +863,11 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval - 1000 + 1000s This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger - interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more + interval value reduces network overhead and a smaller value ( ~ 1 s) might be more informative for Akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` if you need to. A likely positive use case for using failure detector would be: a sensistive failure detector can help evict rogue executors quickly. However this is usually not the case @@ -868,11 +878,11 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.pauses - 6000 + 6000s This is set to a larger value to disable the transport failure detector that comes built in to Akka. It can be enabled again, if you plan to use this feature (Not recommended). Acceptable heart - beat pause in seconds for Akka. This can be used to control sensitivity to GC pauses. Tune + beat pause for Akka. This can be used to control sensitivity to GC pauses. Tune this along with `spark.akka.heartbeat.interval` if you need to. @@ -886,9 +896,9 @@ Apart from these, the following properties are also available, and may be useful spark.akka.timeout - 100 + 100s - Communication timeout between Spark nodes, in seconds. + Communication timeout between Spark nodes. @@ -938,10 +948,10 @@ Apart from these, the following properties are also available, and may be useful spark.network.timeout - 120 + 120s - Default timeout for all network interactions, in seconds. This config will be used in - place of spark.core.connection.ack.wait.timeout, spark.akka.timeout, + Default timeout for all network interactions. This config will be used in place of + spark.core.connection.ack.wait.timeout, spark.akka.timeout, spark.storage.blockManagerSlaveTimeoutMs or spark.shuffle.io.connectionTimeout, if they are not configured. @@ -989,9 +999,9 @@ Apart from these, the following properties are also available, and may be useful spark.locality.wait - 3000 + 3s - Number of milliseconds to wait to launch a data-local task before giving up and launching it + How long to wait to launch a data-local task before giving up and launching it on a less-local node. The same wait will be used to step through multiple locality levels (process-local, node-local, rack-local and then any). It is also possible to customize the waiting time for each level by setting spark.locality.wait.node, etc. @@ -1024,10 +1034,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.maxRegisteredResourcesWaitingTime - 30000 + 30s - Maximum amount of time to wait for resources to register before scheduling begins - (in milliseconds). + Maximum amount of time to wait for resources to register before scheduling begins. @@ -1054,10 +1063,9 @@ Apart from these, the following properties are also available, and may be useful spark.scheduler.revive.interval - 1000 + 1s - The interval length for the scheduler to revive the worker resource offers to run tasks - (in milliseconds). + The interval length for the scheduler to revive the worker resource offers to run tasks. @@ -1070,9 +1078,9 @@ Apart from these, the following properties are also available, and may be useful spark.speculation.interval - 100 + 100ms - How often Spark will check for tasks to speculate, in milliseconds. + How often Spark will check for tasks to speculate. @@ -1127,10 +1135,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.executorIdleTimeout - 600 + 600s - If dynamic allocation is enabled and an executor has been idle for more than this duration - (in seconds), the executor will be removed. For more detail, see this + If dynamic allocation is enabled and an executor has been idle for more than this duration, + the executor will be removed. For more detail, see this
    description. @@ -1157,10 +1165,10 @@ Apart from these, the following properties are also available, and may be useful spark.dynamicAllocation.schedulerBacklogTimeout - 5 + 5s If dynamic allocation is enabled and there have been pending tasks backlogged for more than - this duration (in seconds), new executors will be requested. For more detail, see this + this duration, new executors will be requested. For more detail, see this description. @@ -1215,18 +1223,18 @@ Apart from these, the following properties are also available, and may be useful spark.core.connection.ack.wait.timeout - 60 + 60s - Number of seconds for the connection to wait for ack to occur before timing + How long for the connection to wait for ack to occur before timing out and giving up. To avoid unwilling timeout caused by long pause like GC, you can set larger value. spark.core.connection.auth.wait.timeout - 30 + 30s - Number of seconds for the connection to wait for authentication to occur before timing + How long for the connection to wait for authentication to occur before timing out and giving up. @@ -1347,9 +1355,9 @@ Apart from these, the following properties are also available, and may be useful Property NameDefaultMeaning spark.streaming.blockInterval - 200 + 200ms - Interval (milliseconds) at which data received by Spark Streaming receivers is chunked + Interval at which data received by Spark Streaming receivers is chunked into blocks of data before storing them in Spark. Minimum recommended - 50 ms. See the performance tuning section in the Spark Streaming programing guide for more details. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index ed5bb263a5809..853c9f26b0ec9 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -48,9 +48,9 @@ Most of the configs are the same for Spark on YARN as for other deployment modes spark.yarn.am.waitTime - 100000 + 100s - In yarn-cluster mode, time in milliseconds for the application master to wait for the + In yarn-cluster mode, time for the application master to wait for the SparkContext to be initialized. In yarn-client mode, time for the application master to wait for the driver to connect to it. diff --git a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java index 73da9b7346f4d..b6fbace509a0e 100644 --- a/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java +++ b/network/common/src/main/java/org/apache/spark/network/util/JavaUtils.java @@ -21,9 +21,13 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.util.concurrent.TimeUnit; +import java.util.regex.Matcher; +import java.util.regex.Pattern; import com.google.common.base.Charsets; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableMap; import io.netty.buffer.Unpooled; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -121,4 +125,66 @@ private static boolean isSymlink(File file) throws IOException { } return !fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile()); } + + private static ImmutableMap timeSuffixes = + ImmutableMap.builder() + .put("us", TimeUnit.MICROSECONDS) + .put("ms", TimeUnit.MILLISECONDS) + .put("s", TimeUnit.SECONDS) + .put("m", TimeUnit.MINUTES) + .put("min", TimeUnit.MINUTES) + .put("h", TimeUnit.HOURS) + .put("d", TimeUnit.DAYS) + .build(); + + /** + * Convert a passed time string (e.g. 50s, 100ms, or 250us) to a time count for + * internal use. If no suffix is provided a direct conversion is attempted. + */ + private static long parseTimeString(String str, TimeUnit unit) { + String lower = str.toLowerCase().trim(); + + try { + String suffix; + long val; + Matcher m = Pattern.compile("(-?[0-9]+)([a-z]+)?").matcher(lower); + if (m.matches()) { + val = Long.parseLong(m.group(1)); + suffix = m.group(2); + } else { + throw new NumberFormatException("Failed to parse time string: " + str); + } + + // Check for invalid suffixes + if (suffix != null && !timeSuffixes.containsKey(suffix)) { + throw new NumberFormatException("Invalid suffix: \"" + suffix + "\""); + } + + // If suffix is valid use that, otherwise none was provided and use the default passed + return unit.convert(val, suffix != null ? timeSuffixes.get(suffix) : unit); + } catch (NumberFormatException e) { + String timeError = "Time must be specified as seconds (s), " + + "milliseconds (ms), microseconds (us), minutes (m or min) hour (h), or day (d). " + + "E.g. 50s, 100ms, or 250us."; + + throw new NumberFormatException(timeError + "\n" + e.getMessage()); + } + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If + * no suffix is provided, the passed number is assumed to be in ms. + */ + public static long timeStringAsMs(String str) { + return parseTimeString(str, TimeUnit.MILLISECONDS); + } + + /** + * Convert a time parameter such as (50s, 100ms, or 250us) to seconds for internal use. If + * no suffix is provided, the passed number is assumed to be in seconds. + */ + public static long timeStringAsSec(String str) { + return parseTimeString(str, TimeUnit.SECONDS); + } + } 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 2eaf3b71d9a49..0aef7f1987315 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 @@ -37,8 +37,11 @@ public boolean preferDirectBufs() { /** Connect timeout in milliseconds. Default 120 secs. */ public int connectionTimeoutMs() { - int defaultTimeout = conf.getInt("spark.network.timeout", 120); - return conf.getInt("spark.shuffle.io.connectionTimeout", defaultTimeout) * 1000; + long defaultNetworkTimeoutS = JavaUtils.timeStringAsSec( + conf.get("spark.network.timeout", "120s")); + long defaultTimeoutMs = JavaUtils.timeStringAsSec( + conf.get("spark.shuffle.io.connectionTimeout", defaultNetworkTimeoutS + "s")) * 1000; + return (int) defaultTimeoutMs; } /** Number of concurrent connections between two nodes for fetching data. */ @@ -68,7 +71,9 @@ 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 saslRTTimeoutMs() { return conf.getInt("spark.shuffle.sasl.timeout", 30) * 1000; } + public int saslRTTimeoutMs() { + return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.sasl.timeout", "30s")) * 1000; + } /** * Max number of times we will try IO exceptions (such as connection timeouts) per request. @@ -80,7 +85,9 @@ 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 ioRetryWaitTimeMs() { return conf.getInt("spark.shuffle.io.retryWait", 5) * 1000; } + public int ioRetryWaitTimeMs() { + return (int) JavaUtils.timeStringAsSec(conf.get("spark.shuffle.io.retryWait", "5s")) * 1000; + } /** * Minimum size of a block that we should start using memory map rather than reading in through diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala index 42514d8b47dcf..f4963a78e1d18 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -24,7 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{Logging, SparkConf} import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.SystemClock +import org.apache.spark.util.{SystemClock, Utils} /** Listener object for BlockGenerator events */ private[streaming] trait BlockGeneratorListener { @@ -79,9 +79,9 @@ private[streaming] class BlockGenerator( private case class Block(id: StreamBlockId, buffer: ArrayBuffer[Any]) private val clock = new SystemClock() - private val blockInterval = conf.getLong("spark.streaming.blockInterval", 200) + private val blockIntervalMs = conf.getTimeAsMs("spark.streaming.blockInterval", "200ms") private val blockIntervalTimer = - new RecurringTimer(clock, blockInterval, updateCurrentBuffer, "BlockGenerator") + new RecurringTimer(clock, blockIntervalMs, updateCurrentBuffer, "BlockGenerator") private val blockQueueSize = conf.getInt("spark.streaming.blockQueueSize", 10) private val blocksForPushing = new ArrayBlockingQueue[Block](blockQueueSize) private val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } @@ -132,7 +132,7 @@ private[streaming] class BlockGenerator( val newBlockBuffer = currentBuffer currentBuffer = new ArrayBuffer[Any] if (newBlockBuffer.size > 0) { - val blockId = StreamBlockId(receiverId, time - blockInterval) + val blockId = StreamBlockId(receiverId, time - blockIntervalMs) val newBlock = new Block(blockId, newBlockBuffer) listener.onGenerateBlock(blockId) blocksForPushing.put(newBlock) // put is blocking when queue is full diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 4946806d2ee95..58e56638a2dca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -24,7 +24,7 @@ import akka.actor.{ActorRef, Props, Actor} import org.apache.spark.{SparkEnv, Logging} import org.apache.spark.streaming.{Checkpoint, CheckpointWriter, Time} import org.apache.spark.streaming.util.RecurringTimer -import org.apache.spark.util.{Clock, ManualClock} +import org.apache.spark.util.{Clock, ManualClock, Utils} /** Event classes for JobGenerator */ private[scheduler] sealed trait JobGeneratorEvent @@ -104,17 +104,15 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") val timeWhenStopStarted = System.currentTimeMillis() - val stopTimeout = conf.getLong( - "spark.streaming.gracefulStopTimeout", - 10 * ssc.graph.batchDuration.milliseconds - ) + val stopTimeoutMs = conf.getTimeAsMs( + "spark.streaming.gracefulStopTimeout", s"${10 * ssc.graph.batchDuration.milliseconds}ms") val pollTime = 100 // To prevent graceful stop to get stuck permanently def hasTimedOut: Boolean = { - val timedOut = (System.currentTimeMillis() - timeWhenStopStarted) > stopTimeout + val timedOut = (System.currentTimeMillis() - timeWhenStopStarted) > stopTimeoutMs if (timedOut) { - logWarning("Timed out while stopping the job generator (timeout = " + stopTimeout + ")") + logWarning("Timed out while stopping the job generator (timeout = " + stopTimeoutMs + ")") } timedOut } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index 10c35cba8dc53..91261a9db7360 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -131,11 +131,11 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 200 - val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString) + val blockIntervalMs = 200 + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms") val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 5 - val waitTime = expectedBlocks * blockInterval + (blockInterval / 2) + val waitTime = expectedBlocks * blockIntervalMs + (blockIntervalMs / 2) val generatedData = new ArrayBuffer[Int] // Generate blocks @@ -157,15 +157,15 @@ class ReceiverSuite extends TestSuiteBase with Timeouts with Serializable { test("block generator throttling") { val blockGeneratorListener = new FakeBlockGeneratorListener - val blockInterval = 100 + val blockIntervalMs = 100 val maxRate = 100 - val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString). + val conf = new SparkConf().set("spark.streaming.blockInterval", s"${blockIntervalMs}ms"). set("spark.streaming.receiver.maxRate", maxRate.toString) val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) val expectedBlocks = 20 - val waitTime = expectedBlocks * blockInterval + val waitTime = expectedBlocks * blockIntervalMs val expectedMessages = maxRate * waitTime / 1000 - val expectedMessagesPerBlock = maxRate * blockInterval / 1000 + val expectedMessagesPerBlock = maxRate * blockIntervalMs / 1000 val generatedData = new ArrayBuffer[Int] // Generate blocks 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 d1bbf39dc7897..58353a5f97c8a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -73,9 +73,9 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from conf with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("from existing SparkContext") { @@ -85,24 +85,26 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("from existing SparkContext with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("from checkpoint") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", "10") + myConf.set("spark.cleaner.ttl", "10s") val ssc1 = new StreamingContext(myConf, batchDuration) addInputStream(ssc1).register() ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) - assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") + assert( + Utils.timeStringAsSeconds(cp.sparkConfPairs + .toMap.getOrElse("spark.cleaner.ttl", "-1")) === 10) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(newCp.createSparkConf().getInt("spark.cleaner.ttl", -1) === 10) + assert(newCp.createSparkConf().getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) ssc = new StreamingContext(null, newCp, null) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) + assert(ssc.conf.getTimeAsSeconds("spark.cleaner.ttl", "-1") === 10) } test("start and stop state check") { @@ -176,7 +178,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.cleaner.ttl", "3600") + conf.set("spark.cleaner.ttl", "3600s") sc = new SparkContext(conf) for (i <- 1 to 4) { logInfo("==================================\n\n\n") @@ -207,7 +209,7 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w test("stop slow receiver gracefully") { val conf = new SparkConf().setMaster(master).setAppName(appName) - conf.set("spark.streaming.gracefulStopTimeout", "20000") + conf.set("spark.streaming.gracefulStopTimeout", "20000s") sc = new SparkContext(conf) logInfo("==================================\n\n\n") ssc = new StreamingContext(sc, Milliseconds(100)) diff --git a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 26259cee77151..c357b7ae9d4da 100644 --- a/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -296,7 +296,7 @@ private[spark] class ApplicationMaster( // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = - sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) + sparkConf.getTimeAsMs("spark.yarn.scheduler.heartbeat.interval-ms", "5s") // must be <= expiryInterval / 2. val interval = math.max(0, math.min(expiryInterval / 2, schedulerInterval)) @@ -379,7 +379,8 @@ private[spark] class ApplicationMaster( logWarning( "spark.yarn.applicationMaster.waitTries is deprecated, use spark.yarn.am.waitTime") } - val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", waitTries.getOrElse(100000L)) + val totalWaitTime = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", + s"${waitTries.getOrElse(100000L)}ms") val deadline = System.currentTimeMillis() + totalWaitTime while (sparkContextRef.get() == null && System.currentTimeMillis < deadline && !finished) { @@ -404,8 +405,8 @@ private[spark] class ApplicationMaster( // Spark driver should already be up since it launched us, but we don't want to // wait forever, so wait 100 seconds max to match the cluster mode setting. - val totalWaitTime = sparkConf.getLong("spark.yarn.am.waitTime", 100000L) - val deadline = System.currentTimeMillis + totalWaitTime + val totalWaitTimeMs = sparkConf.getTimeAsMs("spark.yarn.am.waitTime", "100s") + val deadline = System.currentTimeMillis + totalWaitTimeMs while (!driverUp && !finished && System.currentTimeMillis < deadline) { try { From d7f2c198678d1f5ffc1c6da3a6fb189b0a4ef070 Mon Sep 17 00:00:00 2001 From: hlin09 Date: Mon, 13 Apr 2015 16:53:50 -0700 Subject: [PATCH 076/128] [SPARK-6881][SparkR] Changes the checkpoint directory name. Author: hlin09 Closes #5493 from hlin09/fixCheckpointDir and squashes the following commits: e67fc40 [hlin09] Change to temp dir. 1f7ed9e [hlin09] Change the checkpoint dir name. --- R/pkg/R/RDD.R | 2 +- R/pkg/R/context.R | 2 +- R/pkg/inst/tests/test_rdd.R | 5 +++-- 3 files changed, 5 insertions(+), 4 deletions(-) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index 604ad03c407b9..d6a75007a6ad7 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -279,7 +279,7 @@ setMethod("unpersist", #' @examples #'\dontrun{ #' sc <- sparkR.init() -#' setCheckpointDir(sc, "checkpoints") +#' setCheckpointDir(sc, "checkpoint") #' rdd <- parallelize(sc, 1:10, 2L) #' checkpoint(rdd) #'} diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index 2fc0bb294bcce..ebbb8fba1052d 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -216,7 +216,7 @@ broadcast <- function(sc, object) { #' @examples #'\dontrun{ #' sc <- sparkR.init() -#' setCheckpointDir(sc, "~/checkpoints") +#' setCheckpointDir(sc, "~/checkpoint") #' rdd <- parallelize(sc, 1:2, 2L) #' checkpoint(rdd) #'} diff --git a/R/pkg/inst/tests/test_rdd.R b/R/pkg/inst/tests/test_rdd.R index f75e0817b9406..b76e4db03e715 100644 --- a/R/pkg/inst/tests/test_rdd.R +++ b/R/pkg/inst/tests/test_rdd.R @@ -141,7 +141,8 @@ test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkp unpersist(rdd2) expect_false(rdd2@env$isCached) - setCheckpointDir(sc, "checkpoints") + tempDir <- tempfile(pattern = "checkpoint") + setCheckpointDir(sc, tempDir) checkpoint(rdd2) expect_true(rdd2@env$isCheckpointed) @@ -152,7 +153,7 @@ test_that("PipelinedRDD support actions: cache(), persist(), unpersist(), checkp # make sure the data is collectable collect(rdd2) - unlink("checkpoints") + unlink(tempDir) }) test_that("reduce on RDD", { From 5b8b324f33e857b95de65031334846a7ca26fa60 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 13 Apr 2015 18:15:29 -0700 Subject: [PATCH 077/128] [SPARK-6303][SQL] Remove unnecessary Average in GeneratedAggregate Because `Average` is a `PartialAggregate`, we never get a `Average` node when reaching `HashAggregation` to prepare `GeneratedAggregate`. That is why in SQLQuerySuite there is already a test for `avg` with codegen. And it works. But we can find a case in `GeneratedAggregate` to deal with `Average`. Based on the above, we actually never execute this case. So we can remove this case from `GeneratedAggregate`. Author: Liang-Chi Hsieh Closes #4996 from viirya/add_average_codegened and squashes the following commits: 621c12f [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_average_codegened 368cfbc [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into add_average_codegened 74926d1 [Liang-Chi Hsieh] Add Average in canBeCodeGened lists. --- .../sql/execution/GeneratedAggregate.scala | 45 ------------------- 1 file changed, 45 deletions(-) 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 95176e425132d..b510cf033c4a4 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 @@ -153,51 +153,6 @@ case class GeneratedAggregate( AggregateEvaluation(currentSum :: Nil, initialValue :: Nil, updateFunction :: Nil, result) - case a @ Average(expr) => - val calcType = - expr.dataType match { - case DecimalType.Fixed(_, _) => - DecimalType.Unlimited - case _ => - expr.dataType - } - - val currentCount = AttributeReference("currentCount", LongType, nullable = false)() - val currentSum = AttributeReference("currentSum", calcType, nullable = false)() - val initialCount = Literal(0L) - val initialSum = Cast(Literal(0L), calcType) - - // If we're evaluating UnscaledValue(x), we can do Count on x directly, since its - // UnscaledValue will be null if and only if x is null; helps with Average on decimals - val toCount = expr match { - case UnscaledValue(e) => e - case _ => expr - } - - val updateCount = If(IsNotNull(toCount), Add(currentCount, Literal(1L)), currentCount) - val updateSum = Coalesce(Add(Cast(expr, calcType), currentSum) :: currentSum :: Nil) - - val result = - expr.dataType match { - case DecimalType.Fixed(_, _) => - If(EqualTo(currentCount, Literal(0L)), - Literal.create(null, a.dataType), - Cast(Divide( - Cast(currentSum, DecimalType.Unlimited), - Cast(currentCount, DecimalType.Unlimited)), a.dataType)) - case _ => - If(EqualTo(currentCount, Literal(0L)), - Literal.create(null, a.dataType), - Divide(Cast(currentSum, a.dataType), Cast(currentCount, a.dataType))) - } - - AggregateEvaluation( - currentCount :: currentSum :: Nil, - initialCount :: initialSum :: Nil, - updateCount :: updateSum :: Nil, - result - ) - case m @ Max(expr) => val currentMax = AttributeReference("currentMax", expr.dataType, nullable = true)() val initialValue = Literal.create(null, expr.dataType) From 4898dfa464be55772e3f9db10c48adcb3cfc9a3d Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Mon, 13 Apr 2015 18:16:33 -0700 Subject: [PATCH 078/128] [SPARK-6877][SQL] Add code generation support for Min Currently `min` is not supported in code generation. This pr adds the support for it. Author: Liang-Chi Hsieh Closes #5487 from viirya/add_min_codegen and squashes the following commits: 0ddec23 [Liang-Chi Hsieh] Add code generation support for Min. --- .../sql/catalyst/expressions/arithmetic.scala | 45 +++++++++++++++++++ .../expressions/codegen/CodeGenerator.scala | 24 ++++++++++ .../ExpressionEvaluationSuite.scala | 10 +++++ .../sql/execution/GeneratedAggregate.scala | 13 ++++++ .../spark/sql/execution/SparkStrategies.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 14 ++++-- 6 files changed, 104 insertions(+), 4 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 1f6526ef66c56..566b34f7c3a6a 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 @@ -369,6 +369,51 @@ case class MaxOf(left: Expression, right: Expression) extends Expression { override def toString: String = s"MaxOf($left, $right)" } +case class MinOf(left: Expression, right: Expression) extends Expression { + type EvaluatedType = Any + + override def foldable: Boolean = left.foldable && right.foldable + + override def nullable: Boolean = left.nullable && right.nullable + + override def children: Seq[Expression] = left :: right :: Nil + + override lazy val resolved = + left.resolved && right.resolved && + left.dataType == right.dataType + + override def dataType: DataType = { + if (!resolved) { + throw new UnresolvedException(this, + s"datatype. Can not resolve due to differing types ${left.dataType}, ${right.dataType}") + } + left.dataType + } + + lazy val ordering = left.dataType match { + case i: NativeType => i.ordering.asInstanceOf[Ordering[Any]] + case other => sys.error(s"Type $other does not support ordered operations") + } + + override def eval(input: Row): Any = { + val evalE1 = left.eval(input) + val evalE2 = right.eval(input) + if (evalE1 == null) { + evalE2 + } else if (evalE2 == null) { + evalE1 + } else { + if (ordering.compare(evalE1, evalE2) < 0) { + evalE1 + } else { + evalE2 + } + } + } + + override def toString: String = s"MinOf($left, $right)" +} + /** * A function that get the absolute value of the numeric value. */ 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 aac56e1568332..d141354a0f427 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 @@ -524,6 +524,30 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin } """.children + case MinOf(e1, e2) => + val eval1 = expressionEvaluator(e1) + val eval2 = expressionEvaluator(e2) + + eval1.code ++ eval2.code ++ + q""" + var $nullTerm = false + var $primitiveTerm: ${termForType(e1.dataType)} = ${defaultPrimitive(e1.dataType)} + + if (${eval1.nullTerm}) { + $nullTerm = ${eval2.nullTerm} + $primitiveTerm = ${eval2.primitiveTerm} + } else if (${eval2.nullTerm}) { + $nullTerm = ${eval1.nullTerm} + $primitiveTerm = ${eval1.primitiveTerm} + } else { + if (${eval1.primitiveTerm} < ${eval2.primitiveTerm}) { + $primitiveTerm = ${eval1.primitiveTerm} + } else { + $primitiveTerm = ${eval2.primitiveTerm} + } + } + """.children + case UnscaledValue(child) => val childEval = expressionEvaluator(child) 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 d2b1090a0cdd5..d4362a91d992c 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 @@ -233,6 +233,16 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { checkEvaluation(MaxOf(2, Literal.create(null, IntegerType)), 2) } + test("MinOf") { + checkEvaluation(MinOf(1, 2), 1) + checkEvaluation(MinOf(2, 1), 1) + checkEvaluation(MinOf(1L, 2L), 1L) + checkEvaluation(MinOf(2L, 1L), 1L) + + checkEvaluation(MinOf(Literal.create(null, IntegerType), 1), 1) + checkEvaluation(MinOf(1, Literal.create(null, IntegerType)), 1) + } + test("LIKE literal Regular Expression") { checkEvaluation(Literal.create(null, StringType).like("a"), null) checkEvaluation(Literal.create("a", StringType).like(Literal.create(null, StringType)), null) 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 b510cf033c4a4..b1ef6556de1e9 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 @@ -164,6 +164,17 @@ case class GeneratedAggregate( updateMax :: Nil, currentMax) + case m @ Min(expr) => + val currentMin = AttributeReference("currentMin", expr.dataType, nullable = true)() + val initialValue = Literal.create(null, expr.dataType) + val updateMin = MinOf(currentMin, expr) + + AggregateEvaluation( + currentMin :: Nil, + initialValue :: Nil, + updateMin :: Nil, + currentMin) + case CollectHashSet(Seq(expr)) => val set = AttributeReference("hashSet", new OpenHashSetUDT(expr.dataType), nullable = false)() @@ -188,6 +199,8 @@ case class GeneratedAggregate( initialValue :: Nil, collectSets :: Nil, CountSet(set)) + + case o => sys.error(s"$o can't be codegened.") } val computationSchema = computeFunctions.flatMap(_.schema) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index f0d92ffffcda3..5b99e40c2f491 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -155,7 +155,7 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } def canBeCodeGened(aggs: Seq[AggregateExpression]): Boolean = !aggs.exists { - case _: CombineSum | _: Sum | _: Count | _: Max | _: CombineSetsAndCount => false + case _: CombineSum | _: Sum | _: Count | _: Max | _: Min | _: CombineSetsAndCount => false // The generated set implementation is pretty limited ATM. case CollectHashSet(exprs) if exprs.size == 1 && Seq(IntegerType, LongType).contains(exprs.head.dataType) => false 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 5e453e05e2ac7..73fb791c3ead7 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 @@ -172,6 +172,13 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { testCodeGen( "SELECT max(key) FROM testData3x", Row(100) :: Nil) + // MIN + testCodeGen( + "SELECT value, min(key) FROM testData3x GROUP BY value", + (1 to 100).map(i => Row(i.toString, i))) + testCodeGen( + "SELECT min(key) FROM testData3x", + Row(1) :: Nil) // Some combinations. testCodeGen( """ @@ -179,16 +186,17 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { | value, | sum(key), | max(key), + | min(key), | avg(key), | count(key), | count(distinct key) |FROM testData3x |GROUP BY value """.stripMargin, - (1 to 100).map(i => Row(i.toString, i*3, i, i, 3, 1))) + (1 to 100).map(i => Row(i.toString, i*3, i, i, i, 3, 1))) testCodeGen( - "SELECT max(key), avg(key), count(key), count(distinct key) FROM testData3x", - Row(100, 50.5, 300, 100) :: Nil) + "SELECT max(key), min(key), avg(key), count(key), count(distinct key) FROM testData3x", + Row(100, 1, 50.5, 300, 100) :: Nil) // Aggregate with Code generation handling all null values testCodeGen( "SELECT sum('a'), avg('a'), count(null) FROM testData", From 435b8779df01a7477addecb1023605957bca4e9b Mon Sep 17 00:00:00 2001 From: Nathan Kronenfeld Date: Mon, 13 Apr 2015 18:21:16 -0700 Subject: [PATCH 079/128] [Spark-4848] Allow different Worker configurations in standalone cluster This refixes #3699 with the latest code. This fixes SPARK-4848 I've changed the stand-alone cluster scripts to allow different workers to have different numbers of instances, with both port and web-ui port following allong appropriately. I did this by moving the loop over instances from start-slaves and stop-slaves (on the master) to start-slave and stop-slave (on the worker). Wile I was at it, I changed SPARK_WORKER_PORT to work the same way as SPARK_WORKER_WEBUI_PORT, since the new methods work fine for both. Author: Nathan Kronenfeld Closes #5140 from nkronenfeld/feature/spark-4848 and squashes the following commits: cf5f47e [Nathan Kronenfeld] Merge remote branch 'upstream/master' into feature/spark-4848 044ca6f [Nathan Kronenfeld] Documentation and formatting as requested by by andrewor14 d739640 [Nathan Kronenfeld] Move looping through instances from the master to the workers, so that each worker respects its own number of instances and web-ui port --- sbin/start-slave.sh | 59 +++++++++++++++++++++++++++++++++++++++++--- sbin/start-slaves.sh | 11 +-------- sbin/stop-slave.sh | 43 ++++++++++++++++++++++++++++++++ sbin/stop-slaves.sh | 12 +++------ 4 files changed, 103 insertions(+), 22 deletions(-) create mode 100755 sbin/stop-slave.sh diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index 5a6de11afdd3d..4c919ff76a8f5 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -18,15 +18,68 @@ # # Starts a slave on the machine this script is executed on. +# +# Environment Variables +# +# SPARK_WORKER_INSTANCES The number of worker instances to run on this +# slave. Default is 1. +# SPARK_WORKER_PORT The base port number for the first worker. If set, +# subsequent workers will increment this number. If +# unset, Spark will find a valid port number, but +# with no guarantee of a predictable pattern. +# SPARK_WORKER_WEBUI_PORT The base port for the web interface of the first +# worker. Subsequent workers will increment this +# number. Default is 8081. -usage="Usage: start-slave.sh where is like spark://localhost:7077" +usage="Usage: start-slave.sh where is like spark://localhost:7077" -if [ $# -lt 2 ]; then +if [ $# -lt 1 ]; then echo $usage + echo Called as start-slave.sh $* exit 1 fi sbin="`dirname "$0"`" sbin="`cd "$sbin"; pwd`" -"$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker "$@" +. "$sbin/spark-config.sh" + +. "$SPARK_PREFIX/bin/load-spark-env.sh" + +# First argument should be the master; we need to store it aside because we may +# need to insert arguments between it and the other arguments +MASTER=$1 +shift + +# Determine desired worker port +if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then + SPARK_WORKER_WEBUI_PORT=8081 +fi + +# Start up the appropriate number of workers on this machine. +# quick local function to start a worker +function start_instance { + WORKER_NUM=$1 + shift + + if [ "$SPARK_WORKER_PORT" = "" ]; then + PORT_FLAG= + PORT_NUM= + else + PORT_FLAG="--port" + PORT_NUM=$(( $SPARK_WORKER_PORT + $WORKER_NUM - 1 )) + fi + WEBUI_PORT=$(( $SPARK_WORKER_WEBUI_PORT + $WORKER_NUM - 1 )) + + "$sbin"/spark-daemon.sh start org.apache.spark.deploy.worker.Worker $WORKER_NUM \ + --webui-port "$WEBUI_PORT" $PORT_FLAG $PORT_NUM $MASTER "$@" +} + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + start_instance 1 "$@" +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + start_instance $(( 1 + $i )) "$@" + done +fi + diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index 4356c03657109..24d6268815ed3 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -59,13 +59,4 @@ if [ "$START_TACHYON" == "true" ]; then fi # Launch the slaves -if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" -else - if [ "$SPARK_WORKER_WEBUI_PORT" = "" ]; then - SPARK_WORKER_WEBUI_PORT=8081 - fi - for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" $(( $i + 1 )) --webui-port $(( $SPARK_WORKER_WEBUI_PORT + $i )) "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" - done -fi +"$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" "spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT" diff --git a/sbin/stop-slave.sh b/sbin/stop-slave.sh new file mode 100755 index 0000000000000..3d1da5b254f2a --- /dev/null +++ b/sbin/stop-slave.sh @@ -0,0 +1,43 @@ +#!/usr/bin/env bash + +# +# 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. +# + +# A shell script to stop all workers on a single slave +# +# Environment variables +# +# SPARK_WORKER_INSTANCES The number of worker instances that should be +# running on this slave. Default is 1. + +# Usage: stop-slave.sh +# Stops all slaves on this worker machine + +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" + +. "$sbin/spark-config.sh" + +. "$SPARK_PREFIX/bin/load-spark-env.sh" + +if [ "$SPARK_WORKER_INSTANCES" = "" ]; then + "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker 1 +else + for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do + "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 )) + done +fi diff --git a/sbin/stop-slaves.sh b/sbin/stop-slaves.sh index 7c2201100ef97..54c9bd46803a9 100755 --- a/sbin/stop-slaves.sh +++ b/sbin/stop-slaves.sh @@ -17,8 +17,8 @@ # limitations under the License. # -sbin=`dirname "$0"` -sbin=`cd "$sbin"; pwd` +sbin="`dirname "$0"`" +sbin="`cd "$sbin"; pwd`" . "$sbin/spark-config.sh" @@ -29,10 +29,4 @@ if [ -e "$sbin"/../tachyon/bin/tachyon ]; then "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon killAll tachyon.worker.Worker fi -if [ "$SPARK_WORKER_INSTANCES" = "" ]; then - "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1 -else - for ((i=0; i<$SPARK_WORKER_INSTANCES; i++)); do - "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker $(( $i + 1 )) - done -fi +"$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/stop-slave.sh From 3782e1f2bec07b5ffbc8503e40591e96fce67256 Mon Sep 17 00:00:00 2001 From: Fei Wang Date: Mon, 13 Apr 2015 18:23:35 -0700 Subject: [PATCH 080/128] [SQL] [Minor] Fix for SqlApp.scala SqlApp.scala is out of date. Author: Fei Wang Closes #5485 from scwf/patch-1 and squashes the following commits: 6f731c2 [Fei Wang] SqlApp.scala compile error --- dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala index d888de929fdda..cc86ef45858c9 100644 --- a/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala +++ b/dev/audit-release/sbt_app_sql/src/main/scala/SqlApp.scala @@ -36,8 +36,10 @@ object SparkSqlExample { val sc = new SparkContext(conf) val sqlContext = new SQLContext(sc) + import sqlContext.implicits._ import sqlContext._ - val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)) + + val people = sc.makeRDD(1 to 100, 10).map(x => Person(s"Name$x", x)).toDF() people.registerTempTable("people") val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") val teenagerNames = teenagers.map(t => "Name: " + t(0)).collect() From b45059d0d7809a986ba07a447deb71f11ec6afe4 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Mon, 13 Apr 2015 18:26:00 -0700 Subject: [PATCH 081/128] [SPARK-5794] [SQL] fix add jar Author: Daoyuan Wang Closes #4586 from adrian-wang/addjar and squashes the following commits: efdd602 [Daoyuan Wang] move jar to another place 6c707e8 [Daoyuan Wang] restrict hive version for test 32c4fb8 [Daoyuan Wang] fix style and add a test 9957d87 [Daoyuan Wang] use sessionstate classloader in makeRDDforTable 0810e71 [Daoyuan Wang] remove variable substitution 1898309 [Daoyuan Wang] fix classnotfound 95a40da [Daoyuan Wang] support env argus in add jar, and set add jar ret to 0 --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 5 +++-- .../apache/spark/sql/hive/TableReader.scala | 5 ++++- .../spark/sql/hive/execution/commands.scala | 2 +- .../resources/hive-hcatalog-core-0.13.1.jar | Bin 0 -> 468533 bytes .../sql/hive/execution/HiveQuerySuite.scala | 15 +++++++++++++++ 5 files changed, 23 insertions(+), 4 deletions(-) create mode 100644 sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar 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 6272cdedb3e48..62c061bef690a 100644 --- 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 @@ -33,7 +33,7 @@ import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.exec.Utilities -import org.apache.hadoop.hive.ql.processors.{SetProcessor, CommandProcessor, CommandProcessorFactory} +import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, SetProcessor, CommandProcessor, CommandProcessorFactory} import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.shims.ShimLoader import org.apache.thrift.transport.TSocket @@ -264,7 +264,8 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { val proc: CommandProcessor = HiveShim.getCommandProcessor(Array(tokens(0)), hconf) if (proc != null) { - if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor]) { + if (proc.isInstanceOf[Driver] || proc.isInstanceOf[SetProcessor] || + proc.isInstanceOf[AddResourceProcessor]) { val driver = new SparkSQLDriver driver.init() diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index d35291543c9f9..e556c74ffb015 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -35,6 +35,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types.DateUtils +import org.apache.spark.util.Utils /** * A trait for subclasses that handle table scans. @@ -76,7 +77,9 @@ class HadoopTableReader( override def makeRDDForTable(hiveTable: HiveTable): RDD[Row] = makeRDDForTable( hiveTable, - relation.tableDesc.getDeserializerClass.asInstanceOf[Class[Deserializer]], + Class.forName( + relation.tableDesc.getSerdeClassName, true, sc.sessionState.getConf.getClassLoader) + .asInstanceOf[Class[Deserializer]], filterOpt = None) /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 99dc58646ddd6..902a12785e3e9 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -80,7 +80,7 @@ case class AddJar(path: String) extends RunnableCommand { val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD JAR $path") hiveContext.sparkContext.addJar(path) - Seq.empty[Row] + Seq(Row(0)) } } diff --git a/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar b/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar new file mode 100644 index 0000000000000000000000000000000000000000..37af9aafad8a41d3007b63a610b1af37c324544a GIT binary patch literal 468533 zcmb5V1DIt`l0IDSvTfV8ZQHkO+qP{RUG74cZQHKuGP>AR|DN5M{l1yq>EG^s&U5kH zc;iGy<{KFqc~VgZ6bu^ZZ!ex2Q?dW}^7l8W-=A_~szP*9^5P7Ne;0!U;{GkhtKaH6 z|NHN+z(7E#|9>$#A$cirF%?yMIq}ehDXRfS#E@%{?*rWdA##PlTgW0((1C)&bw-KH z4HdLlTJrO4^%oCcjHT;Xnpg-r#q6!mlR1?yY;T~YwH#VwIvOyd{O_jcY)|(_c59+S zB4adtsWsgyaDwRcN1a=-Yh;r1CUjifha%kXnU)z$9KEJ+-r|~TYll<54EB*rKrgrB zE~AmV?7^xmE-DJYx}BsKlY z6aa~cRYKg?+8-nwDIXKxiT-T&gr8c&_=gXGfRd{I*;Qb_ePHio{$F1D?-DS7moRcL z0$7;-K@jnu1uZPy{v?g{KS)~uj9iRt?Ehp9-k)Fq8_Pe~NB>t)6;oRyI~PlUke!i@ zr?aK=A9UgWNtem*R{Q}=@t0T^CnGy&GgGHOD53k`D{*!)a`{tF(EjgLS=u_-{J|)) zztrUXdkES7$vFgnQekiJW%;L;5dH}au(!3fxBG+jlz&1yyVyG!nVVV|*_qhH<8^oV^Nc9KQ z|AB)48bJOLJN|-w#Q)~*{{j175bs|{$v@rpzwm#>xqpoT|Ff3Apc~Eii^Q1Y+S!#|+^ zoraMAF(Vs|gi$DB00F@;0|8P0i+}$%js6!G|Hdpy5hE878%uhCjghl+iw=~J>e78~?K5^-~K6QO3SY3m=rTomEnenjvI};c;P7LA` zaCK?%*vT5t7Q_aWz1V>@DRa`n40n5g)S9YCC%hG$uJsYPF7b$3v{7y>wydsg2_BE& zJ~@8qJK^N)qb=fi^<<9mw|(R7L0gEnw+dO0tw|l(tAr;qa`*8!Wlx*bGh1?dju+-+ z_Y9wQshMotr1o6$@FP5?HTG{4hjz){vT?5!VeVL68xqwaPBGqaac_ja{U#G5zFiXh zxCEYDRQXh%C34#|!q>^|G_u z4vQ%XSfAMmE{_o=4oJxTRRh{4MlSoWMnpdeFt~qPUOwiI<8HpD12o+`K1V zU?`vc1wZ&Y)wG4bBkDR?qNgwac8;xa7yF}z_lo1$nZJV`l#8_Z$9GL-lHxBW&GUy} zhVt>5UyKM4Zj6Gz_&(rwuQ#0^tC9N4`iauMR*Z>2_-{)P95P_OXT-xt9TB8IVSqoA zCHK(M@*CZ0__$#BFCCIV2w2{!@PA)|WFyyKAjt8DKijkNi`^iui~-)amktqWpd+H7-q9+cSwfLxo|@Py zqv*hhcc4V;7gyWUY_`(#rBow`HfGVVr^GhT(r<0C)UdFxphgEFu%-HPu{e4Kn`ItQ zFa8x3Zm38jOnFYUYcJ~S76mAIH4}bY?yDwcW^;q1@Ahz^B=KDOngB_4#5XRqSTdQB zGA)t|YqO%rs;OI(j(Uc@k_$-@-kgYldfma7&_ya_J6J9$9t`JY$T7CLjIAWeTB(wQ zMBSJqq?qAl-ETC9E+UPU;H-lsNf}j`Di#hPd~IuEq!ulu!c;ntVJNqC2ev8YRUR5k z+%e}&h~Zxk91v`8L2x~?IW8?Mv^G}fdre4L7si<0Wmhey3?bL}=MqV(xH6IP3b6OK zC6mnxU1&GGVniHcL&5pqpm}nP62~r(W5c}~Hn+@qz8lt8Y@7kIfMP@3Il^C?@1?N{ z%3bbyBu_qZ5FdvGO3z%1VwM(c{DA<1w%EH$7EzN9$8t zaJMN<{)1mwp`WRj44IspQ!KToO#tI=!j&PEBU@jzca6P_?wAuXD&F40f?LXF?P#20 zHanct@#L3sAhomrH}RQK4X&8esSR>cT7j5$A%!T8o!uNod~1QYbs2ag(Kx0R8t7bZ z(lt2^L^X`k+C|`y-bp-}AQZFpG7&NpkDTOO0bi4D`mH5X z9yikHVC3NQpe&mk1wm(0*g}1SG$J~v;L-Vpm~IO8i&(l=#V}>(gSG6apZy@6wk8bv zaBKa-j<-@3WGuHjuDZCL$(0G9-*3I>x|3yFC|K1L2K%JHe%oEN^k@;Uonx0g<}s{2 z2>Hn&mQ0n)(i|Ce;UPMTE~xETbUP%IR+LrbQ$%QmCVo&kHKxcR+tGEQN@YAKS$n2x zzztsHR-q9dEgkdvc6*<3xsjeN$1z@pC(l|sl+%HMB>)0$CrTb+MJ`=BoP|i%CJbouQ3Z{Yeoc*(o4*D_vp_)2JLqRzomWuTN?DwY=hvZ2X6FF1OiiT z+Za_y39V#@ejqTq4}>22ctFICAvIDn>TPtGu?&^kAnX0%tvO|S>E?Lwkg^<|DJ6;( zl4a1EEZ#+!&#hs4@+ls$lz3Z;MrPVwrLl~b6$qt^r2*ZTUJk5ygX+N6mOOFnH_f7* zzPb<;*~its(dnub)hffr8F-1#A~(^rMSH6XTYE_Iq4Do-xLTE7tPV-`4h^N`RZ_Aj zZ_MD%pP`ZnmNx=ghk42J>wPix;B2}mGo!p+0hYs3OA!y7pfBDRn1|VQcnp*Th7f~Z zuFZXyi>ME>07PvZ1#@m9R+6;4>PlJXktH{6-eB`0--bn2HRCz^(>8PkfxsnusH#*lc?umV_ z7^^Fk1^W4)P0#df14Zd)Xwh-V#Kyx?ItI^90NfXyH(|xri=-E^K4zJz6xv}JCt;3q z4m0X%g%SYLcZlT(k#Z83L1pDDB*+4j3avEgswOzL4P|Flstyd9DYbiAHUlFkli+vt zQRSnm7|XlcLbJ+H5Q?wrfJ$h!$Ub%vx<=JhZJSEx#r8Dr_;_ubsD}>}ZzQH?Er;-a zy@*NrhDBw%&Kc#2)32q|8W@^Z?NzO+vAxl)WElIIDMdchtC>t2O{6)b$O={mH|eG7 z6)o7(s?5G+g>Y)tyPjV13NmnJ))691nGfvaLDV1?gW^gS7#b zEiRbS+9*NW#C^VVstU9Ct$a>v+@TOj)o?A_$~dYh3!b0hez1>Cc$b=tWWO|9N@SN} zP_bp#DFqS_Z`nu|#3P#NUP5U&AXkXGR!&#trTDvbDZYdDN%L@e59KvqUKnukJeL`o zV(<^Cy$M(13)Vr~!Ku4}mH{9GE6h0# zY;+pCY^qfn>i46f#xK958MFWeD9hL;74;#>v{br1n?Tq)9=&WJBQH@&x2IUT0d(Cp`{rB+UPx+^@ltA9t8CNO>kwJewXDKBaK%_O@kCo) zgW4;*IC2+X$~pE{Nl?6vYVs6$GpNwGIyI?FTA7xD=Wm_M2~!2XJKRZKsv4Ea2%dux z^AXV|2W7~wc{a__=Q<<|1-}i--I0sP+hOC$r>`xX9v;kS05DQx*I4c^R0x8@SroQC z9ajq<%`W-mjSY?|IqiLNiKi#$gQYbfW|Q*H{|zl z`m((}7FW2~VB~!@*lK&JS=V5OJqa;3H!r#cBI^X@H*ULrWO!Vi6;US2=Q?y@X4X>D z_)xbu-&humy5VjahzTvIEmJZU^EeL#+^M%3cDb{r9`w+iR$A$1$0LGcB*(Fci)TzOMSp35wvg?F#*#=zm7cq5F1{ap zme(ro#87t_>lmhkm04anIb_j_2g|K!4ZaseJYrnR#ZBuM)#&J> z93?WA-KDjHW4I%A9jRGN40lpe3-$@gD4-q{x>1hmBO`p*VPg8X4sVLi+OZ0Hx; z&?$#|kbUPaw5hmk@ju0U9qEq4$$?cr@rt{}qn`xe4 z+X*$ANc3Ed?76FL9>>ld4PzjbLmtl;8|4kixQad6F*ailpn_D$o`*qH>6h4wXwu~S z?_qOn8#G^ee4%Z{nzT9Ouerzg-1R}%mXi|*ICznA28U>l^Qd*f4g4Ub#|g9D5s7V+ z`Zq$}pd13GIQf>sSk$X2G4n?5)BEbiR0CR^zT)Z|u>Z7`I9?>hz~5(=Y#Hd`W-Y$o z;B6c)!2<&I;ayQs95~&*l%~@oVP%}36Pp~_3Sm0|=?o%Pu84`7JlVC;9bVafStvrnBcq%qrzi>!L4g$g z=ivbBkNA>@XMNhHGQ#w?*&yo&yOzjC&ve6y++EaClteSKv(Sqy%M6nzG1*E ziGue(BBVRdut88>0;0PbmCr3MpI%bYdr>(#t0eQP;_yKD!LA3{n$xk=|`Brvb*z&2lr!XEToWii~3a*wG7e8p=L| z1zm}YO^GJ;=BO*$yHbBl?p3Qm-LUt&R^t0WzIKgPteUHdGN@c(3v2jtV82&Y zAzZ-%h_qaDVpb*IjQqN&-lDjp4_9+0e876kqR`WPX++KCgKgC8wIxUA*dF2KiJA*s z+rowikKRQ3!iFct7Hr)17*t2#)0~`7!1l<>LhU>KE8>pymlwB=*z1yw7yXW4f+C(5 zrjF?RlFk>N2PnTZ{wao8zIW%2bbpDb`-ulGzf|~JCTvA|?iyNbWtG!=YC(g*=J?9Y zhi_G-3{?<5^~DU8j}N9xG71%Z4<#K*dCK->^=Gr)2+G+W0`dmplJ+6|_~jImff6vq z^$AeY_e`(US_MSKBpE`z3&{_t%Tr1lf@eQ8H!6?k79QYA;yF2A$bDh5+!nzsj&urM z5e4LS3e88SHayvV)4L_`H7(HjWOgV(kp*M~fbyW)^UF>c?}XQR&OBp$5+220iNuE< zNG?`Il2@=F0Nmo%l2{KY!TN;kr{#4kmeE2C*bgl4?lIk>W@b)cX0tL7zcvdDAYU0M zCf^WplQTJ`7ap3x6eg-zPBl8=UR|;7j((?wYqXvSC60T({7K3uBq3niy0?(FrreVE zjH|;BV(Z(=`Y(`r&DI z3ABeVuOz6Em(9{r@gJi85G?*<#|N+<27wIHROGHHJ`ley*|Z@+7#3)-x;?2zF$GsO zn)?Z?=%21}3mjyQ028`m6PsjL$Sb)nTBeb2n^p?F2l22;z6oaL!S%w%Gc1yN$lg3G z4J(@pFn485yHP^t{E~Toq#;7*iMe&I&YulD(_IKUKLUblDo`ANz!y4=t_WfE;Z^Db z6LE+J++LyjPr%R1@|9xri@+S_zowp=r-ZP3~Gk;XN3=;R^(TSU_37~sc zMomL_#}yVbiVJ~Z<9lLqn_$+w9fo(r2w>!bV&X>H6O{1D1s^UJQId0{ys6y$C~{`1 za|74?VC4=DW(7q2?1z0D7dz8{o%NGNm`w$B+2!Buk~+xY6V-4PKCtJP)i^o$E_M@k zimm8T{Q$jsY70bnpW*>OKN#R0P0JRVIYb5V{#0v%G8oSQtI_-w}%c(dk|GpT9X%7RI8x>V6 zN4)x=DP;HSZKB6@25Gb6#3b^Jtw~D81YwSIw|fVN$HZkbB*R;n(AKXqv4O7$$QkB)NdGnSeeSqwCbK_uM~~ zF?3RPBVEW8%Bnq~^+x@(xHUJgAlR-_JV{X0 z$sIm-H7)iw2;_Um_z3!PahEfEt5VFl=7-e&yzuNjL%8LQGer~fqv!e={Au^vT)uI) ziQ58Ds2oP%ybY zZoEBj`e13X&)Nc^zLd3IVy7LL&jiIcNoUr#g;X7WFXAeRY zmNpmnEU_7{Mq=hiza^pB@m-fqCsSffv8vt8dlXhQ#!Lp>H60F@!t`vo7%m1@d9TRp zE|*tcjktBY;Cs}>Tp>~}aFCd8;NWTLr4Q(@>4Q*C@=sf}U4P(z?3Dj!UH-m@WwZ<) z2uOtp2uS3w>hk|>?@GkU^!LuIs*$mcsieKV^?w%ZYqelJaF<*^;#=0&Mphe-FsyrN z#am6a=GP%`x|kufk`ir*pk$a=fpc!JG_kunn=Ynbm&h$uBysMnkS&;O@rUG=j_20l zN73y9?!M~0Le8>+A!8#awOUGBKw9De96otS*0epzCi3t1a^JY0a-3&gX1JcBPPV-- z9D!&|%ptQK31b9?(pld#!vcXmc)D)v;2cw@~MPqg4hM83gNfIM#? zvhQBA@?r?1egS#*Uit{u_Mr{%R}J=uKFIdacl0C*!ML3P_K_Lrv3Pa^VxVf>nY(d< zWa=u~>m;Hbc@p+O`>fh~Bhn29wYte#e*$%#CaBu`aehzo_~Hop8RoxoLtu4-Qu^dy zwXMMRt^pbAK^nqw(*(4wdg&j2>)eGl%y5$u);3@<#K86*Z}q8h`63{8Uri{#pNlBJ zw?();aE9mwu8RgWnv#$bsgUd>_a0OLuu;gw6JJMFZQU`wU^b>AS-|+==A>R$FSKg_ zFr_uGEvWGtGYhOAlTT_c+8WQm3w>~7St-V>m~!ivv}Jcn!$wee&GDmQ_*S?z{Wug; zlX%Jj__a*$ha3hdx#-p;HXZ7iYEE>l5d{oOUQX$}XsS%J;>Jr<+g_KT7urm_q(3G8 z09_J$k14S==WBH@H`XFa=K$2=qBU%pAEQ>2^BiP&=Y-ueTU=b7?MmUK$z~}NF4_)( zJyrLc^WV+Jw6x&>I2Zm)`c^2$yR%&*px_@IgmGA?pi!$AAa3;s7UG-=NTRYc4)X zPx7X(6iYrLZT6_&!`mtB(j*>rQO;#0J$0VOSc9Y(J$s6B2Uv_WXti}?Z;{(EJ`0jc zxK`6cjAy5uFt_QYGs5>})AmNqLaC}EDNU4IQ|aW&W}UXBbfz5PIQ5>-mAG%w{3XIR z6&v)2$j4qXM1bXtM&=7$xE?C_sve4vV+U#$`E`s)`|F zPwBIGv05ZrJJ0pX#*R8BHI(amZG+C8*<^8QPR2A5er>z4i&!Hs9Z72uYO;M4s&<@C zFnZdtK2`n7Eq3B&DW%ttLQ^TBhp{BKYLfh27O9;;t~6R-G_71bSEhJ8c=OtPsHqV< zdiUhOcZnQh?%Ql|Y^>`&uRK4^JOa-RF9j@vxRo{5L-`u97IojtR+jge8Q^KQbC8@sJw|P z8Gg!qDx_sV+T&a)20F3AeCYKQy%SLm<4!&$(<42VwYWG(6$j7OF{{B(*ud6GIcGyW zeQ%YN-349k+{0brq%7^&0`P=66e+Uc^g4wray+?aW1ljCRQcF382Vy2=b0qs?A$p{ z(NUB5gQCQ(Nubic#koYEhJ%sf+^Luq+AK%TuvWY~-%>5zbsR>nQ8wMoa?bQS_I3$> z?YNw6lwq);S)_2qgt*uWJ;#svLPHELjY_;1&Gfwo7MvgRRC+bM18814JsPI2?PBaq zra$KC__7}2BB<#_J8aZ%8GUJ!T-((wdN)dx(&_qXU-;Nh;0bMGi!!f>jstIWH}n_b9n za_@Z9SlX^coiEWxu5ra0@fEo9M8wQQ6Rq9|3pLMCrB5SCO=SnjPqPB}-W7M*PFT9{ zjx`TZ#!%<83sYliG9R{ESMooDoPLgJ>|@nDTs637(Qc=!rz>yhk?w6ertwx+Yk*&W zCA+AQRp;REx^!8e{uV%9x5(DFC)U*1d6c%D7FU+`XnRSwvCiG$<&C)~mYn;Y@5*LK zEAUEfU1g+cs^&aNqE7X-17X3{JJT#e6~Cw|<@}^u+t*WuPn5nA9|NBV9ISp4YFT!h zM@L2yM$^3CLjEyno? zO+}jKZ7@S^U-LGZA{2Yd=rOfW_j{(yif)^PCm&DRN70R@0;#Uw-CD=yO}OLL&yYPR zI3-r;i`k5+&X%`dfpQIvS$os}0JKhYeqUY0r))8GeSQ%Ed zem?(PNwI^YkF{!30$rxD6A@d&1v^&xlGu>QG9UDVyj-?YvKD6~gq}Ag9#ynz(VWGZ zq=kDUcV)e`-Bbr)uV8&r)><3OXPb++q0NoG($dCyf6SL>FU?X$u9m#EJ2b|Xg>RVTPLH$20O?95UG9v`pVk@p{VWzfhJooQJNS4Mph1 z&)zKMU=qn&8wtIF`OP3^5veABDj;y<;6& z5{USN7ao+z+;LHo7Ld+h`4XO&eZW2p_{y{-g(MkoqNJQYvyH{yL=zE+gcZX{_J}4? zE=I(cI#q8->7#!j`68xkOcM;EArO^i;EK3RAn9tg8zO2{4pFzJ!i0ZGZ^9cS*0^5? zhe)^yfir5jXf(BC)%kh`zxouCNB+ED>pf1pQn0SRq-F%n5v5NOTe zyG>3Y2&2v(iu!|op5Qo#Y(7HUgBV8|&q*-9>Wv{Dmm6{$eN(uD@8m)#84jmIDQ(%@ z3J|!3zC~tfL8?TLIHt?E6U;OxpFjbMeW0mIkOr-!XL`j2!KWP*EA`Rcs|5iJ5*<{t zhqi}%vw8PNYN+1^!f7gx=^M>#*N^_qQ=Wwqc9^T=`pIBeytlog@ zY=<}Du=zcpRoHw^S&Ja8p6TFvYWD_m5aD_T{F^D{I-{NYsJu*Hx?DGr{4ku^HjDSR z-De$Ebt{1y+_(9jQSXQ=p@U*F83JCtBI{kc##YQ>byk91{^!U@HXTF)aHyNl@ibEeIJ zj`2PQ>`dDh;u~q162y&?)p0p0nkE6@+EkSCL;9oCN^ zKP-P!@@wOg3YD{XZ{*|MtAvKdp_uOr3;{oK638Jv^lOrh&VJ_qn+h8ffZq2s~ES5{8{h zo4`b_1R6mk2u)=dLCDm=j+ZvZY_zsE2_I4-XX{&nz_z}uS=_p;VJX*aoC@ZxeaT>} z@L7Vd^VaU3K6N+RXu?Gi-o&!=viBHeTcALLy^3tq_XVBl)IrDZ(Q@51u2i^l$CY&`5e~R-zsr@-Nvm` z-Ij{ZgaYG=9$`{C;|}1mDw(i*%x;}PTw#1&$+@oUKg66j?hG5sYAT3UEUaZ^*5LW1 zQXR8W*0ncYC=f65qO_$fjg6@Yz5<{YvK`u&C#>y8i{_*yPVWPaU0olUf)seJG6eN& z99fJU?vSQ!QkiDdz^-V)V9VbwY(a3G-R6I2)tzxMBl|9=^wyzGvDz2alv1r{Nmy;( zO(GrVHxPn4m0B-&d~3)^oIsF719?HFuh!*`C%zow5*whw_N$fNs>Y@_ttLC*UQv=r zNvE$9l0@rh5(X6>R`o=$>TM4j(J-ic)$^W@(_yM^p)`s-+7@=t`0-SbD=F`xjFkM` zSH$==j~ivXi#^W?Boe1X4n&#>gBE^W9HS6!0#Kld^DG_)9-~G-eyX4xE!fSh()+Wz zu`hwD!~9JFo!8{!e{is1S8OMtkeD$cr2?g9ckp=px1;R_86PpS~O1HW_lPD0se z9w#KDzcs~)vsyljju`P0JIq+okfWR*14QUwsp9$Yn!Zur^iG+TEutjHTi-EJI2hVy zIiUBZzArN6-08A(i0a}ugScZ@$g3xp>LvDyh~eMX^ulb2ySaxhW13m>kCZ9TCxf1G z*dsMIzj_+&Rd)81nO2r`MVMT1pMTw*X7h7Kp@8(za_th%VLSxmVc*Njw3_n?HeE8Y zq^9~R$L~R~d#3M}yXcJqHE|h}O$CCwI={fP=-YKL*Ow;}|A7Ak^MVhRYa8!ub!TNO z^@LT=4-aHRJp!m2kK_A}k)31NOTNC~$0zR%|+LDcMCNfVsI`uM!^1=+5>)@+e*5|TKlGSbH#}I2TD5-na zH#5XHR6a?!ND}YTI%qq0pPtxmMtD~fbK2>1>ovzwt|pCI#8gWh=%%{jFR*9jBiO&ID&bTt6Ip);~et*Le_x zy9O9yed->0~HP7J9^13(o{=(F)FO6pi(D`Uau?K zL+GO3LD=X@3+Md{h&NYpsdI-q&gwtdViQt6dGDWLmI>g%>xzEfV;NWUG4Tr7-0{W0 zo#PFN7bUTC3VMU=2(fifBvP`^S$I1HEl2DTim#omaRzakDlam}9Y`uh2P2Vk9iU3+ z#d(aoQD96Lw|1!H%^6&BXZ z3=F)Bb&Gh`-orMUSN+ybj33 z-lm`nj3d9qHu(I^LM#L0nEvGcW=e1qe{>VR1eTMqQalT96K8l@-wPOkBlcddit4CS z-RyNm8)SjfmMRk}9Av8L8(pj%;F~XBux)gfPhywMVRwJCU+Uf)ck%&TXnpPh^HThQ zw-rW#DW9_B>sQSSN^kXFZ%lsm+|i`<^QYP zw`l6P<1C?l#L;=$cp_5}xe;3kg{F+IIO0MQQb6dqohu%6G$3Qshc>tn%h66nWGW$N zqt8287h*^jJ}E;Zcv=vgPT7rXmn!f+P&Aw#@vt-^52e{Ks92v(&hfup{JONicEjY{B>>3>aX|0ci$WftFCspW>6Q~q6f^=j?d4E)mLEW2;*BilcA&^pPqKI0$Ku5a zjtrGINmO}N&g9&Q^MM7|96t@kL|0l=jOq(3&T6#EQ;Ee9LFj^it{Aj9DC*zrD0i@gcl6167Urtv}7ce<;-$SviCs?td*NNp+*7M# zTy?#{*K>_vYq9s)$1JTc?x&rWiXh^KB+9R(2RSjX0K0Q1^)%m!2CfFSHo>BwAy^g9 z>T?+ly!oJWHht8`V6DMLOLZv`M=1emaK&htJ4{B+)?UqvNjetr<@CR^fxQwK5WYvU zMpb>9WRbQIUmsthxVh;%&z(tnezUY-%f45weJdl&#SGH$T+@+z>(HbZTfjPV28G%Ara&ijfw9A*W+|&!wuJe)*t*uFv zBM?UfsxF384Q`olEb$j?b}8n;opR z-`&Hb_s&zPDWCQCcjA$dF>v~&R)ywn{-_ z&#LpypV4)D$`jO51Ma1)%r>Vt-BZG9K0Dad?lRn5fMnVxTIW66AA`r8N3l4g5Ev|> zJO{MW^bC1lz0$aS z5y)}KMPAHaXPmva7!SI><#40~rPhBldrWh&pz)Hep@`gbxL?W6=V4S- zdPdhUC#lR*aw^RRzm&WdW&I*EynaatWIfrwEQW9H39+8W&&QURiEQBpvo;yiLA-QD z0OB@Ky0{y%6ZvDi3G-NHQO_qI<^4|87-uj23o}{xD$BDfV6Iy&LZERWh@r_ACgkZ3 z^t2EJo&*c?T#TTFQ>#k?aG9ca*%OBzxMsbY9pw`;+U`5&VS;8$fKVgIgMNl%p(pev zFn7x__z^Wd+F;?{%yY+|hkY6s23pzbu>t+?W0ma28_XX-bebk`-|qkQn&DKp?acy`3Ac~IqMEb{EV%_5j3NM)JVZCltNRA+*V$d^$S=DuYV_hFjnl zX2By&Gj9ZucDHbQl4Xv7TvYhj2_qeTVSUAZ^?2zy-jFYFkA|awt2XBeT#;vh$MvDi zO>hoUpU4tz2)^<+P3b zfIp;R*%JGb-3Jh@`eyfKjIYS36I0`m4JD>t+SQ*Y^v;nK;QrJ*n)im~4Ol?UIX2Ye`p08Zg(C36_q9Yi?1)ob4-9odM=llJ)v-g0+s^o;*Y1y?rz2S zksXYil2vRdQQlbBZYC5%mBp`x6B_*@Xb18|i97Qzh4iSbkD$lnHQyPL{HZ=N+=MSn z;7CzjGWb8Y<+{JRRIEGn)`w3CQjXe9A%qi~^%Lb11c>~6{`tlWYF03_fE$rPGZchU z{nJFVN73J((xP6opS9K75<=(Ibzx;{zm;oDn4Sccna4NrJ8#?mTq~kk3Gsy?XIx{e zQdc!IY#qVuFMOuqs;UfC!og=is&Al(@WwdX`$596P!!xb;X_h7N#)2(ljYq z)pz0H9jtq{=j@k~)V#H?rCC_X^%dAHKE4nm>bBE+W>EPffX zLoL2#XJQCoSRr5)yO}_WVm8lOY%xg`!V$!}P423~#UuG%*!OHK#w>xS(@c_SAUm#a zlxNBp43J{KijbiQ$uy&&#Z(E+P3#LeyGGmBh`08PVN(mFKDK92Bn53vrXT>M?TbM< zPBR|^{m@KniOUF4Jv@&CCA$~LgrR1FGRV)YARf*YwGHB8Vqd5kiMt;T4gCUEN3up6 zco=)hR9R>SBE%<$yQ?dLA6lnu5h*N#)}dW!3QebB5h}d0I<(UQ?JG&dpivIZbWcq$ zhNeBmRx#BsUf6QcA2%DYx3cmj1ZDHAs(h$l8m{xJ&Vbx%r4rl{b+ddicNIlVo~Ea| z`6)k3nvdA(@>e__RI6DB7Oz#3YY8pf)mDCO1sq5XGcD)aZJc-(ehXqd6tk8vv2RCi zo{T1G5)l1+2}b8z(Q8tRlA5fAQ4)pZvF!7%A(_QZhaG`}25oi@UR$_}2oBGOLgQ1j z`h2H!D;1gSkrL73wTt9o-TK9D57Fvh_NpS$m9SrDXWC6ijK#ynbDJ{@i|62)#R_rQ z)}=RC*X4OwJEVBs=ILYIlH-NSlrUbf!m(Xda7g428n4Yly;#pn19mry=k(738CHu7 zUO}kFy;i7YzA@Udutex{qb0zxKks!d65tt=*tjLbF{iP8GwhU3)no07;~87oI&%4| z=0;B8`|qAo!&p01VDB9}WQse_@I(0v=e`%6B)br-&{#IIUE>QsivBFmc=?6x8->5i zS3Vcq04zWU!CNzj)X3^3`QRIduYQUY%P*6^aC+Y_U{D}V^e(pDjO`m4NA+MDi(fi| z`Whu3zkt%^%7!~g6dQIb727SXa4WxjYH3AMll^WP`PJ`EdigeN60-k+E4BC;48;;1 zxwz5%ST3L6C~kf6z)X_fUc%mto{2n9JYdLE<^!wP0diZ z+EgbW8CvE3@Hsj$>#WJa*vW1NgaFXUg<-ascRw=$NZm!xUO=S_g#SD`CI&8}W^XF^r${Qr{nr zE4|n1d2^bY9lB=~K3~z+N_x-MR%Qeoo#|DO<-4rOjIq1@#4Z$iLnY^QCfVx_ShYJV z%l(O}Hy)is9z|b0>T&JTa6jqsCTepldP`!LfvwZFxZ<{4SAA)5I$9JapLlX>*7PXX zcN4HycZ~yy>E)h{v^tY&5zm=YQ@cxLplc-hgn$WrUqErCr&CE$7`T+)@VX3qPyDJE zX-c%I{hM>-WVGmh*gD+oO?rGE)MZ>6QH;pZLuyz@68VMEA#loS7lL{Diy{>=ofs5e zRQtZ4dHVoAUvC-=u1t7otzKe}IqRTk&0m2vR~gi#*-LodH#0vhEbJ4 zdZOzHY=sIY#*pUf`7%e)j?qVyY{GmSR6S~0*_XA877fYgHdaODCBAH6Oqaq6m!lXv zNQ*GxFvRp?glZ_oC`-{*uFdG}Qof>wKB71EWs=h{Z4*kahhtqv5(slZ%)*-Cc5_v+ z#_SHh8~4)AvXj>g6jkFO5o0}r{(r>1Q*dulx9z)P+qP{xS;0THZQEF}ZQHhOd&Rbs z75ikLeeS9H>Q>$Tc;Dun594LjtkFm7t^KaeVxWKZb~nl5Cc z9K(yavUgXY6~+1DYoT%#u1h$(0T}Grx+#5orq+d}(el%}q*l>7wp;jqjGK)p2SXb% zV|;KLX^JJ#AUk?%;tPH6M6B3i~Zz^(fcf9kWu`zA(I8JWV*%> zku!0+$x^VhK|xNk5_i(F@bDL#^<_=Y;RM>`rqxcInO=HPCxx03m82-))Uc{G$nNfr z>t&r5hXKW+c3fNGacS%hue9WYY7S_(exu7B%j~r_xPU-2CP+s<3z-ARv^MRf`!A&z z2yq9KM<~vU!Zb%{uAow|jm{7CfgN`Hcf6A51lyzsZb=Twbz+OkqHO8japy+w9pbZ} z?}^TxMLeWiRONaLwvsNA&rEoR*zq^-WO&A&X=gyMzNtlS0MFQ>+=K@qNd$nW=t3x| zkLbcEnUC-S1=*)=l&k`&i;C zkK!vErAn9b%t`y=aJHqCnTssyD}!)k?UAzh2GgNx>6x8CNWdcHs-~hlvY1)g!7Gei z*OQVpZnYqgzzU=1-vy?n`NgtQlvP27f_U8#mj@QsMTWZY`Gs@)T~BfN7|wt8OCvfS zuv*A&11XGSUdXbC zYfsTWVz~553HCUqUbWt!w)0kQy*%;k7PQSw?4^L} z)Wvn3WguP28}pCyczxDGL3<|m*vo0FeK1YS1zHVpvt5BUY+uSQhexh$E87hhO@Y>U zWI}z1EP;t3g1WyB~;~36Bwku17 zRoCnFhf-V0E#yGKBhf(Qt!MLjnqhwnbA-w^@IQ3bPMA$xN;UC%@`&c`0U}h7NZCT` z%Fa_#%h@&u8tleJQv4tAV%tdjSG&KZ49Kt0?m|}o^AIN*^ zt&A;Ow3b#ehwfc-xrYWkJl*MPxPt|${&^4N%x*9T4)j(2F8}nF+g&hrB_UJ}=#l|W zC4;ClFvMkf9GAS`*FysKA-}ld#pw?dBbcd&w5bPxsYjqQB9fISJdDuU6L6|oaXQOj zyPdHDSRs(8NAXxOp69)BOK9wq(092i`dZ7O&C0QNO2yoxdEG-*^@D1N3algxmz8C# zUNr`LU915hF7kLbC7$bI7iaA2f^JGATL2kn0hi?{=!S}Lk3H*BWs>BV zvUvM@6!yn5bD{x$CgTmHcc_71La+6!6W1+mp}PBy^aGZzLZLHFeEvP$%G+>7rv z6`u%C{?^RfzQa$Ud4bWIZ2euQ+@J2|um*Wd<38`}%M%YbvD)ls0r7V8$B8Gf4~T)w zUSR0kFGp6~YSRz+Kb4WV!Z5|gANPaE4%}UCXs>$DEF-xyCu+GUemIm;1^5iEuf=Ss#?`Q zs`ytd1^fk#tbPpEY_{}KeZT*nBM8{(&WuB3w=zupFugw5bn3b9GW++ooT>*_2W*F< z9uwtl%y!7i2|f|OQokeks|E<62Nw?8d<2M#n{oyIjMI9TA9voN*T3*3&ELpfa5T_y zU^vv7*LoMpd$Z4hmzp3J%N>^j)*CpN^_D6tDOS$Pomj#)cm7TTE|+EQ4+UA_EY1sX zPC{u<*`5Vnsk66ajX>d^21xWv!{%EAu8+#-%HiP!_m4FK4z3i8TCLQ{!Y^9TqsXCD zFmGCN4VVq@ENhD`?W%bTPt`7j#syDyVdAoBvneZ<7W~7%iDGVbukF);#*Pf)@R0Z@T0tN2|t{XX>Oa%y3&Emw%4LsP4rdv|07`FZbA+5~oi(4##@Mm_!D~ zjok1P9SRGtoCD80c8bhq$MWm>`S_rd(4?B_+< z7_-SuQmiHz^Ksy0rXV5=Rbzs#{AR`mTN2T;P1pcJx7yH%ebV^E}9-k zz(>O@Q6gy3W?-i7Exq-n8?KGIE94vu_FBdr%-9w|j=dZtbxlL_RaaV6+*~#Gh-o7v zTPXsBjHP4s{0#IQ#rrWz3j(lI z{?KfwVQ3s-n=%;QP4gE${1)@KT_JXsFDW5*R-C1KVCWF0y?^EpU^^tWG!ra%Q#_e0 zkQ2!+*-H*LPOCSJbWr60)m zFvXMdxNkaUi~`&WHF+8`T@eKHEeUH@9!7Jg- z5^d=(Y`%jpTjuj9<#jhf>FIk+ex43w8!$ZWw=tbsJnWfh1VkmaIZrHwd`v~rfLj-r^Fk{Y*BKnh<1xXi&Fat0R)%-iIgFc&P)iiKiSEH!P zbSBRlfmp;JC|^?%Oi=hxBlmbKF*{c6Pk&e_jSRrP4EJkaGQ(SM-Bhn7WzbA%hw(V7 zh5%<5TD4k}Ljpxa>@stfJ{Qi)*tPQsRNE%Nd*jgpQi?ox;07gfwYV^}SyJstVX~!v zq~#k$4-+i&l#S#*xCew`Sz3FI34>aML ze9Dk|)uN1~1CY9I^;+da%iN#~xGex19~b`)cf=ijFf-qQ)aZqA@1wr#-?YC8S*02D zEf>0WSqrjVnLPZ@HUMg)Ir(g9@NsW-> z3I=qrA*`fjA5ul?tIt5ykxW5e>SUrVP8Gid2Q(G!S;q5b-Ui!DS;QC}TUCBM0uyFM ziQlEcVYJv_#c4_K`bwHcN$4;y^X<6*N^ylQLyQ->|6-3ql%~50^ke+#Ki#iDJu0#I5ROB`=4zcy)fq zOq&mS=B)O8wo~?!=bNvV86<{GBVw;x)#%5Bcu7oeH|fU+N=pX1n_?xOMWHX1Ulmxg z*yHpGpC`^mFZ?eie8w)aFvDqU9J=|*^_pm%-cpgIuR|~hpiAT`xliJOwF*b z10|!FJ%B=-uQ$|*;wf=5Ko_o0$r&U+0EyM~?uE=-lpTUq0gXju1L-t&#>(Y1F^_nh zk_!M$?GEyAxu?+;a#!PMGB_<&dowwh8ON&klE7sM<>9Cs>;4LLw ztHm4?23$L*BYaSE%9s-3EGdj4My3cB-$##u{4bT) z$jsp)jQ;K}iDNhMJ2GAm+gyL=1ATjhSb?Tvn4LAOi<30Qz8ldgRi&rO=ufL5_(6US z5>qTvJ!U2*K_-^T)#h4&O;jen^8v{!Jj~7Luj}AI1o;VD+#!72K?nz2f)PnY-tgFc zjfZ_M(mP=5g3}YXpH z5zUmi2*F7-dQ@kC2LiD65h-&J((P%$A}*>Ntr6u7S(nydv<+1d6bHo$O^3 z^e9tIapp{Q+99Q;as1;cSce2v4RHWRBrN}s!mmov1XaZ+^iGRPm#GpU2Oc_8A*J*e zaMLe{NVW%Ngj(W?28Ayby+V1VqZ7X1EeAhII4Zc0uZu-<>m_ZHIDIJ1^eqszxiPcK z5a+>OLeq%KFe>(h(EU8Q5h1+Z-yo@ROnG6)@zc60_RpXH7Z&{A5lBSO6`a?f>Id+H zC;Xuk{s$rSKhT8#8=xs|;OX&Sy&y#sBRfaq|1N>L)!zTB1p3}4Wy~Cp6QmTPOtJuL zSfkbOFb}?^gaZ{$EFlMqdYm7>HfAwqZk(AS2dzks_^WDD`8dD+M)|Z?XlXtPj!pMi z{rK6^^T_cHCO}YqvYozlX(XHx@Kwj-cFp(fHNzKs!*{>$m*a)shyGEB4waQMAMm`q z_t^#*fOwhO@;=VtUfK<0<_|&uw5bjFAZ;>;jEIwZE0gC#7}NHTzMRtcJRczQBlT05 zut{N<4&Kbfm_D0g% zuhom$Er;wG819+av1mX?kb1%POOM#C3Mw@jdTrs%iNB@Y@sRxuzqt8vk4(>~?DmYW zZ+_X5?L5F1+KLBo-Z*QzZepQLw{MdEE%ykIx0r~J!Cew<1*O!u0Oc1GRH3t>zsm4r zg+<3zK=HdN$;J$sWTY*}-i4yrPI6Uvvu+cX&zfTXhHI_pB+&WDj7w9KLX4(Bht8nG zlKV1gW^T?#{cU=!o@fheE<{=ESMam|}a^*}cHY?idYWPn-^!V{ia$|3Tcu<=rJLkn-kle@&e1WV8g!M4^u3F3l>0=njyu*;-c2M?`q|`+ty{fl)q7E}9lonZ#2(u;#Yk_$W6TiQfcKCeS)1%Ik2Vkm; z)-q+!s%gwy9BNR!aROBhxHr-a5O39k-4H?P_>0iSUXrS+mKP-CAZq)937d&gl@eoG z6a#qS9?YyAVj381Tb&=hbify{(DtmATu+w_k(u|njcmxZi|lrESZH6YgSF!r6l_{3 z1N-aEg%tj=-_>%$>>uV#<}EVtv)_t7UM)9nQqAT7^{&cIq7W4{8V*x9mSU$i3e58` zkQhbb_eUIxJTUj@;^Wx@pSQXcwr)ks^>kJcKc)BtuR~v+3(`l~K%Dxnq>1&mPw4Hi zD`?$5B0~f)9zn_tJvKBNcaR#jTbUmtU+Wzn)Q-w6pLgy|FQTofI>*I2{o>MMh9v0) z8lVbIYiS@VV*zS%PWk{;OxQn~V9wuc+^+V(JTVIh4YyyRhrePs6jR?kXW3=du`xaB z%H28B)LHzXPmJ3r<*GcxVjaa(>d|_6T$38ftVxXQv$*F7TGyCUgS!&D#!@}Uw%GfZ zuTy14@f$uIZY4TzrDaV;x!49rc}QuA^_EIIS;}=G>=|{bq04Ih=n`;$3)AME-@=SX zrv7VY$&908W)K0@3zIMLUMaM1NTDbj5&nZD)l@iaHPOGUrenufstN-zfzevh56ynN zTdfY6<1$5Rcd*{U8?vqhz~yfIQbY6iY#Gmo8FSRC)r z_6okLjsj5SSbA%Y8h8^v7h$!CB}jMm%PA-HDPa4^L{U=$-06;8d< z!d-?unzLHyuaI7v)sPhyQ9Y_qrlnMN{8z&wt`@U*7U5o>@x`Zxn!uPCOg z4*Kf}97>#6gz5{QHd6cJ+2GO{>w@<@shk`5)jx*T{Sfvh7!ApYVaH*hrxYD}nBHFu zXrT>b(A&kg$7`w>>`bTMf$w}%aP@Ux=B}ptL-iCd*%us9n$*Jr%G9m5%N`YF1=ar; z9<{7jZk2vwSFPq1kz;?qe`~2SNxHE)+#&VW9@iC#$!)+t8pW^WFPLa_EE-tWR4QHL zb!-aO&Wa?;WUGOPYj%D4_YC4WC*pTGj)=91ztorW0z4U+B!S^fl#Ic@p<5ISa=VSy zr83!e-anKk?QS`O5ATXy0Zkh_?F}CXT}lU|#0bj?Kauj@@qwz!mzvS$hpS#RK{i!~ z>tY2oyPBayO?J<}fp@;qUv3K`Mdw+%R z@e(WXvBW{(T zZc30SM#5i$44DgXL*4y~#t1AX!!2s1$0g>nJtgtaug9XriaL@Phz_|v*RguJg;i~; z39lYEI0K`t966(YI-LBr&L%#?Hc&%BpZ~_EEb0w+))Swlv^Z`l57LZXZVPWxgVOSe zMAZ(oeTakOh~z+@dH`pqE_>rRC)*tO607k|w6`ayel#MqzY-EYMxSIc$FX%V2AgJ78~+1PQ1|xH%^2g=S;f3~y~6c`jmN zGf?Vx^Wx)#cpUl#W$(Bya;~|gScCEsBTP7#wR1$7LZ>!bI>#c+WY1>#rAiTb7NKG9^ItRCjf#8q_N&%H|Z~MH`V5MWy=IvAb*n z8#_>;QM`ndVcpVNm1J$myy5opoHLxCWhMyFXZgFF8z75j$m(-Tfof3k1Ht5K5NhDe!X$g;?pO>>)9 z?rOV_nWy8=Y*U1?>^A4NWF#A!W{bVLS^K)cU9py*Uj=>asLGP=9M_ak)KY%Whhe zl1tVdyG}y21BWg9)G5YjZoXe1!)G3gqN#mIKLhgt!@ochelv~||D|~2ryQpozUQ2i zuFcPbo6y)_+I>>+9@-Ozu&E{CdUD)B3akKW)gSlSp~x|f#NH(}l!(Kr^g z)VSB&{*@azvU4szcAtM60zCB$tjC`_@}CCawywD(hXjh_x-P)C{MARsq!z` z3UZ&*6Lb*xJeAe*eB+z-_Hn^TCylF+eH&K6+(@6U4D>x_Kqn3_$lEeW-ADu6v>aT2 z*E>gw)dqLm4QE`7&aY9SPrX)@ZWO&N0D;l?4EVknOhRFDCk%-16$ZaGE8IMS)Sdge z^gq%xn=@|O1;MNxTlzAXbJEl-kx#?g@%Pij^^vlbOgi&`? z%wBQfz<2YeO>cV*`2uR;ICgBHj$1Qnjp6Rd7doS+=e&TQ(FC ziv?AJl3N0*1pN(x)>^bWoEc2qv?$T+JMcE>HnKZ9>D<5={%qDxOj}74#$(JcRRmF> zK>gbd3@80(P8`mgAwkUa_1*a(twfdoXZ-2CXJ;z?JKgDcYW!hBU`H325hC|EADhj! zP2Q^X(4b1GJ}N(Fc(Ng@m(g=}D9Xkn@`-Vjk7Y{CYh znVj@c4{X+U&dy$-iE_zeO<_-=1n@-8{G$dy04M-a#gQg_fUM$7vVOz5hcb5=G0nWc zSlfsIah^}j*hL9f&SreN^>Q8_BOg))y6WL*cr%`M7O~MzHZz~-l$yx1yC?e;l1xu$ zpyFj3M7;czzqXisYY9|*Yc;bD*fu(ztayMT&#|bYMkcI_nBsCFMyG-I9h9ZYZ39vc zs4Dke-4F~8t95j-gL+ZgZc)P0sxFN_MWd$0x{(({D89LkA3`{ZrdC!=)IF90-F6k3 z(UcoJ_1kZMWWChCrS1Eg1%qge1Ksm8Q>;Jq1`-W2c|=*O`A6r)bSrZ%sKemsZDZxy zNQdZwBPtT4zlqInv^63=UdZ!p@eyR)AD$hH=4$hmhJhfNb!-06Qx5ttE!=f*wJ_4y z?N$)myQUi&u614E)zsU^+&p+`lP<0XtsplImj-K-4Y4KylRznfw4v(ojkpF}6OK_w zoUWDbxquvJefHGNNG13zQ?9{(K$G+-pR^F_@C~?zTvLugN0gIHDF?I!4c2CB{z||{ z4{bVYqgVSUm)P2Q^tyTUT>dj_z=-zlC{TPFC@YFod}Q**GYYOmkV)HSLTCg~OK>(> z8rEx84UyAq*7fbz^{Z298>PcuugBOE6Iwp8B_NtsPa|%s($BgsaI$bKkQcRz3GRQuHb~-YD0iDnW$fQ3vn3O+SHY=pNz|Prp_L!dVyTa z!!%d9+#fxKt=vF~Z~A(&TUM8jT)4ecvUQ5EJzgW;*c(&xBdNRJv=VmSbYAz|Xf?DL zzf8qez(%*9=hvu`YzqQRGpr4;OBhfrgi}lcmhqt5Eez2GmLeI?;q;5l=;iMSjp#Cj z2|yX9a8_RDJJz}++EdIRZyf^#M1y+9Ds>rA~PPk?e zv-rfkk2^j{4{h?xTC@F!GUY)or$58dp0;N2PxKf#k%9?h8|&b33KKwodo%9K%f|V8 zpcJZ_T$X5x?GEc_q4J-FO8;x2D`WhH4~=7@t5TK?8G4f$mi8#FokK^{(D7L&j5OZB z_PN9KDI;m$oc6IpjVU9R53sB{BVO{p2t(;X=YNWt-uGiCgVCCf2-ryErL_uG6tPAi z?1^^$vTqZ03*sXVS%d=Nv-T+;xIF_ahafNb0&#jHvoFCP*uH}|ha9&QU&Os+w8FtE zt3&QWpOJ=psKtjS1;Vt1ZZNr!2^WD)Qt8Zr-X)6V z%JB?}=gHt5v$)C&lj;)i4q!-kiDjs->Jm$0jcU>-x}si|He?`spuEc)1Sti z6-%6$;l`XbO01Z-;?Dl+BeSO;eugurQ=3>OS!7E?rMHYX$(67$XQ!SONUQyzau#LM zZ4tnSxhuZ6Xjx}V;>8Biz9|_cf>NpxsfoVc{8X7gH&gbJG{Uc45-?Jw`mB) z{FJCV(;}=NX3YrL>9Zxi8G3@<##VOuvedcn^#1d(@Zea7RfG8T>*{Bu%a3*Xf6&MN zU)osOyW5f|>Zsjn%UTNER?8m*??1p#LK^)X)xj-EEp18S0FUsDk~4O)%p}rI#oK5* z@4|wBfHFdA==1&+u8BMGZ^~FV@->Q&>q7u?MXxdc_;uC>so7nhRxr*MqGtC*>sl zQ9eFP_+-M7*!F&!AWXx)&(eoNu(_? zjF$bfB7P?lEUuCaMSzjl+eel~DiLfoFpns5%D@sRu}VKx=g;QLI?#$Ea#BlAZNdd_ zWSWZx?ayvWre92AWO{4+b#w81P&Izf7eD0L1trg8sO$8Nu21&bDae$Gxj;o8bB~Wl z^iLcqK;>YJjnyEHokBwZvl+ljD;GGas75TbdJYmKX#;oKlAti9QAW4Yprw@qf3r?K zhB!7N=#h|VAu>B9Y>V7zYs(GY<&+3r*M3%!u(?@;wP1Zn-h6RfQrXYLfd|FT0W+`+ z**ZTqUk&mM_s1x$Or3?gi{QNm637~Ii5vivoDL>@b0-Z6L)`_XgAz0sJU-68Lp`}x zks?)zE)!0~RtRvN85+;Xf{Tv%yMQNuA`MH0I|`g@aF5nT;O=n8@99}lm#T}ja0k-3 zYh8yA0&VYGXLOTZwOa(0ae)GDC_410epQhC*g$j-VkFFdGd z(q7|4B|GAToY{e#F(N4LJ}cMHEF&0nnI31KXyBl5hY&XcYczpHfwNYXUus=7sx07& zC|R+CZV{x!7}HChn;uIp*J9GvHntS_q5XSGrC6ww=bX@N=bS4ib8+r9i%<(fOtmqA zeRSSQrBJMvaV$LN-3XLo1rC|*iP&h6i#!0S@C$7G!f9f9{;=Km9dr!KOMRL; zh0esGd}}gws!{q59sF~*OW?M>&!pJ~KBWSS*|DU`T5&w4$o3d++0wD~7r^z<8OM%b z?!KaUzoi(k0;lDu*Yf!%w#cRpN!8@RLvRHxvFBES7U-=eUWU0{0Z+}uGBW3W%0W`K z!_lasU~(iEg@j8w!x+23;j)7tIhr=3euLNuxTI}vl_+UUx}e>wtzW@{CRfuS z0Jn`1t}O#}FX{HSf3)`4|3LdI6Fkul_CR-S`}3(gNY6s6k3VdR1$X0r3>k{C?GLGj z?YaG`FLqf*OLGf%H~>2T9qAN#0B+Nq7YJ-q9l(w&q*)nHd7Ri!!B*v9g5TeRscWTB zf@G+cw@*E!UhhiXveZ70Qu#P!sl(Jeq}eu_j5)%;Z;DH{UL$xUEoxO3criZ=$Dh?! ziujDWk1vaR9sG`&_0EJF^EWRtv4~RzwG{eWy#Sv;VPO&A={f$Z0AepcPSi(Yi6Gn@ zPh<%m&mVGL=aPLBvcV%O(zOqu5CAyg(7N<{|;GR9{5LR>$OW&^b?e z7&(2d0cN+N5s*4V{wkxe$KRy1SfjB*26`f%NyPG_kq*{o1wg$zp#VNp zd8;Xr)r)vyNt(5Sm1Wd60<>+zQBWad!D zR96BL6cZ7T&|Csi0W|0jq@>^gatbISqJspvfj%>e8M(FPmfKZ<3lPnxUb6gWJBcRGp8yy= zgnfGLcPlRiIDUZ)?YAapPZ2PBXi+nJl>0aJDqoQ-dY*oyCZJ5 zv9{hAJ5GbhABlb2ZhH{)AC14h4R=Gqzn0-fcg4ZKrr`>26+i@B4|eVd28JlWea}OP zKc?V}ZgUqt+HrdO{7m*!U+h4>4+g$F3~+tPZuC0SpE$sMF9x~?_S_sd%yxY_vLyO2 zgXoE`Wg|k#Aqo>0Ww4Cf#>jfpfgL8N2c5_l#WbV?-Y}D{LfT)-A{}2+-#RGOeBay4 zdOwb#nt^D+vCZcQ&#seG`)#p>#oNrwGGmGg509M|8L{bGk(jl6>R1(DxHN4?jR#Vd z`-BS=Y^jTgE%4~zh8=MZN|3o#b7Gtp)wy+ZM%pcSW$YKBSQT_B+RKy3Cl^sUvL%#$ zBBYrm^UfT}3hkNH<{^l=7IWjas}hZ{70!dE88&mc99d$GI;?6TSrif~%>J+Rn)-9X z9NK*s6MbUB9hx~msLV)~^hq*&O{vm0^K?kd_S`Y)~JNZBy+dMXU=nnJhFsE!GqWm3mdG!op{I>KAP{)wS0{Vz?c?X4vT2aGk z{Fk7=ofaTySWOCqnY=$kqg)r$fw?9@o)_2X-*SXb+;@ez)j~LD^X~a>8FlpMg7MAs zBVjvML8_g@S#>SvOz>A_QYRLT%rWbZq9>@ZB?fX_;qO zPBjsD#aF_sUXMHxx&%2en}$JpOR9-Y!M9KSQ%@;~t$f?Z#Wpr?seMi6rt#ggZR0np z={u&yT5+61A+`&6j$BhWTN;zr)X!|+%0sX7bH}fA1F!G<-kp2P&OsbEh*}}QT9ibe z;)D>qcI_U*HZ>1hCi7G=nz0_rGF=yqY(2Ab7*8bwpD-VsKm{3T2KNM;cn@QTftf(r z+`h&k+nq?}LlV@>d^gt<+T%k)1f7`!O4*SuZDG}3aln|&t~I|s{Un`uF{#sWqK2`E z8o5^ypGGqafr36{r+wky{Q4s`b)^5;#@4DaeN$@@XBmI4JlD8P>smWzXY25vEWBr;N1IG0kkIy zeD>t`mGGi8z*FH`Auw+u{nMM@MFH@9pDytdD3KJ}dWtix`XWwC4775x8f9Sf5{c5J zh?%q>>oB^F<&x;Q>86s;Q%k?GaWpoG%dl{)Kx8u5XN7-@a(1|N)fMemia>%aung0R zk>CX7PT2c^?ggm2xdPf*uXaMxLbzqjoR0<19gq9EGVC3ute}jr^72#u`WV z_h?Ss=AxGK$1H;iJ@u2(7-ASn`BY`}3+-cMAO}75C27$LAzYc!{P-lR#lPU!K_a4w8GKEB7!fN_bWF?ta)@CrF_@?( z@45H=+g%fOC?Qnauyqr@fsi?)=;>+U!d}jcaW#sZ%R51AaZXXt)cF36&uk?^%pD#h z&N5#mpq6_B`MVl0p>U=Rq)Hrh=nF$!pGSE&WnNV8g{?yswf|*0_q%1T0i_n^!@)y; zM9~=i$Yn#_kOK-N`fUB=^?6auhF;a#7DT6G( zAvPeZG!T>{Cp^w#G+l`@i@HIp5gG$w{kK0Yr-@IFyet%ZCte@BS>}S3wDv7osF(dz z?5e`g=x5SFxAhzic=tp*N9mR}9p-~ppl&1U_&HB+5AVRiXa?~f_uoXYd3}0|DwMRz|XB|`01R!qIdIxgf?Q9Dz$U_f2`NGaYQlM~NLD3e8Sowk^omKS7Q7pvOA3P+usST7rgDfUj^y^<;{AF5xP_eIM z_3Sf$mg56nCG*d=lW~JT&>?2t^p(LCqM0ij0<+K>=Uo&%QV`CWbNyjg`l|quBn92Q zmgxA@@Cl*A2JpK&?SeJb5~q(B*0#Y3Xd+JBt7XCJ$TYMnl3 z(6}k2C0BBpv(LcJx;FYfkchOiZ~I(2%FITf6{D3wuw1;=%r#RJ&UIaE4B-@zcCpE3 zmc*aOTzS~t++@n(*2L;NxfgJ&;ih1v_Le9#Mq`j_q@4{zpx)OO2mJPEYTBq#is@*| z8N#jSaL7MnwhzlW99+=vhe`PPtjlkv8rzMT&z>Jr^t+<<)swZSiUj6(c#+;+qCi=} zzv8z|G@5Jo9?X||B~!M8YWHwdqvy%H#r>F1eG>We1aKZO68VdI8=mW+#^VZK6ZZJ` zu`D{E@@7SY;pumz-!frY%XFR=Q92=53g#u2@?vG)cs>KwPU9l90{NNFl-{O9MVU`? z$$ez?><0`MdQSm~UoziJdpxV7TnWihuR$Wdi)^COz3C5nhVd8#%GI+Ob1-ZY1^2*< zE8131KR6Bc6`LcoISz6kfLHKeH%M{P^Qi`zyFa?!6ZT3@S;Hu|92rfqPb>w%O8JKU zyhhZ;?n1hZ$tW<$G&}WDM<3~r?AXs|6RQ|8Lq3o3$M)HFASW9@disT!x_PoAev8Uj z>iYYlSwZ&F43$Pfd!ExiG&8(j#kJuZqU|PXlNcwj7Ppv6SPw`SNsdHGxSZWJkW25G zB6m;I)amvm<(r-X+z;;q+HOrwHM|q7DkZuSPoOxU3&Dp88Dy0jQIUlQ_EbuQ~K4bf&wmp zdsJY#{wn*N>7Zcj3Zu~2hm9b& zIjKBacU}R^HlZ9mVNgNLwyr!}F(ltbIeJ%6Rv%#B#T%`3&_030mabocvLssLc4(4N zE`>JQiE(5SkuRIHQ(nR?DRUq!p<9w!A)V|bNy>l*DNd2yR!ff@W^ob0#=$1Gx;;T% zS;C-sjDj%=mgJn)uafjJbEr6>OhIGXp*a1UGe1?fVTu4IsrF7B#dbm2X$UU%tfcd5 zugELhd;r<6Da9+2Z(5A<+=KxuE_v!7QM_las}$7;hg^~7FuumL8Vn~nDN^-V&X6=^ zMRs)13fQ^ntC1DwB;&I1S&u|hndUS*Q_OTpg+gTe1nV+#bCOcq@Se-wSV&z;$x*MA zPr>(PgRH`&XqnGpi?%Jzp=*UTzj!9dzb-}F(s3I!VFQmM4QbN+u^;ym$gtSoh;uzH@c1Wml)!HSNh!+}#f3Yp`FqFPCp$C+S z^4H6VL-w;;&v(n%$;wia#EA3%^g8X+%$WHbKYqzaIn{H8N@s)EVK11CIo1>58hdWY zwtB+x1hWQleP;QTvC?!d$qF<1Vm9XDKyefit`Te|LMZ7HDbqGP)iKe-g1s zXDyfJcS%y(@xxA}u({6fu`FwZaBnE@B!0Zd9nq3`VlSK!(sC*%jKle}I?sx!JC`|= zK`Mh<21b9$V^5smk(=;P7Uz5O4}1*fYv<%#fGrQk1zCi4-sx8Wf=-JaJEe7O%V?xsiaR2#T$FtD9E%9b z2wcmc6IFW4eT#g}D=NwU>5`-gsIFw3bNlNs7}Q?gK9fkh1isABSyqxO}%#La&xNR;9aaM%hU_pu?w2=hNwNARVz>jN&GH_aGUfOX$isQe~s0X<5#RX{u zT_zoh?;Am&hKu3JJ0qUeNu)fBkDb>&hsSNWWHvo}G+bRGE2EGv8Xc`4k>~`(!t=yV z`JFgXWi>B>sd8{Q-w)eB1psc$R3Y4-d{De}GpGZ}8wX0?yB16kGzmiE?M#&RyBI*?)_$xDa)|bdL=J;+Z&@3R0oi$MmM`Z2AYC!Wg=!;>ScH&QCx#B?~_pS(!8&eQip!um0gp~IeTt&#F&Kg zF$bO38%)nyiPOgh2_tVlS`d(E<6A5s*%%9%)L!mnR@_`Q(L#)COGy|w#snzP zOstBdRZ%9UWUZV3F%q`q%~GH9%_{YAb8(b|2oQ~D9@J*sn74NC2l6D9#Y$ghTf|td z%Ftu?`t1>o>t0bI(Zk99LY9Y)zgp{zYSv5tyLOg;sw#`A)lg;m!GlG;VgEKwMwSSA z!P7}@;x*Pb%YD6&BjgL6yBFU;aCDLEin2zZ)0SXk3@U4Ws@dH{gcBJq>D_1xk5Yp^ z_rlmoSUBJ-E>eE8vTx}txrKAu>BJ_Eph?qFz(4ddOmGzI7aS@s#uC)e_}+y>6m05`f9T=RMupQRLh|6nbhsZRMV$f`2^iUEUX~5Y=T(%X+=sRk6X8_;<#JM(14iW8gL!YFtOc~W>Q)%4_Pv%_93 zhoCFjW*G?JIr>Jsvi)#QUlq$x+Prd&Z_#q$xSg=wZ8S++gW#KzK4ZVn>r|0wjR0}M ziR;*H#GL3`6a9d|@j8Z}@ff@Fj=h1s;%f+y=I}P^D zcrA=y@n*-bZ!F;0Kn?D)yZ|G9fnhPK$M6IH8V3>$@u|#x{`ZJ?xj6fd zbf^~d26PCUKoati1421U@oIa!Av%J>>b6JUC+rP+Dzr69#%K$o4hygDpZ;O2nRfac z;hQ)kZ8%Lj7&O$2@Cr2f!W@LcpVuVH7N`(T8~XQ&Y0IC^B`)q2l3pxbGnh23e7il{ zG0hg3H(ubijo0amsc8%Up6a;CUv?oQXGVp}HlfmXziA#SkrzI@X9hxvI^q7|dX@og zM;X4G?5bi~HpA_6>+RTq&)VKHC-=)q0UT&x*FeORxS_jaR59Nmamxu}6zO)Y{0(CJ z?1{Jn6hf`8kLdKaHT8?YtwS#rjW}={r>rgm0w_Gt3)9^5Q}BgpKG66>3j6bm-LZcz zdVGLJAn<5FC9p>wsTbVAJ;OSqLY6Hum_uzc3h^BX>R8 zFt`5z(youBUlm^&Pi>U5wa|{}9m_)~4`24r^-&`JRV< zdf0&2CvF`8m5uP?_Enw+K06S?15@0;pnJYjfj~Gt14Y$}_Yp)#+3#b{HP>gfF-Am> z5fQ!PgiC0Tht@C-!P;5Utb>T%i}EB?L{~9-VxZpl^TRgc4AUBe(XN4wH1|5F4$(2X zQsDYgm|TWBOgN5Q2SUk1+h~?KQDgxqVDGqdLwqYC^~~!kAL-LteZF#=2XHOb+lur~ zPTA8co8Y>p+0*g`t^Ji*e{*nl>r^gT&3F zQka1w20#li{eb0g#dqgqU+O~&bVFgDmoi9+hSKaCM)U*BV*sp90GT^&%@g8k=tx}- zXCwJ9j9En72}I2=%iT=~Q*eCyT~56r!?(5;<(hMM@W4v0nv-BH5cyRE5~U1UZ@u}a z5OOY(iTT8k?nFc8#1e_?;6&te_9?xVvm!aoa&$^#`-EFuR*Z3J4;qyuvr(M^4nz9P`_9dB^fU&QlN0(l zimN6@Hza{3!6v|-OQT?P@e+6>)KhPQQbFyib*XM`2Xi#2UqgnTTiB@6?0);isa~~cjn7oP8Uo3YA>YT#1@nK6&Je{$n zWoM5j#Tr-k((au;6WdbL9br?)+hI*2$O)`EVDnWVGvpRXoN1z}S{n)GW)q9{nsZIq zTEv6~0JA(x8{W}ko$N94aQvx@$<>{j29v$BB1{%Fd6y#%M5b4y2br98i0U(Q===kO z#fNX$$ScXrp6)5cwGjRrJ66qXeiH7rWTlBEAitO1gnfE1US@QtjdSM79>4dRaHf|U zr729S3YzUq;PUYXj4#S`1~@W9hIB`~rBMi2h_Sg!qk_=icrxSHQwulrE)rs;Ud z97*zSf2o4Zb(H{P4B85wIp9fH0Hgv>YaNUCiZq2|Y)&sSHAgT4(mJ3w&C3Lr>|zx87qi-ZQ%nuaOM84x*o$Xm4Fl%T6b6s8D_*Y@ zP-|2T%Zm}Occ{7(q<)KMy6=as&DloZ)B)QsH16gFz~{WU0j zs}S*k5#?R+_*MSQ_l7vKGeHf+a>c8AZkO0b;l$n%x4_&vXvF}8OW~9N`b$%z^==;5 z%x!3g;a-G}{!FmX%<7v*V0*(p`&AGeZG#0+5-tN7}p8#4@Cc<;X>BrVep$!{B@H{ z`ivIYqiHU%cfdT}1<3W@)%iPR5W6l=+%2$9cbOkp=nVC_v32;vxzpMW*S&yzQ(6q) zw!%>u;N4?eipFN;f z7h}|>=nZmvVx6DK+WdZIk|fyE`im?gWmquYSIdo@xd`ADi5^U&9%;=8IB$yF7=UMp zETXuY{K;{@fc!+N>C6??SM+W&-({f1_!6hKN5y@%8M@(s6$TV^;>AvXLV+=316;lP zBlRbcJA(XB@@_F?XF!R=?%k|JEwZ|UKg*f4#;+2mSlXiBupgTRu}@LWiANL@W8;aS zeDm@ZrqM!7J57_3AmF8S=ri1i^##qtCxU61J@A{$kg(?@G1w!0i(PTX1_m{STuVFs zQZq8}-+BGYxmQqkjf49*9ITbj4#zI)s@7Khw3UO&xU^&t_9NfLOAHCeG#ld`8|Twc zRU!si{Q#1RYhSsiP41fgs6VjN&A`0e{?lCshrThGSi0Ip+Vp9b?G8PqM0UOmQA>$q zhi@yhCqII{$Q@dH<)Rnu8!3r%mrC7slvW|v*Si}ChfgAH9~KgdS4&xN&HZe5 ze~sI8I|?i$yl4#sb$(AdYWIkAT+O9an4NTq@SGmLt??)|jGog%IQe+(r%ghSw_-!^ zqM+}t)nd*B9J%9H_QU*!X9#4}17_-kW#31u#i|K}xrMl}^99`*vEIe*M7J9`-y_;g zND%8xm?YLS#9)$yIYVRbhjT@rL#Xcw0fYZcJp&Nx$6Q>Xu94Uc$~Ir@#Fa;DZ#PpY z4mow?EfQf}qdQdU7!#ydwinKyX_#Q}L1+%OX?qMyw}@17K8t%Op)lVu+iY$a{L~v} z2Vk7kn)ueMKv!RM!tz9Sm!UP&(M?k=(dkLVucX_ww}ILH^szObnRvao#W34_`fx)% z*|vk@DqJMe{J>KJ+O%9TfUf|j(Ne2a->b&swZu5t>quByAwHc^?#X7^=~&Hb5y9Vv zL%AtYhn9BpbvwT&Dg)3AhY`0@t(H6|-^Dv}xtM!AW~%GXZ={R`ezbPtf_n8dI!H zq%{`q20`a8yo%cyP6N>S!{%Q98cHY`x~#`|Da}6#Wf0BEr$)?Mio+6OluTgUF{bVv zr6^(5+{-k$doSl+>QL-N&jJfHEzHT=6r@!8fC_v@EQ}vuP6w5rYrI~SA!k!Jx=8VL zq4=t8fVoB^-w14z`9El#5W&p0QLwZl7}`y5WzR=ox!?6t@Hfq-!5AphT~{{BNg|^P|P8k zy`;OT45^RD$vFr;Y3nPS*w@#Bt1wB!XgyMAo)iw9L>aBYy2}^zHO`Yo-Ia#**7UE@ z=tb(-TPl=(qk|zO`V3jm((`iUNxtFk(i^(rh~uI*1MWeA@0?CbCUG;o%GB)=>V5(b zN#UTB(Q1`hufODgcJi~Y>SypsYB5P_d8D+xf%tS8x46Es@MIw5!u~9rz2k$Q?o2-< zh?l+89d`Sv`-$I=&MBYsIrw{$*BHj69BTN9lw^C7@5Z-mBgds|b8AhU;%R^gG+1 zc7M_`dt4Z~G%W5FDf2&9=dYfK)_1Q-GpoJ?Yd0>D_zm}e)@$aAw&g~&lX8gOSd|`p zqZZmv?BRn*Mkqg!qZj5){~AZR4{di){UEh>Jl-(-ewY{bcl@tj>reWVrR%^!*^BGR3&#Zx6j4ZM*9QomzTI`Go7tZBHc!GGtFSbv4i zE57ew`VvSCSTINbE^}s0USRr`N+`{MK2xD47Kko(i9#zvwkkG0~{FERs)~Pi2knE;<_*GaVN>qE0m{<;^GpS1!7nvq_eBTExsK z8CFg%|NUB?fL;pLD7mEEWL1dm0yoeytK*9$}qZBlTw+DH>Bkz zF;VX5hd0d%IF=Q$6JaOkl7U<{=_&P55RCJzOKFE8()RG4-cLy>Q6}H^0qFL3=Mqp| zJS!NKZz7Di-E7+p{Bt{e>4hUf8{2SS*pQK{5FHS|zjTWisauqINnCqfVTP1%c^Frz zwmC5d^EEn%CJ#$r;HL1{i!FX%1>HdF5U=2$G9>T~1D;yNER!RYr`Cq(6>lMQWKMHeI^mHb6EC~V1KQg9H&}d=VY?|6} z2rSVOoFonNnRD!L{nMz^vUzbYO8fn~V>QO6?O+Cy0xy+dpKWNcBd*I?VFrXCT6_0l zf-WCA%NQwEcmQBi*#ovAwdi^9uepfBV1_*we_V`CdFk8rC>iQtGOI~e23vo5l{mkd zU@sN-jOVW(|Jq9RUYIwb4E*CqH|YOGS>b=TQvIK9XL%zlQ)B0!c20orAA3h}!+*Rz z{-24NDwQ4i1qIa4EB0ku^z>hlzao@;%fBad@)1Wjg$KY#$@y=bE$x?$E_K`O#`O;y zGBAx{{eJi*-EL0XNC~_97xJBW+s?dp=Qv&SeSF-5_A=`oG6wKynlPmoomK}vvu>&z zNToxJ^DZ)08K&$-E54nIHESQqiiE-iakO1R4;;lL$qqbn_gjR|B29$y@*=w)f>n`o zVPW$yHG8&>lFY^JF^TN=8jW_=`!L+(vxOeM_w$RCQ{we8s@wwCZG8eL`?&=k&c*;@ z>zg|#c_lYHVNkc>hV_nvJev<5vy3gKRkj_b@-K2!u%<5whPCJpZ+uj-PNxJ|j%;i2 zwu(|z#FM_3>GS4hf)#)S)pKfu{RncY^U5gzkd%5 zDu;&Oqn&9e<i_wzJ2!GT;=8YyA~{?MuiL}y<@j}A=3&4&?9H~w!`gT6w<8-U>yvX;_j9v` z`D$l-2DZxxOe+C{npW4^7GH(cQq>1C-DZR;NwbNIajM~?Qqn8cdMej?39Ue}T1rG* zA7OPO0kLxgshHp^#OhehCI@MEIn66x+2XW$VY5*hIp=H`cAqnW8Kf1{dm!p<^e#I% zW2oh1Lawqo=qsAdF54^V-C$~rrn*DW1+i&#Ji)MRy;GM(5k|7-9OPEPXy*a|X6Xw~ z6~Cwj%1C`l36p~7yY)^IEzdhtF5iQ;I`-?|bi9dpyKcw7XD_S2i7fd3gC_ld+qVDg z2&_`mcG?g{{akjX8&aJ&nD^SEmi@8zlNvHr&j#3XzOh&?W0(vP6Q6y4SW%HQmNYD> zryu2uaFTI)c3N;JZI6Qq)|_la)@Gn_cGBeeV$JDwy2V`m^Pr}r2RLm=ohxe32aKTf zg)t13UkSh$hSJIw%%R914$ZH|m^y&SuXtOQb#esI9tzCDK-(NMM3r`It=P>8q!wx3 zcb8)wVKiLCzt^>ECLUq4mcvZ;S{ZT)oeZ>x2~}nrL7}P2mPO*-&!o4!W82xk;zq!2 zyP{;7Yer$-tb7$3IFz-sg>l+gE9%=va{CCiT(H^J-kr;+eedcynTulo}&1aBT=>^09j$gze}mQbmC#Wv}Q%BfetD=|m# zl#W*ww$Z&KGY&0g5UIHki^LfQIk{bKGMU%DaO1uR4LPjG@Cf42>56#Adk=@Sqz`D3 z9O>(gCV55axuY@TL2l5wh!w9g94WgCdRnAAYawsO{FK)NCxchfH8EKHGw}FJdx!qs z?BrpFpoTALXeKN{`%e@QD(c%?%quL1UVr1fJ#i+X`m=TbSqd)H$(?{+koCOgfMu3p z>tAApB^R-vj(eekCX&w6=^V0*_yaFkGR=tXLqkfNt=osul`=IxGm#zWcf1cC{6BgV zE+eL^GnjvBq*$J9@K>RH_y82Cxr4#TU%EdlJaZt?b)cvg0R(Jt6O@!1b2mTOLL0j9 z{5piXLYxY;}NhAbt~`QB~D)TbUeT-(;pEy5>WNSX2GDIA&o1te)$ zsO|DgdD$8F35p*rwrqDOXai|;_LAt9{j@aY@ZLY_8+oqVA3!mEFklWI#BhS%rAbI| za{1lNEy*mhi#4faWeNO0P;s&f6nzl_XhwxcJQ5xTVKD$q@CCcb&<1tR>4m6k#PFFH zXIotk^SWeQAyD(O`*hf*e@Z|}MehGBUSs-(E{gN}_)mU5IAvLo@Tk846@<*sDEJ19 z-;j_Oj=CkjqOxfqS-I-N0J5EC?7gRRu)rkvs$VaN_-A-xrW%70jfj{L)7Z6RDmr>dI(mi z@68W18XUgKK$cOr{4H>1|fA9ze)Ieve@{cCI+Q9hz?`*sjtm_L4q z{{MFp{}tQ*t9w|((ivq1?Q=MV?aqC5Qf=->AS5+TP{bNC08D9y3uLhJeBvy5U}i=PYih0T4edD01?)s_p5Q(PZw3pwwXXSq39^Vd_O36*ty z^=D<59vhurJ6$^;+dlT*j{>QBU<$F%sFkAXWZFz%6LWMV^xRMg3D8zIP>N})l0>@U zs-u*g5ynOrt(2Z7S*87at?GTES*TTSCYv=1pFndS3IQ+(W9}NmRA4M~M|~vuzKcO4 z!TBS6?!pfQ>Wm<{s1pKI%f3>>CA=?5o+caObUvy0a%u6pw%MMXTd4A6nq)Fk@^CIz z5={o=wu6BijVdGXbU*vX%}B6U=CPDh0XZN*yN zmI9`JNzg&tI?K=`j|flXAU8#YZlYu?XthRaPdNU1O~#TLZov$j8X53pwk6#kbR=pU z3BUbyOj??LIz8g^K#LpTSLD@qddk$L9b1CuwIAb9QGGS7uT_(D$U&ypwYZV}pCF z-6&?=uW3Z;An6zQveqa9X^ovfJjF%c4#Ei*&uf;`6ts7APe%a{XCq+2qLl{X*$h%t z)ShVOY2?}=bv0$Jfw`p?gIZA5BSX{iAgUC^OVoSKiu>&wtnJaPJBL7@RC=kmp%EOn zFiMf=pgREBGmM_>YX)43GB(OWnDS6-MHMspU?XJ(K>X z8Q)D^1~?q+eEqJzpaynvFk)c?x5}a*1fuTlmMZ;qYMV|Od*rX&dp5;Le^1>!vg17| z877AKuG44FbJ&7+2NU+*Eb>Vl_D;ez*wIh4ZQi_z>rOfvRg1J@w0smFVUAe!SeMqN z+>E;EYz7a^!K#bgp$Q)!I1x)@nuXXhDVW%6%w4gUrcfCLm#MV!%S)plpSs>aA*p>L5exaP+3*5~Y^emPKnER1RDU?PO&4zf z89sILD=T4NlBaPY?W5>S4!#`IA=ySQjN2>`fmEk`?!y{4CmC96yYA7J7yWGT z(S`z$Iox#aNcjp)U-4A!(3{lCbR_Nk)DBZ!$I-3Oxz=5NY(F_+CnO9^0^Q5FJiATRn5y|#u8ZRORmICDw^imu!W_eWxCMTsI%cKr3x!@zA97-H z--$_LU#c(lLb2^PBa_w_*0>haA%QUSC-GkIm(_EsH+f7U_Ps)E@{e&QX>a@5BcvVr zUA*YaZ+byrVe9=gf1K0wqJC^hz#YAfp>z|4Ax(2#wJ1un z6F8dWwPVa=o!fLlW%IY7^e!}=CyITRgjiE12s6Qqm`XbUKjQQ`-)c;{*0=qKbd7uQ z(pBksNPFT?@KQC9jLmI3y-`9izl@%)V)s1XRyLkH5WjuM3fq$|T}wW~f=Ef7(GM0aaK?rr1cLIXC!-X`FxAJfOZKTYF=z9J)1fT> zNgr9x`cjPUkI3nH`qYpSa2YcXo^r@LQefWLICkVsINBEEMLe4(szL0Qw`h z`s)1Vt`|(Cjgg$#d)u)Ot}zgX!^$IopT9U(ShXR&30BxzVta}f)&h68{^e8qTU}ZZ5Ip^v zm;&s$>svwo3SQM{4u2y zsJs~!N?(EIxU*xiY&y!Kwem4D(ekW#c#EZV8eKY#%x@JO7GMxb;WPBSZHl~gXh_rZ zt=t7K!V#MybxjryK^4R)7TkZDj*9ETcK~pSQXT zU~2H%tPbZPu%sic`)_7Uq&WFPAwfEqWkc$08j?a{Z5K!Nz_FSvvn7y3K2E6uIS$WT zmUk!t`-jheBLTz=y&gk^`SC*s?Z*$=|0^^S{gzHt@c{gnz;lrXjCZCg+Sd+4EW<|v ze~TRqdP|=_y{R(-)~u`X)OwxOy6^^E%NU8Y2##c8Lh(hOoG7Am#Hd#%`|+kbD0xk# zu}0|cT~W|QBIr?R(4##NG@$e^r^MnhHJA5`>xqxglkedvzUM5j=?}YSZ|5&Y|1~L1 zPN2-9*O+iR@@{LRif~(UJ0^#U@C}(wJ7Uv)coZDjO-rMW@ay!}iNP)Om=V7DP3yjp zH(FF8+rAh)bX?5(-9b4R@~aku0eTp8e8j!$j=kYpRs`s_4LF2QSzePp#&^Mtuq}5` z0p|k=ca@A0nfKLbtBX2?z+4T=p#${s8)cXy_kLk8U$gz6DDs+V;ZyXybl17k_b?c~ zRM%bTR<}{%`swa!ecR2Xhzbh6A^Rb>tAfnmGy!np5CiP(B2;OgPfvnQ#+h;O5pSt3 zY%s3%6-8AQuZ73(M08*hC;Lw^^hBFUFpVwQq^(JKk-(SH?Z%yf4P`#z0>4$=f4aE3 zG@5|Mk{sDvT3=|8tAMD|>TH&}%p9mlPi zyScJ8=g>xpgK1kAgl?s;rO+;>3=R&a$VIC!3?9eCJi%}9e=1fIAxDyOMOS(LRjMP| z##5t&6E)gv?(0t86IBP8CS^qk@}fr3!;upUo2NgS*qaFEaU;q+n-TJ`!|)JT`^lYH7X-#_zsZN3f5NP8$woq$wESbM1wYCPAv|AB6Nwu`{^oUq-kXVXHQK4;nHd|ww?PMjswD&mh!}`eO zDDXOSLa}_NQA6zS1k1ms9YPLpyt_Zz_Tkpd+Fki8CQd}ieFwC}YfRnJ{bZRp(f_AVV<3Oy1CA52u;(w}X-)J#KJwO7a!&xf8=BE5pK>#;wB1 z2Ke@Zg~uov5Ha%R6^HXJc++TS7f}foH4{t52x27$`ZNN_nYsm)&Fy*PagoMe{f{wP zrSiyo6hJ!;Za-zZek=douUjX@(CSEw3h5GB_nj1yV!0F{uohJ2L_JFcom}Mr*r`}0 zskGs4p`F3pd!L>^69zW$VfIoZ==Itcc_?P27J6uAs26&uX6*dD&w$|ACr9v&UEy;F z@r$b)LrOiqHVQu`;B!Z@Pj}x4a=W3hLq_T$nO@0{Rc8y0`uwRcx9e(hlWiaX}B z{G^(CgAR)VFEVCKIi$`|59|LJlguK@YW>Q&MG(*5s+Z-JHaZzpA1eM8S?qoT#GiDm z4}C=yznXNB^SDcLQLt3W!FjhU<=b~PUZ~hNJV$0XcnxMb-jGj`HTe`K2$K$OpM6|6l4 zjIP)62mD-ZwVIn{{^$i3l~BD_&pw)UJv08TJe(ZMDFqX!svj4uHHTmLGWs)7a%__9-H=2vQh@ySsAAb6TpdQw$-xB`2;PGHC32}F- zHgD_U%yX7_f2u6$b~u+^*l~ACR+xG`SCa5)gWnbPaL4Njb~bY~R}#<97t0^?knZ6e zw}MPILa|}zXhozNmHXO_r{HE8U+aBleXPhCSw%tOkNnHi5R6HMLtASp&q5O=pvx2%2W?aC4%^)pY}AUA zP-yvWBm@^I1v-VILT%Cgn^}kgni``~P$A^^a8i7JmuuJX1XHvvOPQFCJq&{}&~~Ag z;@A&0CO{%qD+yNZ8SJVIxD`Bz>d22NWB&}yG%;8i8W2Zj5Jzf|x~u?5<{w+IyRRK4D%SAY_&dcpE#3t5~Ccu>5)LdZ!Mayv2@hf6o)S5^) z?VHH2I0-PsezBuVaXl1AN~1eYj(CUHx^2)(&lE@Sqb>6Ee%>?5kpw%wZP&M39Pq zG~c__ruTd_kVq!|yxiGgK&@Q~Bg1N=+x)?i*TpEQWcsLRQ-zx0uARBJoVL7?eZa5~ z)F$3!Gh`seV+xP6bGj-|`uE7WobkFGZk~er#p59$yv3a&MrGBXUPyxKLfl2Stc5ge zqO*vHNw-hCSl`IVqsT`DefB2!mWs%DrPO_R7pyCl6@!j-f3HRK&5Gs& z{w5g9M@rYy62`)u5xGb3D|G^+)Vv#i!;xAhBm`{LRa=?ifnh`U!L-nksPt81)?Ei< zrX0rhn?eWj9QNswp@b|i#+;|t)_OKTnJ4blV^zS~G0I|}k~tcTyx@v<1dIdZKRAP~o@dC@bury<+)bzs z*&IfE@kP`sQJ-(*7gh;>hKfYt*;KQI{Ho)fIhJEl5%Ohj5uLGSh(rqPQP!M=cp}%C zoAq-VJ{;F`>RuwUhSU#fyP3^zler>m#x=ds=%zK?T&CMA+E5Brsz(kHFwNYaYTii} z(^^9j_)JSlw!;qiN_{F{?NCac^`|@7$QZlv{;zoS`&X$;mEd$pY*|u$Tz4 zH^l!42dNs`Pu1GHN1AlWviHc^f3ES86#BN`@X|cPP!2>^Rx;d*dVxvos^*;4e1FA2 zd}TU6X}?E#O4wIY;o1WN>azyG~Zg8#IpF;T)a4!6@e$yI?b;AAvID% z*2^#j=&mH0$Gy|3<#Ea%kBH)5naI<=6o9=CEzKG|8XPU@T*1?^2oq-qT7tnC9TQeb z(Ji#5uz=36)r~)IoQTnYRx*j5f4#V!Z(yAd`YuY2eHSGq{;%1Iv!kJ%lbNX_gOjtN zvnhk9shOdRjkEGU{wO%wyIPw3w{(k&w(^1?>SuF{AY@4X(J>riQ;`(vwZCd`UzzeO zlBxJk^H>7-Mq$Gm^C!?(ggn1{Iu%~}EyPO!uDmd)#I2CWs9U}H%+ZF^^i|8Xem8L1 zfC!$Z%o%6w86tf!KyiQ!?@S?ETu1_mC9l2kfVujizqb-xnS!fmuRPQmHvIQct;S=K z@f~{0pZ>}Y9#C}k&OM%D*?E-&ufCKxv)`@A$Kt=_i+T>$h3Oeh&~;~2O+!ucTF2tF zo^1OV6AD{g4)5HvorRNdRU!{xn=8gQrBDeHq9V8B%w`?(< zacMTB+>hiq8G+*+=@;z0*5 zIFPUwp?L%oYfKz}7Mg&XKzRu7!UPtYgPt1jEB^$>j~k69t#Bd*?6yE2!AsPQH97YtX5k?t{3mgN`0#eH)Mtq(PR9Yd&_DIpD#RZ}IqzgjT zb&}0njB>%1Gt__Gyh6 z8-L!n;BL3mogks~zXAoZ%LMRBCgGS0f#MAE@C}?Tepk3+>A#}L{r;`^zOCtV8UMY% ztG^$?|8$@Ke{68o7#{e3K|~QgB7)z-S-{xh`G-Fw%LK=vLZ@nfqMFW`jqOo#{s$mJ2Ex{2Z+ettY2-eM_jV-dw=pYw1F6R*n0ueqG$G=67m>+&|_q zafi4-$#Ka#AQ`uOJ`F@5qmplQ+MZoR$){l`JjQBP=hHxZ5E|yc|#Ef zaeg(Uqc7VZIB`fzk`9~RTA<_iiiB(OKpf*>nr-Kv!1Q7(M+=Ks%b=Q#0Aj_Tmc#q^ zQEd}i;rM^1s=>8X)K>!zLc!A(BzlB0V9VDvLh=lU-3TVICz@I3@QQwOln&9_evCDuI*I95w$M3P{sNK=n*#W+`z z2WpJOdT^8lDNzEhs_Y~l2-RYZj61ZDFm0b**;4Ap>Czs>QI}m5#TNI}h*jg`>)XMm z%^_O;$a?O_mr#SJlm|foXCvTHKbx7SnCc!|)jc^WcM=~kjw8@R7ot0J#W~RBFTR*5 zePL`Iwno`gGekwaGF8+~z^JRc&;p1$ni!)J8?~mV`=J8?cR|}=U@5&_Q`Y!5cF-}u_ft=*7Kt2G-t6eUvNE{rfH9=ITST)hC%!MQj=}K=K(Jd*>sMt%TXa!fxH%$(G+wj$0FJyCuiCYi z-~B_LUSG!OgAL9#BUmE6-jM2~Y|~##%IDIiwn%-W;wPfGEu{F{Dnl;v0l<3s;+;4kRMV=6wVo21}cdnjz2zS^M?aBYMZ&|57# ze@@IA>)IL99SIGLAwY=C`MNB-lYirBsOgbqIP$41i$dk(=L2Z?t+i>2(5cx7uki=v znyd!?bpP;8sz^{`S;fAEnUI0<&-U2LS_Pzgl0DSixbwN0K~@yqK2H0hX2KFzH$i21 z2kunB)<=DV`%%3B3 zwFH5u;G@-Vje;S5={52~xdav<2%?)jKD3u#6a)u|$Tf0+G#y}xmqV0?9`ZNKbEp*{ z+#=@`3Vcfl%bD3xmS1^$C6c^*%^*QKEgKTgv*B-&2SA~%Jd21UjN(5guv$WnV)i(f zjz&AqDJb0M0at}=-VyRqdpxoO>3Qa-jIN0yB=U2wgAv#VU8oSKE67Te3+Y90@FvT6 zqqCB^Y^im>$X*U+Er*{mig-J>(cO@Mk3B%X zgFhQNn)o@wZA_YV>@lajVe$BS-j+D9z7F+=$cQkcf;Mt2 zV6o?4o#3hMaINt-3U>QOa{qx76ti=-boLN7bowtioUAA-4Z?)zW4Y42y!6fA{)5kb z7GI|kGH@s;pU@YNx=X&rA+z30nw{>Yffow@9q5zNp-9(^gTi^$- zVMPS-^0)?raY2-9zL74JM{FnJ9jep<&OD*XFL%nypw7nicf8}7xPKTBj4`eZ*zlDg8dr655Wr#Z^95|DtflOtf8gu(sX&@lKs$wAl(fUW8s z>#=op`g8faR{-smA_`_QBZ{77$vwz@iuAFvvDYEf z7DvP6_=K3&kZYmR!~v~HW&&ZVgR184-`Xa~OuQ-jzt_z3n*{YgST`k8Lz925-9Lj@ zk&3N6iV&i2N{XJgyay5z8d9O8ev&m6p@yi3D(Ia5-dnToc>00VRki46GAS#Y@69*R zj$-K&E_e{3vSfa{xZ-8uVUfRn&du%lA!AGv!Ecg(I^ql(G%9Y9Y05fa5jsa(pegPY zb(|lp$DCol4xFiO<7k7<*13EmK2ZMy^g(t*ZMvLHSG^5e4YV;i!V)c6#0{eOh{}A% zNv4Bzw&z1NM{*#+b?j!w5jan5fxU|?K#;eN0a1qzBUx~!*|H*gTv3lp-(*`!h90A^ zi2*P1IeOr?n?*bD+GV{m^I;t=F7?4IFOhOwPUARPfn>!V*Ssr2lJNz#rcMjr0#DJT z^zHeAQS`dwyBeLJqQw<>eCcY!9sB3mq9E^Fx%$P6CbuZZ#M3gr>Ni=N#;wf63uWvW zjj*2eE1ria+&t8Z*OC3Jm3s0oy4~%?!c=Gxu0oJdsYH(4Tk?GN+QYKaGS5oh~sfUkDj@$frsw zCDn(#CI9sNScA_Xs-3+BrD|G06fRvwlpeIE78CfCFy~WOK~D||bllS5mIe86F6dFB zDQn@0Fq3BzC#dwj;V6rgkfD_MRVYX?&^R$Kl}qbPVSUO?G8-~tQqUMUDimcL9CyXD z2PJ;aPg*MOue@G&4Fex1lTD%^7PoRmE!=6P8tf6Rf+Sn)yQ_xg$ubAYT91-sflS*4 zq_oWl#Sn^`vk;qz@D~bu42!SG{IA zOe^7lj(G)8!snPz*7p6bCu?iLW45p>1NVsT1i8qTSt6X)s+aJIofrmN~%Z^!(xW6L1Q*F#8ukfP}n*ybV6!_iu^4xYS5a=r^YG{4SgQ2gl-{8NT6v zC!ycXhU&@^sGsDZ@IPb370cE$F(USeODvkI@-&J{6-8lFB2xbVn1cfVa-NO}WCBG3 zzbj_=H85POs`;PWdWsR6c4UY%q}1D5>zTb~CUbmVpSD>}Vte%8!FOSOC5;8Di#c_q z3n0J-bU6m#Ae1eg^WyqsWy}FEELfZ3bFdBYS0=mr+7M2mzUH9U+-YtAc|s(rfCgnl zl_wL`ztZE{^KkIpQKn?HUSvuPF(wMyD)H(n^{iNf+zhvp};3C#xi1y$Q2Y!GZa>;0d-Y5)kDl`+m%@6`L%gv5!}p6lB%YK zQFZ|gjvzcPZ&N3Q++<*7@E;ti>P^39z!CYz^d&Ms4}4ibadfBW;kZg#4a}m^a8R8 z=%n;@%TpVny_Z&0lH3Fvwi2Uj=YOXHueg<_0_aYmD{8Kjt2-kTUlxaea9KA4BSlVK z96%+&{x&O_iEf6(CV(6nA&va%ieygFv7m*)T_x?JSDRe!_A9u~4>ZN7fCLq9*}Aud zwx^?|5kTks$hFdvOuxrjp@k`*(tYaP5Igrrm^|bes{qMpeX3TXnvG{!+NgpZZHU;r zw)X;SO|pmi2XIp}0G>{$l)b;K%Xk4$fbej=NKN3(X5che?&CzP2y&!zX_hr;a$AWX zm2IkdPQ7q<^WcKfUmZqU!4+ozZ06_p?+tTR-6g5^ zjxv_))a!F_gv#2zApJY))88h)E|h_V>kH4s`2ZVUlI8k4_du`11uHvE5PN4A)qEsl zf7I|tZI*+qh8?1a3Wv=0mIl{Ua`7IL_3A+k6Ymal7M^zuyfFtC=R0h+d{rsawupm4 z#sOU+qlmd(&z_*`Kmcv@gO-zKqCCXJ5)*DRAR2S zN(8zFKj|R?$!)@T0(`eKpbyCYkO}2k66UGK!f?|Od3}$s-xrhtRs}h8YhoWMv=(X;rDdJLZgn{86_y680ZZXZ z2*V=yfM}kExuhD8Y^q^9xwjk5ri;#N3=h7hW0PEvmkWK!XKeJ(C3K)5MW~P!XR@Hj z@n>DlM|Z^5#yY+8(Uhr<&N1m3H9~sB84j_eW)e@f`Mhs z$r<6Mop!ppTdZc7CVB7J4g6>ArlJ)yhQqswH~1Mody zShO}*jG!7|C%*i$ij%SMWA&H@<^{l?_o?S3&64b#>_027_Ach)Ff|N+h-hGzz!VJ9 zSDmRB@=B2&pZbgo`dN-=V~ft`3uLNMLzK+}t*R>_r$uB7M)Szd|KT#g<~r@4+1J5+ zPwGTz#4ZZ4mrUx$4atmu1p-SZ5$!%fCutO4K+5*sb6*`qyddhs&;F4aZio>ZMiyou z583Phe4zx}5k=EjVrc)@unh8>l%@i)(i8ZC+6>C&@)W)4h}+UhQJ6!rBm9-~ggGHT z_7&s|fz0*Cf&F@yZ|B8%7*=jVZ{L?Z{P9n z=Rde#`Ui0R_xq(7S$H5J)Zp`1>j81vtbk~_LAc;SseY)Obo1gO7kMod2UZve0Oi~T z?Vk!MvgF0{4|m|Syqa0nQB=?1NnN>)^$d4%){#WEyq5daxQTHMBG{Jxkl+wEMLP%2 zLmHH0{-g}+X_~p@Hu6Ksere&&~aB^awsgtxoo(5TD)h4L<_&jW3orA;Nj>P7uOAx}{z6F*}0YNEoZP z#`4uoZx9H+OkegqZqhM5oZG}Rz9s*DpTupQ_hw`)%o)X<1LX7n;_MxxD{Ysp;qKVB zZQHhO+jcrmR&3k0osMmEY}@LjlQ(DY{hhO)=lgLm*7~!?J;wTRkE*(=<~3(c$p!Mt zXYU(Hp1fUUEWF8q(pyY;shc=JsA!OP5hslj-q?uwZCPmdjO9t{CNLgNFXF0?$_r0Q ztno`1i~j6Q8@T*V8;d?lAbIg?0*gKxqW;*By6E_F@RTyspGO{^XnTq4%=VWQ@aft_ir&Jb zXRJ2j#Fc7<#_*aoX`_N+!SpB_h$l!>;P_I8!v<5&A^U5!I5JwtaNbb{>aLNtpnkvv{oAUg{vG)Lhv@>#0*P-{neoot)Bi0gH-5hk5OP?vBomKgDw zm(Xk8wjqt=WH~L-!Ki0)qOhR6HtB`~Ot@Q6$4Rs~Wo(}LZTI2mQKLo6c_My`=5Y6# zcM2+o9oD4+Wozpjb<5^}O96_}W>3kdd!;f{(Sepzd9pb-09SV(E@8;wDa8)d_QBQe zlZ>5u8ox#5E__J8q_>3ZIO{&tw4j71r!X8SORzmU?ov25`SbheSOz>ZJhcbXh!NH3 z);?Bg8O_*8w~6Hn;c_w)?Lw&nA3zQ0MQs7S;eC(ih-q!o381&Gl$H{~c?#saHdI;X zqN1Y}JzCS;V*3QRBuTvq@$v@FB7Lh+Z0#&5i%pcibA+>n6NkzKnxQ&O0>HC2onUtnk`QdjTssWT^bOi(+H-k_`>CGB zEjD=Wf$4{!=C^nYrMGlT{;oNqf1@7KNEUYzys+)^j$3Q_5*p*Tr;qhfz0d!W8^eAR z2tei%mOG^qHa(+~HeG;qNe)^@k&}+c#?&UJB1D4+#f3$B$e4FYuDmM)+)+$o@>ZWw z>BnpWpjnI0D+FZc9eM2(G_uUNZfcg7Ojt#WWS3O)xn$KB7F{~pYRt-rbf;8?piECY zm<9w#=vbxIN&78ZxOCN>i`dmOJgH*CZGr^h?`PH*re=5~iM!54-+$<4=T9##0Rh#b zomR-u{!(qP*va0v>KQCqlg4PGp}@(ba#bx(p^%Z%OAXdwX-d&RhkbY<)kVd;s^{iP zxswk){IN2p+N?!2#iUDH3SQB;2o>U5s&_z1Bb^w25qqu4we=RibasPMRA=%dHBe}X zqb7jRmr;w(ES@_)Y52!4(>1#kDr}{-ni_YgU5c3tG%PbsucT5fB079O+{vy< z?HCR==1TCfIV}j`4ZQnoz!A zGNpLaSDXnNsb#vzg{DY@%n|AFcv`}ecRI7!Dm?Y7!wg*Foz$?yp`nB#B` zoa{We^yLoo2?_~Y`1%~Ipj;e8Gr>b$c z$m}cKZ400Gz%q)B4rZ~(i+)jle(5R32dg*LEMiUYTb*JD?K?8U^`&Xd?Po?O9-*OE~w-*HZ0|_o_rI;KVJ{;S@gsx^SEnk z<3ncIzFW2$Ry*?-n;VOc%647x()z~W^UF!=r$KJ!UdPDTMlFIT^L*Gwy>W4D5j6R& zGRQGYvr`mRlE~iu+$hk{*Pjc^JN_OP+K07FpuA$l#+^8 z8X87gtZP~5`&{0EI*5q@uy`s#q93QWo+9Tr*R)NUuH@?XSdcm(PL1&Biz(-^ouLwp z%=TV z5YXggafVX}GW>z0r|&Yh6%`lahD(T9ZYT>HijT%#U4AIcMU>+r{AN!1ZSbY+xWbZe z#A&*${tLMnEF1usyUg>O%&k~YRe0qQqqLJ@nq%ytER4lU#`H_7>6#&-weqYKExA!E zi(*!uRLQ)~a4~-hk?;XKAwO8cp7^?LApy|RXpn9&)V7s+3;!40h>4Z;%Lh7lg)Ulv z5o3)IcnflS2t=O%IcsP9XluVX7eA#L@jZyBnFRyuh5!8QQd$0+>j{?UHHj#id+Ng&b#EKG1VxWYU?80a zV4sC}b0jTx*>}_0A#nVOE+};_h+mE`nub){n^;G&RItli*KrEF((6gU(%*!@KBy%*#gN^u}RUp)B(4GZm?>V5j#DlBwtkr>FkJ8>p?M|Q_WvAu*>+OA0J&5AhQ)mpL+a<2Ec9OSHdoZQX`x@R_6`(=k_ zF%X8*6Oh&;7+q(cdyo<$_Z2AgtR+eExLhyeX9?_1c{`bEStXD*%B$>)pOK8GM~m!6 zniicVZBvY&^5Ycgl+0300bCKNOzlpKD9&kj4dJf9J~BPp$Y1>FHTXrAOs(nx*`l_X z3|;vM?6ffZqSDQamjNgx8;OI6lp>vaUc=kVwqIjyt4+6^r-_&`&1RAS3xSfL8E=*@ z_IQ#9!Pngm4l1c3h1+Z*s@RU3!L_^Ow zLw9!|411uo77~LWkpJ19ch=VJ;JaGSJd6D z@U1$!$nectj%_Bt9+!*}dS*m>tAlvvj_5>G%TQ=o3|Bfd@mO_7);n&E>lU6Q_?#LW zSg^{c;;nu8{OicH+TU2e@ueg0{E~^P{vU<}NxT1?5&k(wlKd0zHMVy$k#zZ&;cv65 zwlj(-%7+(yx(QP|#%N%k31ZR|9mDtA3uv$p}RhI zKmX4F@G2!GQq`SC`40kV-p$T2J0YkRDW5dH($1N~w`tDqw>SNtd|(cE)NI4I!wADS zCDE-nEqYt^H#H%e*uZIx4vP3w+8#6gZ$TP$j8W2TtEsd)Ep-;V!eArP2NgGJptX== z3RPkaS9fXNu)Ay?v=K(FW5$$L;p93@Q6|;4+!G8?@W|PIg%-k`5XEu50+!9oDvL;+5GpS$pHr@xD8BD|bu+BJ?%<_kyx&mZ4D-eU~ zEAS%@i`oefShI5s3$Q$SG>+$&-=+=4Sl`hGB-p)06?e0wFrYNyJSs^~e(xU}s3(M0 zV+(coGQ+_nN@0*&F(%03su^>)Qv)N)NRB72N1xhtMW1(mXnykJ&K5Uebv4smQ2<;F ze>?a^S03YjwHZ;sgl&G{KN_1@=X1N#KYf0o4h2)ub?hZ`(bTIP^piX%&?%Y(Lw9 znby)U(|C=|g&KI-Dw)&v?XdPv<2xUkefuTcLUJ|FLcZ8sg)Tyy5DLu;>{LxyxPG*G z5{^{~ttwZ9l7Ge_`-D|Xc^c|!+@2UJZbs>;XHie z6XqC-WM>=*2N}6pJUeZ1M96}Pg!8)LvvBC9O6JV0eI5SAM44{C;lLnS)H;mQ(E)YG zYX-&-VfN>f3+*5ddA4b|X&w0Ka=a`yC23s?Iy0_Wy0})$X<``tT!$8hyTK-Lb6f`Y z?M#|=S{B3+^stR~Igr$%o(VL3qlZ4cV|uTwMqG~5X^^aub1}W%2{IO5&>#M(2A_X3 z8b(hyS0ni{8s_=J9fbdv<>W6gl;jr*@vm!CRZV9URSaJD7CAF4AVQFSB0`ZYk~p=n z2m~R9NEkUJAvCtu%=GCZ!+G8(AV@p|Yq7^hbf1p!hKOm)soW04I z6GNCND=V)r-!|W}uYasxZhyQUs(&lJ5k-1qDBrb0LX=@EKvl*nNeoLxXQrIo%h1x3 z-vzU)IIKq58hEpM5f})?;*%gMkj40OAJVtlt-3&Wsxw4EtTD+3A1$uCxb5(kr)k#Y1guJ=C=X=}~2ANIA|$ zw#aILYIy)&M&anR>*POg0~Jm&vFl?&Gud*K8ZMw|H{%;k7aOJn{YVnxV;9KNn65tt zjj=B%bc0FAY=sJ-_HjCT75QnZ{oyabvdJp_NJuUZC965XHWD<%xsaXWEGm`JC>PTs z@2H0!~1juOLj8gMPOp2@B|phbfkU0E`jN$2@;l#rVrOylNN zqVF6W=RJRJT(r_$Lt3w+Me5}H=hk*m<~uvuWX0)`%kuF1gFS*BHFdOGU0cMAfH??@ z77mnhGT8%KVzFnXl0xl` zZj1j_gPqmJbL}yn5@h*vPaBT)Bt7gHm7}0DQ6Wwp=#yOpb(|3t`O$>rqC+T1X;$x4 zdSMDx?=M)@%}N88uBiR0BP|)ajv5`c$HIMxHBheB1Tox^+4B4{V?-Y=8xAZfEISHYsaydmIPT}&%s&jSNwpOUH4JRmx8_S$iC z=c3d+S7wTKpVO~1Q1?s7o9*fb#iSDjfXl zuAEA!K&YoRqJ{)l)g30LcCR@CHOR@8Z0irR5x?3gnb0n_E6P49uxXrfQnr{rDUuIr zToI6_U{LS#dEE38At|*4Q0+qCih!`Tys-l^7Lfg(jHqWGpOPyhy|`5{S0R1aqP9#W z(~lI8iQAWe#_`2GqWOD>XkOJ^wYaFfE-A>UY>fcA8`!6N&?OwNzsnKc zp~qNZcnwS8mJkd_BlZ+}lYYXWYtB2T$Gcfh1AQY70@x#q;Z#WfE8w2AO_ zt`cK*;HctM$YmLRYdBaHiDtYK6LL>0`dP0XW1SE)&u}nfK zDA-1acR;X)?^bd|G8Dy#$@8I2Py4k_#I#ecOwW-hRv3MAQfB@L|Cukn^fo&XZ6TU) zdlPiO%Hllto^F3$`FwdU^#gW*xW}3n&ASl+QN(D#wibok$C^aL=?#t}XO$U?L=}Xd zp12YM@kBAHH+3Xh2`YflkLndIN5vcO=h=*qZllq!9bG0jeeD3)TfCS-_k@%Zt%TTH zduRCu&cqU#6J!b97+Z7G5C07QdvO33b=RE0v^4hMK zWR7DOCul0IC{Z2J$`fWdZj1#hvN$Q58I5DOXBib3py)bxWicaLc4|BUf|<)_H>_7v z++_?qv_>O4)4&oj@6SkDRIyfD(ag@E9twHbP@P)C?mB|?PGIav;Z4_( ztUM8MQ6kJ0(J={Qf(7iu$xgJy(_`eGA9L*>pqOab>6Z?X&#j&tITOF zv^#ySt(Bmm*palA(8x=<>b<-Kq`FlinNKEKXAk92pC#f|_vtiE z9fDtNWqXCSun?P<$WLYW7wekvi2RN6u8;1>%xvZE@fXa>?$Sj$q216Gtq(O&%kr)@ zVt=Cz`*Y#eyQ^?tx2s^67x#J}`1T=|>m!)|`~_N1eMPZw8g*|OlZ+=+=F^GU{m;MOifgm|N4wkZ;?>RRcWXjz74)r#d{$7Z2fXk5q7 zK=4CP6G>{+HYf*d1i3EP9QWq-}1 z;()BOvl*aJ_!+ab{2o5+l|uXMfM{#EPaZ+{Cf`w5(P9c(Q}k&6z@T=wBwl{ElFFv; z*&4!*f#L#JdpoG4kkt%PG77UEhJna0 zTsGcv=6%gYbFE7#D{oiU^vfvQL1C?o%WA@r>PF^>zL~){m2Dot3PrS;T7_zXIx|n< z=rRSQy;HqZ2OZKe`-{7PI=x7zX}-T4Mf<$s)LnNZSu?|SJ9i4sYz7tvE>{}9D0E?j z`IPyO57g#z?MAZHiBZKgL3f0YW+se$`!uOvM?8cb+zM5GR)w;MhNvM+0NrS(%_ldcgQCAF_eB=oplPJ4N%ONicaPpc3LcSba!R{49FV7nQ)umytW~jIXJ&tnc;Pc z@%N*5n%BQwtJ3?@z!OH&zsxwd6(gCV?}PX(ou2!8dB*wX3_|jRy^Z-G2}>ltq2%-e z^FfV-g-&1pd3U5S;JYSQ4|DP!^Ni36efjg<Gd2`F%VM2iP zcbka*BU7wT;pD?d<%1V>W_2chwF_SeZijKQPAGNd(b)I#R}P@CzQnBQWxxm98wg`7W8< zOC277bU4cO6%4V@Vt$UMJ`d@oNJ>naPxw!Q$662t{8-vrKF_}g*UBPTO^YjENhav)Rmwq ze~7-bZ1OBw<%iI%7D+3SiicgWtun5OpJESWJBJ~chNp&m9OCRbZd4%R{}K3rZOUTlR8a&}MEkrlTg8?Ql>Q-* zuHitSl5qUe%h?*umZ=N0Lw(Bv!nr~Gov*M@*QG1AEx1f&HlLaHoaXkPzPRS+_x+wW zWQ8Qj!(?cr<=MidX=;{~&dOlk-`f=c3LJ}B#XABUya8LZpNt7`#((|Zs(Y1u$y4S$q$38*L@Y0CR*RZ?p6aE!1-is$O>Fx-U3%iy8IUv7ZuL z6j947KMI@r){}=m(OyBb-ZdPFYN(zC%p{8E(5lrU-Lw{Pkn@wgsGHd1v>lzJZ`Wjy z$nuDsyzC;ZLHQA5t0ZHow6mYwB)CN-+dN5OhYy!Zk;>-Q$0>mdFMAOJ$_@ipVa{C# zN!I+B$nde(@lsarTq#+R_#v~a)@ydu6vOhO&+xJh9y(fEJ%eA>*(BP3OtE#z9>tWlGfJ{wxPwS%&LCYaT|(p3|ML~}D? zVbL<#ARj*^+`&Wi$;*l*?<3fpyU(qWco=BbNpsV0Lc@#XRTX2uE-pa;%`cVd! zbo;6EeUfoH{+I$KLf;Ptpq_0SQj1^G;Z!bte9fCJ!IsSYYh;=gTdum+ISK`hk0k@n zMC+HWI}&5k=x?#*JuXnE+t?uAg%V~^im2sjM-8or_(GxA_k7xRF?)A!Hc7)Ldlybb z2l(3GaeyKipIaEW=Vg0xu&5i4PBRle^zaoLHGBC2rsCL_ZlGqF=ePj| z^q15DWyR5J__%J#f21~pnWNR#3X6o*-kv3y9DuVOW8H@>pLGe|4Wb&nAx+cw-YeCt zj3tH7V3Tl8h*R)5rRFuBJX59<#(F5Joj+hrtw`bZp%;w~d&)FQ#SH4(x480%6Cl$y zTC$hOjuPUehq+fvg_Du(J3Vn6-_vQnF2nJ;yV9-wyrQz?4#=^7v#xQm9f+lb9}it~ zFG$OoyR}PqXUEvccQ0=xob1tP_izpJSP|mwI)i7crvnW9q=|w81Dkj!SKZC!#QUL8hm9kd6Eb6$C79*F5F}ld?C1ncXq|?> z!&GxOxg0c%@*^UBAE-+ciax2oQPiUoMUN&Vi0Hivr6-28hN|CL^u-kNhp~=x;z+fl z3pV16Yt3zm9^{eV-+<#|0e%>YuV6^~5{wG`&jaEgi@E+KfB);6Qni0d-O-@(EfGaT zQRpG`@(2{|sH#Nfv{8_g-11iC%s8#Tv16-s?Qm4 zlw_GnG&Q@zA7>ag$#D)Qyk205Z4b3>4d@(SrfG1Y;o@p#+F(8GGKmV4WuA%Ca@nXF zcGlTuX_9S}h@J**&8ljV>n78&J7$Mb&n#ZGvSgZZ-EH%h1pKNZ)SGg+{k3xrn5bL8 zN0IlMT4pj0vYq08354g6)&zI*aKxG%I#Vj%RRDlWI-n zPO$HCvK-IV+L*SY?q|KUxoaYo^`Ap!wsuyGoAdqcT@Q2zNL;L-X)3S68pZS%cOJqwa79is$@DQ%r|qENHW8I}kYx^d#VLKbzqzA?`JmFO;H=gy zN)%_x?4@x0eF_DE*UD|5qPp88Y&Z1%;dBc4q(mJ;)Fh8hMJzzd_-AkU7(M)leKJ^c z1Tbz9xc;av>OTr8%!OD#e0X8%YIRt;3`Fjfj5t-I>^Tc1eawkG6B315tkps}{|1pn zX`S1T7oj}X^PY6=AYc&O&!2ysqfwU*z?akj@67q; z!|ZHEc09AK|MUB#F?Rq-4q04(BL{#1o7_ExHj_59nzQ9-WxO%j{DvnI6fKe_CsTu| z-o{Au8_+N9W!0?~Vryhw?L@o%$h(V~ zoTnx0<>e1|DkI_rV0v7&p7EuO!v*UqU2>Go){=Ml`9ft~_wD2pIpB1g+e)4Uw}s`H zKYDmbRB~n6cQrhDmGIdg?nT;-&p+flSpzjdATt)k821Y+6cqxFnrqgf+3{H@?`&g4 z^$f~e34p@8y%s9V*fTj3I#zcg*?Rt8f-9Az{-q9ma$(Uj9r z(tdi8l0gAY=-#a*1=^IktQX1hv!Z#Q2KGu_FBBf=wSfj@w+xp7v{9dD+()?H?Fkc8 ze=+(U>KO#$MWMVfBkqRJoihk+0a$9cVgy8v?Uzws09sbUKQd$_ka$+2PtHbkW&g|@n8lp$mZw9r9vXXwFUto zSG`0I!xqu#Rnyjb6yBoo&GVyo91ff|MM_LB1X?BLD z4rkrb2{S<)Lsy8ST_;Mh6*6ez){EAhqG@dGXA72Y6Q&c85JIwZ{FXSFFM&D=QE`_R z`X0!0%&7YQT|IzvlNs-N3axxR$jhta=g0KvwUgOnZ7ts$xWD#oehjvOHAg-%7pA{F{ZQV4o4tKcn-DA^+4@rplEixxh$-W~`{;nkM zx#7faX|{(HUvA0(Ggr=#Ggr!%RTEh z-Erwf1wT<-R=sr0=`xSBf%U?Z#f=_Ywvw;cRj}%f_^^k`QrYFoN16Z+w}S>j{RHjZ z%^Br-{nIkq5zRH8bIoM(WRf<+M5ydc%Jr^Me{Sp8K?>zE_2E>6W?Jjr{S#4W;Ay65 zYZCok`nD}DyS3?Ojc$qGC{>$Yd|UWM(I7J0C)ul3v;2bLa+Ez9-iEmv$~Su1z?|kC zIwyKNba|)~Z#lH82>VVs)d6~tUuKyDc{PehGI^!(59Q;edw-@%*TlrVB2sC%wB%c= z(PTy(*QwAkDej7PJ=!NFOu@C7os}o{Q11|HrMhx1OwL;h8ZD&4+;q|ez?7b88r#$x zDx}y0@-L6vnZLlHVzjpKRXE{gvqZb`l)~LqAY6FwmKhCO3l}EcdwS3CPF5$~WxGs- zrG==OF3oIsDXBIjH*1qxAjVVeoYT%68VJjZT~oZfP0a1aY$V&pM?-kEgp6IYF5wjQBYb+_1b<)G2FX(q5ZAph2k zPo*~AxSv#RkcYA9i+1wU7@!8z8>)rb04M?zJ7J7igS8mP2Iv>+57@)JqkZw1>~`E= zx&izc9>4=&TZ7tQTf^K{^uT_Dn81GhomgAU4hY^Fd+KE*z}NdUwOL7t@Gz5mVE}&Y z7wC_Q-K>D0k~@wv7#IV%K(C*QV4w}#EHRS{sIt&k{ekBc`w}?xLkYT^oVu+UmY2K% zE`?0>^;1TQ5qgy9k#w_G%G`x}=AU z9q|e4+iF{Dh+|DsD6V3VSp|8OmM3m`)jQ4hC34&mM)E;}%aoa7rt9|E56L3TzL4VG zELPeg&7GdnxoC|_^;rdHOO;?fieQ-+!l>;Tf1p4&3K)PWeIMc4RJqG)K2UEktSgJ+ zel^jIYjAxSmdn2PHKzN0OV)Ga&((fK8(^zC%2MiMoJfWobUWx8jJStIog1F}9! zm2r{}7ulTNmoQ)t>O`w*Z$%zlvjeXeOvrYZSsDSdBi#-2VaM!1G8%!fE6es6iyt|w zR`7yoY7ff}mBcgPCnV^SU79$_&&?m>0|1iwPMByBx@}c;@AdcsYRFaO6)^!YH;O!a z$DEz}^|PyONFRJipYyY;AVZXWSH-UAQa6IJ`_%69cz4`w)$|VDh&r8Bw8Ja!EX9Ta ziEYG!;{HsR;TuJlc-6C)OIL<+2(pmuGbAA?I|*J7aIT_8Gj?NF&TEKGkf@#+)(H1e ziwyUq)(fR5KuUa6E0{Yv?q$rKT#nh*_e^d^F#?z|s}1AWM&kSQvC08C7ZLD|?6!;r8;0 z_}$b|O57IbN^8^-W3Uc5>8dQ%X>LI$@3JP@ImEsrinU{ukwa3qtNtLX?g%Tkv($79f(9~Dbe>|A)Ya?5 z`mepEbo5{!*4Kn5`gIil_ZYadv4yFv;XiapVwR>hCd&W#P4Xp6GByA5S^C$Ztg5KI zpn>v1Cz;6tV){MaZBAF$L0hGw=A=Q?raEtvb}wOEW~5Xak~FQvCWDD<|N2(b!E94g ze8k)@X`eIo#Gi{#VIkeQC(--rwa3{mp~w646Z@O`jXD%pjb>ZSHf$93%RoN^l+BS+ zn?H6J&&&x~AyFYxp|;I9nHbGot87%dMCK@l27<(-btH{ZS%dPRk(zZ__Q_i)%ETbZqt73U6-ovr2V6#a4$Ue5eg`L zeq5@YE{`_(3YsK)Z9K?Hwl__KRu!Ly1869e*O|n@i8qcmb-9P2 zsOPw=H}VKn>}Nq~f_CxdT#2}|B&Qy8oFaRXlvWD*WOUtT1@EdGER7NtaX{&6;bU*- zPvK9mA{7)b1VV-#k%n6fNxPAI=3^{hp5mYIFqVWTEk|A|Jn}9n3s(SG_c`O5_!*_r z^qJw<_vx7RwxCWhDj;^Rf@ViBzcr3fecPyAq|8J%rYNSU`LB=Iib76r{XjJIB%bHs zgBbg}TkocsGdNgBSIZ$AQO4*}VG>#z=LC(DRxP$NkQ6?@Ao_7 zdifE)0bhoJ`+VL~i5>F>$>tW(S92A-(U}7y^nNzvwfTzTm z)Ue9LDVB;is2=?O6;J7HmT2WCylo(Gm7+39X*tCW{A%2yD?>0u;mN#&TiMhI_$Yb& z2x)O@Xvw2Cn#^zJjEL9V`GePr?m#_!u9EJ-TXYPCzPuVBWl(++6@y9xigKU{57~IJ z{;UqU{%o(nV6I0JN_Rbh3tz4v#R4!q!*6`q#*qMw9jYmbNb@^v(F=15zfw>1zwLnkE%pAHb*e6wHqIonh7NzxI9IFw^8yNh zmP(h*9pu3OS3kt7)= zR%@Fof+W+FBOo4%A|#%*eTJ6YxX6;NOJGJLZQpk;7-3pgQv3G;O#VtQl(5INWTNsE zq-7*}h>ld-1dnNa5$$f;hSqM4$~CFs%8XR3px^YpJk-1tbShzo#d?2sW7{$0T4wdH z(&e6}t}S7ayL!Ij6d2?B04<`88=BKs7}{{zo}p+UL=4D;Sv_@Am82CV6KHXl(rGy! z`jBM0dU@o~kB36ayB-%C`2pVXOYl>3_3`tgkqWrVGm)&gUt))&jX!-}DAl})Kj7^j zij*B>8R=DjbH|jyRa%M61^%Ka;J_C}&cIxQw`Mv2+U$TTHSmW5Zzmrht|;U0A5G<> z%De8Rci==tG+tRPxPmzRBimx^TIMiAnw2Nk`iX4{3&zycEpHviTOb!}FdulyWq@MX zFe#IF2t>jO#5Z7Mmx4?)KxCW98q~C~kHBhc?9Ce98FCO!k91lmM13vCGprV_+Vfl; zTU+QT93@~}LhV%h2~52XFU|!uK#BymEn=c+#Db1XAxWoj=}}3A{p7T7O38%Tq!yv+ zg#Z*~n=?!Te$akw!QE6uD_o*lr2ylPef~2^U5Bv&SwOM;cjZ-cl1peCyM3T3+opL& zp*6&cd5GbGc>cehV?(BA+_hiHB=D6?GXK|P61KOo`SK04w72_9PBkan{)cVg$Y(3e zc!wN@e1!aq{|ah8RM@yb2s#=}WRC#j%T-NvVUe}uI-=~4UGaGlQPNxBm;9(>H&<+B zWK9Oy>-KK=pKtunIo`kjj4a!KJ5o@H>r=skvKya8c`%R?#On~EUBKMvkBZU+{lj3g zk>VIC=ySSK(`HpQk(6Dea^YBV5)FfbJLgZEUgb)k zvYunPY1u{Q#^zA~aZlo~O4E7Vn$DcN?s^o?wzr16&IH_OzIcu~xM*r_0Wdd1|y9NF@Q;EDTWI!%kcVhYjnTQ^~eLQ2uG|uz);1yNIcbB zbzjmn_Ngb%Pw+P1)okgVtw!j;%70E;QRE3`i3El~G$k-n3sBY~DX9~Mxdh+m@`RY+ z8=R9ZTvYa(FNaCyP=UubeVnwYJ`k~Lo_hAy#A|veGJ2n93SkGe0co{~?+<2o?(B07 zCAh^4Y47@AD7nqewaDv~@`CHHrJhY#uF; zGkZ|BFonlhHRN3AiLeT>vpbh{z=|E%OV$_aOS8sPF&nl=-&?@1K3$kjB3h=1hs{FtM~CCXh@-(S*>2{-an? zgkLUOGC~?EH>P(9GHH{8>5>{770oSFXsI2GbS=&Pw)LPD0aCiw#Y?|B(6yJZmO88} ztCK&DyV<4tgGuvQx7po)e)7I?(#QHb&G>vHqc$;Sv6~UY?AhD`IPQneU&oo1f_L5!!yoO^aXPo=xp}T!OdqY`!TUyen;6&1 z`0e#~GquYzT&EVoOWfx{!?)im2Y9bU=eFO5VZUv>w1e;8K(phy?lZFcp7z)BPz>(d z(Ik8<+o`lG(mKeuIi}EDCXFTH@r0CadWhxt{)#@{c!~Zpkn`9I+TL@Y?;?2UrYJ<$ z^%x6D)%6hV)2$haI&v@x3RJ^{!i>Mm!;8jJMAuVwShTfZM*)aLO9;UK!j88fz|m6U zlozB`!O*~i6E*gtP^$&!h9R%w7Zqd&jWRH9-Wu-YjMRRV+eBX!p6XfCAVILoDc-pY zU0;h*h@u~yL^q>_6VLGR>b$aN{6QNa$L7mh0zNR+BlgrovApiG4TQw4UuQk_3PKp` zkmD)lJ-7o}7!UJ`3^R3!ZKXpbG9BZ_qEQJ|io==VDoSC!u#S$2O~V*daSAa6WGm)F zfMo%JWty+RQ8Gd?SJoef!bsYV2!a?#%5g`QIIf_qVQ9u~^)T$vwn-GxVZ}oBc3|R9 z?jubc&>|4U?fRq}2q%?bnr{kTaJbv@lvl4u*l4gvMODJHA&~)24;LJD7SjSotW=t4 zp=hxpuRUvtn6(}mqap3DBXgeED>>V{mUgADUEI3vq1gzWFHp5eQj+sDZ!<`PIZc_i zLa&S|P*gOO`iM$9BofRvvFIG0GQ7fjXR@&czLKS3l)JhAr%=UMxUcW}QJ4!nDExvK z_56=mr)|maT|FKuxOsk$0mh>Q&C@f1{M6Yi-s$@Vvn_8N2_;L9{hc;tv~Q<^t*yBW8+pC61&y zV%mR)|ZETqgw#Ho$lrie;r(YM36SHo7KNHGku&BglO&stH9tEKOajk3Qys4P#$M zjFELHi_+-K@(x>T*@k7U>!CZaeG34v-=+Yl9g_jAAsOyBHGt>3U{9few=c;DS5;~^ zDU|Y?QFQ-~7_2!o^k*FUmTH8>$>a=i@N7vAs6ITFJ;eZDJSd`MRa0 zSfRSd8@cS*`Yhso1WgxHv4nZ_1yx{Ae@Qf|bW zUM$^~J@q~rHZ6zBTxtChyU`7|Y~2RQe(zjbN<3daSuC)kZtV*R%BE#{mgPHy9)jru zXS-Pz1x~W9ctn8Cr&z{8wsqD2Jp^=Qyf57;^_Pm8>Mzw_>S4du)}s;lFQw*HwY!J) zLju1RE}7N~#Ff~Q5|HIuaq+SylBqo_O^XD{Dr@^h4opq`P*DOL9Ep5y?4AGB5!jQ; z_vdYItcproYS}g-c%4Rx?m}Md^7A>Nv8trRhV--`Nn|?Nhi;<5>f91 zShq-aP-WxJ1=Skspb$p+hohH%a_+lP6F*1MI}4}+nryWw*kFI;zB~<7#@0;U#dtC?u0y^OySfUUarK~dBWxT@twTH0EE7s=aiER8 z{pq9$xwi&V?hrH(8#E#h87xnB! zGQ85J5F6ba^8G`ssLNW>t_K5#h&Z)q$~%b8q&#RVmors}??wz$K1d?{VU+J#%=WQA z0L?AaXEIT+v{g1yrX=5m(~7N@pG+wF6C&JW2sg3Pe_D{ww#@F6X7(Gb;S}j#GPbOwO8=zEAhf4m9^ORM zV#!cDo4OrvwBi!OigViB)K)YiaoXr}!@brUqx7T70dQNA*s+oKj7WU4w|j>FywExB znk&2yq>QLUM~9GPp(I&IX$U4`qcYDVj*l3MQH*wjhB$Q%(069Q;?ub}4dj-LFdcJ` zxD_eQg3(Ty@KwskcYY9_zm1nzgt$@(Dq^r@phZ5hYw?wCUp05EIUO&~q32VbpK5nA zOg_}?Bwz;H{3Si)J+KvW(n4_z>5^HdI`zd-nW5eAH*zKsk6 z>5qn_XCiR0LtouM^CS+0hJ(dH&dxDbmy~jY^VO64Kp&9fhuuthqe|Qmf;qtilK(u)nIqf;vk1oS#z8O)w`hvR9cqd^s&?vooV;QpTzm!!2mDlC2dEf-DwW z5~tjyTycWm{feP;Qf+s_Td^bQpq8Rs<7q84DMwPNdwf8gNh6-ahdybADMg8%qQkE{jfkVHWG@2BBXlu~bgY7tzT( zrmu>eS_9zUnBMV)Upo`6)4JipwoY99V+zsc48HFNWIHxqQpvB9imb-xH-txHEs>G0 zjBnbmLgN_EdfvK-JLL#dHZRN2YFv{&>*6Jy_=KAByI?XeteiR{zgX) zOmrP!`;4glA({{vq$0#L+bTNT-dW3+7lq}ZU8>K=^Z{#ESiZQQ_K)hbdmf|FOn&Qm zjvo}$#`=Tbmv6*RL+t|IzaNx5L_hEKjQzw2lq{ z|2s_g!N3z(2#PV15L40sm_0#6Hi*K=f+2wNH%9F$WY55;DW*FnD$k*6eU{d(*ONkV zq`PqoEHbN__U-ECXIpD{@9pYp^WOB9?-kuh1MY>q*~`q=nZN%J=TqtHFMk+;R*uO~ zva4l>c3AAD{xNWJx2^`fZj#bWiD0LS+}ekI0vk%|&; zzQ^jL2`6^)&=`(@ZEldC<)u3lJGn!}$#XHJ%S)Q^llY+Gn{cxo1^tR=+s{8y5Zmp?njlKZ|R}?Q23YMy%2)C`3`oYKQUB)@>|{Mxt=tCPa)OVkeP&mnD4+D zbNBFW0KZhxK|S8Cf_=zV*XOj~upVda=;dpeH8@_WDL%;Thd?}K!rJk3Hf``VD#oxd(jo0Q?O!s28so;NTiw-(BZ*7hJ<(%skdWQUtz3;QG_-l7M{wmSLQ<9wVM(Ub#|*w1EL!<>$BhmOCcvK3 zK@j!lmyH!*Z7F09Y+fLNz|sQ!*la=v0`tcuYpCrOx-gR?^r++O(*r8ybPB({o4;%r zKUA07vCgmv@(*sG85H#V>7k$Kh)voM;WJ{tXMh8%K%t>C2ML-tQOQcvQ}WhPnjC8P zR&5c+8A4gUWcPa(?}L=gvfJt(K9Yn}g~Z4?=~>p%jg?ZZh#ddAoP05!T`bC=Wb_8^ zCxrP!nE?90*ziU~1M6r}V87>qx&`K)lRoTXeA7obBs*;li$7^j8-&=v8)2MTvPK!B zXLC#-%JX{KK+8glny8dfNCT})=YKE!Npz5x_L)6>V;R3oYw#Jlw}s0V3~nkd$`+CUR{Kayc6}UftFVJ>Yp6|o zKlBxuS85HPSS*u!^?Sq|g(-IFk%D6@oy=0v3~Dk*(QQ1Arj`|a-(49sbr3D`2R7HbN!zIx!JL1Tp-<4#zgUpW*>C#|GmoSoY@x?8`E^LhJxI-UYzJAvQd0s8k1k0Pkz zWD{~fXtP_ZZ^2~lT_ZS>bI#R`cIKg_rl>n}B!SbHBtLS8$H>pOITWCrDIaS5owx<$ z08$eDXSmq+>V1Kb$z=?ueIJC^X#QG`o`tm)K0aY+qpeEh8wOyIlHuLnfrQJ&RJ z9pp{ac57F#5N(rFG3SniLM-Kd(F22CpwN&sYq6mgMfFNmvz+?={Y9!dIqoqF!!6`B%K$2db@v z_l)O7l%2%*Nz4z)DrJOAF$t}Ww6wgoJ|yH4X_-g%YSTkGwYv`bXzd`FG9RnF1Yyee zKsn^Cimjw+uLo#3{uX}n3L}ztFQ`1b^B>=VcL^@90|Dped}?wX$-njo$_cPQA;pkCm6$B z;s%IpL{N|UJCPoWCfaiKsW%>h;$&VKMu@)>1CuuE>Q2lcx|LzM3rUp5}X zkWCvD6hSnL$N%7k86*0`0*te?l*KNV$YgaTt@6Idl({8yGi7u|FPQJ3l_W3ZUJzcG zcaUYQjf!J`S9kUpfo*u(3_GDuY_O0 z_qxE<52Oh+#y)1|boTOqs6Xl*t6T{kP&~=Gzb|Fbg(Yq^;bPv15{yN2rtk-0Tjxd@ z(8HW^NHQj)mEVdIQeq?;sey?SXV{|Ro1x2W5br>pCe!(& z0dkkkf!Aoa^(o=qDTb{R$K0sV35;0g06sb4C^_=+`T;mL;-GXUS~?<_Ix)=-(3JE5 z)ebmr2TnUc>ZBs*_@K_iqY>U&?7 ztH&n(kfap7vZfLA(PST6@Wozk1$JWg*gK$EXC*?BQ(Wikos&A!uC%5gdc zChM?_F?B==Onkm%&6teyP^K%i#ts0oJ-WUZhyIphyBI`afv3_j!e+@z{7qS^r@BnC z1;lGMkW_-CgAcw2l}l{eCl;$K3#R@~H(id5f<1%}z;rFmn(4m(ttb{$Y6ojd%P4eM zqeL`L){|mDYuA{9W`n5oA3a)B0&lYrHSVI!dMU;p~ zwZn!&X}j_-)mXk2W(-!{`7jO@XB8*3h#zbM$WX@3*c`xTd8F&IC7-(9ak=7k*`iy$ z*m1e(rOA0khI&21;SqS{m#7No_+WT3M$8)|_CSx{oYxWRJTubgwI)}7OIcN9J%t6#1}~sD9D*P z%AqGMVFH|-nkg48cFr*LevjTt*vNE*`PH?7jvHzP1BiHq?cfgO742anJ{p92Tr-AjkxT^CPs{Xn8}J-5&+ddJ^OT zrTl;NxEPN831jQ{jO_e9nHq!jSyky&3F$UM+SGdenLg+bA5#qcL>fP?xhEmtl*1JM z3x+BEWv>Zhv$p zUwdU#_E@nHT>py5yBj(pdt)E3R#Mbe=iHXFe#DqtUKl-<4Sh^n3HO_O`G+f}R?3LX zqX|dA z5^xNZ{A|QwO;mkEs6INg9y+UA{;(%{edSwL&;%v1<;DGs7?bJdD!wf>I)ycQh?Xt& z$)B`l&G>S7{%!v#klHPl!^Sv^`owu`i5=?rQPZ%_i70a%s5jTa-(M#kfy= z;K@!k&B4hSC$BWF*^WCGkuDsPOBb3+OG{>)V)2P7tHFyOV!Vb(Dl;7QW0}a-QfXOf z=QPR;rF99RoZ0AaTGttbIV`ls##{@gB=&3FhgX#v_Tm#Aa=RO|E@@`?NT{RNBZ5pC541B0DPVjLbt3?d{nvVCwK9v3!K=Y*OBE z?l_m|uFj*Ee%Bqd)C%6%?zk&mL6?`K)JAg1q)U!0&0*4bTRS;a_1VsuNK$1I@lB}<3IB?3 zS?yM%TCn32B~R_amm;=Xcau<@2w3%vwsTjHN{qDV=CZrmuoc7jV!f~HY~KxYuuBlz zmaIw6ZR_(cT$x20hULhMN3E1cCQBOky98;}6@0Zfou)$;%$~C;lYy6wG~0@$HLE;= zZ(7Q`56x0KJ7=0=OssX}OGgBEvc&rcny>j@;fmad47C$0GA!J(6)mNsC^l-vW7Axw zS!ya2k2Xm$_saUKKdwFcL;Br<+%mztraR8zR%14`iVWd*O+fIMl%<&}5n&V~3vVqt zz^Bl4SlRl*S{RcpEC5pQ)r~owBE# zbDmFzym>57>jvvszkw;?B^%zr|G5L{FR9`B(3#G3@U%jGgGom>!{{TdNIvqbbwhSm zDfeOU`QbA3*~C1NVuOu0K?3c{04#qw^5-8@WyBbG>!fT1}Ks zb9mo%N*KEumWQPr!WVjj1d-j-*mA5&BOdQM8EY+{rxh8k208_0~U(HnS(Du!koC*!9+r@xPaJ5W# z*W(_36|>>k(E#FW#=sAnjq$mo4Lq*e_2k``Q_ z;{Grki!t=2^gHZWaJC{L9e$g(i8uIGk9E_vs!jbYc7G~=6PBWdu`jJZ(RC33B@VCp zSD2DNM&4|gp@;n7S#Kt-{~3dnoM0;GFjS6^n1gC_|2KMt>Nm2A2S=|MTs{B|CtHir7AjTM4+DKk6YjvOaWa?h zk)=wEVK|!BCOzt8g}`RxNHI=l(>v+rqap4DM#gMoWL0Pt1NkDhBTQr^u|jRJB}`;T zEQ+fUj<>{;UUGd-bNy#6;7euwUN z4Nn?jPmssWfEh|>XfKZUb|ZSt&cuh!8{I;Pyl8n;ZB*1NFeZaG;FI2fjX;{Z#1UhWi^3d zt^h7oiGg;=~EGN@q$zs<-5t;RsFjE zyKV-Z(z52boKZjDnYjoJ_1lciI+^mvL`tUaOgApFAmd!S^GiLR)0IlQ6rWP`Xef*g+XOR zF&UyGB|1Oy9|RzO^mV3x*mZb@YJZi^#Ut@1IXLh9s@q09T; za~y1U3X2}jp@|W_z|9NYbI*BSzvbP%!@AzLEn;5Laiss$*_-kxeFMc* zn1~Jy?rtXJNs|a~uz+I6fVj&fid>G4AJ_Z^7&FoErVv&LDj*r8o)wPli{ouAO*w6E zXm5Mc|DS@85Gol`i#B1L7hx7s3?x5JJxiRuCXs@jAtIxF)qL&k9F_G$U}j%%R-0GtlG)j+PBXx=brP}d!F|k@9RY-W?m0O0rF4FU=-rUyT&V)a`KKEPu{ZglR>d@;^BL*-60Ex)-KEuP9_I&SRV6P*ti&L^j zxlsu1p7|Xpx;XY%!eBhwmxjQ`+3P3Qb}z{4{&@-E{@v)2*d*%Gk|i6lI;5?#wC`9F1b@p2Wo;%Dc{chaT}XoJka3^my#d4dIUdHoFsq zADk? %*<3BEL!W$81o>)Ext(+AfISk120w15cUO>miBe>WfO5ixwCfweWW~jk2N7Rt|=>XxHcYKdL!(Z0^hS$HXfmcCl{-(0Vw*IyjT>a3zG`9_C1ztsH zbc<*eT!qktZNxCLjc6rY4b#LmyJoNs*Zjn-_rf5@;}~)XIe@Yj*W`xjAE-m_lH>n~}sy$Ptp z<;!jB3uqt1eL|T3tSfp^OW!tMyIzg^lKs7;w)_4VoYnb!iAl&lhJq(gD)#`OnW)i2 z;s;%q3#`3LA#sO9VPx%ukF&hMI%P9~vV_mhfm zxqZ|+1o*kCwx~WVfJ(e%J`_-bmMzO*vzYc%LdtlKrv#V5Wjc+g443iPOcIyjF`g)< zQ5;2)lcMtubOuIKO+4%TjN29aQ>h?Dpl%8!{akTIOi^?IPk9P~K!0FNv}s-Bk4Vpa zb1VF=1)Maj>I)vgYj_F__5f3Ua6RlKh>5i+(-zRn-;-%rEChsogjG?Zt=RX1mFnpP zWd&c;tgi3_lR|!yNYxc_sGMd9O+3Rc?-1$Mh!Xnp9#`d(-3FlQR*1K|bf)3Ww{TFd z1-2@;pkU;X_2M0pxwcSwWCo?f{u97Q)TGVol5U8ZS1if8ofB~t4E#iudKu6yGH&-M z(SIol|I!!t=9?<;4Hw|mm+>}a#OVo3KWO#L>PTWr>J%$UKg3@3VqU&l5^V?u|3g{^}Z%IS6kknNeOvM0hslG z+mm*&;XJY}(lAx-#eIU%x|Qf^b6<2fjWmRcgO&ac^ ztm%)Cie%St?hrntie$$)btoUnCaI&FG;AKxCh3bVs!M#ci_{k(KPQN^CarT`4DhnF zNC@-JWihH|u}OwVPY&BAhu97v;m+msC^hVFexhVrSX8kh z)JmYD6`5*~gjJ-`Zb1TfXvZL0ZPkhxw*D>HW=k^OkqZVsFA`*NOCo6Un(#ZyTN0^} z4Mn6p7cB7)kzn1cH7221yeGXa?OL;vO=-_g*tN^ec1JwSr<+uc7**8kQsBRy-rYd) zJ0Lnk^1M+xp6Hz?pv_?aV$te8{YC)^w1AR88hIt@l!PF2Udn{V+Ee{puBu(Z zuxiyxtx~-glqga4QQ4{65pUOB>)P7fvF-NN(7t{V6n*iZ?PboCfdCnL5qG-f+~Yjs z-h1!+dF>$gc^x*-2W61gdl?+vGM*2}_qxv)eWU+s$K-!JIC~4d_=?BuSIO1)ytn@4 z7wiuAwF91t{a|O^xbM#04jzWaPyBmye_kBZx zga;NE@kt_ouR6Jx3Y3Qd{06BztHh&H_DmNEpfDVl5G{{SSgqQKQmXo# zR@_ilv92XBtx&4x!U$8TmQN-Ws8^)|5!UBug`7BLc}+8$^dZW6XuD#dO`NOb)1)v* z9m{Bz5U_?VV-$*FRF7%q{FaVVAUCfbQxd+2YLHB$q`Vj#chWpKDjp=KHcjd>BwBxA9Ow~K@DOh&w*piGi@9KuMO zIIF?2f!hvNsyQ5;>&Jq33D-(;l&E2TokW;-`RRo=_kkRpf_0GRpq-9>0n@(aCSO6_ zv;sp~AR`j1>!>9w6;As3X))>`T-b8TKpvxy<~ zdTTlg5*_BCQq#3PFyXLr58Hf4sI5XIU(>X8(`T6FSUdg;||?NTy%@bH&F}y4UN-i z!l3TkF5>A!S8vhe)xvX20ZVjXaU#{s(@hh0v~{Z?R+$qoX=W!-o3dh-X=Af{t!UPa z)ba&m)pnRnygwLBu2mdN*@pAHBYf+8;wz0sZumF6TZWg#*+zKR6wWqEEJp6OkN0S&X?ua>! zV@kWjXk6{FA)!{x5msw1sRr;-FddqR?^>4r4k2hZI$JN=nt_iWAa4XDOc#z%+11+` zQkjYnPc0URydxs4cRV?2JhdRrMw1-IdV|nMHe6#yYQ0~oo#A(s*nI^mbB9em5|oI^ zjO!UKSh=h(58dcV>G<6(Cat24lY3RVW!fhlHt(?==yq#3k7p>Zj-hRz=fh)8`;3@E znZeDgly)`SaGfIw((s*E9U*ZG*T;EAHaJAGo0Z)0zV?7W5$?U zsAE{ zSxa=O8ABzGoM1Xbsj`M=tZVm5m79D+3Uj8trlkx{la($H!HsemYhr*@s%n`{Vg=z& z*~kc6BfO-rm&|*H!n`A$a4W0-a-B+YK5~3@!=Ck| z+J>05Z^pdKOyyWm&hok9McJE zPT*xpD&cyH{A4-&dkkfMYBu;vV4DMt!B~-RV2e8hik303ZC;gtwLMeA42;X>L{gnUX@bR}tIu2te$>{Y@r(ig_`39)7_S>GIQT+M~a4Q;mmZmFHaZ zi@TIkuT?yk2@wz}Zl~FQ^Y!r@q&gR6h$JSjx%g=-hCZ!dhkf$gN65VMx9fx~vk%cH9wt*yX@&$XHs#=pKX9 z?52BkG5Q>ftL9K%Hi~hN%I>%osK-AOcBHk;9T2>OeEtT#uifCyOD}G^9PP; zF1`riEXd-jfkh%iY%$Y)k^$6Y+W=elK&07KRLQ&v)^j%F2JZEug(_8mC-lfm%dGD5 z{3M9kc}EgYP13S#QW?K`&PAKLNR1u!3(8k!VPdlub&jxH1>a?o0z&duHOZPNUe|Nn zW8KW%eTxwvMpK5Y)vYIXhW7F@>oPKWzcBc2JoU!#@>xH?Oc{`aa?x@Gi8H@mHMmyDOQ>vZ7h3iG zJMG_(aB|=Dgm$|v)${_%6kEMrJ5plXO5XpYx;jbE>u33@i9Q!+j3T%@bw5YOF?TBW zA<^FfAZZZ*FNg@ksJl}EG_JgcUGjYKqV{b!^+$hjk58v^OZ~vgV2H)lcJP~Hwi973 z`P=d3w3*xoQ{^gpSL9C;Z^yqWmu|9l6rsK zTAr|;!yhS+r$S9ol^+Ju@VK~YeJh@EMis!w0eiuyb@05D^*N1(*-+C=RQ`!#OmcYy+%dt?n1HapTs zy$s3wE#=oM6k8bpF~1yy>||6@Fhe>V+D$8Mj<0QMghBh%m8hv1;>-eQSPbb_3vjF( zu`cH$;5iU+9kJO8VbkZ4(v|4b50FFdX6kGX%=d5P>!m=GFPdWRz}NCH(y5ka6gQz%ta{Z|UXDtD zb8;R-;(DVeVE%|86WtjzXNt9sDC3z~MxN~>_$1<*+4+&*II+hkmz_dV>knMvQ|v@9 z%6c<~SMik={c`?PT#bJ)Vw?I@?1WUEQ@TLJ;QC-uq9 z-mO!Y++dJwGY+So$St8XgI5iPf+q74wlzRqcBC3z%j- zc&S;mC-vI_7efhCY9ydpS!)Es2i)Ddnr*+jBOIl3h;ABbwcIX3LM5&_@Rkv8Lx`JJ zM^|!V)&ij~3S^_yLBCY68`6B$qe^dZ@+%aa6dd8&kEV)>5^JQ0=Sq0yH&(M1OfX&P z=pF9&5HL;M{Rb~8xZmyX*~u|k@TSN8Xuml++6b-%p_r3LKB7lItsmf+v0#xcc8@A? ziqqnZ@Uccb`2y_jh&%GgQ}M`q^QQ^=vh^uPBK5uDf#sNqV@LLx_QagN*|c1q9nh47etOYb_tOcehM<=$#3@|1w_v~|I+Zjz)Fu~sCVSz6>X7ye>-(Mi8|aroP2la#*bwsmU+=v8f4W4`bGI9> zxCceOYbPUoMOCYL`3rp6#{p_#s!3?P;m)yX4|O|Sw(>Vq1nkOD5s%HXMw9^dBMVk>>K0k4q zpB(!;`29UK*Pr_Hg8sou4@l`dZLuNuKRNxz+aJ6OuLQ|kpVROD27mnX8Smm_%>0Xp z|7Lo;!}i7heIwnwMFGD(0OA`7XTpFq4XZG4RhUiTsYH(#yGz}6cIBocz31Hkq2w?$+P^yp`XrTymUKprj>phVmiBeZ@`@!SP-E0{jbGV9Hq?5 zfnSS#_`wFW{5MdIZq8C8J~>$QzcUWNd!aS*k?R9_BY4KjN4oBw_HV?+eGQgeD%u3c zbizGFa-;?csYMcHhe#PQ6|+O^^bjqRcD8zDJKS{_j|)tkN=n~K?4oFnyFZr{X7z(Z zDFOw*4%K_}OLGDm#4!7|6GckhL;hfm?s-+c0x~I!ROpX=Oc_G9ReG#>l5(4=lV-vo z;K<2Q=9zeLcc`DcOel4VSiXsK*~DSonbQNix87ZyZi#pR?z(n|TSu8Uw(?&mzzfSm z6TbmQQwpcDbp?>8aXL&S|EjXz5J&lPS~vzLK4|F0&y`$~Jni$(umAaY6gVwUh=KNA!V)(zFeEze&_MfSN|3GND>aafAC$2xgG#^%uHp${nL?kIj=l3ZZWkDQ+Y(OOO zP=k|OI1rPN1lmlrBWbWTnIvIY6jp1!(YoT?Y-mecC~as-^|Uwa0_Fwu2eb<2l?qIE zV08C();~&0ew=Sp6Y!XG-p@XHXWeICbG`c>fA)&U^nMP7fz-xl{O|_ufd~eRH+?j- z>#^RdF!djYgX?iUrV;7mtRFkr>f2{U#{ZuMDqyVaKWxskolxp)Ij3M(@7uvtp zL(N^DYJPd&+49#7y1RVj{QCP=7kKtpBLeelJsuPE&(JHJ&t5$9oe+@qeNv$RW3ghq z#WFF@Jt*kU&6rUtjoO6KS9svB4`ek-4N}7%n|*YCEtSQhB#f`|0soIiAyR6UCJ=tM zJ5*5qXJhm@?|bHJ|GF9ap10DM{)b}tpOZ2FqZOcU$RF$dwTpRsmj`Y9zjD0ay@C7{ zOZ^L{tw7%qYoHv2^I~w0;Pb@jn;r1?3}#A|3rAC40EUq%hsgLq)PA-UFf2Ak&oC8i zrI~TbSUEXn6NF&$R<2BI3H)FR!Q5qBrg+DccP{ArCX>K83>=Sq!e}f8PF-BPzigvY zv-4re5Mh$D%VEi4ViL29Vad(dc%I#CIW%m?c!Azqt92_un|ZwpjTY8SSvWS=GiE52 zgKmN=A1+obxU#10;>n#I1x>_>QYLFqR9R^@<>wzRF^gGwB2;~x?Aw@gHnA_HM_rm- z{8`%sGbAoIP&*riBNYak@?6a8IQWq-hyd+y6l0r@;tdNX-gO+~JugoloM>}56O;{) zFuFIU9CxwiPQyV?*|y|J7i!Xln`H^Qt-g|xds<)bqraM3l0ow9_%~DVa3jnhYd5Y> z1~iG2Ftb#Tzj36?#a;fPu$46g)w@oxa~e#!IA}S4$u@a)Xcg-vMwNNW=qHWEP$A#@ zm}&|ebEZsDCst(_kS{vL5S55Cu}X={Q)^xninidM^~pKx!-9-(YzCiOVm; zi!zlk!$vK(*toUz*?pkq;NxbHhnp9$;p)@lUi@G5>M9KLI|r?U(ASozB(FT zatLRQlO1CR`VW79ze%tO+BiDpnsp`bOHZQuNYho@J3qX@@X7O}&fppJ58lk&s8Ho; z=Euavjdt2Me4Wg;wfs@ljqJ%riZ)wSo z$Ffw3f6y>$eDl zC#AG$I>pSAu~c=I^iSC%p0EUrjh-y5CLPBk?ZjXTx>FwI*<~7asqFmQDqP`3wTf>m zC8mUk_;s1fcqo^a@ua9zW8A7$^K8oVVr4PBPcu76BO@2r2vTLfE}7k6ex_5TjS1AA zf@b!F2%?39Y|~z8&`J3;ahVPougLLFxQNUKNw&E)cdVey2GK1NOFoLfg~}eKhU3MR z;pEApHWWU3S38>E#8q}4(WO(Qi3~sSpl#*bH}a_$&uW2wHAHD9gmT*?n=RB7Cp_r6 z4%KRc1Z#}h`!$YP(p3WW)1n(r(^Sfi6KV5$3hyFm1zmVZ}_!( zO5q5~HHqZ}KIIQmF_H|FwQcN#!6q4sqPY}8Y?gOHJQ|YZXQSz}Hv=ByNj+Z2_$z7- z<65hWtR$1kMf<;Ty^YJMGUAzfl(L5{F)6@R%(=}*-nFN$Vw|PNBd+HzOa!u-Nvi(N zOlodLDDjkQlxnZmCQ@=&_)_K`hroalLHL>OV=vcC1HBJ2_Pu-p0=aUnXSXQqx_YPJ z0@E}yIlJnW*yeVpCav0C-Hy%$>twansyn2}W(pd!(z{!oK-KWNzU zR7DR#jnTRi|Js%(oSKKYUvk3WJ6ep6yVog3x}7FU{amu$Yv<|~1fR*&;=1?Rn~@$& zqbwEqjZVdC0E3)QTw^VLJ8T8O&0^5(#G>02_`}9_nY45|t1K*QQEGY`L34RsWwpwj zmbJT2Q}XiS6-iMxmo)3OXca__#X6<+`Hj+ga zmFCqJjRR?GnwoCS9hPh^lhLYnqJO)1xHAQEx1eyYz_2yTS!q3*3#_Of9p0_Rw?M%y z+@G{~*-5&11v!g!aq#Y$OUGaZ0ko4cH##CIdTNnUigLY^f5s(v00OM+GYOsKk$5-d z2SpFtIek|%Ww79)x=RyL*GqO~U{ujR_S@7(ZCWQa?!ObGHioh7GYg(Io=H?|cZKX5 zwr`EPJZfQ@)PXf($^p=5t?9mQH2C&K66nvS7o?^ojA-~fczg1;?vodj#ELv!N$#$i zIXs6a1MiOKvS|M1v)PRZs|xQ=;+e%KWY`o7a1>qmnYwb^I{Rd>3WP|_0@oH`S3z5Z z!0ZR|=*@GZO~uAqT%u3ik>0%B@f~g~K5b!uv~NX#V(upg|D4k?wW&qVo%IcguZ`*&_W{N*Fgj)cFE>-B~| zlrNZ#{FC2CFY-}t}k{M`fiW>Ni;oE3yA={ogY>}Me9rCpXD9)}b<;4YY7p^R~5%__U(`$+n z;;?b!$e&ki^7D@Y>`K@Os#*jgRIsq4iZt8UEMyH7hqV@u;J$)Kb|FH$cO-Ia<}miv z1993uPIHm!Lc2k!w-mGM*iW(rgopJOuwbBqdpl5JUu}R)IxR4Z`{aat`la!7PHGe= z@HZy&a1hU)+#vv39x9_SS*R=_b37?p7^MzQC5)j?0uDguCW`a*ZdS9Fy5xKDed4!qG^5AOTB+fNJY9rvm{E4-^WRI^l7e9HgV zCK>bT=kpEV8|{zhKThwtjdy|@QaIbNJUsltusZ{gvs=C|-Wz{OyS798ySVbgIxEb@ zxWdd_x}lu6bfU@S@UA-tPpfvc16%LF@-Gsz=PC8i>LH#~?i)Z|pa6MH2eo19j(hM^(lss?-I3ZK+4;0z@&-N7s8A32 zSsXCz;JnIb&Iv$lTL+3bv5SS&2asVDeIokjm+nQenls{5DHs|myMhdk&KY|KPa@Iw zW9frr$BD2j_D!tar^C#LQaW4C(s;M&{oE0QlcOoE6c^J*mmsZ*%|1L+@3B0!>4S3U z)$6r9uJZd6P+qO%aaE=EElsYnc=MD`oK4?6hRWeie&c+r#~*WH^QBsLq^;Gwh17?on}Br zJLcl5*Y5MM2QbdmVdu#)N9{zL=9;+%^gkjvlNq?72e_sqbj3mbfDT?_6g+c$mO)gN zK^C84e5FAasYB-T9~2!7XY|0? zB?!+;M!!q$4!0YZzmNWc*-ho&<1Ws+?F4e)F*y(&eD$gobssOx4f3v%472BXYQ@VN z;gEc;_g*WEg!*u;62-UW5rlYSb(;V6B$rnXbS43G|Q9IWg0$MuLgY;^LK&I8`4JJkdhu}xnZSPxn zCI7FvF7!-rMi=-u)BWNH>Hkl;&d$!$!h~MRz|}zTe}V@8GuEZ3?kjDIA$-HZFbPqj zAZaZmvrz(Kv-hSpOJXgs{7v5&UR!d^C(F6VIbfI=OQfdEhzB3{D(06doq7WNP{_F% z&)|?biax~eYV+#)yuS7!|M}eA{Q=Mz!RO)=)*6b8MJ!>K^v)WF#nrUOh4e1_+Lc#0 zl7;0zdqZqXK5c#2Qmo2j*;;0dVa0vDYYP43GEGoBK!=5AIl}mRyIzaCV}>DH;)aRj zajU>2lVYQ;mfH$ntcMN`=V3aLOPZ~_zd7N1C=)#D+8={AAPCg4ROtg@&3FvtBAY=@f^N zIzt#KJbU@%zsyFzKqU_yxe7f?yRJAE5bDw>ds|^QF00+bzkEwwRS)AG>k&F)V97ky zhwqAVhl_~EOcDpNM|YaNuf!5qeKt^zwSHpHP>p7_8a~rqLHZ9`kR10izotsAU{QmrNS2&4 zBPw_Py+CO4bUmWm0sO#7oP%$d!feK~WzC$oFAYmV7C1 z+Aazp@MaY|7~)_}&?W;3vQC3)K_lGti&h3w6D6uydJ0|uZ&+^`M;*pklivwS@BkwT zK|BM$Q4CiXL<9$`ko+Qc?`JZ$bME8vc4Gmzf{YHJ&>qKr{|nD0maf9+tR983nEm>wrfi&33jFlqNzVL%EbVrKk**=;+mD?M>TRmv zGGbhd&)szzQV5B|gvv>eVsCCE34`m`cU|H@*$G|33PFd(yZyVMx5%Wj0UKOLg!#1j zhev@IxDBRSqdeboU>Fmcb%}QCFK+0FpL%P3Wm-+aj6#{VgYRx6b38Ir;OLgRk9hgM z0_e;*+Eo{feto9kUdZo8H36Lw#Z0xeQ#>~uv$9O}%gSN_>6F3Il-9wN_7{x6zk;e> z)e^K~@pu+nW8?Pk-Rd=Eb8XwvL1KMn&Q)!m&YSF}>{E zcqGv_L=2@tfxs_y&8rv^Fs?Tc*N>f__*s(woPNx>m5<<&sX(*PiP;iin(T?V3Eg>Wi5h3 zMgS6N_X8L-Hc&$b5`sVkgGd4pQ^=4d8QU`Fzi^$_1v6B0Uk^U0;S`dAW(&HSd%QvZBWC{@ z7nKiZcfL2%{Srm@bD5_+P%V4I{`uY(1wFvm`7Dcq8tHT2_q#lh&Am^JDvkU(J7|wG z4R7mkAEn$G(U8O-3!y8CGL@bt#bu&aH0^$6V2FlZ-kXF`CK^EmcB|=AbkCf!0-#{ntqDYI7q^#py>b@+R;z)JNN2@NE z#_TQw8J9b5)8y=-6-$%Sl;aqyJYym;neCXX+^M@nFzF!%`l`-lf=aJ)o%nbRl%@lr zH6be3fuY(lOqq59Ft_M*jOmC8UpJo6;>d|^b~1PA7y{jr^5ubs?xItDQ1~aU%4I!& zl9ldZqJ{1v{?d8*g+K>H&ZK-q|-9NdG zO;Po6=xincF<^pq>L5DIM4#^S5IM}5lGpL@sMvAL&gchmesnLLC&rGnC?B1xQ=lUH zgLic9W701#%Gu5-N_w-o-O;Anx#JYMcVF~vsDwmtV}&$zL4^|KH1*Sk?FV2T?|3h| zlLMu)F9i8dhNd&(&hAh@{!Hq+6uDd%Ty%V=iA&w5_U-{u;D^Jh0&JIeNL}B=KK_vr z*-oCbWcDVSeQvVvv*bKeGGDQ@wzG#O9*-L3or%6S#^Sci2e{C$qAkZ9x?AV@#Rt0Y zcI~v+_A^b7aDKkB$=R+m4-a}jZ^5+Jjx$jYen0PdN__i?p{Z-|j`kT+{6u)}Y6^nN6kEEukj}I_y4xrhH!mSppvTW|uWuK?zIqH=xA!k^AhZfYJU(Bl(>=?) zf8>{tAzakGgnN1l^q)JAC^bmEI{44xYWw9(9$TRtm%_Ljl>Q{~pIP19Uf+dm^56pI z0S#P2T0aN>`?6u>D&+MAIOn_Xea<~gE$8=#l^-Iw1-%r;eJ;>nYNOv+Yw6&}cmnTQ z4HoXH;@N@r9*`3}Hj%XQYBtTbJWjOVFUL@D~{*#wx}zFD91{ zIwm_D-W>D`z2JtcG}Pyhp578ZRG~cdy({u! z5F|AHmVOcC0v&B^k?_$%nl&R!{_ioa6vOhkBIA5Dkk(c2s_;`nh!|%!_B1cVb0BSd+~~iPXf0KYB~Z5ut;8&L36Crhg?j;1lXj zOfc{TdhU+Y0~0+4_mu>r{sMrJfqM(er23+)>u!O18i)BLd;{1|;=`Q`8}sh&qOU%? zP#UGhRX^pt)x=UDR4SWmbpgQ)o@?@>a^ z#whBo0hiBpd#W%n4T6Xy=pj4XP@x!UUGS1ja;#0(k7BcCf#jskj8&4-!%(Cz1=^$z zOu1`oYc&;?R(CM(ip7%?=d*bXD)u$z8v)Xk(evplERWlOQ$s8=O;yw2Vyo+Kd2C@x zC&?2qk`G+Xmyik1BOig(rVg}wiuA(J4t$S^dAB^Iou@CswJo2=Rwp6ETi0oEq?}*^ z2cvRvXY%%3QBnXNchco89k05)!)A**WXp0{KZ<*?i3U9v0OV=udV`3Ed5S+Th{f}R z$Ol3-MA52hboZsnyM?LSFk8USxJ|vR6CZRv=LS#Yvglhx8>ey^6O=-ovYv#kV4sJw zHl&2drKkvz2!Q#OEVk8iMJj<Cwenu@@>~oB4C{*U_V{7o@`W{=2(fOTZ08fPH}? zb8`&r?Qu}uNaS(!z+%Kn(&Z05Q=hy3pmpkA zS;63Vvm$c&V{*fUKOW0nOggo~kTZa9fW;kPf5Rb>jtt&zkT<)D|6KDR&OXo*Fhf*+ z=fR~txHVD@uve8SXgq@J*VE}yYj13pEbw=8H$u<1s2Xp{h%Ebkc zZ?nrGU&Fi$NhwRo7>+UT4l5StL^p6LG&vfpS5Ab1{|rF%LKR#l@2JVQ+7b3%JAb2Z zz_)DsN*YmM-Cg~?S${v1jafv0q0wNVD)Pm(E>cpciO9POaPXDcaokoiylf;f3DT;AJZvR(sj8$5gs|nQHcz!{0GHmq&MjiY;J7cytpHE7BpfGg-Wcs)6Ed_?DOj91B1IyftXO(l>c zl}Z@1+fsXoSh1y9&dPt7oZ-K3iioXY_U2@kWm39EWsexz2qLGC^pLGP@+%I@F=n9n ztDpcbMDF>Deq5sT>#@2Q(e5I0n2&y-#io-68ibzDu!VZ{G`BOR zN@b)OCNBbR5ZE_$MtI@@e}ni?}+Z`K+yiLY74g-)l0-hT@8%-hRA zF5>I~YrcEg(cdaih z@if)>e%c-#CXb4OJ0DDc@p#AIrQ+HW_BpHv%25EJFQoQh9qeDk&fIYu?0dF1UT>Xb zE|vK_sOq^t?ll|bu8uOA-R zz-8!H%$E%gIq89=u$R})>I=~y4c-AP^dz*my2 zE0GT0I7}7c5801xxUwjx@H4#bH?k*hY80CAxAeERf@W!lzVOlnyT@!B%sak6LEPkO zy1gB7tjyFq82z0~=BpHPXy&R15pO+4sCQV$SM65?T^uHs#o!!C-E&|mW~T6@P;<`s zC#?2{GwtS_^K`^C*2{CiWA6-7U1vOnCY-32;z2~}QwR}NrWu?=x`~c*0m6?F(oQuCbJZ=S=@D-_GzJ@ae4{@?t7uI1Q7|?QtspAgijhKdSFCtz z=QD}sqyUgR@F+#nhQy-V$ zqVI$B$sU$%$|Z~6n}vtw?E)vV8HHOq1Uc=!o(XORl|V`XXbX|=EgGqhsdrK-xr zQ?<*hdOMlTX+Por+K%_9av4bgVgP4)!4Nd=mSd)WG=;y~f))~03O2~h&&S^G8rE=w zr^5&hTr4CfR8ULr#bB>--mQeKPLqr`>{X@^=0(J~)i*88H`npUIm^rfx-R-e5X5B8(-|kxfYiA zy^q4&7~h%zJS13j6Pug({5pCuv$1LK53=KgBtwCf6%8vIFP7)<7v4@h z%dKE#l$Yk{mg*fvqGG|?8$VSM`3=9$p&+yRg>_F@5v2Q(bT8SPd8?NWU7e?P6eC^^O#RALaWz%s~S6!BE zAh)5JwX#%+4Hnu-fLStEN@pL=H%crTCRpH$t+n3uaGaml zN;O{{tAjgq$aMjMzfp}2#RUnu=7%`7%J9lezdSuxxXq3P1QH{I zRnhn;;K7bO7T(ND8Z&DMX5-$0c5x=JPMq8i>G*a;os0_?OLzlUKR_G!h8Q3cVPbJx z=V__&V+pPA;K{esO(a`ntJ{5B^Jbh)!gR0Mi%gZdQxaz6Z)1`yhxYN}9zOC;FG99* z{cBCg)JRS5ly_IJe03+9mNLT;WTq}r5id|oJvmn0zhkc^P#GELQ42PZ^{wKF=D;+{ z6#Q^`vpSwO^2LrfJ%4-byMS5WG==u`Jy7O$PyCvpE6kmx$EId&&Aeh>{Kb@)&i&DA z-x&fE2%0Izs6t#Uv|yzmpJ6ts!&;0TXNS!^*U9)qD`BQ(uz8eoMxchD@eX8fF?c*l zo#xiQTxGSc-qhS(Xf`o=oQa94!M@zMScRm*&(Nx{qLiNF$Ry>kLfL@VriHcCR(?n= zN_bi#>1JEDv9+?cvbNaiC@9!AE{^z?nE8-^$DpNFv8L^p&0ak5+#+Jnz60WF3QJrj z=N#$A-=kL+dsIfQn+xd4wYLZ9SkIJdYJHJ~g-nbm1C#yvhWk)hT?(Gg)Xu~Pp{26f zzKrwaj2kiYtLPB!-t7Yu@^~L?M3nyKz~>v>LSQe*wj%M7Ki*W$ed*EDC}|G{w2O2Q z^_o}FcaPqJYL~@zF0afh1)Sp{Ww>dKGj=}r2j5<6X|7eaX|mOCjy$odXHe;RBz^}G(%bP|Vg zPifGO%b^JxENca7qC8TR{5c@eqb`X02L*CSmS|Qgc)o>r0;^reRn~qX z2&!j0*Ve{}5#bp~6NEhxT9b0FYmB9wgL*C;3}jjIPyWyO*KS7!Zp)rOqTjk*m| zM+L7z9n=TX=y1W@vIE=1+=QI*q86+ws#^9B22lo{WgbT}CO(AP|trODC0lh%ZxbC_lEAmj^B9#TBK5$ufd0E9us3*!y zK}(OlxSoAh+d9g)Tt_xtX%1F&SKDyoALhDdQ{V@9pu3QDK^r=;ir)dkKLs;o@e8Fs z7?$&|X*$UQ?wChN7%`chEFxk?sG_zwn`QQs{6%Z&A_GPYKjJx5Wd?Zu9&En-qBC)d z*ROBH!Oi88yo^SQsGCb-iAI~b&80*Wj0|MeJRDXKGrf{0NNl+G02=&C@?9y)$&s2K zJry3s!l%grwqx?2WT6&NHGH@*4J*?_6~R&>&kex{rh%VG{WXL7Wzm@xjkM@cn&u`; zOJuUi^h1WNrPrrZInnr%BzmJFG&|F)qL`8q*fatyT6I{%J*>;tNba~%px4V9810>t4@Dsh=KeS%Nm=K_aXuV>V@^W^JGJPZpoKqRh0mII^2 zHS7&OKyT%>>?1TX8~4ujLvWy}(J0ku@G?;0_b&|u?ja;FfY-tMBZ>7#=26tL8KJ0C zOAJIyw%N@{)Y4(f0XjE3q<8iG){_*)B%D zF+(13$!-qCD{MB`rtg(o;^{e4dWw030xQ*QMh#o5@KQE_LOk_bd>G!6+8b;3$*hfj zi85YFn(^}Ę-3`U8T#PHoKbSe&DAueEQ65xwm-~^T6@j{ZIiI3QrEu3s|Wz4Of z8m8gxi6943M9sRH(5|+bAWs!DtoyH`B8;86N%RbrD^3YlOvsnXiCI$#CM%qcg_*#$cAE)ZnCPy%R;SGS5>1J{YrzoL+ggY^R)9Z0c$hAh}FS& zzZM6OC4?{ZN&&^h!D3~DKChdxHZHb|n}L5OUR6ETESD7aHEnFdrcN7Yuk9qnV$^dP zMcGb`x7V@gqAw105AinndkcE#1_i{U4 zq0LBbk#j5wa{Y@Tx}6_^G>#{^+7(74p)Q9kBwihIbyiU7A|VE=y^;E0gf2LF6K*to zzr!YU*);&mqgXaO{lKyZ?!`Me2Sdmm2HB8!dJJ+0R@oJkMy4mDtPTvaFKYD|>=V61 z5{2@CD=mmQoGb!1p)DMap^4NM3ztBa2=j)s1EVYwt;RZN)eUOa@H)bb$r2q~QK~Q? zqxkf6>FPQ33CxB++6zFV#vV<uA<(5A4Yl)!-;>PyNR!LPi_wE)Ux z$P8c+1Y#}AAsrQ>Zu4DAR7oRbOvx28>u3chfVO5kgH;T-CTrK#70(4e#x9_XO$}C& zHRPX*@t#EuUv||39G(|OY@KEU(GYotv<)?{StyPx62dY0&yXE~5|Ef40T4&jp>N9i zTvR%H9~5mU$e)qqg9hVO>wY3xa06DiS3p!R0OZV*I@F(9UmzHts3Yrd1hl`e-Jm3= zs7|?#PHhOPzg+%+z9cqH6KSHo>q?YrD~%YSR^OTtE_aLq_KAH0HX3VqAHw&?o!dU* zuXb+zpb#X1qLEUSKfqdk@J8_zhO` zA{sgwzv$~hXal>*hpIJ~Tf%FTItmvRxVew4iUXRZubku3UKUL}2s91F%;8M^hC?U~ zuV8Y?pV;~R@QKe#Wb5Y7zzQq1n0jXDa&~ABd3pG~p%xJFdbr+Vdgp$1QadQwjaqg| z+t6`e&`Bh>MQ%HEz(fxN{VVZ;4}jQZ{h$kY^oH}EQ$;(B`;>6Kz=7kE+8{u$yiULd z7>Bhfwtce&5`wyxuaZD_-aismZvV_uOZ9@2deT9iYs+c_Fp*eDZRh&{hZ>5+@(;m# zLTO`us|K5zuDySZTyrYl<%evpXSJ5}?HjxT_#8Ul<|{TV0oW5QA;+;HTd5MQ@PRrv zZxycvqlvgkh(RUz{_gk%;|CNF&n9m`dXoCzTE|}wM0+&%2KR`Qdj#Ela!D=@vil6s zpNay0HSIHr5mf-Z+<)DKX83T5;)c&MV^BfyLxOy22uSf)ZLo{uT8j2P1u5(hKp5ZM zolK%_y(9nrK8$oUUqGt-3P&U^XkH`3P$K)ipx4|K@s{-gMv=T-6WOxyJr!skCBINx z`b~6KBgXfb87r3vvSlOq`N~V+W@Hoi*~=(FoPZ&`t~k3tU@64%FSPKDi^ugX?*L;O zeYbdVWHH<}ZzSW%<^pwg2&PX!J?eJe<-nx>TR3~1zi@MyIx_Im|06~!p05<;P%eDp zS<3tdg*`mZs86R;a*Kiw&~Ume{H^#6CC{}Px9o`=UC{Vbp61nb;~+J9@D~Z zkxJKF3q8*xO^ZSsz3uFv$ux*+Q}XO!#R1LA7TC%*42xoAOwRyS4H17ZkXvSjMLl*b zQ>e2E$=d*oV&({yJ5&SsVt2&T$|Lz$N15(;n|0li=TqYq;vi2r@Sye>dh1m(*ZdKs z-kWYld5NDtyX+(o^i>UpartS#H$UNmDW|{FuxD?l$T<&$M|!IB0r57>?vzKYDYB{N zCd-0o^_uV`vwII6a{(OW9tN73eU)a)aU#kIMNK2wDolk|EDXNqlPD_9HX?22;F%!( z6<{m9rqv#FJqn>O!2TcF;4^n6lApidk2u6?-kJA3*fpwOY1FXZIn+J7^#lEsZxSB5 z-6Hec1bT7)t+ZsFZpRDAoZCr<`@-p=cEig0@L6hxkd9oTfRq&*>pf zm+}^DfQ~qH%3>O*0|n}1Yum;49cminzw_&Xsk`4 z0?Rz)yfF9UNT(DHtNsgT(iOPQK;MdCt|+|xbfimb(iPj3Hskag(Ws7}9tMV!T6QCP zr>|-M^pM`jZGP#qIHwy>xxGNQvwX_iH@ED+1^2R_g6f+o2TPw94hs?^Tf#uG(3GE||s;Ih3U{Y+Ds<6uTw}hUM zYa;6RBxZ)?$|kYyfUL~DL*Mq zGVV~nngup#IJj-XQu(=m;1-^oewu6(j3m9gJR+~_hq-(b%!@HzTw=4sXWC(BP+n(E z#4tW=V!N^PPGRO%yif0lp?vs7x5H;4!Ya1#J`NxPdeQT4Mo$X`Ze{Jzz2HxAghJGB zy_(=~kn3%xV^HI0^~I=2>tapEoC<^2&5}3yk4GNO^11Tr<2lY{Pjd$Ag0tgto|+$L zvg@RcCT;1IZc$3>)Kn%4G)V>{3nTioWS)XNkRvX0jvq0og!ILIzvyme?5U@5aAZPs8br%k4BT%)Q&K@j%e9k z_#~M;sKg_{Ov1Xr@vh97c>Mb59hmfn)0A~)q^TVylP}3-6m!?wMI67nMvKYh@iIbS z^K`H|N0VM)?>`9iS}GeBqHSe(1^71h=wBRS^t&>lU3=u^I7bS6gez=--w-VCSeX?3 zQc1@xt=skh{24DYU z5%{(05zva2zZ;3J8m6={62GLKzXFU$u03XR9QYaTmd_{|5eU`uiJz&%FS6~(lN5)} zf>r1#g8WeaQ=7J*yPVUQa9h}R>};pz7T+T8qrAyWBF2lK4IgrH{D%Lpg}BMHz^=`| zWLNXw@;>(eH-%Rgt|r3P7ACgN!Ujg>CM3-NtItw-Qx2OEftSppl_m?xyd6bql5{R! zipItr_?eW7N@+u?`FNXgW7?WRwdy&ktHj!{`eEgxYRGgLba*P#tw!yvu%b+T>r! zNfU8m=D)0V?^KUfJaCN|1Vh?E!n6e6Tn=fcMyX8Xb?gbRU8c^uK4n$gZ*Jpw56MkG4@z5)L0 zM7YH>;}rb$UfUA$jYThd(`+e*maQE@-MpY8>A}acq-@*pKbIaBkv< z1kvI4>jR1DjQ8b0#dXL2`Xi6(U<^B;6W%xyf(t|4sM*IsceJmDAai7deA@Tv@DhZ9 zAK%3XhUrrD_h5#9U;b{kc`{>0@7e>&bu>sE`a}%l08b4hKcu)oOE-7-lE;K|MYSb$W{T)ag|AO*$T} zKq6bJHj-k84&_v+}D_US>wuNGP90`|V`EEGLvgQRcUo6ITmk1v7^bbjhEma*TP*^@bLmjYuEV%^HS>9i|2a6n zw@5`}=sN5#GN!_!HkvNB$d+tiwW1L8blMbMDwE~FS<~#8T(ov|yClS;Or5XFLd4Qn zQ%S;wbz)$frkw zx|fSH&FBf8RW_5C6_k$KVVy>wYtb*RQ-uBiRykQXTDwmEiw}AB{ft=QOoUbjqM=P> z;-bY%sQV~=a{_bNGqhvF=A-2=FqxRQ1h*O6L~CK{cBv*FXx-VB9(#P3={vztv<`7m zldwa_yF@6fUzJhZYC$OMJIaQu7KezY!Kk_U@ZkMt#}e#X{^#5%U~-$d+`@P9DhjD2 z)>VwsK~b3pPlH~Ulty9ar7;YWl3V__BsmtCszg2FQ_!m;yU%Dt5Ni_>DVAt{SF@V zr+B#6A9LjrD1gG=-D!BPrvDGOi1{FBFSF^_O4DS8K&E1HeRB?Vrn%|~(K&(~Z~m(k z9%y8u8M2+Bq+|r4dB^(DRkpo#E^Mj50jz}#=#SDJm7bsJcY4~Q))d21JpKLHN*?UW zd*M^`vEYD}_m*bp8tZIdQAZ2ykqL~Z%A4=&dVA-awO4Ar*awi&&KB|{;USM4?7vQW zqV416-FkDwz$I@!UK~2DUhfWp3@@B3hd*;14^RLb}DCAz#iyvuur8qy}7NG`qv7qF;r}4}}wyI>S)?nwQYSJM8LRn2_&J&!IA} z9!36>r1Df;^@^h8bczk2*Hl{brH0h46>G6ks@=c3rMI|9_eZn;{Wh&VnE- zpHOisJB&o_#EaKp59P7?7tiMK7eM`K6Mjj}_fOG>%a-Z#Okj7=V3C zh1BJ@lHTpx$UJF`5|Q-`6@}dCLoEk=qDZS-(bTFlohb*tCKjIGQ{<*8z? zp^$S*C(cdW$KNz2nV7ncl=e3wP{dc9W@G%O;VyoBvhn30g*Q#(Ea$w`>ke`X9K++y zO|mTB51oyYgm*9YSLR&?&(-W7E`B|Da~Kh69ZAon*krBJktM&V`HP2$GIk~`B>X0E z5P55Dw$aR8d^>M!*+(fz&eq!$#`?SH)&k6y`W%GRSoNFOn;s^&$?}rvli2>d%m4sl z?v0>h(#^e#B<}?9|$j3rbpH6K4mhd|s_^a#G0$ z@r4-yDzXhDo07AOnfP@tuAb-8w;+kdBkhdh8jbE9UDC_#=_I?xnKgV#N#+>)-vhRP z?a>4=W^dQ7w!uTg3yr}u^?n=(XL;T;ywv~p*}A;2hBi26DnNl0qsV-WFr@y_5|v~y z6UB$VhC7LMEqYFigEDtZ*n0q9Jgw8g; zLm`)E9Xb(CUz%sn(9-(L5!@q!U13>417+&zgA$xACO~)tuMGuFu}958WA>SquRw z_`0@YmP*z`AX8b8nt3&(Z4t?}gCdmv3Q}PMjd*Yqd(IEl7k2k=YdnZ!vfv)N@6-2+9FN=s`)~eLUdgb47j^5HKDB-scpg!sG0Pi z2$)>%@-(-^H1Chct~=X@+d-vn@LSStDdR(|t0ud4hSGxeZpHFf2h!sl(9z2=^*dis ze0NmPOB9`~=^}Wt;T{4sxZ3h|6QClWTi42`Sc99ZNL`AxU|pC+N$JQ1wIJ* zD09+*=o!n=f25PK1|C-H1ZpT8cpUy$7ceuRnB?WHDXT`JK+&(tWvCW=g2x!eG7-rh zg;xZq%u(Y;kapTYZM^EtgkvK1j6;8B&e>j<(`OvQw{~S+atKx=OEW*u5MGN=;PpxI zNT!6#M~BRZiDkIrnC5x@irfd&+C*G~=|X~EMT%;nEl!8sBT8jYgFGp2O8k5AiB&ch z?@SGhqkL-*l&5$jIWx=igq@(I!G4{RgOyvTAWYFxGIhe9WRYJ;A;q6o;x7>{zo4Py z4OJa&;L_#V9Y71c$7RiN%Rc=H%ukbYu;w%{d=_e)FCpMPIvO48svS5!{@YJ6(nn z+UbWu40#&PW*6p~KSR-BoHAp!FFJ}G(;H4pB;ZyO{$}cBN*#SN=J8E&;CsnzUaIYg z7&oA4#P}ycDp)E8Pa@<+yI_NdK0UZ0sN)WDE?j;o!yoD*9?uCvuNU$`tN=&4mS;;` zT^hE^r0D6fKK+u?rr&eD3w3Bii(gHnGu)Bjgj)E6HgXcUmW`d&^K$m}-{?Q+j4xSD z|IvH)+a>z{C!DhWXDX>t^^;!|!0@%jUP`$jBftU`&9SAHc&8*no{vZfHw+}apDn`C zmRu#a5xR-Fn>VdEHsX&+7$9A2vkO zOjb3zm}4w7P#Lw@T2LHLkANW0A2;H)L?_(5~;$MVBA3f?b5^ket~Z9j0AYwxdDm*VL}ewe(3$*9~yf$2aUMV`%yf!*&r!lu*oR4<<1cDdoc=rfrr zx5w;fNYUtr5zQu)N;6<+q>uvjBopdIlj;jGbyuV!H|38=AX`a$v2H--T7YJmVt86a z+4N4+rhBzqx%k^!KjZ=1hS_K(H+^WZSZ2SAGm4{`S#X$$G;908C#$bYNyF+<($tBb z{J^^YWOKi!c)5#@%~!fKcSGyZN)0?(g($3RYnje!5AFmuQ@H9M&>g_HWpY-|1N>Ny z(H;gm;=9{V3NtS#k&O=X{>H=%lhZx}1OLPSgGsh2`Bs|DEv^Imi((2$Tx?KT>st&f zxQ$z5t4dLT7nYX|uJT<*=PB-Mi;(WulXotgmF+B!3{J#?P2NEhrgY`mM2l^;ocst; z8K1P#1`#xH&6YZHexOqSW6B`?!?x+@fji{Pex{P+89#BVx&yqlNSNP6Dr<-TG42)H zdlDZ>!|7y8r--_M?1wi4d!)b1apFcOE9qY!DDU`uk0{T4k% z{RpfCepVRkBl1ev?8ex7Yhw*e+dBC~6-)gNKFqy!&@xu2pnC!+H)63N>M~+ptK>yH zs3%T%VYg>`L|rIC=s#!CsSw2h>@tbRJHZ`(CI4Tu`H}}Ui0!YBu?^|}(k=bJvbm_E zqn+dbBXCJkTld3ON99AN?`%c&GPO37=W1yhYeXg|v{*E?K=Ht-nn#m#r7=_dyP%P) zlBrmdfSXpLkgDVgm0WX`+%hL(kvX@3v?4SK{3*AO=zfU5H_EAQr0FUm`DNe7yvuvK zb(`yTlcM)N`$!Mi|L4sEn`H=mV?XGQ5FJJLZr_yyGH&q>G4q`Q7Qbj85{qw!;O`#t zwYhI%@BiTJD}yTQwj?PEx5C}s-Q67uxVXE!JB3}`-QC@t!rk57t#B=1=y}}}@pVk} zOuS#`od0L-l{<5-Oo2hd-ko`4{!#tRUbB(Ew1LnNIV5*?aPmygplN!>jH5d;#=dDn z@x1_J-{4-gR+Gn%6{^lJ*qOrP~P4B;YJOfEU9 zx^mha-X(KYqDQDQ)hW#UfSCbAsZzX`bH~HHkQa1glAo5(Ik5R_6zq>UtECCL_hGvdjWe}9jbc1S`3(fu8O4pg^1WQ`CG)Z ztk5JbJ`kgk%Z9TgZFRR*hjoJXF}V^ArakS(>M&mS4z(EUgi)FaudDGCm$|wsq6mlE zN2_a{yX0`pQ|gyaw*PvBb+v6Fjd`kPYi7<;1hfR~PpmYg9scP_E$r4Rv4h;CB}WuZ zmz*ik$>L-Qb3Wr#ppWP;e~57~Q26BR9(Av!p3UaLsLHa{L*gTi{TG7nEv*S2u_-GY zlUN}N5 z$B!AC%xNUqb^MmG!LJFxnNjDZBUeRL4qnOMLSh_$QGJ2O&f{sDj6Z5xccZS$ab#;Z z6moY#d3S)g2jC~i*)TQ(Gmb+dts-QBZN*LT)rqx;Koh0sj19!^ov2UQwC981MWrcY zTw1dX#V&jWW{I7yvm+^Y;iKznb*6~{s*Ow~CdsIDDGf&tkvPbSyU}OiWSh+vO#QTp zHn?V$hPn+emiq_I!7Zgq)F&23Lp@~7$+}9i5$J5v%{b&EG8)+v)ax83JYv32q8Nv{ z>G{q>dlh0BP+vvdOaV+y z%al(b(_)!-Unrp(;8LVZAUZChGkEN@wL*=mHgkXc6_&xuOevr?wf|A2w76VaBL>|; z8}nt#MD$iLZkf0wyM{WZOG^P==LO+(b)yw z9dMy`y&SkV!HQW;4*+u;>-jX0P`QGLlY{X`Ex=F6u2D zH%}XE`1ESafEK*=S`x71WofX|rr)*|NOYF}*h&Isjvyp3*y^RH=9sy44<^pJZUDj* zT~njfCt_Rk-eyIPdHQHJ)V*+AQi@%v9M3)~%HR3fxDK=X4+Im4Zw_kmqt=>t#X6cDc?Bq=KC0Ddpn4)L-k33~Uc%K`_P3beSOx zQn{ye8gNS^;a|pu4cdcv1@HOeY95HZLMVv!`xWuDrVF&ssrC$lRi}^P+mXHXq0{+{ zA3iPhrkr^0ScLXdmd}fqQjQPexw)2#!dNcIr!POP zC_8mph!umJ-#hWdS$+sDIETJ3QsMnnOKE9AJ0)}R>9LnxE*sC~##@}sZKu8HaU}|V zbHC(c3(ig#{wbF?tSE`j_Oqhk;nKbM&o4LbD#l1jvyD9;^tEi=Suj-{lt#`7=xciS zMj`vRF@$tGOzmH898jh-YQC)owX2LPP0Y`;5WR5tfltbTYF!f@5UPWEzm5twW=YxJ zMO^g)7u9kTSac?p<^@dRSdh&Q(92Hs968K9Ip^1@+kA;?b|*BaDxIXN^~TP*8vI#s z(hgpeI>T5pxEjO;YW2q5?4@@XvWSP?#jp021xlL3F`uMW?T80961`$RdAU!ZjfLu*l&N9^kV_uw1DF8b}wXgTlH_oOUfLmBr{Q30l zW+rz}nb!dQ=mW-!t~}ns0+C{(wqMc@r!%OMD-^QPhji~m0KYsQipl5$$Le$$l6;wA z%fV|0?^xM>CN_#!Vf4AC;@H zayLk^#{>pbUZBs_N~8w*^cmT=T$AD=X7~QB$fSAKbtf2p`o#Drbi37|5kCCV*J}Sw zeeM6aXpDckXrS`V7dlu)z7oVoMGozIV^~WM z27JkDh|9K!WRn4!w_W=MPSZXU57T7D1T7B3!#W#xvpqgPe(>@d zn_F8HY_f|~ZE_iwKGo+Ls-IXD@S~p!u8J^8LTOx?DUnL_o!xrDE$LK#aU)=POk83zG!rByb6YUz(Cid-ENsVy&22${{nns>? zBPL{CB=WsH2Wq8c^GUz!_ujhr=$)7Ecv!nIGI==1P$6|HhU!Vb*~IKJgx>=zYn8g@ zO#qW&bGnI4tnn@45j3F-|bi9w(q&TLb+VtAVmZNUePi`YwpXBrA#D}gEjI49z ziRO#U!EiDd#gjWxW8<5j1GQ@iC7TTGjoJ8PCE^TnsY~M}C=@?CfU6I7`g*5~fi%WA zXsr>ytnI~nRA}RCQc@TZeEAR_e=<1*tWllmk)v+q0IsJ|(cPqW*MjdUzH%J0v%59w zRBJU~&RZ4ptc)?rN$;Oje5T$Z{`u$BNuBe!f`Ndf!hnEC{u|1y#r#!f=G8E7)S!iSsa*8sr`mHeD;Jo zq!l#A4-nH@MNfCa<}Yg7?`~~w(ob*cG2L{r7bMuK+Xz;DfZI}vrBa~4+!SAT#Hmtc zihfpoXk#hEHlO8?<*CxLjMrl5o@|t0T4~J3&Jx{4Z=V;z{l^k~ZOJsysd7I_dtQBh z87zn%vCCpLnS&~q+)HEzFb*0Q*&$O3B z>$){esa1SwvNGc11Q49HS$AeJDNtoOo!ImtUGXf_cNbr!LXMIitZPDutvePPmEw9U z)K}$_W`mAEP9B+QKUX%@x+%ZExcclb{cY~)wUjWbT=Np1uD1}y9Bj(Ds>*v;fn(a6 zA<0)r%l&pvy>i=c>#iK=F-^&v&QdPKZb(+3YI($vDXR!Q;!$#iK4B5z_S3oc_e2kj zcpdHTcW@ea%3Tt0f_)V55_`?Q5*YseW8{+=Y8aTQoyq_wOvP(t-DE)%_J{57BVbcB zj6!uC3lPg-rEGY5Ra%E^^NaPqJ;muiVZtfvYEi9s;(ocNzFH$@v;Sg_2nWSk29I_D zYVNfDHs(s_-=^l3X0G91?HCZ>q_H??HnHMLwvnJh8h_*CsU;m!p8+oj551IwJgSdy zyN$Sm)v#n;aBs{Ws|XfbWL}B@O(I!jvVb!7>=5)KSK9R49{kQ|NmijWe@p5VEu?{n zc>sHmv!mkZ@pp%3VZoM1E;+s{yGTF2tO(0z%^`!o#@T7{DgLR-aMOnUz}6eVYIhBZ z0n#LE<>fv8jD8hg*)NUa`|c0%RN(ftV9?ga!lHTdj`&f`!ua2~HLNHfCAm4=V8#<{Ej~*9EZJQ1qnpI4HG8(1n|0O@)VCyEg2M=>30vX~{Hho?f-CZb+DoBwO5yDv0w4{jHl^GYSPA^A1W>cA~+kU0jB6A`?qf;N z-UhC2QI9$C4F$wy^U>f{vwM(n(Q_eimqRlZ8Cq~f(O8neGTXw#JL?vPeYq{QgEH5s z9m|N}xTcsaGA-)wm>cj_CL4;pQluWLKsjAIz|A1%JSk2yVu@M3`pCn`QCXG{!^O3a zybAxUa2n}cFH0KGR6%@xkW`bNd`Ht}X^{7d`71x`Al+WK`FPclH!i`Wuihi>(bFKA zpWUtr1N*Kx(u*E_r$c=}32U5Qb;h+WT!T9?k7G!o?(jAIyk{HEXcQ9eBbIgHd`Nr? zgFZ+$-qS0Nyl~0N(#=w~HO{0f#^CTN}UdHHbbj@N-Mds~H z!1OqjU4}#qk*=PzICZ;Vg%IFan_Z+~4cYj?z?p8;+Y~!y>($XV$!PRLlV9nK(&Cml z!nk~^x1i#1hR!G{oX>d<=3ageck%$$!lu4z*kaP}yVbkku1s`UEG`+`HU6RHka$17xqsCqL}Ln>r!Hy~c8W|Fh}#iY@kTgM)xPz=D7%{Tog9e-A|&7+RbB zNB5O8aW()r+d2N{z|?=WUmF058mccn^eQ5{hr~HfZ@zFTtysBgBi#_lEQlp&chg>d z8!2pt&W1I}Z1YmuN1$)n`#0a42o@D)sa#KivAC zGrx5!*vA}e;9uQEPgQUYv7s81#HRirQH$Z7fI_b=?NL%dETuy$(n39vp*tCGSG|>$ zVZmLeY^!eMlY7bxurMe2NNnUwOSq~POwiV0)=9&2$r9G)xK$F|K!VB`Y&}FZQ6=2V zaZhD}?Q|2%WpOyyydAn+m31U$%4kiJk}XaEBs7eUu7(*?C&|v26$*-M)J0_5da<;t z#DD9q6Fdp9$k?I)t;le6^TJLirzj2_H>NkxsceJ;oEd{JW$BnS;3)y8RNHqTHgsw)VXCL{u?8i>qD|iyE$c%@hh>! z*>bDVWn7ztGvwpMu-#z9u+Ga~UvS5By2B(htxN-NrO^>sB|0ZR%63pX8aCSuUz?+& z=74^zti6|qAp_dwRgJIep1ViMts**Gw1?S_8rur>bMWm%Rk5!=9E}zhH^=#6pj_h~ z>XLcWDVms5&N`a=s&#>mfnZw3NO6!FU+JowPMezF)NJ~$cIeqbH);o$Y=FL3;aYOg z$eYTZdbd3q=dd;1m3}N!0#?*0&VrN)Wu8Fp$oe(?Qtdh^yyixpcrXR{ung zx?nH9xnga{ul!V|c*qhBrWr9|v}0;ayyRrl2<6sGA!a)~o!x)#2UY5VTgEt7AG~8< zBdsmJa^M0bj@-JviT|Vc#YO79H&@fpP~`@>np1r|TH)3vKtNww(#^$98SQ>>Q771Cgh@yTTna+_x$Sd*>2uuy^G%6(iXg$Sbx0$h zbEpAdp=(!tyrvq&aBy9rT_Rks!1n%0e}UWWGaKh~H3hv_b%N&;BEd}2s?dB5=R6VI zej=oPs@V`gBf+L%$@Y(T$c~(?m-&NlHNcS?Y;VzXLaC{l^mM4Im6`_DyR!C`7^p7s z{zE?c22@yNqIIOHnaOEY zq9AIhVMcB7nK9dic${4fv|os@gD=#FYxRzZ7}R4P+;YGfOe+iTsN#%iBx!J6Fui9O z7>`+5I2LWlG7-|;PzTWg%CkXCY|*)pKZK#4ui66?-MU7rGbQ}ulW5>MH)ImjDc$EG zV4R3r65}{If)5=8lmxm3Nuq4BB2B9}P~oiDVh~k^*l&Zfi&Y59Sg?YP@dPQ*i1YCEolyIEVRPJ!4Wu3ox(nWp|4zY=j6V+7?Pk z8LZSdi9<+<7F4%G3CF_E=?N9Wo_fS+A^0N3&r&e*yaPWBsrOfaztn4P4;+(q?0eQ} z_M|Dpw&~M_@9XOs+b{l&@O-%9Xi87#`~06GzoWq*oJH*w<7BYuty=SUsYS|i>YQx& z;JT#WeU+%-<8Ep%*FK>71!)^yhIvIu zsW8Ar#Kv3)CoEls+?~98iKJAdH8;|Zv#YXU`$7Z8Kp>39T!K@a-RUVL=Lp!{wFvxU zq@S`!#@MbH?+26$Exz_Yc?^d~VE&>xn-`hXI<{Il)hL-tw42B!q?`qECfgtZkc<;2Ol(M1%3z;TEq z-R`|~ytHr8t@IW9)`z$mw<8V*(3@DZNO`NkZ`ue^@Npo$Q|Wr;YMNMXJ+5X>^%xlugI|VAG#cn( z!VJeL>cja$k8mO3Gd-B`5VnR3J)Gp`&A{xRa7i69Lo**KY(|)GpZQD!B8~j;jXCbe z%-P=YIjg3Ugsg`~fO3-vqwwUSV1el@6_o?^;Vt@N@ttHc$F}Az{56C9&syN4lV%pm z$MZbeI!Xje_5^zDWzZQQ?+S?EVh?beC`em!O>E9UP}L~&Oll`ugQbXnT8%KXQaBw{@@_G}2)8&dE=Vbd>9 zb<(eR>ts;MNlMjS7zI=}Y!|Sy$iE8(ao#yE-_*OT$g6{&Ay9otl0tz_*qUsO7Mwqd zcl$8L8u>=J#Tjq4A%g*CxH{ij!P*r?wE4gpL_SIi;qQGiPIx0u-2Pc-GNBX)(O)$N zME`#zL;u(C@>e#rh@GvIvw^L%)4ysgDb+0#Wf*OEd(O5Eig6f=GcXDs^ZSt3kh}w& zLnlSm6?&mg9gPkr&!{Qc%1)_;7x(B%9z%@A+z zN8~4HeDnD;nUTg}w>7cF_t|x4OWY3pV38%cKO*uRXY5MaYK~u;TMs$GyLg#Oa>nV3 zc<;>E^;%i?T3vqwp1c!hp`Ip6mZa@G620~yZW7I$Byv}&i*r{Tp~_5Zo@QF7gGL=h zuMA(_v~N~MW%#b!EImRuQeU$1bZ_35TzW1eolXck@mbQ(LGJb{BP5~HRRv`?371jM zwDafbQbWu=04&(<#I86u-+7LJv$wriMveR$_!>rg^oSc@ebXzZ(gJN!z&y)L8A4Ef^aC$fo zdX6FsfL0Z{1yjysgr>rN<^|Sn>@OzHb_hKoK-b?TsTZ1s@!ewM7$eTxpR)$_P@i0N zQ~?|tp`--UgZ47TFq`#WNvM!vSSpqswXIZrFiacucMYM9hny*~lDA53)c1E(I2D-+ za4rl;mV(ssJmbm$Yt%*k-6o|iW>~JKP@xSRw=psL$kS!?+1i;wfYBZ!b9-{saBq;< zdv3sm5(QhEIx}i(m?nz|isg7>36ZYbogoizrukN;=m45VLp{(@vap5%C74*VNOm-o zc@O92R)j~^`e?djifW6kRN;C^a4Fv;$NJ|3sNZ@%XUcAG)5@XGV!NnFY%~C?h$)JYF#4HM2-Y zeU^8rYNYv>|n4awyXWHX?ydg+HjeIAOO_;RB9 zJBdI0eUWACd))1?!zhgQQ<*REjF?Xttw@x0#DhT8FfJ?J!UyftNT*?XrT9-8AQov4 zoz#|SdZq1w32U25E{(>T2A7BMsKz_HBQ}rekkVbSHu7MO(R@ym(7`($D? zIV4Uo>QR~O6JDw(h&nsFp zU|QVwuWzl7be3~>CG|cnqPb{qo5knbgfnwl9mU|9$&cj-(WjZ~&W*MO7mX3FZCAB4 zVQO{f^RDYAxX>vtPPP)cH<`;{4it19%*T<_ZNoQ!I-icK#UUEqf?c-ZlDTFAJw5JH z=e-YUzB9L#`}WK51eZYFS2!|N62QH*XOf`Woz9BNv{e+xh3a})bjDO&^*kJ3>7?rN z^q1qy<&RkcrF)52K=_cka&Fi0qVAv9us~ED(E^eC6x8>ukGOolIrH$ji(@@Fr-@VW zKr!5N6wgal+_I}=GIS^gd+N4HjUUi%s(i4S$~Wl^PbLRm5?RFy4pvmuN>aHMA2lOS zQR)l?4%FR($l8*5uvC$++7ZcG31d1fR~>R?J4>?L zR_koZj@AeM%~z``B~ijGguUNEm;3+XapK`AuU~40Jk9%C9ho@`B4gP~4n!7}1Ke<` zr?H{Do~H`L-?L)KEL&ECmi{!KMK3hsq>kDX!jQtko9UG3Ub4WEhR*7TOhLmHTP!DJ z*YTkr1q-Ioqe?3_J0o5LqxxNt5=eBpq(|`_rbMaK-Li@qR00+q&_` z3-#;WIJ8m?>#^Kb?c>be`>T)g7mc%J1}D**iKgpQOstZHv&I#huBd+=>dH4R_0O9& zq*-I>BB&KW3X|2`XzntXv%!IE=h^DcY$K%Rbm(LPd4Ys4ga z$I@?U(@Ib-+!Ki^KFdV@s!w{q2v2_BLM84oaH+R#H9_Fm>=1wd^ijjuAdmO9;LJjB zZDX}ki`}@3UpHSYGKYP{5k=eg>>jE$#;G-K;GOH8BVcr)BVeBbPOb8tS7O9;U-mVF zC{>2jx?XxpPG;QOHqvO?kdgFR&PfSPXKvDOb6e@5N1C!DY~6X2ZQ1zdC$E33u_bmQ)nAMC?y+aK*n`N=PH zo`yuPU)oPwU-^{cDMvyN&d7a4*u1^C7k4!#i`zRX+LzZ9D0*{hk6bY|;&=M*M|}l8 zxf2)-yI*e%oN@#*96cjnqenA<>kZwgUb9EH1Tq}ELq5Yqy9Ky8bmDlE4SV&dIneqK zb9G~PrtDkwEIWGpU4Q5FC43gb)r0h999|B5a_A)Zix_POB9OR`)5|=Jvg3?83`b#h z9((AH%sW^I!{2iwWKVyaaD0HRe%$(%L3u-6+k0bKJ9>l7NH6NYZ6W``cx@1E*2Cf8 z&GNj9D**aeGMetXfU~*RB${T2nhSb5z$()h;lj}y%sX+PyS3K=<-%bDlwiLV{>uI$ zumhI>QXqXFWyd3$;CpxQ{&0ZZ;j_ceu+v|#YZNYD=)bJE{z8Wnx5%jf!2q;37a(yJ&{PSq!tkDDEH<00~p?gWlhx4!loS>{OG)%EQJ-SF29bFYi;!U2nsBzuysd zXl2o0Ob6qYZ(xpuNsEMA11eyS&GtXd4;ehHx^)lQwH+#)_b{TFKRE{AjRq?uw)MypuC6AHQEcd#M6~#_*2F#97 zzr)agf^%!6?Y)s2Z1quNKm&|t@ibNvtP06Gr?FpKd?r1s?B{T=uu?|}8&aDL*lZ!p zB_IhQVKjH(zsDb#Ym)E#nr${wHkenkiR~7sBAN7HfoXoD@ov@)n(?UV4%>Av`HD4n z@s+|MX#KZ-1s=AZMo6EjiDSB)iUbkIeeiay6Su{Yg-O#(pGfU`QJ~Sd`$l+P4N+Be z{p1lJo&6DR=pJ%Y=vl%*{)TK8T(>W>iUftDjEJmXs%ct`nAxE;Vb;;@IFoT6PH5OM z(0~*923~Jc3%$*#x=Qx zyCkw!D??;J#Zi4tPtP(mu>Jv=<&4*?vBP9Xt5a*Dq;cJ;$|74*z%tBf5v>l><#mFI zZi$s>-j=b)pbad%HABr}ZkLiy;=a5A$T3olp&`wJ=9~DM8<3ZH!Wr@A2Fumo-Nz+C5EKjjP32cp#4SP z-P}@3mFFdD1OE|LF(_uLV@&6`;SsR4s7NUX=7uyzo-L@eDYhNDB86*0tT&Bq>2h~ zf;Q2cKja6}F z)RwAd&dlU+5ac6L5JiHZ`D}ycVOyGc5{I>AC{p~k^SH+AgRk#%_S-S!Tc(%hlylx| z!Vg^1&#jcq>Rp6e@@PEL&)F1jaZR0%|Hk(4-zbf9e~AZDUu(2X|DVUy*Dci9#NGM7 zv;ix%FKuAch&Ulgh%yw7fUSv&R0yl7NHdRe5NZ~YRJx+mif^kXCRHu#jayrY`&b+A%6@KT*dD@b~Gx@fQm9v4= zqbROsGwmlJX^_iscEU`DxLk%yrkKh_X<{aeOm~Gxu@OL5ej@US-WUcMqS|^+Sw^M7 zyMTE2OWD`hyiR2q7hHELwmEoroVA~}5{sa51mPY;6i;XauPQG~HceRwI8Jx9plp(2 z$>Ne&uEd&T1NYn%U0E6fu4q9{rvl~H@5M+Ij~qCYs{M}}CedkYrN%bv_QSMQN=Ozb)gBjK#YOU#5n+PimRb`*L6`6u z8q>Lu(KK^*ILnsuf7E3bZ}+HmL?-n^mKz#;O@biSV6uz#igbUG2PPlPQ7hJv0l37= zlW|Lou_}Zxn!@ws>0rWHBM{AU=-4`$`hrE`3^vQZlIsJn(xl0SUJ?sa5*&3pveIY zH%XB-Xx@Q#=UZ2Ow$STKOVD1!ul=O^JkV(v@uFDUGO6y09!9@s=!Q-GZI6XKaYvOM zdS}G6u&k|Jgg7i5lL0?7(ay-S9Vw^VAf~q}v6Q(wP)TNLj_??!Wu-Zjwg2?LCKX4M zvpbGxyqofo3|1pe(1i6L$+bRi6rsy66!>-j|1-SxHM}mZIu~$E4uL@N9;3MzE;;un#UIL%$)wa` z)Xr8|u@N%m=WX;cnZ;}2DQ1EbPKG)-gP42|vxT8c>V7$}AA}K>vOXZ}Qhsjd-gz;N z>O%(hq@4MOx3sj9D3*&0%Y9%TZXZeW^lkcyBs=_NF#d##XBLIDgd^CMtwl%b-4=OD zz)H8(XRimYx^WT^{cbWQfOiLX{S3b@n!?NLaDPE?1^VsNI^*!bx)E-AD$_L$;O(Zg zOE+O}K;IG$DLN`9jM!&F9rV1yDZcvP+J?oKAj?09k0ylOY@Pf!ya?^4$uIatyVHIp zhl~7A2eH$aI%{BNVs2n-Y;EGmAnIac<01J49)(<-%@rK&TrG@E9RGDR532Smp$H*f zN`cZqAtNd(qM|X0njngap~3{BYf_2TwYodhV{wSsxb_>P4^qE_`Sl0l-y0`E3iNzl zlXR*-%CZv){*o(_M(9Yp%zo1Sy)`=V@%pw;0V3PyOfj(A4$eWFTE9c?|Jc!oVwo(+ z%hoJR7{jZe5gu_d~p?2iGiBCowjV#PNE(A(B;TYIaeYZ0u4O2&tg;3@G zq-l6tx5)Td%ET`C(V-6gsjvdAEv;#4`VVCTsn>Jj=|i}0RV4T z3q!bzC_yNa#X)~86xv|0WyJb1Lz?{~#Z{zEJ`hj5ua}O<@MyH`BLtCYmdIO6z|tSc z{9k}Y8NQWmjT4uGFik=jObQ{QL+#?E<2>enIGC5=DqJ@wo8zaNv@JqR?~B|+&Ubvy zOi;V2`xIu$sFs4q@%ZPF)jMRQr*5ZTGzm$k&r$uo=;Dx1d;(G2`AWcBpuJZgOiTGz^{xaK?G2S9sA8{36R zkxh`>*P^44(FXOnJF%fWZw|;UAz44#s^aaKa5|}v?{hLWeV}il#f>QwBtDq_v^4M6L}(t(8uul%`A|67f@kL) z8g`l0nSr=$eQUlK`eoOoHEwI^$*>j_F~G{|bcYJVT& zz;y-e=jb!ED<*?A62As#{d>ZFPVj%>lf)r4YPg=MJ>0BgyWee5qv5+%-AME3C-v!R>@+*-e zFJa%_P=F#$XN(g%Q2YWl6CrD27tK!eWgdA!=Y@C0KI(w{`_=ijD+heK%7c9vsj$OFK(fmj`vsVNB2c>)M_doq_CcK6D->W%!Ds~LO2^8wS z5uip8C|w)OPQ&06C|-vKNDLlHm}H{74};AbG4ogNpcsC9?UhnLCr19PkOZR@DA;MB z<`0>UcxRw|&R=;%_c!6?w9XPoZ&KEJ@U7-m~$Q;G=Q#S zqOe1a!K@}bdKe7lFj67Cz;@cO*rddH!2;yenXVqdPg8Qwzeet!dBJl49>!GcR;ZF_ zr&%@wsQqok0&~VRu9nnA_MjlAL12VCxzMU z&Yrju`lBz?oVFB3xCpdH?<2#HTgf@--g9?7lg+qXS6<3sp1RVA2&DLTXbc9S%r9?= zy`VF;X$hgj3BTNzty>{@(`}9E?UKP&#D<=rlU)g#f+Fr28{xZ#S`~iED~b1=O4uzdh6)xklp8LLm8p((J(nt3gR8cj8P>=DmT2wMH%#N zHVrdMq*%yy9}$=CR}<-*t4_+9)y#e-`!$~21}0iWhTHb^$X~sy>JbxOL|ENUup2|Sfhed_LN($#L69-K_*m> zt(mPeKnC8*j8!FREbCw7-|_bQTbo|YX@MyF zX+9*7>34DsXdQq5sPRT-JqK#DcdG51y_Uuy0cAs@z?JPNaLN7 z#%poZC8k0r@zX4wFgTU2vR3lMH*g?j=7=jqC!#*#0n{QR+xJ-^eYJ>fx~%q?R$N() zxiKA~yDfkEFZ>xzfBcYsa*ghR)uzd*{g_K zqb$QyhdB!Yt(#qzoFUA7wy^}xwUNd5w#MDL>X|;c$Fv3SV z;SFFOkV(PAr%M&X210vOQH(L&Fkf^vf9$hVJ85pXD#PNuNN*3N@cuz^P^nYlJenMI z4Q;?Pzj<2N@<4xEI-Yg(Z$Q65wYu=vLh7|?#n~gi{b`7?b>*}X?RBQ@{AgHH^~BZ4 zyRMJ;fRP|KU<1Fm+1euC8Br3PYyJm_c=KL}bcee{+Par+KY`U)THJyST`Q2+`crh2BN|S z`oc7q=HA!dcBQg*r3d=vHgJ^h3**wum=AM=^0lK7)}+HJdAd9pm&jm!m*|;Dw2kO`TSK zo4Fo4#c+EA7a4wDV0~FLIBDtJkTNTNn1q$PjCAy-CsKxzw6+JDjzbY}Jv#_JX2=#} zp9ej5VzA)uM5<|ceIKr2%jQh;0os`%m!ZWRe@weewaVYW)f7K4Wv?D&Rl*y!oWKV= z127A2ZJX&wGYy-*lI5?XHrx#Mh!Wqd+b)O`dE0i2EMTkUqZMt)wk{HTTH(^TM0r&* z4c7_lA)EV)Sk~7bp<=apk&5L1%SDHcbt?wa_ix<9?5E>bKE{Q2i;`|gc zemj{}8G>cYuJFx<0@ElOx`QZs*d600Bw~QB)rY8TOvz(1qRvp|7pZE|ZRU1@S%Q&ywqHvNksfOvoM2W8-66Yo?v$z>4lm;IWuY2Sj)MkIuq(D!Gib8LMmw)~o!*mP=)BI{Oefb8(Xb#*%R%Z zq-l?I1ifskDx||3?ubL!!*qIh=FFN|7G*nRGd`2qai^L4nL;RrY;4p!+g1}i^@yFLdC~|RA)_3x+B8gs-N$3nciHcnTN56>_*A|a7 z9$H$R9nl&>?abSeW60?&+OhF>IiNOrEvznbW+}_U!cTB$)KC#v8X;^ zf`F>o2%1BaD1a_H23tbBQW*~`g&3xR#Y6uh59UgXvI(g!fcEAfI0z^O)O~NF&OWr| z%csV!{W++6p;(s@;bsr$?V#T3r$e)G7aFwbtl2}bg)yhOM?oXl5own|BiJt-sIobM7q7c%O@+;+c5ZAc~2Od6s8N?k~(JBxM{F<%v<;3F7^xFhA zKzKekXs<@3bd+a_x%RcWGXgR8S0FS0LjdPV6`VpFEKwhqC>>izJpQCasV}$P0aiJ9 ziO5=iD}!Rpv2?+2Le7X8!vzurD1&cJ&d`=+F|WAq!mN&ozKP=Ae^m(tWS@6)j(cS~ zdkeUFcFB5narKk`P^Oz_&MAL8i65L9{)BXf&3?+O`La&{tD32Ce$@&Z|5K(BH|8$7 z<-tnvC-pb6<1`OHN+}_6kJR;3A?PKSD0VWU#dTQb^+`mySfnO2`S{48MY&ahqN*dp z;Q{CZjCbIdR)swg01a_9#Zs1!eL}8Pq8NqmIM~2tZJAPAHfh#C7bEveAcj992P(Cz zAnh`awYOt}dWA89jlb$5mjdGsQ2e_;-FzmeGvlWd!&oV=L^iu%l~Cm{xOFe$=Xe2B zt7suTN`gVmE3i^0hr5N-Qgn(a-3b9nyw-Yih~|FiQ*0hp5T)heo9h z+~phmPbgMk!4CyCb(C`G>HPExuF4FeIEz`&SDP8LT>JH>kW+&Clpc`=cjmJn(+bLU zf$qnj_|hj5LA=oT`ujV$mZm+f?21lzaMD-P=MJgzpqK-`aEF*B6*99_6qJ^9`8nbY zxJtuCOQg0t)V6WI>)Gj6_b05Ja2PyqCadHb0)tMw=EV?@NH>;8s60mu*@&`;ChVhu zEl#Wph~A;P;AUv|RH5)xC!Zmk0hZv$CsRiyh`CI0=d4y*r`%HvVapoU8n`tbs@OGP zK9z*`ODZDsS1G+=?y_kVHrPQju@ zOPA(3XWO=I+qP}nwr$(CZQHhO+r6u9b#(pR5f#<-x}MjZGjrrOawv(q>2$&jT*p#c zIxnxO1@i3w)QZ=gb8Eh>;TWwogoPSC@)w=UC#`Ltot5NVpJlJT;C#QK^iqD062$0C z+E)1fvqsT9h(^0sZ#B`Y=1ngT`anKLtJK0qG|H#qG#vdZ@}xox=)w%o^b-u>kqfQ_BDhm12QC+Fr0p zRd1G2eQ&De8b+}Uk45AzR!+PETzVeTO`Od4`_LWC=D3c}sw8CVqN?8s+?mX8BAN%- z05dYw<7;7a9*wiyT#a!fPy?W1KlazN3c$Ve&{Co{OJK1gs0hrvR*e}CCdGB5X;LPV z7-Ojbi!mIIyEWDbby7DYC+>gb*v{KFaOmggMv^O-j{ZjYeHO7${liQ7CCpC`3 z3&VO<6Aj_kR%J| zxo<~J1JC-cuWiQ3WsiyC_*9()5XbJJq0E)kk846E zab8A}cIm^IRDjI1;ssgb0>5J^S{3LeSTu~(o(~YlykL?(5eEp`|%z~I)LAgS~ ztqb4zB;yj<&IA3;fcACSy{2bN62R_=y@$gb1Lza|K@2Upi{^Jk{_6q$6cvmD`K>?S zVE&Kl&;R)V|3`7|Qd#>Q2BQ9K!J>i&=*r^l3l|~7kF_qY6N?(fF{AhW@0btN@g(|;>Y8o8ezeacIAx)7Mb$V@jlH{04KTE4trKCZtWzO#1z zU=Fb0HR=^Y!)eCe@`Ea&mJOML8lb|=?FxY*;4SLcW7Fu92THh-1x~c~XQWuvh19f- zXQ&XCesW;>2S);G@lyw8rN-$7&iY%x8w=3JrRdW6@=2>DUqhyrUz}45IDp)+X6tEO z=kAp+6ld${rl6X;Lr&bfqm{FF5Tp77I18JYN@65)Yfj2sOtb1t!b)$Xn@g8Q_x+>* z2W&_3&C3*7k(87IDP_zhp&bA-rJt7-XDMJ(S){hpv(pOZJecb#&LA>*6`mr6Z^vF8 zD$$6oYi?oGPU&yoE71#btb{5uG;7H+Zj6%yETXpM7uf5>lj$+!=(0ApQ9(Mb$5Vsz zh3Dx_Dm1ClS05@FNWTa>O*zviMDN)-P^U1E)F5N-MbTHcH+W^2oIo4~ADua9*H`QZ zFU^nT#tQST0IEW6EUYYIlVVlhN+8S0UgX%oE4IpSgDsEb%FN9Os%CRbQ z3j#=Vv?YCC0xf0enNko?T#$RlCj-|Xih8^eX}+#@N;un(uyUZ53;h}AFkKDQ&#rXT z;-9~Pjm|7V7c0fD2py6T#iAzQqrGn*BYNY!RE_?uZmP_tp``N|2_StG^hj`2Ct8IE zcl<+!ZF0=mg&Un1cmXf!v6ULvpjdM{`-t;{XEKf_wtXjN%?*=qev94WUJxqFJ$gEV z_(NM7JGAejP_BRljj8V+Fg|CHf(3<-;#v^$8y=4v$WrhWhX&c#X7?TuZe~oPH(+SEU>=^ zlNrf2DdDO#Y~h-vN^$_zDYlq!hrFG92W_)GxelZl3gvrhnq zGw}+TLqFK!yie$?3}#OBhNZsTNlp+DaGdPq>2zeUC%zg4Tx^~ zVjd}<(dlftJD5JNTrh)HE#hRW!dkDxS`8z2R(c=%HfC`FTSdZqlxu@R z_c+rAgDX&3zufTrNRKxfu)MShZ*N)Rm?dVTKHKggqmq^}oA=Zoo~ZnQJ{$Ry7vjJYZJZP42A_qsy0p6xdYN5xn?@W9{T zgcUL$+xkgMU;2a%?H{@Pb*aZrI%pR&(qFu#th~o%;{fuf+n1*tB zalC^0$^Y>*4`2Gq;87-zrYk@eOAGKLjQuZWVY4u#bAEvOnvU;?U1+zHtN-xVl*(Be za_-G_4qR){aPy6<4~mjP1n4(5lu8R#Zm>ipJbcR>iYvTcCT>DDe7wtB$6OF8t~(tj z@!FO?3WNfSG_tat5hYX@wr}s0spa2!uElJgPT%&u>(v@iI?7ybxJbYb6xOe;n zDuubw#?WOPp;PGak#b2Z9HIQ2PISVG5Ytb#`#s8N0;%q-5i-byQ&3`B@t7&Qy(^=2 zy5F3-J~{g!OfsRwm*_d^cBFxP(5&22%K$!m>G+EdJ0O_@w0W;gMcf1{v1>B~M7lf@b6hQx5eWYr zaW^AJ|2gmYBYytvE7yTlm5I0Ji{TwG+t(CRP1o=38CabNlC1&ywjrrM724YQudmD) zQOf4jZ<%QGOTzK~|5rK0tbfNS|EUuH^T~^CQZS z6EXi~K{+36)5S=6wxX)R0@9eTT**4l!U3d#@|$<6*&q>gEY%1yzX-*z=5IogYYcv5^BSwl?_6%B+iT$ z3r+U-Zkosnuab#a8n=2QyPS=W{&g4Z1$~F14eg0|=LIu}iq3Yv4d$1L{J939xhD_v zfds5-dq@V7JeaSZ-IZ@3>+bfXie-C>z?}M`UGc=At1)4<+u&Hp9V2Pqhv1Vp<&Y2h zKlT3#!*`q&uNAwA$n2-ux>mww26 z-~fY%+p3@FEFOybQ znj<)py5#HB$+*>7WSQ=mOhI+P=eLioC(23BTB!hFBs~K{h^_aSmoDB$@CUXWjw1HN zdBg1$H?byRgDqytkjqSqF5ko%@xkGAoG(E7i3`P!f~@HzP2QQX?UL`CZJYP_w>FEc zl^F+1Gir>r0~c$WH42=t5rgB?3oPWVC#K;{B`zs z^rgIN`1{SV7slP){kv)E9|W0ZNl9Jkl;X~Dl&?V~YIg*q*2pb3SBdfUgYDUel!n+0 zmVy7C2Alh%=60+;D5a;n$!xc*mHg_WMI{eRa$Y5p7fz^~_E^#3hJ zr_{ha6^GD%Wa6hbEU`iNLiutdK&aV5@k0sWAjM*JMH#_nclmUjb>c%hK9;#;FUy2H zHkPxMJ=Ox0t&l7nL@8DQG(1|Go1SZ{x-=|q0`$*(j<2kpS!i1GMruCm86J+k9=vuw zdn}IcPI+)VL3_#ThC-@Z8{mJZsa~i3AchZoIMM^&INX6S6C)jjV_Wu)F_|BsFgQb9 z_jlY-AdH(jVKIAuak!)9pbUP^O0B_QILCUM5g=Zyp)#hghCtZ;csS*Ds?hy1sh|g+km~m4CNve@EV&^j!u8ScIwk0S!v0o@g4@5NGmqRLtw-xalD}-+=6n9XZ z#40`t^gqeLHWI)w68<-${v?c^Du~P}?-lQqtqS8OA0}9-YyCn5* zWtrbTD!Qqs40nGWQ#E8&q>m>Hc*`U-%nS znyt+R2J<|$Z4qJ_rQ!MgWf4xKUlauvShgj%9!(-CvPwH7sO@A0+uXADsAXU`gs3DR zCX6O)X0>SbgboW(;b`!pV-Q_fMzV>zLjUHU$>Va6qMY@K2c{yVbj%c-kH#1{HYPv^ z&#?1rqOMq)t^s2ao-E9h-JT5V?hGN1{<4oP2;2q}VVHq_h;{}JSsx?$|z zQUm_S@$;pdsx1#h^h9)^?5J8?ur-WbKxZi!f|1o`TrBQK)Yq8ln0-cJ8-JYNXA4*u zuiRQUAujrZ42$fO<#<24T?$v{pMrjONOY_{JeJ9C4?pKACSmQ;0VkaXmZn4!c4}34 z7o4U<1ok#Rg?xBPp5c)Ga1@88Ml~rj;$e`M=xOYSPo>}&k6ll!gM>of;)MP^oL;(j zgC1QU2=@xUq1%tXE7l$L!3WuYD0*i-eX&WJcPbt`fc$ zd7@V*TNDRink#LedQBFa!F^Kt5pv9{WzBxUVbS4Z`}9yTX+SR|3A0WC9A7%y=^x8f zDaF7pncv3G3+r_6CCj%?VjedK5=?pqWm>BFCbTa_(K~zl+`wY~0c!SA9hAyxC~omV z4r?{tY#=NVJ3p-xDr(|Qemg0WQTpv^zNRUAJgVUrO+RMFCE!3?EomcSLwv^Ql4fvy zpMZU)36PenP!O?adnqD&mMQz`gY8N65TzfK(5N*_NTed`8F@AljYKQGB~*3dg_Tg- zeEtlb+J*jaOk1v=x)>h^d|kn~`&xNH7+ZTPMR{TRgCm^|B`m8+G4#x10*=hQBAFH? z;!-=S(&DLp9IF&c#1Om|C6dy=rJ(ChI-?)~`;;&%xwF9U!mF8NS08q_kA|dVicbOO z%^YrwcM(j4l@9`C|56~4&j90l^@Ygyf^++WmJni;kC#&Rs}>NX#=HrqJI=k7DSE$k zimyYbd?UMu^h6;O@Mg1yj zYS;_LxxxtJIeazk(ugPh0BY*^aiDNk+X{m)YV{%Ev05pVt9${J48n^DqXxC?l5jAs zmdCwve~3^JcNDlhOViOcVl{yMB zq>cdCpAgGH|EyKuS1t;&+afsXI@o0w>9o*^JIdQyDdSp(-#}-j@s*Di3CGzzx5ETw zG=5Ctq8J0vTvoF+Hi3y#7*SXk9o6hH!df=-awNgj53Wa`ik#G)E?3XAX#~WyerKf3Qwtv-rmmM*4pCw>R|8S_~6j! zV0UqDgjoEI1?4ABY`q5@86-Qj7fHfn(7RVhX|86;@i0+aEDz3d@ZJt`upS;~Q<)Zr zcQ&%gdiaHXsN(;!xiO3{DxXwqB#kh6ZBR*&dPx9Kq|T(U&pJ8VkeFU|P+MJHk@#|L zb-Z>~6~v5q5+E}a;9-l0*K58b^3p{BWRbXENv)^R0HRC*x}p%7h6BCz*Q=(?g8)&3 zN_w$44R5J}p%^FuXyOv`aQkc4JMy`6Gj$BCV4k6UVZdBL7Q@0st@Ox9e7)PVhGe~PJ6&g$s zbdIGYI|LTmM=cEdV{|PRri(a4oTsY^(f>@ql95+qN;7bWwHA)Bff;RW+Q2{1E4bwl zNsAetI|(l>G0+MvgBwlE235uWkmDT!Wn2|`nj6xrdYF&kLfgvvPpl?aj3J#GRd4{F z3h;b@Ymc@RC{k%>XbPC%4Rx(~VL2}DxZCCSf4Xu1AXlMIKw$KXqU6MiNV*P>s3 zeZVH&-4ruwKD2!tW%-L5!`>=94C|y-Axh+*W{}`5An^NoA z)0=FEl7QBK<{bcZ1*nlIU_PNej?O#p4oE$DH|_?^RV-@@?dMD49JItkQWp$LmP<4K zhr3kmlsH=6MOD=tA}($!-f9jIOe^V45Y50DVGNZVyv0_kRb>l?5CxjQfYwntyGr0T z++0HE3|QC5)qhASr>(Bhq>BjAagN=08BmJdic=DucMMCX;fPuyW!)`qMGXc;M&2Ew zv6f*_QDjClD;s42TO!%UM>9xAxfnY-H%P$)TjZoVp?$6)M*(ec$C`}TKFA%vlh{xf zPKJ?iDGx15G!f1ZNNpz<{;kyi$(RMO4_%S;<$%O#ZFaKc+yJb_I)^Yp0_(!`q2Oy5?@t z6rvNMl!i~MJ%{&q-TUQ3_}2)IO=+Gog~j+e#_(J#PDIC9^-@dUhArw+2qx6D73x5_ z07$DD6wIDprobOU*{;-?XQ<4sqyiQ_wrzj+&ZNE{=gb(FT%Z<)k4?XngRa6>1eXRL zUk}<~k1>{=tSN+FrsN)BzHOeHfT8xia+L@VJpsHEKY?GSaov86b{mQ z)y16JF|}}QQ7_WR&F0hZ!5GCHSWV-RleLX~bkZTBP=>==VP2Ab+;p;KC_+2GL z*>WdU(}T9u+yvZ@r~~Stk=bj?m=4{vLa%%Pel^B(;37SrMT6ZG+-|t>V%1xaLkw>!j)+>7637Te1gE zv6XkFXHf;F8$O?N>vD}d!4s=*SRRt~=4ru8^DjM z^b(Hqfd#Ab6amliVgl@Ouc!q_8L1ICv?SbN%AcW0KO>auGIhw`iA)zPfEQ-yS7^dB zH>eRkCW7C7cCRG*w<_uHH?Qo``*xoJn9*bfIx?y)v^CveHpDl|=~q;YJ;SS-C6hYS zCq`#2HAZmu+t+*=?^IH`n19V5l{=-{1``r#wE81#5D3rU9ngMmWThJu)%Kj{-RiCP zXLnMIdI%(ZK+cLX77IREmF5*?yA^p-Y;tdo21ZtiUlj!fpUk)H}KUvcUU@BDA7s>(yx^`bvk*Y`g-gqeGPbg^JdBUi)---zA2)9PWcH>kDY@Xazs^O)LfeLH-oaFSmIze zro|Y}jFtxSmWof0D!_N9}+?5R=$v7yro6%uSIn%>7*y1JTsPn zv<%M+Lp-Ra&{AiCU`m_l$WKfOqC|^~>kGfKL}Scu!P@BPjSV#^%#e7%Z@I_MI#DBw zFq2@9IFo~SuD-ONNuIXdXBQYKodgf|Lfe2-8K;2g&pLnrnjfT}-w(RjY+7p1x4gY) zb~Dd%3^$s2;W)9kP-}%8E<6BrY3!9C;v-JU2J6!%(zA;}F{JM5kEk9^ri$z49Ph&2 zLF;SizuE}mkmOc6gcSK*c8)*iTxkV=aX>ysEN@lU$dj$MU%{!hD@`qBInI=nJ|i@` ze*|x|Cb!^^CyA80hpi@P!Ub-VjzZCS+E9O~uHUa55q+{CPR0r_LOW>EKjHTh-bN&J)O8nvO)$z#ll@BTD{sqv;SK$&119|O_Vs89l) zesW>ks1}3WG@~toHR|Eo0KA-)dbQ$~sd^GJoSH+%>7P2u5mG%}B_eLbLcu7)=kW^v zc7SlbPaiYq;ACQqQ&b=~htR|2@T$etv9;IZb>P zq@-Xtm>qo#{oRL4gyaVR!!fwkC``W0#@04Ak1P9j#-ps()toFVt}XM2%Jgt3-!}C2 zplN@zsJs!iYZW19-9qERFc+Nj#=m%+69>vGb-DF|^t7%kSHN4PuaKfJr#czf{j~Lc zMY5BAUhHM$uaB0@3>_wjz%M`s$DdU%X6;J^qVH7q?!@T%42!4E*i6tFT^ur-K!`7n z`m*#D#5iWn82=fh`5mtO8La&oJ@6W^`5nf$eg^4#M=q_@?+R5iMr8{ufb?QA)-^8} z{{RC!$L3bXKI`RM!uWj!fbMt3ysMT8p}Pv%S31dp6q%p%^(<9tqGs|4-_e`>rdqBs2!ZM4r(Gv{-Zj7~6m*VdxIOSp5 z9i0M#o!5drzT3RsMnyuw{luZ8$6nogvpt}SRExaar;Du~g2sWk%iT}mbW6Z~DdM<8 zla7DL_=WJNB4GM_Zj7HcEtq>b5Rt()SctA)eYc)@Q(x=V|7rxL5`(7!JBM?D=JS}@ zEdzDfE&m4Dde_N3mR%Fq3ITo6bVKL0GaC`*OT5!B!JFz-f$xV;n)d_bzp7zfx_*gq zzZ*wEe@C?cOTOa&QNyI1EFI1MXDN*EcZagFo~4t~f3))=1u2DJOORKB)F?;_ zkdWBF`fyjd!FpLza#D~|*IP!*;{M(GjP=R<%ugzBMYmi0? zJ(G5wm(FW%hL3}jmYYAuwj3dd#0{eQ;xsD@weziU!&1NpZL3D8nrr(UoA#|54azLd zfmPS^hAW0sN94CaMO7(aVpp~#pk3f@Tn_U|=Gl7rvu>RTzdU+8gm%*OOO2=sCs;E} zV9zuQW*)|^<+WlfO%qOlSQ8lrWLo`dm0-}6(Qjg_{{RLQU)%>pwoZd<)<#DEdLl>T zs;1pYh1y2vb%ePt#D(21X&ZW}doD7UPe#QZG0s%1O-+Y}&9XK-_7mor5dM?}8bn;x z$xRuU(y~7cO86F`3{rUBEZTU4$a1VQ$5Vew_-CA|<(V2xSzf8TsI{vf{;{=Q@Mb|u zb1B7HTNI=5=nGH{wfLT**oY^`N&QBlq-cs>sYLAFn&zAI31|}as%OfIovfzzQ)hdz zk2r$)lT(L<_U?wh+qopqGnu5ATmfzMUap;>uD z=lXmqbwTb^##fFrt?-X2_@jDkZ*F$XCFR#B*x~36*KUTMHjZ@cmdq%{I@J1ygifz` z!iQ*L@w46~oT0AoGG9KhM--t)0HsF+!X=h8JU22{!7CYVV}AYmJP^WYi&0c#y?qsB zLVbR-G=ej*$w4u8Y>9(>(Tu34Wz&2C-!nI*jWC$qnL7HBbq1aH+meD!PsRS3f3Izk40oM@I-{n2G;J$A z@wXOeCVDx6JEE^t0X+JHAZW^#%<%$xOnJh=O}Dk^Op$`AH-Q^xc`!u@aE1J4?+*Tmo;$S?AVFmsDxg%T*bxm zkVeqZe)z%r*j}X7>1Tz*vxnO@YDapTq3NRLbG9a&am*~JV{l8B)0M7O|*4FO-p;&(^P;+O{aESmPB}YWpU#m4fQe5Wm}>0hT<$2 zc2yKy^tnVHOMa=t1vY|VkhzW8T>k~ z=uVqyg>0AOl-Ne9^fGJ4NG(W`gi143kx)`0w9+7!dHMl<_+!D#;{q#s;B}##b7(1& zqkL6XKIdQ|U?*E)2r}oxU5pFNEr9c4>k|)L_`f{~R8#;Kpk-82wnY7LkFg&sgT#kbP z;DK{bb>n5h>4BQ?!1TraW921GVzoK0yC`$>)Rx&&AlmxcdHEWQz3FSq2f}S5>IP1I zV(^9$MS>2}V*KU(^Y4H_p7~mShloHF!q2d3Pm^dceihV;EvF(ybG#YmeYP^uGROsSnj216SMqz@rlKi5f-)WQTExCB z_e78~i+5zvqI}@unKAGxp~ebCW%)DBAJ*tY2ijF_MRty?G`(N8Bj19@>huO za$St+lb9Z|(?Mb&jn@K<3{pJAPb?Zb%E@d214$gy!x4j^F1ZrVqDr}2n9l_A{wySS&;1tKkQw|=T)Kw#cWA7gv z&W>b~#zcj+JtbZq8a5xoL1eB{`VEN88A;@p&@0NpDTvC`O8(W)W9wED#7kVAzDUd| zN}`qWdPe6v(eDORt(gM&{nciBa>zqL3@t}nI>q{soUt>o`PbM!aW&H zwK+bww2Q64dDH=$G9v}ir@{AKq;FnVon&Xh6J5ZjwK?u-ZRhsrEUurJ%t`H&#m%gy zgjICjZ6hdQE4Avd`%!k#Ou{Qc^2up1SPsH}_Sd%Y-H|=a|Ql8ZsH8PzeY+ zeVQwd*9fzZ$4Z46imQhoC74oxVGbJ55$Qz*x7+tu0`J@VTh}R79K5OMLPo$-Ri&q? zkk|m84o~Rijx`tA?6NlZ5_(XwrS;{{*#|<+$jK50Jr|^2bgt<1jT&xsp;UEqm~`gG za!kCe8cWRhan5SjUz`VW8K}d}?(n!DV3kT*;BRWye9crT;`4~%)ibUAgAGzs7iu%O zEbE7BY_mrOJyMgDjKrvn2AwUEX(hyUOphIMD`2M|I8DFB7}KDRK3A34`MNI~LgakJ z3tu!(v^(G=Vg}6~5?REx<-W=KV@mr+u>Ugm$jNr9RR(V6TGWx+AQrZ8lHgsvvE`az zC@4*L8@wk6FDa%thmDCi-HliITE-!l%8C1n*HrU?sd7l|G-wDO(~ij+32qhLwW?uR zOO0AJdtA9m#%HVZ#M8da`W0n@D4!)2Ce8vSf;f)3s^*xWOp;i6z^ezQ>vW{=lgx|$ zbxJsWtp=;|1B0U6AV_|GKw<{Fetxh1nA_G8)7Jwpgnv{QPZLw3L-d_WR}B7g+=1zt@sm(>0X zsXe6&(h4L5Ys0mm7%2obaI=_a&yqW+DN&uocQjohExhl=L;Sd9J+w>Ygf6PkLu?SH z>lmYPmSE_R3ex8y>7WKjmfq#(PfkjS(XWP;r3)X|sPz|Al+va1iPGQ3D&_P)$KvDB ziNRwHM@aufJ_bF$K^^eF4=PYaq%0gEMK&RQHgKbsgCZdxC)#JOysP)4(4qG;|AuFt zwATH$qqK^DHsDl!MO5~%qawUVAdwx?@t4?xJ!$iQu>GtTP@S$tv3~=Pj=|!iQbREZ zH}ut~@TCxua{+yXTOkBF63C6I*04nnK~DwM0ZT!m91B(e1AZ-^3vuut;v4dR#>v8U zykqT`l9x^Am6Amt67v|NqG04310N)n-i~Qwwuh{sQe0lI6e_tK$b7$N+AI@bZTfSv z$a1@mF=F;~^w->`L}T@f^ore|!6lJ}DKhV^4uG@3bE*4{H6R^gX1t;Y#nYh@ z7Bbotv3fEpkKS4YBx!HUdN{ zm{;M&?hGNM)UVm^8r%~h@c4T}Pzqah$)+hS3%AxjG0%sb6x*Y6Oa2r@uOuF;RRk!d zOtw^C^)z0&Od(bwG(_mw0a-UIe7=dNu0&Hp5!6SH!MsOLHY2)BD$` z!#5GFWT1L+Jskt)q=YP|tVO3)z!DnmO~7p(VXuu)WlbmJu89Dd(<{a7Kq2RVB=jk4 z?X>eFgD}GYEf8KGj2PAzNYR~Ez5~8xKKJnV9K}Sg$2DRj*#xT@&{8Z-O^@cw8tJj+ z3+KOf($dPM7oopgN$W3H!uEeAF8&YpOx?{>amDF7nsGkCI9m-6RM3*Zk~bib(Q=qs z7n2gtgBR4_5EC-&3zcgybgpx6D#0p~**a^hs^R8iKDilBI*bZ0ZJ2m!e^9KUMXa63 zEVEc_VR0*CndxoXqhK!5I97Pk2l4B;yCc)#dhMF+WE)3@=jRpS53_hP_Q#znDhO?P z7oF9#8AJ7yAZpBkXKEME@?J^Piy)xPK0g*W!f?)RgPtdJ4$Ey2ZFWCO(_6e>$$)8Z z3ep}$`)3LJ*Ai`aaLvUn2An6tZ>aIcD}&an!`40roF~f8LQoIr0nMEX@VENhkFZI1 zO)AgqaR~T(3$VB0E*bz2G(q?r`)8K75B%`XLCN}u8Y`+E3?1P zaL?(k4%BCc_s>?Z59y%K`EF0*k&f_eMp7TFPXiU`BA{;e|9fo0JQ7 z>mCat>DB7q$dZNTjkNlDH4ci>>ou2(+`JJx5QDYFx+#G=Y!lmA=N7nmYW~ipkUDzh z{Z=fF?9LFW{`<;A22_k@`f4uP_*#P&C-v>W9DA(*AeI)r&MHZ^y#0HFp%?3BHZfk} zwauZGOuMXUs`Fu6d&FE9rsLO+!3@b2Tt}oKuCwA%tNus~1@L-$(0buid>2JWsJkt^ zfbgoei{tA-|G?D8FVRVHl|e-h-@6ZW*YAZj@7?R^E3c3&43O7cY=7L;X6&l?KA0)E zV&#`>XaWu`b`8myEj^TRc}U*FVU->LoR^&wl*!$C;MHBo_yN&tS%%xebHln$v6)nx zSMK)DsITYF3Qyf1Jc27dDpx~OB!`~W(fYMiI2*W7iz^>>D`y-M&ibs17|2V{B3&ID zxa}D%!I&rY_bOpdFxYwJau$=s{xxkRoE5eRz{Y474;P0uKj-2uofp7^(CU|v(Y#Gs3BPiXwHNf+l1E;0up5$mG8|*MWO)SqX)gSe>tnC3 z7fp1v$Xml?iGnaTgwLmBpQLNYhZ^D0Y%!;+iSQmcTTu?YxwasZ;x&mb<|ec+5#Pfa zK_if^I#-vKB|c%Jf-P0#A2K{_LZzPyFcR&+@Xzta=Lw#ro||^g7O49pJlXJ z&fRh%a#(u3a=yP7ji{GISkclTmTMWARsD3L^BFCn@h`EWJ1)FtIBKj=oj?_=X)_LG zUqGf|ltH#}L@eN(YTwYgnwZt@89|mcKjhzZuRakNWl($>Z>-RMq^w}AaJFBcapFde z7eu6yRzRW3{-wD^A20c~W@^+Tq=RZYP2xe#jDj3>Xd3f~TECL0y=F4*&8)a2A&J?a zMXVj1f??k~sBx>@xAaCOR^ujqGF4sUDDW3R=TS)1!}!d8=Yz& zGV2*~+WuI{Q9@R;*x+9IAtPmHHm=8(U*BaZA2rlBo|rPB)!=R+mPmJGU6*(%rvOi{ z3krsnz&DJh5Pgt!s8*ywyTwl|umD>SpPhtITxhr&+X#(`fV0x@M%=kJIN(~KyI_R* zM}WRKbMY0eijjD#aGHKB<(igAAv0{=NrXO|OLjMTPTT-RLBQ-YXACh&iJ)&m`P@05 zUWd4{a^(h*_H8~qVjAt<{EfWMz}g5fi?@NTqWEwZln7E$u^CeomQ)#V>`Gbb-$>xF zlxm2Ogu~Uk{n^VrvpDO(fb@C_tE_b)uJvTM0C(6lfq*v_ms`{ZqXn44HF0T>qDr5U ztrgoqvRr2I3(??|Qsq`PLne#p%;Yp&?xZreOj5u3&R6mUk<)?eJL*O8s${+NRiSD) zFTRz-ADCuB0LnXhW|rlWf}n&Z$v!A=xBjtAa{mh#XPM-;jDm$-lN6;S_bnr&oyac3 zwq5F79yhz@(ti&mV=u$tF@~MD(DkVpvXKDKymOC`E}Na2M3j@dFZ&;nV^UA_sSq4g zMl()$$A*Y$D4K!Lt7s!niaaedmx&Ht8$gB=BQf$~{uCm1>aH7*;T4BQy5b!T4Qg*p zN(X*ASVx$qQe^ozvmloSWWe~@qEY&Qk%dN8W2a6L@cTaAg@iC%D+dt$84%eD9Y|R$5Wli*G98>(c zUHOAa6*f=7oKx<8Q2rmrcU9Cm_vf4>rwze04H|0^mG`af_$!8z+1?U}f!9r%$H(ux zl^-yF<8pI4%tGe9`OU+U-IdNFwZ{j>9PxiM;?bOvO-|~k%cqebT`}oDO{ba~EO*Hr z@kd0&r1J-h<8b%B?d3mrLd(A|$m)HH3I-?gQy3LUFIGr8S4Y@Fhn=JYgkBB>0Ov5m z?G0t1L!M_ka?EdnOu2W3o>D*-yoA1 zY@L)P`i4zABO%J!?~F}I2qusS#73FgW!$^yi4bhOabCOC6*(LU_{@B&Ue z8TogScNGcz+<~9_xV=2|bVj{j{N5>`ls!I(Q19-0FK!dhJpruQIx_bxfe79_M?N*X z>$LNj9GGA1Jxia=P1yoU6_NSs8L}1|q~_`?QTfl5C>3#f9hgm)FSvo!iv}pic_0zB zC=;_037TfmpnXEiCM{zolAqc2ia;@53_JyuK$6k6*VvmG24@(hd)SB_R+z#z78V{C z2Bm5)&`t%VGcf&C7sRl66d4As`j`*Yy50BRsO35Qg-u?N5RRia{7J{0=+Hp7C$*-C z5ha1@0&I;ekMJP-`pw35t)!=(Eo?T^sptu##;@QT!BajyB={pE;GfkWWW>R0d znk15T6q+l*)`C?Imd+}C!??CrzFUG2eX9boPQt_9T;u~vn&@rgv$}GUiI835-G;2@s%UowUug!$v_#hIbt(uw8rE?1 z{35lU&GWx^8344dEN+Mqi@7{?ClImM-7^f1!8!$9_oIi*i!3UFj#PZ$Q><8M(+TV~*293oM3N*=u8 z)PLZ=E`JFl`n(?hnuCg0ybTl7Z*PB_y^~jF@4f@)d6!l9d5hkTTWfn-L61(58>1Ua z<~=W0o~kGA;C-0gfW}#=%rDr7z}N(t`TZNBM*+vrnB9Uw*VMl+2CrB<1ld%zFUX56 znueVf%(orD-ZQ;spc&@0-Mkd88ajRz7*GNtArauZpuRX9aYw>b?9?yRt1mo!S?+6TWDm!?8jnM};1f7Osyn=2oY5{jq*4%kF4F|m*oBUX}6q*QEVd_Y&&pOY;yGrcoa z=&%+P5a$PbsmSxTqF~Y+Qq*^6WGp3wWQqr`$YUx?Evu4B3rlR1X8}fPIB!H|q#`ij zl4xo*rZ!N;q?D}@bmWRFX^{|vX3QvNPo)j|=hU&L*p=P4R4$_U_aE7%DAaSgaLPPJ zL0r8vSLNU`MKJMpLx51b(W()8mUr@$v)9-x`_Lsx1EyEt=x)r9P?>q zZDihBbbC5Bd@zKTlu0VGpfhx-0HnhVu9%0ll0Mn`LerSK?ufqnY*JpbiGuQahdP#e z&KWfihtDBm>@qwBgwu&pyweFggY5nU(K^QIv|PF}DV2 zF}rf%Drh`?GGs7DyQLF_eeI#fvY->MXxIf9ca3H`m;sDmm_56%MUf+SfsrIa4~W^u zr>rfNyRO2+HMbUZXAPJ$W3N!&78Y8}nP}8rQhpP(SBjHHpU^WCu77F3DYW7LV3B}i zB3eWYEtbMtFz1)?g?H6Kj@o*EWBXa9ujSZD`&vix@^Brrqn#=(x$=eyE{3w6PtbeE zs2G0oF_Ldn9mKpMjN};oUyQv|aAyIx?c1?!chIqI+qRQ`)Uj>bwr$(CZQJPR=BsmW z?Q{0Sx%++9dRn#Ss4>U@tz8B{fud@e|*^$Ch<3qkV zG2cPXh%D#+)|nHB1&;^|0o6F2Xw;FlG3*(^Vz+x z5H*Zz#*wHXKNA+MSQZ?u`6@Fd8Qotx`Q4f`RS6)yz-<)^{c1-%G>Uw?+ z33qE=QI(7(-;N2fRuhbs(0*gBo7v1#{uk{eHkhi3jZTe9qjNYt#V#I5Px!{ z8qP!?6ZGH%w1zB(Wk{0JY3HDc+NOIPV_fza>ys&aY*Y_oFS0hQ?=`4G{ai!bHf}2_ z?K$0m{drGXv0e}a;}mWs7I+0#P=P7tbbu`lJLpS#5MEPAN^7Rn=FzqN3QcQs^QO+u zDF^5bIs2H~MEHE{2foGpZ4|$A?@eCNP}Js(*e&P!!o^E+QXxZC-Kd@;>QVvu)g1<0 zL8ZdIl(!A`3Cq6^UOt+fZiqEqx0W1&{z%cX<6qj|>*e_&#%|ar?{D-U#HnY(FS2YK z9z@MKur7IN>q+EA1Gs(>TVcq~7{v`83Io@DlFoFTs|t<{?{#TBzmo8T{CPqNpAnsF zL+s!@Ks(k7R>kfiR*lREb+{BFK~3_|QUv-Dfm;0^-)H@%TwzrG3NpUdZ8( z38ltzdC5Fc?$5BCYYm5OC)78Uhd4fzBVQJ$Zzw7~o<#I-M+bAtrg|E_#pqFlwMs%^~ z7J^so)p35tRdNkkh1=L(Ys1fv1Zi%+8PwBl=49ffIAo#NV><7`3mv zuLzRt+fO$8Hu&HR=^{-zL;5UGTxS~ELdxd9i`q| z170*`sfOQ6Pso3*r%kEcj6hVLK-4V~g~Fh%ia3YwD7YB#49eAYfZi#DWYkV6J@WmZ ziY(1)pVIFqwwCcjwqyODDYE|@aw`9)hFHSd#O8l#uPW8|{}iBnSvQOopZ!pa{q9x8 ztQL}QDWSnYwPFRO`UIOq0)}yPBtB;Kd09|uWksuL2I#3K-9x|`Tj{M1cBJQO2R`N z-8SOFkL}t1A_WJAG?EWNlNl2!lDbDMe1b>l!`q{O~$7C4J3Nq>O` z4^D|OtRzj=+CBaVEWs6{7@D}rg{$fa6yD64Yc@WtmaGT-)$uYLDp9sTpI5?Wh4=-) zts&TA2&FdIYnVCH2=+M(oVpCR zl+&25a5WdM1HBpSfAy)RAuFk$xnVnlWWZS`2m_PAr}wPTWBs;HO=!m6xxn3PV=^b? z2rDE6khm5Lw!z|1uKQGv)i5(m;+BYKwUuRJWg1RssF)61#axnQ&kxtSI_r5Mz*r}n z!wWHvYnQD541|r6V;nBGcPt$Kjp>j>cPeeQznnTylwI$4rZyQ-iGJaVBgctz+LTD$ zvm8d4Ze4ry@62#kzit+CRu3V716E1X&CKT_WW8JFx--6CwGS(3cE6zU_Gw=Rd9mL;3!Ybmc!D!|1k<^yGO|=Rbb6mZ~*B4p4rU$#n$w^F$#l zP9>kf|6sMGtL>rFdvo;r{B(`vrc@Xwu>YemXAc7KBB_~MbHzK-5oqtP*ylXHB$aOy z=hT4x4uQB)|DM_}v}TUQc^m`hx0^$kbdTBP9EAQn=i}E1Xv^xWNTJ0+6?vo+XmcC>eT3@0i-9&?*PVZxUNjV{3lmu+Bw9D{N$D4?>7o{Z;879jJjh^gL8MU zfM=Lct8YSUs8Z`7nckkMXVlJJl;pQPn}NpurlHu{gd=TV%_Rkmbm=8%ghGW{Xisgb z$Q8saQ`?$|2Y&RUU2lw+ zlLKB1(|h!j-S!Bk3n^{SIpgx=a&R{=me=UW_iPn+Gzj`=5>i|A6=lieHlRW!#R2Gz z7~NnplsSX~i-=Yk{PC~h*&s{5>rUa&?eE${c)8MWxBLILMV>&sC0PObnPxwIIlTX~ zq4vM~3ID0hF)+9O|BQtua#IO!ZMO#i2Xj+)W_%5 ztX_3})&c&)a50H<7^zJnps6o}Txq<0-2Al1dE;gH!lON38FXy{fP=pH+ z<2r@j0bqP6Y8Nw-opJe$(<}v5Y(Fv@t9GCv)Mw+_0`UuWOnFOV)JgDV&~S)yE)p(Q zMg98KvVRLw`-$+Iv`+u(%ODuAN|ruFuMqm0z!OGSmgLuhdlil(p%3K2Lzp?Y8&u&9 zHx`Ac_*U2bgH$(wb(O@lj})cgjxz3YiKp_6e%rPb37xl$k(=tV^K2*)`=vPivgnr* zPyW}=HSiaG9En)ec8VsDLdx%CrAkWQDSK(eb^P+d$6~&xTW#ebsk*Vo8*9d*sQuR0 z`=3>v!$*zK_xax_b?|qYdh=^zeklldOTjnIwl5oP_pO=jg}`xg!-h!QdtkaRXIoIp zh-B*6D&>uBbUp}tJ_PP@_zZww6|D8`NDsRtwM~7=0OLgcp-;fAOE;0jxGFk8*M^LX z_hjuDg{yUFSrvQALyAA2<^bN*F@0c+9{=@m@!(eR5bb$uz)Kg;x_5r)@PlqNlLQE! zF^Am+rIcCX8rUM;1nqB_r!D;dse>>m!qcOEG|&gl|4(Vae?|EJgZTa56a4@Ev(bR@ z)CMGd=Qz%DWo>rPV^Dx8of6N;?(k!86iY}h6ibNDi%Ax3PHvL4WK9OTHG+E1QikV_ z4nUwHAvHmkqS8y?wN)J30Y0G+b6xfG)l}Tue93P{B zxCMdZd(4CUJ{{oO%UPQ2LEY24v*G=K{I1>`(fufdY`7!f{Xngu=>J~4Z4dswH{kUy z-h-q2+6Y1Z&>r}Ht%dx)82Wy@h56`7|0q)2Q?q#d`{cS1(sJvJydMK%}pHNgU?WI#Kk%q8Q zsEwpts!*g>z9_|_x5&ym5l(bB!7b7?PgT)buNhZ$H?AqO{8p}3^ocME-qfs`;8Jmf(^Wr4m4W7#rPgewcoRSYcRp7-rYcj7CSBFkuj*Nln_tO+ zeD#C_BAPq%_=43ZZrrB;dw=a@P!FqTU&QEf}fS4tH z+2AQS%9Tj+5BNjl3s+T;{m!H=)3!A@(Om97eNnJ?hrb1#^X7x_OG5`H-@aFf4<^BV ztuTyjR5Kzm^!=puhS+R?-~%}Hq>e>^f!ySd$>2LGAa@2DY$<{thJ77MRo0Qu{|th7 zOJ*-yT#VZy$U}vifEga=D;DYciOYdxK3>>==`FC~VC-rdyU3CnKP(*{#D^D_6V;2k z%{^Gt>M?yI(w4q((>ZB^)=HKw=;PC56EZ%>2ZxNa9z9;gU^22CH9M|ZQW1cl9z zzkgAi`UZ^JJJ+OHfW*F$n@5u~kzeKp>vZ;clriW=nA6m%&Z6pZQMV>n7qK-c{1r2} zrDLs~9S%@plT-q!O!MFRfF{WV6Th0fV_y!$7NWgWQZBHWPyB}C7qco25*S>2!3LvV zSi}sfksT-y_AZe95Vf&g&^=cTS}rCOkIPvlM^sAK=BBn#mCE(BzL;bmvYq+lP2!d4-ek&D`@#fwK4)T@8E?!U%d2bsxK>C6lCc=keu@3s$rFe2I!J z)6?`uBxT61=%cefZz}788$TEJ=wZlQ^p!f?=e4n&SWHNbNcaYtIlOv9&?$`j=8-`p z8++9Wh^0$Qr*bS5LhCa5RjWW`(gL?+5Qwl=w0kx!h9hyF*wAwSc~MP8>4t<+3T!Z* zO!Z4$BW1362F!x1u>p*_m*ZR)gH3bN&2mk60GQM;xvGaj*qXRa6U;G#?0Wrk(J2_$ zW@eI*ZZLg6j?po21uDL3Ce|^7{RP`FdQ&D=`x2v2AG}9YImR+_-zMy}3@m5B;Ky;( zANkI6^oFOr)d7i+s)?^dLv$A*C$ssnXEodR!m0R@%DQ2PC)^41$;g{9;e_4u!Iktd z)0b<;O*9du~1NX%U$(CG|CDj}d9H$HZEoIAWDm&9Ymxt8O7xk#@L<^nkmNXHu4 z-_|O-Bw>;$>nFTYlo>?k0F*`C=q@oGoXnoJ_z!Y`yvTjFUXAfvicWP0Rmb0y7A!m+ zcljuEv+@|+sZGRs>8R_TS7*Cs^X=zt2?go63wFZ1rDGxLm5yyXypa1!#bSZ(IrlAsh*kgjCY!JsYa$~#IucZ-v56Z;77i@&yy7hd?^`KV`1jBQWjUBpyz0mG#CIvpkT!;DT5s+M%1tq)o~3AJ<;eV5lge^v@k2r;d+tc1$+P^bQZe7#JO_zNrE#WiHnDiZl){cs zRb?F~ETx#+;6O)**#eewcHszzQFwF-puPqWmW>*_@q`YhElJ9M2P6i$Uq(B+>6V?Z z8Ne$-=$RLmEP@kSy!&^Gdrrxo<#OULM}|~`lJ^K*9ysojt4FC?Ic+jK z^v}Y>l0xqGC$Dx#?>Wnqh^U@Zm6E7PTM6{ki>K+V^S9^B41ZgLGueA-c&8}fY;oALc%H+b_VCeu)mNIJ*=0x`EUcL@6`JG+YfwlJe!|H)~Cno*1!5y@& zYU9uJ++WYwa=gte3s5L`Ku~MJ{M=mHthaC32>)oft7L`Y3))=56Hr_JS)%G`XT-s71#M#Pp)#Dltb}}JzGd9`YvgRp}f|Zs})iOSE(KEJDZc>r>otl7VTb4nblGqH=y>$sS zr?jZvmNpU5e;ikAOKlwpQtVZVH9^&L)d4P4S5S^3+*jDGc67EuBT5akrx7@6?QuST zL-bOZ_Ck`-hEH8*_=A)y*nuvb@s51cVPG09q@vm(TVD%UZ%C=NW!U2V2wO=<5QLvc`Tuanfl?zpjsZ)%(UJgT<6{K9}}%6VV| z_b>LmN**~1V9V}KeLg|ba2|uhA%svh19?LQ8Fz8g0Z&o3g z*vpQjkt0RbUS54&K#zJ9ol>7{V;W})aJ8R7VXl8s9V8dm@r2g&4#BkT63@yu%&~N^ z9(8+)>tZbNy}Q}RbSv8*mtA7P(oWA`3`*fEvA2GtggJo_j7xR%4p{j^&4iDQs+|Bq z)dELeoMH^D)sEH3XXW*C=-iEpAPf$Um(wC!>>d+Qu7V+!_ps81A#lHSO# zlx0`@raEU+5`tuD=Ohax^gG{jUL(u$x7TH2r5P~{YM#MDp*{N&TxqjK_Cas*axzF1 z*9Kn^ zI_F+|k(!_tKkT$tG>z`e>g34Wq@7V0gcqg5+=B!gZ7!uW`FuoPb%t+){8Ev;v zJ*az}j!E%FE5XGR-3@$3X!8u~% zVW11-g0pH=uM4^e&63b;2z}S^fhYA{q`s((fy5rgBnhN6P1^ZnjQ+C5WVJ*cIqjUR zLGLgFh_WD5p3g5^DBSg5h|>MJx-8+E6(W@p{2_8h1y; zt}FDCe?7y6h7i$Hq#bzS_rj^ik#`WQ3w!^R)HePAc#&@wHXBBoPedmV9k_Nh>0?y& zBApWpP%7$RGHM| zCZ5vlj3Lgcw_O*q+5LR|)m4kRGi1dsnUi<`z#V`XGhopj3>lpeS|htN@C@|c4j!R+ zDtJ85<@V*W!|-}`*G#{6pxD8EBRGf@`nH?z$C=FQu~hkLPTl^giP z<}KS(!sdnR49X3jkq?DV2d*3p7|Z35bt2dwzMM_edUc{aiJ7aMaF&Fz>e6=WtQ;=r z&+9CzFLy^p(@}9Qojr`qIzLCMWcT~SlC4mANqGlrw3Jq9mq$cS!95`I5M981HgMF8 z@RxghKyjZVh(!t0p2F|1+`K$?c`dm2y9)|jske)K2g|DYH>Y_6TEko9LMD_Vt^kA5 zE;4pWc(9(tLQP)5@*G3PX*y>+0OEwpeDUHeFu-xZfOD5zreR*mu{CK+N`MYASUh`!+cC|T=AMVm$YSH>Xp$+8NMAj=%h4s#CaBK zgDXae*%J^FO!oSnuw72Q1DV~gmSAhQnGHs%LKI`DqXys{W9IPV(BNPQd>UvyKVwL> zM_B4&15u(}2v;K|R;6f@6qy4?WmgEK&~@#UJo1=kh*NCHeyj|x0i0bb`K(I}&wVxq z4%^5fD^>Ht&q#dfIzY1sc64{lM+dx@zjYXZHnC(=MlW6tBgdFRX5dcNnVUu$I;=32 z;tVGuqx+&v+x$*2;e}GA26V?$wZj!EP0?+JTQG%v_?6wl+}@$>K2?G@ zb7t>+!pUxVHm2XhkfiD&)4pKas>!~%ClmqZ6{zSn>2QHXHl`J4HZf-Sy^K7%#746Cdzk^J3WgV3aDqLT=b?z#| zv;|NlecaH;udk97Z$3eu1y85&@Nk zQ%Q6N68Bzv_7d|-(9t^Bm+dsmQ6&`3RX86P=s-yWEOlR56K-L+c(=TC(}FMA75m?@E?amqM7%GkON6({pJLs zw!;x(w#yL?u8R>5uF{s;h(tdT8_-sa0(lbQ!Fo#JkDHoLT8D70F)e~G8xUlp~4enjF{t+@#>OI>Kvk+ zEkGMC>LJvCMDSoe5xc!K#ISsbcym9GxRE}w0MmqAs{B|W$-sdSrjbB_OHfEK@~{ZX zsF9A>gx>Mc406scX!BH*8TH_2<6nr%!kk)Y+wCVy=Gkh9Y zuCB?QmK$Oa41VRdv!4l@>|$Xzb#BOhGGp`?YLs41)>$hVTS}crbW_o^(pKOt7UVYa z2S%+h0V!l2gG3l_@vo_sGd4V`{Ft6#$-Jm~N4}M}Uq)tfAtKV-%KDX?xu6tEifP|5 zR`IX46r*E@!BZ~huu^*xoDyz03TIzfAv>9|8C^0;@VY{_asv$$bBA$w9$`wfRG5fl z@n+iTX1bUxlc1p*<~f5Qly$g*X)cu~%$_Xmq63kBRR@S(o=fBCZM4Epg=ID-omSrV=pv;5g59x40mflTsx!cl4Xq zt6XEnuEE*;bxwiwefRL+V4GZqW;DU-!41_hREH5^=5>h})#WI;iiJfC zE$(XhBF06$#F&V>+Tc%KZ?$B3$KR;QiZNt=_qZ(&HM@bpX;wsPfS6atErqLR9XwVxv z$S9_iB}`5ii9AjbYUACEpPGs!4uE11efozdbC+ik_Xt)t>6|;6Rmfaar~H?Eg{;%O zOEMPA=LAl2E&niY_&;V~tJvu&^XEhSwNIStzu6IHs3n2>z!Kv(ZOGGv)^Nv92&eM> zovAJaMXiJ4NYd z{t^rHO){hsFbjzJA`s+o&cp72`oIV7{K0n8OYln9ze}{kP(Bm-S;s@*6(%0@2F}Vt z+qUq9`p(@@ABG`V;v1P!pxgNtzn)Nn@8Z6Kh) zWustzgfnu{3mD`;v1*;z67c-RsBH#?QCIVaEqXB}2V2f#5HbhB0*YQ(htkCB1&2ij zkTES!0+fwSlFIqq*h(8zR+&@Wfq8`{p$-(ufkJjn9Bx#Uai}qBh^MA^~a!z)U0? z!xp#~gv=xK~N7fL8`V%!Mac%#{Wo*lCDi_UmRUdClYo`uje?`FJYh#;dEV#pAjPLzg69<%2j0e(nPLvq%!~=+5%#|5cv; zmF}$}<0IrlMe9>CyAyHdgZRs)^fBg9j`EY~Ga~g}E$FK%-DI{WV#0^~F=zkDCv3t8 zCiVMn{=1wx<4Z12&+Y76C0)*j^6_sH`uqiDm&i#)+Gz@OgJeXj^pX8@r~+7Vn_`iH zs`9hF3R+Zc`MgE7s>T6!kZCD*f++>JRuO46S_&r+yVzbEWz#+P9lhQj7-iL=dk{wW zJ3zNv`-sXlu9X^n)NAPjURN<_E(w~gbWyg(!;Rv)6P9zYLiyRDASGjP(%#Oz`WW51 z>i8F(f*V-P<2_}P+5B5u9%K`h=CPwOU(1jUVu)j()}$;ur`H1bUVM%pI~4* z^KS8zUha8=UZ^~YAIA!v#E62Iu$B+(h^Lje8y?(?*RxAmP(wo?R&b#og+qd2E3(S! z2O?9Sn|$xsu8r;;MTNi@Ch?gJAETn+F_-bZep`+X3Gj5vwMgG(rarC?kj~a5hCZd@ zB`Sm|*4AJ%J4x%g*3%OdQ#SwN=GU22iw~{*wA?nBj%0thRdBfkAL#=@NHl;16On-Q~%tE0p6JU%pVDZ^D7<7!2uGGoQ01w`2F8ilAE(e;~>M3oV z$M$q@Ere+OphI8^@jZ}O=Zm>qq=qRo(0t{m3?ZZ~{D$7cgOpK$@WUEMwsvYWtahtk z1M0Ke9JZs7>sxWIuq~xY2Rm4hLah+{sRTSPv#k+MA%>9HRWW`QH5qp{t%9@Uo(NSv>+!&)cV2eQ3 z;9$dvX11{!v>VTfY!kxS#9T4_provt_XXvFxLi#@KCeSzO)erv~H*| zi_uXhh6?>a)JEGf+T@B5%Cc*o`2h>VWDbjU^&9RMw22wom-fI53P>i1sdU4+M@}ka zNao+R=B5r|`M?_0job8-`?gKZ>ecSZQplD|kf^lXoKlBUxf1|sC)_OqE3dlBrxNr( zsY*)AkucMe4tLrw$lL9RS(xDkr_9nkmfPl+j`zL>RzMXlI+Lby`P1ooi}Rg_r*K{ld$vQ!9w zsoG3JPt$$Fb%y{w!9U@Z)D_K#b&vhvRe$qfCDNske!8wA)XYd5PTdc}kd+CsD_Rqz z-@&k4Fq}SspZ?8L%V0Zl+Fn%-5bz z@8V>dts;jlT|=X?d2HHj(Lv+Qf(9V)Y>Ux~kokw;1p|2$$_mw2LE~}gCQV#~jZuI! zwP%w1sWo3)h4RgyS7F&THvwYKm?O_;dki2!{^o`HB-=>I&T;s^b^hq#9y9FF;!Xc1 zu4gDY(Nk%M;8))`d(H?yLG>s|#T&C3fvJbR;h}Z}xmhwaSJ{h`@oY!y4m$JO$(|11 zPITq+-y1Aezlw?05Ga^HmCU%>?r#n+!X7$>e--M640grd8HYgYmO0qa7?~D50D`x` zHT<2CB%^sCFbqhQ2|_UE?l%Y$V~Ms^KENa}E}v)+Sa^kJWo`x?PL5}#!k&+)x=U@v zHPLDwvtMxBsKtj0f`h5Qiz88C!am*p31~9VTu1jdn~a_?Kk|}Qy$RD95&?n+_Y1Io zmetgssiWG^0T*0vt(OKX(x$|DcU)Lz+eLRBZ#>vLe}rA>dgKboP8Xni^bR`S@T2&8 z2^L>3qOuhZLI(eZ zyrHH~o3l7$^Y!N#LMN_vn}3w5tM;%lB8TCJy&CgKrfcz2eWVV@JfL9vr1KTbwVZK5 z`=}nw+$CU(Kt8#7Hy7C8GYt?bcv!S2(7>5^Vf$p~%)c`G2<*{GRuJubib;E{rtp5m z%FF%z__Oy*Z~62OZ~5&{Rc-n3%q^CNbH7mm66j6?Hs8q1-w&2APVzs+T3{n~Q6q0| zqluTK#xo1pSTG@45CzMn@cn?6N5V=bp!gftqJC4B>|Kc{ySGEYJy`R1%SaR)=ugB!>J<=0@R5zHMC zMq2}HhUlRTof%-*G1_Di{9>7sU@Lb|b@W-ZB30(79x{KQSiPzHOQPxDXkXpKe7g=< zWvgq-3{})f8UvDzv2A35^r|Ks$$(2>99Xsdsxk3@FhaqB8(N!Gi!UN`tJ{*7(?l-L}HV zJJ1X&0>RaT(Iu=xZeamDc?JsUxRe^7LP$r_gOs)_WV4}Vv{anuqHQcW%nR(SD*Qs3 z{6x@?^5jxd5;Bb#x-W*$Jspn8dCdXQ_CyDvoA zV24iFe9#VNyc!~24(P>VCNR%w>nxVP5!Fujkud$mUO}toALH2paJn!YX zEMr)3dXrASd2G=ys#!E*Aoe$0j&Bi1Up;H$!i)Z30nTSE$csx5q|s2)Ah~}G87bJu&w_OcCMuKt(aa+S_|{ zxz-D;q^2EZ><&oLX}v)8@$1<|^7zw_cqYJ{vDWi@jSRTqu;I>Y=o(w`-LB+JCOgu7 zpHcA>%=R!dtbZkoCnNw?gy~vRS@zwnt><0PPRUBlBVP=*LMi44J$NwE_4YS@#Hv;EjA3H^SK9*^jbt0ZVY5~{$$)6fyZggR37V7gU}IqQ!%dH`ei z0i}1B=%Oq>rvR&HI*725Bc=--US5GRE#fV{BE6iWMff;;X3_ojQNp1?dLeXicd|se zF{cPGENXGe*v7GW^$T^)i?pH&R#Za!_v712`f!@I5r z%3mK|kp={PFB)Da_-u}ezbBHHxyzo($5}uQ_!NuC^s0I#jNM~8)xpF zJK%H9;sz}OBz#kaMvi2v(|G{L%xx$;n-WfUj4lWVp2ft}fL%w7t`za;ZLyn>{LaT; zTQV>FK?J>*+?!cR1cQFMe*2&i!$ES$`+(B^0Ed(@CRV%pAk55iCRmOlbdp;WOu&ywH6=_Ms` zJLAxV=ApqUsnEO5t-FoqGSro}UA&{*ZLOt=EcKs$0oON06wp#{3}4Ys3NxOUe&kQyNN<~dP$K}{oj z<6?T9A?FlE3pQ9xCCKYx3nQ(>;&6p@`h$eXd< z@`;q>&2RO-gZ(3N<%E=!TS(ErC2So{IitE=h1~W*BURyd%V2K6VE$WZR}sii7$HAC z3ID-F@hS%7BgmPzU!zI?!d+QI=_m@GBREq$DI2KIiSTWFVzmwi+I515k=vDuQmwYq zh-;!9GV>2=onAsMMah}S^VhhSeEHA`bzY5t0*TmxqpG_!p3HBLq@vKSUxrGLQJ{5V zwX%g{(Z}eB<`3LrR!WQTdZ(pbCYjYiFuH8AI1J@mff2WPQwNJ25?bS>62%i$*3}k8 zKG}h@2z|2%#0oj_n^ksZvutM~V>)8t9q{BY017FHF8`z>^^WB3AnGGKyUK97KghUp zK%(+%>5`Y?WGIVEE7~4B#;dHdTkjcvTa12;`RB!F&VcUqEc;)1&QS~3cI|l@>Uo<< zfdm6jcIiRBov`@Lu)3(e5LUh?j{T;AmL{{+fclcWI3G78^7sGU5fir7_&blG-j9h976e2Fgvb1^8K9Mp07^sHTi z9&&LQbzN%UM)S$<`kkEX8T3bAuoL~+VWCdU3_|V5hWOeRwu#JV&j0ZK>1@i0t1mO+ zzLGt9gw6lem+i|SGf2bn$RIoWmHuMaE&-p77%au)+%Pas<6otfRR_W*3-Q-xK-l5@ z?3uYnsnF<&U+;}Grh&hZnxsj1+yb3At=``%iH1llXn4#l8^#5XI>1=ysPWqxA&2)H zMGef{KLx3?5*R!S!2!>=ma`CApxWaU0qKwr?7=6Htg6i?TRuNScHB9{V zE#VQumyF{dg(|&CPA*?Ei%4>8I3$SFVJ8e>r1VpUh zc77Z5C&dJCJ!J{{H&*1WX1Sjx5lyhLg4h6xF|VA}%s6tqb2)tFn?u_@of+y&&~G+> zX_< zUmY$a>!z#4ru$|p!;Q~dR~Klte}dAJyCT>31kg!tq|>`V23+H@4l5{0jy-NGUbig4 zUOYI;zA0xFzs_SdY^(*G1j3RUD-(s9dc3jZ?IDyszZa*F94%H~F#?5U+7uvr5q+o{ z5<%XyB)k^QKRc@7+CaDK;xws3_p&JLEj}_bkIO0Qmy{iCwn$cx$%pXIxpW%T869*n zF(s5b>QTQsLDXKLyn-xQz0Wr5No4=wF#Utubn9h{b0v7mud&0C^m`G>j5WP&r}spQ z>DyZLU2nXnYk|S&wOi`BAhI#6flZ!gwk+&0+x}C#=MH>3Yd+DURKFmYND_GC+am=y zs#6S|oH^a=8*L_ahG_)kxDjjO(oKCorUX)D7y|cg@RUI{;0Cepn9WshjVgEef>`vo zezwf^9XP^T#Z3L%4r8na)ANYs3Z`jX(T=;^s%9T$jVz6^BgfD#@OBDyA5>EPt|}=# zF~u}kC&v_awu$=pf1?1$54fbUK>YfZi}pW09{OKjgrtq(e;p4sscP7ws-W^@0Z64e zNzwuXLD96+q+WTgSAN z7)Lt+4W|=Cl+nnTM%013I?KOUYolRuBQ`B^S!*V{iNe}$3|{u%b48zk3>a0Y>NX9} zPLjEzhrUdh8$0Y^`IVU$Zs_aFoSBr^NCpVYZ zg*;@ZG2j6$k>-wuLu*#0wx6Pb@NbKh%_)*;eKSl?)g!$*ZDu9?Vzb1c#u%cbE8 zQrTQnACt<68Do}k0Yshqa}$f5YS+-VGbs{vJMEDIF;by00IJ3XNiCf+*wr26aVj%UUxao|Qk#vGM~8?E<4e$ejPwUE)K$@v8@(IIRa|IKdeZhfFznM-1BIsC2QjxQ&au zkX0sRqatP$FG_dHHjI4~HG_XBBK?Dc6~8zCsas2cMP1)g+djjIfhGv*| zg{i*eBRxG@^1@iz)cY)9EoS8dc&zgn1owMESr=OS`Oq;(m{{ddm`F84w9KnhTvBUm z{;~d3poM_;A!tpE9MCsfS#a*VgX6SVbj?DtK zXMcEmKO5bBGdl>j*nOq+V{yp91`1;$Hta(5QxUaU7>uf&42QKZDq2ZFSKK!x9)K3J zMhuyS>sZ1K?t0*`tEQR4X05Y&prtou%f)Jm=@~?NVYgC0(5FfU{leKwz|{0yjCz+K z)+BxxNFwTh*PVc24d?)k0%_K*-F-Eftx4hQec)*(99|qR9Q~$4_w!1fjn2c1fjXqs zE596fjrBllxtm_McfKOxIRI1{JsQ73{+*{vH#B^~GW@ViN*33UA{jjmN8c`6&NpCt zo+alhm2sY(fQB$n9CM)x+yLygB=*1P+x+e*yZ?( zYm?r$#gBd_enS)4rH!hQJUGD-c?ke6OHl9X*j@Acu_^+j>KcGgPe$25UBi3jLTCe# z%^F&JKJIT2#;4@h*9bSTfxUY_XNfEhER*UWvy(lm{ffKDDx56Kj`f)L+Glr1YzwKAW5nM`eM@!G2PVUuYAlXf-)|Sz4duBn6eHqtU>MpSM$B z=p6ZvJ0agjC=dVpzezLRLaGj|A6XU#`|B6m|7`d5pDSJ&16xIa@qcZ$-ibAW03C1S8<86!mQ5r4wkztf#NRKd~+{2 zPtb+v<2H>Ti>BiB*^?IF>Xu5;0-s_r%9$NFP0`()Vzk;lt!08U!>ySrkEJUr?>OmH z16Aj86qOF)YtjK#!xzcqEa8G1;tnUgH8-L2)dgxRnA7MqeR%oxIh%I43Kd-D3$}wZ z@B;_J5X5w8CS6hGE5Pq(+EjjpjhBiAd~sOuDG( z7=uyLcDBmpjeLXFeJZoyvux#Q6U_U8hE!mZocFcLn+6x5Om-43&anyJQK+g7$1}`T zULB{9(n5-f|A(<}2of#ql6+;`wr$(C?W$L{ZQHhO+qPfXwyU}p6VuT>F%kdf?!L{9 z69;)R^Ds)g6L?a(vl)0Ack$Y2Vf{_J*9n5|?UAU!xp|dotRgBA|7WExVy~9kPQR_u zmjd^uRAUHg%QO`eq1_)7!Hp~VSrftIRlSRa5?fchtA(<&=ZV6%TOR?XE`M7H850mt z5P>)WN$KDdV_=h)iKDmg*D545x3MvO6tW!jqZHJq<|$xfxQ5bgYU%#hk(3T1 za*vpzbC2+ly8A5k{d~i8J0#_0hc1PHQbz%ELhSuJZZO7~#S(o$Qmx$@O6&ZHFYb+& z5%O~6VneY7W52Y<2;1#nwn~g*DC^k@g-3JLn0#%PK@9KO4F5NUe#3^mXG#w9ImQ!G zxtYH29KN4xS+*0I*&S_j1y~#oK&@gLk9?MODEE`amW7;hgPmnwsB0gwlpDWY!zJ9v36$#E1A6t$h-WwE*aDdtt@1VRXx~NZ5@W$NVvhun z*)6g^`PHelc9}KYOne_sC%il^7SXsvEiMxXMB^Hs*e_OR5{o^LSI|=9eHov0@aKQg z$3#osXsUj*jrZ>XkpIx6SKh(a%)!LbQPITC*5UtJHWjOGDq@KtUurK^Pj+ z!lELI_Kd{zqYL)!85%){u*WDhqiP8XXj)gR(!K_M3ikN5Dk~#U4FKAWync zdQvHH4@xotshKpI?ze~k-Yf?+Z8>~>j-j*jEI&MAOtCa+y_`yE)}U-pFF5jeHYKq% zTdBH#?mAHYJc^igDe3sDjeqTEy76*V;7Lzu_oY`4TZ=DiOn7tnCE>3%{y>YXfd)RC z#A@xoQru<4SB|W=`7h(Cqc8qdk-`#Xg@%r9=ShEUXCf8k8ryK9*C4Ukt5SXf1U%QH zILj<`a@MN#@U2Cxnx?WRdE3XT9a6UaEtp?v{h4D=Eo>`Y`+da2$OJMaakA=N_un{0 zdVB+H7+x-#`OLa>#XuGJwRAOFrYrVlCWZP@!*nrRz*?hhF+1k(Y?;7@|uR zAy1ndvPj-Vom%I$zuuLYA(>+0K^5b1SH>*I)R@uSl7bH7xz~cKY~2+w(QNQ~+=C>F z8h(argYNWVj1ejA*cXKLX6F`3CHVpHPyqjnghl!8Vfs?V@0DMftpOfYFpFiHLkW8p zcS+7Cehr*PE{)Bn2a-~&eGnwV1p#|ScL!tHym=VdIJY}rPD(lcZ7HLZ4l5h*Egmrh z+!1qK8kMdpp48OfW%oy?Xo)3s&O7=L)*&NvvCzKh2R#`EZBbunp)`iO;=ZwTDbbpA z#yiH4+FQyH3GTzQlt7~fjAM`|JX#q4B?}7W+Gvm4KF{S9qmda`sfLAftmiEXHEocl z_REbNlE?~Gozi){G!lBJknQ6ye7J0#rXvGLGI!*;SvYwrx39ikywnk*B-{9Wl^;8FSb z&l9u}Xn}1&*ggJY^R!KPX?$LeDraF!Sg0b2vO)34d*+a4(CEqb_%geR$9PYy|2kudssBy#aSVOhDcV`)m+$WG9}@&5>jg>0>@Eu4Py;{VbRS1VatF7dFHR~gM?!w)Qg*Ubb-=t}4GaA-`d*R+PQ(sCRr&?UKsECRO89e4kvnO{zh&EwIC8~a)M{y?*Qz0JA ze%PDE=n^**^SjgIC3FwzF}(f=1Ew(<2Sk2 zNrpA4Ckuga-vGi1H`YU*_*W{s8l^H`_pF(857;R~Ne%%+<(3$KO;$inB65>cDTNas z7K9KUGn@3%A{muYw6!8aIaN5xZZ0lW&&+x_l`wGqo@IGYA@Q?X(((i6ez%-?(=7J&BaqdIz(6c6YQlk*zv$)OkQ3kzQuZ4ycl;Z${*H z51M{-TJzrkqebpN?J5I^Z@K>&LH%)59TvaQ^Yz=ZGyG=}ByZs0^xv@Aw5pUNmI(a! z%F1BED&5{%{dnf?m?f7KE@=q5P@NI+L1-9usF^;s%%O9&%kbZ1%H1&p8*Gkyusq0M zNHCv2X5SEOu*f`(R9c$RN-c7y^o!ezdUCxb&A#6sA9#Ok4C$hwR}|6vKMXa{CPUO& zhZ{&_p)9=ozyfHVjj-TAnu*H90^a52@gUBP>b(+%N+rBfhKyhaov$^f{_W{*J7kBx zp}=fpg!r4RmVZp$xu$LJ+5_{+49Z8l_Z%hK)HjV*N)JXmuU4j2jfAniRFyU*rTvyY z!xRg)3&q(f8YVu!tz~-M7oAuYw-^qSOwUQy%2hUvWXrMNrnq#fDP4&nvd4@AMG>z# zOLI`qxtsD~(`mL1MvnxqC~cxk2cMhL%_chWrmMz)x~+ImtHqtXH1pjmRhgpWS=n|)&*>VMuQLw=;PX^)0QIlYJ>ZR+n!~Mo3m9#^}(%O zVV|*alLi_?np!jFh|c1b%5#6g>5uxlvn9Z_)Mk}Ku7q+7@NmpK1)&lddqQ8L7+ZHC zLCa-Ids)cQ?X;+Ks^`7-BP`17KZB*Yq0oV}UNZ&NlNd)ee@a|Z`?^LUmMspGzru~q zZBHuIM;m-U5AWEG!E$QD(!n3Wr_}Aq*}TKd>BLIIg`gAU>B><__5+8>N=MY$R^?!m zVQu7$TC~m3&K`5$1+y*Mr#Wn2LAreC9O)ifvYJX{iP2q{@!i3f<&w+EuPY=(IJBUH zSF*}3m+`F9YE{m%EZM@Fp@CP=k&xbx=2B?eA!_@TDQ>(#)h8Z0?D>OzC!D186 zXKg`~oHctDG1Mi_UH~aWH8F^&PF+!s5=qW>@Yh{)K7?hF0Ttlq0eQOpDlklBX;{c_ z2D~jUFYe=|ph<+2ERpxQEAO0cpWPY?QM{6oR-`a=&de3aibFyO@gc+l0V|n&%k+7f z$lY>h&>5b=p`T^vIJ=DAe8&8&bFmUp`_S`7j}~)*_6bVQ;WJ^M6JK6!df+dSd$P^( zA1LE5VbY1&Y9u@&7tP>uC`Bq{>HZiA(}*u@R}veMBeru<>7*V}?W7*j;p#{XMuh)_ zx6;I!<9~N61aHvZGDT7AC*qcI(!KhRaTxGZ`eV>vf~KAfVi(SkNNus`s__G!=)y4p zGfHWe$U~eQjs7532_vYudHnDAFlVUJU8KDk5sWk$MK*IM+Ob8#r`-vN<6#n#K{%;N z;Ozcd26O%E$k`wi?J^DSU9y$pK@y#F;aI%|CuZgN+H&H1L}!Zn#JC;8a+z7zprr-7 zqms8+l6JnxQ^~t*$-DXFYpMY^l>I;ob~+HcLt2n_)R5nz5W0&0wy;2aD}j5XT?cY# z=4dB!;lDV&)=VuZzGCQiq@axaR|KF)2GAVV-~}0-Z$+;Wo@i4BwKjCHV>(^~2ZsB` zAjhw%qK67aeZ2DHRXoVI$0dcMKZ~SQW&a-i{IALf1n`#g@0VXY=KmjSNB-L=tzcIwwM6MOb^% z)HYuceJY~qjwZ{Lg<>^56vT=bPwTTdSCz{BM_5bx0eT3d+3u1pyioLHVi%ozL{BWO z<%eeT#2TY>$FZEQ7v4CrrgmvKZAmV-Sks0#!BbtZZgz14x^i%_PJ>hXTs7U*FkkL$ z{&Gx?d~x&R1gQQqI>34(4d_abTbG{JI<;qg;vAjm6%(2%o^WUV_V(#X*5gl{eDmSI zp)DF!&HkAKb?zFQ@D=Vw(|8*OenYIVD#qnoc`IgBACC+RNmV#=pwtm4&OQBmv{ z*+}C8zs%Y zj~qj5xaZZJv!U8CI_j;v?^gDV(&dpw;ueS{v;L&`mh#}10m`kvGkbI7+%XHxos-Nt zp<8>6{48evW~4buW^POA{d>^6uT}nNxgWT*_G0R*3qxua{{Xk9i6PNOYVDM6({4+ekSAXAo z{<*09&Gzj|wPSnqQ`ay1El2e;HNjVEqo@9ss`^bw`JFlPJv#AYdGynGUt9HQsd)o{ z^%H_e{3o>PQ!Arphm@bba#xk0tZZ*#cudnr5!(zB!VWA# zMN&u`Cf<@_z#x&8t`9HYSEoeDJB+j#gDZfSpolnCkX}YS5AfKdP>_UEgf%4PG2NRO zwtq~M$-GUVjVayPQTpK!{>hB{-UmLnBjk~7fFxTGof^hLcrX_r(;D^~aPwEm67-%7 z;~{xK+KCKas{ggS3(X>+gh-{#R1d-;L)8*#rcHs5DHD^0mpZGD>R8>C4cDwJD@>Oy zOgU0jkn4r%FmH-LS^8HLnXFnbiB@U7?^G{}W{FY6rc7IDSyQI_Pg|*_DnVdo;i)FN zMzOZCvb%}*^u?)pc3AePr-pxS8AW^V{<*Ebr^!XZdn&IgLWKor*LXRVj?R)?58I+4 zg2_%_Lr2F%M#RzENJZsI5)=?%K8x4a)1q&a)zkY-QO~n;GuAQ2W_#i2{IO}N{Zl}0 z?%+aD858JQ71W9)ue6L{Lb3x<1D$mm<;~WS+y)*%A1DvydWf6hS_*8g;T9A zP@qK9<@Y6x&hh!ko6-f(l}?R*EZ%A-Yz)4$FjI>nLo1U~qH%$(ZaoV~TfIN-1FbD!+|!1+MoH}~%w2vxmK z^^LM)GN&h8XGre-Xk#VfLBY z^)u>uPyep_anoVR>%>nIWz1h0>Q`=U?O&%TMlqgio;-kIY5T)K$6)zLER{pYpI^d; zq>5@W7!?J=2cov~vRIm?Zf;}dHE2E|?D@Y>VcLOtcEn#pMrb>wRHw-K-Z20}VzQCu5+CIzpsgitG3S<3S4?qhQB za+D`gfn=Dx?XNMKnmi>S!fh)(daC|&3qM7<=w)JX7`1DwXuJu%iFr$ukr_|hOeHe4 zt&jtDq8lz^(K`*m7TLz#6T^c#U_=!O5n(K?osy=V-k1Vf^=aV+p#Nb@XP~4xCHJ=p z?GLE(1#}6?1|jlxi{enh^y0sRR7^$27||=}5>ht6Hxc{VLuN~?8pZj6<3#nch<_!j za$VQXogLC6S$<u{7w= z6`TZa$Njy3K`=%S)V0*)%yjB5l-c4*OM|@#edVN_NJo6Uj0cjjixX}sK;0OG#2ccpd)JT^4AGcMOP6>&?pO7rK&Y?|Js}GUQj$;6QrxYUmBiJ{MN^=*8z{8rpyzWAmS^>zJ2&J#zB(NoyMo}lqvyGX%Nb8T~zBI2Vb=c=J*nT;6zfOPfz zM>+J_R5avam!+0;b@JD7=#f3-#ljd(JM8%WV;(Hi&4baq`Fp zJ_k9_XjhtYBl??hzi&6G$cvVR5!FMwWH3NB36CbVSn=z4^d*ftQ-Vk;o*$JQXDrGUKt|Km zjTIvucy7A|4zRs$^uRjVz*(?e0yUY~f`NNrooAaIW8yl5C&n@~skltASbKr7YP_2A z#Ut(|W{H^Cs6d^^js>M?VX>pT2Gun*HVsOr zcg1h~R+R&=Q{rPE4tNp;I64aE49GVUySBAJHnliO^(WA1tvL(RGWgyf=7oD$Mgx`$9SpEtnyO= zPL#E0K32duB%y}ZwXVtDouydz%Z3iKFPiarC0GpYV!qLwg{cogN`lr~D^~4q-n)&f zlrTw}&EHi}*9$|nJRu8v9TJL-xe`2Dqy^SgWoW|;3?vu)kNqI`?3!sH8dg@VZeF!Y z450B-qP2W_!EThxKFWjwDNXPre`<8d`J$)WBmrFaCM_1l99W1k)+o`5x9zJnq8tan zLKl=yswRwIk<~P=1MkKKI`P@w>g&0uCT9{&gp8Sq%S+XR2O!Smw%L>0;e24SXewNK zRD7L@sZss$G)*)l>PV+EyAp23iVsTP5zMSkvJpnhV zY$k}Eq2>feXyeMx)11*>e6W^+Se75PjBv8MSXd%0>*+vcq^yq9*+6NeO}1WXBnfm3 zp-Rny?%o`qY&}@4xH}2nJ@G_HzG`bijY1Vq#qjd-{J_XQhfM21Ubf-QA|0LXc!aX3W|-0v_a>kcw%LTd^s-_TBcbA8ld0w=R~bYE#oP48;x?gL1_j_eyKmL{dTDA=wwx)-bL?t`tR9!q~%DGOkg2 zRGF4y?Q7a+_pj77gK`CK^yDcrSu(7JU%CldA+28Qg9=%paXHN!hGAJ0WDZT2g(gf2 z(A#0p=+AA~Fbq}imv)$&#%1D<;@LdCN>*ADRWskRGHVNMXbNq`F!$@aqVLPvhtpm& z4s8qPMPjh(jm3F&@8br_m=#0Eh&f?f(;64AoW+3tpoU7z6+&})bl^CZ>_2qYp&TeH zukuo7`^>akmR{`Z3T=^Hvkd|53p6!{yr#dl_s5!AVoZg>8*^%SB@0+WFs(kyruquy zl?Jy#(J*ML8yCWaxQTOH)jil@vCrvdP0uh4$=-q6_%ynAiBowj!FLLaqj`s{O_wsV zx91OD@sFI{HgguZJBO0(nmZFAj-6bqhE`TCBb?qoxoP3(^yC?3Z_e-EFL=Pziy@q& zg{)E08qj-$-TwKCJO1tMLfoXeW4PgU@>UydI3*G_51kgD#F)b&>0WJF0e- ztPV?sxM_41KsB{B#uB$yMykQwQQcVzeg2B@!d8baLSC!g2H|agq5S<9^rIx!%op^P zrkeB~eHlvxAo?M^A+3)^A+ z{BGtUw*5R>Sd4wG`C3c(RucrYfw{Ccjn%e%2>Zr-iwu!fMQ9l`v;toj#)dpolQ@>R zY8V=oMhs0%N|0&zT6{h8vyha0lY)NMCTg0@7@&he8tuYs^M?;ki)K`@Ik8>}SZLX$ zd!l{_73*n)Y5(lZcnd(#k1%g5a}MH#XVCPzzpic@*z7RL;!5l|r9@-R;F*wxtDLM7 z>DKm7z+14Xg_}1*x8_b#A&1fnj6JmDNeIs6mhB-arAI_VkeA1H?q4=)7pS35+CZ>! zX#1Yh^@Yu)n->J$d-U7#`wONlvLJDtRemLDvIHV&()`_ z_<41fm-wN$og0NbVQsL*%w>PgU97A6|LNNjQe++oj$%8GgaMMi|78Y$bNa&{oSYwx z`Kq`bVA$F>1vybo1r~Aramu#HH!D z&u(HaW}dX*n99{zqC>y#47}hRMoh;cToz2n(k?41t(em|E7D~DMGD$FfpvYZ#9Zmj zp1Ol+y2dmKvD(kSk-H$Bp-ja3>h$bBYx%&XNwhH|BEXurcWAy`HUF5xMSrYnV%Q;! z+Due%4^RSmi|q4&2pYIV8#PPnCM(A5fB2Iey0k+COY z7?&u|l$f={CX?f|RET-PiJkt~#LgiHS&le3HGc}aDN?jgvLZ>}d991{s~1$UHqH$B z0^fi`(?AR;EO{B-HZV6zf+y!<@WF_d+QNIznvpflBmNP9{Dp_tf^S<(d!kzkJVRYV z^QTTAIGNf-rDDIZg+Q$59{2t0*4&n6W`1*`J4vkY#T<7L(rTfU9<32c?UG`=-#7sf z2}NTOd7OSzsyRY4%%O!0_CjdFK?xYQc~RHl(-fwAP%y8sHjacSWTA;UcRU>@3pSfi zTN^Zu+@QEVqxvn*P3l?3>|954V;oTe?>#!S$ot&HktDhwNsmQ|C3~GYFq#9pwB&vV zyk=9sS`sL`_c-?1kcc!4j!!WohVf9Tz#J{!F+Hq`q$0NJcJutvQ`3q|r^*C=2u;+Q z@nId)#34xF`1m0)Xxr{M>0)A`iS`I3fnZ>x<)FK6IBXO;a_k{?O254XVGVk|WCg1e z>pVs{{cy&i?K{pmhAgDY_G!eG_3U2ouL4utP4>DMhM-pQ+;Y)t@wSe`g}_;FWwOo{rQGC9{5rVNsk#C)Cjhq zRg)~rv@O&k=wS;)gvwjX7f5Zb{D+&48XB}SwdKpPzF&DHFx+a{XX&?KJjZ+T|?ME=_xs4uaY7$PZZKx zvLtoUDe!H->X)o5)eWu{jWXF3S_$pWnF88+J&?T;x1wnAB5lwc#A^0oZ`IgHSk?n^ z#Ot~Y*(#H7Ae{WY8s(~z8sko)3>#%YWaC=o`W(1S5GC>RW;^ zEjbR7VJ+~*D?&oNOk=eUzTGDfL<)0fswa9+#71z-xzfLFxF}==jVVdX;kteZVZedP z!$}MA+?4tJB65%Aw!@sxrOCV*;HgBqQ5rBqDi+Ft6o9A>IPhv_4W#b!E}1rhBo;Yf zvB|C+fRMop2>${VpvoC-myU-l_#Wsv-jF*0B~alEOoH}8$Taz;WUBXQHOV9vUp*HG zM8u9f`mA;J#%R#n=nFJ`s;(1kLLypZdG0quA;86@n+KR>&j2_IB|Rk~;IQSuihNq9 za#Gyqo!J>U?j#O@Okk9!$#r_^DQ@>Z^P$5bBAiIg zekA-w5-dzgO$2gl8`}@V2h1qgpN*@ zb6jF*;sR1q0HP8Q!|h0DXuj}?RWTQV ztfHmJeVmyd=;pf!(($e07JQY%Yz~U1Lsq!z$pau5K%o$4c~$K3R<%1$ z3V`P6*R3d6p1`l`(@eKsE|(yce{?luf!R20=$XeOB!?!s$XW6}5{WEvTFUxXBjJE( zjQrNH?&0kDP<%;{%u1jJ4vEo1M3?6&txrj`*&gV!F;fhhC3|y5l~`^@O^(DZW<#yI zU|M7}_+QT5JXVxf^`sP}UI{j&87daZA(ugv)YC^I2P;;(_f>KeKGDo9@db~xBU+)8 zx5^6w`hi__={oIR?pOE6Ce2ionUxb2sj8~_cHGP`V1Q{{pT+M2+o88h^Hj-Q4hv(; z3PX~@1(kO9FXCO_>{mpyQedZ9K_BoEOBsy%i5`^=jhPO@E+TsoBOS%dM;e1=z zD)QN{!>__X1)wBK9-KZM3umHOF{qOvB_w}Kpbnk?z8kmBpp{uy%Iob>SYSq1Pa^7O zH#d6@iorpTPz?0 zV<=&ZB;|Ks{2iV?pQJ1h zbq2mAxja+=OxzQuL~v2sp%G@1hH|OgVy6oE5O-uuU%36xPu0*E(;~1#mc2r|vT+nbpU`Ml+l8?S#~g(0?kbKVOfdQwab zlLWKPjS;AKpqG`T?9XIB%JiSHbaagZk&Q#HP)$;5AFS{|5&UJ!wAa}0Eu6m(FSA4% zZf5|@hz`;>)Ai@n} zzW^tlMnk?>@Zh@6b((;yu|n(>>kv37$)?+EQ;gzwqZkdLJ7n7#9}2Rye}Zp6y*eTf z)Dc_L;!t{}~IZx1!`iMW^b+4E?C%r78LT6J-$0@N}e37qOzj@Jtu({zFwIXI88}lV3|= zoU3uq7RqZC_q2ZLcF8A}BG=c8WOPnuz(N zR;0+_40ftXXTG+ss@?>)V5esSMg)A;NnnI#d>ve+65n#kJ64`=ziD79A!`GE?;_*_ zE@~YbUNL*>O+P{%tUpB)r&@Le=t-vzWe@o=7*?#W@|p#7_Ua@ASvB;f0aK_C6BWps z27E=2!%dI3B1B}>mw{FTg0}^1)^823A8DtYuSugdRS_Pdt~nqJLMkP-&R8VRHD$t5 zg?dskxPp=Q6o?cwBfGfnki85vXvnvRXuYp)47Mko&0=1KurtqcjUj%5QYA5u!Ep47 z$6rIJEouNhLZ~qstncMyfYFBWCzC&PIu6K8DoCu{03pZ`VJ{gxBMzz0VgFbQCH+R) zv1k5o4|di?*=ZJ1f#b1wKvt+@4xEJ@z>KTGR97LovCXZhU0;$_TWke6Kf_@OmX>hFc~b;8bW{(DB>&S zXE)~2ZbzEMsAvgJB|9N^HIK~(Q^ML@IMB&v%;XS_U#4hJ{&bbd^k^<1R8c`6Dg{IX zD(X`|v8H~s=1UoRKq_*mz5`vAqE70;Y!VNBaz8G6ioa5$rDvWWfhL)v1!iDr-kilc zlt|?zD#eJm-yg|KR4>_^U}>`n-L(3NoCPqZ+{-K9lqLY@+~DnEeY{n$#%zke4Z*KQ zI{=BN&IH|j|1Oho0yw2t{X zYStBGvbunvJ8f2*;HPW}1E^1fol1$P0Xv_@L*sL(6u!Uw-czCtV7)lkgx^_SN}--W z^K9n!TgS0=~pclj>A2`c16{87;b^!SJvC+}h>? zI<-icWs1EVFh0J{WG++r`4H<1^F05dURY0ALhosrchwu)CKhW9JG)!eOmmC?7gz^c zNSZ$`8hd6Vo`+rz^^VP!y2z&+lLm4@7?AF7>5llBDAFEs{A@5x+#y2?UGENQ>n09a zmlCY2YyGLtcM|9Dug0IzD`9tFQ??0(++Q&qu3{lkP!;JBaZ}k$Mo&7VJ`Wt0g}SF^ zbXkTKS1Ue;r;e|}Tt<_Um-_RG;Cf@Necx$5{T=0X$E!RLDi4R(k8ump)K@wO;46gt z&Q)>wWryJfy4rs-Hmpb3`VGeMfd2^yE73p$Ml~>&h$(|#izW$su*MeajrTSj03Ty^ zVkCi|WpN33h|LJ*Nwi^6JI;Y4Pv4GC1!l2BZrhaZk#6_7ZqtX?cWY#Z(!tPOSp`^S^6dbnrZ zgvA<6Ckt?;kX;95HAKH=XFb}Cx=8aeeAYTx4Sdxm6b*aca=L89PB+-bT!P6Olk_-m zhNNQCW_J3sljpI`=^79&tW5XU&00L$%I2a_oqtK4mP;TTaJKr2&|EQi{3T1gB)gtz3|%P~uo*`qaT+y}bmbwd27)7_1&2?tYYNZg zP-m1R;*B2@7oVdg=_ahvW%+XyJa8stxxQTErZC51=bdN|G4>4^=P!%;I~5vU-iTtWoqp&%o| z3%nN}@JXSLit8hrSvP$`GaMB5(r+PnhJSpHNQ#Xy zTfD~L!t`3nB%%n3ss9Z>V zC#>TN&ZUiGO=f*j=9hh8G5Rd%fmYeVIuB>3+W8v=Q?Gx8tpPLyD=T)82509D)fg2! zI8^bzBBi zb$n;-+y^4B!@}t<@a0F`wlpCHhUYfood%qWNV~b%Y>MY3vg1?37?z!k+8x-@b*202 z#;o$^C3x))oO{T~GA9iP&Pa0DHp#{q{uWIIvmQj2f#8HZwB6&MKNIV*d6?c&m@W)f z{MFd{d?D4BjBS7ytAt1hQq2ZGF!bSHQD4|>#R%JNp=*O}OUvR5*<+}jRl!JcDzwGZ zgLy<%e6@+2W5#C)Te!4RvHPr$8XLuHfKmjkNGg2RkvP;l{BN!Ls~UKNagG%;E<|U& z1FQ8Z4sajA5)T|<;0|!w)G#56Wdv>79;*{7xq8U+QmFSo#kv{7BuaF}$dtC3%5n@A zxG*>*79@C-@y{+^5^MjC$yqcBVXh5JO;sa@ZQ1~_%*BP3$Q_+%{)S)BA9nzdfT197 z0yK}kMPI54pR%3`#1JFWjj&9KI0`4E0xC)Kg+Lz+BRJ?%9|evhjw7U!B(V7tn8qgk zk&*O6ICFMgGRFy^SO_FZ9dT$H@AbU9`TgH#lU>%B2j|_lNQY}wsRTzgkp0V^yO^LX zB&25%QqmJT7fTTo+U+q}1J=qNx+%|nFaehfmct5$Q3(ZZa2{elnRqR7 z=`FpHUbVGK((Bv|`~LD_lFmI;l*fOPA55{_;rn&OpG|N!L_MXet%aTXJ@`LfKolv< zudSji|L*LJJCoR6xjPi;2-SLk6YcNMF4f3ydqYSrS=!s)*uRXM=z0Zlq$jyq8m5#K zPQFEHca_H*wNrG3deFdI1G-~~n#MpW(YwKgx-W)ZoBUPz-}C~`H6(jrQn>=ZQe^Z; z@d18eYDz}+f1?T*ZE$E}&Ne1dGu=#8jyI4OTb(l5o(8hE>LU0Zo`hQS-*`?n{jGRI+@Ts&@@@c+OpU^c6L`W?=%4MDsBFh6~!)+CU?=C zKrFmkh`3-WT5+L^JTs&0i(2Tx3w@9cHu*-J8Yn%2KcH%oRBne*+pcDV{w|{>2PF3D zd!w){%j##Gf^6h=faDzMC2ZtUfm=!;Cg7d|d=H^Q-p2Cn62`6QfuK6~(sAsnGWORE z+H2769daBsWvm=>3R}uD??|SEaPDzCFm47JpV1x(`20zq+1NwBaC%0+!*lLV>`*=c zTk7fdM4$26GQZG#275wU>1#NL-qVwTGKX1MW3|*#Je*J*P`aZlbZd%d8!hOV$**>eWSZnb@&>GqJ`ygqe?O7Dw5)pch}AC|p9 zZBLlo*>a~_Z#lfE`22S!PwsiVaD78^r}+91KHdPoS?G|W=#VjMk&)bz$%l84-V_xk zErtEq!lJ!`*)!R~BVQ@Y(Rzaz%mHPMdWAyAMx^9@zj1L&op>>U4D3|(txZmwh2kLG0#{ufZUogKqYV{d0 zPtq6g*0nQKLuLc{XKVYOX^KSNyAukl0wQv&g(cQcqy@HnVzb&# zvO9Q&Yj$9QfgTs241jJ6D0E1XkkopGXK*Iv|7BU>YqTcNx(Ta}&(37v2Be-89hGi_ zyLkTPmHeS1w=NHmtWAJiaO}u#vASF32w?@=wyjJeH}m0S-G*&MwnM^NkbmOnC2R~d ze6w7DlPt4asXil5pB*#c3tk6Oz00io>qgU+wL1806-N;k&EtcG`y%SLm5IP5J_Boj zH81aSA>=}HzFif_Ab#T_zK0Vx%|jnj8w%2elwyZhb-X<{H5q^dUiiELmX$%SAqCDsyzSaf91ca0HU@;OTF5$< z6sf1tc;OnBl&{)lbTJ&UXKCG$72gL~gDb1n(04e=@ISPGCwS_)b+cDrK577=V_!Zs z?y7*1*~1_8_ORoNQHYSMpdX`FA@E$9(@-P==dqbTJADH{W3TlEw<7Fd6cg}QR@wuk z%rq@HQ*?`MNr!znKRVz(v3$N|{i9lrO=&^2?!og(HjwT?A7RVRojDJqS*;Y?g;FKU z<&7xCknV?aGs?(Su5qRRwn2PjP)pa8a@PPTx^LZkMr?zF)u&glfyuJ%Hjv?Z(T;~- z!mr{Ve6k+^Qd|Z?V+X{DVe!OhB6*zi2s-MQiH zn=rZhv3#MGNvs8B!ZfcG2r2qk>0Z9CvF1VKXJ?UUj{5b0n`Wa zfZ2XHoE}1VvA@Bsjym`LDnG71==DW@V!X!jL4CYA-qQ8N0{Hry+@`Jze?wC3)i!8; zb6W2Anqa*$T~hzfLt{IMh>w37`h8w`KN7y67}X)B%zi`%Gs0m>;swcZb`R_rob@co=hptxV5IgLt#7p!mIqfNgj%e$uaaMAK$r;lkSrN zRjfd{{1ht&FR&aa$S@pa=RuNaIx9dNdt2e1!=qkau~OV(LI#mN>j zjIo>cr5nG-6@|?FL=uJm?0NBPqXHIJ2}7ijKEVGTH_P*5q73yQ(4hXDB`lleo98Nw zaQ{p#;QsX%Ojjx`AKOt(-&cnIBNfV$CWV)Liaj8aFg7|2szObD;z7~&rco8!+j{f5S~bn?`ANxZ|gg>r&p zt<&*Zqq&l}+I-5Iqsw4FJhmWsuS(`?thnK;P{1RN^{AF)I(1TLOKd=}3rVI6$Vr_R zdEv8L5owud@YuJp<;0h4DR0L13fNb$(_g8JGb2A$_Ed(A6c%x!5thExbOO?h%k)#$ zTgZY`O~e9A+TTodoZukSgzS10qTg*;4WIb-$(_X__l_APog-M{+sgAXE}*WyUpJOC zLVZ%Z|fcCANhQco8V7*33ETeN;5y4Zv&cV1Dfy%}#&ose@hdLxT^Q=rRKA6w=tH-*Ix zkx`?yShSkE3YE7geJF3j6)U$!3DS7Bnm~V1CXNfvg_AxAF{k6@kAFgOrsd`ZzNwM# z<`%SnVkzH#nbY$9a^z3UDSQ1I-%{-ee%LZ5fBHo}soJmugRDQX+QRXIxaiB)hClWo z9xro{pmh-d{~=?3N~o(>h?^A?24>TV)yVB#do&7M!itJ{f+uriJ<&Pjr#B)%+jktz1O|3 z{2ix0f``DuO^o@vSP|LC%qsLL>=;4S>|o!lN>Vd_KIFV1SudkSWV0}Xz-Gx7q1B`%EZ!;f zXyF>U+2lENwIOTY`kZmP{7rbh%!|Ue_#FB1+roi0*$xSz65zHJMTk)s3Xko>caCrC zs|f*Q0VowSRW~X|xl;sZPG8pPW@FcX%!3$Sx{MjD_Cv3yOn*mT14WOCl2N2OQXd)Q z(MLjErbZHslsD+X+J=z2hZ1X9`NL=rw!ZBvIl*YQgGgE27GuZLjP#8n>c*sj+KRjK zsx|e&<^)`O<)onH$Z&ySI|#=dha@7?+9i&(&s3iRUm@mBAUk6I7h~_(oLShd?S>uO zwr$($*tTtUY}>Zev2EM9W83Jwd1~#dz1MncSFJB|{(x&%U1N?h&f{>J_2xy>B?a|7 zrStrk9Gm^v!TV=I(9bJja}IAp6n+9=x#drL#Wt1}54SY8NOs72m4`WAM)GXA!8b>? zYDgkMdziNk}vR?%q6nHck=5Qp-^Seu$l%Z3cs`f*UaUYw6S zx2!R=em3}S6xb=fMObYgpI85+=#(%fZsiU0EJ(N5nWtqgz@&7l&l_H-WoC53>5QX3 zwam-4Mq^V8)pr{-7G#8J|_#} zWe6|Z6(Jp~BvP8K2jjgFLScHD0xu95OhaM40zcP#zZ~P&kljaJ5y_R-aAh8gNM6c25HWMo9wp_+dctDFBm|t+&l<5^N(J=Q zaE%TuaGv`ypUBf<5QGl+CF#iX0OdxO7Urvxm=bB1hVa0Z(*+ykBQ03JVbCto|HhSz z9Sd<~5&p?qW4d<-C02oVy>ErmI=cDQ6plkYo`yX(yk8Yh?gxIh&%vnG1%EJ1U{L4* za~v+v&wEFdP0SzYbVsieE+B%pf>Xakx)YSL?`tva1+l`hXRqLc%VmoTr(AqISP%a) zP84TE!+2?ZG&=UP#GXls0PrIISTa7Zb~%m5mc(m13OXX35}B!6cq{hGk*}3lX&t!? zOpymDPk->E9I+c)q)d`unI#bbQ#9a2wIhUW&K|ki>-DXSBVqF>XOHWYtpT(+`*LNa zjs#8+A`z68hCOUU*)LE({%3#;pd2cH{Nrh8rUwGz{@(>(|F7uuf5{q}wP1Wyeq;?j zQ>5<4Gi1R)0RU5fAR>9l@Juk$U_udm$Or?$fUJy369ZJ!g3p+ACyznRkN90H_=t0^y)ERj3;s{cwD%9T((#zu_-CPNCbZUDr& zgbJuZl=5I&xyAP;^okyl(v6;#19~aP_pA-fx<^>N$^cDh{d|WCX&BFzs1tv-T$)T~ zgxQ0)Le2bJVFbE;=n$IWR%u%GDozKunTck6`V+B2Oxn z`JV%GGH9)X|2WO94eRH;hk-M1a*RpfR!aNWp%vS+{lef^=)kYVHrIK4n>8%x=h0y+ zq0RS6dh7Kul`Zrb2x}BTj*RC(n3IvR@+KNx1(p26`}1?@ zl0~3CbJ}TS?XL0AK>?5hdPeX^Qh|f5He~{srn@;M=6n(-sIuma(}5(vx`B)#wsR*E zg3<14wC|M{EFp&3>>A-;3Eer&$lyepblf8t4koUwBH#!D>9fGTDs8B4r@9Q$T5gfjBe zQA8d)LmDpCoM;hr-hvhsMmFCNkUo}Ts)-QJ<0j^vZkCsX3rl7}I=wArte;4wU`Mj; zoZrBH!uSVWu+)bScY#Q6IFH!p_d2Y-+vN0q-|ei*?hrH9=-C2FXVzYtZ-a>!zh}=V zcElm>XybZc)9GjH;SCyaV8v|g0U+avuM_sqg59wWsxQ)-9Ce^3MUXqE8_q{#P(%nK z?G;nd*h0fzC=Lh3r((=cTWP+@ucg&>`+E)uPEHV%Ecjob@?UCiE~!Xd?JrArOnt0xYv!V1jPanbj4H zdDS$Zx9zp_n>%lqnUrD2CYTx3-Q0yhr*U8l@au$8Ynrm^|G?YoqyLhgoXQa|hf_Pe zomy^mM5QvynwY_}@C(oI5xl%SMnKcVQ$s0|Xy}&d-zV%MON}Iek(;k-n>u0J$WNrd z^m9L&sAz;Q6^p1`MwGq1lRt$2cxQG$oN~;8(Sua z_yw~87i zaYCSZ4iFQ2c7;pM{dyB5lj-!DAjb{!Y|x`h?L2CJSC{;f88T3M6W)x918-F6Ys3CB ztm;#D3mp(ILo(iA^P>1ZCDOc#ejfcA$&9BIk`_-!WfKm?#w!wvF>;9!r#SPB`8^MV zy)DRLwI$KohyNdWLy`X61*1xg@xFJ=h@<#)!ctq_i&arSRZhR{lY0eFy|4hT_lZuBYvp#k7_(b`fSFc zOGWMcsRa8^qCm-A*_&C2{roG7k6_W}sas5Le$bC!qLs?0bHx2NN7j18rS_-T>Hjv* z@G%}uO+p7Bn6>=$?2!R5Som=427OFgT*X#A>Ws&n1sWpibJ~DCF+vv?Cxjy6`XpFt zWs1t>?V&vB#@LztC=rHUtb7P|9*n*^#?(@zZ=;+#Mk81~qP$m(=pWTtJ~F+x4d}1FO8Mdb<1aVz$^_OIOvfR=+J#rsfAqrIe;LSyGuw!Y zI%P1b@E;%B^l^V}eB}zh`T8jW)LH)l3xr3bEGBicolOGwVBz)+jAet8+#hb*A5R$>lyK(lPEU`NO*Y z_6a#xdV26&Lj1!O9xLStmFNOLD)Mwk3Ar)pw8D9CJ^9`Xg4T)~j+PXJbYW%tywx2~ ziOy6u!?IPr#5hV_8Os4KDig5iaA4tNAs&y%54BZWt7I{J{XhzSSvBipU;9PJBk9;C zm`{bvKM0yi=SDO}cJjRypj&XMTjV!H08=ByW&i7Mp9sjfQgC(?Mq2a%PE3yB8mjCE zG^|+D;I`u{prqLctX5HP=2FkS%*^QGJVxSW;vd*m-@Z-wj8-j? z`z2f=NRMxx=xR%`#{njZg&1P}SJttkj#@wT1F&(;ez(?;<;64oG5fjqJX7sinmGIV zh$A>%+-@~4GV-tw4s4b#FHdpv3mN~2S&QwJb0WT39{_-#Nr1dET*AI*ZVB( z%iGNHrZdaqrGzP33@I}(o%6W-&;Su)mx)W)26Y2v*=@@2EJg(8I-d(H)6_TT{8nD$ zS}yFCXf5h~&1)24CyIA(WVI&h20q@~=vgNBbcAHv9tYX+7*lH(0a@5cS5R`aoy%Pz z8$C>1NLg7r>?5WXv8!hcC z;UkUsO{<{*ps+-9myG$Dj331XtF4XR#U`oW$a6Z1wU4clir@DR@z7OamS{4#vx-DH#ZKH}u8)Dm+J^h3 zJcI;^-8^TF`8tvhXU^%9piSa=quHQ3+hY3l0IM!K^#fCF(zZ+kN4A~ZDs`bNwz)T? zw)Z=%_KwZas!lTXmpP0(G#x&F-?!HpPNCuL6xC4+qwdCcFuC*4V{-2t5tV(y0>Q9i zy>NzkoWp`Y1!s?SiT6s*e=_f3-;uXzTv>vyMHu9tuZuq#Rz*+*?}X-jAR9}Nv;6;X z!hM_eWhv+bFOevncs^3{nQ3ku`1=IH7eZWnk~6k3?rL*M#y(dj(cn&IC*`mZ6kJ!1 z&vd+hk|etambwrWRSfR{XC6;Wum~=OS%~9%J~UXG`3AHa#JB0)QSSr-XM%(4_(SRw z3V^CWQv&aGjgLdf6QCwKNJ~wHp+@`*DxJ%!f7Vw7{i3KT)K7}@li7YmKhuC&xG4^1 z4;=k!T`hb+TfE6Mss5P-%UVbi3d^(WL|Q zZw}E>G70D6=;42NO3H%^?B=;bH}Toyo5gbxFXJ_Ht+Z0S5V7(L52>>ZgVB4UES_|= zxcQa1n)R7T)2vZ{sdE?Qa6Q>L10bC3vmC1inNPjUQ*aR!Tzz{u{)Ag$&balSE;F|< zAgA()Mv3*pAIJPdZoH{{rUq?l+PQM=l|AUPW7wYyd6{39u*guyDvY}KFO!wkM!`6V zunlsN>W<9ObM-$*P`X00LX3(5*&sO%DA~C{EiXf3z?eBk?nbj5Q}JT` zabSXh=#7#d-OR&J8DYu8*?_WzsH+cLOM{S;ugaKn9D5#FDWt_GNhi`o9>=b(ZUz?; zpeo41xRSzi?~YqE8IK%lL8lyAFfEOvg$FxLWpvU+t-M<2odaL7L|w20e2*u6@fTzZ zNniALPdap4E_8c+kSp5#}szX#QgIkbmQ-fKHWP@PB}7GX8Da zyw!1}h^+v^(FSU@r)3i_rs%L{E}LaV(FANYA?-mQeq#+pZvHCo-fl25TOOCNa57z< z>Zr_^!EbdOT(sPkQ%;D@V-_ireJ1s0=m&Ew ziI31KMyE0HB^+2b^h+y6Z9m;rG7SB}p;xLvT07v+@MBi9qfr41_cn!7KHj8W(Tus* z`{w(w;C(SiaZ1=hw~^^Gwu1}`!DD!HIWrLci5U|}gpCV??fT%T zkWOg@h#Fm;pi~(9uOKWONr+9?18?kjBxCR-+Xvc(EAG}$C$-Ymh>lnE-{mAT%htAUt*hE42MC4^If zMPaP|SC3aCq$x(3%C;s6C9F!7XiFyDzYOVzv$wFhXV$4$dL>fW4FmtcDqU}mbRSCs zmnS^`L_JZPNAy*q^uo0FRl^#rTrb#2F?3VV^>X_9k<2iVGB^=sPsAO0bED9Yc#h9; zC2E2&z;(Qj6A14H=)^I0OoKYmj>9OByCGE*Gp`zgBPk+t$bMVU8vS+Emf-ak%gnlH?=HVU1-i~RjF!U zXv!e{+kG)}Z|s~3m3#dDNZ`2X_Ppt=dolBLdl88P>XW6C4*>d94S3nJ-|DLCvwH0W zA?OR+`V+cUP~2LJ@nUNL;MAiqOrp#jdXfmsZuuaRZFYI_2=F|gls{yDb-7?(;0 zU_MoSc56k1gw&J;gCW2Dj*%dz+zX~a(_mn9$|*I%D+9$ess)+g4*(83oPqVmQbe&s zK7r96?GDM&MQFE-%LD*Szs5gA9RL=rQFLMO*n_Wwc8-8uxkIzW=n`n$0d2Pf6D$M; z1NeXbyRwE#rA{p4A&Z#%0rZb;4jqA}oP!}p^woqb4TWKCe5p(e zL50p{hblj56%LY)xKcIZ!&#O5vy9r46FE;k=fk^Y%VZmZ~83yp)Hm60mhZ5%k9gq=$6ff5Lk&|{7; z%3xlS)5TSicBfIH;4^M*2G{) zjiIr!pUAP9K2_n6axkGA@Q#tj0eW65+Vagrm`j7Me$%Y3%$bT*ovBuQ%0tw@)-BB< z`Dt>ZV7G6qi`al;++y!cHTdQ|ns#LNLd*87SY-{pbHvx85!->mGT7i_!!Svp`sCj21L{c_4%zPDQM`9$!}1z(>bMC2<7P`kW5XBiPLVjG{t~v#k#yoWegp0 z;$Y^Oq4gHKmQ%nfVR|e?oaHQLsW!6v$=*LcchUsrd~9;FTMk`tSOhbg?`oOar7cK8 z{{4MP0$H>ep*z6F`eOc7u9Cn+X)HN{KT*binp~m5iB7jAzbau>iqJ-c#{Chox)I_q zXK*(rd5G48r7J{-43|`6`|l5_Ywol(iOMj;y4$%ha3aymamw^=8n*@lx{YB7nQnwp za6VXcIO0d;O-33lSauWSmC|OQZ3muRmJV|cbZ3_x%~w5uR;udGDO9*Q-x^=zQ}r)_vaD^4(k=8UTz_+@W&DzLP``p`6o!C_>>J%Ye6P^a`TnIS z7)XIZN?H|2(c*37w921&2X3}HJ)9>+i-&V%T0s*V>xilMW}NXI7w=%vW`_tARg~^%d9{a~+uoRRWrU=M zA8vgj6{+?7F_iL^?uffIhGlM*sCk(|b0gCLN6i03hx7J_A<;DDSZ9D__%otSx`c;m z?x=&wkzQnC3`kWfQ=m5)bj?LOWSR{s&`Y30U!PJVtqPi0aG{f1Fm-X0=vZjkrj(Ra zDzL83phCuF!`-_wiEAQP_nMji$qrxKdO^2Q@ARpxlo^stK*FOr_AHp}&p<)#gmYB9 zI{#1%u!sB@1H)p=-DB{np3ZV?qPawRvFwF8?FjoyHKbr)ivsot0ZGji+ieT>E-HUZ zt3gFd07bv(XAC8JpY7;d2Z9F?fQ6ByI^kA?$kXHO6bvUOs}S;J3K4O8K#vQ zC0-IUkR5Xh7>VH3NTtf!Jt{a01-hM%#G5KT2>xjef7}6MaS%s0N!?T14alf&O+CL8 zz8JNM4fA*L*m~M>RzEpesXNj{-R1YE?o#s)VYUDUssH^V5T`83&wq)487*wdNN3pD1+)8mPg_dUoex-0{=_5Y8yWhzCLtuOLM*H@kt$5}Arx=n3 zKtmfasC)(KDIdt&o27p2qoV!?^AL9+OV0q#Y}zBt#sk1(<92(wkd59TljY zwB>hmx`EV8rsJcbVyv%MyxN>Mi-E@kv`RJPFsO;lQt_3S*t~Julj;^)=bKFtW z5nqkO8DT|ysM#MZ#fSMWPu)w{e-A}9?tN1*Ad{XQSqqE;B3oe>pFA$lr8bStnQneJ zA2V-uYtH26>Od!$7W;AUI>ltL&X?lOin}$f=|!WG^^NKt73;vP72s#Ko9RhTxhX6) z)OI|RjIZmsRWvDhWHnEuU=nAirHlk5oHz;3Or&jtD>mf$^b$E*N{BPACc(tEoO$cV zw?~^SVP0j}{TZfkw>C@SWasAf@pt$&OsMNOjaTJb=PwCjU4KpF%ayuX)U?Tc##MJ zJ^6tV{=VA*LwmQDUTTy6@?6P765ktAXxt3ZZfblX*hvpvqW=MAm10Kwl zbZ(-U&7mBRbEIhj({IQ;0P4(awleJ}YVh~@r0R_QKt(s-x}Kgeks&S085qkH`+KGP zL3DBsJ2Q2aot;)^S!r`)XKrV1t0{71@_Z9A(Gs*`COhIn<*Pk1#O+o-`h-9d_bmWqdvf3ft;U@Jo zWM*KhBKsI8CKoFK){;h|*zu9{eP^Gfe5qrD!rxW)GgE5)SPiO%@c1RN9^7~$x|YUp zBHuQuLVxU)u5s$~q3R5?8y803xtE00?}Tj0Db%CsEfb=N-2CHK6DDFIM`^lN{;*FA z0zzh6w>bg~afjfs*Ib8n_nz5iu~g;KmM__a$^M{a(nqU^$3tAP4teFVzyOiPiZY{Y zK0&*QU%_%78uv8z1L5XyK$iYlPQQ}<+Rq-V4>1m89mA}^3W2k{Rm#c&FywAWTyX@n z%&>>ixI9y1-g^YJH#Wbxdha;!E?5~S=o{xNoi=5{@cD2VPd$X$28kQVxC9m2_Lg~? z%kyqNY?IudukJAW8|rFSQS;<+dIpjF5ubu88~Dgd=%o7b=ZO`SZhUwyBN)6v3s65G z#IYoZ0*xwxnsf30-25%cY$u<5GXZ*o)7ql{d(ByE9QMyRFo2R6B{M;$Jc=;18C%hcLi|Yx9{q4~czYo4J%Km*p=`1LV+AqJnhQ!lxOiUg&)Cw( zY{RH19N#`gEj-S?VnP6XD|Bzm`?KowDc zKl}Q*;ZKeBAJ!Gw(0^a9nr+pXb`oU_OgqI?ajXAnHIW+pK>)6B`z>)S(_0*hSgW$* zG(%_@Yt98Kg#YpOXJl!m$@=%DQyVw_NwrB#O)T;`SOAg$<_T?RWd)7eOBM;kb|B&K z(7TC8IDoW!cZ@(jXax!64?1#x#qostPlGVmkZ~C0EsLbt+EPVKzSvxE9lZ~toTQ)I z>S5RfuDVRE8C{SeTY46R6V6Yv%EOtjAmLO~1sycvA1=##-#UUQ5(5U3|1ij$g*8() zyg5+#PR1<>h#&e>&LgijkiaNN?l(0OQ#2+QaMr{-Vco`=+q00LvY02V$-DEDkdUjM z6YKsIACp4m!}gnY0b+rc+oZOTSR4GYQynBgk+Rl}Y_(ID2~$mXk3PKr_QLR$9kaoR zBsn>H3*^IgxBi66T@O}#*no8~qCfrOUHx*F#$%xylsx=I4Kpep#8}5@`1LjNE`x^r zHu7&`1XFrb;l>jy>a&m~fo!eYJT8ixAnq4JZCFsm8WoQLuRzPd5-Q`OG`tDp(Pm$x z5sYk{(E6{(A&6!0l@_v<7A#2Zg6!hLwv;v%>UN4^B}Ls+dIt=<5NV|2EGIO`gla&WtTlk!p`=*8|?m8?TG%azlnn(h|7LNm8>UfQsJg# z4`+CWbQ{3IZCwbll;*x^DJi&`N;Jh=5HMrE_B)QvbqQ#z)sS`88(FDhN`(6b1K&9& zx11NN~}Diagiapi&ncEMYUZtgw*Qy5O|bNKYgB znY8npJHo4br>E&xX3A5C4Lo6EuYwNV4a8v85-pX?#XZF`{Jnj|2(s$~X#Ka2m@pmj z9It&)#cf(m+!Ht3!PRW*=#N#e_pcD@L3yb08UdnrHe`RYt}IR{JW1aee+T_KXCTSb z0jAf08g}4~4ok`h`5h`gw+D6#MBqU+aEhwRJ>J*};APyCz8%a#jleHHeW3034ax78 zP~07~s-?$Jn!JLp8z;b}f1y4JOaUk0PRtMxbu<^sJ z?%_Dl`GKqg${d1JfHmm)&}*bQ zuvsOHHVMCj_|M4KsJGz|q+SLF7lP__yF3W>k@mK(OiAz8z-MZD#n0ACTKs{6D3mX z;0n@+2PrFO1AZMKj6m6~n6RilCFXMRbh=~I{Y#)WSbB=64O!r{6P5kJd?GP4P2qj;$p~}5 z|94IDZ++Xg=$6)}4f&a-i(TL+=RdijhoGhhN6gMJu=AgI!5});yU7~$pa`=~62K74 z4^*KWrC66yF;s$Feg~YybdA69R6Ct?u;|f?VhXzxXJk&RcwR?~+ZLnEa+-tDjpFc% zs0P&(_zSWo5x|6nFK@5`rTl{EJhKFsd}HEagaKRRhHyssMMmVikdk_n8;EyjqBW_s z4>mcTjC?kp5%CDhT!NsCY$?Q?fF6HtKP_1--@oYCcak_K#m5*6648=e;|SDP>!5-jVT`B;oXbSbT`9*RP|{=)x^h4k%S{PF!i z`gR8(oF`^K!VDNtARwXt-PYHC_3Zxtkl`hXv%d!f5l4DMgiF-!7|#2HgRsko$n_Kv z5W<9sY{E({?-+@O$*z)fNUMqRtmN28bE#xnLC?2idpoxJR&G%q zyKBz*xiUyFx9>RI`S)vhL4(6KP8o$Sm(_OuA^6O18+)64Y$(J7+$i_^Dv@>WP0@xoHq89{_36f zfS1tq1!{(TSQ?}$ED%XybE)#k{l7O~fi~zEHo<{_$bZ)N#Qt}OC~9hE=xX!d`mg`P zanh{rt&Spr`VChXGf9jJCZ=80Vno=tp;d)GbibqllA1Pb<+x63$Tn`rfs@>GIoU(- z{Po(ZpWngpR_@)cgJRzyKtFV2%}c&=rhf zqGG@rj7c!+W*{(Ps?!(c!>t%4B|Dxz5*;1oAyvAq36=Kx`Zsve=OZ%OtBcqeUfsID zP){7s15Ye7+7oqWuI+#0e zy^&bjF(RjqH(7cNXFlYll({*muxMQ>*BVzq!fV9^rP|2mJFkGz+-ACNCEbWeQX6Zo z%X9H=zG048`KzX_UqQgC6i$9GnLV#KWkgYCHIXIL_`bbYJEA$^RkaGlo$FY?Z z4dB4URRoV2y~$>QP87Fkp;lZFzCzoOju98CoJvHdfqU}VH)YF!*h{89Bh_o3NtfPy zPI^6auwo~+C-=UK&$oCsR*EHIWvg0*dJFRO_G&pmqK@A16&`<4saVRHbK}}nQb~ql zNotj85vrCB%1v&yX#tj28c^zt5^BfgNI&O*NpEihig28q+P-tsAwGdFRQz~h$>5!@ zz)eMXwnG(Kq&-|vEdz6aEu%?}u)748rJb5!Amfn&i2I&T{>C3D)J85-H5&C=IeqohwrydM}hd5?O`NU9%hE4jXUkFARy00(ZrPY2bC+7;uD zTFyD)K<>duKlS6rg5DYShJ|2{3JiR4%)0mp2u48j363N4Dcr;4KZ?p@g0*R8I#g~D zOj~@-j`|QDNQ0i)iw&+r^@*`Z^@+Mc^@$@u{S41*F>xC|Wz2>pGz=<|BhF5ynZNUa zGg|v598;9`9v$DPMPe{0pSRhR9L4GwKv-=Ii)zgy=>ko?{v>tmr^p;Ukx~1OxAtTm ztaXrzy{D{QhF&QW*~+~yDGSeG+yrG&{=NrrZleQ90O4+AN7>;?#f}UQm-eiQ zHUiTbM`J8kJTr%1>{4wpeU_$2R2O+7A*AvChI166+dt%|Qk9i)xc^DjOX?;BTGDY{ zWlDwSbQoLIG8wL=l5|Wyvb$kC^=L33I1t)1eJcG(P>b#dxZAUd4wYu}U1IXVFz9u2#q1H>;yc-}P?hupHtD0V?dn z3~B=op;N(U3sDu3JKLmHxtKHDYAdC**)y1qJ&qN=p&f62@7hrDK0E6Xl|-5N;E(qA zQklB>4S{uTqH2BD$f^;8k150FOe#NsWTfSNM*!r2fBwe2P$ID?4XYPTj;Je=Zh+l> z(F^c2*92oF*lZ7*%2aLs-v;Wi^Ie6;7dLq11g>B|v{CpEXb=jpY6>rc+(Ln|t#ISw z-zA8iphSiK2MOM5{qaHBB3IPOYuM*zBzRnxCf4P7CELG3(RdfIG{p}1SXCEIRzVqO zXNXwiV{vc&DOl}K9N+n5`rw9qK`0_Vk`)?2W(!7Jo70}YQaENjNVc zzLV==E%b(ab`?bo5sq#t;ZnGG-PD6qXjUluNJtBDT$6b^MW3@#W+MN={P){qIA{1k z?xzG01OWo#{omal%BIGyPL?j7q_T!yo|5)9CZ$~0 zdla%26^jQW5*si=h@3lUgXjKxJ<&$9)|-eQ9Ox$~K;6-)$+ro+#qzS{JPkXLsD zG^4;4FQhq`8UQw-S=>-#C?OIeWY4O7wQ2Sh3s`$r-Q737h7D$mSB8d1XuEi+K_Z}; zhru&!4o0EInySTOW;+|-3ngm*GiNBd{FQ4!xwe2Z?XZMaIo(OmM7HVPwh53He@m1F z2jwzkwE4&DeY>5j0Bbh*5M0^m!;GZT`O|*f$wSm!w2tj$HE;S<;M&&{*M1PVq7dOf zBLB;X8LOsYEF3iB@FrjmJ8V&R=))+ZjP@$eLMN z?SGE(-;YU+F~=$z5(sGY=aT>Nya3rdnKKwV7#drcGFVu;{XC2fT?}pP%^7SB9h^)} zT#Zc`{xf0ykL2b5HObbfTe_pHV)>0|i6=sUnxPSi+6awcf=g01P(d+~Q#Qo(!vJTs z8C%jKB$y3NjTyjFX>PD;w>HTY$ZQnXbe6zL(Fj{4)i%jx`Pm3OKW6(~Ay_Eq&*mhU zLebz#ddK>@p3HRe-#qlFd%a9(+k>t}3rN%>wL`+D8#WOV^ebvGUqXWHG>{Y_3Q6)t zAEaxy#@DCq}4|j*M|j65+AcL z&rG~V2ESDVUZuC=FJm3_7HNhCTbv-d>d2wu@@3UGMZ57TdM%R^wf%T(&BMU-yDS9* zd$(n+k&oTZLsxXiVl341xD$`0jLR~V8bK>dqD~eHcCi+1p|ZH;Xp$5jtgK{ZqR$R= zlq!qNre%rCj_K%hB`qWPt|}X~XRW@X1N~04Gi}XAo1dN9kbmuS7+Ki&k7Hy*OeGXE zS+%GZuaq2b!n1ioXfBVDdf~}%FuR~jWvmO&oT@q7k+YGW^F;u&Y3GOg3^mdlK1@r9 zx}QN6Ni=6eK-d7Ax(4$~X=a{Ve29^C4V8!xk3f?iW}nX$saCn^?$98)?_;JAYx!c5Gs5z0a&O%vFfBAFyt7s}O zmS)IDZx}JgHe?&h7yJ=;+uaj0viKc1TUbK7=(43=u!NZvrVVd{rGL`@tkS|NPttAH zFKuab*Z`ZdXkS!cW`M#z=^@q6aVrN7wMBGb3f_4OU1I(e7HALHL{9Y)eE3U;>I0z+ z-4k^~^~&#;zZd4A+8=jo3|5g$6LKuHejVldoq0gqviCF8K}f!8MjfJEML!2D~(@-Ej#d#p7gGsea0#D zG>*lR$rvfmHS8SW^)MzQ9{F2>zm6DnIhqcta$E5v+MAD9PK%?>z$-WlP6_6yK=GV>!;M*Yti zTy~?Y}OSO6b#E^O^3Bv>@BCBGviET^=FNV zjA>MBT!k8gOaT^EP0bm%yPmP-T!-qTI2{i84(mngQ{1`u#+TOMB%1-xl4D-F%Nl+3 z8rcp*EJYF{k|6yA)DghYD3^$EqL;Rn!PLQOQP;Gv4tF+zRH?W(=jE}8G=4qABUP)Un5S=k**#8%gwBjc;DKJZVCdoad%-IXYouW+IstX z>?j9nW%~gBBq>;%%A*^H(8yAArq~gqAMMVnsL{7A75rG{%DTh&_@BP|#qTPp)NE#F z&zJo+_%|z`oKfk2q3dG=?6e(iN^LL>vss}R==@}_R1u3$Cqy|1rSR@fa2;VZ4jAgg zD1B-^!Fh%@gWJi4PTPc^;-ZrB1^m<;^yRFmv^dm8fR(wW zW59Alby#q;Bgm=7bn}~Ity$qW+oDgvMjJ6|xC8JOZ4g{rVNyGvW%B{%jskl5Ls6$g zG^YU79)!BWIK2W`RfE)%s3G9_GZVxyMR%wudiQTc(_hawh+ixH=z%P>JVPPOsFqdE;DaeSxNb+z|^ax>LsWmYlDg5z16Y?opjcRS% zW(!@b8rLcqRhezA)?m@|1*>JNZEb6`nij1W-zxv<@9dWyS+m46M&RC??>onnosXIJ zldZ;GZ$QNN_6v|d)dVBL%!nPw%mBV+&(q$Y(_{R88PP}x#fe)5a#K&rES0yt`@}3> z{QJaGrPwK0q;egEqj?3aV;TE&=|NR&Kp`kE=5Gk-A<4JXq#dD!=tO|USxmxCKQ-Py zaW4eKVgWNg%`~SabTJmgG?*oMrOIi)C^a(k@P80idYQ5%delXO#auciRC>t^DsOJV zX|cshDY5rP&~ix`MXKE5zmBL?Js4w3Wt^+S)N>0K9Pw2Y64K0|f0sb8oO&S4jEOxQxd*T=(^on!3cu&FM(L=~BsrUyyW(rf$^@?{ zA<&7mVn{4FS)yXpAhy@4oS7ih6e@!`G>82r+m?AkmknZ#x$k$F$4<#qu|mHC=PfU5 z#y@TPotY1N;!z$V97vF=R~5C=Nx5G@Ieh4x#rnf}7RNEA$x_*D$GLp)zd- z8M$o-yf=%BR8ync9_2AE^4l?_qYD$pDySuO0M&UK9302*AX{jZOBe-c5aL1yXQE5Q zh=bG+9W~&RMQ*M#j$`sYiNPAKdxM=TdXlu zKwwK*9wF}X3IUy^^RgbYpw%{E4JL^}FE~rW+XDeGzg4b6#ApIi)%b|J&xTp3pFwn( zg1!;&6I+z6$NhGC>NnN7{=SD7FK2Y73Qi@i27bj$%4t_O)1_)m7bk6EmLWxAWf{pD z0{j%p7mF^aJ+D$%(!2AeNbNAXX+tBHZ|a7g&60)vJhHpmzflICR14S)2lo5t(F>>H zF8}m9Bz5n${3@a~RZP->&qB5rz^s9pWe>EWlyW(SC~SasTv)0w*-)izE!Er5w5S{N zww&jfzCJSLoxVdo-*}xVh0S#ox36av=d9KThsh0!g<4+(EfBQ%WhiFog}>g5^dwSb z0Y&7j);1rXjs%IdY)khfUi=$3QDpMdAqZcmPy&Rxg(BiRwe-NjnVk)@hFEe((yW@R z?>;*r7VP9#p!j`muxgHGlENv{cXKHe%3?AUz<(*m#xBO*Z17t?-`Kh~<_zv)O+Z$B zYKTp&3C<*znBPdn`_z-0CZL-gni}!&s60dX`JAG3N@dg(g>r4POC{n> zm6a`EOyWs&8-D6-11QT8*%{&2RJA8U>Hj+^Mj z(kdWWV9cwNTys7*H32qcIKfprPIwA0m+<6oqA`k$f`zRwWI`<_jCXU~66s;FQ7Rqj zvnD?2hoF##o*U1!7qBzM2*>vZa`_RH_a4)mG#+ZOxUY%PlVvg4TBr?x`t_wt z8cPXRNFEfJq#g}+wRv}O!W6l%lFX_vxgOk^V+|a;C{|1*>U!9Z8cb<${Xw-3(ZRR| zvB9Ud9~Y*T3)PqGU~^lna;iXQ zJ~8}r_v4!ebfcZ#91%ESdP8+-u69fJN%RW#{j3=hPgJZKl22Tmq}f`=_D;^{EoRMv zJlZ9m|Mbwb+Aaxis$VQsMBf~4mCPd~4&0sx#En}otKjgQ|BNLl30Hd))dR28RxK7> z_=m$U7mq{%Uu5y;OmR@Mj|u@iNFsj06o)^KA@);FvE)t_JMt`hpwIuV@ z3h3fyYo>`*nx3Nh>c=dJuVLC87>!p@+Ye$2W8A4rnx4Y>zO!8%fq`Qy$!0c25n>xU>)Odd?}fhn0!|_~z5xeWob0cz>KnW~LZNy)0M+X`_ zHq?I?9r<*JQrbIEK2_klEGw1Ic9aiS@6&hN>rOUkiBg(+XHKe%tg5ScjWTqoT0yge zPAwMlbL^f0TS&={GyZ{Ax7D}9psEta^Du3@FEVhe&1->tUH!m~oW?4xID}~!!w>h! zt(1(5+X?z;$X`S0E3ujiGUp28J4U-vF>=Tdw-Zl1-WyI?lmL*(5>Fu*mj&~pjT*`7 znV4*BmS16*JUlxgKN;QaZ^V@t9xNJG_DaLbkd7;tt}~g|YtSF+D2t`TQ1KMD;ywUa zev$61$+RQxOP!+0EN=M@hU<2G@D!bXt#)TG0`2PJaoT@gQFhUxM!4~mwT=Oi-8-%^ z=&)na32O3WH+?LRqoW2ZIG9T?53}=hoke9zti^cf8P`4Jt=x%HTqL(}c&8_HI%>^U z?)vgG7vQPOQ<`%I2KpTZHY*pQ?#r!yFjs;|j|MV3!EWz2u1(2~e>0on9b>q7)Qu|h z)yex-^@UXISWVN(xFZe07wG1XBj=YF@vR-LjgN~ut2(Q)zW+H@GPg;4$!NyINc-6= ze2OmBo$-873}KoSH776F5;BX7EoxjB0Ew#M+w0&+lNSP?MWB zi3|EIBAl9ZrtMuJB7Exyn~r%eaN5e2sh6;(H+5r~gFx@f)827mZTQgWCOB*b_ zhsWu)kHLRr&oJo%Y0dT|^5`TezJ8{nrORy~(9Bw$b|iV2p{2W# zG~;WJkn4}1w74@i@f3LUFGG9s@{REAp9iHc_PzF?ZitRsk0qGEvb3XmUt>)M6Z6Bm zk^I7SsrEE_x&J^;Za@4DX@3T~$-Hp$7V@*&SBY0*eS7{IYp5qNK1-HH)0pck#s`j@ z*Xi#5L$qz^t6e|8>s zd#(V8CI^7YYegH=b!BUnSsoxcTxzwU-eLKEU*0UY`b94 zuE`naqVXne)aG*&Dv2`skLvp^_+%dCmVTA{&=1x&*pXqSWPn)4Rw#;h6hAQ=kAJw2ua!kBm1(Fxm@C7nAz$I3=(9@q4;VE}0)?sh00e z{&o(MhT!qbycW98{+kJ^TY)aov-;D;rH{o4L)REPGq2tCuwL(&COEv_+`P{;9yd_D ze>1O^bk{N&jUjt;a-V!WloN+0oLIYIpBqhBn4@MaAp_=YqK)8g#z4DIGO?l{Gt2D^ zY+-OmN#Qg{e$jEIsp|D?p_ZZ?>IDn#E9>?^-qnlmBaF7Jc*Xb@oy{xtjZmS-SHChUp;W)~{>WEK=P!rpK1^6Zvf+$8q)nS?p;%~)f?xfX z{*^0iqqu-4rOgby*8bG#jfg&9K&r1FL8jL9!@5Naj?NAH&@xK?YK$hu#)4Xl`LTiT zTHx_S)cN29#s|&iDVnhE3`rx@aegQ0+dY1T;my>I(2xh)+t#MB zn~^+Z7SEsQ%Z5D=3L(5=P+ROezX)#b;nP{ebiymKDUXmzSAT;{=AXPsZP*XT_a=i{ z%9uLMN_m#jTj{Hzf3dYDi$%SUq+>>bQnE2HUPlTsqwjkB%mU2313C0a^jQJu`;Unq z^LcSuMTFT{=^DB%*TxFoY2pI}W0S1y`$WWJURS{&{KFyGHarhfQ)g=VHn`hV=j#%u zG{6qIWZTV^f1^hVc6a8fZbrp`_cFMruNiD4@anQ4#fQqGy{@%8&jWtH2xFP9`;vP% zl2O6f$}I=>+<_qlz8KPLVskL5udC*l=sCydu#%IQf28Hf+W}&|Z-uUZERzH2S|+&v?2O;`i)*2yXvedZh~t z`nEvc(!Rb3Q4ftxI=_fjj+ReSzsN!FyPEU&Dsddh9=rLK<^wIt70Q9LgnRY8l^AnQ zrH6&aY&dfV+)sI;RPmk0gV)dhT&K1>X|!nwK+@KxdOw+WPqZp~kQnJ~^_bVc^L(0G zjW^s12c=?&w+_?G8DoHS)t|#<)9b?m;!!7iahDYj?pxvph>vycv?5%4y?`2IxfGGJ9JqCHq9u5(FN3K|PUV;-7Lz=t#asmXJ3Lv_=)e)LcH z3D%OmZR34Vv-*@@XTVEy6`c!nF_Anh5`P`OL&p6;@pCMb2HPw4%1=)mxRIc?8u@1@ za$yaV(M;p9Po|{;`O!WvJOGKD<;nqOIhqH9fJZCj)Ci<4Sb7Pc&Ny@bfEygL86Z~6 z{x)Vz_2J79wr&h*S`ja!coeC$sod1sK4ZEZ0b)~_xsch}YF5sJ)L(JqwR!h@xV;E%TRm8YdPt1|L!WU2XUu{lNi`gPAiX=LE}HTSqFf zk=CxFP+I@ha_L?PhIkXYFGy?cn6@ z_TPA&(T4WIQ^)`M=h`!4wm>Ss$dHnhw&0LNp(`^dPhDcYQ6it*at_}j$FXR(F>}+@ zMGYTCWb{*5OdOmW>}NFO28e@d(Qi$4G)1uyT*YoiG~|k}{|ZlM+a*a0yga@)Iqr17 z?ykP~rI#D{UOE4WwyqDe*s9(a1t^N5-mKpJ5TFYub@5U{AWR&R)PKnUdjW)n-$BFY z4=Z!?QrtE4lj2b9E5CkHa}R^Z*=A5oJROFFjWH2P`|0!sltINGLc=Q_G&A@yV7k2! zM1FV&GNK(sb5Y-s!yf}yLm8(|HIMF6@OrC^IeIA?^BD$Lf$-dZs(ZR1yDLhew^E~X zRu#jZ$9FX;e|>R!;|Z;-`dv|O!%>bTaRTUs-+(p!ifnZ0DgwXrCmjWozk}Oe217I* zA4mbj;f5IdnlJ9i0m0?q0knn&2|xN!UwXm>_u^5$6qBa%!9R=k>tA$P5X9}g#IdRt7&HCE8mzK>XY@orbYYNbHXPEM$u0y)tl**+uw(P)X(suF?Seus5 zpfX_A9l_dgk!5=+F&<2|HBuVHkU0j!`ZgaubRoU(aG6PwF7BVO_9NEUFFAYUr$<>p zR*o%HnYM6ESL!OTRb2-;6d|CCSCNt4E-NokiH507f@rOQx{WAktahTwe*SN9`uwPTHSB?71-H?I zJ#`<0gM?Xk#)h(c6J;+Z3pWlNI=X31g7`UunG^?BT<+6~nvd4_4$qdLa7m+=(*nF( zo;g__D=9Ob@p%6#GwG>gJ!*0qiGP$>I!s6&4y(Dj)`jdd9U=uv#x*r_VOiRqTzatN zM|nM(O)k0-vY7zl1Jvd!VZ$_0Fe9$+vP8i4u;cZ8t--!Dscgj?aoV( zQEmu2>T=8!O?dNwkp&rzXStmGYx7{-8pShDVgr9@|5-AbkS-Y-2e->~1>A%%LhvOD ziw*3m6&NolOMDuyv<&vuHFg9`jhCa$X$PBXwfYdM+sovIPM{{Z0V)?cz;4f5n2ai1 zY0beMJXXtbm@MnL4@Nkg<|7}h$`)NyunTixL$n9EZ=yA+TIoDtwp603;Cm%gkc<0d zu^;|sWB62{0yvqQ%S53P0uK6li7NB{BidBf?`36D@=D{cRjp}VNyv{NigUKiGAHAC zYns$EwpbbHhnyv)K@-R-nP(dCl~59Xi5T{RbDq?BxLk{*Y$p3Zp6-be^OVu#nw+@` z#f#uG3+HfIwTqc*b&GgeQ{sx{%~k~IqPeqWuOZqpm1-eNcLW#;8V3~$u;W9xw-GLH zqWt`Y1t=`b!q8QJDwC?9Si{0*vJ7bdP(L_&&<%r423Kj)BVN-I9%(y>Yp;Z=(TC&Y zc3+`r_#`Tv{xPXrfVf?_}1)P{n^FY$-bYtpYL6b$Sd_3i`7^Ci>b4<9&(o@c zs|lXVv4|mjy=4K%UZ|jJovpBG4OwV|S2=hrQJ%}v?K$sIxgI)u;3$Oy-6>TbZ5eQAH8HOHiO~7%vFuW{jKG(7PyqatB==DSafEbp6k9&lZEcXivmp4WDRFn{z$A}&PnfB{6^0?wdz{<|(&)I; zVJc7`Tak*an%Cq~Ke9wjZaX2gZheYjVbVXgZB5yX&PGAP7I7l{!Rc~46`8ZX=T!7S zDKWExah2qaWzH=BoJ(@-jy$gZQ*rgGBOjWV4+D(75i|`qdBY|;!nA98iL@6qg_}v^zc-^)ZhxN^d=H5c%>m7M_-pz|PyJHSxvugNKVsVy1 zW7hH0-U~Vs_^ixwD&%d6meBa8 z$>Of{9^t=99m;F=u%d>1CR~V$UBDfy3Y%1bnUuq07-@ht8-xCqZoT??zHOCX zoG~fl0;0(v`cSm@dwrEbw|`piv~(jTI4o%5I_dDkt}?N9_N3|;e(gBOO~P+Gj%?KD z1M7c1`1Z1Vs5WeQ-v1ajNAKf?iU+J#2jg(_u|>b!0ddK!Xy?tF626g69%)ZLFM{~W zaTIKzL^%|QUwpKChO`@A=B#twA7lY1ngo#ehtddbR74#BtZTjtjvs9%xPq{a_`Fy$ zB&$GH$vXAXMAhZ^En$$Xx4mH4)+nh=_=wy!@QQT@z3~vn18)LBMvahXGN@ZikB*NV zO>{|uulrZce!+Rf)w0Q?!O74i#9Bge=d)lwO6r^$Dsvf2X2OZ~t+L=AfEX}0eNQ)v zn#n6InIe#a5z-#o#lQ@jqH&!=754R>Mp2~xO0tph7K#GVXrazJ!w*l@&-m`ilJ~SuVCEyKdNZcu$)=GpPc(1d^)*x8VY#juOYnkM~_L`&N+?K z#~75(b-Gl0=XLLmMn9T8`D0iim1f;eQ>J4NYlcs`PDY%coTq<@FwpJg75>rp0wI_% zFV0a%$`dP9#1&d*@a)y)jSKx(@r2$Hy`_W4?sBKmbc*rn*3R(r%~=62dK=u~AK2{7 zKpbg#9R2_cr)^tED4I+$SK8z$3-0`}2fPutY}=rO+j@lhO%Ruc#;cgSpr3>Dl4cFX zh+;$FLuDM!Iv}=1SYZ8Q!5?}00EqJG&;b00hm=b-ATTY#*)?-u1?R*B-bg?kHzGS9 zI)!zLR3}Urw+E9mtb0dSHwX*BVZ6Vf)|2Jtl6wW@xoru#I+HH%QRoC_TeCY9@^tHl zj@xHUd!bLyh#{M%GtV)^y(Mc+i`&6HcLTp{U}GA43ytWITC z%U{dOUC(^f!q;gklQ@O;>qaT;`=G(-Q9Gsf>t2foR)g2=3!Xf}=z*;EhRkqBD>jb4 ziV6vXI8I504FTdx_sxis*=Y_K3u)r=bg=iU*LF`|VXDlHYBT}|BAvd|)^}hGL^(qG z5>&JDp$9nmqr^q@Znl(4`9r|2G}5b%pKMoEZm_dq?kvBzyq#+V<3@`s|71*o{Ej7Z z!aTj+&WQA7hI}@rzd2R%`VG4fJeg{Y;Yu&6H%GCoG^EhFJz7MIDLmuQzo{+_`@PA& z-uBFcrSD))nn6zgTW9Uf-9{(HPHNC^=M zUVES~RKr&r)-o3YL?q@G9v-u^na!7meZM}y8e-I8O&i7%$GTZyC+TRKOJb78hWH4M z8>g9&ql8@AiG!5hcEW4nA)e9-0XM$J!#x!P+j--}pUe=|xG(z1Vk#kU$Je3()42v@ z`j*dRUqQsTxAk3oB}o&2;bVibO3A=S9vfYRHhP;~j0qE(n2+C{!?-uC`p9J352cT0 zEu;O_mUD0U(^S_w!2?>)n?Z#w^!&hH&n2`00>@5~nf-Pt3I&sS9%-EUK(eseHbOVN zo#R^2eY|l~7J35b)mtvfZrliudh&`^!U&_O&1dN!)uctWfe?$!I$2U=GYu9L<{l%O z_${ndGz&!mGILEj<&=XL*Ac3l3F*0ihx+J3ejA83q6 zQP)2R9tHbY!b-J}k9C$2VJ(>Yay~p~Rexu$N5LywJ(t2cF}J=uaP=y{@+3m2?c0br zREOn%bR|=V_)slSDDnj1{6oXYDAkK^RGhC+LDBja+u?v42(og(>WsenNXc2=!Gurk zfsIdXpp8#$5f--u$6eK};BOL!;2k=>zMJssA~Lizq1mZV%pkSFT*RFId$mG`o%OHP z5HVfuHrceTzX*3753wCZ2)1;d1R1Z)kv>i0W~GrST6a23Z!*kmY>;4C(CvW%%o-jz zr|Q5nkVsz8RZUPjW61SU6|)jYlfCQ-@_$d;cUe?yD&NSC`DTYW|1)GuJAAv#9E|P$ z%SZn|i48A5%ihDWBq|Yo6~McROWY03N8;Kr#r0Gk-i^*?owEmCs^7 zy?j|~N9_zgC@X{o_Kw!yXW^?i_MN5@8=dYEeM2pQ=Mp>Z04K7VHl;kR#-qHVgSVbXcJcQe^Dxeyf_?afR<~7db z-PRdt{%<{sBfep{L2jhEYoytI*>3lk8PR-E;E0~QJC62hajG#5yWfaMnM{rP=(^2{ z741E0`<`X(D|4;iG;H#`pd@vB&4Kp%myUJ*_M{%6MZDMAe6(J^;aasp%nx?vzSird zoUmqH3on0XYxv6U-!e&Zgt4ZbZEfPLTJ9I{8!|nOJ*p&kA9_9`);#9mJnl7Ib(WH( zM@$h@N8u5&mOGH`Mi+IYUh53oZJ<57R zKSr5@weMZuhLa6Pa10U^i=}+l#g@mI*2SZ6C2$eQ%ktL@TO1+K`H%H@9leUO^;%A>+Z8h^A&}x)S2MFyNT)S=YYQ5_wBL!^{=gO z-y`gi4(g^e08jHEBG7jbu+rZ{Ap=APC1hs~Du#&5q#COTe4m_;Jv8>K0eL?B35pm7 zObCq6&WJ^Y5HQ|LHujqw*2`f7G6~db0&OJrUGscfbf#Q69A50j#XHDF2UYl!DF^PsqCnD^wPAfQ=b>t9sGAg(95D0A z4uXwBcd_9}*kTIJ%Z>Dq>kmyyFxz={VjNOB?< zt^7&;NIjm{@0kobBeN0t$eRpVNf5ee`<}v(ujWqRXGyXTw-y--1SHEGgU(z*U%dp{ zKQ*k-{mZLU_$sR@S-He_O5(o}T0mG30Jo~oj2dh!58iicSiZcJyLrg6jgC~gAOrnE z+SO&b^~Nr6+oLc!G7sIM*n;y6HFDT7w%mL1*rSHbIzQee1sJ_`&%^ptrYe91je~J; zmg*lb!c_FtZxR%J90#hXSlA&0W)ed%;74ThEb+*T9W@kTTUl~Kci~O{>L<9hyr$*0=S%FeJnMnUEVhxGqC^I7~Hn z&Fnm)h`kTTcaj}lhr0@cuIJ?uW>lpfg_*<2p4@NGc{IajmPVW!mNNt~n6MtE#*BW1Vl$b*9X5@=r8_NAZW1yN z{KAqYfA0}cFPk0ibdgdQMb&<8iI(>Ds6p2g!vH-EE6@hEu@Q1%J#PC8*KqbFVFTgZ zfG<=s={s4G`~8lVY&b0O^zlSngB25Olos*nPlpO!)hf@^jVbFDE;XUKYvvPvSH`r= zGYOj3wCQE;?Pe1o+Ef|Ml#IU|Z#U<>3J%5Tw8<2Wv$)*bCLU1Ya$C%p^BA-|!bS&0 z5PiTeN5LXGE3Fn_iz*-Q8ctc2iv7cjs>-7X+&<8di*|agBoZV<)3ln@(x5NO=6^N( zvudww{j}N>*M>%&hwf38xBq##juHA#YVU}-S9KeYUzL(n%PH%P{x=DV`JPUbUKKsJ zriNCNN>yeK4n4P`eJ&$=Oh{)l?#V#zUW`N8@&6pBy?2*jmj$JanVOZW>6MGAky|rn63_4{OnN>2SkZu%6K9 z2x=^9(suiWv`j{So@76rm*zI*Q5D9Zo7>gPy#K^bb0;UdTEUE?dh?^XhFo(}J?yHX zMACr|HPB(4KsJxU7n^%AgB2!+IvGT(w_Mmr{8Nadx!;jnTkfhQVQ)K{xw2%)Ob?+BJY{+YV}2d?PoVhf9Gkr3&$ex?Mlr9CtFw5 zZI-S%JUo{y)Q(ERIw^5ZW%0Jsm=Vv*)Ve2%Qq$$pHPwFXq;K7H^;G`1G5_Hc*T9RZ z13M;ISr~Lzhf|K>k37zMoM4YCU0)fhO2bf-(;8(7f_kS6=|Md+d6wv@wwBvu+T}#1 z;@J0BQ=4R&*R?s26d&WC`Su1Puj&vrB&DPbxU#-c?q=&SD|$J@CEe(=#uu9SBle?e zlLCiT7^F@zCzQ2WixdC=C);pPh0BM+{bclbx1*|rzHJkxyW-7GM|C;f)@!-6Xr-_x z75e3V^Hj5YnI^=|A0(UflS)RLJ-#cxGhmS-MD{#DCFISFt;&4Bdy37$fEgh>N6)Vf zaFsH?jzC$O{bcX4U~#d&wtL&Gj%LSj5-KiHSYt|tzr4dE9FHlu2c}IkLZ8&Utpz>n zHGU1r{~j|lVYTPbdIo2U%cw?*1S3=6 zs9sOZH)f$NXx;q`&HsTP+FEqg>ZFNg=J0ArXs)19f1m~!jM=!E^8OegVSjT*KOuzF zxY3(g*6HQOE_7Q~9!F-d*sJu*@&jcKX+Y>Iq@aWkv26&yb>V~eZ3e@*L?v=Vw7^@z zr2b&(@rC~4dNd<4C$CJc$mOQ0gh}0sUqDWAUykGhGX>iZR}uhuxT*6sQlF{89jQUt z3T@R_Keanl@V7YRvbe3Q=Z&g@RBShNY40^%9EUp34cy z3-4$aKPg&IRCG{1amWqfd7p|$1F~h4Tq2jp(y>NjA4H$6N|^8qp&d2}GXhUPt_)o{(3JWU@+o;Xu{=M~OO7`g;nr2w@swni>((@(kc@K{%<@! zH^QSM@K(dHjRcJ4&;}D!-srJ!HZ-5?d>!!ZdoyOY3VwLEOBP+o7no%j-M_gg3Na4F zy_hR&$7t_RvWjyqk#XM5yUiH{q^cN8ux+g0G?5fK-Xe3n74LdA&%L+{;rT`D9rmxu zecWliq0X0Wft@wp#e7p+zi_9o;MS z8MH^?+J+T4FNHH1ntOenv&o!4soaIMt|FQip?TxeUBEm#R>A$=q8%B74YgT7MF3j% zdV>SjKw5G|tto_cAM8}Nh&sM}>2z75%CU1UO}@i+tw7vC6l8otHaOc6%fcMlp9d?6 zJ%hhr$9#pV$N|`KCfxeQ5bW73=DM-EhlCU%lDB6J(#zo z&!u$ux2w)I)9A`4I$;TI;I(%@%?WTWZ^(5%SV*XVLu%(*+SRif}u}%e8{(pzrMfxsr+AGK$L#y zkBTBfOf~2J!R(5YJNaiP0;23PD>P?L&XagD-8k>AqS^upsq1% z^`t5(nnlBt+dhTB21U2Bd&haq0x$4oc|Jo6(o{*sxipf zrUb$g`~cNq0BtR?70nyyMLW@mK#6yo7s-1yE3bO>ax5*>%Ik0!~m-060A+and@MWK;y~GZQk8X zvu!9)6ljyv>rB{s5QVA7EQ(QOJq^OOy@A!rAH126-zh{3=2aC9}Q5|r%poY z3rz|$dFp0t!&-GOp$IP|jpWewJd&a8K1+A1YM^BMkX? zIBjv(!+m{yCFx!(&$&{EH;F)*PJdPQcl^ zs9r=psO7KxSDEI0x&IGIY6@7NIc4mjYj(-dEUO9(8?gl?MOrAWWIVHoZahL`^8MDpt9 zT>Ofx4{GAP-Zxq|E&^^ou+(esJI3gLfDmE55vZ?HZZqcSOFz)EedkXkR%NC91)z3V zOFEpp^2Lu<1l4!Y4KG4!9sDt>S~%vrXs%#VJaQ}Gw>J=r+w6~&Ylv0Jp_kYKOW14m zO}}*9vq}|>U=xL)N;2UnwM_2G3)R89VWNuo05-Vgxeyx`I-BPQ`hck|Zpy3}G*0o@ z=?V)#Py}L0|I9zGcemkEn|`<28n0~DtHhfiTtvE#fCvP;0GDv| z-&4TQ9P!nN7?iIQcL;3;oH5LzOGe^Qjh~15Y~92@Ik#T=DCg0YIrmn@2rU465YJ&E zFdQiuS&R=X(AzsXAs&-0gT2LpIigfYOUgzn)QGfpUp;JcK2jELUzs;Kf@{TZ&ctj< zflRC{z3?q2lVw{VER*AJR(VGw5eY1x!{)hEQaLM09=n*?aGdyc3hhm25BW6-sy1h? zd$9+Lbtl6BVWv5h*fqeYf{Xr{4 z(05pm1d#?c$+M(yk5H>R8Ya(Ogi~Y@U7uxM8Uc{rJHcHazfU1`daWW^uePYhLFniW znR<-op3I^Q0cXUeiebT$<+`d(%s%2A+>s-hQb^)k1<1v=5?vKfP0sTEKyR?K}F zXZZ2Gwz5n|377`xQ3=M-d6OgLRJ5skr`QOuMpZmzQ91E&$5&8tp4a4wSuU)>9u(A^ zBP4w@q!^0iB#(dO$j8GH$8*6M@7XDy>y0x>#CLM&Ml#SCJoR`<(XO$!X`fExFw~NjAVW{fAk^F&I%A=XZbEK?im;uhX_QD zTmmu+7;_~ks zGwvL0&_gI<586b+%8~)ttd$FJdgp=ky{2cs?iot=eenhNB*S)C`eh1he9Sfy_anQ% zeQZTDouLO7XWls^I+%0kp)vkrwkKUEgV*I2Iwe`e3m#^R{OyhP1hk}FqRKZa*osjv zF}6BzU9mAlDCdtB8f8TpLv#qsk=2C-HAVR~ zAgSOJxlGXV923x0Ll=cA{P8_sn>~{u1kDPyQpjggGZNf}FHZXWQQK4efY|WPR@?v6 zTtoO!S?v%juT(qQGE4&xZ0QOK0W~G3bstK*eUVQ{k>@O5Ae=ksXQvEVk3Tj#K5d1b zL_x+en2oba#`ZhZ*ZtmC4m&$hIyAhkv*0+9{`r=2zBOokm_hEpB=o z2sxYW6cmOn0nrSmtO+5*;LSQ60sF|>Pzb+Dn*$HvsGXZUD?g|o!OS7}V9g)~59FibULmW5EE zrs8ZrmWR&wmpALC+M&5kdIQQu?${2G@#t+%n#=3c@8Roz-!ISCE|FmnKYkp4-!K2+ zTmApD$^Kt|_kX|DYc(!4(af>F;OXFMf-%L=*u+^!z_-M%Ny4aYOrXf15YEJ*tbCyzl{}dIoZcm6I2vdyZ3%P)CX!o`!g47MIqWM}n9fX~$b4gpO#U;{Cg1Olh*GSyg;litd*;C+$q!2=0UiN7jlw_tJ=A?38!D22?#|z>bjGfUSTQ$@AWX;3KVM}$$0EFNK}Hr z@@6fN+e{PK2-3RAJZApBv_U`Rw1MJ7L#~qNYwxqwmOP)q#TGuD0Y-$0{gyz$`c!^c zW;_xzBcK%>rHGz_kJFBNXqFkY0{%O3KG%mU{xvJ9tY=7#Xf7GvOWMWByd4!zF$MF*=cvz-CWZd%;p4uzYDl&Npt31+t`!lh#({l!jpTFq8{R44v} zj(N^^*dTA0Zykpv=~{pNUQ>yama4jP(F1YQnl=wz*;#ZTZzMfE>Q*cNtF&kw;4{!A z(?|0b%ulAa%P{K!CWU*2@W2jUsLC22$EAzz=jhXR6d+A!Tn^^%Zx{!OGnS>@{gIaV z&hr3#@o&Ftp;_yQ&@&+w3|@U`|4Cz`Ou3r<5~vd*`ARI8=4?P6LGdg_{u(p>mwbR` zl5?Wx`XkXXNApfLJLh;pq0UD+3MqdBE+tS()O3+8Dj*G~iz`h9=4{X*4q0)woyKCY zj>K}%BryjXcVuA@)ei+Cmugn>mv+&rH;eXT6P-_H{AbD5ugO(kouY=H zug)Lx{=nBAyi5tI1t&!FRB}#an+!4Q5@!^f66+*_rI4FK6NqEbMW|gdX?ZIYSqJ56 ztv6L!Ga`jw@D}b^)yzTeOxRNukIw0qKYf8QkEPl^)#K|Lf}k@CnM(%rtCw9~>*ao?LuhHUM(h3e_ttcR9KqWgSqmh~f~6ZRx>O-1EvSvL(YzY3c@LG6o}akWSl3gUa|%t^cI>%6f3QM+jBUD`+Qmhn}ep0&+EmqwI%<%7;`3Pxcb#n{tdgUT;| z#ZBXgTKhkoy;H1cQIu_awr%@t+qP}nwr$(CZQHhO+t%54*Q?6St9z4|%1hSQ`dI5{ zW{x>WAH8+d8Ur&)cFh4&^lgQIwEw-ZL17|Gh6hZ-dLNM|kQ7Mx zC9^;kCAB1pYED!(^^G(%dj`dDh%f%JZqwQ#-ZHq0K3#Nrj!)i+HF@&Al7U- z_PXpjj=bf5&3g3Zot>{V?R^3C8-7T_8Q?kDc^ajV=jq_8!Z;H9|T!{Hoyx;p>HoWOmH82b02KR&1XZ|I?J!9sk ztIvKp3Jr}U*B2i~g5oo!@;CW_Agk^BAYS6R-26b1h8B(xW)nY7Z z2^2n#L;B*|EZMw>62>Q;hH>dss(D^6Ry`x@bQCeY`fO(Y$cvW)yTg&$TxKcXNrB{m}va3NYk=Un-P(__0^`lsj zMzPR1D{*o;r%silyFtsF_Hh#Bt(=6Csa=7N8dDb5WJSnuB}2%fZ^~(S%1J|RvvHTp ztYvh{&9{?#nf%T=`ek%_N^b&Di{8WAT~d}{dDn(ydW@&iJ9j-*jA#>V%4p&OD@yHp zn*}9P;1u$pLJK!ql#vp+Ms5+-zmsxav}&1qzHOP4t!`Apn(=jrO>s3bEUg~LgrcIS zG4W=qhs<3|MTp)bE;cNk-Z1s%JWtQ&A`4GuRba-_7G$g*zdJH8N_bO(m(ZXlW4T#ZE@gi; z2BU*sV0sfX|7|{_uCwHj1cex&a;fNKb9a&!;e~qY+9JDgc+#-eCz2;Hnp8vIrBx9TOp1g4~4TO1L}0#>0J?zKnvpvH3ox4BTriR$cATuc`=zj z9}BWrlB;Nq`-@1mT5evUS7og{y!)c6(COko5TQ;+)Ip+ng|Wp5?|$J#ve^^`;rw(7 zgw25Mj4-!^O6RMTuF#BJxtkZ%5z3#%U#YdG> z(&Cg4d*jnNICc}+3lPS_gy##fqH*N;GL45!fCXi% z?6`a_uh|G_wUceYiOzJkjKTZAA0X-+LkP^w`{XuQgH3CbdQABV9Fr=si++IRB_aVu z8_W`98EM0Y_SqWE?vp={fjMLsrCsAx=#9AqIhn4DCP&ZvKbks|Gb+kTb=h~n>ggPJ z^f;KJ?Rt|KjmuXh^#PIJFi+na1Ny8GSQXB^Q&Lw|)F=`L<7S5pt)3UK$aAa_w=Ts$ zw2dcw$%xqHDS3?Q8IcX09V3&MdNrBq%1H{H=W!H-07DEfZwG}~g z(2|QsTTCWHc@OxF72G?%U+HRYq-a{#AO$v#;W~@2OUL{#8slTx2?WLF+6ElSlR3KTZnV>WabmmjqvTJd|UQhTyp%d!rJ+@*no!}msk6Hp52>Ht2phX(qNtO*67J%Wy>?FA?BxRyP7>}Th;|RzWg3QN zxoBuI%cs=h3L5c}1+M$-wT*Er{X+xQS&i&29p+02a9$!Jbz`Z)<@&_$cp zTH#%ovka*;ZlmaSBKQnm`gw_6`I26Ov;hByL%n*Qi2!kgefek8T$@S+5hy64{ie_+ zIXKi`B){QBLn@C-X7=D{&lgRE<;mIXA+aaC&R6ONAu^>3Vp@0AGfKS*_DST%*OjX( z?N^D28wpBjZ7qSAg=i@V+eYfH;SH?9vLlgmac#e$>del=^C*nixYFl*`c~~@7dzr6 z_D1;47Vq+*=@`EwS}|suiDMIIUrY*n_2cN1Z#R#5kN)n2mRYIS4f{h#W_sCCPG?!Qyly$w{EmV@0+YlpQXL z;3YH;rJQ#4c%cWm_=HyChj3eSv!n)6*6ISK${PdGRZ z$S=BAPWj?))$R6VF`8}ZZ5M?!CHigYW1XcPM*nz8vV5wl)0fP!bb`XYzt&nx-z#5|^> z3dDWF#ZW29YdRzWAqrNvb(e5~#S9IzQq@Etu4SF&lFDEG+~-%!!%uRquHy=M#)vN% z($9#Ul6CS1GIX|Ch_`yvDX!c3-PHSAQdE%1W;K!_H)GO4^fv}GeeIQT_FNVVP`L}O z6vQ6&1=K}6dbo8gr#CuQg)y{aY($Q$)agPu4Ceak*Joqg;WovDCJPQF+h}^6E_^tO z6m>|ecioY?C~c?!F|LOkJhi7%oV#4peT!bu4_V#Epgfe<k=N;&V& z0*6~eG!CvvyS=|21ZMI?1d%`Ak58SOd;V{e8k>U86ir&k5Szi9F8oRS<|kpDg-m~V(&2#&10fGddJ z24dq4O~X2@VhzzpkDObx18CRfKeGjNi^}1+LmS+Xz;H%=-Ef4j z53!x1&%l9&Io-fxhT8cD%CI_t;XpGt%uhpsMf)LO4+M*b_+I^)qQ&dG!`iUdDq>@g z0oj)*d;q3t0bmy7jgezp(^x&i6mS$xMqp|RsX#Kb$Jeq}3bl;NK;y8`#&L#mhH0LK z7nK&=KCtWNzizr=@Dk}pU<(N*ofFyR(IWl*zTMH)Y#G*tS|LKExkjsXIVat!i(>Sz ziF0eYNs)Tm7Z-**x5TDodBZ$&_zj-rkVRzT+Ub__%mJ?-e=E4ujx}bFt|!Rzj)Nzp zcc9x1DYtL44eoZ2zbi8IjwdHhf{XmwO#=16Tz&XegNyPKqMYjp8hYXB5Y@m5sW3!S zoLY=3y2y?5-GiIgm4Q`M3g+l=uq+irO~~d(D0)hS(!B$#nE_3D%awJIBBFI{S2X+w z{$G|B35L$CLFRQZhr6b1&8=`8cFn*jguE~hE%7yHdF1pPJCC@ySJJVF7v$?Ar7PkG zfQ7JN>C#%4TXL>RpzbgKZfdDf%NooY#9cdB_K@-E>U|+<_CQ}^o$B$PJ%-CIh%Z{p z@y#pAi=z{F^424(Q>o1t$zPx0c;PN6WY*FRv0xv{6n^e*_?2FJQ1Q!GrM__gT>pt6 zZFd}Jh9Na)zam6;@$M2c=NO$Wy4?lWJtI*6B=+NsgbrCkf_@R#Ck-dewu~q^xCP9-sIl+mFKzjBv-hH?2Zzt@%YtYF`=YuhQ*Q~EVbw#<>6I$|5s??Y zSZBf`m_~@*c}U`!9!qCc>bh@wV!~DkXJ*1md3<&P4VH7wad*NnQDjvg6Y*?kZN1r#gumZr~4>r(}1cO?kV z(!8F}a}v0FZiP0ThWD?)s~g+3;cKOumr19a2QoARcjy=7)KnPi;pJji{S_KU)Dz0! zPUUwNs9{-+2+GN!Oij_O?fNluhkGd0#nR|gk!GhU<0Qdi zbyme6>`eliP3P?P>34P%R2klb{A48yq=ViWnHm(rlr-B@Khx_jhEM2+BmjqU}Gx_8CFfqYG7>xU1u2E%A5VYmo8v+18*lM z0Bq{o0WIOuxcH3$9^YI!e`wgRQbHz1?bAC8AHY#@T;N#S@I(~_j}6mq6%x z(i?WwRQ=Bl$lng+phQtgP6ssVF`n%|J}JYb#>_Dj%5FLVR!6T~S}SC& zP}TzyG3epN;SQ6)s$4$Mx@?MSh_2{4z7MIZ&Yj6 zp>FAh?I%bhjg{VFhZU7vo5TW~OX(x+xuWPQL~ei4raoqGSg=YsK5*H>us23Mt!duN zpQ63M|GnEpm2OEm|0|>5{$jWPhn$@M=r;Xdou+10Ygepc6d#h5?u4$Ea(eNp$_0*a zDlAGi|E@7+7#9KH7^FIToxDB^<`h=->xktE{GzU=<8ELL$fTScA8T3J-Z{sW$3Dc4@>{sH;fhN1!Zg%4ILwVs;sAl*sj^&YcomYOsPtx&#+kUBXBFG7``t8YLCzVIsoh+ z74-CG9FDw-(-LBTv)Uo8ha2ve4Vc}b0k@3UrEIVmzr)y!U zsngQTweDe^950HHx1nOHFl*`e=xOLl(T&N5tnegnvH0Dqj|)poboJdWqKvjiIE+(h z0^4obYwdyy+L|U#jNsaGTtyi}+lDfm^n~`!yCh&C zU=QC$%H*=5zO82iqw3y1qvFtF%)n`iBkIA}B2Pm}_^XF2M6NU#gMOxGVvtUTkjJV% zq&P{jD{xa5?qfb4hbyn@7|(%A*&a}Vn$9#(|6GV!`-+Uh-4nv_F5Xk3VwDfO*&vTgJ;O%L1w8PpB zv^XF^z5+)M1r4|Df3Mi%;90zhjn>{{`9G0N*7PYj?$?~-$}T|~7!2qY(OXaFntA*#74Eai_O?~xFv@IqfrES<6TFGrarQ;06&d@+ z&3TEUUH!esC1n+DOs8chOG=$uzWiP%n@NSa=E)|Rv=8MbUD;B%R|qDE$72IkcZIZV zZ9Xomv&M3hBiXt=bM5T`IT%Z)=-ys6leeXP&X(6oUIq0Ly;sV1xEwUzG_Cd-fngWzb4}l`Bd#EVAsv%jEL+ zamj?aZi%!E#Q;LU-oNTzyXVE7tKKgQF>Kz24hkvXA)y5KFv~DB4(655Y|u5i(q&7K zh~LG$DRbdPuY3u_|D;8bTz-4aR(Tpl`Aj0{4V95u9p9DR@RLL#uBmdkrckYgADCo| zuyk(4$rD7@7bZsnZN!uY4MHJ?;e2~bk6_OCWrF_@M+X^QhvEuJpR4qRc>mRRcgnZc z1!fy_uW*pya>;ZSZue;aOj8P`)nIuqx9RNyzSjE2dBF zJ{+XLU?C}T6U?Rzk0Ol}Zq^L+uPfplTcs}Fh^^*W!^OUg&7ATqQ9xvTOeQ^@{1T8> zeUKT_g@@5F37e5MlbO-9jBu#D*L;91vow0)+r*|ysZI;T6O#&^eUs+BB0b;0d?ndn zDXMcss2{s#?hhAjvGZI46+@dldW9O)=$x#c(Rs})SLZ6C`L_q zl3PM5q1zOt49I212z5|p9y>idoi5uRAsLv`5_twQhB)V2dO5pc0`jF6gb<@V_X29% zIlqJC&uVn4n5*NC(<~EWj`&MWqC>VwOuTdMZ4L<#C1a0&#He}u#Jm5pR_)k_hz3S2 zZMKRHbmnjuC$TzrF(1CFW@u?Nl*mW}qhR5y&Qu0*d;_?_6mvstqd_r@OGRp2o~Bsb zPWAZCV~#^V*$JazjUme8QWSehJXX_`K#V~4c!qa;!jz%_Hf5w07^FzLg67557`6N> z)p*0@l!^AHd|jo?06i&`$;vb|6$Ew-F#+)f5`^YJX~2sgU)$!~st#;H1<->`(@^{foI<@}P>$?5VgI z3=<+^jw4lSyp3g%t2*BeZMNurKT92A~6Z;!bu93Cs#};ZbpD`#(&NwfnOiI-^*aJ4^RyUB!R9W6bjoYuz#r8^c|2erIuEZqMJac5Ll9 zIOwGfoWF2>vZm|RE8`f~RWeiN(wwf+X>fjqz&Ln@lmYHh zV!EASadGhW4RIdZN)C~TJu^!!CvB-WwciT@y9sWrE$#ZP|Z za%=ORRR%gZKEBT)SRT<6WZ&!{y`dOk({Hv>ijmqiM$6$&~ zpfpHtHZ{2_09I(*!=6qzP9!YT;}(a10X34l?Wg4Om?NUcG3G-rTC92#@592F5|h^t!R_)?Taj(zXc=?5ksIYzG0xP{3Y2Ixl&CTqW& zU3oErSH}e5@@QpN{mjGv({t2I_Xaa32doU4+D&T6s(b1=H;{TM6J%eB-u{I@M?li7 z#a?wh;ZD8E`9=s3CWru(y=r@Z{zo;~lCAmAaR%}yOD%oPUUr1w*Pjm3pxZr~ngPCE z+dMw;WXn*xIEh_8NsC>WAO&yMJeqYRfPm59h(ojCzbps98|>e~*GZ+fo2ae?0Kz-r z$Qno4Vgy6mgKP($GfM(L$kdO_stE1unCLIA_Dp>2OI2Tbc@M94t+igIwb&>*l;EtF zKY22|8jHQ9CAzm-)+#e#9H-)%KwL!l?ZQI;FbE)PV0{D3G%Y_EhXl;JDVR0!ed2

    + zb^G6qZ@8pYXXnlx{byQ#|HCfH&Y^l@CW@4@5V>Z@(QJ!=)jL3vlIezft~qq_&yHXU zy+vtm1y5XP333Ao!Ny6c?wQi*r4l09-^P-kBELakSJc(_4a)4ahwoqu(8hXGr=RuX zO}9Tr$PIw=hnng&R585nsaX=0N!L4k%zH>8`5fo?)^bmn;Te(T(Aeh_Qqdj7^Byzu z4RZ6y+2Iw`d8c3{lWqDn-MmYhWv5@!O;z9?9AI-s0q>+$-@?T(=oxE&5q7{)q-Ew< z!&7VI)t>hd{E%RTPR6ap021zwM#vV3Z8AmP+!tulElUCJUC$GDW|@mi3J&lp+w2`r zP?0%n^)4(#^c_ri(D0{UCc5}baH8y~^gg#16&J@XwleH8&dgufDO)Bne%o9g1I03Vshrhu2{25&99c!!A3pdLTwOZ>j+JJnysy zbP(St-cU^5B@s*zt)+j6NWU$+KjLV)W3SMv>SFFP1!a_cHHJHK%@b$dvSBUzmLG(K=_?wAsVs7gE%eDEhp+VJm zSFB}}e>qgJ3ZaTe=nFBh52RsLcsE4@HfY& zUf@t)U9s@b@ghx!^WeCHhh3fmgNgWD#rx;rQe1-s zG9q=^sbo1Y`|aR4qfnSS$YyEHr64WQ8}CvBV;JQa>1p%}E!mP&80}qBXo?K<2C{JA zRgIA{HD}P9>z&h5{Cb0DRrUj7$T*7(mU~w0WeB>gjkbQ3Cv^YsnVVClDG6A%1@uGxQBERlYPBzPg_hJHT>I$dhe}b9EEnQR^)>-= zq+#WJ6IF?%bwFSfv(nAZTjgU!Q>HE*ZRM%rq9-t;Cryh=40h*D%jz6TOw~;iGS)3w z)qtGbQKo~4l;c)*&}S22sr0-x&2pv7cMTz`WK$v(#8AOAJv}@RmN_Y7LE2NLNBoJ6 zmCZFX9_slq09T9VfY=CYT^6X!Ifo41V`Nit{v00hbMt}Pm^vuvAG?tmC_2P-&xo)W zY{9E(ug)-~&Rzbj&RxN+&Rv6(on2j_uOKb80;x?@>GW0K?^5(!D7P1eWOnM! z`8cl`Gg6_D-k~wOhd}<(IE+qX8#1F!N$0Q@d<@lxf?h)<-c^T4UKRUr96W<~n zEnmEH*gcYoiKek`un1y;CrU!V4ROyvU~*tjmxj7}xN81sFAv7{BnHtmc#ayXAD1L} zM^uP*4VbplX)pI9Uf_dSh#)oUP+!m()Y%R#|A18tsoe3wnx3!XSi(H{=*npfNr7t>ZIDhhq9Fg0iJmx~Fh8Bo;wK&|Fj@I;m zIc;|KV}2yRPMI>v!U#4`h%q-8|5srdOPq-IyEy=cUQa~&3beiZSwHpS%ly!JJq^n| z3SOI~@#V>0yAp~~!T=RKAswrHobDH%mfQ%xb)?SIHG}`K19u#r!gE#DXh~Zl`|x20 z>uyEB9H46qA^V6ttvnciVS55J8x{l8%eF^E>_GI(4K7DH@Lats&&6y5sZ!J#Y8K*e}1KMWDiO2pj1J*Z@1Q_9s4rVcS4GBiGl~ z)Ki7H+yu1#hsaXL3nOujiu#v|DV)72lF9`fOPe5+$3B+9u&%1_*^;$p)Q2I!Rq!ZY z_S#0S=lx6sM#%PIjeF2qr@jYcFmqk!VbM_ zuY)CoG>=#0o!YGHT7A?>xv>zr3RWRCu~oz(?S_OTL?)X3IxDxSmh5D_C;a8GU|2+G zy(m`!7nSDft;d?~y&bm;1)di=sRj6f?pon~>eGn{Hn}EA2qgfUa=m^NVi0|c6NGg@ z$pNbWU}wr5lK)~_-t=)mmF?~&ZR5YD>E57fDaghAAXpyT!^f9yY=IY3FluDso;?TG zoKckAug^v{_O8TV^yFhr(As>C8Jaa~>YTSq{zT8K0s()PkQkR_yiZJzpH5$x8$#j% zu~ek0JaQ==t3uYc5grOpq*b~8rp4ZG&47KpKmWbhZX}dQDgP!@v!ng_Bl|zQAphUR z_W$+ff7`$Qv*tFdzWXALqWsL9ttEGjfT6?Fhu`yC8xqoupkp|$`Rm=q55@Fj2RAp3 zSx#yQYsjA^H%V_6nJy5LpJ$giZXYPEr`||%pcrBFNWF|N1j1P>Fc=6CR zdOV&?fA+lg*pB>s?QXULIPKx_H0iCxstggM`?SHMB?RHg5|D=^0WZQ~*khQ}>!J>d zPS}B{4nK$v-_i5(#K?-h72zqnRY1!gDjK{wVekZ+PTZ+`$w23(-c?1N4o#p19iR>0 z5!*kd8hZ&(>7ooiGU&iRt{Qo1PwmFtOVjs~9TL;~N)9kX&@w7vnlr|;d`7RUZ!?bCE|%az!Ip>Pg6@{#Mpv3}l+!Q0l#^Q%`6~7zuqt!c5Mw0yvrT`~Gbb;M ztsHx4f-&$KT5w@vGbeg*Z7`Y<74kFZhm)LB&g;+eCQk;JJJU@iOeou-x; zO>r3$8Y|Y@5|FsT#*J&)i%T(G>7A<2|LE`g;jT^xLVG-JNF7}(5QfCtmBp5qrxPB% zSPdY@SgQU~O6);hdlBY^Ai>A%JX{ncW6Bfh6O`aE^_f{}NSyCEDV3JeYDyYDF6~hgjY6_3FX5fO)znv__xi%NN6676 z+wuMm8!=?lHHQi2d$@ll-Mx1DQT-(gtW>YttPL5|pjck3kHjVFF4jd8rk%;E=N?!K zIYFJ&pl~U7*yP^Y3dLHJgF{aNO9ceuYqzQ>~E#vtyBx@t9aEbmY*J`%x&XC#puXZ$4k#z8F133No=A*lUJTKQx#=ykWZ$Ro@5L6c#vsy zGUA@p=s4xP7UI;`5vqF`gBuOTHDqc`YE_bF4{@`>40UaWs&w_ zFhD9fq}hy2@l+K5hTJ^6T~_%pOexgh0^zG3{AjLhP~gHl(b^ATjWO&WGO*a;0dB>@>V)Xm=w+oECaCUFjOQFIuZez~?hUeG&P9I*StDk` z3&4$Y+=0X9p&?qF*Kmgk+9^8mL=Cb5!eO3M+y?T=uFc05`aL^GRmFXVE!+qm&D$r~ zfan&%Bm@CKkP>Aj(TV+_1MLiBK*vXNjuDHPzJ^*G5U-agGQfok%Xh)mJViZCJ!lcu+sBfh zWzA2u9r+8V7hv1Z{?8AkRK~#s{@4w>5INinB?e(96}Ww!V1SgM|9l^(hBInMjJ_$DcPlKzXWYB z)r#*T)r85^a%QYc8+T+&DR(H#QA@>+b>PENPt-F&?4Of*zh&*fqnyCaWzN<&nd?gs z_{@HHs0=sCmj@j5ea?F}I1PBXg%6ztM0jp9f(`I68zp&ps4$xSd>DHwsQAQx?gRj> zeKsMd-b(ZWpFA((ZGZO*F6lf|UpV3`+o>4ws>PZNw(uvBB-LqBCgMgVOvbS8fV)s1 zNPfG$lw=`M3g24?m6GneGF_mzMP0YUHao#KJ#e9Jte|f`LT=!UUw;m|Rw3u?9h_O8 z1EgS_6lamV`vt3P3Tf(d2UKGLI%#Ouob}iE8L^$~VQZwczraj)3As)E242MiLl(^E z)Pg`o$*{eJ0(}bl2ZaI)g#ry4YzfGFChjz66#`ND1X3AUG`3WGnlw6%Xxu^}q7?_kCb@+*(Aq6eihz3gD>( zM&>3iT&G#<7@yz$o34LW?3nprCSlyzY^Oo+QoXcHz?;RAaJl9ez^CD#lW<44PRGTD z2zlfvVn?}F4dh4&b~4Jlr}s!H7!QK=+uk+TSZH&08AYNT7qkC@YxNQ`n%^*RU4C_+ z$Q!G;pm)_^f`^#Mniv#fzXtD|d?ztf=SaA93^q5NnlCUCs?$;Jo(88Sw?t*3n~CfP zMtQLw3*(nP&F7aK;id;+ZvtJ*iSU- zcdhTDKxmI$uXXQypbEZ3w$C}m2s_Es{^S!?Si^xKi5l{|Rj6t#^=MJ2gOG)kYR0`MbcYVp| zw2HqJ6F`%KC<=rys$~GV_yXfi0dTTut~;`c^W?eW(ikB0tN! zwxIa0^?OBA@Vo$Z7-q&QOB@M4Er37!X^4%ZV(U4JeWv5g`ZPUjc8UQjYpEgL&1Cy@ zdFH3fZf54D6YTL1=PR)_B*fCsTifl9<1F`U&aIa(k$AY zL3XN}J2@?q0kh7-okKS5(}L3aEkO;lTd>z9v<~^M@`4uCVkJxsYg_-GNZ=~^Q&7L9 zu}yaW<#A{K9!WMWy<1`bG-#_^d4#6(&QbBOfIDNa_$jr{2UDwidv>ctaG(^bYYZ4K1_7oSr1;HpeUa)@5-t_LD=H zE7~0{ESepjXi@k@SF2x(otCL)4@6)h=O=EF*YsX1)Lk7kY~)=H(zy<;(~kZfCyY<3lkMF5whm;VE%XlIbTmWP_$2G2 zCyY<-%x!9OWS7w+^}<%sW3|o(E!4{+SkZgl4Bo>6KA1!1)4;o$0)98)LM6kSOMLP+t4LbO-wOzzXdG(o@T(I#!5EXJ2P z`$0FT?}()6TUmeZ@;gr;@9FXE#WMA)SXWsX9gXr@~4>=K7vk~W_P}e zvyN5{2Uf_VViWORuE;dlYU(d*GyBG3&|(na{#-vF0djOzcg19cc<8uQgiw{6A)DSN z>WdY2v*};-=#dIf!OyCT8Z%v8CIuJ;O{Dxf?Tkzk8Rl7bDKd?2W&={gdNVG4kX4b6 z0{$#YqXTb$_WB| zlIam8W0?|r#Ig422J1NsNnpej_^Sw!j^OdEf(R#7TfSNZoV`nVXJo6R+2I=!FrcxV zG&@|hkXe$9wz=yIs0e=gXCI~0T7o6~xw5!m9Fh$c0iil}T##xZv8&62 zhkEg8b}hsb748Nt^(IeiA_QK#Dx2T{$7G5+GbW`8CUV0Da{XbT@G`L}93bi}0|%D} z2KoscxS<7_c7jbCXAFX^Cd*%Qv6?-J(K2KaS{+Ll`u(7NPTxvNEWa2_FicrJvvXPBPrsnv zvWS=yeaCwT-i>X2DOf%{|5k+ca^hZNfr>*jzps8#Ci)KcGQWfT!y$8`v?QO=vicNM ztJu5AabtpT)T-$DcVo7dd)EBbue}U04&%dV;vC!|s3MobWTURR+p}UD$?;JI9>9Mc z*1chCS3ll+%1aj=%T9jLC*fv{c=}U#$4zRk=`%|91z8jJn<0osC(T=ljsWl3?^}xk z>YJDx{ATPo=%<#;U|%20o9{HvZ{|&Oj!abcN9rHrPq6t-ngvH9lskb1XyVU=FXOxL zKTbt$P9sGXALjqyogF}J>#_!rDT%Y6G}Vi(jkEs%tS{)}cwsajV^vtm*1{7S`Ac~71GHkn$ii}NOPBe?mwdE*gCFRkJ zBbgh()pJEZ5kX$2s9*+gwEY?lsUT^~BpDg95PyUs&&tZB7b2vs8(4;yW{})=j9&no z%3!6dg(!_W{<@|~DkILfMaw0uW#Oi8C|PHeXV*%zHe+l^X+LIeP+4P^fp%WkBWf*L z$HEV_n=(%GxFY1S02?-F%BMivS~xZ-C}v8E;ukpsj8UKM;z>MxR5#t&uefVq94S^> z9+$A_R_bXy{E)@P?QO{8y(207UH;Jb#5}TivxYJ!GRwvN%s!%tQi+k_OXukT*y)vN z_Yt`Il0ZR%d#|MWx$5W^r}WWVw))BZR^CRBC?5d~*Ws&qxxG_;{!_vQ>z54}29QUC zVBVLp?u@WStx{d32q4*4qEbxF)NDaqJxSY0IpI1)`Kqd;tls=MC#*UP_NPIMNMwLt zv6D{|I@4(mRhfF%-7GvXfN~3q9EHy+zJ{8hRx`*luc*a??fRNcDcWx15UY-vd_$@e!I5 zmwJ`RO;)mO9#>ce`J%)(Glj}#j}l4#3uuCF0IEc@FrqFN;&~s@nriw8HC4PEm$?mT-^YjJUfmdI*v2|=YZpCYDZ&r zESr|YYU0xH3YJuzl(-Vc)NnTA7pP2~UH8~VM=*ckp-U>4KODD0W}PYMQMVC7sc$xz zOlZ6YhFQinti?Ra^%A9NOa&a?@-(@h=$&&OLvpXpU9I4J4);M}QJkIJT`i_C=N|Br zn&Pzk)?wMpO~dSnMh#Ke$gq$;1Js$kVy0UABC1T14~=J#!iJX`C6sTJ*L6GVT%oW< zxl7T*%|5;z6m1`u{}%S2It&gkfE+xXeZ0@g7P)g zgkeU_P?xU`$aChUfgQ5_h(M{b7m5<3D->?V1QR=%F7|y@-EKKHtVx23P<;Ep=WPQ> z;x#%1XV>hI&9K`_c_7#aU4inw9MMt9QDKgol#hzg69`fulRkHL4#?kUcj3(%}+|dZHwnWRm@V76s2H~z&b&4 z6tM&;B+eQEA&zf5ucvjeq6ZEoW8!mnEFa0?&F&nLrc(ApQs3b&h? z#?g@O9@K$@M(GpKR8vJ5X_6{E=lD67~y)FN$f%tKM#?VoLi>YZ`p z>~lksGHTV7J6xuL>vHF;icQ#qhZB7xcq!kf@qw+e$e%ZxICZ)R^aNsBZwYg*uJs`?1u&JjActuB;fScFGd=-v;M^aJmX1})Ej3S`vbbr>`M7HTE{DVGK+bwye@ z%q%7h1!|@sc?_haC6oweHt{#RRWDA0BgD+&ls;ocgIVm<^{oVMAqsYPivPhCStF3i z($B<|jnQ#QsyyTTH#1mudPo>$RK5ay?(klF85f^>SutHUk?q0AAjz(VUmyc4k*&MTy? zdWZf)ssX{Y@w6lT#ud8dOm2UYaT5+dU}?zW#W*Ie zEx}KJguSH|;d{*JXL z?Qt)0W`AAg^Gf;j5d4VEFE8KXR_mB^#rC%hqJAZ?`LZ^Jeq0UMS>YX8fRvKt<7R^zM& z{hs6vNfC~O94=Exdb^P^uHej*DLJqJrF=57IBm>dja5xgU^BE%C5CDwowcspT6+rt z7jGz-?-4F+*WeVA;Y{E(cO%MG>we9vF5LCQ4`$hep-A7LU(n@}UVtd>cdp%1`WG}~ zZm4fwu-s>i`Jnrx5uJ@8ehp(N-M%UAuU7m<7{l%0JA#-ZtZ3J?puG;&vf|Y675GOI z&_t{c{$o#)39!FE5h9tP58sHXMdazfL60?@cEkP%CceWDUf$uA1OY%C!Bo^I0L+>J8 zL55bvbMCC4{Sai!HJ(GpF*qQ|55^z4jVmPP2FvD<8Khl+Ji8!mR_%32z)OcgUMem# z7WJ{?XP#qA{9*H<@K+gWZ!_Qm_phs8O<<(WQpKg+f&S=^9{iFp%?03cJDHtwqGlbg zlogv}?`T7Ur-VF8R0U4ax(m{8V~Trg~VK@p$7n!_0s3NKZFH{_YY&tftE05SI{u5SvavjIcK`lV`&N$`sP z9JXZAbxT6+$85JVmzpBiE~Inkk4o%kMzdvO>oOGWKboV@rH*FGJbC|N+>6qmNLf-e z#qEXY^C$rEeF+RAeF=^pxQ3UwMp<5w=8g~_Xpetz5K{`k(ePFjt<*898 zVUJ5M`oQ;yK#A534Z;pXIPp6nV(!Vu)hAa$1jMt={(Mn+V`P5k7_dmL+L$5W+z_Hy z1DW_K52Fc5!F?lWmw=c)K&*1BM z^p2uJG4!7ncj6iggG%r$Dm}jxDx=<^uymNFvK1|;MdGIlwAE zGaT}abnL81Fjp->>1~H8D2e_R;>_eV;FQSk(>*f{e$8X;9QJznOCj5AV8`MQqhubh zuu8VyJnkvdt0_I`OAJYU|5_duO!Z$?Z~x=JLq>hy`HG&z*2UKAbKt7u(G{Riizxci z__`uKUR-1y7%PV`ovmfIr zD_Es>ffsoqRC40wcav)`zf+#Qx7n{Rztf{r6$k!Ltl#K~2!)7uA17cIYCn7^%1s z_~xw=REkMg?FoNa8dI$?x&3B;r=SjRUfBjLk7M0|uy7-B;E`IrNE}MhY}%E^G@Fdz zE{QuE%ub4BHI@QBmvroWFyW!h8%ue*^I&djsg%ur+47*`*SlveY$YwkB+O(isJBKf zUvqqSR!KST(>?M-Ug)guFRqSB@X-`>R7jhx#-MM>=UC9!TP?L_4%^e5sX6M`ZhsXB z;vKm=90@X{HPuI1Qg4JDv&B#%AlBKY%OV#Mo*`!UfOw8e7TxbQi za>#k4c9xcXi}q~U79)!@(38g?t4)O+iFFucPE}k5vjn4k=rLG(k5vlwuK_$#f6~{p zN#>ljxn`_zkV2|m$!~1yo2Y`VU3qR|tkr+j=#gTj*adNo?C+#uV^w>(mUD>H$-a> z6XFo$XR-^%kN!xyFWhFsM2e9+*B2x&kRhq!sOTk)pw+OvWZso;es7OhY;W12%t+bv zb)C1=PNs}N$StDDm0BrJ92HNvzSnGBDB#NT%hc^X`9N;z>=uLC)NjpN9G*bP5Sy%~ z$0*m8Dwl5jl8w|#0@8PrDSRlK=cP1JuBOXxeb*~%*-G?WJY_;->b9gzZW+XCA+w=j z{sZ?99rnX^MJLtvjSmx|-U+DZ961rnSXTOk4dddZ_#9f1M=(hw;>1Pb%s<6vtIK+r zRnoMWb<(()HR>%Bk@b_JU7@GG+r<<0FzC^Tik*m93D4;{t^sr{ghgI}!uE;f7)iO1-3nlWY z%qJJh3`%;b6Zdy>-rMW7-C~EITxm+3)$vteE@RAgez_GIWI&yQV7OuFiYo zKNQlTcbtz8-I?>GPf4CGYvoUDxps(S zoHc42XR@9(ife5yr2@7pWEYPDPngZOH?2na9@=9_w1yvy{YhgG{-C9$&gvrPFnz+7Od}tvsj0G7SjF zn0whM)|t@-bME)?JFU#^JV77APw)LmT+SXxtCBP2yix7W=w0*IWw$cj+VRz5_0)`W zG|J0#%LvNJ;IAa`{i;A`5J`>u|g^3wB$#a+BdyD z#O^40?@Av%?4-(77I%3pn&`qV!yTRJ@a`l`3Q_%@*`5&v$y{X)^u?aD?*-1a#TPRD z%M6?D9lDB+_$*JmnUI(0+`#Idv6(Gu3x#76xr@62xHR|eHF5*Azyy7v;?kOW&rK=w zA}t0JiJWRHai3RUkVf9z+Xd?wci!tUNspEH#CSUN4VmxN8w0zAK_nT$%;YtJPufJX zpVM~`!j2HkEWX9~L#KDd?X}Ke+@u*BS%O`evis%;GjYi^I06Hc+74Zoj}CXHHhXhR zo2|p;*~JxcNi;W)kI(k*WRp!to5LK+99@&O^mVIr#s$C0+Kq`Wv7}Xi8&%bu-{HG+ zs&}idDhoYx5X?_~ebnlV&jDMS2JL?Cp?og3Xqmn^w%s&tU22qIU)=Xa`y$Vq93Fd< z)ci-+hv)8uX)ldvat`feUIyxW+>DqE^5itYO{UAu&uh}5=gx>yXe`D47@MmYUy%8o ztSyNX`t0RK-J3}z`sdqZT)G9B@H@&N_1YnKZDs86xfbTyYWH3BMX5>pl-Ff026{h_ z7fVej9iBw}d2J@6ZtPPj3}t!F2kI;(*Xi9ZTL`kwvCm_XZ*i=~l74rQN2)$?YK#j* z4OBf~x2u%Rljzu=H&fQ6=Hm>Rkmbk_-DH7lyt83*~NSIg#7H2 zg;+ep7dqU~`fHzlzn~VN{MT&M=Tr4Wwk%`F!sjIIUIFNQI0`^2qa;e#a%+qxmzOLO zw&_=W$+8bUH~!I3RX~~1$?U&5VG)Y?@)2XY`-p;roWKYxn4O7hg{wt6LNjv|Lp3s2 za@amE7AD;bga19B8jFzp4v#r)X?<$3b@+2-b+x_NTGLQg(`4c!BXRitjG-@DbeJP& zHPT${&5X+HUfpK*1j#-imjwP4id{rvYwK%!JnhME!}U=pzKdPo6vJ(uV5zaXIb$Ir zfH=L^TvF0d7RqG~8j-62@2AYfESj?_dTtH3y*F7&swsf`QfWOJ!bhfj&_3(N8_dOK zgzdcmDaa_+^QlF1=s?rjaC%iOrUf%~zeyS|QY||$^ZC9ro6;U?u!GuPvwltO4GAah z)L4hdFKf1TDHFydHqR7+&9p~)Yga>!_yOn-sJ4gNq|O_jBs;0Ap}`g2cT&X@yCn{p zRJZC?3)~3Y=>0hQ*SP`WK&y^iixtElQxU`@~ViIiqtFBTH`0}3gEsrb_tMwzn(9OVB%f<$td@NI60kBjx#u| z`6fH9OxCrpQv!c*w73b0j5GL0lM#;>FzM)sa{Z<#ES__odVoPp60`-rL=AS zLrJfEFa(9VT;OPl^#BH0FSSCC!3&Tv_SPgEEKz$1ciiBMpeBnWYJ7sCuj@31(x@D; zPE~`2iIvepFeuNpNgqvrZ_|oF=N(kuhBlkx9(iev*Gdz?#YCpJA4y%?OA?N`6CSQpe5qX{S1i2y_s221ueyKmT*v$uiTS*J4#sdt zRoaF$lBrNFrt3$Lt{t2EB6AMWw(uowh~ zST<;dDqZjv`hw<^vONr|7d#i|wQv*Zf}HO^;45f!Iz3Oa?77s~Mx^)3I!^Md=+he~48G}(!;wpPKo z#wqhL!7IXwmzsew){_d<*BMeOAK+L-bD`l8Y%C1{Seef7hX>>@vaTFPbsaU>oy^N937?sIE62gI7W%icNSu{fWhK=VAJH z*MF$K!Wr}=VyV0dFwtH&SgEcQv4s8zYY<|Lxs=kQt41&H5?#@~XdN97Wt>xD@j(B(R+wPBWc zBOpWM@V=wfj&v2vbk5D+%j*u@5EIMkl&Ansi_ZB&57n=mw`9 zdv}xQy>ijLW;`CKZ2%0SA`KF9``oFXL1Yhc`tuf;1vHd~-(6O7h0-s5<5}Ne)Z2#_ z%RO?}D30L>?FqLGM#686UMu(_;^LG2IS%5GXad4Cy*6zZ8DK5v^xCrsw8-gRZG4k+ ztx}Kam8SGiRvO22zD#tdbe>G;jZy$Vmw_|CO7)D{@|ith(yU%zsiv22BURGn$#)8Y zyYp1EtG5%_^rzRHk;i3<=j!U&!@6nOMe7e#9Dn^YTjttZKF$oE_AFqb-yFKpBB%i^ z2W+1u27kXm%FI68fP0|-wPTLO3KdZJUO>RY`SC;Iztb`Qk3IeWXB+Kp=KLQvzW;2U zM>M`P@K(^jx_3eY&0Z53;%&8U=F3e3Vt$1+r)JIFQw6nK zYH+qHm2~=8D^^#_o8>-*tA3qXwe&Uf{rx8FU!C@RQ#sT66KA-L5FM_!U#@%dJZ}76 zcHUmT_&;!e>%HkJgK$l07%1#cZ@v0#5(g$BmryQe_gp1(cHrF)&arc8Km!Rkr;qx@O9%82GX;2 zN4J$>YpmV$`B_#D&2w(#e~_op&#v6I(~Zby&dSYVGD!hRtm0SDQ=Hfw%Id6Fo4d)x zVt8x(cHXT?jGbINxpNt7tJI2}Yx3**S-<;?KJo?ghp-#KDt)d((52!Kb*_JN;Za!l z)uFN-2AA4Qz0k7k$8J&u0>I-7OD#ro(vf4g#QWIn$$5D;a~Tg+sCdj=J<@kUPpqbP z97<_PDgavgA3-(ZIIVl@GwRPi=B|H;Su#cp5?qz_F~a%+tt{8LW6aY7pW_ErfW`qTxkwIF4Eiecn{%g zv1(&O`SB}N&(7+T-%yR=qiQ1CY|CQl`i9Wxm~}2i_2Y;B20G{U$QhdK%6s!pvGGZm z7tUp(wOgoh%ydznT1B!|*QLu5V7M<=H5#JJFc8 zV35%^{GoOYe{(|Ij5OU$it>ygq=fl-S&{U7(sCDsDyhq2i=r51F>p|9=Zdmf-whG< z6zE{8*{@loWFC;H4W>@b0&d$!od&m2=;K{dt(WYo5ogwSx*4gtPtFGb1((wxMwOVZE;q$N}mu)H^Ybb{TaSWc!p+Yf+Vh zX~WPTHT>Z}wpo48_{?L)VU1s3qTtY>Ue_!F5fsf+Q*6_))hO<2JA}_|KXmmUTmKmT zRfl%0=j$LMV+2^V4cz|Wdp4dDV?Le+L(Kqz{kZS#!D!lEZhGKPbrXET15$iK6Y>3L z(8?}=z##*u2ZlP68=i1p|18^{H>gkT$1Li4{d6n^-*Y5DoYm?(jK_$V*6~Gqib7%S zi6V-Xdx}SK%uE-#{mVP9Vi`4QC>xL8Bu&eeWmdRdJ7P0jCD%qgEn*5*W4{)ZZ=$lX znFqzQe!MC-#nb3xoW5>edq|~ASyqJ#d-&gUk^ySt@cSEZaLXelE~O;#KKR$iOc6c& zweBGghg0_NG_^5>$ipyQ!?DHMj2fJoHO49o84amGJLi255Qn$Ms^Z-wNlkl+ZCT{} z{?zN-f!cy+35ZTazHOIkzx-20({h@{-oCafmrxxyFCMVg8YB=5W_$0x!|^(dK#4S( zl_^`FSi@GrDNc?gw^*7{Yo|ON^Hb&~K+x3l!)AF#RQd9u2df=8%7xkCr?;9P0(@S5 z;XREU$;4wz)+r|Kb=G)e9J)k72 z<0m}c(=@29geKL2SPnY>^onyW{U@~Vm#En*%x4pnFHD}P?=*JbqWTDhBO>dcQSR-u zvKIQP7Ofie%W=~@QRT|E*W{oS;ArWT==wTq^kRmXHX+ff-|}j0v;7Q|X^ZrQ{U6m- z>V{}2Ur|N}9p`JZ)2-s4N7WR}9bX@Btri3!?fo3FzI9FrnTBy{?%>C#$mu{IMWRL4 za2R&NKt)4@&i-%0ooY#cTDdh9bjoz$`%O7lI9=2y3L9NX!D2xEcRVB>b(1_h@}Wc`+iweBQnp4PlXn&ay<|#;o&CiHD3M&ED26 z5gh{Z9KScWpqJDYg_(*qA-Cj$1XrKW_odb6Q21BgRmr|Bs*Aw}_=C<=eL{1bcRjK( z^Q%yL$b-vr)CBK^Qj=!poS4duN+&6@9JdIzu44%R%sn+^T7ha2U`0H!=-fly=mjE zLX1B7T`%Xo{RaNAN57=-L%Ptq*oXUaL?EO!W(${Z3w6D)5;Au%j6qj$6AY`A&u1_(;e8J7)z)jtSA#{TvB%wKz zCirO$_(8YiZg?>R@*3pu+d?I$oqB$d<2zF+z#{ut`d?4i@ck^{J3vks%+jwK8XEQ}Vs)-WZT`re{Azgf+*=025M%&$9 zG5@^XOy^&{ph=^PI#Hr?Bbr@Svn`hz{j3y+-7I?0jQl9=bY^WyZ=J08XZJBQoE*3P zETfD$IYX>(m^iHb>^K^&i8zMbRvR;EtCr?1?VM!C;^Xl;OfB?S21S_WL`vW=iCb@K z^`f~sewpb}4eX+4p<%M@TGA+3pPH3^a;9?{8=L9r#ZZN_a)%&iMYo`Yn>%f?I*aKSygi{oR`Aerlier>%L8IYiJ^|8 zD~6Y1Iu1Jdgptj}-KTV6abK8d2Qm2tSaUnzuuRS&7ouz}zs^ocKM`Bxif0%9x)HBb z0GdiOdXnnBg4Zsi!MXK3P0#!GV(oe$6@(hXJ>5*;rhT;#4>ybVs{P^XMno$|?TMDj zAqHI$M71cjUithDK+|D~UG)_9DY48J$Qi)1pT!A78rr zE45J`A5pDU0ydj3lZ!8P64MfMCN52ec~IHpY@=eeSi5cdx*dc zW3ARY`vIv%MRFAqqY{gUpilH9xD`rm#gpiw{k2}n)~s(e>DSpKLjOWKgZoav1pbKO zk$%?0yO;~?!SSzXH``zz#Q9qF>)~YuO~`N0C?SG7{+JL{qUfX?nK=-lwmI28zOJ%F zO79HH`I#_>#MMND6GvNg{*X&%DM6~o7b|1=k3#!j*t)?24i}DHT&o2C=tl0dWjo_M znuB(bKl&D!f873_BUC4F&iDHI>K#@)t=>;C+QuNCu2xShtN$b7+1DG~?q_Br_F8Jg z;lpX+!DN8I`CBp+2s{(({mU6i?g0ymr#mPQ7s1J}lRNx#nQ@NA#`qvD5`^|crwo4|}0Tba@l7!|)H zKcey>@xO8ab+3B6szZ3B5SW7mP<)TkqjHk(I7b7(=BLUifNM1 zz(1@NUyf&b!XJXa2;M+*@(K=Rc^gCctk2d@g5iByQ z-IXLPB0}!U9(GA@eEo040W|fd`Nsc1`+VmliTrndp8szR|No>6{I5BA(%8)Qwv;eb z95IOkaacdp{RLQHJQxa;lw_f#pd)h!`Ok@IM+;%7npJIsmfk3hnrAT^Cem~=s5I8hjn*;;0%@Hk?voDBYXQ~-~y^fV_pLfZ7}-kw?(|4 zIWUm_^alji?@T_6f&!G4VjjF+ErhPHT1k{v50@SqD`k(_f(7LHz|Is1}Xxzbm zc15k-%JBGyeks1)yns76hYwr<6)H+iyv zhrYt95iz4;#ZRYbMXXo4NV4#PL#JLL3)dY@j~VB@&?Z^}qg^@A9I6}g=UQ<8hmoI1 z|CzH_5P?J6^~JL*1gMZk+fDp6_xiU5Fed)G@&YOpK8|C%^2PhpD6ehu<{X=|5Z8^? zKx+IxWH4lEOk-(t73*wSi^L;m?gqBbVxe)W&Pb%PBTI38|KT#$uLr+#;pF@bn0pZR zdu_=jcI-CNVH zRd?8?Dx-*h8~!u|oQ%OVB#|lcEZT%tOJa}Tw}kiUhHUi-a;96^9Mb2*dItaOE_@ph zq{K6bZ;dk&diFu^()BYkh^rdFgwcrVAy?PS^}Yn{=>h3$(Q>LimRh*5G?hwE9Ub-P`KedCS`U2i zVDgl3LuTFWHq4*f!%m@*A4yswWaRb|D7}_{Y8hX`yhgz4CH0n*8=C#?wEwk5mQan4i#yY7@sbc{-z?K>^j|&3{mm~15%f>4lPGJh(GYdGokd;XfAhB%nlK*o$#mv;1 zq2;GMd!i#rDfL^_Pm_b!Hc0c)5pDfAgR;ea7bb@3s$0+NpoFd5r(*-p!TQO+&tCP# zQV>PDBUz`ZX#riSc!$=k}F6CzJBa|a*VO?%;Xn*SNeTd}R3|8#?LFQh+QL+y_ zzxh>aTMZ$m;94-iwn_NB2tc-(InF?CcEvc?7&g$rn{*r@H5Qs-PEL!J6#5|ix3|7c zC&GZM_E>4v6udIMuSKYUbzMH&H(4tRb997jAml@HY%<18l&qM<#{!P>9N&nweE1hH zAJFrP@%mTg!Sr22hB>c$3XESqbn@D#ds!F&o# zjjCmea22|C(Y$um(MO1|eQ3Iznq`i#h7(BTymu`x-7Gwe-4U5B{3Eu$Uy}lD5Q`eH zSypo*hK)N-;ZK`)S(snD{*c>xl@hw0a=CU9P9<6OOuKLuyX&fHO&Qc4j=M>ma2DJ4 zZ+hWlu&3dAyNW_ORSC^(qmhwo4gw3?dCxQQXt|# zKb1R4vea<&@m)mEz20WL}ATE^4bw~-XFtfXf`Ie`MR>m2;LD5XxtzF~cMyth%i7{5o z;}EQt=UfVgSo=9w&!GGg2dWp!&cs9Q7tfYlSx9w?=H}^e{8H`7s^2B0B#X9&8QYfd zadxEgRLteAqhUt}Wu0&R`cn;-WwZ=f{g7*}Nqm z`DEiLAIrrJ3K?u3(Hr;;O1z0b+1#~a0SjyB?_uI`3moX&wF`%=5`5s1-4DvWBBz~^moksj${Q)VibHgF{xiUt#@J^4>UP4 zsxr#OR(7&Epj@UdR5H-mT?w)YZC9IB%lP!wMd@<2wiXwc8|!)-YFbMg zDvb=TT2@+S?4IOeVTxj7=~BYTbj+RY^{`2t7)mhMw^tn=iQ}k?cF8(1KV~6`QaP3yJ7o^;MB1} zCQpDn={ada19!g5KZHr(IHJC$d=X`1_*7EeTfx8^=I`UH1I3XlXI(A@9UYrDnF^3M z(*X2EQz68W*oy|=BsSlZB2I=5LR3(DU(70;5v!KD4n^+zwhO#V-)Z%hmVDE1fJcTv z!UuXoxGI6v9eg!+)5fkMg_n++7WkGG-7y8!+`K|&t(3saGVUkLNy)|41=8U0`{^Lb zMc^KNDfLZjYKcy(qGYv-O>z{B#1R+9Nhjr1eZ?=YZR3QUcQ}z33Zhn1Pw7au(Q=S4 zZ}AG6P1_9z6XEI)oBt&lSKCB478@lnlY`+|+EN~PiwXr&VUPNNYChLs_<3B#hsQds zHH*-bW}aU9ue1&Eg-jvMz_ZfZ~GkSoG_n$J?S_+JxZ&H^0hdh zl`sg@$({(q2QDlua)hnH@p>=IrBf$b#&u8cB_RLWqzH+$lH#m=2tuTY-u_A! zH!1O0yW{x0adqw%yePq=I-M0zJWYGB%#|?Be6ZRGO_QXW2DD}+i-ZdCS`T&q!TxU}mQVeX`;S@XPZW>2&&S)WXsXEF`=$Icx>f}L z6YU(E{o^tWGeHX1(qFY{8PTD3FT~LShBVZ5o)E23P>r)hS+~W8K5>LoZ(A7leM8Oi zi#3V%Nv>VRv5D+bo1>3fr5aN&If?o-R$awiWILqi`cgfK?$VVq-tp%|OYyz}zpMn7 ziewR|Ube`3L>76-dNdY4$oQzweFC&TH*GgO$5W#v=(cBKjb#qxn>&H1nn$U>8{t1XcnBv6&orebVz@*( z`|&rX5BB#69veYHmdZR|ZS}+2yfRxwzM8IM&k|FR|@+|*Av2#@!;`D+C7`ySC)4G?+x#;;Bd>{6Y4>y zTPunX&^8qJqcL5iIS91s(H*X8J5lp^RS3&ggc3ulCbwSn1`W2S72FNx{Q@1Y5tH1o zeGg_V{LSNsglYHgtb-QlvrLqQgyZ>hR5d!qp^{F^iFL5B))2uk0ZGllD#aa@dhzX# zZ%f|E>}Z*j3v1@-_K1TDm};6h<5PV)SvrrCTwyJ+G!}&pp0x~~=_K3%l!2mTo)nexhIVO(VUxUf#8-92_&={1*F{Ytr|5^blBEOsMQYqBl>bj z?rc2Lx+p_GFx+kMU`kiaT(NU{3!s&1jtfRMP!P(aph?l17M)JUq@}BTTg#NrXl6fL zfbjFyp1n#Pjz1*J#N=3_K7-FFT8Fevq{f3s^vk7&GA#nKaRMScwF zB$)@ltUkxBDHI%;b#xDssc*YN>x$gS#|4ZBOhTqR?aI>4M&1spX$o9o%S!jkeK>W& zs0PS3-JAsJp(zW1szsLtXDSV%Tv{TkEVf&Yh+Tx<8&HGW@I6Wl^L5ToE;S@TH$j#g zv8En?$PCoZoS+(tbk785m`p;2<4Pc143)ZSXAIK%%kY$!*oke zt@~*J^|WK7o5RqwC8DkPYD(a3#n88uL}HsGGb=JTtytjaF*-vp&e$uRvUL)SHOwczJ@3ywY}6)Ex_2;D zc8F&ImM2cZsR~Vpz#V7To;OBWPbW;=+fN61+GknSKWWsU+Knq=M!g#gX&6;MR0Da3 z1evWn8;W=s-QD+KYcKvn`_OcFmb{P9^j7b4!-w>bg5Xrj}49tUWV?Q_Ee9u+J8Vh#XLO~ z12aqcnj6$a1U`3SAeu|Yhp0`y*#A-Q_gf9--}d7Dwpg{I<29eIH!>_buLIu~NiLYQ zmykjI!H2;?T{YHhWKMx$aF)p@AJW$vZ%%Lf!kff@^+4QEAb*W~dmv=de*B>PKS(0~ zp@4`wTe<%)qH&34bb!h+wtp}A6&Dv@Ny+>S9Bd%;uXu8B^b)cD-Tci|X)J{GD^@N$ zP$_xN)8r{MCvK-g=C%&5jO&x%9R-6Y+T=?exLezo^UvXDshI1Z^i=bAv7D;j{=U*9Kb;3hAk}=eheSAwfkwXK`~(Y*SaIJ$^l>(vL35L1s4)DP z`OR}xuR%c+Y^(>-%JXTgyrB#-wPomdYIo@9bK_4I=)P(qU!rcO*Ct;U5%5hb9}5VqQ9PLl@jy}Ba?D;zHb zLji3Mdd-o{03LEeuSa~~p%=3sRaS@rxsQ%>?Sh015m)OQE97swEIn1R?j0+jW|AwN2XEKea>M21?KO69$a z=f*Q~vk&??ewh?vnEIwGgQn~%5~*(zJ9DVSD5Wh0cr8}$dx7sM-Rgl;pm*o*i#>y5 z^!`q*54$R+;+ufUrR+&}g>K`S4zg?Ho=KincP|30SQ@2%5{i*JZuwAy;xhn~fBXV0 z$?D_p4ZXN^#}J;iqw+a>iH=duLQN<9{gv7*cK|4XF!thB89;RF4r&5*p}1K*HQAbz z2^28)I`c%YgVs%(K&Rb`Es-6YIqD%A**1n<;)NDzWz)sV^K>eRbNj@qTUt#{uI7neHeM%1X4{z%D-)R^V}Z2D|F8p$KGH$szz*9}?M83dZCUK# zr*t(V`!2qt5?1d#K7;Lir8Pp+Wt^rEye>+f0j*QJi+2j=5%3yXD+}zXN#r`sXFGGO zf%D7$gcKEB@4t*j=Bnu)e&@(CqU8Wm@h(wl=)ymftB(80K@2(IkoU{t#P85 zftQfR9|StSv`mrNqv@E9vFWMnx9e`@9hBFQU3xZA??!m?9RSY|J!B?e+O~R7iFsRG z8H5)D5*u4gk?Kl{NF)mNwGUENHz#$gYYa-O-0MYL@VWG{wg~f(nVUaXa@Kh8Oh8`6>Dv_lAFclix#@1L9 z6Hluu($Ad98C)6=`X#S&J`as!%_%9oJI+jYANSzYCLOP@9We#elF6K#%D0jsU=-F(q@tsgOM&Xl%@xV!lO77ug&QRWKH?m@%PZxG7d3I#>4?0wZA_f zgvMnE@;IC@PP1O%0Pd+Tum(-OS1M7K%?e&c1Be{aT+-NLg>{b8z~KKWslFsi*VR?j;SA6^ zXkNk>l~MMBQddTmTv-Z@E5Z2_;9OW}> z@9o1^M)9%<^}WP$cdE*^(|M*TbB4zvABaqkJ?5mly-v>7C9V>{hPKN&hpIi z7#b;*e~j*$r-U)iYY)F1$>up-k>Nq@Ms1o8g|O}S6%T+e+MO=1GvD5t5AT|f0}~(| z3pd;!k~HC={ga;>XUxDK7QD?hoI{G$pCoypmF|fU>=~HMU_>Erf|+pSNZ@tQ5Ly1) zqaODwFc@~7#@wi{e5j`o1X2|2CGwk+p$_zKKtPO;=cQx^5*G^mLh;l@yNy^6AF~`W za{3V$Rphx{6}-k2&Y773^n*f@3(?v`eCsO;9q!9G|BLXz8%U?iBe$h(CG%So!Gz{I zOnS#=R2(_P%0vKk82R3l&DwgHeL@IshoFwS%CLKoGjV0uoH)18!M)zxJd*Sks%kde zJfS3Sm4&)WEOA#`A)qbt>T$zcT}694L}A_|N+Z(2=sBQ4bl}J3ElHt1+oZz`8TCrf zdv{Sg&v}+=ymEh@e=tng@6B&Y7o`YI6^|bXGXEBuDb5!>la8+FG%Bf?C(3fgFV8<| zXZy_Div$b4??z>7jvmA`e;)<@YbgZSAbtRc{uZ_W;ZU_^Y|LgbKr3r4n+<+hvdl(tJyxmW?MVVdvI`GEk zgl6_{7ey^QUsChAP?COeGJ%y%V*RKS?%7W1%g!Ue|3%;YT$$XSOZAePJ-lx5RcV;Jx|HOuFxYs;rm@2RiGd9ZXBYdr{*9j zGB35rC?uJ$Bzei0DeXY`$Qg0NGbsu`&cK-I@?o)O0Bum5#BF;F#Vs^;-f$Vot1DPu zLIZZ0_3bZyo!F(<><|{_RbX<=+#xJ;Zv;c++V>;QU(mQ*aoD+ARp+qO0DO1l% zk25HRPT}(i_R7-^!J1J@1<`YicX}EXpH#SSpf=Hf~0@BV*Eks!sP|d2VhNT?By6tuVkc~w`J9rCW@Yij5 zt1-gawNY<^8mzW=2H{U?r&{SVu_zfPU@LFT5I;M20PlDb&(UQX<(G@NOL$?WiLR0} z>#)?g@>7hv^*H&9tA!oDyjEKSGA0_hrLqZ=VBwv6>+vEU?a@srRJ0*us9C$=f?2Vk zLsOmINVltaNSGp0#%x22PgNlc2XdxQz}o!cjvn4)?hTR|M%CwYOJ%2Kwwmofztt`| zm9Pm@VHQQc9hV5h#SAkO(JY+59K2M;vMu%-r&?3dy~JrB)UMRKl7iCz9>(Zz5n{(o z3ztC>AcT(#sZUIG)OLsTGo@6sUz-h9)zWl&%xbU<>$6CFRKe`Zw9DtOhKm$y?U1nP zXSac;J+k8evq?eEtNe>E*vQ-Tz?h*$CsK4XA&-mf;0oq1v@I9-)XqtURbWm4%)S7kRhCV0_Foz6jJo;0f-Cgf?CX`p~eHBw%>gW!-T z*I3=j?<^{+O_a1PDDwzI)s2~2%^eBHU+!UrVR0XZRoXupY-03+B&5HLE2Yt-$??~k z!nCKjx01yWOt}J4P&)UfHYImUp_iwnJ*#9rWT|p_gi)(Hc}n+P*Q@ri*X4N%4X`1c zuBcD7!Rkk6q52~2Dc{)oh!47<`oiy)%j!Y`FxO_q*bOmwYB5F1?Hb3l>?5@jDk$YN zy3EeUDKVjFz5?9%rJcKNwX+d)%J&iAqTh?5^ag-T&DYihmnY4b(qtVkW*xMV2SnM7 z%TB7Y3^k5;O1sjm%wkXRH0Qi#~kK=Cc6wQf>7A188wi+K!q1`GvgnUrml)OVoxh}p>QTSI`%Arz^Ma+!^#Lwz}|TUPa+9LM;)18OJr{(gQ0aaAQUT^A&aklZU8PLTzM#xgcetL(RWNterQ?qGwlpichlWH1d=%h26S zR76cYyz-BdJb!WP{2z+KZZ9Bb2{hGclxWdq!8zEisHt1p`z%o%X8NKMKrU$yNV z9{2-q2WWs88PdwvoE(U-jg8(~y#s1hk@9g#p+{_f#7}FAo9yqjZ^zl8j~Qc_AW{rE zU-?^lk10T^2ANl?7;2HL zi}}~SynS3zv}@pWz1e_MMTa|f3Q9*rH6CHXUQrq@TDy#pF-P2_$s8PD;43?gk^&9- z%SME_3exCz>=h)8i3?_?4T-TLQ%f=lV-meEh}|Ha6LA#Q^;j(nQS-R~`F`w6K{Uea zk_V%P))1eUv;qGGq|R2ilzFnR0ne&G*!6drml4Mk{KUhLKSFZ?I zKD9)DZU=H%e#GcoVC;+bvLDhER#FKo+-HJ6)GVA8+6^fLdjFAgHJ4gh&K|08)ox>J zN@e8G3r)Yv6;a_(t72E6?_DK`PBQC3#u+TtlesOU98Cw0Bhoo}88*b3Rnup5x+at! z!~ki>14?}1rD<{d78Vm~;TlY*keZv-iAE|`lR}F1u!6nTcq?`p&I+_9z$=yepf4k3 z4Ph9Dw8P6Cw=8V41zrx|SK#FZOO`!+;n+?cjefmvTLS(iU5SC(VVP zN)^!0;c0=It=nauHhoJS6NE7$vvcDC3V)Bi3cHp$f z%GU18oEvLA+TX3aba-R~;^#H|S}E&YL6ng8PX z<_#A$;+V~-%Ti{yk1@OPzj>bZy-zyoL6D{6b0o!xG8nep9!yxykmf$H{e zaG*41jDYJWooHkKI5RY63?@yOdg{*Z$HG_{sgL&Mjb#(Clky@;WxSAuqsw^%P7u6GW{E4Nr(#ySqedQMg1 z1`#noRNv6g1Afxz66j)TUIK?exHcnJ*4W~-1809EK0HTx(T@8tGhQAG(DMdCl#|lp zj`6B}o9>=A(iPwG4&vmv{xzwL8|eXx1uW;vw^QEFiS&Fap<+!_#TsMvkQwA@q-$H{ zZ(@#>p1A)iJxKt2!GrN)X8z5WH1_?4pYg`0ALVEEBuznhUFSz*5^N$(}L7we<{FX`gULmfRC6x;@6q z^0qg`#)p9!l{;LxJ}AAYFu_TWEFF_#WL|=jEK$znZ=9v70LD74eAw6#q)z0CT0iCZ z^FP#Zg&#et#9%-`6EHwPRR5>nRm$1kPTAB+#FT{bzZkgmEoWp?jIsQB*)B;90s<*c z3J63_33Y@j2(Tr#QfYQc$t5AmbDQMWji%W(SuG0M!(XY(vFLFOP8TpU6S3z8;)&Ge zCOvLq-$3TaOVH-WQxCrEF6_Z7l!r0^o_YJ%t-FT3Y+1lj{@yld1Lhg=dYJmKM(~ZH z6gP%8$+gM)Xj3-n+!B@`_GYIgkxXT9W9TwonjBiHDZMm$^jMOKWMc{O`550V0diT2 zsm(!(@rB`q>BhWMKxzC5fi(X#VcIZFy`+ckP{2T)#!H3fBzp3P#Ecub@kc}+JyxTU zUj2C*F9vnO_>_xZlUSa+H6uH77iXmP7c4}d_rh;?nNLU3TeU zi6Z4D7^Ej;Sd1Wwsm%ZvCRvyXP9SN6%W0u0jB_(m^ej^lAtc(t|=R(GAgHxG!yVn{c$mC|fwFJ$1{_@7!6Zm{MOZM)Xx z-|PjI8#sbYyRy=gr%b%IA%l4-S#pvb)D8x2!t7G3Op^RhK#W)Gt@^x-Hrqb!FNR+Pt}Ogkpi6VIE}hkJr^t@7PQ|pA@z_C+ zGG%(?3r_e~pKJK%nQaRLn5Ip|)R)@tAD^HL_%iZ zUL`r=8P1P<{!JJEn}&uE=A$EtS!?kN>MVOFI?Z~KXA1ZAi{f;$kBi>nWYatPlXRuQ z#VbB!`ft&6WVW|AmcrvB<`6=Bbp$bMt$LF>%WoQI*-zCO!+dpSai2KG`Gu57ge#Z-Dk2B!BYW82wx77o6WXJ_Hn@31dDFRa&I^gnGoY5wU0TI+oQT|r~1r(OP#$iSVK|V_PC@0|YFz7AiE2)EskG!9%Y^0qegJ+i3fQjDuTQVpv8yt;Bj8j}8`$HdceNZP#DWl>tahvlSHTd(V;lv}Th zJJ-T&*)lXet7qkqbXGdN=Xh3L#_OUdozC;3D&5Y5*&}k-e4$5w2#o}vpqDvUVwR-2 ze+YfEsg|XXYcnBz62v1^rTmp@$~vQn@Z(c)0J}SVp+ACMk?d}4%ftOOssDd)$ZPNH zTL%9^?fF09G3Ea$)c(&ZQ*FAg^OguoSiI4=lMc6|6~$g@sqB?c3tgxXAz{mV5vASI z7@+`2N^A`WRFeq_r|psubV{aP3MSpY|A1`9LC`@K{-}c+=`=uVM&f0TWXX%P_s(-} zW3}%4{dK+{XrG8@Zh%4qb0U~FSa@F6Yd!fO3C7*DgM7FT<~P%J`avMf-o$+^47{la z$*?<<551t37&D8xT9AmNk8sd&kESXq2Bmda*HVpr$@7 zoS8I(8e6SaBuhk7ahe;-A;L~gYLy3_ZZ`QpOe;gp6^quYig-|~u}AtvLXuppR>hHK zwnFx&KOk1uRlR&DYxyW}T)o{*byVR~Ka~?(exi>W6GnwhYB2=M&`5Se%nC$g= zan$a)^C(yxU)gLt-+V$`AL$NBkT`zwlUP1Ey~9)I<=45jCmi8Y(1yu$*97Y3Y6-a8 zmcV}3^@Vaxabeq&=pfFB(|a_j4n7IbaWRSg}fjvY$!Q13`b#wb%+>hAwA*z|liB z=x{~`>)OdH7aq_L3(0d@cMv$s22m#jyzzY-)FqhuGhteWs8T~gfMN)S%$y##t`-(_e*@kTkaBG z9G5impmFA|92Ot79vlELyR{oU8;aD?7K&j>LWu*kYGqOK9dUNHM` zGzl{irugP{;Xpd$rQ!n`;<;xHjyorg<&1lV9F}3pM-FP<{hr^=h>mLr(*p4|G9H?;S zxO`BflZ-hy@y_kPb70>%h~|7pJQ1MJV8S0EeWG}GIEG=&GsZnpu+P|(*GFE=?>o&~ zaf4DJ40y$A@Qn?91LyYw*ZRU4&{9nBQZ%6}`)#q2vv)N-=GJ2XoAP^`-ViUpZu6 z{@0~O-;tB@-jyl!%G zsyEWw{%f$?y^Jqy`2HfBzN-4Y%r9^H+2cPgkNq)oA{)hb8UuAIK)5G>x zcls`y{?GS&Ef2V&_wF~Jb#qVnWIxKE*DF)rdjIjL!|=}o!2jLX*LVK}{+-y@wf{(wFuQg^dKTVLPHGaf-|NDq;*H#I})vXt61NqmZjNmKBo499K? z5sH^7ye_dgDaQxI{Sl2&$n&juQ9OJ3ZvYc@d88ur9ln9mfx8g;Cl*x?#dylf7_X7!M&z4flsmznd z;n~^136`>)Fb8RG>x#L(D%|iH{^kBP{arlhwv>5zG2`AHU&Ib5iAh#v~dO+ z3@fPgt2ZqaoSek_4M?D6#?L4aomTddq>PNFB8;U=^Q;vR(O~{ldr0^m^wjUnMF)$zenPak8gndrcsmGo@U^$C;uG@^>^ULl_Z}5|m`Qtu?O15_d`ydo z)vzMlU$%x>OZ=v!og51YaZh+)F2hi&S&9D)tfJOHi@#TC1I5Vn0y$;j!-5(y6Py(v zn8?&w-;A($_%WbXTHEX3MA9tGwSQrK(N2FQ7OWi!sN2XgR2AK%Q|7Ar5-1@9A?uZ* zJhg+)4=!YiVvZ~zz_F&PAEDaV#lNL)Sz1ChP$1bYbAlEvr>KIbCB>v*57pLcDC#C~ zUSR>-vJp8{pJ$R?V;j~;*tkM3N6g?Dh$?O3hu8lxy)0gx%&U^D%b*<{sNQ1g4`LhQTqbyH=+>t8S7U&&sth{XvzsFb?Pb9A zT|BX3DZ7yEFH|!tG}jwi+UTJ{1sViNbWmacrtF}=G%-~Lh_Xsr7=%AV9jOaCwpH<; z&BQN}u`Ps`TiBb#h}bL2S4Y)WQ=w%#At_-OVY}05iV#10=}UibB65NH^{b`P0qrW} zvzV83SB(E|$wbWb~y?aiOJzT)V{CpCuMjSDsoMcw0g%U?33w5lWvI2!7B z_DYFuW>)(9@rH7UG;gD2I>74iSCIs9D#AR03B)YQbfMrCnhZONPyL#cEVdLXGoQRx zn^7@v)7@)(OL!WG9<+MVXEYSrExxWonxtNFVMg2>)x(1R>!JL>>;^3#TPz->YY^Y4 z9J9^9SGgPNIP2T{(Y9?!JTOpVws1$$FjH*k;;$t+g7eH>__FB(7Qj6W?enMqe4HshCLWN@&szehFs9PLu2QqgnF`sS+%yJajHY^V)lu``c zyyt6%-0BU8=qc}V%TeHQSc5b!EimplKP9InPe07W(uyTszF4f|=n{YKUo(}1LkJejkD$zQ>X+c0$>5F{EDDm$>uusYcP3&(z2W*`EIRm(M8^*4u5 zcD~3Bkx%pALhYUeRaR%i1f_ZqX+b z&CM%vhpi@EU6rEkdVRuZFUGfta&L$agIkw`NRmI3Eq6mS6KDDxvw^5io}*vP+Zkpo zYN@vbR^(6drx3H2bC>zD3YITcR$QDsx2+%;WXtyPAw7;>tjpwW%Ab|7D5j0pV?F1alsns+c;y*^Xn+<`H zXd^>lZ1AaIXGHejZKQjNS1seu5v_A(S;rtO4wkvl<;K0_NgG-_KRQ z(f%gth3CmM9iw%wrog6cleKF7jKs!bc@u!%9kL)?$Mbu)ka3WDRyjFu1I@AVSN}>18YL_4W5f?*&y2*c${xamG(<>& z!A9}7ku4PW;|tx7Lk-2oG92bl)6=|{(yi8;6jOaX%A9;=nrZov)NRM_f}0VcbCg(h-i zclnu!x9&{W0MnjWO25p+1AuXA`VmyK!`!%p(x1A?s|7F^qEU zU>sr*GtzGG*VsbRn64f=NDJt9$15xQs3gn$PNPr?Mux}W?(x%K8KX!=@d5amX;bz* zTN|igBE*|zxfgI)f0jH)InJnO{QYvM-~^cpc6s^jAG$Gb3;D_Xz@>&5Z>;4|5Me!Z zBf-RM(022hP#>_tIKs!pv&*r<0!4eoFbjl^O93DqToYh9wZG{tZ;XK)6S3J=`ZjR% zrU;v}9Q0dRTF@-K-zW?s;o34(MKj>OPj!z?Qv~fdM8PJ#$EJSB8DqJszFnayR!Njw zQqN@EV40#layWvpv75-S}ltCX+% zxWwj|&Zt}S&0!p7GOKdtyQQ|RNQfajHV#KLVd(V|pd5$ysWicArZH~&nq#S=Dc-EO zKqSG5kqN&Qu8X;LHn0j6XNbk#zee#+%-hl~3x4f-7AEU>4ZiC&E%JqH^FD^-or`B~ z8Ih06EW??ABwSut(YnI)aJG73lL-cOv$0gv?)8qRyOi{HY1B1GKPx+10 z%P67uAv-Z5S{5BvlO|BX**4afRy#^lWJ^1!Vb(QD%}j`~WE~yLY5HCqCx)li!UWRn zTYp!6z2o$u(?&n4v49y%NiLw+OS&jUWJ-PO59XEkJK%f8SLNaY#ddCzr6vNRdDSaw^t2Co4!pB`dwn>4jU>w z`%1nX3!PC>>S}|~sI@gqX@sze0sLqZ@m}L-rF;+)xuCMx=gGjTS=R_J>JJL+eHQe@ zNPCnT&(db}dQDb_t38~n(oe~eU0Eihu5J3px|6%S;`1jsYvVPjm$};9=6ctzo{G6b*Kp^0Xw}&Ztrq#(H^;4ch_LI8rT)1W8~qLV$gTZFi^SztpTgNb z32te(tK)V3XtRApMFQhTU$(>C)u zoEKVXR@?Pld-}qYtivtg^k&zUe9tI%JuiG)eYN6O82peJ8x%*l$-ausf+(e&Q>!~K zc5dL#c_4CgNf=kV#aw&IdfS(UVOJgTn$hX=dU4hY`-t__Tw9yt-%JgLda=L9#`MZE z7~)Xj>Y=Zp+dP!6SJI!~vcSKZAL9DCAn#J za4dl4J1QtX?=qA)fq2Wn(qv8ktoA2K^l!{pfDj|o^~ry`EAi`%lWA3mK;ssL_C}z+ z2&qD(qgb>l%RRGJW|6DrysFk~CX1BaygFL_T(jZaBR#o`WLrrnV%psSJ_5qc0&=_d zrd#e}3=ZVp{X@I)(KSll5)X_!Q*=#m)c)Cmx8c9>kTTo%PtL3wI=^~r#KKi?Y&9TW z0OyC>MJ|^?ejdA*J$K#QoM}+?Nj3p5g8N%R^K#|79pX?5{)EIly~;`7c_5qDTHMKyowc-GEXt{YG%ajI0i7z`*yJxEHFZ|X zJED#w`8O| z`I_ToyF#4ErPq%7{69!q|1m>-{&n)p$|v!xim@kj_xHBwpkAm$u;>sg?kp zme`Nrq?6Z@|Ap_;&m!QLZv@`J&|RTCnqRmr5%K)0tQ{e(sq7_5b2bmS?9}uviTzGV z56+iL*A&;}rhu??o6;?Smc@b@_O^hK)YMa$%hCe;~8!xe%jU(P=ULoGWT(hh?XP_s#soyRTp)e}M=&6RH}kBcM1V zBQ&-HsRu$z0r-+`N6r*l#ZK~<1G?<4&`By4;f7Q=eYg=$V`$uUN%3U?v@+Y`;;Z7V zT9rng`wSYFTlP?zdKdfR%BMxjd~a}=Ab_tOs>cCZ)$Q7|IaqGRjo7|xp0_ePuGuxf z_tq}UP(cdebIF`~R_E5)wontBF_95Lf?2lNzrF}cMp=|CJ?zjIm&^l1%Py$GF6?Am zNYw?KwvcIqqG(b_A^BRi#4erFkIh|`Y{_T0 z1c+O2MDn@$tD{fdmw{Qbw*bw@c22xSeFb#<)!ArJw&kJ&>SKJ%>q6wJUQ&v%-Tej!U@EqqZgw&xERHIB@lLrsJ6dp;%O%Fn;Hc?HPxi1w+-f8Ll+4-`CF%GVl{hm zql-F$d~{!Nv)W6d(&#c&HzlT4ljQaRcRz-hpl(v=v_FJi9DMd6wQKXibM=hf=oT2g z67Nq>b+mpcR)KrAPniVsYGF97xdG;Mof@EE&a270+@f$8Cow&0unyP0Ja)ZiY z=Tkjy#~yL=_z&JecNW6M_bo>sGvTp(KmIb5zyX3rs4DDAyM=Xn_sv}r@f~(Ip!>R_ zu~5FV_3p>$peVi~SvoYU%1l3Ia8s_ByDKEQx5qyE8vop%(rFdnKY9-a>$;Q;f%q-} zcTBJyHm^h+&2F#(j9b@|7yxRv@!}@yc{ip197f{dUOc6wFsrfGG9c}}l-OG#bGIAM z;`!ha7W6J%<( zMXqkevlOQKaJGn#-4+PCPn)d!Z1PX0Z_1v-08OLmwJX?O_f0MI^aOqs_u6hA;$Kx0 ztCDADWQXeoE_q>TZ{Y7uQod=pncuhApz!Cr7TSa=`MnWHpy&n!2HpwLu%dWuNbi3z z&TW3d7{IQ4`y4Bw6KU*4IQT*jJqkn2Vk4&HhZ26FNc~{Ve90>;x@%gHgV37~u4cSt z&;BN)iwW-L4085_fK5x>8<-<`ca!r1=9is6WhKf`Xcg6qKV z{z&LUk3R+PL}-5s`Gq2XY`)K!cV_erBJ+)#H*)8UxisQ5c|dGRf=^oP85MIywEMXb@SVPwZM9W6*uo#TCubW<+n{}gleX@9yRf^E-{BjXuee4>Bv{d8V-d*Fpl~v} z0A@TGyl`3)E~J4?e0`+X!(S};Wk4_sbNrv|3M;2E7Ujgy#vHWx)xedhxB(>o_bjSJ ze$;*jy0b42Gm8u1qL-@tIP|JJPCXg2^7Vd6Cbc4K9`dzHb+5xaR*JOe#YS0&*HxDFT zo=182J^9v2P;S{F(=b{3Sh^WGi}Z4&`;pC>?53Q$S{KX7dl*^$`gn|N*7`Fypgbjt-k?_ggV#GUn0>H^l-| z$@1q-r!*^O)LUm+@=uw1_r+FB&INb6nd5j~_P<4(F))-bmRN4RWbf@F%Gc@0N9nZn z%;>*ju^U10j#cN$lCMxpem%=z$WzNuioD~LLD>ag_{UmtKpBn@M7Tj1(S|g{X?gPM zPMj^E`q1ePpDmiT*q_ud%fB&srO}t2y&-vJ(wF{u2bc`$hAU2A=U5{kT_h8#p#$Q> zJvej1uz&f58Uo+afH! zTqvU$d4Klv-3n}3T~5}xR0ESbpfL7!MX;y3F4#mJy|-sPEO5fpUHb2i+073*;pbT& zaLkeO%&!OXx{6z}Z&9ioQZ2|%Cc6uQw>{lCMWmz`4(wds>H=nX1Kv=a=n3|u z1!O|Ee^2~%jPNN*fD`DePrwHPv9SeqjAta!q0uTuJl_rJ*Y^WWdl(5-~4iE!( zZI$>AVR4KSTGQkMecH6Y&0DR?SX@|Y@Ll-a(yQn%g|q@7Gl z6_0I6;bgn1X7Ck!F&f*yAwRvI*PU6GX8-Ea$UQ8_(2+pF5l0b93E%iaER z+7^v0`rFAxQE;=qtb+6kA#*wBVtoJaF5Ol1&rbzX|75P6J>-Ro?WB8IjZttl^-hj6 zPrsrkX-R&8o?7;=F9qgd?zWg72Pn0&O8O40IbYH=GS(#}f{0`S4A z+`}t(07f8RG^tGgAuGRwY<`dzInkbS||2irIQ zu7uVHopnus^o9GA zqAz@b>X-X7;eWuM@YFlW{Z9Lxuy?BarSAB+H!^)U;ozq}cF&*i^;g*pAXjku(_nw*1LqJ3y zt?fsy2?)yoqdz^qwGK$a)r|1|bz_0;KBNVBGV~I)UGpL4RR4w2qAc{rBFby${ zM~(@Mb8&2RXpUm+B?29bag4Uq7K$CS!OSp0|xwI=Tha_z!l;S;s$TQ9GR)&n*u zI!9&aAGt~zH35vFwsj3+5inStFrLAE#^QA8h{dBlc#R7S_lDzSP10vviO{8wEKVB_ zRV0kieEgiz&r>j^1M&0ra9~wBml&5xulUZ_Q;`&GYj`k!)x-Cg0bh@J9 z#H`oSS%#0NakiTqlsa?Dc|4im##UYls#%g^qij=iUlz`zeJTt7SVqc7a-+oOr#*3C{ss4Ejx%$@iM7T^eI8?Cg^i}p^XB-?uKsJh z{q6ECa=L^A?qChQ9Q;gnqX&U>m@2?^jm;n0UQ$?naym$E0 z&^DwFNhfp0mM5+3m-d62<>IqJC<~ z#xV!K?~7y3(3&YB85NUWIL|D-=bXLfJ715-+l_&wJ(7kw>`W3zMbdh*qjr}Zv=b%4 z+)=hy8A=Vq4oVk2&P3a^k8Fj*XoK53Z=i-x;!2f($Dz!HlXu-Sk6b`|g&L52{)^ z`=^EFAN1jo4|#`0o&)fmmd^yIx07P)Zq#@VxK9{T4xok&h!0<|t9arH{Y@rVP1svR zN0m9t*elAq8cfe~DBFfm@laL0s48*R`>s6%tI3Gexv}Bn636Fx4ysWtBr)b+N^F&})|oul=hpj1hrgxKKlgXaGFRiMe3j`(7+D zhL`=*7Va|&0p1?lPRgy@Jhq*lWhbZawDV6pPlT`CRokNq z9XVOGaOwB`c#2q>H*&jFo3qyJGRIo2SS4sRwYLmCzV(ydB{yR^H*;z96&_1?Oc)}j zO~NB`2_*3p-l5#ScwaFi~ zirLhBX+qE)!&;LpOC;}gq2Y?E4uMWlCE$jk4r%Va>oiaO+Jo37p6FS7fMw+mY>HRf z4ymNRp@c!46Gx1!g4*a4HkZ#y(}VE|`QJC}E;yO)`aiu)(?2^Fj{k2|=>OQT{|i5= zRl8Hh7RB&OE9oTWr4W7zU%({_SiMw+nLVy=l6#+K$Bm>7}6*-A9Wdr-~zA!_vks@)lv=(UMg8#(K+lT1fo- zV#KuTL=<*~%3q_+v)@W(p=h$>Xr2s$#Ld^`LYOMF^m-+i< zncuzTMFNcnl*a?2p5aBjyXA@N-9w^ZaSy&S4YuXX=T0lrC3HoJk6=dq3m*1+{nE?$248?EKFS+B~2n( z^%`ja8IyOFS|)#~(T+l=*35aX4i`;o_2>-i)O}%+*+b%Hf9YJnqJN^I57ZI5|02a51`rVc|F3BAU&q`33L2vt)*dM8n119N*EV5<0v(+i28Gae9kn6a(1GAX zv@V^j6Rnn+*K%ygY}{KrkO&knV)W}(wY6(rwW`r*)jBp1pxQNUs6E!TTKN7S=ga+k z-wzkWedcD(-5Zhw+55&xob8Z8zz7A4iffkh{#7)HrW zg@2-~5YR@=P-vXjj4l5Qy&h!*QTZsf;3Yg12)&1koEd`Q@emz# z#E7HHhz*60i~Q|?mxmA~_jyY>&sSi8l#e>eN3!JLZ}R%z*s2ZtC}4}~MOnj*qv9QbWFey`

    cF39OH z%W%-r)dl26TCMPc&-I`I@tBy=SJjtKzF%{h7VsUCN6k)aDVZ(K5BEAbEBt9InX%># z6K!1O`3p&o2dRlMTRD~vvec>n7iI4homseTi&kvgwr$(CZQB*w72CG|*tX4zom5=O z$=YkTv+r8>wsY5cm=E)5w%NY%jnPLR=q0>cKv~=D-ea77*I)1H7kF;S1 zxrG;bPcToB8fD9kI2kjSc35S6>sS*hAY}Li*Vno-VB+x=c$`X2wxC>NHk_Y>c&Eu+ zg#>FCs(8RL(5}1TB3q>RUrNP3;5Ip|X5b8j40@XrUj3O+mC$ zYrsxq$W3SCl##Y)9OMI~i|!B)O<)iWN+v92#!Of3&gDaRppa++#{Wp}h0MR?VD&`~ zDW#5tp;1fzqfPk3qws}qC&lr-9zh=~K=BU$lhP>rf_OlLdX`KgJN%-C5sz{S~moHj-A99wb&7ZU&e2q?(3pr8G^;JRz}kq)u)ZSEE`jR{roCn8^utG zOl|BfW0AcN*#Wk-1G~rUpuZU`7Y<89eT(26nt!M`=R`?~3(fuQ1aP6@j}aN5-aQkw z0$lbI30Vfjm1WjE#He#co7E|hXFfFz9EUTuAauV-r7la`MWOBaa$}rCb-$BC;P}XD zaAV;P^ytVokoaz-wx<3`ee88k}#t{xb&{6e6#5$g^3JLYLHaQ(G> z&G0IVE6Zk{k1hlB%OvV$!f=v;U!BkhZPTnh`0rcLr?uzWF$dga3$E6Xwy_@532!mW z(=Rt#v=u=(3Wq`OR#gjRc0-a2X;xzt3&p@k{F5MW7nb&X5Pe(o4u>1^71EV z!Q;cupVA1u#;Ef*BjsRahF`cpY`YIK7%d*U?8|jDWAv}j#Fp`UPk=`xKgzK>y9#|P zJPJ8@ZjpgMp?fU~pMghG_L(bv7(ao@e;KVh)w$kpym#MTX57?q#z!V5Yc51f`FJyV z&3gG3Z|ufW4dALT6S!@#o}ka?OcdfQ&X~O63?z1JLtbY3WV(CV@~#-tPX5Leix@?m z!3t^0?X`hC1y9X_vShiFbbKPIR&adeDtzQ^-Gjb92=xk&Jrp-?$!p((lv=L6oj9@qFb*Erv=tx2uu z{lAERn4k0pUB{TO%@^sjas&#$971Z1SSPa6`h*Ft8~iLP&6;1ePUlLh<4jU~AL^RY z_eFh5Bc67}zDY&FgxTsKO9-)M1E)eiDjX|>Jh0Y3eNOxgQynqu+b^03!mw*im&;qU z)Cu5x=^`~W6)8+PqtBDw{tgJoE{sF{VB3(o*tu!txaf!~=`Q+|2{C z&J2SM@vfHSc-KaP&n&QXxUoo#Ic78)NTW!Y#UecVrhl119;jurNBlxaf;o)#>2QxE zmW@7ASiZuFULfN3L3n=#91vwO*KEa%I{)gxP%x)L88p$FXg=(xHH#xmDVZV@uAawc zaC=8GlR1ViqFNq^>HCqS2#_mBkStFH7UuHy`2Bc2b9P0t#H{Kc5fQU8zsjh>NIml; z7*mNFgUry;Z6VE1zm+P(TXhKwB8wBhfltdF9;HuvOt0q*Z`QFA!@_O>dZfk*nJ$y_ zz=Ut~1k0n-vZ)2yo->OYM{goLq)7lj-+Z(5ey8#Q<*@wz1@o`?h$LH$zW#fg6CLu8 z9~A#5d?fn4(n;0J@xR48OEhd;b=A;5XS%LUkB^sKqg2XTHKmiG5ErZiN$E^CnnlaL zr#+I^S2sq zsh6poLAW78CwJ!?x4P?N#Hotov;NxlB1KT1`}}n+T2R$`1MkLmEl(+pXR2FWoM=ll zVaZNU?o8J{*DvB6y(>N&QEL^)T3Z7$M?!~I?+W)sxio;3g|+hXCFC+#}HB&jGprQVaGv8a(fYYvue z6x`^6i29-;ulZ=wS$`?ePF8Be*(S4X3$^Deg)&GG8t|WaxQ0m+xp7VF`U;VTInZ&gO{_4e*!;`slA_{X2RPvw)Y;TvDft7^>lt*@jnu%X zQ!}a)6Q`bZ&NyQ}*exjIWN(Ld>OvaJZiGtJM4VKEa3jmZvS>%+el_D&hEg9SELT5 z;b|j7)H6K4?6W7s#ICBI<4BVl)3V8 zg$HF$!>C+Th$FpOSB7S+KS~<0{ec!Q#q`6%hj1oFv6oV2TJdWMYcI1Ciaq-svp&-u zwLhInFB`s|l{+L;UB(NXzs;Tpx&G*lR8J(wEd+|g0bT7fDwN?t`~sVcF*aWj6;H$b z`5JCp*p+BzGXHPWRFa~|qo0U2weotYJ;ZFI$I{bGckG{}(|KqgF31&Jp=2C)@Snu} z-Bpg~Uv#NvGKmU(%&Bfph9%kAd)1A>sd}e%RSxNey6ULH=O#5dO4=RJFxAmIdRh<~ zS~inYFd1y?x(hNkWog!KaaL|xce4~5rhd6c~F1#m#6iCRROe6 zcC0yN?#>SgbmdRw$PcrLXz3#t$~+z7Jc*7HQTviDbz+9nu3}OCyn8S*XP)1ZkzK#; z8<9JL@9&Ylz<$HX9LY}@P$l+Cb7>RT9T_Q0F`0EnWs>itWN!?F7vL|FV~qF9(O49P z*JN-AGQ4@#B`Bv7RHH9$2yL(2$~bm5?&IRlk8pE7_p7+br;d@0QH;@*YbQv|xeA5l z=M&!Ww|4_mi~dGBLOGfZ%L@&lm^T+H%?A=n3EF6kqs_ZsKyv7lQb7b_Ye} z%p?uZ?(#V>5%V!+_3p+sMI6h9TU6iLr>Xw{S140n5q$QFsRrJo)t{%439UR{sIM6g z34#%>wm))x<{n^ryv<>de=bGvP@<8t(9`K)w_>e%c|&SKY*V7QGG( z|Bjpa;B4RKSO7d*5KRP)b2-{2vHNdEx%#ZN9FZJDga(T6vmiKQlJK8~6b&rlSV4Y6 zJ-G5ny5m($0X#vokt4grJ%8!0^DoMMS!nM;d4<}-I+fIhK(tPXNen@GP0A4fz#~di zu7s!bKS^Fbh#mSSx1c?ncwk$;hIG8S&7(ohT0MiCVdlNW=CUAq1jYI|z}rY@4C^j~ zOwwD(EpbOMv_G)D!7_yc;G5!&%6#f>y;J zu!b7O_DDsv1ZqPyu)hlcp7J>NOer5$a0kz9;7vZ8-*B;oxkAd=kdGWkjIv){EmtsM zNB!G%QQl9vuHhLIw`8Y{$koBoYex|6kLlafIt^j7Ua+hVp>zW=GQR2#sXyR0Z}D3X z{dA)doN}B^!Ylj#Sd-f;PPZ%34alAIPMmz}vBzyLLvIzRAILAqcP^E*>(q?$SwlWo zvR%DP&mU-67KI8tL6L98#hDt0+ zj8(?prIUn+cqeyF-He(0OCm7(*HsQ;X)2h`DS!scep!3#3~|{B!uieo7Oz*F{D#ot zGYhjP@y-xshRji*8EYZZBhFB-Ya!G#dx+v=LH%C5(lS}Ir=$G3@(gI%iy)?>2Vvd{ zbbGC9!uGow76~8;fOja@va)oyH?|&&1fVC3Xa#)S~_tLx}s!jH$L-EoWTi$mu3GTagfzJ;z4+} z8H#8ZUSnlI{J{5(fr|pErUBW|f+-Aj& z4vs!gfV)Y@@wmk?%$eO{VtFPFKTs{*4ib-TqZ>cX4nCyEW4z>Yj-(yZ9KEXbDCsY9 zv4tPvWoMZt{n|sX-Jt`3wmJT>r&@ZW#uQd5*&Qu*_2D{J*Bj}LzOmX2p>K)evM?S~ zx%)R#Y`xgG>+d(%3kLfCS>gFVH6J5m+y91sW(l*>gMx@7y;jC17rh&m9Cewxy0_X$ z(qgcvpfKJ$>&<~tilrr*=->PR~h++;2x}P>8LTjX(Mq-!`ZQ#LD(cH*Ztr zgqC4>qZezlc~J(Wd!0H7<&_5)MfQELOHe&T$8IX^d|I*^Zx^v@SL>1vQ>d8yuB?B)>vCaUP3t|dF&!xNGRi5qBCsW9+QzxcU3ChCVi1{7SV4m_l8li()hp}mv#t*4By0uRs*717QSPJz3#q!F z4&!5ro`P-bIT{l0NeZ?W_(adJ%_gtoG8Z$s>9QR&>KB%u+~uvh+^0P88KhRD2s|11 zAWY!u-~1V}K}WB&8zu5P@yx8=Ek)hz1I$$CSCzC~8Sslg;0~f_XFC2g^+SLv_!Id; zo~pHk!%R!ip!^U!!ygu`Ta1n1OurD*Z;=K#D} zMU^o)?KC?ct*)U5h_gi?odKq8pra+@KElxwA?ObaM@LldF3 z><6KqIFSQDNes`4)*=`A7j`(?c&9h<&Ip7B#**@dI4Yu!+(THrQ)v67j8P9mKZ)X_ z+fj+oMFSN>#t$;;=|-k+6xyx{%sY@ z)=RQ`*0+UseLp<^7nc72Vd83Wa?k^Uu%Ww4pOzX2RIVn1x8TICC37f4?D9I&WwN9A zGocW-iU%<~O`cF*UIbnNxePxbj<8a()S)BMHI|^GuWA*?WFG2MvN!cn*-K0)G;EGr z(K~NE%>$g0=B0nb5pZKyMM~6S7mVDrb7WfYj%(qUc4}q<2Y*UJU-T8`Ldd1*P^8G9 zp%RBEHw1t;uO|z4=Hh!6j792 z-#ZDc-gSLTDA6s`lly6p8IR+ssT+aa0O0gtV|cxER!4?#Q1T{5#uABC`%<_m%yc%# z-9C`89Ys+3XLnIVz(PPn$4Fl;vo5WSfa;b8&sMbP+u_*9#X3T7DwcBV<1^+64Rg76 z*T|Kh(UptDL}rNw?Z8#XGUZ;>DLNLjg6^)G4D7hDQ}~|Q8Dv;))J1g7>2xh*?t@-D zu^*aMGOWC@-51LEPdp$LAjJI^&2$shwqXJZ}v2GWI7g>O;q|zw{2bQ^6L+ zvMEG|Y}Dc;f18_SbjN3qeLP5t4EHoWU_!^N{^Df=#57pPzb}VX41KI>m>pmn zO&@J^pR~^&tFjcoAP~l5foLI*%z$z?fi-m4zDYPUw?PTLgl^&+d0BG9@UP))o;6mO z@jH~QeDguf|B=D}10Vd;=>KW$C2BevC>p4r@|c*C;z1GRHFnF&uozejnl%Lt=mmrr zLbT{t!-Gc&uwfal%tEhJf5H5Y`x`A+&(Bo|-nIl5`Ir1pKm#oS>diDXCznmLJMWuq zO(*dI{-0=lW-n5>!&mV1C{nH_$|?q@W)mZ>$ad+n`|#$<>BG9v5A9WHU^vlKob47S zQa8^ndui;ccD#u7U5uJ2r+IKiK{S&KG}LXB4@R2qe`0xNh8a{zu-=Ul&?I@)fbDad zDNMAWVeE4|aU___t1HI*5MLWIA&mug3NwF0*c9}Yn4^&#jGSv#Bhi=z+Q?C=8)MOU zNY%k8wV$1+iMo~HRy4-^Vxn;*W4T4=C#A^ss;pK~Y`9AiyjaX^LCIKm>!8}UzD(P) zsvP5tX#|-B#*^_UN^oK;RIN>1S_|l8VxvFwnSd$sWw(fF@!B5=BY=r9-&ymj*`?MP zj5xP5@!uQe3e=I?T(8ND1yNbA-0nR4FyKc%c_ih=@V8RW#MnR@kL795&+3lN`xNtLjgK}!M zbOO9C&MkNH4%z*bt2nzfEnAER&y0frR%@}t-7#KLkL_i>u-cu#e_Dz}H+!)eMG`o! z>ILz){a}yNo9)k7Khq;8XQ&70TV@5qU$=^T$7&>uQ&? zKb1G|D}n*gl+duf<2V&fZXRxWUcRRp;^>cvP#{GBff>yZZi;n|It=Z6vLcktT31k+ zs2>ts;s~JQ6DwNG%vYa1{th`W1>lX*jm)!5W`vsVG>qZ7Y4_oU3xoOj2Ux=^b`%Y? zGHU4UC4~($7MW-9fo2!9y{X$D>DnI+8)3$Or~QE`rf0zS>~(-|4p>2VSabuQ$6tR~ zByjHS(N=)Y+#QQNb`!rV@rBz6%&2gS#Eo```dB-9LKr)qav^W>wi^DlDeLf#Q0Em+ zf@2?B%*f;nV}!+&?xcft10rEPNpgSU!WggM@)-YNa2yM+zYZdgaGfu9s2w~wI}flE zW%J}1ZV5oYKQ(M~Z)8EhmAv6eAeW2}l*x)AdpUI3h`wQvZgVzRV(iVgGlO{An!6c; zyDHQ(#J~f7z;I7QeE|X?RR-?u?=XVPYjk%KCE#XDVP8@>eY+dZ+fcIEELP$1hQHE{ z=6#Lv!PBE;I){@b>VxeW1tvaxp~UMLTZlWrHdA0xSiOscGItV47fTkI5tU7EEjMdQ z^O1W9lbR?Try0dROP1`)XPO*Ik5#ZcvfTpxEKSi~5g-|$6Y-043ch3dk_vW$(Aebu z1p8P2<6UMuVf~$w_|@ zM9~a~G9XsbQ*ZB zXxyKH;xhr(r{Nqh3I9izvymF!-ttrx56!BPWN)`pa%Q(BDk&0(PhL==Zj3knJtYq29?u#jZx0V(ubOdwnF zB7{}Yho(GL3a<7ADGADi%x~FqftpmE#LW|DOVanTo_KC-V4p82M9UA~bG;j-F!jAX z0r)JgKkSNoyD*-0So@t>ZYvYC$;s0uS{w}B3dJW33jGcO@Udpv=Yxn7lrS@@)dE;V&7 zp`0Qy!m)i1Cwd-oL^Zuf>#UBhq|BB4TTH2Iq(0#t*?he*;2c*{Lm;WN(WpDfS5!P1 zS&Qk^TZragpxsXE(9lMxs;33mT-9dDmTa|w=kZ5opO1}d@enP6*XLf*9b6KKqV^IS zr_or$C|g2`tzXS99oK+pl(hvDj~DY8&AiJ*eMvQADk9vZ(-XR^S8j?<;vUMH157#Y zyXGXT=_r$GE!;vL(u%Vk-6{^EN86#_+zAcByR-2cK7i?4OByPRB#A*MLze=)GNSmv zT)r$=yE#Hg4&@|i{Gph)TZH?489<6cS_SSM#|@&(H#eYayzPqK94=L4QKJ`}4%TVm zoZ40uO1Aku=I+$L807=5hWtR zK7S>wxDK$yc=0zg`~PHVT=U2TDPF*UTtPD4){sbV(Uu5tYmU>T{tMD}KBpa-D@>M= zFG?kyG{RU)dC6KbgSgOrL^2KP?<_vcwBOAWC-sH(riOxo2sSYO&0Q%Q^F`^r87x;~+)S&;< zd+|@k#pr(s0d+eS6$w;-c2Ig!3}Pf-D;u#AOnuVfG;26zTHyj<>w2Yn?X)bi4(#jc ztu4-bl{b(Le>#>HShv!v6aVYFv+xZI4oJ{N`R3&D>-FuX6R(-c+@tv3fLF9WFHfdN z!_weE45mk8br~1$mxY08%ne4IQC{=|&7m4j`@=Z=M)cH!dZ*O^4euGNVITDCHWRKe z`)W;3!+o1T$$U`plfU^1sb*)WYJk%*#ncaB?pZE8@P(@C3c?&UW`V0v`}Rbn2=1OX zaz;fDg=4?o>=lYLDtP+>?6J8&d^I=THa&fs$(+>rfr@+xUTn%#FuL{!#$>+=q#4fc z-1h`rtU^~0e+#CVbhFmhR9?y(?mvaDAVR01zRw9Lo2N}=^TbNxm z!AUf?3yYc4Ow?fA-;g+p1+Wn@nidBV3mk+awjNxT2#4pkJ%oy4&n+Z%biQ${RaFMl zVrFaDw84z!t9#p$&N~VBGu40!t`upbgm`46wTrRtT>%r{IWAdg=xlilNsU+=3p%W; zd(Z6v#Oqq*Y|`#>T*&2%QUADzV%WnbTb9}ts*#PcIS9dTMPr*+FCy031hiFsg5m@~pv-4cT{MtR0 zLpk@=tZFFYO?q?LTAO`d%)Z7ZW&W@+8tiCo%r?jAfsoMI=G`J5qrBoB zH}X34C)zvL>bR(aJjB2FISljN*|PTgw+`NiqaYn=u68LEiX(yA>haeJ)0ldRM~r3( z_J$iTw@|%Ihl9NIEp8tm|LV}5)FG!#zXN^Xw^+e{#smFVdic*sPb%zYW&1y#?U$Vc zvJm3v*W_ZqS4%|Hp9<|7=*Ruyr+u(uYa5)FwQ|fD1`JN4Ki&yu^N%`bFpHun0zU~R zIDK?!LXbgrqSrgtGj?Y#WuI1WZUlg;4Yx&spOGU-ip12Tv(aI|(dCBfO%2z7P9xGl zE0oKHIg;xbMcB5T!w+CC^e)VB0$;}%vwgLufc#`_P zkGOzM843glLb*#B6#HM%e>PvFb62-l4VSXCZl(`rp%V~|@erzM(#;JY4 zXp$TV#cckz{ao7EGb55PSp&PqI)x|?E4kpLcuAD~At?c}sj*w0VCYHV7uaIeFJ5)8IYW zU-C5*aHBZmQo6S$tCA0-X}mx`GP010J}A$hO{6usAf<^xwU4~?p#2Er zJfc;JIxk_0*OOB1XIZOMmV*+-KK4MuHcJ6tM>1uD10wan{ZS-_EI^m{XZ2%sg0A9z>W$JC*k^AQwr-KyeT&7TTeUF}9CbxS+?ad{ zUh2i+^}l4f*AdLVzZrp-g*c9;c{s(bpjtWI%PTi&Sc)-!{@bX_8?Of!+&7um@g0r- zGeYklDD~e7J#`%w6bnrM9$ZaHbtO?$1hoX~A8yOqHb_Wt%0>J6xYbWb5{)6t^;y?K z{rFvHemC5UlTRa6GtDe&;oOUNfPjL<$xo=V8owSnN#J6Y>?!|~XRo|xzdQ$y-oFo% zbwA|ym4G`LEB3vCkz`x*K{!}t$q*S7N|!CkBek|<5B#>2i#DZve*?vWV0)bldH4s<|^t^WOb0;*kF9YpuHj|~ib#SXz7ug!2Q zX^Jj=Bxs69eq8ex+3FMo*W%b(Nnf7w*hv^9>Lk}wSKmH3yY(nxhgTJ1`vQeUt^;8@pWy+0|js%sYsD0rFi3}ts*c|l|+6wJrx#rQqQ3?!#QYArMgIfDf{oqt!^eQ z=&CX!AwnlFDvMNA#9VM0L63?A?C0%^j&N1s^KQ7`>Lh+-bXD14iDKeTZD8)~A*hlY z{Rcz7T5#CcNt?e=+PPeopi_Fq0i$T33Nh)f61(R5qgkA?e^>6E=iPocD#yfua8k}S zWhkF>hT!8pD=yWS4~7*L6d$UzbMwATWyJk>cH`|szIi^bBzH7T?pSDu3CHGNyyL>f zGuV>>I%KP3+qE@9j#{8sRhE^jKK5;CT^YQG<9`g$qU!ZvpRi+F9h#C=RU>t*Kjzi@ zmCJGcQY3G9auGK)h(laoRa%QQqT<2OTHF2LJLQ&U~cQio1(-x zt)5~#nu#H`oimSK%z7idMHTex`95~_NCN}W2?Obujdlzo|-{E7UXsJ zDd84{2IZL&80{{tDkz4c!X+Z_pKpFR=0PEGPn1zZTC0)=bVb#w1FH5hadQZR8!^~a zEDrnxNS;hm>{Z%d?D2*@hlQ|?P*-;b&ZNoDg_{2miogkizUEb6A}o(;r8PKbTW!RA zNMzsCQ%>AzA=61bfcT`G7>N~sQDwfXY0_9CX$lbs-P2Mry1Rk_`r6i}`_Th2*0}vI z!L+TKuI{P?Y}R)~1p3feNksZ}&Bj(4BWURjX-$_U0?Imge!4WD4&ZF@5isM`GUJ)Y zV2rt>ACeLW6vzw?r!B3M(GRsJ9f#y1J*KpFddL2tZp5OOyM>^`4*#i>`? zj0#q>j7+9E@=qbO$<^Q!W64 zavmrqY1W@bG?3HjA>2XOtC6aiu84o(TGcL-+ zi^0EwgM%N6U`^k=@yGY`pZVASCph?Tm6ViNCV3DhM6s{Fuuw8&pAb~5{veus!V*C$ z%5sXG{)-edlFi7WWlHCtxYWkP*xKfgCl@jX4sm5SAU`Rlbeie7g`ZR1*C(P8gim*y z>Kdb#SL?zfpJm&1#Bh}F=DVfw7UvF94VB%)1lHq>#V1oloXr(E*j#7{lBNk3?h}6u z_~I2Eo%EGR&Kp?tK;h=B+fp>7IDRqM_9~u&`Ja4O{II8%aCFbQmE1N8T+ZtP{;gDY zOSV-W2=hHmN$}%`$bZD5TiH3ajMyHVD&8o%^UiQ?|DZBhhX{A46ztG?CL8H%BL9>?zl9# zGlm8!zKoz&&&Khscx!-?s2WEwj5;R)kC7j6+^@}3a5jEPZ$9uDjiIy z$P;wvTU2Sp@;*zUc>dy}8#wIdX_V_r(8rtYB;=@jX6lVFjcHb_5s&Fjoy~^XC6lic z&(iNrL(vO#do@bS2vE6DC%J<9b=(_!jvDo%?r0V>xx}CIEz9-7%<{76YdHn!0~5GM zVSpmN9yhqKZYFz)!|x`BWxKUFX7wE59ewAASVg}~{D=^bFmBOZR~1z9vWrfE%q1@? z?lWeOHM0;NU%FN_XpbehY%QTMGP;7edZSUg`%_O7$rDPI4<{`EDG6qzsGo65T|X0_ zJ(fEqa_yv0Ub@{Hc}LT>sn}CUkf^@s)T|tE4r(-YHjA_rwD%HoQJ5KbJ7u<71EU5a zowae+yX~DYDHacObkp?>J0%4sLOMuc>?Rsm-mx)W)XN7B&NJAQd~=Wq{kRO_(Bs|mfEBo_;Q59+BIV7Fa-<>W?2#Mb9|09lcs1uPT7rFjTf z!!>EdNe=30wYy?}a${(Ct7F|o_(GLeE5F{GPc_aD) zA6kpkHo8!$^_L`i)MA2S z?k!EOBk|Q-RpZJ@xMDRYDzHkDjd`q_^ zZ~`m0D4}TUx;a&B$Vg)JpU@U3**{d~soql=U1Bxw#GlFfp*lKy8x)r(^8nK|c9?&d z_L4F%WdvUsmo`UT?O$`q@lq4FyZ_9sKSF?t9!q-xnJjSb2}Z&8AvKzJhXEI`e8=&j zGFs&+H`wd?f%7$g$NgD#7{Dp!a0bm;!N??2s9ZiqZh(sd#WSE%oT&_$PUFok^<N>r~5XO$4^%pbmSJ6tp-dRsaKa#&%8>w^nSD8)UaX7_G|g@yk?Mf{1j6sRajnT z?JBK0vkt2@Uy!kGtvxR90A!XOH)*$LGBj2^5$6h}xQ8)gyZR{RZ?43x~UCHszv`&FnYxSs8FdQ^cu91k5`E+&U)hD zrb~QJ+26kMiX1ay@seDJPn1-{kpOZUYF*QxGM~(gvewO!Z^%ICO}e@hhAJxQmpS1% z$cpwqYO2dnlHT9+grk8|tt1oe>c)Ogqu1d%@=N%jhT`M%}5kY?ZH{7*>e{bfD-}cztjJ~IbU0nHDv83d{%Asp`=w#8k@sZI-Is4Ni0|TnaDK zEOmHl#`MP}uH9IV!wLr$l=!T@V#(G%^ONyk3g_RAMTBkq;R}8bsSe-n5akGD)E_Ek zBw{>91NcIunQNLBTstff&(5x!+q<4EXI;X z`83GM%DC*)hO*7Jl3nuRK(K%A+v9OCHjz*K4xY-|1J51o?Y0(lcb8T?l|&9p)xSo} zd&t6TFR=Dk5@Fl!?DHSdAVSx&5_yXs;nP^()QXpaGhH&Zd8a~SM3MjAt?8a1&H+U_ z%x`!HBxi3G20*m_lB0E95k3n2Hveb5gRI z5Dxdrm>ldV@RZ#t@eDaP;5n6f3YUwHU|~MCQc_P(LcvarShEWHvr>AIk|5Y{ZN&M0 zI=41f0x-@gJ6}C%bP%9^HYtUaXpZjCrosAGE7b45XI2-t0ZkY0jP>MZt`B8QU!E96 zlhlXSKt7g1P4S#3JVNoN0ng%y?twRxuRZijS%<^3I3kifVQ82peFpRLRsy-X~PBY1J#xd)+dNVSu3&cq+ zWT(dC%N`5ABFQ@N1KqTSJ9*17l=A1Ya|WIraSZNrG(fP};1|Z}r=Jd?p5Tl&gO1TK z#$7qjteD{QsY(&#NYz`P<)t)Fk`CpFl8B-n3>hH^1yDG(NB-QE?gJ}x8!AFqTMLUG zXA{M+9VNTGa`H~&ro@-dKOIh43uGteKtR$356;{cJ=#Pf)Js5Jx+(c2Qae9$kQm2P)Vx}D*UTleiym%kXsk}^zYgnpx7w9p}_ zcJX5NW||FbZ53QbU-5*2bh0aGQ{;Zg*$>(%tUN4NY=_f#e>w06PHD+(lk{rASH^KxwL%@$fg~$lO=jSU?_=GpRo7wd4%n*_7S{*Bh(br@ZpQ z@5fFE1|{{FM3uM`6s0y~1Z9U!p(7ya-C*Hb+*0~?Oq?W4a-K3B&r1pfk1Oa}si|#* zSt@xe`6<2e1fQ(1DpuVZB%mouf#Djl3>$gV+yPKj1kL@RChjLx=_ZFuTw4>1-I2?U zM;;)1=gzRd9=Ec9^+&MUaj(r>$iJq=8h`extH+B40=eUw%$%v3Cc{L9D^km4aB$sz z>c*+TGQ8SLk|krs4c6{&b+&ucsZ6-(cIQ?cfODUTQ@~PJ82S;59XSolA$iFYNpgGh zJcC=NwwuG=8w^*9BF`<(e7R9o8OUmOn>?xu#I!x`W#hsc48w8_zuHeQewfeT9h?Il z&+4>$)9|?_jNr**^^lvr9Q%jeNMiMnEZ&CEhMkNHIFegy{d5^H4m53TbSZn<<1N$h z0d@m5)>^;Euim?yz^^_R@-wKxx!7}~m|f6IkMG2T+eT?5ESUyA`E@5O=rPfQlL2#d zLhwOz_=11^<)Z|ZtK77NRn_(HD^t=*r*2lhH&7&h&y_3w-ywV{`~MB_|H&%{kIEbY zZdbtSV8#_I#TxhT(|zRj*}eFpjN67ysLmrPbeUsJyG?jz6E<&gh%_QmVGZCe+Z}(F z5Om~88{E|hsurqGm8={;V|we}5n8vqC0(aLYviOqM1z`I_9OVUj5(Haa(=GMg~^gt zqP*N?5=1Q*w~!sNbK9WAWBUmJj>BazxsrZN&s_3~Fx@>|MD%16%w9T+!)00GYZ6MYXo!K(4is)Jf% zNel~tGI_}YcCWrZ%R$IJ)hqJeX}7}jjo+QQ5h(RRze(xt<#9?6y&JiK|k|5O^pe??Kw6 zF*~B3Qtq@=tQH43WcWb)6K4V}h&FO*mZz&nT416GGfZa)UD+ z?#Ie*sh@EC^V$s=7Wr4NaP`eK3o|O1AJ)6yJ3n=pFLNuzJLOMMx!k>Fg9C9bo0;n< zF9M(NAzT4<Hc3-vP3e>@g&6r zWLGRnR7vhuIMu43U=?|mX(Q3iafzVkV}gX?IGpCZv21aNpoYy>DBXepkwzEhmaH36 zt2)Cl^_(k(sPey)C3%V7A&&!h1c}qUr2+5-kDo_Q|AA(+p9ISw&4KpnydcX7RD z+FzptJDeWtaKFzG(kH|=MYHUNy8sZ%Vwao|GNU`_n_-UZG9NHWRw;7qV)&V^=C#77 zrk}|tYWa_{E zt=GkR-<)mryND8i@}IVj|8JJy|K&se*+J2w@#=x9hW@F~T`XCq2@1x6h`CgpTnai& z7>UIMHMa&TC(H;mEbT$2H>bC(t}*)aA|`ILnw$Hb|9TR4^pKR9OCsQz;2vW(X1C^? zHs_aIQ@YFjThFG)-)p~{p3n1uufM>3_#ZTJI!r4uIh<5WRB-WQ$Mc-nPTJ+82aHVg zGURbt31}0@*206PxHbvVgCiN}*<$*?py?dxYB^%n`6nQPPJUUP$TopC2Grn5?t z)RZ;VsE_Ro3fQi^lf$%FS-2+0GPG9du)&&6wbq?Sn3d%lE)gP4(}Y#puGakH2PWNU z#Kz2YEe1}18!M#pEw<^EX;YiLneXJVHh!@B&bmsYq+0y)U2KaX(js`=qT;df(z7Fh zj_tNzQPxar#k0xR#cb;`a^Y2&32V+`dLSvanCuy_eRN@Wxv3Z|(4#zt3uGKS&c8qd zl~egZj+9009kBaDRz9alP3@|0UI_i%vQj|3|A(jND(^x%kEu+y=yKERuv`l^u`vfq z!)Ubx3)!;Dj>@(&>`B)Nf2Vu7aC@KKfje1ttSM)_HYFP$+ZMl(0&kJQrJl2Hx52j^I5 z7WNeip{SULncEb7M88NVWLxwoakwdmny%O4_UU`B5uP`I1&q5#LBLG4_aD}zlv@#Y z?V!i~eg%Yfl8{2pL(M*st?IY+tsqv8Y&U!MHWmfxJZFcUpk2p;7In&tfO?M6`Cwh- z(@C6tK` z+_QY>jxvP2k@*j|b`R~KbkW$Xjg@tjreSr5a-uZk84HCG0Kn1MaVSK2K?%sOf zef32{{OU#)gA(3ue zY@FGmd*~DLS?$VyY1=8*iQSuiqC+)E8P>#V&}l)v$Yil+#_JSM7^x zO9wX+Ydr$O$84q6#@CM2$Ktnir79HbnjXr!S4C44=Ijc4`-p7*<=V$VGjk9+4Udw;XHxHv4R;u!!dK(#AKg7*26Eng3=mye2kw% z4P2Mr>^z3TEwilo1-XzZxM#M@RFNjE*p?Z{`e$KB^zJAmSZNIQKHpJ|i8H(rWF%^; z_7ZrJs$psy%NYpUAf;yzQ!5>15VlIvCoZe+a7NbN`;FS>xy}N)fGJ{c8O1AVZ%`r+ zst9_0Xc_^rRtK;d3}J^T$S!o!hanI)D;h(n>UT^?!s*Ue0Iv%#b^7D7_+g{MoX**B zetJe0CXDkpE;PSA)if@=p2aiP5KF-<77d?V>6gi8v3&v>Xi4Qhl$@bsOX$m1M%I)L ze|jIBW^>ci$-V)V%V$RS|ug9P9 zgQO|2ah@5SoV{J#Jy){wxB5vQ32}zwXC!#N#4ssVLn4QK-N#SLx6$@+cgc&G<@bqQ|DGh^2$-D+`N*clW`;GVPA%JhfOCqpr^!QWT7m4$fGi2d%zfXO z{DND*(lfR(6oH@IZI+(c`nU9Ty8NxqvTy!D z6RYhbYx>&Or>$-Lmaoq{Fny4zdBKfba1_)@>g;}DYJ!PGA2BXg5zDw)iZMMr^&0wR z)Rl%zYOe1d$k(uRI+rn1$OJ>dA3J+l4p^*f`gN<6$Y7qgZcjEv|b9mn( zcFl9Wao-U`>J@d&a60KUNgIh)+3SM6SG%e82+D(@UEX<&top`4I3mbUNK+93{dNyoN^K-^5gi!gFe5~3kyiVB}W06sQXQgF70b2o1eV3FJrA1U!JP|tm zJtm_eBKTjN13XhLK`lbrrIWHWiJ+;#Am(lATyUvH;1&U@@pc} zPPq_&JE%PKI-*09JH`wiiY4#6Gm64Mm!PidAWKPTFi4!y$}H*yc(+z@+h@ZsEcx8c<6J_|6*Ztf(f@VH zc99(u#@ z#91fR@VS<}94x#u^^4?Qrp-8G%+;F#Na|b18cN0dw3i&mSuj`Q~p0Q|R2r#n5&K-B5A&q(7BtlcTV-I9~pZgp^Sr8D00y6L0?R z!8!h;zu9+L^lU&(65 zdyHm`+Q&Xt0|J>tVZPr27P~E!X=08bv0h#dx4c7-XYZU}H^|-aY0gUa?EvV@XS=0f zPu}%vgTgQf;6!cfhBf3AYa18PfRBc1(7YtpQ#+@1XbC*)Dafd|vd`N25AJT?W#acn+Gq;aut_( z--!sp?0CuSY&iy7olFPi@0AQ4hxW=I?Sl(v9nXV|CtF^4y_KhPi2M9+?*4qla^8xU znkYE`;xGM(?s(taXN~jzK9STvEUx;xg?u-(IG@H^N;NcUJG#ZtU*k91d;UDzQn%Dz z&S$W@*yKzt{k4p=yduL4<6tcj-x#SdhXKrYXxtoVlQsxlVrPY@LlK%TsaN2qj}El% z5K?*AwboA`bzr(^7B;ta9MfjR8!gjFwg(CNFYJGtBaDVuzg~K` z@b$gf4b1-X_2V18y*Gj-DRLUrOLRXz*uQt}x^vFHe7SwtGyT!mVn`IO4{{`3VAsQOEw3VJY zmuF7hX>#~j;1+D6n6DJC!h_a9iSsSeJ6 zrp&IxkU)!PuGcmdM{AUNxWV)|HP(3Ol4Mc1;?&4YYEf)BOr*;+_!J#u2P!%;p4s^| zEwt5CATu5l%HpLw6^5?0R6n|YCrnrq#B%25pY|l7aFG6NPFgu4H`#q8(I(>njEfpM ziI@%{sLTBe_RH46fx)4iq6+i^q7lK%Rrr)79{)THy_z;%ShIR;s7tAhHj|y4&dH{d zzWj#y3*Bk6GEe~X(>(HsELn^O8!tUcc1Soh`ImbH{F*#nDfA>M{J^!XgOBes!O|?_ z(MEUkK*W6V@g`aIaLgvbZ90-t0w@JErd5v$U`AIo&bS%{0CVOx_%OC5DP|~z(GuH3 zQpC6%xwpzVL7(*F!6(O}W+9MySPI-5gVB!Zk7mk3(%L=u`y$L}jrE7yrmh#Q73Ys7 zO)o)o198=jb;{cQKXGMihOY@ZYkzhmd^09&e7-tfB&)P%4e}PsqvovZ}NR zgd4+N#(g%pw9Dq9>Vk=z0d!NH}XI7kqen`-8|1+lD3AbCZfCmSb-j&UoJtOA4Opr z1{H(B^-}}x*lC@0lV-jxX((JMpVtrkQMW6X7kcK{S^THXRE{GbSKqIf7np7&2`CbV zY<_4M`w0HQ6N(iLBg@!IU`?>5q-70*YsT>4Z|Mj(W39z*Xs9y~q*}F>s^?miMWfSL zKfq7jYE4Gk6L=7CvHDrW&m6yj9AeF8p+TVDq}@5kTVi(AS;p?fq-CR&SYOPYKDA>! zzSXP=T(^FGs=%V`tULBmcAGd?GKOV-;yG&VJ)iZyz0&2g^V;+k8K*IJFDf4VQso$B za=`wwAnB0{e75Ra&fV6CeR;<7%YY)D##cc%!RoDvc&mggcQ^2tvdcWs0;j|KB}Fr> ztM`=48E7pel2d-T!6275c`f?GEdR^5KQdTh3U)i(5 z=*p7C?PQgAwS;fy{@tU60Aa#TyUo7N$G(!FH{>VT=6;LMdXEvuaJD7co0PmMe&~0m zpF2I%zMp8BN#b)5KR{ZEh2Td*wpm9&&8Gtmm~$BFDrh11T9Eb3I&%Mja5A_6aQAQd z5HAP@J#s+)G_LdzoVh`S&iyc?7n@Y@gV^U5)GecKos@6!LbPi*V}>1Gyi1&Ih*iX&+j$bBpWHS+T2%VM}Il2`^4CYyh z*X|d;f<3(vy0(!kuJ;et>K-?w<<~+WLmEJ|hEn*$97$qBAd`!4NW=R4Z$J%=@8X#4 zyG^k~{PBbS|8}bX;Rf-)i-3Pt~8rx{J^f z#(uPE(`bs^0^K0S+OtFJGR&lGuNyWni%aH38rLx(yD9N)F;}xw<77? z(>47wLD{a8NSmw#=R{mffmVa23la908LgmSg(VlU5+uS>0F zyycP^2GE77m|`{I7na+V$tlOXI=O{D>=LO=3XM(Ltw)2iCRsIsQrQi(P7E9WX|G_{ zpkxqN_c0;d+I3==sL)R}tE=jg_oC6mVJv}OSySU8RdA_pVqYz!9BTqBur_g}Pe_*G zz(S8)k6TnVKsu2?E!rmuY%S~7H1!?E&E}AETCEn^u57R|eVRu5%QPjMqn;7nDc-Jd z?IN~AqCe2qz39rLj*rEQqytA%+7aE=Y2MRgNt z<;`Y6qA(^pO~(zZ@XZyjj3v#jdJE`;zfIiAdO)ysxlktXz>5Bo9xi@MDYHsP1!!6h z=73kuPsuDrl%$Kvr#*#dtvbw`Pf;s6n=-1aiemo?{LY`pD<4Nm?NVm8<9S^UN^aMm zzZ;)(V1>2nbH4Qv>l*|2`J)QE9`O^T&jEA57H&Pn9^|iJHed_e4RsGohv^#PUA`|H z_8n6YTxUoZ_8rx>rWiJ3ARTP)$9sGFl>Y&zcIF3X?TO|I_^oVVK@ z4j=1Tj1zAl#b3KclEC$#g4}KcwIB}6f>hZK9ZgGgL>RUujyr0z-#=|m>dp_U;Or!L z&Jry=ZbgH6f?*`45~E~6ssU9ZNgU!VZCQvv-imq0ZmHekSP`YRgE-F0jFghaOhCax zSMt>&h;U9yVE?DE-e`MOQ|`)VQsAP8%jyM%zqT`qPM;Q5s^#CtV=8EqQ=}-5(xWO! z3)4MJK3tPlD-P;GxpIRF2}ihFLS3)S_tS~heo|mcpK16uRIcOFE?c_+X3hH1k+vei zRFNS*YV{C4!=_t&g3vL-BS)?+gW~~>%MnPedTva?46Ym|S0JJ%N&`_kf=^kJ&#HpA z)*?!vQ__;gBoB@*On-QA`Vy}gf#05ViZY{5qK|EKIg#LdT%K)H27;IPVj;#Ap*zEl z4EC3^$+PlW9Bo*Z1w?TB2`AFbU6E7NHgYn}3bcP{yAGhFCBQEaE=97S*84v=`)TE= ztMr2jhQr?r$;B)Lm13VCuPGDZpb9)OvHZUm!K$uYF?j%|71UvoDA@HpoQ+ZMfT=a5 z^XA|td|2zVUyuIu;Wug2n-Z04F$IIhXC6k{L%3xMt|FL8~I&6r{X)1s-hv1moDYMpo*sYEo*b& z{iJ`EaYZCyKPN$tjlA#|Xl7+Co^fpy1O8Ag9zrZ^l%pGdWKEG}jR|pu`nf6QORYXJ zUd2$F&7cQG+2onN&0T0LLHil#V=V|HaJ4{FuZRDK;Fo~t5zB_7pycxP@^GtUwrjSn zpH@yP(VS5&hYaE&&e38N!4u*=3O4a%PL=WNl`32!LeCJz8@~_;Ziw%|t_6F?LEb6J zXBT9x56YqQxuO-xnzJ-3@g>|(gg_#p4ob%bBv!f|iz?Ac?N=+mwP@mJ?Qx9lJX~Xk zMe(SzA)6|p>1GB;pnKSzDt>&J+-Aj33QJUe+49M0yNqo*1x@%5L4DbE^vGobASb$9 zQz%WWBWL@>8xM@=z`X|wynGRw@W@_RZ5)`n_$V#<*DL0zoA`!XC6p}C&Rufc7x7@9 zeH#%l-Lu+m2?#t z+n%+6rEh2n`VB4F{%@DCqoJ9xwf=vxa25aSQQ67d%JF|PO&XOfzcWoRd?qo@0D|Co zT0+95S{=%Ah+wgBtm-(}xgf&KH`A<|wGEP*a(`2YMfqY%{`O2+7(_6<|L{$?;Tf=k z*@Zw7Upl(YlH79KtiQWkdeZ(uoMPR}aX)B>b7WXR3%N!vfOeSY=K}-+3=Reh)k3me z=SPK9CDAgOGja+uw={a!LBwSveu!F?2-iC9JXAVt>^K&~)3*@$b46W~jd`n~6FPQp z6sD?G|8+hWk=f|WCS|8f-eFx8Hg|zq&`F_QYVE!l0}DJj3A2|e%7(sSUutdG)}{i} zCbn7~#e9qdt2wFF68eiAgHea^et+==O_kavIiVxp#^NBtT+L(E8P_!+SN);&F=G)a zBKJbiOd(TRm5H$^CEq*BLBZS~+iQf4<|-{El2L3e?ZB-_JB{tVE?*KO$&NTmK4Ku2 z-x8;p9&gBcUH|yg71&-X#3mm25;_IRPPV(jufPvw5EEctp4eHbGK%%odXRvUsdf{v zpWy92+6Su_Vrso+XG(qP4j76lDXdck0Y8XAmQ2xiQ{tnb=pcfwK1bDAQFS|@53=PU zKwH^eEE@hg*`EH^!*od(3rL_sw>mLST|bNQ?X|R-J5sSPI!*SKCK{wAacHVZ9EdTS zI0PBRYSg>;+EvB;l@)NVsu5$&Lqo48LKHW?8yP?ajDLx<%t*Ql!fc>Y&rnX*$c5y= zk%z2}bO?h`Zuha(Y~V}lBEC}l!0G;kN9QGaC#d;Z7s)~u8#!;4P@z=MP&boiO_W}8 z<}LNyc4@zJ_R0o>5A#mme{3!afi)A#dykAT-N+`vE#h0CzRQf3gM@wqi&Yy1&251H zz0z{S=5iBV_ls{B7pqyu%^>pn?dXc+3d%00Y^a=ARc4@kqI{sb;G*1FT}JDH+Q5@A z9r*%6@J&sRD>hey?`aDBH)!Sk z4{Jrl+}O(KzYB)so4{^t`W*!H5BRG5-!VY67mHOWx<6v2y&yIaD{TpCLdT&vaV?HX z)Zd*jjg{J?5+?imxAsg2gd#-tOuvgIV~jaxP-r>MGi%QC(mm_W>7vW~>lv(DnOjU? zC@=^+g_+(+Z}m6+Z3l`1MxXR%mcCQ?40S$H{#r+Z)V(IkRSE`kENd(SJ$Z~rhnfzB z>X>BUL|jCt2$ct=UUJ`bxP<@u@--`H5mjZ$&akt(?p;P>TZHkn1v?c=CS2Qm!XytT z*Og~aSo!TvcI7Q_=|H`PX&t>tI@c8(d=_*=-II(B3)BivZ*S@@d~6 zJ~}0GJH4kC;H9TsN}hs;9Ms#0m34Rsd>X#HQ-+uAcT-jWo6Ntm4@4th7b`0?e0-ik zI`ZE4O_Jk9?h<~xdgMi141G6pfXE|>0y`XX0=u|6CX^RL>)#!sOR8qJ6v*ttmI@cz zd9UB0A7?`|g;g$30{9<({i?@aIhITQ^$n|Q#u$SktK?7S2WYA}GwAdpbg` za7OA}4rL|@30gTH)}->HBg16Q9&m=phw0J`F9U^}6D}*V3%q>A7znRF`8NYbv1hc&{qJk(?7P(X|HD%I=W9t?-|k!ZWpb0?P4(|T>Ip)=PtlB7 zYc%wFWY~i=?a}pj-bY?f8JnN4A0H5YYs16GB8(yQWc_x6y#*0J;*b&uh*~J~k(+<- zyNUKDz+$?RuE*PD3wTstR9q&oKrz7r)fzzZF-T8LVyxRU2kUcMu7Cw4w`$BMQX$_vCBKw3NI=Pg*lBGb&wcR;Okl&80G4Q)pcEKxQ_ zBo@yTsc+0@DNArn=p1nTq5e>pJQ(C`cCM|)2CE)|@!2YwQag6N!5G_7Tr81z>NUDu< zx6QC5Zhxu|tIr;RcOT5)Cjjv8(u?T-|qVW64gasn>V)@L9_%{tW+3bcTLpr^9vTE4LqjA@wnf-~;N=tOHf z(ca0&cV_JmPD>yBkz)4EsF|jf#)~)<%Y#nR)7?S-^HmSQbLRiST=mlmM4+P|@}ido zQp7&X1OU?_wi$4V+}06;{03>IG7~F(neQ0y?Vqqre9H5zj;g(dc&~4We``_~Hck8T zhHcPE7R81!q)E4E7g>#04kNGCYaGU*GL>)A`n&w5t%c&FYF9&hn|lZBPl8xnRsW?w z{R-33G*Ba9j1mHg%5DdScQx^1#JFof1;5e2-e=TmUWj;+Rdj&7u zl;?;n!GF$rsqODroFTuE50rJ%=(y&Yv~GW` zuWWbGZ0$`Nsnv^{H63$Al>{~&(Xv?$A@L5}4m$_wI=LZ+@Q3k)wRmgOQRL~JB9Mh% z93`QGbzh6{@$V-cS*Npjil#V!8{FP|3H)hV!Lz+o6DnA93DxiN#DqN^T?<=ITITP8 z1K@U^b#i5?)yY#c&z7|RB~CljfDyD+C&Dy&uyGnmRC~hm5w2Ke%oO1wnHmLMq7SS9 zhrKXsS)q`-_7$v#j@MgYO+P?xSBapnSRudVT0oR&XH!*52~{kMys1{5rjc+zaJ2cWc3ghg!JXT}>H3L}JBKQ>Z4rWGikiL_4 zf1%pZ?~Wgad;u zY0>uEw~y&N=j#8ez5J&}=bymae`B^zRZlym1&` z-$*3(quQEI*U%KwzHckJ2ZLv)Pw}7ET%STS382r3M*j71DoPr2E zN50L82@d3@HuCk`um8y&dZOm8)am=6t|iFC0H*WKkREbWEMs|W_imz8CrHDgG6c` z+ufS=*SJvIwCAMIWN93zZPX5nZ5B)jSR59+@$NHdlgdmdPQ#lGXca-`(|$|ObD>^w z4ECyR%ax5I3Mm=(n~;tAkx+7?+T%e0Cs?CMZV4-zG|7dTL;S|v)$`6LWl(Qa$lP4l z<|V1Eopy6)AN?;p-OboGJ++u@flwe$&y-to$7m;utrIfmFZYzgQR3#nekzVAl-e_K2 zxk&p(@Y>>`U`8U3eRgqX??SnsJh3L?Y-xDG&0?-ia$0e9W$nC6<6;@js#p<;)?{w# z;~Cy%q)L)&cc+F*T#|QKu;F5xzi|}$OARelByNl)O4^CPINk|}jy9c5XH7OIM(Um# zJ2;0L0+NW$AdHeibR2ZQNojdq+2tknrqsR_r&cz{M>Tye*{EUe^KGkrY!Fec)sMj! zE|5}?UZ*o&GZJ>ZJZttu>I?Xq zT(l{sX;reY_Ni#}H2bqMo|@FX6Iq**r01*f8!hb|TD3|cU&sSz=;G^Pnl31UY=lYtsC@6g|YJxy&#>xyKfY3PgP z$>O7enB0aBs?%akbioX_a4dpB#^ns~MmY>)5G-B9IE^InxG>UOdYAjC;??LjM^I-E zZVv2_h7+v(7>VhZ45@+?&cl@Ppj0Yq=P0(w3LT)Q~ zgd{b%L+w?UXm2Ca>eSGV?4_wC#DWgI3;4u^`BC;vCaSfo_aPE~Y`Ji`abptH-*Hp0hip%yx-F&LPMWAaOh#;Q zalqwP*p?h;`~V?PvnQEVYP^@xE9UONtLkV+uz=d=^+x9`lG@b6sB?~(Djjd>_<_6B z7N=NYkdryC>PVGm33X`XU??|=@Cq^Fz3=)Z%8mx#H4g3ggVTzjalQVTK8>CAZY#PY zluGTbu)%|9FKf8&Hh!2+m3$j{7hQos2#O&v4Qnw z(X2l-EqH5o$d@Al3Woyn%YhS{ANo;sN5vA~>cE{UJY4nhPv$!`tBIZp0F$UAqix?K z&Un|(IF|#XF9~_QFZ*3q#yCxba+V3Ntm#YtgR1$RL|fY~3h)7{LxX~hz3`-REX`gl z={-=5D@rss|Mgo9BEqf2-z3`F)giQk<)+Y#?2s^!F` zJ&R&J`)Kui6oZ)yG%~vyPThSucMy@Rz@K(x1brkL>wN@+IL?vh?2|iY*FGP(fP(0# zycffi=%Lg+*=Yq?M#^l1Qy%Fk(-a(tlD>Mo>x`!U->Tsaq@1%!$6}g!g2h%{S)tNi zoC7BOos#lCAowhL*k&x#gHW!FXk+oFq3B{`FQl{34Y!!O;P!4@@n+C1OU@6A%_LQ2fc_;0DSP7aTO7S__aDKFD|{sPst;YfflpBk_iDYL zI>z>1k*wGRTk=_)gNoI2B15GkP?Z-Bs?iNi+U9Srg%?|elEz(G&NIloJjk-o!9u2> z8#&3Uv4n-|ggAfvaW|Pg6zWCd1A-=yu?d>(8DO-WNa zd<>iU_O|;qnhR{TL;w(=INGRQC{wy`3HkY79qlgT^umoAR|JZN|~%S3GS?iFQ3 zEd~1u{Y3Y6g7R>T_lnNE<(=v>7C8%#x>SH;k>4BnbsEbRdk;}oMYx`5K z^k$y`OB*xU0Q2Vw$&}tt_E2A=*Mg`~*`2wx3ztkD_j9BLwGIO^E@zgcs+1SLe;rkS zy|nR$!~ggZK>42|xPO`n`~$&hz__6-Ie$jG42YEjK>-mlg2N80Rm7lyE`u9_iHi|< zHtLBFqdmtaGK+<)wvS#yN8rXUhvK&|U$r&9*=4lE6xf6>82u~?6<6Sj%nZJ>Ln=0p zFA~RXHcwa8t&P9uV|tsFz^JUD>S|?jylOpizIvHzt>=DAC+CC9k;l8%`yq?%Hu#6A z({~euE$cA{%C~$1o!bp<%6&ST&f}WYs}*JF8Wm*9a!Qql{)Xk^rOn^gYzn5Z1z~Ic z($@T`8Ls2FhsaBG=u>8>t9--Pb{n6&1#)Zd@NM-Yn)3hx{8QZcefrPAvpgmE%@nxK ztsbh+dC%B$vcfx+&#wNRE#5oVr(k+E*0Va4&!s>5Upd~t^M`zr*Sa2Gw$`7b>0b+n zs;eD&6XmP#8kmanw%3{#3cRPN|;+YFd1nc6iN zQ={ND7zDgj&e$zui3R4xDrFPuxLjGmk+nepDr5BkPLqprBnf6nEf!1n(Cd{m0+Mo< z2*MTJ%VK{~&!8TIR}@d8;X(U4d(_0X6f-aT?Lz97-tub+b?unN7Q1%*^@uQ{;9tPq zp9ZYWyDGMMLgm=#OC_hy{78q$n7Lfb01%nmyp7HE?4;X5JGge_Z+1IzwW(=9d(R3rorRh3dG)eOrMgqZPGXZK zp0azTh8|g~$%GZ7M{8XcNvl`gFKSd3nLkX5a@{Ny%EH2UTrRD>oYQIedSK~GLWFC& z@ncdz{EJXgGNI4v6bV-V_0m4;SzO$3IJFBxu+|`V>=psx7?XAutoo897PDQIK(Gn) zh{gb#`e-u6Mag_h?1Gt?$Rew_F=?D$ZGe{kXky(;0*za9w`z`}bO%0uj=w`9rDkGJ zkvdR(X!Ne(=uXLP+iC*YL8CsP!-b>Lgp-CivyspIDxfDi`YjHxad4?1E6Ip zk?fQ{OY8}I!gAoz+LOS$LRFIb%RklPZ*`)ynOOu2&;wB_t=S)16Odp8janf=UuLYP zfLMS~10jn3NAa3b_PW`q^l4kVeS%(G=D_KVouC&920WE>9|aKmX^K~P++X4J9M+Fj zSTm-Q$e?_{Ih(x(z{wbo^(V*_^qCZ95~5=}CB%G^=0cKlkVpZgJ|+Vx(f!hQSYk|4 zHkeeXeSJK&4H1w*y6d65kyvj614jvr$8bLee40Z#4(owSYhOaMxzGkBTMhFtu%sh3 z8uxi~c=fv5Ai@LpSqYpO#tmQnCISEGA!ed#4bY(J&q zW#XF&Mg@l%{Ro@P#}t!7ZcLpInjGsyaOmoZ8?2D1NZwCR53(FkY*R-A`h6dPw=+kw z-5qy}#0erj2iBc3MpUlgi=al!A}$*$e@BN^H9gcMjqXzvkc&||ztu_29~0;NS5$V; zmc$l`4+|`fNAVin>J3h=s-mW%vZ8c7Pepg$>M1wo4p^cOx6|_*bB)D~z ze8sAx7gxuZ_Fcs%WR@_R#8PbPTej@g&BdB~yN6X(hrDQ>nG@}q`QE07Ye9W%4}_I$ z7yvoF4c0pLf*mWAIEaN$eFj+C=7yM(*8W zIlF)VrggyS#E)FOaPct3z-8xo<4!J3m=7Mx&bb=)?65jot9jOgw(LY2$nr-Ff(yp# zEr0S#x2tB8Q`HoRt2O_$kmLm3;F<7O>FvEIuP%*OcrI6t)kEzRLE4gI=0s_loqr7- zX&#f|>Y+bL&JGTIMhPqa#lHrsLQgw7Ec+vh{QkJR2d((nKuWg$<#Cmo6ByP#-*}1E zSJTl?mIv3Lx7~K~QQK4Fow==L<8Qe-mjK3K&)iJ=nSf{nYWourV zb7GY)Ey`q(Y)+Km?hGOftci#q z2QM_|=c@#^SNg14oZ~RJ0- zoV!lqX?vp!YSxsvq`?V+w!a2IcL%^cITqqn@biJ^_#PmB!1uWX+G4w*Z-#dk6UbiY zLjG)e9pjLLV0$DOC7&bgrPv(xEX~qXgr%*e^p9hrIOPIc5jY7c(`+ax)BUco@7IH}O&@}G^z>^oc`F=RdCfUy{=OPGgg$ZKY*FX zW*nk1Bri>bA4)4>0QF*P&oza97PvbfSh#R`Y9OONNQr7%(-B>!-TQI=YDQcN>{arF6D$v$o(lUKHc{2V#dHhCVDo#=}DiDhyNm_`I^E9 z;;Lu~fA3(rGkEV#{{2@HmfKZW+MudFmfKcAkOv>{_BnCu$|Z;n-jKwVdWA$DS*Rp7 zZ`JRt`lp0T^9}EA>5j`I=ZGawckh@b{Oebo>Zxc$XQX6O`8UL*!tAAtpys;C#QxP_ zInJn8QKO!$E8n@rj2{WXGwyb;`?pB*{kq*=>B=e3Y^3a(Slu8F<19z$%~1 zrjQU}SN58oxBs#4{RzX{3rBZg&ke}eAKjjjd|9{XMAS`HmQHwpYPqRxp!=%FT>D*; zZ^jkD!H`{kIR!WY34A*dc{u})hunlOl{haDny25>Wg%g%Pf44@JL7J>gUPS?xwBZn z=t|2IOEbu^?Afkyn>r2H`9ZwnJR`6GnZAZ@4VfoJ0P+DI15lN3=ayhf+l{P~kUOl#+JV#?rg!I^4#lW(_T-&ZQ? zk=pGR>no_szu)&GWj6OHL!(i=wK&p*c!QCYdJ=14Vv(EqxC^ifk~Q;*r+K^<2bnjZ zf;aqNJ?7yVyP|InGd`&}W=uH9l99|hkN9g`#nyYYAOgr_)lb=7^4bl0C#%pAWk)Ax z7LV^|)g^nk1ZT<38mFu*wU7Q z{IPIJ!Wj?gRPa9Bc*m@_XYGs*Ec#2~ z8*i`BH})dh3Zh(o2L-wpgq<)L6`Ql?5VP->QJz}srby6A!19jH9EFT1hl*&d&T-F& zizFwnyhg^zMyirZy4Ut;@$pq?`Am!)kAIR#n`3w^b)eZarBBw`Ozgi5mMcAT2VY|F zcC|l&RNC-5FI|rpq6_t)sy2=9heJNTqYAl{Hqdp^|8-IGLg;!H9U?%d&*Wourg$Kh z=c$xEF~1kjyt%S6Z?M@R&PitfOIeP2#@GU?-YNbv5ZnFxa33nRkZ#>(@2hnoB9C*X zk;ov+;CNeu5FXr(^or z8F%A4IQh+z+@AfpAx7cZuY*7bbP$v< zmgoECZmqA(G;Hw`m2^G_U$CJqf^=oy;l9kVaOB3N-NylJh^oSa3zF2%Zz3DJ4 z+FKjXs89UBF|<)aY}+~wym!(ZQ-qO^!B32jS8Hj(6Iw}}>K9t#5VQw5) zb#7o+ymW8RWWs%2&PSk1kvB@g&1h(ADBiFxa;8GPa<2Ohd(oVqv*r*P9hNxLH}`JO zVhAxEVGb@%{b^aYDm~`3NST64MV?bDE08lMgVifO$Jlhl1S0YX?@cg~)n@GyV02cH zVO?F{w(XZfZ_NiLz#-6cOJySKKc$tY=Mo#g5f!_D%QaT($7F=$I5xznS#!)5PYfGp z1NKm8CK62kQ`iHSr#85lU@d|ll~3WuFqQi1*Rqn#>Ox@9cB=$F9xFBD99Fvlj6tWb zSet`RT$r5eZpWBgK~peU2dJ9f4uenLr!$CS+XtJQdzdv?`yR9rWSS=RyEN zt!~fy7!D{M(o*0+i=+t4d2h9tf|9H$91jN;i%1AhF(C}v-$7gEceKBA*Pi-JDo~2O zlSWpOy_C0(%E5aWsruxn%krvJ1;sOr^YU|HXqnNVd3qc70|#f9ESb3q*--KZ7U7I| z<|6@>y9ijHA$;Z-z1iflwe00AutbGBbTtDB4HTbcJGQ`*J z)vEY50-16OOQgQAMU~CT9dxt*E|%Zm_OA)J7$cL*-$uNxFoT{Xo<%c*^zs{4pMPAMz!(~L zg!5<6nnSblz_t*y19s+>(Io;*8K6Z@RHZWkUI7H*ZLQRUc9m)eGP-u)aL6e~Lo!X} z1Gg`{=U&6bIGs+{C>wBkRAX={Be!HtN%c(ua;m$W4MYHC5`7l^&4{U(1|NkaB6A4w zr8~8uEu!XK3ULk*g{GNaPKD{{i{l;g;Y9KC?u!PjAs??awD$ml`(5~#_?;F}H{6@` zP+p|g_)aX(^;)=!je%DKP}&(Ge&N4i?Wm~PV6=L*VN=@OzfiNk$fFl{66ov(XuW25 zki7ac2ckUFNHbwx&#JJ8dL1W>>P2hjcJ19eG1B*Mk&SAEsG&(2Mw8fr1!NX-GWCEF#>;B5x9NuEz zvO2mmEf0AD4$v^+ct1fbz<%#`Z4T9FDcD#iH>g}qQn%<(;iCD#_T0ap5^yt6&CUy5 z@^JtP4O?>8wep6w3iN1qQ>nl^+JCTL0{tJ(-toD!AW+wxnb@{}v2EM7ZQIGjwr$%^ zCbn(cc5-vh-nZ`AyXx!@w^sFEuvS<1>VDqmMJKRdfd4}wu$3ZAr~>%22V7*QD}_4#G1~=x$&+RFs%WLu-Q+(@pnY2z*y6F zdFBg$e+K*67l=&{)a@}YK;#T#gmY;LK5mZ1fA!3D7x zG}UtQ;nvQY^jr@MhJvOUIPWVfV|J7bh&Q3K@h^j96nLgxpBbuS4%%NH0Iu5so`RM5 z7=9)9yAx(pX+w-0P+Rr=!;QL3GiniZveSPuP z2*@L%1mb=Rf310x)`FRsi@=BfBI%PR6UZlg!HX~(1ZL8^6np#e@aN>p&X6%mm!!Ge zb=WTOW?kr5GR&B8C`vLOpImHqxIS-nz1?4LfBm|F=oaT6+GHAAQK-$=BaJdwLcc0Y z6`$YneyUZPV7M+%HLNaAHLWfiq34y0GxaIW|EHx|_s^wcGbde1<~*#96}6)8i~lLak{nD|wg4F+()le!;23X2u31WMGIqO-9SG zah;w_%#)CjWPO`wwPc~PS}B3PRc}E8w^?;*tzrCD zi}fhsoFOvdvFSwCrG3x8-KlyuT$=7&Pr2$nzec%^Xv2%aG=AX7dSajD>hS6pFjDkH zaJf->7^2hU>agRCdBx9%DlAmvO}VQx#o?0EA+i%p-GAlsQL8)w*S?jyXw%-?y?*PS z=0?r#B+G^SzRaO?n2z_h=?tN|l^{agBiqm+%5y0&_N1A&e(6l602l^3(_joGz0OS2 zg!Lq~3^?qOfhf9kshNLh_L};bNQFi?}||xOYcpRXL%Mo&Pp6GW3T64mCB*j&SEW(wcZpg`%e&=tRF)m>8$Wns_6^ zgQ_AT6!xu=Ino!7*A-Qvxoc#XChYQK1apkM_8kVXq7US7|c zjZCG)JN@4f$qDA_;V}pr^$A1*TVs(8l)$wS_VVEk%i;G0wC3J@Qi9qMIdxFZfV!^P z2^I>o5zo@)mLD7CSOeeWroowuEPE1Ugq*~2*ae1Mf(av6WElENFUb&S)}k@>H(Z<; zv`GN#$^Zr#_#%K7VLoF!e_|M&aZljG-X?#QL7-i_9cY46;r&Z!GZ|EZMwx{5Ln77i z0?)Sm`3Il%Jz>*?Y~yU0o1wH7B+V?B1xPrBAr>nu$<}`p5|z!Sy;LCl`c*~s>le-c z#G(Fw@m@t^2ciF*x?LJjdP<9FUtbeaS9EQ=2;lm1Mv1@Nr5I4*iSZE;0kcH-{P12f z46LL7jBGduKfc~&d4cr&I!^+^SV?msc<=Xk!*6n4XlZ?} z`e|<@7hjXmzNML|AieewTP)l=6;6<(JB{#>m~PD#3d64YwVL`84RWLs z#V^K;YNX_^5%I>1E#jmS<{%if#})z+YbNB7dq^0W*#||3O5WT?)On~9=NAlFf>M5g z#LqLc7l1o6h)zekY)~baGJ&e zbHgGzT7V3u)ymPLKdR&D_aMa7Ci$F+kh!*r69Z>@c{PHCj=hsm!L7q{CdJf~s zH_djXyeLsY8u_Ehq*7)vF2nt0Ux5T<(-=A?&xX5huBF^bp+g-hG(d=?*iPj)*@u)M1UcGx$yD)TN=i~CDsQ0d#Tj*dEnmF{X_n;H%>(i z8UHz%CX)JM!-&~5i)hmK7X~DM;Q}_7V8z|kRh;k}9F0wQnJPw(Nueu>`V0+R8HZ`5 zg-BAYf>ArsiBNE(hFm*!CR}Oik~>NYR`3lNlR(VO1;i*U>(4G65reLDEA=8q>~(k< zA?0m?c13Ra@@fk%fhMdlk%0Uc741#)2le>c0+yIy!&%Fb%g%7FvoofB?L1JP{LRm> zT8(C=aXCvQlN^rZI4ukD8WX6as2I5E%(GC4;$zbh#+biTd2nRHk-rs9iWzoBCib3` zS{*GdTc_ueNm3vLP()Rf;|p~(4KG#Zc@*GI{daCF384u=C$m&8`no)s_Hz4uZ!sbSSliB>1f~`X-l@urgU+PKP3I%`ZiPSAb6}=DN zzua)4+}ck*^Odesh|IK7hJM?%0X-!a+#_MiajWJTAUm#y$wH-8l= z_1s+|I#VGADig>1;ZS+l3Rv$(ck{(kcg-)3vD$qx^&vz{q*ZuR9mE^q@oFJEX(u$_5Ec#MO z(=vM5?j^%h)2=&n2-~JudjOQ8*S-<7>~zt)-|i*fT{teYnzt`n-XTd|`W0A^AFX-| zq}^TFTE#YNRANXA3WBo#`3r>#Mo5YfBQ&Gz-%@KEcWoDGrh-1c0cf+RBKPl(Czw=i zzpvJGc(_vdd215T#5UWcj@Mo!a1A}Vz#2JC2RI!hDGUhUbEII30jD>CfOITvfSGVrmdDG@h`y%t9NILVwv0qIRuT5I@QTpZ0{kRp< zX-%_iNoAeRV!_Vt8rH44-IIG%<5iv$z=XRPgkElyerrayY^aLomKg1xlk|U>O4xlx ze^Q2%@mBc8CwG0M_jXvC z^yFW{mA^%*`Hl_~0NWrE`V*n9B_5TQWIod#@!oZ26yM!RDMyBM?4&UFQbD_g))8$e zDMtt{R(b6wY_=aXM;cd*38FFy@zMjuXN7+9?nW??)flbc@%HBIJzJaN=U>LR_PBZE zl{+Gm((+B~rTKg_I&l>R8((#29d#*%>}}>=tf?5}vPFranZzfPuCvV9P|IU*E>#`} zoxDd1(YR$yR!!##tKexWam$1s>ZM~Ki{+iU96pL}Z6Wu>u_(Sc;Y0k+k7d|pHPfLD zPGuM=oSYzOdpQjAfQr_c9U=wTNG4jgS=)>|rkbMREW((9Fi{)bo~ym!NB>1q*;c@C zezQVJv^~w?5G2uKotg@N^VCT`y#&SG0HyDcX;s>&2%u&w%>B^4+w8v9PL-*`T6>e? zE~P%<`l~FmK-#@JyOjo;fT~!P=@?D|X{q{$*woqX)JJi(Ft;h;(XM*5bo|St(ca0I z&JU3aO3MT~6iPRby9OWqc%00^RKPtl-~;Wv^2|S2aEGgpIa#=NS^ypEtY7pPm|#7$ zNKWZ&eZ;=6V%n%`s#FHNM{c#;V06^p?L)&xx{Wd*pOmOy&b8S5Mpe>8Ve-IG5HmaM zM9Ik+v(87z*%#+)seddwUG>Z6MYgc#E3aTBCn z?Qo|h(30HDP*je{T=4?%D2|W8+N-XdW>@DA@z206xll!8KU%)ln zxglEi3cP$Hdu$ZOUf=D#`b>zs3f|y$~e=XhtgUPc&PX_6Qxxyo@`4jVW zpKUM)kgWxgE5GO2u7&J%rM6-vR9Hxmz!==Y7uR099?Oug!gP&Y|#hgqh>{U$~WTq+-1C4C1EN939=pjlG%(<4`;9hFS#7F zDuPKVib_hIIJU4T!eE^7Uo{)_suZTC$@f_k-yB&YZbi(P4WKj&r7TOSxRG|+Sw9&V z4^`99GWtoa=396p=F~I2L-Iu7EbLox1y1iYY`F4t2d}R5d%%~#?M8NzHePBAsN`Ml~kUtz#RDCSAx#351bt1SGpFwSv>MceGSoWb>QNrQ+8ziXK2xod2?uPZb>q1pbk{J5ivm`Q9`i0@DxpFPA zW(QZ|0K9`uQZ1()2q4cs1=>w&XGD3~*xHm{pS7uS&%jkLy31qc=Yls?<_M#K0VnBSl&w!i{_pn!OaOgFb?s2zg1OJ6E?c0nvrbRP!u~Vtz80Fz5TSE5D>Esl$>fz;4P11e8fR7;aU@H;fx>bNEQmr|@p)CYhc|*sB5&>VtsmAVq&EDaooYf_ zI02~b{OIMOLu!= z;EK(jZYK;wZ^vektgh;UKXUcR;#rG(Z@=>kby^dXt^sw4qC-2JVznTp(%=;4PJ>Uc z7~X%LWJ@=n0cu>Joxc5y-4aNe6tr#`hKZ_(5-nIzGnkD`l9Rdulgp9#9>y}@7$`hb zHyxI+dWsyO(1r~jS{i?&R5IPNII1yj15iW_QEocstt_l;M-3iCPzz9o2gTZ!5zv{j z(FhM?*dwD)(wPgi8c2qj8B5oL)xEwl)!slh4^jq(Yff`}@&dk4P4}p&hSwtlZD-ki zhg%;#^RMBk2DkL5z6k}tIV1gi4Vc~;aZYIm#`!Yc4`HkZ9|%`fW_wu9g#7++O+Ek*1Oovtj=O%R-3b)CSQ%omd;#e4n5(q zlHb?7kFv!Pl8Rz{ur060HYf6*R$SF4F{qd7qe}DFAQ$eEGOf%AtIhNeoEVy9AXSu_ zFlZ}<6hz5!HQKv}CFP-XmhnI4R)+nWhqRyM87Y@X|I~4`$dM~9^o4AAHj?NR5Kf$- zAA7~ycMg%R>L21>te&lJqZ-@Wb7@_1d4({yrMp;jf#BeRBuWv~Mh*EB5`gwSRz9uM zHi!9JrcSHDHN`NWR9>VHeord{S3uG3%F4EobzOcvv9(YIeEPxG&f zih*Fz*>^`4Nsij?_kV-+znq+{Xn#r`TYqo{%Kz!u=RaCr{J zGKd2w(X3f$Hd$y|F0L(UG%cxEYBr(161 z4C+(x)>Vkt!+cX;{=)g`Hu$fi{LB*6rzX6+dW5%>`MY!`2U-nui|)qM&vbO}ZPh>T zUUhJ3pgg`#$x_*nB|)rd30&f+P!VTUjiQ4oc|!e$Ue7YO zQo1b}m6Fh|s*>=YIZR1~E-%!bDqE)IJXsP;LEX5*-vR16-H2n6BJ~Ah;hudxWFxCl z!Dt?tFe7@vqnx5*k)qkI%?hCIY*CrzR2}_TC`Q6THnPLha33~%3hau(@;BY0-TAVe z8ikyah58oc9#?syrn$%Z%`m!3o;ZRSLoY8riOIxp($vu~Bg>B!V=>Z||7M_RQpJ;{ zS8R8KFt|h)=8v+AgP|gF}^MuHsU=vA}Pa6QNDbR z=LJ+mnkr*y1cB!ExSrbiyuN)cYs%QuJfdj#AYHAV?ZZ=$oB@7$%}nW%{z9+evA-M% zduxs>MgL{L2O$nwb)XeZrNg5O$v?D#(-UA28d*9h?C?i z+*SGm=9<0~Rz`mdw}<*I}FZg+;>l~_howe7x8yW{P+5R?@gQl zUd9XXxAKW1(sw^k&{IQ!uJAxnY=#AH?bv8tG|O6qL-_E3dT81n3f`sr--AFYl4cod z>u`dvY;wr?+FI5>=4S!Yaw(ET`x>Bqk_t;}E*wkYDV9y-%b66c#Q6M|0*NJ^#95Mq z<49$9(NvbxXXeGJ3i$#!1|#^2pyTlE6N!0y2-pf2zDVMj(!pgPcw%K?HM{2%>G^ha z#8k$?(5%LRLMJYYB6BPZ5+Z2kYgiV=k#V!d*yZ>QyR3iFIRjT0@WEhSTMi zQW!Nf>NH4dA+S=5r1SLYwty_6kkMV?-;AtTin?P&Z_*S-4hX~TP8{Zg3dKUoTjUH) zp#>&s0ol+WHe}fBk8MT@lJbLN9MsbN>`%Pd&Ln1vq?JaQ!o^nqkiwO& zR3hH7EFZwNSMmd4yy2?-SuL#)j_BbjjvljAT%R(7n!l~uJ(3&3-@2Wiw50h-hSOLM zwM2XdH@Xw2h{Oe(nX#AG8Yz`pQM}Ee8Z=fZ&caE|25{@FWY@8n%GIL0PcPWHdrKSF z`kp2)!f74YVBFd}3~yU4n0~PrJrAy}S0O`CP=em5@hau9*saeF^02$DMI%)9Qg)=f|Q!?O2xZd1ta-XvLZ% ztjU5DT0~YR&KSx$nT)Lm-0_H(&!V2I=~WRoks8Ad2A>#T8aCz_(0hLXLoSQ>=0Ag; zHU%vgVxBOmM1VY8Dd#@Xp)pNsFttLl9f!+{fxspF14_!}}Si`Xy< zarh_d6upu3g&W%Qa8n5C`Q~(HhT1>_+-b4Wu7M0|zmGkE?sNhNw8a!0thWlYrAYetz}c>VPP-cgqR}3?|~G;6Xn9$cFnT4B=`!FYZ}$)okTPA5~Z2ninT7Oo_hjSe?DcEs+mZWdR^1X z-loj6`j9Px%RTV=HU~3n4JIbS7XSLLOwc=v&gCN5*PZka6h(nMFy%eCb1zYS;D}&3 zHv-&j03e$S)Kz~3_KHzyCP*JLi2_-rR4Q+m6MM3qP9QH-fY5aWHso2#cBNxj_cB7n zs~H$UcRH{Rs~|ZfN+z=Uslybs&#$%)BrcS9Er@=t-M$oSoCVnE{ELQ4YWIKx|{RmU|oceH_)&aZ1~=r4Q%>W z`I!OF(e?)B85i|Vr5PMA58=I5%}WjNJCj7vtvhB2k{z4Um}FQZlVEt4iXDPpJ&=SQ zXL4?c9z=4AKLbY!ORi6|ADt-= zMd)~4i0}=5_UjBo$1+WyKfH~@*A4r_&nnRq(^k=LJAbw7Z z1I~;|<09?i#iE_!#V|98;@5vhNhVSL@Fh`S`N>YxX6jjAVppk_+-j6Lw4f~0#i>XV zlnu;HZ3sPj6|`rEePF~cuXli|p`EFdU8m_`q=hY0-zy==6Gu=_;f=h%MGncZR2~o| zays!!#Ok65lY!H|*<{Hni<)(S=5x=SGU4t3~crxyzocNDk%O;lrXG5;dcelY;O}tJRh9+KY}n1 zY#A`qhm+~CljV#0R1>bods;zW7rOHL*1$S1VnR>G{a2UhXS(?7!B@A~2JOfuQgmzD z9f8j9h!?94bUX&M=Y2%`T?PAn28dk;i~!05IjzCSEwTptChRE7LtrgZs9JHs)Vm!STuSo7Y2;7#60S}R{#h|sFqDJG1#|ZU)n%IoXN*Bc| zX_sQp*7ngUA6gx~FIIU&*xiPJTzKcHp{}%lG2SA*peCaSRg3Nu+7FSRzozA!RKxj<> zGW^>%Bhd#wmn}K~2b;4Ax(o@6t@7W771`H(!VtG$u-yw262V3`Y+yES;Zq@*%&w`!suc}T@BX0rOKCyN&*qQK9jBk4{70WUVy! zI1zWu*d84>gVAgQSf&GW8;=+-Nt;!Q`%`@#^+Dky>D7B;Vp>g5p8h}mbLOGws9VmQ z&@a{CoX};01|4JUrfgV?gp{C}7Eu=o)dl+Ve7`lr?~?rh5}3^}7rA|Q+|}jDhK`CZ zL0D@lQz%pNdSIQYXuDWDil>&Pm@9+VpcZs63-a`KU`huQrPi}!?!zp`@N&T>Vo7r9 zm*y}1d_eEl@sio`c#9LvxCzDnJb4H0 z(_s*zZ{8=8g{dm`Ti{7aYi$x=J1ob=&3?O zC?AF)IDy>+GB}a=NO0my?;ota>ev&N|LjnbHj<4$<%as` zv?4bfIMqX5{+)f2?J@H0c7lFoJMI|M>)g z7-bllq$(3@H4!~6!xi8Xsf3q6`OTNpuY@Ophc|n$Z-h5bnrdE&y`Ny3y2;s#9h5y9 zS@mS^Vw>6vV6sSw?Q#w$|3RHve0^`oEN`|F3a1 zNpaogr=E#BqtM;}8*_|0@ejX!22>*|{?2cuGRWxA^hSkwC%a2Tn?>iq>Y6q)G#e5Z3R;5 z{n{`j{e%tADg)8WhlYy5f(bEg`)$R;U<%Qb1tZuUyJA9dv3{%JESagl1kY+GV?D24 z0PwdQg6%5XrXx&;|BO?wL;r22N50v#j=%E=qjJpce(}Hs?fLqXKjZW(9H-{kjUV^U zti^YQ?KcIU!(GrN?lJ}o&>l@;y)_es!2Zj9J|J@1NS4D4N|DCB9_-UqWKddz#nu%8 zpu23YBmlvTW2!(a$us8e`}=WVf?nHY{Y=O1LM z*_zV+x9{se(@NR6@i2cswY-os) z(o0y&x!%8;$a0+KzV@8zVvX^3+79?7msjcBZ+l|~%Hz-LxLZKuwiFe!JH0s*(POu} zPvhmkouhP{Vq>>2kCNjM5ZtqOw7kJJx!Gec$jei(*Ff>5GPvEg%fxlocl`3~&T6y| z_3@C8HMA!g`SMKhF=d2k2nsMkn}UubQ3|%Fcj9<%Q;v(HVWJ-+ z7U>WosUxe`GKiEk>d-GjA19RAzh*^s<>cF!?sK2yhpDK~QpW43A4s&o!>6k8UGRU=K`$XBmC7H4f8 z5MkA#m~!q15>OU~>ap58;$qbzWO2NQu$giVqn6qM`IxK-S4lB*?2j_y%-3cr6EKbN z(nP_*s8w}xm_J110o@uC*iI(+_;Ba+3;V^|>(sYY75;-tJA~C@!G?!oJta{0`pg!D z>?}&RJHdksfH}I z_s(r@VeGRgs#OPJ!2jkhg8cC)UAu+}d8G4K=rI8iaPg*FP{)08omkh;IS8bT>yj)TWB*_p3N<1c_y|Mx+xDW?w?ql1b96eSIMW z3i%aXagPW&Ii=iL-7VIdC|1M*fVXNf?cw%pzJF*Zl`tSJwD--{^FWv2gTfFeotM3O zuCFe(u41;==JVr*{S_wGfT3nUA=h7{8RR^7%+5^`uv7gkw!3$ld{bHZPnBy}+csLY zEu$v}xkaji^=lZXWL9ATA=@NoJ88;E-a*V3D_0?1Oh|Xvn$s|+XCZKnXB*QKy>pd) z5Q1(0;$9C;A_(NL_{>%eOP8k8O$1#!zU@SOS=Om+Dad1R8>q~O_}S{=I@SU+I=tIT zjD;Dh1h580v`|ue14OVe-6T7rfhPmPMriICk*7hkuugm7W9o(^>NohzD$rOqTMP{H zJgud9zgSN2%y!O^6f=bQ50c!L-IO|gtj`)1d4g0K?mkeB^ZCz9X(OKXuE%aW;Jo4! zF029rSL*F*(av2ly25!;UH6TVQYO!lwQ&rX1_2>eXgs90WL9Jfp;XR43}&^ua5H+l z1te7gym4G}0v@fyvYo~-B#v77uGd%59&XNv7XVFF+mh|O+RQF4@l53iaaKbmdqQ^p zgc`=3sE0uC>*^T6ZbAZ3eGPerZx#go!b{E$u|cxPH(au9NQ{yTV90 z1W*_AZAM&`5)ABsi{vaF6`)EQAd5AFy?@wLl_Uk9gRK4x%L=-2hRWc2dCEow5A}Wy!2nx zZQAB2_K=6lc?Y@dz7~@Rma`f1lut7vQIdSD)Q7fWZO^iabB0sAuv5`=wg1Jscq^uAVfJk!*@ zsSrfkcKQAs=oRt}Plu&In} zQ4#6x6Y+Z?BtZqZ|Oi18Y^ z)xuiDJ_|!MX#?5{TErrN+wbWG2d8TMZ^|#Dj!{E!BPX^nT+>ul#xZ{H&g(w;KFL{K zs4?%t6H9A`Vj^8haiYu*B`n;gPG|GnF7|W^!ilC$X-tMo_4CI%siDRe4Yop0q-n!x ztVr>}2n3B_-XQHU*3yswA=o;imMSv@Z7fKIA^6+dV)+{-7Hh&H#mG<@!#8aQh73dN z;>D`4mdNlpwc4YI46D+1Bk(ykdw_Ug6^w|ukrbQCUY8bJOID}FK4wYHeQw7x&1o{6;jJnzw)b1{a~OCJwoagGlk zJ?io>mzW<2Ix2%W=_e@Lc|udEO_7m~nrSuK zDBFtzS$m%8E41Cyre%4!0x64^=T!HvpWIq!LZQEdUw>1RlX4pnwZVCpiW!CCS?v97 zl|*vwA5U);{SXU40#Ec@&14kuL}Ek?ybXxi^7?YonR2Lf6bq)B5zNyot>;5}4(E}^ zM*d7^6M~Vq&mjRF_ym6?<euK3ggZF4|k>+MG{+ftjSqMtE*$ z%`Unl^T}bTm9}Mt%9ttO$EnGy2|H_I0@DVg2rFLgtt4N(qs(E4K@hb2+=P zm)2q)sr=5!fEy_QH6(quD3+W4_4DF*_t2^5%PzOg_SoQF#WLkkW{`=hDE+a5u)vsC z{MMBR%gD_)3LN(LNV0;|=s(&ulsAM*;`7MU8N!X6%Mr2kJ*ranoILr7)OO%J)}bOX zd;U?CL$&9-iCRh&#}|9H3bLA`qAQ-6vQTQDPs_CW{fi# zXEJ2>y@`*jYxu7{!rI2C>y)<5a6sIgCBqSQPF4;^+Fv$ zr781&c&hh8*6WrRG7-zBW8z-(U7Zd*&pNa2_8O)ZD&nK3p zJ8&=ydE^yKshXILNfm|Oxl8_yY>ix{DIh9{gbQbKL)w~4R1XK!myl|6_Sw3dI)|1kEH?Y6Q%j2>v6@KR*bx~)EzK^NCcBGhOIlQteojbL2K)4~4#Y?ZV%;bqI&uPrq@|$O3QpXR zxr1D(D?hxX`A-xN^8+@!U__74?ra(!^&5nw4J3GKt>|!kv0F-cY9SG1k0-ECsop7Y zUa_@NgoLFq80Hp%LR!64YIXNzy&ErK}!Md(yL7DpvaEXZ7#R8Zw5X>_bL8hxiCV<8I*N%(_UPF*(S1$rA<2*A7~ zJs&2#=mI^8mE|>BGLEHzso%RWScj#pgVfFeUClohf)(sd0? zyd;HGacutd;UQJW+7z>G^S`N)rD#8(r)i`?FXl>-xL}b^5WTP~14A#aw z%cUyK1{+DmpZ~ewBF*yUg&JG4jXAmHh1$k8592$ZD><{!`VXP-1Vh!NpH5PO`dwo| zg%h4MXi_4(2A9+Jqo0R4N>gRI5j~wT6_)}I?@T2F!}LxHcw_Lr;R8FWnYn~(LhABK z>y<8q=ze3PzL`YE7lULpcyLY0_362CBr|1cIiZ57+PsoKRdc0r9?g8e3y-Bj07y8` zFTa*h3tdniU65d^AV~-&xK!*(q4-iwwZVzqc_C9%v8y|+mV>prVqcflcg3Jv3HBS{ zXCFJ|!r*=4eHYlJr=M@1BH6}PxT!2GNf?09WjdAkf{#da?ERl3>Q?@=fYcX`tk z=R=a?R1PC(Rk5l~PXF~lP^ZLljen!AfTWr5I%O`bExIwDeli}JFpb*v#ZO|!o1*Vz zhKHY6Hz93{YYB}sz>uUatdJtn2l#s$D!y+nxK*C(v_sc}EMg^x;SH@YMTTBS%I zg+}eXkzhCv&}oavPKhNP^xj+eduA9g+F+%&#x&5%|59$%;E~2#InVjg?j)7a9r?Gr zLFTIqU32R!pTAXso3R0+f69(E37?31op-iwT+_6;*(M&e@nq4u|6wdbp$*aqCbuj+ zAlZjroBi9 z{(W1#16@glKM77Hgkd}>Tm@Ep0A_jc1?8k#L45thTAskqby31{bZ6U<+{)v{DI|_PV?Hw^xQSPA zpd6D3N?g?p0Lp8lRX7deh+){RsqmOeR`lc6UyM$S-F z2T=bb-gdEM&8n6tF6WjwH&J1B@7m#s`uB024C_M0%M&?FW`Aq!RLIX8tY30^f3CCU z<|($Mink$d0G+_#Jtd~UeeuN(fXpqvg}5F4;Qs#3%ntbLDjv(LcwVgg*YTg<_%4da z95nt5RHsDG=?!M54dA4Z{8cmT7#{r2oOpc}OvYx;=I@5p;)Tm2KZo-sxQua+VGmK_ zF$T;B*vPQ}g4G{5tLTgh;4etuf^AkJDi5I4~l3nV<9*W1i+*K(Io|kw*;?p8*Z(?% z%E5~NdEUT{%Hn(=ZLz+}^Ox$Ne1Uuyqk-Z5n>WJy7ee9+m%34PgAKw;&k$D_JYeV( zQG!Q;6F53Bfqc}E>!+5PKC;fFxg~XXqNCdU>41|a^vODVybDchzEd=G3rQAv z*eqM50Z5Bqpe@{p3 zAUFuUXMn2}h8dl2KQXxfFz!%}{x=_WIsi;xrK=?Gpm%W0OElVWjwe+ue!sNT3H6x$ z-EX5&?cWeLEaDSoyhTW%2e$a19gY8ovUlLNwB3>gbKACU+qP}nwr$(CZQHi(-1g3m zo9CXY?*3|w8a>W$So3*jtcaKqiKi~ctZPUvYYD}6iOIC2zmo)p7Sbi;F_8^XV+kag zB0j7OH3)*TqKz&hOf&+@7Sv#AMACzfemQhs{1%m~otSfnCo|7i8sSDElb8BTl96fr zZAl_~%pUSy%m_o4G0iK9^vDcFPAqOmK}bbaLDnmYfGG(pfH4R&{6Ii1#tzto6(vPe z>m}EXJ=S=D<8^VpHN0PQ4AhpevL{>c0LxgfDY=kgx?zObz=1Tiq#%2gGJ$Aec!hRL9}wm+1NAI{9ZS!ZO5qH)sU5TEJ@!%;W`)32xr!c=eT>27}F9`WgJ;ozU# z^mibJ#|Qd!jbub7vjan-)$lZPRkn%|pv#@|Oc{-&qITK+OSI!Ci%n)FWz^-a(L()v}Eg z%UsFv$Hil4D%;>GP<@lB9eK1FFjWOYe1UT|FBTRKS>2k;CagE1%=dx?aWvS$%pnp} zvu6TzD;Wo5ZqQw)LiaTo25vbY;Km1R4~tShAu1u3XCbGOsEc^H3y)fENwBp&@F|#P ztU`0P(ITESJ@DEPJ=XiEDes6Z`#2;oz7L69B2<=Hu$M=-@tWPD<}*Bd(h}X2JzgebNl@5DV3iTL}b}-)I zD*IICY97WC%pj@pdhBLx6p10lazC0}86V2jAy4hdsqr_|K}*cO=v5oOgA)XcWKq8< z{~|l7W5`?MAjX))yi}6>Sn2uT!yjlAvv1Zcw%I_78OGQrBrP`|mK}aFb`#&nmJjyM9q74$pv6 z1GvqD4y@U(9L3*V29MYK56F7}vSZ&}2S~Wj!)x52zGB($Ttqr?x zXRxiA9x~;4$5JdO45O6Lj>DR6Hi6p!92>C!niq9w=ow)`eF~a)fak=L1kI0H^47z% zwxQ;UXI>)$VLSQnMOOj}ubVZQ_7F$`q)mz7fd`U~&!3Zr}#GASbO7idJBMseS3eO7L&KNj6EJWoyO;q*BX zE=|T8gY~I^Z}d2Kc%J9EM1maG`GkQ+aDWZj0X-z{So$;N4d#llOrWyckx1#pu63nN z2$`20o_B5Uo4}Z^>^iO8#CP`@zP3aDo4}ba`Vh{Dq)0 zh$iVQeWPzdJnsg$T3hmoZ^VU16AK)@4fbDa_L6TZA3rRN@m5$;fA+89`p*@ah%SQc z(&bWMzmt7*9Qd^DjPY_6XT@yKLl~?n*jkPEhDGmZhPs0Zm*ZvI;+bK*%JD6LzLOWP zijF7Z6qWHM; zrlBdG0I8LOS~GP*D?#InTZa#@#?PQDk-$~zQFKP>gpdV0>7ZVGh^G8`2(g zXY4$`393Bk=aEdYSQjr|H&BO3h*E}tq3zcvd-2f*oxqy98R<;Dtu2XI<6NICUYg+% zn=UUw&{kiUp`pvbH^FRYXObCw)_t}G{@Daaka@#{q7teEXtJk?xCDA4d7RtEP`c$Y zi~}A_HIqFsBzFpJ>5xm=fES?ayGfS!vCq{*<~y<}&D4@Hy4f97>;}ONHK)<3Ez*E- zM^`+hCXrT`K>H&p^1i3f&!3%U>|NTG~(a+;Arf^oX1dX%Hh!LLguTLIdDhC?Z}aEB&r^ac&`}+j=X;r*+%U_ z|GwP)j0NaeiilFlE96Nb4SLgI%_1>W?YWc0ec^87HT6P!-f);$2&s5LLnwXSIwIYC!DCS< zY1n>JrK<7=(}(`}e)d*m;V?5uQLH&3D;K`z>W)|UVfiNe*xeJHEA7)p?20IvjE&hS z2x^jpq@}}P6h@2Fgxwp)ie!!!XJ}=0m`>Q*+H|H?+|@xB9)P_GF%?FW?l{cqvit($ zu)5*{54+^qczqW5Afivj9k&G?pV`2bN5}g(_$Iphb-T$+jqFU zQ3zQYMRH6&RSsAaDi?lIFSXo%CQP$UsX7Rt045i{a@SoR1}6+u8X=lZIXfV@5M5%( zLyJtKSn?>&V|75m%|CnMFLIZuQK-`2<({VPA?{%+TA0wefof!H8r{96?NKWD!LWKq3lC2FipP^qp9usdWYV?aE|ZgS?1gI5621n8L}$*Z<7}9xh99(-2h? z>Ej@+if2Rad7#b~cqZw~%CkM7YO=j?4xe2(7dzB}5U`_gtg;nbxF6w4T<}z@xE9X| z<160fm_eJR@cGqoMXE;r%(f0!=^3bG5X|CK8C*UrWG|6{kYa5WLKwlJ}E7W$VRhnVTV zOql-@5c9=O(H_i5~L7q_4gh}uA zp?Z`IYKS+8I396nf)5~zP#nC9^Aw=U}L0fQ0K!av&2FARJq!gn39DHA2Kx)FpdO{k%?ob9)UD6!9=w7B~O3=k>%nU~!U z!%`4aQB?GFt>(GQyTmk6K99Oy6%4nUcRJDZt_=|Y5Pj3E=$)zB`R%fSL(E9tqx9c? zz>7_V#upURSNPIUsX??= zjHw^$(rA0ab$*yAB7_$NGRUj4)w0XB>=kpXUSaSF@f#}BOx&tGNv51{L}c6#-V17& zleO}4s$Fmo^owH6d`8x|Iq32o=$~G#9SxLkEXZHKo-zLS0O0@jYW;I~{yjP~>K@wI zD>y%}Yw;Tk#QwwPi}{Ay`1Qf@qo9~1P?*zzGWcMv3^=Y~I})K7oLpJ(Tb6WQ=2te% zFT6@xF~@}XH4&RjyRSa$Tf1RA%_~oTOw(onp#tO{!anbJw_bf;Z~yq-vT?r;62$?Q zCq10$gY1%oYK_=uq1&Pj;XvLD>oDG^;veCHK)y`2tBv|Vdasx5`*wT5?gqU^uKXFk z^JVbnulJQ74MRqTs)RctGJGqC!yEF&6!`C|CE994$jp-dAXF(jJ8XnYv4l9y8%4IARKUs}98U zmK{Aq@dnUKzAp|;c}t9@qf!s)K|}>wqLFe7 zEJcmGa=_3_kS9J*9K*fx^!eMVn2v<>$53oAQGWdfO4x?W@)Cgdk2MuMsu)0Ae!y~$ z=)zVy5C5vJ)7uMR!t>io^N{Mbr#02!0* z%jzw7@NaWvH+SML(L7r+bmj^bL5F&Z_F5A!HL0c1(^#78Eg_tN=Rk+BRw=0TOcWUB zr{D8Kv=~s-KK(W^;^mULq}oHp0jlf=?8{j5+Ff(tV42+&D~`{_y@G6~=bFOwwBztw zC)c6B>rE5QHC~cNqpukv26^<9-MSVHoxV_>i0RD5(n~rBh8DMwe0VySNC=Nb0t#8J zVhMc9SRc335zsu%a9t1rMTZ!-S&U!XRIN)H#crG2EKJA4yEp;J3k)UWt+C}KEu;oZ zYc(Ril{`6(AlM9VTN8U@=7b#MtV@o*HkSBKmku&6eH8>+n5{P*dM22YyzXHjgxqF$ zbTu&!MGh?Je6KW`Lrfl6zx~NO z2IqE*4QVkcO9`)vUfSnz$8(DZ)g^<2OPG{Ia_v7r+7F?NCL=MjtkL#pF!av4hR_zh z-1`UwB*?gPsnv00cqx22Ot9WDZ zEi`2Etvob)>xg7&H*|GW8+Vr+`5R$|(l=OV@+mrEch7|igCckI%J`js825%m6&}fV zL`V5e$=&loO`tilCu|>7dzaB6ZfbBstZce|avR+ba>AqslL@_o z!YW2H87tx54dthwiSj$bhH_K*^C!fwsvWmZX74L%*)}E~aaK3(?S2=TfR_%^rAoy9xb!NO=H%b|&`F4$?fLEj51=jMO&r z&?J6O`-p>e88vlfhJnabBeL+p5)&e&87`Q{q(X@{MVyG)sEUyJ26LA=UE19PQ)Q3y z&yyV+SKEXRFjrVcx7tyC7VT*Df(F;IvC5RH!265r={h1Z@m>t-vv6>!&^PGdw!vW$qCRDDrThmv=JjFhCt<5Pg|&_b3c#tDru z?U&A3-zl>-Q>$~3-{{w8w8`mfcu8=SELwf2!?T~!AfldaG^ba@Q{3aI315rMXon^- z;|oi}Y^xjO#%rkg z(p$+3_S0xt?O2O@SKkOvJn*##rRfJ>2SExijylDi{dcFo z;~qf@L7q%q11vt^y|Fi3?aP9=s@SEn=Qx5t36Qh>xq}D6cox;%FNY1FSnZXQg4e@k z!+(75d!!*efZxZDwzWN=ohZc$oSk?hwkej#-x|lbNfXEWSzB zF(d&TFbtkr_pmJ>i%D)y$?j-pJrR5Oox%;I8)8GjmrEED?_zR-fE=R_P$#hhKAT1# z7=pN&8nX`w@kf4eBm_}bUxAB8xBKUK=??>&GdPEcl(=qcRFH15rNxC5`E4?tL+QK- z2x9ptZSF0>i0!hGKSa;&V@Ym8HU6B-x~!Ju-JbUW<;4`2gwun`u*n@1FM(0o^s*!)Xl{@<;#4U&5jem7F`+-EAfcEyI5E)_ zTm|4x5&!%N?d^{9%PkV{we!fz%eKOYRiPn^}WKf zGa5}DZ)oR{F0NFAKPick%Ed~`*R}`v+0iefC;CYSadg~bo-p@!*hfGdgAxqMF;j*b zADo2*saVg!dr&x<0_X0(?#4v=?6__kM%%wX1jnLT9^JJg%*gUA)iBoGC^9jc$0>kE zSUlS~z_l_8VzS*}mQr#6vil=QI4Bu%^(OT6=@HE!CF5Rjo2+V5lFZE3-3zs4afAy) z6z0cs>=Q^=Aa7(J$%r~b^vBRR#>nK&wvqdV!EuZs%GB{mSb`tD%|HjP5ROA@U+&_V43LWW-^WUb4YQ+Ej zMC`wSrii1Xo#X!-q#L7Z;f1A!!iz-L*_z^RYHg-h>WZqNksaJ=Y!yx@XV|hyI1zh7 zJ*kd4l_bejX?i?tVnLxbB^%I~lGBKqNiKt=sX3@=e&Dh028kD7_X&VI+krXNNEG>X z&}+8qHQRB^^SVvd_nCL53+(@O_Xukq0_Qvkb|8dH?zP__%MOB9dO*bdsDO!Ec3{Np zoh=YJK*qW7N$lenDG)h;WbB(cfF3j#|E>Xg1kEdUdWMm0dI!STH?AKzkT&j}D3Cdj zHt`7=Bmt^N@@NG;OaI21v8QjJI-q3in>4@yx<~kE0eXg^P4Gw=1RX?~v4*a#e+Qkh zhG7n!OY#UBL;=b+@yQxQ9dtJCohuMK;5P0Z5HPzh!q_*ZpE*Fo*f)khxBtiZ+kXH$ z2ru!S2oy93FX=rI%$KlVk|iOWgpp{Bslc3^r2nN}WfsnYgodjKZpN5Jj7t5zpjw4m z^E6(QR_kP0TY`_8-;PE;2fXr9oD?xV91IbCPp2r!5)x^2EdHW`q2Syd9~*O`7;tQ; z^4DRYo;CFd&RU1!=;47peY_%5GD9-tNFm_U%4uPPxoFI=(#1Qo3-8Etn0tFW!Ec36n~C$3TwpmjNizOZ9Lsiy5jPE1Ty22y~0y6c5# zu^|Juzd0vUVk{_Ofj((&B9R00{HYE?-Aa>CSt*1jht4koV*fNhG3Tw-2l0u6 zsLg6t<@xDP`9%sWQ(P~pRY{VaX^@u4nl+vf34l&$u7aX8Ru%ryCYN4ku373fn=P{z zL6J5P^d}fvuJRVrJ}&EY&t}E!#VHCkfH~AG)cyF|ml$YTwm$6EMok0lDx2w<9ZO4* z;1{A$3wAMyyKAaoK^aw?F?F-iejZoinC=RW6vk7H>pytC-zKLy#AhrsjdvM2Q!+z6 zYpgRvW!)Y-3TU3CBW&S1OFVL`TLo)u>(wsvIDdk1CQ!E<2?8YGnBOW*OWch;#GH_s zTuEyrIjZvL&*fED&R*mNvvC%rOmL9SIq0_K`nXK!K9O}%^{*UN?*>WSN1*J2^E?KC zr=UnMX$O6D$@vxh0d6Yhb)jqYx&5=09n-BOn_NalHPKTg4(uKT0q+R0IaL;>oFp~c zs6fOP=*(Pr%FdDtT{p;8BO@oW)EY`t5>;L4%S<*SRms)U#C;!#-ea}}j_bn(VHV$<63YOFHi%=Gst zUU$cr#uF6@%tSbdkhNpi_Uc;jv_(0uVJBnEsA5ZHETVLmgHWy#>$OhYug9v4!j8Z* zsm)s~x-v6GiTQ*IZX>NYayaU)Bbw4(#)>YSd#0MfWwc8Wg*`x}5Z>%HTJ?$=(O6s8 z6WU&4hd_^^t&mq`G*cw==*dF6m_CKxX^PA$-NlkzajfUhtWCyy{dE5s^uCrx?Hq(c z8+TIsd`UW5p-rjL2*O0h5~NTmg$(660*P6p2vk~TTGB3LtT_M~X04hRQ9H-QST|8R z?&R8M_iQ+phiP)(kBvo5Rbo0z-p1xF5^XKm>xnPPmywC3wLMM=EID!R9_mA z^$bctEf$f2IH%jl5P92d2C_06%mIAYQ`hY$0*|q3_2EfJ@msSci?w{6<1T_ z2KkHONVQ{25Dv37N;l4WROq=`GaC8KU){<7XBi)7JtnlWrIm!dgOq&_p z0EA}}`(EcXe2(SsRc?DOuHWZPFl_w*008CNN}RY~jtYEk2ufT238XIseUuSbo>tg* z+J55|U-WzU^<{wSkBoU2NR8SXVGMv1yQh`yoj(<*AAo%^GHEkER?23d(-DmMYnv`u zT;B;N{ZHxD{DEHBUXI!o$C{CNLlz{Va6VjNJ%FJ%4oH&$tveGPpoVfNrdWu1 zNZXw@oFfce2PUp4+~n2|Cg!J)%{VRj1~e|V92krfX{c<8HISaEuR#*D3+h8{q zh#T!qfwU~pwEYCq0qcuNI2IVzEKsjhP`*Xt^ns$g8_m1OH21as$m7TP%+xN;;#9x! zU99>C%<4mFx2oj_IgR7!VT=wL%lEYHp$y&GrxvMPz40oJrcfp;p2V}X2hQ2*vuRLL z4_^|y<7v`aYgC5cYY(Wh6xp3Z!|i+HU)h-Z%c=-KepGfx%dWE5@La@mx5jv7pd2S? zTqJY1Cpp=hBNaP_X~*wD-C;^X*_%|JBy)HtKG~ad6>AYxBy)MkTpTRZ6U zR~#<>N?;?l6OU-`@MW-0m*M1+xqXvvPM73myZDlcXDaXLWq{e7w0|XiJI8n>qHM(x z+=)lGci=KvM@!}3DV?^H1hP9r6<#B}7X4&l94H6M%GxwA(31?ZBO@$y_#28LDih9D zD8*cp@-;{e4HY*CZ~(zh&K)D?9OYHA2XuS##_P_<^5O^ZPwZAVPp5YD zmtijW&lzThe=^Kl#ST`{o&+?>e@OTdC2`>Gk|Qbo>ygies(S5Mqk2+$qE@fG(Yli( z|1ZNFy@M~A8D==ab+y4^W;z?|_xC8OZY*R0_eM`Lci9h~i|bf5c^lDmgP6kPPY~Xm+M4&k zop!u|iH;Ta$%GaGjWmEGCxZ={WFi`NyQ2Q0MACAVc2;9X2cf50FbQlSFl==_yB{9s zk?x2Smf*^mnmU;I^Y$7lrpWF{cT=VAq^2Hdn+w4%UVU6t`EZTl%?14YDq|4hT{h{P zsZ?T63Zu^mYxs>27p?Ro*xp1iwoi&-5MtAPn!#@K0ujr&j%ZTIZS6I$k1OqneSQ7B?SGZMO;Asu1g#^_gat<9be98t2hQX~#V#ROSZ zN@l3eQN|F&MqB56i!|iPmFZG-zQFBCo`=~}Of{1v*A-6RjoTLA&~|spVCT%R z@fe|Gv>_O{?gtN(xBBc<5yK8S5nMQNbGX;Jk2zPk7YluRzF_+*?2x%5C&Z)nA_MHC ziSqU;11#FNV=%|VM_Pf$AQ{OFHS?C>C0UHN6zKgX0+4~z;Er0O)q%FgLOtvQ2-VXRMY{?#X&_27Qee_t`oXqO6+&TA61??DK`Z?74gC)J9_i zwE!dWaGiSViH!LCVh;g&1_(q$cGENoGzxXrp6zOCJJ(!lr8e++vW81QN2)t4n^CSs z#J*2-<%kU(S@ibd-(3Eo){d;IM$w?8=@lK>P2Bc{~s7xI?9f*Qst= z#N~Jfx*}m#RqZm?o*c@iGx5C5jBm(SSLm{(4BaZ^vpiOf6hue%omIvrsjBYZF(xyiX)Hi0-!8wZ%QY!>%1j_L5r@f zFciPcff?vnYAzYwp1bp^um+MMA*BVVQL5Uh1oq^7IH#qt_*LRFA+gekvCpR)gM+?P zPtHewyLKt~jNE+XXia72Rw%8f)}R5~AeCTJ|BkhuvqeV;wnb(UG7Q5>pP195o1L3I zv1ptxUhulCwxPC6PS8azN2&;s5E86yvuZLTT(Viu*#Igs&}<3SCDyb@R}mP@Z|CX_ zUN4Ckd+p^7aLr=YkbI?-njlvCd!n)E5R{JA6>=#?F{^L-Y1E5K=bw zy(vMHb|+WPPiY!LwazK4R;ywv`<#&jQqDPTyg#JbfV)F=(DHZ)Kr~18h;q*+`WkA> zfeGTxuoKJ_;X$&^-%?!Bc99nuwNUG_7wD_)!hhu=$06a1?$4?ZN^^;RC>| zSw>v*1%Qx7m<@iZS5)r>YnW)=zuJT?eqPoZ{~aYl^bT>sC>-f40s3$2!Kbt+F2Xq- zdyF1~AlVQAwXP`U9mhfFvD07zIOFh`#Y(&gVi7x5C$m60f@TU0dDP~;%#&Y-4a*JD z=cyJuqIk#wgzUa#mpw^#kgW24q>A~g1zf^;@aImLGEl(01w`GxcloDAI+@UvZ$R(5 z=L;CLB~7`;hhJ>(x#x=)Ig=K;`gUpq|}gLb#DT6ouni z{%Mz^_~Q;dREK?g2+DnklXhX4>`p}bvn0ZQVHoVLAlhxRGD{E~hWSF;>$c;}cRD9SudnY1m_GEK z`+@vwfYrcoyuUu;vE9rNo`}?OpY%c$W}A)PbRQt7g{ZrbC_7U-@fJFo$+^eg&3MGA zB$85wV*oWJQD6kgfLtiF!-TKgv7zR<`!rnK-8Y5Hx95f`x1J97G|SLD&eB#L+XP)! zpyaw#=R~v#x-xn6F8Mxn4c!sNPl(Z(wo9_ta(T_?o#L75arg>=3K-GFrJMAm|Wd9gL4S8tS3x3eIeU>J?qwu|Zu{!;mQ8JKx#9N9L()Lq-B_!mSU z8A2o}&g%jldM|f03@^M5I^xOvv{6?;Ks<*GQL68oVHE8U-pTS5h!Ts_3)+RwUA*z0 zhCafkFxJd?r|k;vlHEWuy0VBdmQAnrv=U30%36V^q%m!V zpEk%x@31$cf6I8tLEt&1)DG}z`S;?$;#OY>e!%{@*@sNI&QC#p{W^g9^-JbI+wA|> zbYNg;ZSt=bFJt0tpyX`l_+N&E|9Tp#J4#4u$Ui&8#zN}HSQVNgB2Wd0mCc$8TG0S# zK;;6TmQhot5~yrl4sJl^ebr1~a36|)z}^m{m{hE}mU>_5E}jbT=yEK~E}yccFaZ^( zg7|NGI9_x2I9}a&Y592nymi z2WREO1elPVMYJV@F~vYOHNvr!wR&cq*lemT)Mn;9cX*3pKuoSKJVfh(h03rVjB&ov z|2&;+gMbBCdCFdz*66awcG28Aiz5W*yfu?VH&S?2?Md`ZY&sVeAF&9oC$PW2YD=CE zEL@{ST6TD0GDHo6%23u{fV0!CQhE*^?3DpN!epcLdf}C22YPWvC{{-9sxXyDnNdtD z-rThsmA|j?DB##>8CuO|3bfPCQIJOLlkAf$0UDO;NJQ^)8{Ktm{+O2CC>^@qM;H*h z!rkOTmZuF=j9jAhm|C>8iWBH`5l35!6?*Z9CkzY>gt3P%#+sC*nY-OoKOAPb$##Qj zS|Mtwzz1mmmsiapd29vyj1ZnAppRB9hH86>A=^v&`KT_+pL#Wb2p?d|mm)C{j3~;~ zdUAXHMVG~?R6O`ijLtJ`kXu9qj!RE7nH^Zjia?8P<%a03JTq4O z8(YX?md!LNPy4a#1{6YtSJWSaN2jbvkC=h{h(gU4gF8Lz4GD)=x;e&MWAFn%D!^?33EQ!h`9sf% zZ$*$02#tvZaV~|MK?yWP*24}U1se}VaXJvZ)Iq`<<(JpU*Hn^;*cDW0=Pi2elnPre zATsIr_?eM>(LKmfkRJ*|&`sNA6~oNK2wRD0Ip^^?7sxw_y~zZa=g){`C3;FD3*3lB z&9w)*yeASaOX{ZLf9XV^H1nnaSL%6-yx#%a5JNJ+w3xmU6225j{1(ePA8>WZKULLX z86AYTIUc*HEPz6>P)F@;h#hw4*mRnf)ko;j=M;Il7Sczt}u0!9J8Ro|(;fzB_8C#&1UEB#> zpq15+V^=9@$qDC)U{dVs8cjhTgfW6#gc&Aw`6JmuGtco-7Gozi``g>!J@o7w4|gI4 zd?0|gGTVv*mnf2Da?Qa}OsI-oZAf72!#!7t9<>H_b>kGBIhrpmI)#3fn8;TnE&2>S zDVZsYGKz+L={p#kjDcJ#G}rKhjw|BB?|*cVGkDvI!T)6zcESH+K*jj)tudpbp@bv= z{|$QW-`!ZN>RddHMxnpv6iSCziO7LnC;YEmVQ=$ zq=+1Ln#o+FgB(56F_z3UO%iG_?+A+4LDO;Z^oIKt=UKecECU$#?Z6B@%yD zAGptF2$+Mkr>l@Stz(KFc`l(?CBAV#@>bH-ypocM3h1mr$Q0)yq+aG04OWOV5g;^3 zB#@a1Fc+>^Z)9{HV^AqU_S%^nhh&s5gYZ{P-`N$JJW;$H^mwRYh6@p@m**Quu3!i96yc;AgT&g!^omv8<0h~yMji(DYv61V;-0JkzF70zJA%$MKfoq5|CaJnFLm3Q~vkuN&!ms%it zBEn_n=8{~wlZYPghIMa*-xz-mCvQz;BWvwyXdw7$M^`Z-Cf>SkS+cY3R-xhr7mx7L znPf9XV^U8L3W78@4Og#v_LE5kmAR2aPmJ3aX;Ns+v*XFf_TUH_V*n}Oe5%aoT&SMy6)bNbPi7cs2YGfs*YB=$YUt5M(fYl3My^Ag1#h{H#v9gUmzrplfR%Mjo7I4bnB?&WQ zkvKhFo-WuSN{^P!BAfZ5Ic4ctD*znJUWgS5+L=Ke!pmJ9cl~Gb4qfNJTo-(?8|?lG z*xjQ^zL2%l#{_`$82|cVR8F2$McG?@ceYkfLYcSySO!ae*F{+)10x( zA$e=K=4O}Ab;0xdtOt5;>}GfFhL>IvPkJFt@#z5BCm4d4QxXrWWC>7amg2Fo!t7B} zfHkE%gGM}*VBY;8-DPmHU9~EK;WKv05*yy!Xw}Nar|M)*%XUt7A*0|xlbKt@>|h{1 zPT81+DlB(Jv4nbl*k8UX8R#Olqgt6&YO>lQnMEfv#fZ%fqM2d=g_(lvisGZ4pDQ%D zoj>2a(qjAsd1vMjvoZcjgPnVf%Q%T4EN_2C$iVnpB7jzsS=#vYI8afUevAiDr?C)n zty`I*|5(X8KVffxpO0o<*|4CT5_&QPk+DcpghS*N()qpe@|*jW<~336SdjMNA>%~% z;%L~l#zbm>HxZ1ldO*e&PS|SS9CMfY;gzW#vt#5>(_HA;>;qPJ3*W7wf-bOV0z=eQX0_f(ok}DHR+k zK5F^ZKsiUmmqjlA{+~1)$7MB1NfDLBKUi&9N2jn2DlZN@{m|7$e&CLz5pl)NNIpuV zU-ad3D77j%xx^!;Ni%oMq#V1%gKT!m-w(u$P3DVaOX+|1zZ?V9;g*j@TrSU0|4Cv$ z)jzL)DRa<4F&WbDI`6K=8en`$T!~}$kfD=AcMRiSt3PCbT8!4F2eT=rzVsbXjw}&O z9La-lzNiEuEaMY%8ndcTY(oA5q%o#OZ{)O_mEedeAt1lrubJPo@{K?QC#8~1=Tn=Q z96${`KECUZRHmQC5J_=f-lzl4gk=gUuJ`S4D%xYFA$M^OElfSpXx;kJ%=bAn!bnTt z@A^135z(QKDO4t>XEGL{vNWjfLomM$+Ir5>MObacw1whON^8o@Wtwv8w4SzD&sn&m z_^9Dm<4ZRdYy)1 zPJF&R&y@iwGpXt^UAalFskCYm3Q}v@_NY3dj}0dJPFxakgw!odX!zTZIx$PVLcQnM zBU*aOlBG+5>R75cbUAI=T;X#(J-!E^Rg-z|~+GR9Uq$9Ufb-n_q zGbU%U!c&)Hp;=#H2K~o<&Nk#GS+(oztVed~LG0qy;sfZ5K}Lz0V*dLPv+ntGSRj+O zbVA|7Df3&@Pi~3NoKXhx&pX+W(DX&fkQU=A#nWchM&XsRNGb}QJtb$22wX^8rSAlX zK@>|}%UfTx_pNIeiEEB%rt@Hi?dyAOKYzXEbGptW2qJ?fv{;mZ;vji!SK(0vI{<8{ za{jaj^uAMg;jw6QPN@1dN9}F%aN-iZcbyT7`bxt%P|cNwOIJ{jsrcWr-`Fs*wgYEJ zbsHBztrN(Vz)8>hXMELW@6F~;bWLm^sVKN)Nf#Ya(ZoDu!qQ7s$*_}#%-Q54O^}_s$E;ER^jUwO8O_ z%U(I;Nm_;~Eo<EjgOVX?iDqt24BcdibfJ{^{Ye& zlQ!7w1>nvD`x(H214`li7T?FT*U9J1 zTeavd!}$Ws18fgGeCETBPSsH@oW|ys4A^OKUyVw0m1F(^lO`+SndL#y>Cy*v6iT~&6WJ}1fRHk!3l%j%eQ7a>--v^7Jmj1876 zBHdcuvN}&`oUU#YMm+2-jUcIb7uUDh8f%w@xMEnXb)X_l9bR(}LGR>WQu$VCeGckL zChMqq8ACWPry~|GL2Xptr5|k;A=j`2+N}0Q4+DX>gw`J!gFh#jbbAX(s$|p;IA3vj zLyf#dSD6ETAToJzW}9+Z7T9Ou7~NWA=7?oaQR@|ft^RF}-7wK@bT?(|Q_StQXHQ)H zY4+X}?qp+W0d!~XXm60QWdmxC-Y9RDv2H=H_T6aSFk^rE*Bq$>nXvW1cIWP&_p#W0 z$=^g{@j!jq$Jhe!2JVRlz}R`geUZo5g79YUY4j;3x1h=90-dm@_PNKJ_Bmov?h3}| z3}khGp>e1VV2)u8ki^pc!aHzBLLZCox64RU>wm=N1NBZm0=cWf-UIce9*YCw%QZs! z1a3sE4YbPcg>B`~1!iN{1?0lU1KOKyfU8Dge4jKRckouRU*_=Xe8cIXDJCRh-gOokux)VqN{$~{QH`-D8*LC^4MfIzurxN^T)&($`69 z#Sh>=A=p(z_nqEfHLUut8uq`{QU8A+n25WPiM_Lho$Y_~#XCw^D*qzI>L(h#fgx*F zrpAb{)Z4&cs^oL!V+y9P^^SQHg z#GdQvB2Kopa^9Za=JC4W=-YemWb^xc-n#u|>Yx`A*sPAK-0d$NUW`TH9WDZ*0U|}o zD<~_%mx^+%QmzhRsM5`+I#G^a%+i1yGSfM46MkX}-==KMd09BWjf$N|{+4V|T%ZM7 zXML_ZL~nJq0Rq}OOrOqb1(UV*8%EdS(#IZm6n>FrsyQ-m%_Vdq+ag7&dsfQJ)3R>Z zMwel#B2l9zvqkxI+%~r0CsKjww?}A&xgD(%7IZ*!hY;})M^Win){4#FO37`t|~2KhIZ4(#UK;iG$LWKXoG%pWaljmYEVDt zOV*Zx>aC2~d9fm;o>-r)r3~XhqR4z3P6ztzNKet)+645;ZpJFd{zS9SfseV8#SH%VPe zMCSYWrqSCSrZ|)>A%dt`o%1vgfJMO358kF6d$V-CsExS>V$QrVj?_N%TD?eM!#)0H*Pyxy4{KQQ4#M*-08-!ZDg z>6Gx{i3A|jgZtejjw&<#-xl|x{a;r}-|&77JO~Kw{|CJPpVK1H)D8H*(0kkeK<|B0 zz}aCUm653pWvo=7VG@3O(@3F2D8*q>3em2$Odcgv+PMsvaY%3LtgpQl?@Ra!j?{EF zmR&YFpFx?wC0;xnZph{Rm}62;Oz-l%<~!y(<~`nUdVcQl`GDxv*L~`b4nm@cLz|`f zJse8^_ihB&%X}*E+h~yUbO2PIZC{S|Zp0GBo~;nVKGJO@(A9e-=D9WKFXsG*+$|B< zkx)ylVhX3Kx~ROLIdyslp@RG%1UPC)dh{rS)>xgSd_0{NOFhca=rIBxGZ&E#YKUDO zoVu!?4$}kK4{u!7gD4|3VlN8e1R1L(c2gNPU;-u>+pHHY-^MAe#7SLu#5{krR!$V4qKCL|Nbb zE0Y?kIy+RiKusjZw+nMJ|}B^N}!R~hDH!qqXn5ZUJsK7>1l zBd%_2onLxL9QhZGQ=(p6a6UFAVsV7ibplnTQ941Q0x$O5g4vH8e!(3GO~`O*e;r2`wFq2N+t;T`W92Z|8wuKcnv4< z&3JXznGX>~Bh#!EOCrr0fv6ydp>Z?lhtkBj&G&o4S)|@2xiUUKf+ZVns}VW)JMtph zU-}~3f2^6}fMO8#qTH|1s(Rau%Lm$QOPh=k$8y>rTUS;Ke#u8%zR-E7iPuL|S)p~l`8MyzI4xbXP0IMY#ZU~{=QewjL%~g*{ z6%7iZlboamq?N}Z@L=s8jn1XzB)}e7!MTafJPNp%iMQscoB&Kx=oi4Z`43P>ays1c zXxNCgHP*r-+9T7*zjxib=WoQ|nBjAMywu5-D3aZiPRzJKP5f+)+Dk z_938GIQ|;Y=Yr@lLA9uV^GI4kHwcA` zE)p2!EaVz}$Tp7Jz^=<@o{)n}hT_|)J&s=Df%_Sd#KAg>Jq?TZ|5d^GlJ(AN*F z{4&nO09VBHT!rf}cOc#;poPdxA+}f&Jo*w)dr75*MUGMG6#0e`8E>r4N8t4Hm`^Qf zj?geD!&n7w>5>3cBBvDzt=%B~^0>imMC25|Ky=WFyHLqbHSN=om0JpKv8b@HPyYiE zRy-6G&oFY1v2JmnWYnfqvn($pic3Upb_$O=YM~qi7;f2nCTW#6GsFbF&+}K9tD$E3 zWm{fPIG#;``M=JnJSr`}HZ&R90iVw(LIW0+z0s(AURk{;P?Y-d-)S6cj!965#MvxZ zc6UF&ov0q&!v*n2-aG$0dBGB!f=cEaYiC0M0pb52PIss8rL2*;sfCf9iH#|MLDbpS z)?M=Zs26euS||eST`WyZ|F6uWqI8rB=pDC5?!+a&9`Jiigc?J@Gq zT;J|55I>kS%V5q8seY56SLC|{{zvcqs8%t8ynLPVzv@obvjio?-9EJ9rU9=3rg~98 zQ?JOjb6Hq#LP%6@8Up(UQ5Gf(LI93S^>5<)z;omw^SeO;F>5#RiIb|-7X+>njL)UN;!eWGua+e z7@(to;x>a!GOzUSuET@nSn3-P52e- zp1Vt+&Is1+8HFKkK29@oUeZQJ^#(V4$>G( z{qUG5FR(jwC?-z-V>48|01V!iH>p z!#r%_=dJfoFbOF%#JE<(Koqi&m;^UpKT(_z66q*IS(z7d0_nj&r2JUnZF44anA(-3 zOc8&+3sp|lBy-2i6ymh9jxSiz*cvZZ$BgwyOVg;qf1MZib^0>DzWGIq$RHqm|3jHt z7y%e0jQ|SHKnLe<(B9V85(qQ}DB3##zl+q^)XB;I|Ekh#jaOeZb9^82)eBkndB_yO z)j9S^V!3*vKt4|Pu$j2glwNUmJhVKN7W?jId zA*L%Hmp-9Bi@pMvjyG3nWte9cyb@fu?%%z0t{=PIAA1<|Kpmj;BMsrkFI#c-!L60T z4KYed@N(iOSD^`d191rEZXBStLZAP*kp>_p9d6Cz@aOFYqErrq9vpc;ESxKTqln7?r@KgLBNy>SEI9&Azd(ixb?`5vq~P;7=-sJlz|-eK3O?rLr2 zF+}zK0kT4Fp{6=p@(@*gU7_efAz7V@Gh9}ydxFKBR2zn=v)e*|t zDx6puLmy1N>^Lo`AYwR(lLP>3=2<;we1u^KHA=6y*9!%I`3YcMci_T#+mn$?}n-?E;qVvqCUq~>2=PneQ$es^~ z(>3xW!S>Zj&ARQPx?>jh>i+&q*K4pUz=-NW|)_t*}x;q<;r0L zRx_$w%{D9ORN$8%`^@6SC$i`~3r3^Q|GhdMC)FPIgSHrXvXk(}B4f3)BxoShO*_Jx zLiXTYxH!!%nwD>xqR~)&&hq@plw{MTL?yOYs?avnH4&8(3O$9pK_Fn>v{cq~<{cH_ zFyUo!wd&ETC@zg4tZl4aC`FwPXtR zq*fuKQ+?#m3s|6syCYfK zKc%5u;0Ly^kqNkHazGOhuz~)~x)$T7*Ijnp=Td)bZM-tT#^L@9>0wC=gmsT`++7*o zBAU1Uxyp7S;}}dePejj#XTHgek_*T#xS*yWZT6*HdGpmr(r~Bcb5}m-ga}A6E}E0G z0F6Rd)ja{?KMz~?hZ4$ipX92XW*H5H+@bT`dm-^o(? zB#jwoHRwmH+|>IFttMD|tO8br!Khc3SlU-K17*xTQsv`QXrv)jFu+?nHRqaW9FVtb zwWwh-%e$!rhv!fqgHkyZqFX8*@ayl2FFH9Ba}wf7z%eh*pq$HpAuA8r4zG_FFDN z`~RtxDbcO@pv0@=wf)DEp}q^3#-ezARFeAaUW^sZ*%Aol0XpI5ZLZT z9>IE~(zVaeac!_&@~X{u4?8X@ni7rn)zCXf7wcAVVbf}C$S-QBF46*)5WVGdmz!j( z{zKK1V|TXYuMv|iw5}F&Mf%4vC(5xZm{3U)zsuDfX4m1R{F(+)S^;fSVOc zH(BT1D{AD&jtTy~OTCq#T4>+phag2)(iKphrarWu6F>kb!7=`Md4DoC-0yTE*F3Uj z4ByaA7rz)!G0PV#>e9!%&<1cWf`#0x;DXnUDgS8_@KUzGc?Zut@KvGV0|m1l*m1FL{mxYWra->3QoU; zmgWq7zXjLjo%mW~*fWxLM@~Ii^}RK4Jq?9QE49ojN-Au;P)Zx!!y;3t<&zh%NUF!b z?t}5z#k;@j(hXM>27Pw4*Yqezp)GqPcNZ>y{D@P#RrsCx>x#`S8v1#*ki-oqKyp1y zRb2GMIlxwfsjz>qreLoqLWLTZIsy;ut51*G=FqtwkzotcC;PKR(~`&>+vz!x#rN~S zu{Q-97gB5?K|nITI}6_bp|b!On_1eJGAN6PfB(oCIVhW&I2)TPn;P2#Oq5NH{&)Ld ztf8fXrh)c(Mav3bg{8RP4Nl5gRz;2jAuO5{miIGcM_AIvj9z47hR@|Wuh5KzeDDHJ zidyE7jbS)K?MH%U^0&n2^@kw>L{UsA$QqQ;bh zjjw8Cq=>zQf|jN#8aq+BBW(^|=KT7CS-GC5rA_VEMQi&dY}h`ikR>^|Flq{&a7dhqZ+XFK5754iGA&$#})6wLpM(*Ty-H~6xsrtQWoe#@5#M62 z-jRc_ch@?wg;<7APxzxdL*YzUoGEp=FbaWvJpbpn_}s!Ke5t?hnuqpy>)$@*$dnzY z;(BrYq5=o%z)52O5ezXCjqwQCc6e_}sOU5OQtT38yl6^7fNa@HxE^(mcwz)ynMPf1 z$2yAIY!p580Pg(NPw$aXDAhVHfO<}_lD~24?AcJ86XZi;T}o5DmvjlDnu@< z=)@OdoN?I;d@jo)gQ{0JE_F%U3#5aZJ2v0f=}8Mc3@}MLS|$v4s-Eou-qBb&qplL0f1D0L7{}_6_f!Fe|{%CrQZb1CdZb_GQ=LsE?+z zF9mexhIq%}2jhe*NENF*b_F^DiMh_8Soh28BmV&q!h5L-M$$<9PTxXbMgTg_agaNHVZy8vqw>Z6miXLHfeS%XN zIcA>vjt;H4dSxGNn&Hk8Z=!aEepVihKu8DV%6?wdR!!fztYu(SX3r`^N$Xuv??uN zE?@8!pc{h1M*_JjG@LijwSugYx|#Y ztpDe0>_6vaaeIKR5%7Pd+IFd1E1(LY@y!dTE{P$d@8dREQ7uAj$3_@Jh$;Q_@>WVT z;%SUr&vtgf*o}P=kVQikz!G>bjB;&8XHut&TPbRAJ$9UGbG^!5t=Z=HhIBxDt{>ei zu%thX8#u(X(Pf%cgIlK6ot~9k?f?E@_$B7&d*&q!axCWQq|)HuwdMhWWf9YblA`d% zTIE)VNNqZp^4S{2@Y(Gxv{}pD`%>oJ%$Kjk+qz!ahQQ6pyFz#kAFX4+q_>t+QitTc z&L4ss2YppzjlVpw_`v0XE_fH!3035tXnZ-C7LDNc+P9im|T& z3VdJqY)IeeL#r2(oJ&Au zRR+K@%K@N5Qw`lj5jfgUD)JPs0oNLXF2u$iQ{1=a!`Y!{AU^2Xjc*?qO1d6Q_F3*g)F~~go)lPM^N?OS;mlI?Wcb=;iPQO4PrQKtrAtkU~0>ca|B0JX?z9Vrk z?=!vaar`&u9gChB<#|$s2wep(x$as9OQRfHJ_og#TleePDW^GA>>Pv&1U5kkaTzn- z*9dm}!ryR2;xM9ic_-j)$=kb}%a4a5)xsS5HzY$2JrvK;_6W4OhYOk*FA$m({$FYz zVGleFXrAShGom{%kmXIN9Z2uBghw&?fAfbRkHp}}b-ZF#EfMq}e`SLlSd7Y^^i6=9 zlyTKT{H_$?%mOiY1edVm(i@uB3sJ%L$!S}6rzLE={@0RCq>2(l{q8(p zkpBO6p8xlf{m(^Xm&U5fx;WNX6%%w6^dRYvP+eA52p&zYjKYvPKmTEvAFwb8Hk}HGF2j}DsTo464!HrX4!$AQ-R{%6Qg>wmkXC{L@T$1wo`g8aacJrVpM7N+w_u1d_{Ct38obZFx~IWSE`bfp?&J z!)HLPRE}@5x0+ei88>D3kx{kLRhAJv(M8+5jF6euyEQIzDSN_;MZ|4%N!3Z%8UE8C z3*LhT(oulNghxwk^3PgS^3Bp<$-EV zBzRkUYU8Bv)Lk)?==!pQ%6xOS2J8AbrIb~aO`lR5wjNbJy@4TTJ=f0=99t<2t=!*b zxX-G#DM>Uqs=13@OnMF34@Jgqy4lvn3uEQ(5}Jj&>YJ+VF7 zbPJBO-~JR7t$vFRFwEz!G#<*TNZFHVKa+>^drWyP_mJ5|mVxEArLdkKd z4vA9)uf~S*^Y?qKQVyhXV{H54OUhjVB6y!KNK7i!2`2ak2WL(k<%ooiRUCKF#Q}{z z!na6)XO!OeIae{K>G~r|$Eljmk8!CL9c&G0zq+@%Lq3c@1^(dSw7|ubAi=ufQz!cD z@lG}N7*LQ#>h!pAL0xl@LZL)w$`X_{KTf6LX8Di z5dYC)vN4P&^83A1VVGAx{-z>o`G#9aaTwPv(SdTGu9I#bf2%X{*`zZTiPAkqK=hx7 zI9*x}ryw)`pu|u9lv@~n(=hwX7pa%ZqQ+=MyyuH!RjOtU%C#7$HwG+u`&j+{ROasf zWq9-EaHEjX5UtqV9f2Yq>RwQ}gXD!#z?C6qZ9p9w8R!Vr0Zi~8_D0*1Y9+)XYH4`N z@}?ST{rwzcO|2fqkn&o-0NRlAFF=mA0jHJJMbYKoH@mfM{Rp38m!w`pE?X>|`j4mD zVf0Hf)1*%L1(e4#tYUWe`<31>w#6#xTo&EfvwnYN*;Lu99n9Jdn>7br0ns6_alNDC znyOF>K9F%9@6wxm-}ZkX%!A`HU4{w_U;(ri`q6ZKBQ6mf8ToQ7+OChMl#bZk^*LI7 zEV9&b*ec6z4%IUZV^vKL>r@kusxl-L`Zfk5)(!Fc`MBxbLd25c1DT^9@XzdF@6<^) z@}bOqGQE<`hb6;fgp$~I%MQLGvZYj_PzU@ z!rlluf}Zg&++EY43)la!{dvmlUlE(0m6)2scCKd~Oea7qI7eXn9HAb2se&9cRJlnw zuW!qx6|#L={5p-hMG*Pgx(PuVqeFw6qfCZA}@(DM$s8L47HM6)*id55}zl4 zSso=;fl)_6v?(#K%1TUE@G7y}KCqWet+FnWBblHER`l4w>YHQJ8kX29`h>7=8D)rE znAXcFtBbOvPZA--x`P#yNM#~9Tj&K0pgfg_hf_$11sz=e#cw|P(Dl&B7g;L#B=tZd zbPa)M^ z!h6Nhn&PsFYWBB13d?RG>8h3YV5{9>na{4ZhP(P{Z4T4U+dVkMz+DrwY6bN z+O@5mQTAGV6Mma$n;Xl-`BKQVN=b|Kq_Y%)qkx{bSjV|fN0{Y9Ya@6^OpHI(}Y z0JUgwCbgu#bPFdohZdWpJzq=RA7xtB)I}lC2V>Bdww@m=#B5;4t(+u;=Dm10uIv3T zYJNO*7~aQ;q5tko#Rd4rU`D{dYjfn}GZ4A*jSN`s*d|tX8c278RE1z%n9ZN})ukV; zxDJ7Ud!*# zMGP(QSJv}J;d$}aei<)9!HJ00Gf7$s)NLdtGGtaacIxVs{yTi_!2`Ic4XvRraSWWs zazxfgvy^Y6h-PU=SsXfmOW>;l%JwLrBDpsDr{R}Fuf_gRT^dq_u4|g5NgJzNjpt=0 z`EVupLPNO%S|gM3ruKM3tf9`_u2&Jf(cniCSqpErJsUR(u*`mzPX{6vr=Vz$5J8kE z?5@Q07aoP&KwI{gsTG>e7*V2mo@P=ODRU>>eZE1WTZsl+l?f>2%cBbfHM%Ya@4`k< zsw~bI(DBrLOS>b6gsQ?lY}Pl6r8cG!3MCLSu+~LOWNbmU-y0CJgktRU-=hLnWecB~ zwCQF?E$}keLEm{^KOnkBPK&LUv1%xeQk}+rJyMLKAAN?>{TKOy0g8b?SFoUaP&&eZQZ6{zw#zK;Pt`FC-Pai zgmkM=Kf8U%`~F&z$$tRd2NLo?$nr|KR5u2XJ*WRw;jAW?Em6s|C5MRUb?@K2H8{D? zB^r~Reuog2HOEwv9oG7*71r`!Tf`vVybPXil@cNM|J))vnHoC-EP?I}k|t(O|3OKx z2UvO-{Rdh0zj6~MwPAF17Z(N0)6zNM8At^snYB=uh>)?OBjN`sei3}5kBPH8 ziN7jhC7_}!WSyGfoR=cuqDh`Qo|T_EVv;AA*jKjX?Koi8r$=SYIoXkt|8}xu%h@}? z;}>0;KtEH2I_7$1%%Q%Q|0WgfpURIz5VK-HtDWp%0~i2vM7kFy9E9|#BqkB*%x98q zc*VD~^3umYDbVjt++Cv-wS*@DcXhgF2dlgzV2Ge9JVGlps8%_DmFQK26KQ54 z29DsG1nJ=p>!j%^V2=_H0G+(*6Q=RQZ2h_^!%a90OD`r+%Uq|Vy*nQIdpqAI`ok07 z`P8^`Ze%qH6IA_^$cB92&2b*U*>`A=$R`dh?$gX0m*Oj^uuuUZo^2EsjS$2{#_aQ>L(WY!I7_ru zHI+IlD$QjHR1I&yX)S`5>3w_W<!Zz62U=a z@OGQ#sJvR<UL#uPn$V~kTB;O-{0BI@^> zEeFe3ODdIcew}N{brA31<#RL9(ij9`D!b}d4QF#WN&DLN(X&7P*U`p9Pts-Wj4(1| zAZ+0f>TNI$G9(Tkk#Su;fveA#Ae$eZ1l{=7hUUH{R1b;Clz4pYsm>zUo4S2z^pBNk zZbSnwGZ`n6I7BCx=o)p$PaF#>a;Lm42SaV{yuP2b5)cRqPT7uTgSf02n149q>M#nU zwrrOSe<6$G_W`#ZhCCcz&+%ycVfV~-fHWMuqms5pEEs6;_>A*31BUjLe3+gAMm2!> zKpJ?`#9siw?F}Q#CJ(9mhwtuP_E*N!Phwyb2O9sVmzCTPgCf1Vy|V7!w7WOf$gxp46z7w>YB>5-xG^}q$ zABQ;rF2G+>BDseQ*(UaqMHzV>`AEce>uKhfk$#segDTyluuNaLH~z4;Hsej)LE5nh z#pAh%Q6YZ8YxD53mDji8GYTwupf-Xpes%4~1;>y38?@PNF%7bE+&X~OpA(^t3@Sdf z(=h7cN~|inPmGO{!AqH&1@QiRFce>>0*Df~jHvGQWfL1Di)$AJ;6SSEIoQ>NIE8~g zI(uG)-$n3c1jVWesS%E@kqjZhhU^RxvDu-;6XcMs@ED7&fgv9OIkP2#xPk<0xBfmZ zYq!!qm28XeWII7h9l|O-bFEg{wJ~C`2%oW}N6K8%N~mR}+Xe3c=1ogT3b;qgZ*JN# zj8c2`tnP%;>bwklkKs9PGy+BU10T@+a*j@gQ+jTY z!*(`>&RgUREiw~1I%THS_8olKUghTSE#KA4O9F zN4XXsNmByJi!S#B*|`{J<{wd0)3A?t0yX#3?vcYiaa4>4shwiSm@i`PUecvHFw$P( z21~E#pULtJ^E73IA$n{=bRt#hao7o}va)S_fa*J;Z2|+WTDCd6mjRe;AlLB3Z3y54 zC0AkJ;QRyklWu+#@pkEVxYdwPkW92M#nrth8V+~QxUb-3{ypIhjY0fUkUHV zREK0SuX}oePFCQ-Qep%^JCHztSOZbID(mD`$5OsKZE{#-MP-Wcz4whncK%d_KVtdRJ?@(O_=bvA%kNV=6`~p7Gpw_~j;EJV1gg7fFv)+&~jHPc3RDLO$ zU6s<}G=^5Jq?y5&N*a$}7ziMC{go=CTNt5Kv+;X=Z&w#gz5aDJ}jQleI@7^RH$>~8&>cB-`_n|X~0bi*mfg6xVUE@>5v9ZuR zI!`pDk`w-GS~QLV`flpsa0vO}Q8*5ByL;jo3Vc*ejWN8&2O7j(aQ2Akw)3NJp|vL+ z9O)&;(~t#3;n&K=qo>!I3R#-dl3I#drIx-%Q_;_k>yu6DXxY^i|& zk59X}hk5yk(zdo~ucqiyTsbp3fpJcjz1!dl?SIhRhu_w_dLG3Wn$b6R&Fz99TP#%N zf}Y({MZsW;{!QXDS&l%Tx$N-G`B|fvAJvqnEy+Ku7*_E^!0H4&UJRiW$HdYQ`F|4{ zr$BawTaEBOUE)uR3hze;0a}D-nS5APt8!9H;LiOnf#+p> z76~N269Ue|n|Ey;iG0#4a+dBZjAIe6pT`F8AhH;=veR;AF7TfHSLcX(PXnYpPX8b<$!zR3e4L1^oqE9k|mVlS=*9gIuh- zOm8?YCAy4eVLzfS(cH80+B^>F=8_A)?Acv}ZFjfBQXzm~1p^tYJ zvVWy)nr1H{zv2(Zo3=`l64JOWw^;#l0i|k~&@t^y3RUCEK=w)>=W{(V(R8-bD!p^e z?Z(U@*@ZU4fw~?uX##Ob-mRt1OMLzsEah1cfDF4lJQGLy7I-ytU9llbBKHoGQLIKc z6t3aK+3VW-M7 z7!^bJ9H=>^$`~gTCji|V!HM`*maFCfyFI#!iR4ypM+$8}Da^SR!BaDe_K0e7WmH|% zY{PE?yg{j~qT-5JN{-gtkH{0!zshpujgphB!x#A3Qf2n@O|XN`Y4Zg~l@y_8Z25*1 z#jHEOZ7Kf}>u{{L#4pLkNKiI0EF|`s>-B@8?!n21(L3;2&|D3@Y$AvHQmug>Bn9-_ z-P9(|;Rv3@&V@E`66Ek-8m$*C%vACJjp|JiamojeF8f)P^uyoGMM?dZ&&^J; z05c{$X82n3BW9omzDN3_;zp;)M5s~q&Axaxvqkv{q=)IOC2^SDB<7ranch^aUz@3T z*8?Y8Fj`e;lF=W7vNjrH{*g)PutX98JoJIEPb*1WV3F|I%_)F zJWab80js+fMW3-EmY8Dj`*_OR9^FB<&9&bp_k98(hyMdXR%P|B=!%OX`7jLTj^e`ffIh?@ zL+eSUQh4Rfnm~(VJMWjitI~z%19ch24E&Q}fVYKaE+?c!BuF?bMIuwx??167$C5no z4K!VVeYj{&;#*t~X7cPlxb`hwsrSbAHx}+@f-bL(>=H(6|3&kFPl9lYDjqT7De*or zDF@r>N-CM9hggr{A37oF%?$(X`31|18)LLdtj$@!puJN+MG5*w=eQZGlzk;}3?*pW zcYyR`X}~co?H3{~&v)bsfm(0S&oC;i)QZ3BI1R`;0wxL>qnd-X^;%VNj^*epF=qvZ z9c3mQrjNY9S*A=2z=DnPEdQU#n4zl1NV8OJ9<_NqBJq^9ByyXAZ~~0@xOltom?Nfd zm+8uGu&B%ab&6BQH<-O8h1qNKlQyWaW6QyQ^HWt1`EWz}xE!>_1ZZPtU93N+BUlp= z5irKis#u8NZv;kB#2vN_a{l$h>qkP4mj6iaRaMP3#GO-kj5)ZBOh+#B9|HP`Q=1}> zTmnSrZ_lWN{L{xh^ug^Jgwktz-F>q2IyX4Cz8~G24xQaewNVwc$FvBufMsziOL=pF zIM0G#|4x3%m<=MF2MNc!X&K1Y?7cw&eGq=S@m*CG2}gxfi4Yly4Vk2&B=7o}J+Nj# zj6llLnI)+!at@d}uX+$MHhgrn_FyiPv=3{@N6L`U(MPkc^s-3fSi(lDDFqB;#4ecC zn9tInq+n;|s@&wnYQLV9(Obir35wp6G7tg%&I;}h3kD+Oh`Z@4Xn|TSvmf#qUs&G! zLa_>#I4owbS!Of(53isw*D_IS66ib4A#}F~$AX**jizD>j$yfh8O3D=eie!f49PQA z`@gHz?0O?2s9M=8cG4s#-LJ!OwpjyeWQFh6LsQk!b6tsG_IgB5G3Dn@#6I-TE=A5z zOD!?H5Q^xX`^3|Z!?JeUo<*&)1=!^^ z`Nuq);F*a8<&Fu&V7g7wLZQD7tD@Eu&9is%8PQMZ9jBCS$yW5X9MP_;dKqUn5@eL5 z`oJqsjf`)tV7}(0BXMUCXLMzy3O9+7NyK|XS+Rb63~7cA|mkvUj?|D_vHs{>|DIWBBnp(@_o zeqFKSn{XhC#ynkT*wrr1*^%~IpYsFS@;qE;kp9u(17uBJi#wV5UHN!@b;wU=)dzbb z?zOm^iBnt6Q2eL<2gH9csx-nA9bt_wni^@JveyF`41s{_qMk4YKuSHavg!h;P&x)> zlBP~X0l%X2KOD4QzJ)!;O-_nw{{lb5(-{H~zTp0`nno(;WTfRJQ~oiF)H3~1>?2Q8 z!TN^0`c&AJR6RRlpqwI}OS%8?tj{f6_|+=zBP|9q-1f^VE4Lp@K!h{u(p4pCu@u9k zdYZQ*a%xq!F1kps&QZJHhXWnuK|9&2TM(YD!Bh^|b51Fm#I+p2h#{kN^l}trwVn?y z6Y&Ppe+1r=m|`xJ{_9yOf;AyHoz7}7nEF!t#Opu|=dc&&xGuZ$zx#b^QKHLdep|Dl z;(~xk{ST{@{|(CihoR@Uc1+aN#?<`(1ZDqU5ZWP47+2I~^v@hxPZM_=<8)+HX&TDN zPQze9j9Q{0_CfG{EIbU6dB&9CIYam1DDmN`IdaPkpiDc%Y4<$4C7jfEdR7oH-?sfL z>@)1?p|R_EQgg{nBF1}scJ^;3$Mfd3&-2#vmiPUWuf8W(FaC>X9DPiHEHXdm;z~Ho zRhwp@m98pd`Hm7qECfpLzK|CdA&hVzD(w?#B;U|)h1-1z>l;>nj6s%fhWkjM#UMq% zN{|BNEtOZsHecz^>n|7|^04VWCm}Ba1H~IL2m{!i2_c(74bYd008Gt+wN$J+z zL|Y)<-e`GH7KfWOm1D`9T1}%R5{H8by5b^A9yN#Av&zek5N7ZOmlqj~fNOx7}@pBhBzg(D^E&6Gb2M;u+Z}jL$(`k9sbFgen>7VTjz#M&#TP zo==G}74ieDS^q%M(H{-1n$;FHj zt&igh-N|bHa&urViFG1m(i9oBNw(vLT(wtX?T}7aM4NgA@c?srt+izi`F8!%%a`|W zYR+Mti?Jgd9YgfLj0b>Dk+YR-Iy_jJ@fNRdQqg4}-C)4Lu^yP^QD`R4;4>s7fh5Xs zS<1RLa*~Otn#oAqpUETT@>A7vjo4Tk_8j8XGn?_^>LyeIH*9RFw$;anip~lBJ0`G) zQxK21Qp|7i3Wvlk*`|4#8t1R!9Sd6303qmrp zVrFB23=ONSq24C}KO28ftUXm6of4lz)y?!*Dyq!hw(bZdw=q?XGAyDtHcts5^NMrT zp&G}fY*@(C$$qnZW;=mK9hB$y0a_0{o1|j8e<)uG-lXRD0|PD9us66YShR-gDUKPV zS}Opoof*wPjDuv~Sx*p`)gSE{*^#+^onem3T`l^B38>d;n@UCfF)VRzWjbIJ@du}mbJvr_~vJT zFot$Lk82p_g6{u8**iv85{F&B-LY-kwr$%<$JR;5wr$(!*v^TaoERP3=p^0I&EUOv z-kCLb?)_SyYOShO^?&xW_iz7|Og`}^KwbNbWG2sgV-yC86Zt%!RpYzst!dIAQM<-t zYN$Y15P<)w=I)dw-7y4b^s8-i>QHJ-i!T9&K;PABQa?UjAmos?eEYhw2`O+2MpxIT z7$Sq8x8FOnW{$naeryPA&MLKM4(f334&m=Z{? zVR1Iv^+L~5s-D_aH;+0L%oP6#msEBPn|fhiVNALtQE^tftn?p^el986PA(_wCyw5_ zP-S^<71`+ksHjv5o%Pdl)z3pr9u;oNjx1fS=5CWZOFR7_64D)CEy*qg>2l4XN0ZhoN+pJ(+63-5=B z;-(vilG&Aom~GA|BS?okLf@|=zWjrbyP?wp@4`djfU9rL6j&u+xpUDU9w%P(JiF`X zG^w@r$;XALnRq-C((Ebx+|16D0;@{49`=lqr}S6TsO$ok?GR%@P;n_@KD61ojE*rn zLe@RAbPm-ojwW{ANds+5k=#4-iIsp_txh5mw#iPq*^!`ZTv#SDjpH0b3GGUKsT=nr zphR}??6VlxA5t7{c0NU{P;3y|?jub9Az1#XG(-Bmv;b8DceRbzzDmRdW)TBSS`WwF zqIsoq_&9gkKdmE|+(xRZ_*H6uoQDi#bkm*Yn_J$>9;?-dVBa$lIyB*8 zu{0-Y>%@7&os72nX^h|0fG9JR*np_J7A|Q#r%ntMQtZ`qMV?VNy!r8iHzJ)bNzi3VhYHp~lD3hWn7BSvYRz8( z%MJPYKZoXt$%$o7zli_YUs#m?o!R*RJ^}r2*V6xCHZIe!QwFKv_#@$8a78FcNRId% z?T0}@N~o6FbD~X3>0qJ@j~6lOH?}oj93Q1beSAaz`<=@6t(IyO2Z($tjl09%k|Wh8 zvnZ4SdhYR?<8^v{A6r%YhMlN6r`Q6{Zqt%c>O`4rMLU9(lY<`Smg_alv737AgYE(TVbCgYh4ZO)-c6A zp7u>_ZwfevesZpv2wR@6u6l*gGd)kr4Kgp!Od^iCGcv-w8Alh@V7NR8X>R3;(_4Kd zQM`>exCxRDP8IFy%G^j zm2})3zS=vwTzos3)fWit{K^!6gN{769NNhKSn4e!I-D69acYEb>syMlFVzJ}ILfLc z#_@6WYZ#cH=F%Uw@|*uLb<8Kq;4qxREq-^z%E*zmTt*9%Z-;vb+jk-L$rYMl@GEB6 zr0$ZTp=iYJ9ey466^q5Vy@p>qZhxe9^~KP>{8GX9eTGkRxJ28UEhw9!(iM}_^tdFs z)|+ga*4s`EhuXueLY-o6(1e!nyTg>Z-=RyBiWiX!(fxSBzVD0vdyPd`RZ9VX4+>0)1$9V$Xqt7Vn6ipA{^_C@cAVb4KlXIfdGO9_H z*2;t>d81av$@Io8k#>#EcQfp2`(815Rem;@hjGxVlm~s!Tx7! zhv8i_zv35k;+5>%H=h5^6yadv?DAiuX3YNuHz8*7)8c=1{(cP&o*K)TA5LxDSpsdL zj4(VjLy2Lk5(^N6i4riQ5-BJW^p6(DlwK@Z*j7{`5)O4OZVoleS}n`AH61Hu4kX3F z_)K*h8(kW8?x3!vl`hvJoqs#8mQ-025LB(LzK6e`ulb)DK6#&WeU2vrwqL&mdKHhx z@fqj`UfdFLr|)_5Ztadz_^!s?>`DFgPN7(*D{xG=YwrNiJEqjPcLSjNZbrY3N5Jj- zBO+}7-BIK_azwwsYvEUp7)3uEJG6mFh|d!a&+<>m<0cgZ?wH1GM5#;2HlAy7A%Hsg zgz>+syqE5Ktk^^u3Cc2#%5wL_*t^u8a@YQPYynb0Wy+8O*E4(`Di8M@O}!<2ylS&CWll%d9(_5YbQYcBY-?&XL8R&ZSsb*hshxT7)SRe z8ttpj)K3}FesmijE^O57MRjDv*|?{5M<{qM)IJWygp$yh~Zb z?J%v_h^YTeU&}!v0|&La zk$PhrU2qV|19w3#Xo1jDgVpx9k4e06IcInrX3hPXglp=m&6Xd7lUs`Zq%;j@_cDgY zQm!1Bk<8KGtW07C3wFH`=ikh4yIvF-E_~NJN(0OImp?kzcyn zI^LX=g`WMZ3A=7v3){9GauRtL9QySLiPs~ZGA|&nd1K4r-`VKm^q$l5t(6okA7B;4Fef$sbg#eg}R#C5@B$kQx~6zdIihw(*A*rVVD`d*yKkK z)Y6dhYTaIT2t5J~B}=9O$HJC+CmuKVM2^UC!y&x9wx;NX_h4~_h_>v|eIT`bi%*G+ zWQ4*vq=8zM;#8@vquxMl4lzO@t2l^~FSmNYwyae9B)oz@80d{Psqt|k7Y!~AS9~Al zGD=VMM=xfQDgz0_$1-awcrR(m)O~?{XbE+8uK=Q%`h={+rzE3|R}JbVD><7o5{jzb zVn^_FN^+<#t`$0zZq_(;+Qj{MY%LWP_1f=i&*Z$U;_;!Pfe9X7I^8r5jk6~k&jd{+ z;17*XpC3SI21T&APufbGv=k8!^H~4|Yg%xBDhJ#pyhl0i1^+Y9bmufzxm%3Lz0 z7Djf}tGJC?^;Mv;^jJ2P&WpkvN@-QHF_SDJ1VV21HgWZ%Eo_F}hoRV+FJ=q3D^uYgft6pW^tJ{rBF+f1_%@n%dbo=Qwb@OTZZZ== zEhxow%8ga~rPAm4L<2hlM6R(5V>pd2ZEM=c6ptn??+EV-h~?43##CdzneVu4S;^^R z59wnnHyJTVvo+L+4oltxbm>rFUF>^KbjcyfOK>vRk>4aQ$|)D1?SErKT;ils!OG0f z1q1HoJG!%kz*t&n?Rlb394gT`zsS@dPe04zrcC%**Ol*hp(>#s{VDIxtGG&i^lUyO z&?q_HM&TLF=os|X^J zyE>8%ume*uD1Nh_;TfOPtrHOEq}v-WsmD!x))J9ss3tg()~z~OT;ZCJ99dVQAm-(? zCl?>l@huhXXo0iUHe{P@04~@8+Q4t!B2AU6;av?mADzmqxZZ*?2iC92{-6Rd$tjRW;T1{B>3(!Gq;(JHasd9?RS`|J$_YU8rA7U^=btJO=>m%3D!5H&1sW+4B= z5=q9mfE|2F1^RFOH!=4@f{VY$2MjzX>GmNVImo&^QD*si43gl|+oU4oO2@f#@vX4y zwhwP>mL9UgO%QG5ufH&m9-slvCAf^a+PBOR^mRH_7|8_6M@$oq=@+`W(rNb6N`5xP zF0YB)150&6Oyz82^rPs6rnN&ha+26eV^-s)BB6rfIq|uq9{V`|> zF76|5RHCBlj8B8dSz#wAMB5>x!dZ$K2_W;&*?Lc*Z$#$k!o>FqSnH74%l>M;3ZZ=o zLZp~v{K9qp+nW)yvA-V6wXfZ}MALun)iF}2lZ*a2IoV#~N?*>EDjZqN=$v*VwxA>( za)0lT+5O9nK2KV*t@0jTZ_0~KA}Le)lp)+21% z%XP_xIUpSw#?yF5`tow>D6gI*otWDXNG&d4vnEVb7`Y^I36um=!dn5B#p$GnsQdvP zd5Vrm@8voqjA3mFmK_|qC3^;l1)y6m#1oqru#gz~0V8i;rGF8*`@_~Vkh%e~%t6R^t%s z!NaUx-z3ig6_@{~VDy{a4R{lOb z^MYSJaLDiW$Q1ld33UT_e)Ys6c0P$H{gB~(XBhvHOMRyh5Z6<4+BP*Vey}c})hV1% z;UjfUVX*WlhlzGj`OK|>qBs`vLUrbXZn8gL zkXP7GCmMixW|5gn>O`!$xKy-NjC9m_{)v^9<-9~>el@})#C)t?42Mk_d}8b3n*VFC zed*1i!uTEkKaXf16RLJ)Uy$m}uVs(a|ECe{e?)>%{MX=&q>GD_i-wDdncaUd0?V`< zl<}oe{IOVsX05=EIa8@JpwKocxD-L5p&YVac7;?nL5YjcM?GuM$1A)AMFscZWPik$ zW$H9IKS|ZV6v;V>Yo7GW?v?d4{Zu#B`9ox#znD>7>v`*)R+jmBy?+vv``(cTXlpM{ z$1d=s@WTB!E*Dv&8>t>m41f|agY`v(Y<$I;NC(y7c6@p)b$g5! z-uhz*mcUHVLe)Th4ZtW0AMf`Z>tP_?kTZV@uEi=5X3l4dN=H|M)b&oNBn@18U>YNOB#<8U1ktE+- zvlfQiK;zS6El=|5ChpX%(l26=5@UxSC50= z)1#QAN1bQ7{%|ncxuu876vBGOGepREt0tQ`HLgypvvmNOC0~DEOy6W7Us@mQ!c5~P zbzAHF`Xqv+&XdLgY&@ZlcxfiIi|GH?QC%OUunFNd#y;g6R{_5@Q5GfB`K$Wg21d8X zpcstb)|(|X>J9fgN!oR#0rIa^bK7e1CtXW zf+|w%EGwVG!AkKLOtYM(06^Bln!QDn)V&wK_qvwHWt;UK%A99E;n+K0j?0h^qvkwiQj0@loK{o=$A8gzD5%y3ue;3ge!yAS@qm$y%e^Z?r)oa0AWQEuVIG-D zX~P6qRmkN=+zQ2ql6-1V(8x)r*Cfx@DwC90*&^i=VF1C(;pnJ`AwyECl1IItR~KFf zn|8bfyyg~_P0qEBSeSg(RL+6!hNYA|(IF?!;%2vJ21r2T_}hQ{4Efc;F(fRnGt0O4j${rwRrWyAm%VJ)Of zKO$HzXSNX2dsp?I&Z!JlPI6CVy^(6wTr{`9=RpW5*hcI+enXk*YV`;*TogejRQoiF z(w5@e`QX{T`1AOc#6bz;=#D};PgQn7XW&JtiCK0k+yg*i6|#W(wJ@~;So0pYR0J!6_(yhqf9pF zNQj)JmE#P&1{Wy0b0VIHG4S{{x99r}XbB^w*;i(a0BJ^X^QwF}YbZfX+J$2GdpKwe ze22cyf^?-zRvW1CRtbAW1tI3Ofm`qW@eLc7)n92#>P%G;C~;vq8odC?b~ z8A)Kc$eSi&)Apz>w|02x#y9M&1MG93X|y`9wE+qGVIFFFE$Q1csZn)E!GfxAiO)wl zt=R1RGO`wQ-f~Gwm@yloUo9-Hj_|&X zB^F*{hjDAbZi&RLLuQ4FBXgmx|5jHJm+3D*0Jr|dNAw5mKM(+JL-;>`2%nC8;S#>| z-pBtJz4!mosubPqU2XnHp^DM%t^8+8^J9(pxl8wo(E2>wrPTiwOh`O%haCZjNaxJR-5;D zqG1MSM$G_rYMUw*W$RQPNh}@4pJ{pRYRK>jv6G8DxyGSHgU9N}kAZ5}kq2Wd^%s0! z5Ng-=xnrfUSGBRmObz{}!9?1aUOA051AyTkGhfdFN(+1Y`CrbM)y`hZTE-muYfnzQ zh*dvTM0*oKcmm2*Fiqs8rjr|I&IJk8*S}B6K7TCN$12KaQ{>HOg1PEZ>i$&u*ovjK z&X9+ha+j!wnOZxA-|vUm%KGGbU&%|}S=%>0)>1F2Xb6Vl8x=-wIpmPGKaOI#Q3|#v zjWUe%?Jir;=F)X$I9dJbP&60oLI4_a;Tp|mobV;6oE0;={{4c1PS)gWufAZriaGQ+ zdThh{>NCjuA^t@wu;N&dAimC)AvY_=&f>Uc72vrFV?`jkN;bLdcSvSeG}Btqds-C4 zXmMMC^hm=Z9JiBBQ)KC}ig$ZvgCoEye>PUz$jtsaumR>OFs4;z3bZ;`O`%zR%;W107+a&Kt+!e07@{`TVyuf80v{ypS= z867kCMc>(v4$vR+pz!66tgxsZd8Y9FF}nWpUHBB4@=ZpH9!chf?#Z<~AaW9pL zTo6N3-PT?w2B`)Bl#%=^*AhqcPlc;4yDQ1wq|$n4z1Qy`z3%-gZ}VR_xBdFK%6hB* zc8wIetqaYDUHP9wl&c)^SX_0*Y(HDFnuU%)*+uA0G+Qbl%iIW4c_jZ*4Kk%Bdk)p? z@z9?YDP#-jB@Mk8Y5tU|+I5psX^mzJ!SAwCN(xi<#@7H?MJ3-FUc!faRE&VUK#mIUCrd+dfwf9iSr%wv6L5Pi%@m=ZI@EQmhi5qIikv10>(H~KE!Gi9tgWu~Hl@p?4k_8O)GZv^I>eV+(lAuU zYZ5X9lf_4I+M_KP330KZ1#Go-1^Y8u)GtZ451cz0lQsd?aHe-3pTY(o*@n}|>YSxn zOp2EcRn@mnkOex)|-YFI6grUkQS*+`@e2lRSTm{*bcP=TP4^$ML~Y zBCoaKk+xP3CsT=WEH;S`Ls5!dcauSfTC+#6SH!1fbU4=w-*o01m_lE`s3O1|T>52U z+Gf^Vo*(I`sy<{$%HBRXuOfh^Yva*JbOgDJV0Adxf|atjWeQ&ZS3)v+_r&NFXZb!L zNb>~GG@Y7C*%|87s3!e2ne18q(E=lY=`2q}SmWL(rb7!2LUm8?YLT6)o*FE4g@Vn< z%4OczI=hICcjNA=2d84GC7q+a`j};K_?eQs3bz>NOv4OzDbCs|K|MRtLN|zvNDIwm z$F90j&|K2Qum`Rb}% z@Q$&8%b2csaoksVevGv|fL<>-R8KRVD^gC<*m9Mn2&QN!im#L#HC7YW>uL!Fw`fG3 z6K_Mv*FQXDlwlVdLE>P)8kzQ1KSKGYH|TU58|-+wpRhxBI^YGz^&pKsTOy)y%P%n1 zFb}Wj-8=E ziI1{`uXv2ioL*4rB;U4jP+>`K>u*?gUd}Q)nS}5Ak~!TA|3}eofoDGc{tvj1xK9=c zZle~zgSKl#Lo}nr+v4EQeM-nr;1pauNQn+i!GzLN%cLw;b>#fSf!eS)Wxby?PwV*( zt&oNIL7t(IB?^+Z9TUrxjk05C5+>gGn8u!oK0_yL=(tfXOtaDUT&pfc+TLvJPs@<4 z4kHka$N+UaveYc(@Bs#M;8KPy8RJSf6uNTgn7w}pE0F_BYYp~NB8<^I#%1GZl_5NZ z>I#%4V=ZnQ9Gz0t=^o0jk&IE#9>+Ji$R}Ucg`u0FFp9;V%PT%6y_K&S-ds?+>$W!F zJXo93YKX7V4Eu8<*Fs0|RVj6EN!w~;sXRN3!3~m8zFm}hhU$O`e+65g^2On%&+5#7 zx?ipGHQyT(t1^!{(1Rbq7fIv9ls5^#RKCG(U0U}QlO3}P(bm`1o3qn`T;2=T_cQvH z^3#pBtR=4R`}Vvbcz0`N7PNyhYwqPyR3r!_G}JRkIgvdCVJu*>X0rbhe7@1eEpJsAlsV zwX|KeG5<~v@(&O@HXY#ax41fogunM3Pb{Q#oZv}`sh?RVTlaZG^>PA;hb;Gya2sS^ z#q+`JGL{#V^%xT8lf`?=L8jD4-&YMOIbUoGwy*!Pks(!Y**O$TEt;C+F~?kv&o4#V zQg)EYR)z0-=(k8btREe7(I$_ctH7BA{9TH zGu)BlpJRCNaJG7c=QK!A+`Stn(IUZ@b4V7mk;RhLw|E-#Vf-7NfonhYW zuZKKCed}emxeJ>=Xa=?yxMiq2S5BS~eGvp95X0u}DiV!cYtcF+?;IXk)RFxtr5<`H zX~$3kqF`Yp*oyFIC7nDR-Y$unTr#lq5*@=~DyHNt2&Us$0YT1iGm740rm^r+Q(e<{ zP(T*f#@PmZADBZ^==?bf>rpl5(9wz9%tedRJ~92+QB|3=K|rXSGsZn?Dq+~-Hb*pX zN#^Dydd*SivSq+p0KyctN+bJa&OlF|S!Hc4o}}BFa)RnoY>_+01LS7#7G9W*}UYK%G8e%9baq=B~ECh{n3U`S=N9>o^zb*P=n6UIclngREgXzhNH;Bvb82v z2I4VVgqb2|YXW+lCo{&1^DGthV;z%q)ouocyM;-#OBiUN1(w_G3u`tKk*N63!U;Y# z9%DjdjJSBN_+icGLQ_o(bZu{fB(+N&P294a5GJ6+Fr?E@);5PoMyq+Rc= z1pHf)-_aF!?gAk>w*-!(@Se~{LyXf2`s==1v)RtjcBB4nh|oI1*QBx@jX2kqvRm|E zr|&dlZWt0!m~r3M8Kniq&$7s>W9KvWT#C&Fg4|?Mgh-QpanDl2BxX8fH4c zw9$r4+rGa>0wZAf;~pJD3Le)jNK+o!_$KhZcG<|WJ0#YF&5j%ts8@t$?l4A8qPk@ z2~x8{RT$Er%$@#@`|jXwTThPZ#R;#tdzlw!X3aEx8!|N8x6|2`wOp2xRoEnCZ!P|B3qDXjvcbCQ*F&p+P<=utY}0!2K(u;ap?rss+lKf>kO^ zLvt`JVlrbg4CYe1?(^b)cLZLwX`OUTW#I3h#vWY=Vw}9m++gP3l*p6?)c;TA*PJSo z5usnc2waM9-#Gq%i*Em&P^0VRYoP7+5zn!hVrkSf-e%7WARCL3N-jf8Hn)((oWr%#~Pq9WL}tg@Rh%EdRp5)+TNIP8c>40ClUrvdo>8 zV<(J9Cq&<|0T0ACLmX9w)jgko@2@N$-#;sR1K>&^ceF19R*s=j-{6fZf&EgqPA?vW zZxKPCOE&?yFHT`E#Ds4ux*t0**>?d@MoTxJ5smoFxE;foL8NG-E>&=5^%{1}7)f|l zagn5M8%y|TD`zq~3Y}s_>5}H1wImL@vURc{Tr~)~gz21%*5i7ElYJWAi<>uH@%)#1 zcj|5QEU#*`n^#JDS+RMj8qdUiiu3a~9cGOR_rvjbI{oZimfhVjHrOQB4<@xOn8G$= zN~4T(a(rfuu=H0OB`E-Ycc1nn#0|h0Az@3M>gKjV6R%D9LCPVhW&4C+?DHS%*MzypN`L_*prTxOuz0lJVm;TO?s9s zmGANAK1;J}jze-(L~3}OllH!BfPO;q`O03wl@w^jOU>t+EH?1?)r|AK@NK2!zP6r# zt*)JDMeiu{%fvLQtv_Mx-61B6a&aT!0^CV>o0@zjY}vD=(kFOi+~``Y9+2)cFC;;+ z{@Qse+Q-gz9@J{EP6>t7U9N(duuFVoFmC)MZIjtD>)3~{EE+70w`P1Q~`YwfLQt(M$-8FOX}I21(e*r+xCgTU(>xRd zAFo{S@3GlbMd9aeTd*-*+umNOeO^4poDg~!v@l8upAt5)zvu)I@?s>WNjI+=e+o6m za6)fePCs=$pqcx)($c#86>#U0Z*xd1y#>A8^*NlXV1fiI%;t0C0D~qTsJD~q{!BbxX9kFjx|jr{;3_&3CDGz zv}1(I^<12QSX?NtshVPI3aj(X<=fm^Ud9|qr7b;oA4^%Mc$g2qel)9uWwJ7z$=)Ni zT8!K5h223J;Vn}yBUedKbe9<=;siiTDJbmLMgel!l9s5Dm}2)zNwu?@M1OJJu@JEW zkgqxKw0dR|?G$X%6(x9MVnzP@3w+ zi?(t(EpU#0og=hIsqW|=`^)9_DM;#kl$J6XwDpNsn?R|C zO`EAX773*7nJzV#oE8O49yyETuQ8#z9p-E%2cfjRO$(`;hwkeCG#bEN60dtwS%xNS zrn4|>8HH3{snBJGH|;N5PmxMyIt6A&j^8%w*T29}=Hr315l4}<^`_#Qh}k^R-3mQW z^_=G8!~C~OgUg+avh`j@Co{k5UB$x{J3}hjf)iNzzDPdhTUERGSP&NEwAmQ zp{ve6P-8J8+*{R-_vxcRNck3Hy=tQof5qU?H;YL@mm9HbT{K;V#={P^*c>)GhVRzD zS@()M7gHgAD=KN0V~)79S{Nv?qMdchWV4adiW3l)D*9^7?aduvD+q*-Wf`X*--T~k zeUV?gr0EewGZubi^~izdI{OIUPlh|A;HWbW63`znSco|99{U3SVwr)%s@u}UFHiAnj9@=({RugRJ zi@v?#9qTU5t8P>-mZv|~@WE0?Y$)5c#2x=^h9d$X-S$H8kBBTdgfoDBHc#x<9vu z*MspU3n-=tV^VIW`Wb4e9N2`d6R;GRonNPA)tg?yxR`o8Gq*HuY}Yf@40JP0XGfwe z2IDN|mH2;ERCk)0KpUJ>GLYyCiKWV+ZK1)1yme!#7*KjVd0jEhS{^;i3cmUSQvT%C zl|}a9BX3YA$8rn!;nl+6$gU$lma*lhMYHP>#-^|J4B@CTmEx}B5WR#DZv}&J^&h8e z@J~=4sd9*Uj=>BL-hRsMVLnH>aCKdR8(5$*MeUVAIL6_&YVkQ#WG?X2O-inNYdAGZ z9>b*)86d#l7dO;`;bUM%fY%1~?9;~reF#b~1 z?)=r1G`^e~2MOcl1Uo8A&%V1k8-NaAiz;lg)g4Y~g+u)H8gk_8OgmPQ>rlh4x5g>; zI_a5kUP4ljjcN1K`RMc)3UVf zKkwR31H2R;`g?f(T|Dm^!7%g(5Fq~R*?smRed;&k$8W>F1tLLxJY%;gM$%7q)`JDbN;P-h zvn!%T!$#mv19Q$AQ-84D7&asBWVDu^6N~9MS6D+h`Ngk5*14k_B{onO z9xEKn&U=&PC_%rnN5WD3S68!}c6OUnsghpTJSO^1n!Bh;a?Ci7n<@R3>(q{(Q_pP0 z+V;{XhgGkSBBeXgv~huGT`O@7#YNlk<8>yWGbg9uiJW)|9iVq;cK;%z1#o^dldK z!#j{kvZK+kH`;9dI6V0H9Ynlp?lH~Xa2g|yD`q$n`n#8=$YVFpK20S#wg%-wrqUm) z-q;WH2`dA9deSw}jvGf8O7lJqXwg3D9F5?4sEK7FmLeD0zgd;xL;Vnp_V~+l(Zt^` zw0jX4^yBc2If?&AXf@QmrSlc!^2pD%B`oI#ac9(3WzuxQox%fcr9CM^I+_6P+5K|c z9&FG*qS3o^n@bID*mQSIlU*=(;$`w)02cSf-WDhTsui3tVxs`aCRrG=<$b(3#%z3} zD!2(+^B~#=U*--CR-I#bf`1I1F4pb<`Co09*KYMSr?V85Z`eHgA;l5eiCrmvOmiS9 z9PHxO9%GEPDz-MC)`Ok=#z|(+m~D4cD57_5`^V5aKNXohz^EG+4zjGv7ckfXuHO3j5(*D+`^s-C`XGVgZw8rDLI4{Fy`IEAC(eYw6OiZiZmeKFNc_XF zy0V&yP5n_^?g4slexd%U@t-Ddc&tQbh3_!_+s~2M_m_U)=lV>nvzquXh~WJ;y9l9* z%S{>1aX~w#pY1<}k-+<#_aiDhORq%-(3sz8W-YhhM{BzyzzcXNcrqUTHr({@lq&2@ zKD;{T@4TKkj~EdZB%~1(*dSmK7yJrKK!z36N}t~=L#qa&FWERF)!!G5(<2uwCV}y} zAF;hnJ#N2i{j*8~3xIN6GuAosIMv7zSG{ z17nK`{=f=rJW& zq)VDX7hjPfCSMf<;{BHfh_UA_aW;P@pr)Wuf| zVO92p;1@QKNc6=k9-qSZJGWfr5Vl_jEcef?p}Vg}dHKwc`pCT+wCYA}d;hqD3nO1( zjFX%W369pDKHBlV7V9PYKY#s1zp;q-j+uy?{XiTf7js|h0U>6o;Rh|ibz&jWQHLQJ zaFnNS3;-?eRgkP!uR`91dd142&Z&lAL8zZrlcutqEg5(xK&0_M_$8zCigJiRE8mlRgzTIg{ z#?3a>;147Pw2qeN0kCwS9(8yUe)P5xtwo^sr9~KyH2rwv|)A9 z-cS4CKNVsZm_GxTTprPJPS#UuSN3gJougez#dM?W3pHJrZ}7kC-yDqyU{hU8|Dt7 zsZ!_}yoC!{E>o{zvzFF*?gI+OVqjhr2`uf8=#6|eYa_sko*evg`t%(^W`PZ1e*1hT%o***rx9jO|K z@GvJP%T}+oZ(%(@4rSZ{Q}{y& zq}s|K2TgKI&@kqn@7NRNh-~gAIzKjxeI7?DyqnJ5B$#Sx&{A1NTqNB&zQrm;Gk$0? zo7TQ#c(wYt+8LoniCyZ_Hxq^fvYw!WSGMaxR@^awQ-0gG+w>N!glHJOdD}{j1+>OM z6J4~1rb(E`3#kFQ2WOE{KOWKp=oiwlcQ&Gd1oJl>;Ub8%I}(36PHK2*@wP)w3C<7~ ztI!=$MXo0byr1U$4{IolQd~CQkt$};6LI3DlMj~x9xt{gNvIv^>QFuaNXBZpU zCdW^bi{x+HutDFwsa)CTE2^Uc(c-FAO^$BN--wteUqWy+deldW@2KO2r(WPczV)Gb=YS_xN-H|vPPAj%2mHHiWMbC4w$Oj7vi8ntMWGkw+ zbJ1#xMak~f9hfm?_)j)rnm{LyNZud1GBgNFS9+7B@lN@9YrY^OA4*ju#6`$NK1{Z}%b3a^v(LX4i|-ZhBt{pPIa}Mt zer?{Ni7^7pPcIAyH{uu{dn2{@mL#SL8tAb6mUjo^7#$qM-o)@l+w8G7%ZG7I9ulYB z7Ywkq?!zSC9MwSuZ?&6y>e=}mfN z))nToaz>kR%nz6$8VIn#3NJ)jtR4|SIpEkp$%nL#&||04tBl<#sZp>;Wcb8;e+f7diZbu`|Xia(mq zom#88dw$|t3b`9ve~q>Bk|+7n2%-74AtY~r^V0v)64{t|f!G+iILz}(f;KrbM9TmU z76`FRw|^r}-L|W!10GNG&kksDyC_G!u@HoIIciW_#trK>Y(iM)#RAW5D##Rp-rNYS zT^Q3+H|lm;o~0ic&y7y4)7*|?6)n0IYs9xVkkl1sml+V{b%s||XfXuNf{-}j^+ zv;4;bl{<`5x-S{exRN7=E{YQRGdGau&q9C1p#%yg?Go1!m>AUz&($AhzG&XOm;_VD zHO}z!-~SMfd2KRP>sKLbF?F?hul07i3fD!J_;fQ_rT09_?>&c4^f4*{`1_^KUD`odis+ zzz&r#!_tZu8&_;Dk+-fcc?cmqXY@8Nx&OK9F4o)3Kjy1&#OSN)j`ja|2>UOQLk$yC zdka}dOQ-)ZwY>gB*Tnc}TQ>)}>i^r-oRr6HRn%L7)+h=zhF7};x33bASSy$BPBa&9 zDuU^gybGH=9gk(_>-*;RF9cZ+CksFjATqY|4o~m_JBHft!!5hX5hlgA-u>+FGuwHR z`nDWvQ#YP`hda_jSg@#*UYE;yv zI*dZqYHYZpb@TIM(C}kuwDCu^*imhLXHRc@yZyj`{Fiu^9b&n0&U%m@yzz_t7yydQ ziZUsgQ(TYQCWpEl~cEHwSiy)x3iCr6GBzNqz+;CIFBHobdPb@D>3{}gE0>%eB zt1|CfvRNQn6k~x;OsKIp3@WW%Q$YrB&BnIEsEGx7Fz7EGsiA>53Pl_|!ZYW+4Bdy$ zCx)TxH2O34H<+m9EHe+GS9Mj~r0t2srySl;^X230vFKk7*u7ge7331PmYvmrw1{&y zp7}Y*`i^59rZZ0VbW)6Y!S5-T)syE(XN}pml&ZDySnjjNx|9ojw6Qk%BbN-hRkL2S zY%~b;Fvo>iLch~{HUO%jUnRkpP7G(R@&ctI0v9HpZHDn9hf-`gyfCZ}17BNlp~zKZB_3g8QmVt7}WL&9wA+ zQJoqmWJ@RJYN~cQjIjKSh08-Mlza@MQ@eHl7h~_(ooU#$30CZ+V%xTD+qP||V%xTD z+x8vXwkt_Zdi6|qzddXEnJ?Eb*z4T;I!K5Mu4lp(Ry{;Ud)5x|deJTq8)LDEDA*K> zR=t8KhU{5!NZMV!kBVDoR1`ScK_+_TtTklKr8kC&YriK8XMgU2eS5YR$P7(HnjIC} zv{r7MgaNx%cc|Mv_7js6q=%N!-d9(I@Kvm9NTn%;)QDRo-|G3d*u5VRQZmAv=e>?n ze4sX>VDP#UY|47Fb$vCHB$>4v!pN!rFIMeooU+x7iMT_MT3 z9n;IGUb$^%Fk-^^Mn-|vaFl1)+NoEb^YaPkk>#-*+K2K-G8@%4nEx93+qaF^tQ6C5 zyIDOFbW@6N(ziWoS8J-k0E~rZK&lpXYbLIUsamMUf--wGvX(cIkgTzSQ0>uugjI-9M99^@V+A3X07uZi zvdFl+w0vyASY3sMnHsNp$R3TzoMHEc1o943e7wS8H+$^r0j+mkbBQ)oPUP6Mi42Er za`Qn?)!E=pKThNT(D_ME?n%2s=*-Zcs%xS@mq+$Jp==bv_DWW%W2Si1GwcS&YzW}D zgP4Ebn0$68Tq4eZk+#T*=XnEcKqD>UM3VtR#{-3vaYoLFrhf_=byYm26^{)HoEpm? zjAeFYOluT8o6DWV`A%9ikD;j`$Q1Yj6bd4gm1R_p5d5fHQP=qu%6&S=zXf^Z@KPj? zA9bMAs(V5sdr0bbP&*h$y`Hyk98-n0f6YAyTb4``2%b(2*o0W(pz()4Pi z!j^(p*!$z6Z-9nzSCxe#Fho7{z1J8~KS9~H{C?MI^fog+WQ{k*6J4Ql&4Xm5t0z#O zVIO3TWc0gZ(7;@Qcnycj5Cb`ZM^F}{4N$aGu;y%meq>0NsiK-W7*HqRO7eUp<+6`_ zDIiyxU0f~2iF@Rg?=p?(xq4Xr`mYo8F1cK@LEaCC@uZR9X9d& z`SARSj%k`Dhih?A-!Q%*d5-F)QfOX4Cdx`-Noz4cp{rC`$$Cwi9g=vF-=p)W3&1^| z8HKQ01BqRHq|{~zFOf-a2>%scXxiC9-v#>h3-m{z%=sTT+y9Cq2;1Aa7+Tu>4?sah z+8&wl_jmR-i7e!kC`_`tJ;}K-B5IjP0s*9BILADd#1>hXnvujo%Av|}Kf(bxQaMuC zxBoZAum^hz5+Z@9gT~y@;d?LU+{fqNHs)V(4tCsCU0I_R?ffn$jaR5MJJ+wdTfu1s27!zmVY#Z77n4ndbxA|ds1&yY85$!IE>wqdv z+loRz76R8Y76&AtB^AD6m#iq8*YTsZObj^A{FhMU^6ECF)DlH3vZK@Ls!uu+iigK~ zA}E_VR8Jhegzu;P*ZZpa#YWFg%FDTYk|w1(TJ8)&AFW&HKhv|ho`rDJyFy7e*{o>O z`yVcgB9D6Yqo#ws=U-}U1aY2Z=}3o(iV)DN)Ho5j!yS4Pv4UQ+6;Pdy<=@RujJHpY zI|6QvRL@o>jf1#94k(&>hVo%0V~rZTA>}uIq9@GScv9VZ(r#`Xb98xYL#nN`tdrRneVQEBlO}AWZ9`(&$Mweu5{;NP7g%5sw4`Qb{J{3Ogh- zWOgJEWOkgN|C`qLv$)li0`b?cW4d3zc>m+n{I~Ak|5ergzfX=9l!rDTgDiFe`YYk1vDBMJFZbCVmu{F8CA*9?AbnOTlOj;=T5}J_I)EgLSt<==kb1_01 zr$A{TN@?i=m>joK<|R4v^JZl;i!$b=0x+q6-g(6mlSwtjEW76Cy?e*Pq(VN z@5;#Gfi$S-|1w19ef%Dnv$~Ui>p=f!v#0O#rv90Q(AVGReJ}m|MUDRr(#LieiNN|M zkN*u*SDCThR`?k593Xr{<{vG`OX{v8^4ZnkRRi%C}7n$J%$~&`+OwN=kAbJ zcyGv(NU!2<0_EO8iK-${r6Qq#ubZy|O+RKi{5+|`qeiTB|5Qg&P62-jE<+HvcqPm$ zqk=0*GhBr%QrKRBOil*V$wQx0{l(R^GrNh}D=-$&rdpu{>|q zggk6P1-Uhj!H*sgJ`0IXpq8{K>YXlmcB)aY zYxfINra*DmF1m{C)Uw!7ypjk7I_uD|q3oDKupBN5)ZyAUN&5J%+vzycBwTiu{qB2q z!)f4(prhI8j|^>OQSVmvPEvC)FJmoC5I8(D1=U;O6aIi(w;DgLb`8}TplUtY$mdOj z9V>(qu0`Q_QlF0Ri!CTkB5GfP8rnhRmUyyF_dteS$$kiis$(J|5AS@G%1DQ5X?nPZ4`dOwPbF;4RNQIJcss+!Pjl=ogq1+2)fDU3Ba2;% z4*dnFFk?Zv2BD6HnbRYy5*K8(lvGF)tDAAMT|2zr6>`MtN)`92k!L80P&biDH_2vL zGV_zhkD@H4IpXy)Try$WsML+D13N@FH5^?gG$q^MAZ3hVF!~+oBijms!a_DRU~PD9nv*(JYaX?^#x1gI6SNH;;2}A0T5eYi^_=K7v5>dW zz^O}(@|yX(FQZEL`Boq2JpvIWB9A~;aU{DX0t(6WW>3(q9t8D!=o zr8zGa$_l7jW+B!&7gC}`>slkwR+IU#m`H4Gxtm(bBs}xQzetHWriU3xiH1T& zWFw%A3->Yd$C9l{>?8)0i;HE}NxL)$WINQv;MR>NT_5BI)=gc&f&c7>x}tDtvmJYp z-sqNY5wK@EC2i4g{CPl@@pGKR%%fGCs7#p?NE`2}P?;!ABgj+qU>ByYa;2!|v`1W) zRl`BuuHi|f)L=d0E$vpfQ+B>(KK^p*qQs9!$b29?9!AJ|KzuJJ#AkL4^4W@)uV2h~ zkn9*r=02udS#xTdPVO?T(>hcp8+GVUmY?DcPBXSg0cCbAWxUy7KQT)=@&+rDT^3CD z5V&DJ_Ir=c@7PkdsO>!2vN<uhzN+QsyJiBR@-j09!YoL_d}$%N=?w_0bHKd5>(| zWmE2@LO@%m?9z*GG*;YhJx*phvM%KzL8tT1G3?s%>ev6Be zKwgAai{upghPIT^ssFiz1dkZ)9MEW_yM%-bs5*4CR@u*|oSSqOF;vb?%aVOh_4%oe zHm)Ho?aoJu8F$w>k8(GejFM8F1NTnmdb=>%ykvP$*i1X>3f>wtfv-R|c@*)NR)OXi zw)0a*`^cr-O~b!CbIf)}cTP(V!K&@9Mn5(OC$0tw--LsJ0OBe!%{7~EYC z0z>T3@tvm-huwnV;tak6ojNH1?J}4uPmelhx-A^uNI2UfS&&vjY%o4+k-@KPNignL zHFCd~MM3hBj{}Go6-AlEqWGd%qASnYYC@R4X>z@%Q8gk!dQC`*iyv3Vd#MlfTCw+73;1%)F=93Q4h-%%CIa?SYlOaN zYgB7o12awow3qid!e$+$$fxM3n4X!f#a<(-^H3$S3wNxPn}TJo$ToD|&&p(}G;YW# zJ?d_|$1+yV&8O&{gVlfIkgWK{=@3D$k2h+f|3ca`xx4G~J1U%0VYLv!>&U^3+UA0J zUnOYhs8`9_ABw6HP`8>EKFSve;#2HrbZ_IyqqN^y^qZf%nU4Ovv+kT_3ny=aC(kE6?$fMpoU2>!#u6;Ao(-&xmmfaHj49`l z4=)~+8WyrOn=Ve%xkB+2sFaL8eO^lDPHTu*qqplYcp-I?8W&i(TiLtc&)h@cfLher)N-9^KZ)tF5d#+rD3$A8Ptr4Wo`SoBqgIsw?B+xFrqfDX!xa zI%^6Z(isVRJC!oD5=0xbiY-0Xj~B5ot;7W#F>_ffPtbDO^Lzk94mw zgwzWqbL+X~-&8!(Cz&^yCe_5Po3U+7c~P_Gm{{tq$~m+yt*A#m0o5tM2a zD$99hA-1%Nw&akZlGui*!xv_|9r3)|tO<<|ieZqU1>ZxnlozA1#Pnm1Bnkl}d7baN z1GkkQp^W2xv84nf@_D(saZ)z@bg^dKDr+&;K)hHZcv79Q#!0)g)7+e$xrfl*QAOAe z#3ha?k(dk$L+|R?@cD)N1f{<@c$pP5#8k1w%m%Z>T+I)G-g1RxR|{FDSxVWwv(xM_Rxa!})D>YKmo!s_@}!nLPt` ziZ{%e;;?e>{nLWYyOBxl7}(kZ;hP>ZlO8lx|698OCCwPMc34YiWWA~ZnQ#$dpLZ>w z#|NwER&PK0p&xJKeY^QowmpMyD@yv^1*THcna=KbyzDCLajq))Uhl&IUT@IW1E06? z_uRo(dk#|S!R6a8J_%d3eL^2d{L-}5gQz!1J}G*;QQaT1TQ06AQCrrtBUXO)BX2KS z_@CrrUt(5QkjkUtJzCleR{K)FF8B>w{c3+1tj|V2FZd1Ej=*@oZ@Ye&3>exWb4L6- z4Z!zTo-?rW%bhLiseP&DsjC#aEwBEMG`dL~Y-q4HbX4@P{MLRsKq!gX35}M^zts1~ zqJTMe;MZpkD-DkxO^q7R_rQ>0c0Do}jc{s>&*q&RHgd}uZmNvounso2Eia5|G0X(| z^h$-L;A@}nHZ@<-ms6}6g}I@a!je9q3t)niGs9w81dBQkHf7m!>a;?|ZOv5M=zr(y zJaXzZhQMu%s%?*uQ@FV)5VX!{&KpB-|90wh$Cte!%)|1HD8l7{(jQglC`@sR?aOfL zlaf|*@;$zRwsOM78(u9ha+=pH_lb)1Xb!?#-0{RPGWZ7$5_fm2Xm1=~4;r_B536wn zXc7pLI&`GDwI3(VApFbFL_ogCVGn!CuM=mrueFwoZXs%@X__90^f_t@##11{&APENHPrkfsi4r-g5rezda&9c6ALHwwUsX0#o8BYX^17wXyB;y-PzzHYlJrB^o&FE2d3{`q2Red7B)W*}M zWx*070>s26;VVtFvx;OVnb1&Dbh;gI+#{PrI@fS|O>wgv&IvRXHNrVX5T0+4BAHil zVYN_P?kM~D3ylqjHE)vkC8m_Knu7D>J|O*pON&~s%biloUU^^UjuNXD>KVVuXT0zmfBXv!{YUD2sLpa*7tvBTU)@(g>L(rcE!X8pYU{|Y zwx=DODy3vs+@+si{Tufax6>!GG1(35>5SATEp}t_;va8kt5Db2gwXTN-wyXV2RY7T z6c0V|xvX@7-35g&A?0ne2W3gRtA50LB0^uWBHMgcgBZeO0Z~EZW$5& zOnq^;5A2-4(G)~_et6!yzd?yh)itrwD=^36x-8F%J5Km*I*Cj9*r-8@{`*zW2?$JY@vUr_Fvh z_<)z*jQ9}}63c6d^tP>I_dmD2f9<|MFIRVgo(?LpwHadc6p)e%BM-^H;kp>%WM#~UE!2cWgqp!Sv1b^zn(>e?j+@bW-m?g?+b}WboAD9C%R_WkYumhhQN*qN$gx47B#q z;xP6FlE-9NlNFZx6M*bu%+EnFD-S9;%b9GZluihT!of;gI9Ub$61K22&<#uIWipb5 zJyQl1RE2jn1fxo8h~qjgCWlN*sxnT`uQ3dL6T!z!V5&B-@ZuH48G?^qz29AA!>YYn zcA8xqbDYi?Q^T1Aj+y_Zm5v#$1`=*t2BX$c;M@DfV{C=Cu;iFo;M5O^$PY)HSsP1M zm{iF*Ku+Ae$YMmnbX))?8l#z6piGdm(u`()MlE!f&ozRKNR3<& z9)`>y7*@Y^!6%o~-%61~fys(-gF2@XPq>h2bP{?Q( zshDT4Umxz^fKOP7-PX?Jojqnl@Dk&xG^p%yjT$M_adYDl7-J~HXTviA?6K55yw}ataj?Z!eYz4IlS_(C#^S_-Btba@ z`rY7+CQL3gA~6W%fpl#B1~+5hndkSu7l&qEUs?(mf$x;$1_u{9OesJ(r13=-dyj zMCk`R!$$6cHe`%hg&oS_h-lRm&8B$Binl;lVJmsd=RfEhd{4D0>v`(CQu-RcBXbMs)wr(2&mi7IT( zr5Yv)U_oY_Y>7s`(8I1H7_(ZpR{%l6~nT&dM0Nvl*fb;FD~3Uw)Kb{IT@ zSE*yyrz~7`qKYS#-1uh7RumWRy@D9c0Ph~@hgMU99@#dHSSk{`5&2h|IosV--q-9> zFF-~u#$t8WTBe*TR(IH0oHRZ9Al17xIi!DXAw@`4rn+5B(o-?ZxdpqV@1>+Dc|(aV z`0z^v4lDeQgVkon<=45l5k1oMbq4{G2(4~?MfjNooGydx1)gqBncvDmr%=dAI{TQK zWwrZ}w;C3tRU+wP>|)Y_E`j!y7Ft_KcK$VC#$AnUye^sxp5vlfkx}InR*bfpP=lQn z0P|qSN$0ey#`=qC{4yC>BhQa3QH6uwmF{#ZX=|6OjpVdxDruZs|AZ6SMHa!F4Gwv! zl+oT@RH!{8_TE6wd?%qT|M+(1GAD?$Via~sjZf9Dt38!@tXkslpxcrWQK?mA1-}sI zQnQ!u)g@!$i0HveDV{Yrv;LpwT|ZVdOR1nIkggcNE07cmBwM7IOpz|*KNW3_ealyC zNHWcO1g&c^2{3k?xs*^5Q8)P@@>EU`brkL#tIowdea_^Xh^6m$lO2C9Wozg&id z|J{nS;tBA&AwF$`x+B07NF;hozWkEB4GR34Uyy&uXP~mY5&XrA0uL5|dj@v3Ah#vW zH+;s|ctKR6PHI4uo0W))S{VPWIC_%B0O@O!% zPWZ`np>3xv7|rPT1$AqGkX2hr%RASI1!u;21d9r0CBal-kus3R0Lxq}kqKal^r(A?1&x8;5~1>GNk1;c+_Xp(k6 zQdT=d8&wxeoBuAc=+T1mL0--H$EW==e#jU$V~Qj|8Vaecu`d;VP2xW=U;rBt1JyCM zxP})`N*F^J&vu^BESrgEnN(Ul16W)vww02gw#n`i*YYZPHFIwgZ|&Ny+Wvb{YWVLq zCUzl&)Zj13T({?A&u#Axc8=HIBN&n9#6Ja#dS1!}eM%L9JJlun3;L^zyYqF{=l18f z!^gL%9}tXRalm`*R9~szk|lpi{~$X5tDNrxEV6vKJH8`?d@FZ<(>uP~13mKFedI_n zpZ`Jinv(5Px1{T-nYTu_WwLsXee_RAQL1OOdHn)I8q6f^H~X zw5nD1#FAI{2$j;yt#IOHr{Iw;LCdA3a|V6V2|2AeIu*NT(BFqqf%GgGgcsLACNd?s-zgD4`P@=&>GJlHg>r&~D zb)oj^?PI5hOJ2f<#b}sI#Er#pD4Wi`RApRv6EztA;bCd^it8h1;;|Xt$f)F=uD8^M z!nkb&B2zafv0#wBWgHT_T|3K1yj+D44%NY#7Y6Z~Mwy$COk*_y?vW>2De^KHzKa8O zkwy>={VMWeF#{a~4(;Q%!EC6<+px?ajL){*G8=jF2bhZ*iB)s+pcqDaH#wXQNRV9M zGrnSfD-JCl0wdAO_GuI$U}ML45SWSG&+jye@7Dbt^WW%`jccXP3QU7lbWrM8<)9Er zq7d@(z^F4e`}t5!la*@1A8p4AqJw5s7JA9xkN7IDMFI_6+2Ca!HTNSTFxBvC52P*oIrBd2(@QN#xS_*aJ!pC#M$4~ooc_jSox`kw*O&G_Wgu1xT4C(+k*_VsNRlu{R zmjS(+1YNu@N-yG~T_>%92rV;GC^4)q2aSO( z{+2Ol^^F5hq&7+FDS%Q(aO=xaCg$RMCZ`iX9>y7rYcv|8F|TnRC-Gzk-P1>~;D(sO zz_1iaIS)mFtn?PbPco80=W)dpqcDF$i7w3ug(K}R8WpX%z(^fZ=-@?B2ztjr5?=xq z$UN<^)-V>WO~gW?P{&6lOiahRukcy=SUhR%MB$zs!H!;Z_g%`94VJC2pB0}T8sn1} zi5e4H=tjHR@bIia4A`dss-(kW7MF#&ygqGJdSBnyNps-rn*s#aO~Sd2=XuCTNo z1v5YH@aXl1ve)(3g9%Z1LMp{$LcrX-Mll#>3j}PIJ!nF$>iwel4TbQN^~D%Arr@WD zvr#57$F@(KAh`Lwu%t3#OP88F4Ej(;FxlM9;zB66_QIttkRFW)wS0g zwO9@$);Q~eSpoJA;15x2cj*T=?=02HoNlYjzXhUrug0^Rchd(P|1OfB6v|rHJL{B737)^s$qF991unU9dk7sP7sk7JD z?7b(fHg&(6DRGYcl3j*IXWPfr3_ZL-?+iG!T8OT^8z=kjMC}bZ|K`u0mDsbDZx(lA1d6$cf)qvVc{b$bee zwM#|^@$PREbPRr$ipYxti^&*15~c(?6W-_T?G-lCT+R)Poh2BzUFCf>eS+8%8c|K^ zGT3JB*u{sogb;aDa>vc;hrUbbt@0;h6C~#L2n4!M`|$P7o7FFG@0+eyyJ+kjDrC3r z9^x$sYj^0?>XY!heh?qyCf|B5&cC9q48K}XJ4P8bWG^T2j$fqD?NtGg}13s)R?@JIw~KT zOtgekP6I9C+-bqkU@x)5!$Fu_z!_<;?tUHRr)vzdkC0mK{F()ZBYW*VXx_3%Vjf7z zkxSJcNA3Ni4-!)e7v%bObLf~*!AgOIjf%Kpd3ysoVh&uq*BCX5pWq?-J86vmi8{+S z_@AVT=jnj3WCom{VhNQL6rcy%(kSU_b&8#b8%*3N?Oeg3A(cx^kv%xk(Luqu!QGWN z)n4Jl+q*C5KkA3dJ`hVME^<$6nCr%&ybDj+m|{7xLsQZ9rqo($Q1bH_iMFOBSJJZ< zpS*^#RCD2WS$lsafZ%1ke@YJW4mJG=yF{JGhw?Lm>ow!z#veQ-JDz0@@x$BM-@_;@ zO7d{2K8a)W5An@QbiLDp{hC~OWU1%LPx7t?MAmh+;1`Lw!NrworY5f`iWIE#fOwdD zW)njzUT6%U4%wD`rYC^ivw5iMrR%PysCuSq#aD4s5t@|l;A6rc_-XFwo=^IyVPm=c zMYMl)|I~BmZZqO`Myo8@*%zsKMdfN7tL_fWN!XG z*32x!+S49!Y`_LtM;Op&Nd7WaXKFIRCh|?^(Zx9$=gI=MsVAK!DZ_J)mwZiKJWu9J z>Y_&6Y+eVm2!9WB)bLmuQUQ7CO7zb9py?rNy$jH8)o&|bBhV85je^QF0BkO=Qn z!8}AeV%*IVQcBrE{fS&!`xpd!^E(1e76m%Ph;V==flNu(6k29KkI?W)mU&yFRl|~` zgXY*UEXOS+jN4a9J6URj7M*is3)p_@aWY96S0X3KBdQ&5#f!x{ zNYOF{lX^$AIpfy#oXBq_fMMpU<)mamVont%5^%yA7mO;&ioxvkhc01>Blg5Y~Fca=ATl7(YVq*@HW~H)Sami3*EY+p4 zy)t9t`n8Simc-Z1p22KrfrYS^!%+ObxL@?-p?RdoWF$n_smpiiFuSRMsqbXkU!XDZ zAkUpb^S=N{PLf{>e^0(YO$3%x8WD~mq_+Et=RmITcwFf&Bd6w{&;}CRiC|7!-TB#= zl1JK!@z=NSLUwML!K7+X&m(XcjB>oTe#<($3&PCJ#1yy;5w6LH0FdnUXy~b*j7z9z zD?BJZol7sjfBMN9wdak>CxpqaN$;6!-w%Gl7ioQ77h#wkVAKreIM!M534X$Yqnhkf z`4xz}kanE$VGT(%1z7Oo_HWC!EeKYkK(^%zk{kW zC(p3LRD7egI09K=K2+J3bsHkl9kiB>ndAqvvk-6mKtM+ZNkdc z`uZlC-Hd*CeX_23n6)g=rLGCixCfr#fH>1pE*mk(H3O~6JuHfK{-6Tbu1ht|V1+gz z4n0b1GruRKm(G80$EGw{Q1&G4fH~TifOiJ%#B|_U$;}Encp>zpNyP4my9+AXb>`ia zz6yun_g~>`U8mp=OJQH&32aB*>o|Sono&LxhTQzcrS*(5n;c;Js5il)lgCeBd#8x^ zgSDV8p4Ap;$9gn-tWsf@bar2z;}(pems|AF*bB@(^2L5~+y|v~ICEB&%7fX|iKzvp z#Uhv2PLYS*WfY8}wosQg25{(_#Kqkr+!n!g zVRjXqwU!*S_`OO1x1^wzr^^z@Czt16<_%BEl zZ?qiPgX(PQ3A4ZC$x@g%G1LpiSt##Wh4b3hO68349luiNQlW&Z7l!1gZ_NNXvD+y- zRX|=E*~Rfv+2cDh8Z|Lh%fHX;)AhjK{Zabp+EQ}9(b0-8G%VG&7xX=!dMoviO|E7h zS?|pjphtA7)~w2^7O|dU3y1+gX#q+}YL-syREixptgx&TPc%uy7Wt|+m&M$RE*M6z zjovfG*sloHaFWdWRML5EvuNY7cAH1^DmMwCfly?s$i4+kmc(F>yxU1s=sejb2B!}WwEbyFmG|1!p0)o zX9LS8v45ZXN5KK$WQGq8LIKrFY&1`3Y$=soh$kmBs_~2M5X{aiwb#rW=z2K&t~c}q zE&ZP9vj(?pioCog(aCuqMBye#i5AZrYE4d}5zYme%9!&c6Fm<8H!diGAz5Y`}P zG6|3WdHle(>S&c$4F5KW`m9cNM&+^J30gB&?gv6Q7{-O%>=?+Vs?ao@t}9o2ku)I! zw3DKs|0CgV*S1~CM?Vvt&=t?@f`0={DW$@fzY{tX^^5ifh#WgPR7b|KNyd@`5trLY zow}JMJ9c)is`p_sxsY2-(dmDw!Fpfqw|?k(*Z25^;RuvlBe~cHe)tfDsc=2q9k%lc zGVkf4_ot-}x_{nfwcMr2c}%89^GfP8$~ryS*3&**q9$HLSL+Ga2|>A;{T7|feTO&8 z7JH-1ky`DNs)){%yx5J&YPBzi zB?w|Pozi8{1A!xv_0&x@HJmdU-pEWR(>>?JIPi@3SrF{Z@e`@r_p1vwrJV}+Bf~GL zrVru_y@L~A>Sq5aJO8|38WJ1LY6nH|q&V}| zU9kQ_4uF*>hJWz2#`;M9PX0Z>HRgXW?v*R1i0b)K3CsUvy8maD@c%*US)(j#za;S6 z_mV>r8ThNP042UXp&dU}%TX#BL{Jq0Q33_5R@qy!RjP5tO}8TKyNpOt^`IYNPrP83 z3kVSG~AdGnP1*UNn|G=3H;%3XbQ4HF)e$45|qQ!ZOjJW^yR6OK}~ zD^e^aBNM!S8JpyF0h@6nZNOv$mdtpm7FYQVHox7IU9AveJd(A_0QYN4)_wpQhz?gk6`JMYyG5X2;p(rv=aA11haNrol*Y;Q5E6g4a5>Y?{-n29Tm#mY0O;oG9iNk$ zK+NtdMu6?f`aW`3`;?~d#>{SN08M6Nor8{Xwpyy+S&dV3sH-1$OZ>t!N|5|rZ6i3j z%FhLn(Vtw*flM)Nabrw8#D&KJ(EhJ@ohJyH+t3dIwTt}!#}oD6XF|%}=zn^mTGX^1 zkX13h*<_R0ws#h>QRq2u>btmvV&eNeEAIL%v*Qo{%ibzFl7g z17be-gNolIGGX(98PM4k#Uo2L9-(k^;Ers;(9`uJ&Nd#oaP*+}Mvwpx%HXtn=uOVC z&_|p0{!sPxJ1lIyxQb)qVpXX90OUza z(o`j;iM4Q3i;N>Ss1%X}C(sKSzg@>&Rp|CJ`keTQx?Sm>Eu9r>>P3W^>DmH9pd*Y@CHe%#lgVqa(7{ z&1;yTMu)OAhRUW+P8Lp?(T^t-%?j#4%z2q8krl7j$c95qY1vBfu*yS)FXjx-W`&jc zCMwQB91FTZOuDNiBV(-$GFH+W($Gh}q;8b7d8Yk!@*QsgrdO94qMGRUgmPqqKCdWZ z^D@*R4-H{FgUU!@dW?7~5~=*!roAGfaQCmDCSf_zx4%zNDW>vMz`3DZ3C_mqV+m)z?&`6X3tVV}`V*>GUDEyB+3SEzy>JZ^PRQvl|MPM+#Jm8Z zN1JK#L0?Lhj1eK(z(T`xLo`PFM>ug%({5I$O;S`j=;-CHi?N$~?QU=Bo2$n2$$KwH z2kDaZ1QW>{_wAOD&W1F=bW|aH-=OFU;7}bg%XhLlb+!boW(PllIfz!0j^sB*RP)$% z;`;T)>G$|PkMb*wQodTB5_69MN^THYk_9Qlfb?HCOYN)R4yN@?9Z<9FLfirh%>qPT zbCP9UtUgSgn7S;=c#%G-jKXHd8{8?2Y$m)xc(!h8LXHACs*Yq*W7L~?M{;Z;i!x+l zsjTSdHG(2!T_&Tm!q6VrCTli^tXnWg$<4_9Sq-9M2%Uj% z3Zvrr%9`x>xdr#gR)dqyOHXT%|6-SB*Vv76*yB~=YLf<|3|yd6exhL0rIvVpvreNd zI9x0*!x=62=2Ih}P#2$X$~L$J2?N?mDIqdDm~&dp91wgrTIcP-e9^)*h0Y$Cr=5i6 z)V4yp&bO^H|4J;54UV0n2!7ifI`O6QB}P4XLVQtZhq$tT8RfE;B+pTPf37P({jl{E zq%8UYQZ^%B&3^uB;tH`$UTV5wENItf-O#(JTB(NEF+fTX@P^zmRkn;{m~6&35R}Jh$CC@?mFVn7^2-L+RI{;E2>Q)B&&DBa>yH+ z%ZHq?TQ=6HLYQt=_Ix%rWnBuoK_=$q2CnJ3GW`8C6OXAvKCd{8eEKG&3iU`1R9-vK zo0P3Yh%d%0ULY!`aPy*q{v=mp;Q}2yRX&pCKEEJvk9UmzU6{jcwEoFUoHZx?#4bUe zg~cB{1ST@1PnZi48FkPQA-N)bAU*d{4V1`oW{TY_r?3>XUo?ZdyhPG)%c7FQ;#Pbw z<`rx*b}>R%bDh7uX-QP|%@zxZ?d-qis@vp`zlkF)ZY@*k5AE6d{2>L`05=lxXpc z^a1_+Q#TN@D<*xPg$bA_Mvv$pWW*WRv7opoDJAX4;fYl?D_Jz4MIpZ~fMs{wos-#=GU5y-D!{Qr@``u}vr{6Fo!Isf;qw8`PfKo>kO zB%M#IKwv2hr}G>7fVxnrq*xT1N{Qa$fF>g9uw@0?5B85g3_6|Ye><1h35Pw9p;Q&% zd9QA+uCmze#hUp2eE-7l0Rbb>sqs`B)kI(7+3lHOUeeihv|k^@3YuO#yr8P*x*-CZ0_6=mv!WmcrDIbh@SQIJX zw~c$K8H!bo1O!R}8-EyR7z-Lw{z%|cgi~UI^A;7EjN3o*3h)1=D`~|+cLk~|BjZZ$ z;FJ4PI9-z+v-!jXWt_sOt)7{H)o$^|0Wvod8Yj!5xu;$F?1-*y^XTPr(F>{aom{`l zpG?SZ@jSekF(KhgW1-d7KcsB+TZ4}=$DcddZ1u&fVCPM}t!Fx(Z9R)ZoPOnBC=0v; z0%@*#029J34*d=3?rbCK-9%F8!KivT{mc4kk4cU^<6IX%*d7iYLEZ!v-E1Ig5sZ)C zCeNTwn9ftvsm${rQk`s&`EW;zzi26)5mcH(#)9-x zd0u&khTmu&lzR0rR_(*N1E@j+My}nI;y2uX9j%&_@5-j1<2&@HVBtS!yDB)@n>(30 zJ1dzw*gO4?$e$8bTjd{xkKYZM2_Y20KCOZ!5bC24R3Sj^cL5@bh(e%7&4%VsPJH6k zjSU;_8^jmP7nsjwGjk36*L;KiM*)B3b%(osA}Ims(!|ltw)Y(GoO7=8>~`Px_qF>k z<~OovyRBORbi8mw(J?DfK^~N$%vjg^t?)g@-0*cXP)?iB2?uFdc~_fDkJ3YGa9;Zi zx?OinyJ}52LgO_tzch2V7~H8wX44_JH#ychW^FN3`;q3V8!A$7bd*mZ>7Il`W8=j$Ok*ctyP1u3zmLbf})kre^y3}0<^C=JI!j!el6n3f2 zP-T=`E5@@MH0l|a$589!>Oh3c24hqly$TzrO(&h3-!rq>)<>^+n5|DFAwIM7iu-i7 z?W#^2c^ZFL-e$*>VVmM7I^-5;Bti=!nZk3n%puV^MwCnZaipYt(R3MQZ%m}cN(Rr* zaQkjao3wIe{S>OBos>GN$%2tsVwghy6s*Gx^*d(3UuCNF$hJCHylYZ1L~^Rrq@yGV zi^0rv2di2y=VlXaQVO*mHP!5CsAdr=u{_G@F7+Z4)kc1TzPtgMPEf#bgSSpHcxY)X zayZhpE+%uy0J$s0nj#!uDJ2lplFdSv8B$alTk_A@wpnDL;?h$PJg8918RbgQF*k31 zg8}=!@uQ;R&d)1t=AjL-hHBHI>^tJ`XREqC_;gS}SM_ytl?JObb&j2ruoCY^6ym!V zoe9!J>OC<|H~Wc--b0BMwFg(n)?p@|*5W&w364!s=WG|$Q_Q&TXqZO^*XvJw01HG>eV4RC8{l96mA2}F^AqG5}5N2 z`|lIYM6C0hRG(ukOklnn7Dl5~nMbP9Ji|2eFHqv06E|C0Fq{>V#}nc$9l4K&VN?;1 zGKhWh4FA$Vkgcd6m+((U8IOlptP&8y+7ad#iT?r}kTwW8BXC#>V^kT#C@e>A&MWp* z*@#F<76KH&DaDjAQYzn7m@t$K9*IbSGF~yBCBc%#&EwrEL%vZ=OF{Fbz2baz<)K=spoLR)@-Xjaf;S~@QJi~oF=@PIEcFDaW|rBjRU!U*j5qKz_~U zdY8(b~X+=E0sQ6?ZO@|A<`&e!vkMC^G1il*?7N-SR4^fOf4s)M$Y zOv%iZlJ6C5=G;)!_j>NkCVI}z=ITXpl1kG&erUbOQ=s}n%PI4I$|O~J#UP3+GyrZV zzfP;CS*@Grm$~G^+h^kX2K%p%Ai?Xe#EGB%O84^+D*ea(`u~nX`=2%7!uGbdmM%Xp z9*7{u2T8!+57iGOFx(iAx?CO8VvUNR@VPwyz9-^z-TRIkCO-ka1or{$3NA zOy+JehW2-!S))gxz5K5Dt3oO4zrc3-aVYLQ;mj?{CBmv#3k>@b-uc8*FBRd}zGyBp zBy?aa`id2^71~NhNO)>@T*jO7e|8c!Pisw`Oh5rh|vJciCoEjF} z{|6UBe&`|?%Jk5|-J_hyZtQ#m5DIgVKV+XjfFN9GOKnzX8)bIszd2&c2h!Vf2pe|o zbjvMryMSnNdykl1$ND4%=e+KVpzbFijb|W+Q0^Jl?@?(;!kR~d@B2YF?LSTnFdk14mwUcwr$(C zZQHhOb&@}}-LY-kwr%$0taCAI&RMf&uD_e_Vy~)QyXt)&9n*^ke)J2CK!1?`S~%JO z(|Nt0q8Wq*0;2gJ3rE4w$>l$%6jN%_&e);|-)i#GdfLqnm#c@e4kkm-VFJAf#*j?I zPA1xsID#H#B(jIDk{uOGKlJDQQA9f&PN#4xlnM@qgMR(|R~QuD25M=oc*qw(m!^_C^sK^0Q|=ezNnC0cs^Y~I%2Te^m_p` zd(?Ag^f4iNyUdGLg8LyLg9w>>fLV%sUeI3#v$XT>ESO5Oo?9-(KD|Ji%nPoK>T=5cDxJX*tB>@gxQI)?qL1I41#}H9UT$~87U6BLaKvMuOyT*e_}s_nuTsL zP~Zzf9~;uBF%mb*wJ-Evf$N(@><w6UzA4n!26B9x!)N*+8EI_l6p`6zN=p|r)fy2ZC53ay1GdLx<8dlCw-)Wx@9 z^G+NTpG``yw4GP#)`@O+pNxv-%3n;57mnd7mm283X?aneu9wW`;P9L#E-w7 zx>E$vOM~eAN*(5u>m?iAQ=|$6Nq7EL{-L!>OB#y;OZm)GKq|OKw9a{t@%#7x7#t?n z?ujaX7N|Cx|IM5J|2#ncciY*L4}^=((9*wbOiBADFDd}-hPCptE`W3b47LipK8kkZ zPb`UgR3~70ex16yv%@KHHVB!3Bw2vK4I~5!HlKa1zws9XtpE~1NCtw^uCxTCqy)pz z&ds#3HQGTezBXWOm^p+Z0?pG&76<>eDO#NfY*rRKcpVk!J90Sp^o&&x>sKP z?Sl#Yto`_XdAU3DJ3fBD-qe3%L@lo{FJDWPd?^0TQ9aS|uS5JW!f!jp!_WV$)B znWS;piY!NM$YlYXVUu8(OzrN5P_+8H@Tz1pibS3g?VkM@ zlNAS`Ia-Fbn_@Yn@k^Ux!96Lm^5k)!VLXJAs~AG@Ea0J(_9|jJ)G5s(KY17c!7CJ} z|8;UW_Ng(PD*9Vs$G^$B9*c`PP)ySZjn@XC_rS`<}&rA47>qxO0*rI6eTrJ_)~t z^T@3&n6rJ8(ET1{+GVsMZM&9H)BIsM8x(S1H=HMxF#9Cog!^!%?H`En^?345!j8)* z4DLMs^fzbxL2sK-wI)^%V;Iey*KQERd{Duo&OG{ z`vmXrH_-4;LiSqIvGzDjSKb!C!BlLVd$I7&$jHug!td3D77oi?HLSK&%C;p!dt)O1 z{e%|taqh!xpSB6Y8@F-q!!89JPyCyauUW9S0A)Ukj@Bl4EwUHv`b_sn!sA8YZl1~b z_}emZPkp?Di36Dt`L}y!y=i10&p()3HlwAlc*l1T>0K~sUGfhB<~aA7dKb=<7bl*w z{O+UZ7(>xjT=aKG=roo+hrm7ip`DYHK0}21M?VzJ>lK`P_A1uehXL#`p9R95{ZxDG zd))BPYC`{$=FRkK^(_1!+r;zh|ve#S_7_wA~0LaaP-F*R~Jr)sJCB z-x2()zauL9yUhfRy7CZl=dK+>`8QEEk74dA8Oz(zuXN608==RWC@y1OL5o?pfA5Y~ zy|grX6K4}9G_$>3b#6~!9tI5NfW3wnvw&?7cjG2*lh9&gkTnc%?Acm9r8IkG2KU9y zv)MxkxUjuRrGm2qwDVYV5M_C)+W3%QJ+*zLJFljI^(AJjNHAo#BiIpaBm?S;HnAmh zKsC%N*&CTqZ0nv(nXj0j!;Vs-B*vrlO{z#;Q;EQra<_?JeEiR!jH8 z#nfo5H5yBQom7JX4H@f#lCUWS9QB>%IYvu)bMN{8l#_A z-RslMvE2&6?#8{P#T{eV8dq>{Z)?{w(T(o|LC0`^KdJnCS3oh9Pr!a3R~f*5S(_X|!#Qn=0_m_Tt4fq=>gUtf(0fgyE zBpM^!3<4bEUOs5PuOCvRx5{pW4gk@)_ZOZ=`(x>79&hB9djOdgOEwy)Iq25{(gu|S zZDlz#3RQem;-h>BrX+*Z(yE)iNM#*#c*d_-*+5_A*5oP7;lscLE`8@L4Z);$v|5t7 z$MXCJti>BxV|R~Sc5(rEc)gvhML%V>Klfu zA2Rp7g1ItIDZ-UFSv{!cTOg0UmBmzRYS`S;aT8C5-81(RQR`CAlzKa>!uoNv2P=Z@ zx>BqRU{LVQO6Ke~bp3OWfkRCTRS(lUjk}3nD=mS+WjZ|ddmf`lQUE$`@-@pb$o1SDlH-0rfoJE=@VJtv@_uiCJGyxUYV;x@YP<}u0VmT zRD#rrFGIEf;-3g}Htw9p-YYRQ518s>FQfhGMki27{a}K>Ab1psd_?&7j>RWIs?VX@ zo&>_Dzd7%H7cOpKcC>KcbX@)OaS8xDE7eNeJ7K+L1RravYo#-10J+r7VV(a4ANZ6| zLn4D6MJS8ud$BJD?sc6}jr9ZZoa-RD54@3qJa3*5-zbm4!#v_I;Xf)15FKk&8O^)` zeq{BeC}-bD;d*qbv*-bFSo?(eOeAaI-dme&3Wo-Z^8SB$o9;ue88ffq(slEfHsvyq>M(LV?!+^Q;R$NMnmZ@&v+ll;O6ea zBw{jq1JlWenhW6*niMRGcgf2+eX+TDo8%$&@A>OjRX$8}$MKC>abS@6_qQoU@Kg_SrH!O!|Jkk_mf~&SLWe9U$ za53_^BV%u)LcPUfHVrQuj~5+OR@eCh%Ni4bnTi`}*HMKuQjQ=!x#<_VsOqw+PYQJd z=?t)C4`ANgJG4%@_0b;-Z~$#DV$L?_bq}ndD9dq6Snupx>+Qp}XsIp!Wzr={K7f!a zV0Q$cGC3v-IZ7ZSz?nPVnp;TB-#~=)%A^ID8UAjei82r*s`dyWw^%`2cbbIxAaa>S zb5yGJVBxF6QY2pST@(0+n+5Zx#Xywq3|(HHlBZ|!-W}Xr%w4LpiMc207%RocL)MSdG!x?Kig5{1VmQP zjZB|FX#L)mg5a94U9{Zgd`NbnLMjX^WS7m|C z5LTJv6j|8>x|EQg@yP$sAMP!Xl0PL}<~@dw{R~Fu&>Sh1{xH>`ni8IxDVZTfWgNwl zXiLSCYD)pLgeiGIA-#k{3&)tEw7h*VHf4Ygs)Tw+9z@ z&Xqlap6lt~?5Q!qg!UGt*p%Q#lLT6&Ij+xV3vjG+DoKkb7%a-NilHSQq6En4h{k;` z&tYnuMU?Fu3mcvJw0qNJ+jbn#96-N7@zbp;Ksu@A+!@_$wLnYnkmoj!+eO)*BgcyDBG?JXOa=dEqw#k8`&Z^~{1 z(J!bpEZQjJmJUN_@Uc)ERve^PI;Vw`J6wpc{_dO43s}u9NU&xN{lxWPSdu)qQ1)EZ z(L3N#o>G4(=*YGSf^F(mfms?0`;7isqtr%V!E)%7j!G&NQc5b6qGg*zR5Pt8TL9%p z25v{|VdL%V^*lH#F4B^Xm|*_ za5w|l95mB(8}E|@czOvc06tyQF%Grg}tV^DgjSov>oelczOJG*$IDzHe`Q1hC zU6ggFCpv_D)o1He*Dr68GWMW>WefKxaViWusriDEz*|gJUguQQD=K9in_9v38W<@h zW?JTkCjk%%^Bzb>xp+Xd^Dn)8CEiD&%?k48p@M6J)gaXaSyHz%*e9qJ+|9#gP%C#HwT9G<5MFc8^I(E{7nt19tj;<8U^ zUYjcXrgHyecnvGa+ioabXHiZfoMr2jvVv?V#^ln@EuUh_bB4Mjq`^eIrvKD)xiO;^ z=2T8UGiPnxL2~rxZ28_oNJoPoBO|ImFKAQvoFCw&npMTCx@;=c7mGVz)xl(GR}{?zE%}k zNbcefe-U8meTgj!Xv$|bNe2_;94p75`2u>fp%~0NENl18WZ$*do7-#XW+AufPS&z@QnLD+FUj?2pH*?FxasJ?v3%FCtDUHq<;?-`dMx%`VL~WeP9+ zzKnE^tl}PcEUQOFoM`64YK`I-LUO@aVSe-_cSfe8zA8UkC;G4=n%JUyHb4}qN0gzV zzKJYDBWaySmN_lYTvjcI5bfp1snReT-xl-*Pdme%wW242rAtz^AhFUDapB_#&-iH3 z7!*V;>@W7FjK@YdUu;E=rkB!cLL3z>xLh>u^W@(594XFT&EC1lkr*R8+29$F*H_OX z6EEXjJY2;>kd6!_XXqI;-f=~i*SX$Y50$7H=B1CBN6P_7c#VYpzbN#C+Wu}4AARS2 zPVNZetM^_x?!(LPO}a9!E|o8O%)Xdvbdc!VI}nLaHXmzHw9w#fK!Zkrb7mefQ;o68 zJmYuJQDSr0tHS(}<#}O2)i02nVE3mi6KxR4G(P$VhjD`Nd-X~I2H3@P0LE$S!D%uG z`oGhIh+nkl)t0E()y~2`70l~#Zz3OzS&yR)ST=9c<~_gUFMsB~FI0h3S5Ja|kj_u& zoqt7z6sayU-aAU0!l*uamHhs=*7JwLRt%a68uR~R;8aXd#OYql-T&YoVv`^v!&g_< z`|eCCx>Pg`62Ugi{c5uDFq|=E`hZ5FjrgVwM*U0wF>Wq^v^F;1wC+Z_W*9q11#9vB zl4YN{mXc4i2g_s&i2X8YDWoCjaT_~FMqrdvQ*};;xW+GTmx5j8pvq0RzzLnlSg@N1f zakJZ+sB2iuX6a9~xM16+j2FK>WE%N%8l03I&wSf%EB|C%QM<3CsA^YUsQ#G%5R6|5 zzs}n9{+RxWFZb(*{kSrCFmJ^6P3ge5g{EzX<%M)=Ehipt^79R&(x|%{2jlx}MsmUc zE&e-j5Fh;B-AXS|GJ8hOOA{elbuecwAWlXOKX#UDa6K7U-O2xkfccw@b5tg4Z??4V z+?PRP;15}$wVe{JeEb-YkS`raZE1C*5u5fFst!#EXQ-&Gp|SE%rSDmH_6nnRfgb4e zrfIm3zNpl;hQ>xH6tiAhRTEopjx$p2v2Exv=9KCX; z;Tp{)k;q0%T?46^gB18`SWaf@_T++ha-f$ADAijqD1J^@mHP5hb(wXQW?g%IT}73b zr>W?SFp=-n6|UVAXMMawi>-7+?v@*PzbtJXIqqH^>2{3<6%s87o+NCxiOHimCZ8z& zqIvp#ff}J-#+apl>|BAnjN$!-UZHc7^Y10Y8D9tPRc-PryUhlab7PawkBpSQ+bCI7 zpKS^+K1&oYYb88I5V-f~6u=2G1ZgqlS6~AvLiHy!mbIklz)!H1)wJmXi+Xtpu%wd8 zUSm@hD$ZNMFa&Q}fMhZ1FJu9s9Go?px}*mH67S?(A4dIPQ&nyTZ1ie#l?_NU{&jZk zMXMLF-qrDdtIt_BAcxTLwB4rV<2zMFg>KdyZY(ml;0(xhTV<6kTPOmqQ1`<4XVakm z6=|Ld%fM~Pk*DK8($x7K_&X!Oqlw%U8Z~!-eC;nIaOEw)n6`6z1rhcPYXXR$A6Ki5 z78*@OcaD8?6|qmyMm36X&Y^%bq<50(Cb%jZ7x#k*OLjELx`FmzY7d#DRI<`=!dE+e zHt1kC#iC#6zWg|I#G2ymvwH&bI|+m5F@e+0;z?|S_MeBbLxEW?doI|j-=3L&pr3sI zQ5ojd3-^pHLzKNVL~7c6@dzX5Aau{1Ryr-b#5do2o8A~sm{v=?(`wBS9R~D>VH;Am zJP}L6ssD2^nn(-_3zo?Dvd;*F{k?2d`kN74ky}Prc2aP4bv1F&VMPBPs zRXyLRk$u(1DjrJ2qVMXq>ImL%d+a3er>(5bN5-${h8MbfvQJ0T}G zyKl-|A#QxRVoSdIt1t9t3{Zlb!?O2;^Q#R7K78rp@IIreFPuQBH|hP69nvPGyXn$} zq3O3iII$pAHi^M^ZKs~`DBpw=RlL^ zMB^YRlQPr2Mx)YZhF?VZaz+!2DtSE#J=} zDAUNuwvI8vP!X&+s0bc^S)j;^=>vS-F)E5qmWPWh%sq@0K%u3Mv@RLEIy=XEryFGI zhaJ=Ha*EBRiK6*>=p{uCvHcl*cSR;X*XVn=54NmzftTKU`)8jHt08q+Q~1IGdqY+n z>b#^7%yVAnr=g0;+j1dli9>zlXyZMDu-N2MZ(pwRW$BJ4Yn=x++t(X^GQr6kyU@Y> zuoyS?hUXuHK9_7UkmnExy@=8|(`8AaR}kBUFjcrgf`4?y=vk?C4X2(`+M>(29&3%4 zdN5}IV@7Rb(y_GEWwC~%w|>?ef5vD(V#4b{*U^*~j5r~o`G@D%o+zmIY}4=n?-;}}$HvX)l##o@ z7{+ntjAaQqV@OpTQ`VOrZTFZzHEgj>IL;+<4?{&JJjoB6J8QfP;({FWvDiA6F@cP3 zMkbM!G*d5ptq^+o@wk7aPLstCv6Nv=ER7VWkn62_mXm;pd54`Gn%n zI!FUtSwL2DjhgsD`E!EeIU)Nz!l=$TD@soY^XU68K7+K*QV|OD=R6;oVEb#&ydSX! z?_49MwF5rrctv_37TwP*z2qO7e`E1yi&+F0?e;}AmPt-x5>gSWodvjbmI+A%Z>!W{ z7^Q&Iuxc%)t(zNxJjak4XjO?R44<%|)jLO&#C^x8(O(mS3DHERuYd(iC1lxv-lJa^ zCgEQYLio!uSWjc{mD-9OD3Jd9GXSXe21+@IN?)CMEGJ>RLLq(AFm5z8Us86F^VFR7ge{&db#g|j=90#Swo zEE7sTk#fY;UREDn@#Czc)7l^B5?H!;wX{U1-4LJXC_4sPGU|tpA2VvM&y)+~opv+b z&~l(@X~bIebkZrIHs@b*R}L;>^@i`c8b2>krPEklis?^j&Ruqy(`rQg1uXsjPRHfS zpS>(v+Q23F2tPmPb5X!tQ@989zB{%pU21u^0&gYvdtucAG3jEwDUvf%J0lG)r!y!^ zm(z&-2(=BT#wK+krTjUu1#+sBZhci$wx-c{5wRUF z4uE=x^{iM)4uQ+?^}RLdsJG5iMbQz>c5&ns{!GA0&=lS5q6kD~=}08bf2awcRtGs# zLj#Csp~XMxg+S0sPx$4botC+`x(9L}k?4X2!p;B83AE7$NA&L6`V>9%>pe0Jwn0ro zKC1kYMdt>>2s?K7KkV>n$ps)$;Ala@uI7NP;`e|Q9jk!@ySU=1zn{L`*N-(-BLAV8r zkzKik4U(R-ufaYtwg8H6hIl#BY%f^T`;T6|1zP6l?@Y{8i#o^bFtsq=?i877(FV#3 z=?bQh9fZuLw)98LB_GK`&Dz^fPE<^z34hCw4`(W7kzE8+DP}vyOPV4$|0^O3?-U$@ zy~#p5Ek=@pY64$&Rrz3(7H=jU%=l@>MI7ebYE1p<=(JbRQ?ggzZm&lRZE#Un1K%INegt|_m4?QDbc#cErFuKDV#)i{=|^*|b0 z-3BL;VqeMIHSW;YEHdnU8HS4SyiK10dsawkiPFc|o;}Vd;mHvJ>gfFT;YAZ6mv8_mRA_H+U_FI!|$N&@T^{)7zJ1%_-ShdT6fO zc4Hx(d>raC%RoYC(W)8YnNx;^9(v+T(PnA747P{QSA!}Ik=+axe?uZ%;gw&y>cJ5D=UBb*VG zvgzv-O~R>y&Oo1-}YLNPxrPa;+l?1v=rFt=WT%}&2o!NZ)NEEi8IhpA}WzfKWoJM`#7>zKVkAGIq?jr1sf8OMWPTiDL zlFA;?&DeRa0k@lN`96M~Hg46;c~{MSoI=DG58n=KW7J$l=2u16Xfs!*Wv<*L9?XqM zZsTF?fL_HH5ZJ`bQSRUHf^W}S8=!BEKI`%rZ~{}r44C2+v0BNQ36@NFJc?;uGyMqP z`W8phdARE)_M5ottZS~vmv!+*^u{++px_;ICma)wJ8;%>ac-DaTX?ue%8yM@O;9=P zMP31#39J*w{R?S;Qc8&(arUe6{>_~$)Hbj)?^9dq&o)ixk)-nA%*oG8PB=%aaF;DR zrqZ@7H6)}HA48B6f1IOF+R0d%Xl%5F=(oFfUKb^o%WV~7Ud@2x@$(XG(luB&Q%kq@ ztf+?Du13eQi9*a~nYrY?(M3J1ME}f@zv0I_!@iCXpWsu0(EkuBylG{TR+P(M5pmAo zSmZY$C%lnuQCoWTZi+7kuUs-)e>7Jwvm5(*MMN)zdL8(6>%m4br$Wik2uSmw5hZwN zP5X$7^mwS1L&gqj@zK_U@2IdP!~jaGYA5q~sATM@V-5H4bFb1$U76MJ6b3VuTV`MDj=R#qvil3Ke>z zyY{s@S$h*R`vR*E6LS0CnN5UOm~E9K0JtT2u3e}lc~8XD!9o|{x@)`3u{b3P<)5*w zP`;S!ztk4WB2V?eTXcOkJNxO+D1OmLXb+ku}j8XCM?Sk4Na6%{i@FOOfQ$pf=8%Gc{L%bqU7a*u1% z+s$20j z?_XaRcsq2oprw8s7m=Fz-WEa)vFjbm|CNgmT)g}KglIn%To%(H4u;^?!L&JP=Skn) z=lg5_FL)8xJ}%UBRT$Izj2+App<2{~Mxx^fu5JnMlpnD6t&n^L(R&4QIWh1SpXZ9t z`wsNuY5YELG~OM5=0&oBI)jy)yPaRyZb&D)lnvn~(8>sixjG+mW@tfT^~kOO9s3T5n8 zqbJ6b2Z_iggqA=kF4&>NA{AF$c&LalV@&WcGS9=AQ*ro{>qhi*EdNh|NV zm+fDQ)*$Y_++h${qrfKVbr>L5K>Y077_UnlcOTEffvrnmXPN7!NPjL=>!jgSb-mbi zJH)iSfqAGfA%8m>L2SWvJ?u+nhTjQHl8n3 z7Vbi(ju8;cwjP>eAz`A}3C~y+YR#__A^Kq=#V^w_ib=~ieR3*M>C3LfzqKm)h^adG z^bcg(xR@0<4QXjFzl;mxN@IuO^bCL(dS}7=_Zf(HK&rRt%=fWsgo>hw^jM?{n-ri` zban|@cZ&CH?0Cf(m4`Tss@@(nbI?quzC+D})0vJZ-dLX$8S%`CcJlSelRPmz@REB| ziq4}+SM9~_CF52=?^XZy{D+*Ji+6>}0nqLgy*tV1gxLEJ#J`8e2>pvXtr=FGZADnG znOYddEcUcSU1L7+;%~sl+BBEl+SxP%T-_+E%n8pZbgL^_i{4d&tmoy{Po1wIz4-^1 z!It>jy6XXxTjYW(gLhGtjBe2Ml6+9xTzHhp)?=b==wf`oOs>`%y5-U}JuObxBbC0# zJkYx~`Wt9oe@pr;hh{?W-`vrrxWCur(I=&DLnf2%^vGGp$R9mVF{{a%M99VYA@X~T zv@S`Pd>?&3k)*>DtL6-2-by%zRQ&Q-pB9<2v>1ngLrR8xSB8AJ+x*I1VRhwv9ngn1 z0M(&a+RVZktS7i_Vk}A`c9}BU-0wA+6AAph=2>?WYF*MVm*`C)_X!}^F*Yv|@Uy(j zJahycUn=(~D(t`o#J6f@kix6t2YU2lg!7k4>|A2-e^6Uz5Hoz z5?V&>+SKuqTgFq5Z0~73J3c|Zc2ea>;g6E;i9J(4HMVK_Q(r3U?txi_-OAVQ12TwT z%8^Z5Dw5;Nu-?ilQsXP(?xC7635h?RL=TVUwYba}2J0X_+FBW8OyG+{ii@kwTAhYC8Jbo zNO@gQwT!Ykc%ATHIkPM1{aro>MNU|7yTXrt5VV4+iJ2DRH0h*CqnJ2456^wyuCDdVNbU_z|;2Tu!L0SB}^?xw`tX?gDaRX@ z=OK5az&Bw0@S;(oTaeX*4{QeRURH~ArvT2wZrB;-8u#9v>*s#P6DP7UrAcy*!zohT zmDU{*z%BcU>l=*=$*q@g z8^L}(cX9YaC3uuGi*H;0UuUiqXAmkB{yD&C&;HD6b7LPDw?lU6VSW(hO=Y$uSavfI&bq5L8GfI7qQAQUU@9tfztm7M4X|X1X*JLwfqOjE9A`tzfmU zmUX4ChPAd}bu*8ZL`1QwwW^IPetTud#`$&ohA#e=u6xHuHR@-s^Noxd8M=hh7veSV z3HQ!h_l?hu_aEnznJm%*tnb9=_0yjF+t+N2uNIHb_-M3+4LazBf9+@gPH}-h{L#Cl zd)()|A0gPYOJOt&oR}NnDs9{wg17sEoqy4&8xDzrU8-=*h%T%hnXy+w%R9}kvrDfGX$EjZQl_63(c20HTi@#P`CxhuLMOHwlc`Z^P;RK|bgudXyEv%nT_2G2$naQ$UYzRMg=lon zj#GS6l4{-VS%tc_>WbB3&QWUI=d?~k)$gp({R387+X7djn#35XPzznc-Q`&(6o& z4BN4Uw0J8AG7H)t!jT4O?fqOy;Ie7_Q}i32y@ocz~NduljQS0(vpir0M|| z;uCFH+V{%I+D{tr&$4nGD;1)FPNrmoK?$s9o9_l3X&O@|^xE@dhZZrvNPIDht9xp- z<6SMb2zAd_YeTu3G!!Of^N8oLL;6xBzpNFfb}GHSzqEv4M7=tFwOC3Z-oM(}r|_4> zjeIj^`%(O5R*<~fc{?VUPAvi@NHWC0mTR?{^yEk}f5KVMU5R!jZ>dMWUQXwbhp$8Y z82V5s8fMQ-N7FJJ4!eI5DaWBfTPalR%QJOXX^F9&j7{zLMvNb><}rbeoXzfkU?If% z&WS+hVBFM+A-7H5(XcVl;{F3`CNOlL=u9f=?CF{5CI9V_7}EoNz0UH7Ys zi#l<5^^vj@9bXFiW*{}XpU`InqmRUUA|PQzWuQbFeLY$Tps~imj|B7lYoI_OkRywX zz6xI=_P|Re={=aaNRu>Y?Y#Mdhycw@h{oG2^g{vPYkspr_F^CrO%rEKNv8!3(nSRJ zKw!)GK>{WinjhJfX~q7Jlciq|UVJKzc00<@UY-vPa(qT&6ILA{iv~UE%v73&-$^D5 z6=X#720neNvz{Dde&dN^VY?3E?L&U7=2^mjvbGjJ>blTDMOu;81+vPO>$MPU4wNdp zL@=emB}vNsOf*HWz`nu-d#lDNjSu)lg|lBVMX!{7w~1Fb~tZ@{7H=^1d?0C(s+8rHV=EA;JIIqZ6a=+yh-~UqK zgYW;R+^|y|>&cc@q$k&MD+JOMRaEdonTo$RSM60W1%H2LH`BCAZ$!nXZGM=={?`yX z^ck-fFbyUBz8KE0n@g>S4B#ZT|3_5nyKo#DgM zaa1rR{1w*c-iW+fY^j2+Qr+@wD(FDcFJz2cPMDgf>YhD>x;yV)(~nVD8m=zQO+rVV zK^8X<`YxZp_Xl7s=B$egNo1hEUkL55d}RMHLFFH=KgNItZSQJtAm0hSHRM87ber=0rG!WK zlqImIsFpxMZq(3w;pnj?B&jiFoCUWSyF%HVvGv|kv`7*rAO0&gX;^@eeF~s+-mTPdMe|tNTF0ZuV5y9B z5alWn!>np0KG<1u*-m1grKZd0B29{x(sZ@IrG~DuKQ3JN9E)MA!m%H~(^9}n%s(rL zu6nfC^SuCtQ;qhg#9AVABBOwqu(g8f+&^0)g>vAJ*WY-=A=rfJcyhm*5PTXo=d&EQ zcH%$1M>E5;&C?sJ7>4#h0>czeiPTjAi=jlR@&AkpmA5tHU#$FyY)Z^^VMqCXHxUnj zBUw~aruVm;dkZ;`dpxG;9M>y04!?q#xjJ%FXKRVu(>MwE4Mq&KI$#Xw z?`t9^JPJ&3!jp2#qD+ zUr4-;&V19r9~0>49;3&7x}`&(`dPR?=A=aX?_5L{be;C+6cAu*F*Bz()zx& zwb?qu-Ze?Ee6}NT&X{$$Dj*BeQ9AnPmBQ>*Of`iTm~wZ1R9JN^wr}?S%e)=v-|ZD5 zGni_proCG|!fFSX!%~Aygbp*$V~F#74y1-Te6ks#hw@l79no0SD|MkP3scMM;g8=m%HK0~D25=dC3>pPS{9Vnihx;MFY zX}j*sYex1RK{uz}t^%V76qHv%r{4;C7X)o#qp?vm`5qNA!YpHQwkfJeED~?3#GTM2 zv4-QN_6nF?0<11-_q6O53thADV{^Wbk`B|QqAvUfYzCE&l)aY;=fLlcp(;tV#ThiY z${{MsvgKsj(oa9{ZXoz-g(_C)sFs4!lz1tMy(0L6TN2J0FDLrEuNhEXR8?|HESusg z3^TUDCZ=vV!pdgl`UBj?c=7EER>oxQXK~)!4mlb>5CDfLB66F*!W0PU-kXxt%nE{%5KHOy#a1OJmiRIy3RZ+>j7{ev0 zQRRsucMqru06r#cI~iXALIss3;E3aRT@5uBLjGMIDLP9u$G;%+PTKjUUl#YBvOJ%8 zSm<9$_l|0*ago0x0cRT;iLiJ4=k~^ zDiK5JlPZ⋘O7n=^R@K*@;NB30!C&+F26$90vHm#`~Nit~w3+zgr-`shaS@33xjUQXtt%F{+1+ zH|k(AMi>R~5gJ(K7l`tTOMp3X^1^IYiAbZ?a1^j28`&(fJ=mtXAle-1g_w-uD5N&Nr8*gFPC8a7(HGqE$VZQGhmY}@MCNyo{=?AV;xwr$(C?PTU; z*RHSX?Dy2E{r&0xPv1{fcdfPV>-w1tEZ`sc^MqSMkdh|Mqu`AGmitRc`GkpahS!Pa zAqDO;>4T*g?z0ghf+(--6l-U)HGej2iy{N-_?6{z;D(^;yE&w{9Yg2>#&D=5gKlwc zWm$()>=L5#{Ey1!*`~6Hs8WX-CrzlSHh|mI)qKNu3ZpPi)t}Y|9l?Nt4ux1 zl{dtgenyZs%>>`iu6-JvWD(vyjDsCInw081@2>QS7u>P~ZS8*7hJG8j|478#nl4SU@b3yxIy& zUW=w)ievzP1V4Rex17#n@FGN9#VP1aQH(GUyMI(+HyEAs!FBN?BiN;I|HFcM=9X~z z>~bV9hH^S)v^2O)$`|qMH{&N=|FrjyvgL~K3&<2S@~b1TWn2NmrT}gZW zJbHiMTE6CL|93x}{jKKrUpX->-`u?vcM(SWGUl7)2!mbQ)(`DLz88?LFJ1q3sXNv+ z#*UA-;&0^JI9);Y8?Q8gs9#Sv-3v0&->`oVdkVCaJKd@uIR7g5{BF8w1$+N_+?H-HrdRS4e6~u|dtd zu>^&!QYRWkNfjkVnZL3o=(y#3+_+TsLu_*C#3^7*7;r=3`soF8rs;Wes&4_r_v1G( z$%#dyZ>O}Af5^<-6Q#x2RWJz{QcN^u*vE%cR#`L2`c2F<>HAT&l*(pwlcUE}Gv)79 zX^Y`J(yQkj8-5lt&HYN_(`TeAaUsnjPf0jbrWr3s*Cy9~LA8#F2<77g zNn#i&H+Pj5aTT~__)`j?a^wTJO;Jjs|G1vU0ro#qjl@8}Y~F;}?h%Bv*}5yun*dSm$EH6<32-O9YMuj$j>fE9Yi6uxubgmUH zbZw8>VY7tr*cjvGj}|8MYiN;-@Dgp0)NH5qQ3L3^QF{%Ko?hG=Y=)1PyG>;%Fx;uw zHH-@=Rj52pOQGv9Y!aff*l2_;sm%7O4KP-hMZafF=(KX#U@|rfV1q;mFc!U8_)Jjr zewXY1p(pzhnL4`_VO3-nlvhO9AkO5Ojbp{di87{3?UIHEim)Rq)0uS`*hvA1W&Mm4 zyD&raY_`34d=u{^EeYkN)6PR-TNwRD=#0sia!-lL5s2Zc6lew;Ieo~|9H-y4%QN1X~^47-#8rIy5y(Kf{1>W zZ#CGMpEHxGC$3&nQg=AC+POLku9ZNWK-H67We*WCjW*jzBT!U?9l|(=7%wzk-9cnp zkevldoGQXkA+Pl{)^pHZC`z9ia05Cu@w2GRic>Is=lDRl{cxfagv0odoZt+wU(cvF zLU|==32h|k)#!*S3pa-I%euB?xKb-bN@F3L zz_R!UE&<#7rcCjAVcI53DsNr?8MCm$NIn|X(do8VpmMD-isc2h*LE-#SA|W}D_Bp| z4fP>rx}g-~7YD28^wmoCXac0$QZrXwlVo;S6+yiLwaOqMqR|7jOSU*~qa`|V_G;K9 z91gClVzIcMsnZYZhHOp?^h6o1VD2IgI5Z~R{MlFZ-Ik@1yb^JiwUI|IHX#d_@3zV% z8u{?yn++?IP2bh+xJ`AqlA18yBE$^}BViDMA@r;wvkr0krtdPdkbs7ANo0)ET8q?R zHgGZEC;eGiPATrdDlsw!;8QhUUm3uAXcPwM(hEL4LKf)!YfI zQ4J>u*Lldj1!T!2t=3!-Hgnh@mTwO;#5TwLlwWIR5ednOJ(~y27pX|)c%rj_H=1+^ zW5A1tgc)iR$sh8g&?}r>bU{3}26fS>Yo^}aleRn8MN|-y83I#f3&`+|AaXruFx?qv zA>>Y&sOQ{RG^j{Qzmoq;l_OTq02ptw%NW5$LKO9`IDY%|D*7VTbV_lzZ+J2CPU&AkMx z7s=0hC=qX`uG+=GpP2#0ino_4^iSW!hakdOHG~q*TQCX8-ypAIT0WRy#(sXn20vLo zkP8&fDx6pb`1|oam1kqC^zxhNX2uFmbWaA*Bbaws08nl(WP5q#QsIXs*`?N0>>|6X* zlL$5=;QIU|LLHN6u2nrD0|^#wTQk>YuTRElFfB5e5Qh}}(M>qE$Cj5p>Mv zEa`N%W}sLf7Iz=Ps*ujGk1#pv`K~H=4{S7UF5BVWa3Oy`MMUIg8~n1~u%?#4jEuD2 zGE(g$Hz265Eq-x~56_ZQxyXK`jss}Y?|RNh87{aRa12-<#lHeded&HMn?T^JJRPY6 z(#||xGuSgv#?WIK_^3??qx&}l^{aS@G%4#SJuAe+s(N?WA(wu}j-2e;(ojSm?vtwQ z@-&Q&LJdl7F6F&jya(tqRB3p1(5G+KHahkYHc=HdykBz>0lFsfT!d$|(z%P=)=B0{ z-2A}6B@27bHDf@H0NFn?QBhlR)aa3`^m<`iXJ?c9wveGER|dL_bleObYirhNxDf40 z8+8nG=+ivpEz!3DWVC3_E8HtaRw@EYK17O|W0pe9C26xJ)eZS%_W3e-*8%w!rbax? zI|jTCUle)SNh&gM!A zj31fR!Lz_#;df?TxddTJpVG4#cdtJ}0kYp~k7?0Ej?bK==VjDRim{*1oNjBZ%RIrI-g%M3L+)J2hALAP+DY~> zSU1#^ncr1kr@HMhL|Rl&IuTkb`uHnM1== zr1eCSxJqyiFXX3iwcauEBV=3OtsSP6l`aUotHr*?w7CLoZ`bP zZ2h?FCqWRMuZEN1l7gPRqR&I9g;kg5No|O*6@}PYH)bc1kiQmt01>@js|Wn{xWAqB z7+R=DF?W8%nvs%G*wyauC6uIxyH7e z&ubMo-E+!^|6L%FZp!f&uX8#g`Zh%QHjaxqme>XCm*n~4M1d54=%%aZn2D}PTnN^( z$FWZ`zvr=Klz21xEVV2&CmhUboQG|1iQo5VXSBW5`onOL9S-$KUAm4ds!X%IVC&JVW1fK|hL#vfqZBH2db8ecN2$9e&vAV56pW#Lk@uda z+eBQOLM9Y_Nw-(XhKQ_A2meiN?EP#+#%M^br8{oQ?@gqF8yf1z{q8nsiYCH6>ZD#>E#>6EMIGeMi=FhdopcFFCpWCCl}M)| z4W-j@0pX<6*XR)V4@g*E_`}g6XWhafmj-T3=5T?Ryil>Uljp~2`UjZC<4^rin0q(= z-460vzPCfc-ejE#XJgA3AX-JoJI%E|xJ_;+KI+u7g{+y5?RwONmJ_P+GkxgySM ziaok_+8HpCB@!7Jr%Hc#3_z+CXN-bJQ#IAQB_3ch)B@W_`qA0FSO^$8-~Y&w{bA*P;ZWE_bfY{m7VX{TnzE-Ugl6;4E5~6 z>(D*i2c8XbpQ*=*aj)tFPP?w`w&L&!xa}xSt6$vl2W@8St?YYzSMb^ztj`wO;h8Sx z-g(8-H?!@C1;3cS-5boTbq{ZJBsRy?PcR+uRxnAU+Beqivn(d8cckeWO6U0=dF!eR z6A9#Kvw-k4FYeZUQ<_&p3 zMbtHTub&J+=wh%IkW1aKI9wIl_0-RaJk4&tCD<780A`yDb&bE=)vnG*W;n2+FO7{~ z)HJhtV-OBMJK%X&k5(53mjiZ zNiv4Zv+F=~e!YKT`EpEjX%40)0{Q2s@kgQ6vQdsOsS4C(j)xox|2;^srUxiMg;h|d zgL!yDO;d_a9lK%e^6=YNd;6O|?+*su1l}~8?G$fR+?L-t^34v@@3mKqJkqB(F>ESk zzVWjgeOuB+D#EkDj@iTuSJ&_LhbV&Nr*rb;L5AM5*iP=Hm+^m=!-R-TxCk402@Q1sJW)?5>U4oxgL0`IV2bB{+U;+`#QFL?{BHt!9cJbnVN?$^cuGth z)+(E3pF+_QLk+VP2lt-MB8T*i1XhPNewg6nuhuwB^-x3yAbI^xU_ijEixH|Hsk@sw z;wc79Y*9O$y)DEIREo<_4{~qUhgwM$)pWo!+T~Lc6Bk#Y8wh8T+@I2^lpy zDDVt65ZLFYxNjeS7OmHhGT;3~cnwhu?Vy8dL6EO~pgTN~)V4{;p{<%Zrm!G55(aS7nk}_e%jpPx+fWT{K#gcg|Zbr_UwlYNl#h`3W4= zEoa9?cO$}oHTw^;Y^+@8l(GE$_J0J0MRx3#hmhaCp?rbS|5ZAVP)jFJlK+@fjz;w+4@;iIM4gL=m)#FK`f^_>ny#X(B-6_L^fY)7 zJn{KlMV|Tk26Mm0xhJ7#50wSjeAp7F1^YPH>zmEj&s$yAkIzDWe<61qKj`;R_^Awa zMPWvnDPOo-5!)Tr2kuk%=1(an4Alqlr^4O8-;-WuW8IJvF~PWp+Z&+I2FQ>yidgo6 zsiI6u!p^)nX^TR$%4rUWbrnap1eEJ~CjUEpG0^`_(qn3oohpX}lGnq{$dt~`G+N>D z6>BXTX#(~qQ)O-h06;kE^0n_cxX=$BcF3#)r28RBaTqE}SR5WoQaB+r$LRBO%6A1O z8PS(H;a~Tyknw0hxn{c4*4aHcW+XCnw)-JvrpM?q$IgDWTlx#|2;dms%8co>$o56J z%P3LR-@>y(4L={k$wGr9j7jiV%p6J3Chu85x=nQtmQ zX4BO{wK;FHJHXKl;FmitVqdIL*8Kxu(QWBQ+(=`S-aV6AUvers7meunYcUCaI4AX8 z_)(q(e~TUoQg@4^sxsYd?*vhet2HjA*hLKTy2NZ-sm1lS%WR1w%$Exk; zym0XYrlnWR7hUHw5QFVzJd_^oQMpPiatt}fCy^UE{Z|YtzWg^;8lAHV0}To;qEo7s zFGlM(79p2$W%!|-8hRzVib=FO3Q-PTR?C@iz`iIbKvDK#zhfKKw`lkpdlSYn=J-s> zRvfv`nk;~bv~WfRwc;#p*!ec)hq1fW3C^%Ge~d&KVKHTDG9CA8mTGyP8n?$|_+qp+ z*V~;maA~O?0l6eu+{|2>4RQREx?X2R>42lwwAcS9*5Hi{$A^pX(V&ZG>CcfxeS!1z z-p*Xz!zd9f;+)2)s6q@lJC*)k0N`TKNeR#TcG~34iWpE+BzgWU0RKNs#MjG z+DZo;i46k=CpG1JP4OC~Zr9r75sQNPKy-FK zDIcNZ19y2dc5Bh;6HbBHv(sR?I_JDnOo5PEYI0;`{_Cg@oR@0+M*QL36AL;C=4=BPO>4x_t9wNl^a`!^k!5p|5f1Xxkj4xU`Vx_S z)fa1`-Wk0T+xh#m`!5+FVc~WH#D?^SLYy)QzOx#_F87;{4z$HT2&hjyxwiJ7%e1^l z`xw0Fv#n7C=L}hN?K@Sd4k^gGNW&+uw-w0Fk?57hwV@g54@ST?X0*?op7%$^L{!slVOYosRcg}Nck>p5<{&Yr!H5k?I{hm91V{uQ&&eKOWYw{K_oPulYb+}rohl9MaIs}&ow z`Bm%jmRIWQhv)k@hv+hYgfUzo0=eQ8DM$~|3zrtbvwrbvqY2jyfn!|*JW`Pua)m`tiF&)g0eSs<8(wZo2Y%x4rS+kK`G=>_ty!VL?Q@^&1fs?_=P zZZc}fqce@FGP@KZgiIO$kh|7XB6@8+vsDpO6pG`Wu&lLtG#`mIJoQ>+48+%wlVYFc z*_r>*3E|wD+mXTt?mBeBQnH^QnE!j~SBxssVtj^LGO;HY`>o>nitq^|LSZG-3}IU* zA{$K_8H-XBgJpQnZ9A71J}Pr?%I*iT(nFaWJ!fz$Sel>YC9!Aee?X^v@4i$vh#}Rq z*R)QJDqd(A0r%xS{h#Oa*K&FK-Ude0WUw%M9GlDe+7{L{u7pPL9VrOF|>(@Txp@xC}>^T zs9uq$CevsoCP%EC)TFLczNBsKvnWyixcTPmf9N+gaRG2+lj7WY{_OsAI^NhlAIMh(5BV z>Yq90G#d00Qe?zpW1Q&NEy<)O6-z%Xf}LJTT#6rp9v*)l#ATdg5r!r0D;|*4N3Zil z__1a7P5JfHQ7tJY*_6_y<`T=;R|BO~DNT#qOEZf+O9N;-*+EE2fHvt+4<+Hy@piHf z7jm^S_WsCIm&6JDO64SkGPU&6UIbq8bS*;tSIwm8`#sai&kB;dp0hW)3?7nvrqvGkW-5oDly@I*HAfsA$^0H>!`$q?2Q&HG?EYZBmB+t;T z4S;GB$GM{bx@D!!GGCtfpZe93Od*{Zu7X`vv^!&QmuTVC*pLcn(zxicrXB74_-*K@ z(zC#DT0Y&0iym&5y)6{g!SBRzZ&U6P=KQVcQpL=%e^8?F&i7@F zW(5^37EG+I+%KP^TRQ(pksD#*VD=BBKdBi9;uOXbr4ifT`570Ka&DmFu;%-GGTRCy zD1X;MaVCu55I@2uX^KNfg%7B8Z+tO-Q>9(i#?-P7F~(e_W5v6Cd*i`D8k$#?QPiMH zP3J8TU<1?_QuQ_XF4DRqrBfVCBZLIrY6W2wuD%-mE1E|Jjmp5b){}h~Ar)h`S#WLQ z!kHC!6hWefC!O@PC?!N3TgfK>TRt*0c~oW3$txN%A!*maFp|gb`D1qyJ|s#NXqv)% zEp415fxnbaDLbZ&AM`ozu`(04V{-XV!PErLF6?~LdWB>Hlqz+`oW!{68I&dq$51p^ z7_?ru@Xpma-pP``TPZHSM2@e|>rw4w>r9PmLO`%2^bT$#YO8IY5spf4)GQAR)5u?` z)4H&}fdiV7=Ppi=e2ZxfLNNy6-1`1e^Ar)W_cRm)IqOn;9jE&=SvFZ1Tv+n8qp&x{mmq%b8G;Ym#J`yBeX{wIchSbCI)Y(ly>z7wkPs8 zsUo{Z>>k{2NDV=S@A$Y(C?iTq%I?IlOFX6xmJMe%6`rs@8Ui*lIqI$sDqJU_>ePWr zG--bqrsNHk<<(Irzz>`=D8kDQMPPp~k@WH_c^eecz4Q!Hi3#fcz^!jG)}pc&Askox zDN~bVeI+WKfkIsl9qmF>l%RDxa;HUBoFZVuXhZ+#C9|f4R+lAJNMlbwUUtPt7CZK4 zc`9zzoxl}Zhr%h!`z8EHgrx8HEf0k2H9z5jvQJ+$gkawg6`Hvb&@C`xvr|2l2Kh>A zShf&yIaDp4NVqhL3l&+bzJ-e~&rv2-&CkdQE#BCAOB9}SwE1(EyX1>=%}#Sq0Fqi>DgcSec`c~#e*AmKeTvAC z)2ttyml-EOkgt@RMGMr#T$W3Ucz66RCC-+d4u@`NPF*Q?nhl#ogX9YP8oHJLT8j_> z`;x`4LoUUx!@qXzac=al;RI3iz~@DAU6$kuQ%R{~Lti(wsYAtG^SL#GUliiDcEM&? zvlv)_{jU1R)b*?b596#uExU%Kb0NEilrsn8F7~pu`~9OPUNN(9yGH(diAQ@y=XRlhNJvNF@XU0bT=)J}NX7BS_EjSq zoP1p~q@84IP%G@Ve4zi#QUIl0d3jS&;f9NkSD5@3n%isc5AG)OAid+lK{&UM#@(~@ z4MZG*sS!j&VeH{GxHi8OUy{m{?d4a?KR)rW)Hgh%8($4GOG<^!ueW3#RZ;Vl%1=4P z0O=1#W+JM;1v30)PN)GLawPpNo$WqxMI_o0N!pq@*?yqh?P1>zEB1iy6w82S4OSgj zJGK34CEYkYW41I-g>XMp1Qkz3K!SNO9B)B_7-u=RqkX@sG_k^~O2bm=R3XT-G!P5U z6l+T7z&I87e554}x(k1c=hU+@xV46N&78=*Nf+wjUKUkzyb)a*RnqO$C&OfZQzMa6 zO2iLQcjBN>7+pqJbF_@mx&1*2z`AEcaErZX+)edqlbb=W&MTGncVLxOf#dO|5gYp= zV(reeBi=hT1yo{i6k-!O?Txa!nL|AIvNp$>9cR?0n5zqRAw;brs}IgZ^Jiku2mWMV zTkVZbLmffJd5G$fYPlu}FJFHMntn6~j9}kb<>jzxC!4#KBGW=88v>SgaepJ{N}1>; zyPC1B&~V(V7l(~bG=XQ^!L~Lmlu{>`Xr@8YU@)3wg!)-$Bxw=EC%`NQzM0~3=k^Gu z;wf3{3Q~9KpPc#XQWP3%_8kdxZ%Zv9ZI~-~RNEmHAz}<;#uUKbZ%mFR56)X39ho+o zbg+LK*UqBG$6~@p;$v^(aV|Ojj6B~3M6*#PJ!#SL@yG#bBj7#ClWDY*u=yl~+So<3 zCLrokIoqAu2sD2JzQLZ8$du++l22D{t>Rxhi$*0O1vezPAtd7BhmQ^V*aMrzjIxxb zfvi1UZvVE!l8QpXVO7lvk(y~b*+8~R<+!Z=6rS`E17|v(*V&P1;whd9-Y391b4oviXsFOBc$JVV5KC;VZZhG`@}MhNZahya}G62nkdU(KgC>EHxfa;#i|fSm8=$ zCg{m@8UP#z*U)!=Mb@8n>3AUVygSI>;OiTMk>^;|Ru$C-jWtDfuVp;r=T?|YQ~Hf1 zwN?IV`zL?X%154YHoMX@&kmG)iZ1Rk$Osb|iyoHCSOYXJs8Z8&W4$)Aq$GwHkOQT0rGhOzglo44!$n$m`N)lG*`WgO>G&GK4cPB0DsnBhGMqzn%q|b6b8ceEeLn*sX zXVttv1pC@wG2@V=hUoB_(Tq>WQ_9I%2v1gcyl=Sf^@NAqevCTANi!oTzwRAkPnGsi z0`xSZz2w7)1uH{fZyyjh);6U`?N}?8x^tp{W4Ne}sEr1fmTkEG2PIG5YqQx8gMH*# zcxyEEz^B$HVw^jPPxN*70|0(OU18UHw|`V3r)LonOARcPJzH22nCUTNX0g zqlQz8qq?W}>6C|rc5ahs_KEIZ$`Ws5Uarr+K3_jMUZkA78Dh8!K^U(e#!uD5h)2$B zb_eEy{To3!B+y*|Pk;q$6|3p~3VJXZOf<+2& zs3zSlCt6|WWH$m@&G?utUpW)uH0$`Pn)$k#b>`NyX4bP>k`9+)z-_U6;9zu1BDZwR&iPlRgbp$yH)m>MJl6M@zmfAJl9+{k)M)~Vh~+0LkERMRt|Vaz3Z zI@InSTYjd_^;?;(`iB89^o`#6bOC3+tAi~)em2ZkFLyE?THnGwJB*3q2(ONIhfJ?I z#)TdIc_4b!fKL<1l?A}HVl@z-suRz#YP`aF#rfT?b`tS|w3#xDbb2d2RSsE%g;j4r zA?esIvyLDEOIx}tifG?cQ5=CI7(S>}fSQWahRCf56_}StyPPuqr1qsy$X>?r^hPd% zeaNMmsYjccFL%Nz7)V(U?1wmU2&RfD8A}K!gESOZ>tfm-J_0T@-e`8_H(#{+!NZNU zDVfc#j(IgC`jW%7bf&hp2EDc=u6E(`Ms7c@MI^(`8QY3r;!hc*DT` z7Q2Oq9e2p_rVEdy;oE|uF$z!2lT|p**#v&ve48QI3FRARg3Uj4E}ZQp0!M#CFy*vE zgThbL4sSh7jbVFJ?1##B)%4jk86G_*EzC$6WW;z2X=}x~k~J*r-XhAta*pUuhDy5v z2s4KwS$d?;9ozm?8ary5Moh1K_jyHPXLvqz%lE(o6MW{x-n2Lsx8_=NC&1L6fsizG zG2!S}Lu8{0ecc$eju2U&?Lyow`<0^sknOP(y#T$!d9bXm91yx3@Tt`K%4{9-3(fH|%Quc$Co7%5-nSyVVQm9_ z53Z%~^6F`$?d@1_rE06PXZjr4v&}K_e_&gE?bx|CYJ_TpGDFW)j@ZrdEcX7PWEU{v zf|&#kg6%s}7k!bdhf{_UI@4FK(8K+VOLyS1hep69td*>7<72Os%~zkT^%VD1xLb12 zkP17Jg7b%O2w>ks8>^6BVdNTrBSp4UWiLX zOSMO41}UhnzK>Q&kVDJjT~slAJSOg)n;D}wR=HBS5nP>ASwJ15oPCFVNFAe6b%%c_ z1GD$^^T>#ta2O3gYRI?@M_ac;Vse5+{|ixg^dao~4we-q4{JcVr$;-06-Uw6EiA}VBz*FaE;v*^|J{8kJdt2G zBb#^<;o=ikzCoV2V~HQ0XapQV3aL${BY1SfK;E{w{Y!jXD8$jU4o#|hek`Ef%UgE7 zH$CWLnq=g}bw;Us$a_2dZ+}7!9U4)ENP~3~JN}FkrLCP+LFAU7_4socrP2bzC)r#I zdz@anP3PC6>2PmdC zz^}^@8!x}m;xZDGv$WMYqT#kqUY#1a9*<38?m%|CvFxlI}0=rCPxA5<_b zWP8b~6_&^}AHLZYzQI=9;Vemb);stgIj(`@8>IuLR~REwgY8Ut3%`qm01Z)s?zkkJiL zl}$dSAMq}5K`K#M2y*YT1OF)Y_Id46TVgUywOmR!SVz_yG^*VkQq=LDYr=3;SU1t3 zB6I#{&>g3gE-&^ax+Nz2zj;fg>?~a@jchDE&7|xcTwVUVgc#I@byr=)_^Z&w=iy-l zOTrw1jY&dI9vB9L_I2S!mYCeXrtLRVg2bI6t9@n~aqXgZVacMlwQYH$_Ib!=MF|1w ze3@QBrR`-^<>vF+q+Qce*~7*=fRZf%mM!;jKBo1$>wOA%{W|^CSiC%(#|R)9(&$y> zBDz_(KtEeq4Eo?+4U??$6KBMX9hua8{>|cGnkS z^QC_EC71H8*rkd$#h!-!Q0>>68+*THd%Zl4|BD6Iy-*{KBot;adbCXOg6OLgu)T}p zk99D9P0>pP@wa4G;idk-Ps!>vvd4hq1JREOaQN?sdwA^jk|SYm#U#?BXW!q=@hA7S zi!FUNRRwBoGsub&tD;ivh9t!qi|5p!QCsyM8={kIPLT|dU3Y1!Pl#VRO5K5%O%3s! zc`z&?#R?2=m*D*TF&k;f_SNG9UT^Fw+YXyRWkoOR$C4Sqnwa~CQDs#Pj*`zWb2Yf; zS(Ei%@-0`On~(Y}eb}isc>AR%PxyxbkjqG3y)~#bPH-NM0-l&YJB&AAsbkw5iY%D` z2L2E9EVr#Mr z8|J~Yjg*9LMyK1txLgYz(B^Bx&-8FJzaQ(ajSG@YIg|bg2~r$L(@?_%WwA&fjUkJJ z1ySHEo+r9N{L|2h&%} z*=7Zo?c#q|)p>CR?{*z+79XTq)==h%j{Hd>VpL2aKFp6%N@K!|t$=4erw&*YC8$X+ z^opT7;I>PdDAYsgZ*~hrQN@J)kg+9Zg8#^HF7KfO`-kGLw!?vh@9alz`a+ro`@1t0 z$7tV>t+{n{k<)XMlI;RsOBlAG;IuX?ZQ!F?$_&i0ZW-TnCe#atPToZ@lLn^^Dj_Ym zX2VGNcsK{cZA5GTjDE$^p zDO`*w@-@@w*_3S7cv&q8H_+6zv*#Ra7{VNqbn6oG5O$0T-A$|WI~nT_*6^ZkSjq9( zH!h5rDz(g+%sR32I_O`53G+l$$#)IX`SQ3^>~fUwM&W>%Xe1Xn+L_YW2p5|{`Dl+U zh*lTcWDsQr>YOWWp;S-;+S~bux?2`|d32|Nc`XTV+b$ZrRxu5GQhn%Gfh||;lrEo| zBNI)&0FcN@AJOJpuoqo%FPpK1np_x_qe3YHc~+62vqG5>9$Xa8kFv|m(C|(cseDl# zcz7oZJu(+u(iMsw+vY95GmuXx-%IV}M>iK6@{fkNmgR|cWC2)b&7YswzndB4eW0jT zsvt5@PDl=qr+**cChNBa4gkI`Uu&n6SX9&AmCmv%mFFbPO0FSu?>$vl?WEp&E*@7iT+%AH?=x zR7GHke3>~3w;W;VLTHXlg+YV-kSPAzwF08yUnhj;V|9UE{#btcvpHy23 zr?6_hB{x5FG-jKp92L6~eQte`WmunIi#u3Xsn0CKru1Hrbk+Qd~HbE#W^_#JIQafMGoJy@L{9V>Gf6Q#^oA>tOl8SUzU9k7rM=cLNb4 za9>Bh%0Qo8jLj(`@4~ucd{pPvW}JT(6zf=e_u3+-nL3Yfs8t`dZ10j_HGTUhb#xF1 z8R~E}g#8WVplQhfvIZ3fW~Dse@@Km0NPPw5DWtfOBnnk4} zl#)*ACJ2mRlqR4K2=)sZ0&7B@NlH?~%aL_LmW zzoKku-v=mWV;-iMv+8IPHJYiSt8NTmDS4|mw-|oH)_3W>rOwEo^NFL#REqaiax*`Q z?dJh)4CNj|gb(vqMuoa;4N?=j$Z^ox`)Ez6U2DNRTg82f*np*IG~p&s3yFn5q&2N` zDgYw3ulY2M+tct=SX6r@t5M0A$u(Zo2q4?7Xgz)(VzTjf&%{&gww$Neg4>PgvJ2fDQ0zI-;(S7CV2^lW=XvJJT3H0B^o?}vRk7Gp4DGSb z;p&J0vQT(V@S?=e(AI=dxd=%lMlr1BHj&?xD}B?(HiMi+@e_z26A31ZN(YMwBz~&D ziv-X1U~&nP_4a6m`O zN`O>{4PI$h2jrqet_E`Mf{p`Bf&{bWFuSZ^0L>PdTknsersbhY{9kF!4iNJ|eBsvS zk|90p!3PpmQ5F%PIa0q|3u4R?r2-vOL_$ZOo+8j`wSrn+>&huIpSzy{{nll5H!V>Nj7v%_}v*dSjhm2bzr0DAoZecdtscSjB=N0^j->Z+dl z7F4Dk+Kz$zVTa#$4zX~K45uy8M|+-S{fG@kEVnPtsUE#L1yT+@L7}bV|NY(H%rC`EMwLczJ#~MCa zFDy*(LpIpZ5zLsP!lKB|dcHle=)#wR%$ssF%$PB`qnE}t%gHVo=r1cjD$%Nz6)7EB zdoU=59T*}YwVxSzxQyfhkndrwl^BF zAF=pmU>i-x{PB=z)Nx?1Nc!f4CF;0^(!ihidq|h_TL%o7o5RLdc-jlh(GDT|u6{_L zPi8EkViiJNnw+d715;@TFR*N%lT)u1rp%3Ct5i81bnX~9Ce!33+jS;2j9ybD*a6Q~ z2VQ*B>{eeNsd>)X5h_fk>BhG;-cc14iEr8LF#QUI_e)?b6W?Ot6h(8->G^*cd&eeC zqbN%|ot3s#X+3GX(zb2e&Z@L+JZamuZJU+0J=N1Q5pQ?I8$F-yKd{d^_geefYrjMC zBkMnDL)xLePqebcmmSk&ay7^apCd2x1w4kT>JiDVOr#wT zMBLNs5q(|v{8v>n*U4lN|4%z_|D$aGr8WIv5&Tm@-%x8@d?U*qbxh8ag#9bb>{Bk~{>9yW$x zk%^ylVFzql^R*znogg0~Fr2JKiwCH_WK)-t@2?8LS*z7@Wd> zNA;aS`pHAYqZ134IgScAl@Lj;tnp3PFdcq=%BF^@Fr%fmZk}^bQN`)0+fM8?i(c}O zXikR26WvsMOm)T329Q}-OUsMl!FPJSRep~}EmSvQN6Gg5liaxTX!dX7-XtCq&pLU3 zZ#Kx#DC;bMWYH(TP=!18pmjr^J>GwDLfaBLu3U4AcVdOf_0W}85QXJlUB0t+EMX}6 za(?MFr4%ETF_)*y<7g|YLSMp1Ml@Pv$JG}uTUwR(Z=tETGMYHNp+@fQ6dj$dwQI`u z5H$|bmYJmEB6Q{>H+GRrz$=Nlz*6lgf;Oa4?*=eCk5AIF1Y3rj^;k*TyBAJOZ|7Nv zla zk*q;#`4)Fdb{qYMZoq;bU6lqmxjX`e@34W7YnW&Qove8aVqUWz1oZ#|mk%Fd$r^D! zdkA=1b|5$(gj)@p88p?I)7KSsHsn^iVCGiptmjsmgeNG&aZ>S2ej9`*xPwh^=pnv8 zN((7VWN_^e(ur#{7AZ*h!%?cm#`0)UKnws@p_ystC`44mz+}Q0iY1hwjK!d>$&x#< zVAw&U>>`!zZnP$Xv5)kTVEAA?&p)t?JUYWTyM*|ik>HqC;3@}J&Ho1Z zuf>gsTmDPwUvUTgGlrP|$KsZ>`v;WS8QQ42SlawwU#Ln&S03#j_P3B?r;SA&_9HB! z{*JspU_6p2ev#eI0@Sh(Iy+=b(%o>)vRHdj|F9tg+ZbNp50PnSLmH;6gv`&iuIH?^ zbnmOlt_I2;KYVbF;RUc^N0wM#in|~?ORtSy0$GZtwu&h}aAl}soNcYgXTgg%w(a^t zE3M8UU40Gy=RzAzKL@hPW`%5x=ELo7hl8J1Eu)gAwq%>9cc`(V%H^@}WxAon&zlH= zRAKteh33a@*v#aoFd_a{$l;IEd)W#I-B|OZ3aahAn|m~~A2SA0J_c!}SNEiLp}bb> z=k?o;pHe@CYATadX0YGRncR`6?oALb=X<`+_$JflQIAh~(pA8u@R{yCba z3;c(41labi*vB`63E7H z>RewxP-nh0zI{XS;}&+F7VR#4P#Rr7=ZP{d1oM)~o4Sp zWXulGn5|)^P%qCF11*}xokgO9{KP^YLL9+;v4_x$$7W8v<@_KkJY@N!Nls1CmcZc? zfF-I0hoKJ97cy8?E{biDA*Td_S3v*s(ob8eyC1%?@$E4web{U7W8>(iqLpa{J!62s z`q7_I|FuH38#Qf){#ED(6bK0W|5%~_k%aOeSFoYEsf3}OiOv6oQ-LbB|Ed%@+ZIlo zb~JK4GB?1!F&AL;Vif<6PR)R|l11xDv$;+=PdLA9Ygq~FkrIfv*hjh%$2oqIqAoHz zO6X`y2YUYX{r&m%g$A;`Yl#SDuofWBCDCULrv}KytVG4$MDCu=gqVL7O8!+$Mj&U< z2=M8~VzbieGG{{~M>Dl#sTofSdre_1I42>iRZAH!0vNaf7uu&-?+c~z01BJoHgfLL zGU|;glSdts?!46z5N5WgL_au`K8lD$-jXZ36q$FW;9}zNURly63H8VAcr`kZA4r77k=S+2Z>^&sx_i^Yx6R!u_7L?uU&}0 z`EW1LuV}|tOOtmq(iO&r%I-Oc8ZhX${_Kw1h(~=3tCzEGj~wB1*2-k3?6^iAb=T{L~qp)IN>#Df4|SNx0)gDXZW6mXk9K; zcgeN>Td`A}q4@`1ug#-x|C1+Zk$DH;gibBUUF!mWi1$_aGjvCgv=d4Fh9`^aCjsav zaw}{B6%I@L!Kl2;?aD6q9Y@^EPhgN^nox9<<1^+CYxl5f`+(|HYlb``n)+XRT(e0o z|4hb6#J}sHLkZecZMLl1S9@}Bj@v=)AnpiVp=EO?Lyl zBaH7Zji<3Ya*An{Q3cG^TLp>%_K?Bwy1>g{7Z5@}9sv3m(k4JIppp=!i@X#yI`?T| zAejIDAcRn^t;K1sH$(e#c4N|4w|qUxSj0i$_XE;-xA%_swfB41^~d3FLT)gFCLOeX zOnj=5qSl>Wa6uj0-2>vJeL-+iV*Ej3@B-qb17h&diMi;|K#xK&@tacr0zqE0-CN}1 z12iap%Uxwbd^kgj>>qmzFHu^2WokL972twCa(n_cKlMv{m0ye?yMq|icWGbB-(EBP zYoKqVVF7&{2tKr7_7z9&LR5Y3OEb^#J_11)oCdql#EW}>z`kn1)xPGkG_IefK$UBHySE#!(AW=Yf(LNLc>JENf2+{(*$6WG68i2b=GvwQ_Cm ze(>2$$;g8Wvtb>rJyr-rPjazq(@9Un7y?<|-Rfjc^V61VD zZJLP~t{VKc*x){LnMQP3|0&p#Q`p~tdHmC*gRgAWz{`8VmC1Y6eG}qVDkK41YUv=2 zultw`s#R4#=(o5LQFOkAH(hMRQN30|;0{Xc2DqA!FTjj4u}wnR+lRKfWde8k$)u5` z7iVcPdDCmF%PoHdM?%AIya4Q6-s~?O^>5B2d$)IP)Q+v<3?r?ib(o!vnd9OIXEj{v zBc5BTA&ug`!OVRC+T`Tg#f|OEoh}BG^46wvlBt|1KrG9gQrS@-nUHcXB&lFuo%r$D zRaEx{?X|XZKKN=3c0nt)V zt-j^tKp88H$a2+TmUv4H<%b8637#;Lw!~|PI`g(yAu8vO&d%G28Bg>2ZGQ}i@ zf2q>l-7CW)tS0QCxPrCmFo}c$LVR}Pk!!XwG^@%Zu%?0Km7ALuiPT+Y&zY+sm&KEP za)J-&%KFJQHbu(=@zl=bfV4W6k|6ujje$-BkZG19ne-VaM<4sZ^I=LtbV(i}LC*G+vfW2jtsJ{Jxg5L{dsTLkMTN*?vd`8B7*`SxpjY zSxt%wSuW|6dI`2Pr)Dr!X#~7NYTCl(IBw;$U@Lhx@(61gP3nr1G?M5{`26BZwGDIh z?CCBz8BNj(8J()M%4AoYT}p{`85{e?WUKYGS2|+e>}kh>t|)Wm>3C<&SQ#$)X%3W; zIO^#7q@wX83QP7RWICt3TN<({N0^zVRH{ouL`J$Hu#~`p9Dma_)7?)L4cpk@~iS3Aa%}OGc zx~qO_Fxn%XdL!AdLAFh&n$a|rHS>Zk67rss1{2`ewpR+PdDxm^EOez(m-57VxluP z303A**+Q+7p=1FSVuE6!8dk^Q&eF(;&iMyKg*;6Q+eN8#Rx%n2OUyehwASIIk#875 zNkK=6yRZ8>g`5u{>okC_wbbhaK4$~XrD3!~DZ0aWns1J9L@4ypi{X9!b?M+;CVp9& zg>VpRJ@tz3?Q5Wwu5-fK&BcYYk}6*svCw_iIj4r(-|nd);^#S6&hXl3?|DN)`ozlQ zSw6=&v$e;noWj_fQ5Hq(Uiw*t_lF(U!&cqDsY0N%=Yn&R{YbUU$T@#pds6sXM^Mz%d8(U(o1@TvQRV{dkq zx5cC;UO6=6>DvdF&PH61O`Ok0W_Z#lh|DT6!2DFjmX)XBA@?1!$M>9lCX2^s^J(i$ z#yrX~MSZr!@+Tl)+amE-{=ZDpe;pkHby$G|xtHid=66dLUn~<^<&V42n6rHRNIy_& z9EGEm*bURnqT=MwQzxEf!Blmu)7Jz33}f>4I>7jH zML{~#OSt@=phBk+fpOqEGJO~y2)8hmVj#yx(2J0gqe6mNzL zPLFBAt9JeC4Yjhk?75dtuEQ~b9~B}7sh}krz1b587eHa;YYUvjt{4*9-&>y8Ebb~( z++l0FO{8fHy1(I%Tpvl(yt(uTqWv9qvHz}67Y3%OyHarUgMLLB_(7Qq8e6tLU^jLt@hjGj^*5tw1my~VG zp;43C7i~MXlldEFc=JrT)A5F2*v2zuYmx5>e;%K*7uLp#RS!;i@K5RX-`n-N>Ogv| z(_f4{!f#{6nX%W+`{+w_pLSt6%C{MTg3XZF><&Tc-Pr?;c%4S9*8aNL#ze(C?`=%i{i#6YR=iKg97OhDtC^=b@P0AOk zu^Y~$ZqwQdXffHu+tYjd&_)^0aLq~MHru^4+Wq&n z+|_mAyTMn>t3(c7yk34}`lY=&CUr|lfORc90;zLW6wv4dnP}&$>&wX)scI)|XxNoX ze!_2Co5Jd)b}EH?|EYC4UOAtvSw!0)rEZo)$S?_g64A&gk5=8nq6MqtX^S-R|wL))NIG_4%`lZ+SVzq+Q7ZiW@2K95f9Y)@wnF zxktR?NN=;mvE|O5FfX-!T7P&6l^3$562E7_-b7jR_4j{+sQiWHr4jzQ5K71(Aj1En z;QB973;%!N8Aa-v>gfMgHNZl7VdMHc3Py#OmX5OHhT%nw{|qox)MOo&@bZet%PaAo zUpPqc-Ff~bIwWVHI7XfnzWXcerz|IDr^+$F_^_KBD(vd|Sat39T%p}!_4Rr^(*wdA znM1l7J%Mu3pZufRh&g~-Celodb!BkXboI$&|ObBJ+LtI&|TmUMI%0}@gA=wwhd&W9X91yp)Iyz@IHm*~(HxE<=BMlEGEL72@#>Eqv7(Z_3#*99rF$VOavZ z3V+8%XWlOQ1tA!{8P_Q01NJ9Gdg%sH{86Ov?fLZvN> z@+&&)BpybZYxI|3OrF*Jl8%tOz4M5Y$|gk;0He0ESbuy@8(y9jTLygn{R=Bgq4 zy;`-yUrzkEF7|0SQWeKw-vxJg>zU)W~-O2QU^^( zbG4rxpsv~(vVzf>myf1PW0cPoP(ZUis7;eRNr=&9bitLSGRL4dlPli0L9^Z00lTey z%g?WHOUzF)Ov67+3IBdqv22SQZl}m>Gghq~u?b{Tg5_7e#o<@Kh3rti>%h<=Fm>qJfxOQYm`uc+$nv*++oV{%(p7q^~Mm`71tV#c9a6G z95h_5ix2m~hxN<@?oFm3oaqcZV*`#;h|7Ya-Zo*r;Nt5{H|D z%mZ-@ImnPs_erKK(H$^ZE#t`w(9)mBHsjETlqQ1kH}w!+TY;J1RqRqx$fX#sX(%eI z;@Ut=2^F`o*b)h5R^k#j;~~iNr=>Vw!jz*`=-oH$U%w&a5+iK#H7Jpq#xLBBVb&{P%rLuBhpwY1O2 zKS9TW_s5^|SH!Kx1|QWtTCTv3zkDHywDxR;=;V&GOGL6$Tj zIP)%dB8OHINf?LV!EtwQy9|1Bk59ieACc%>T(@PPUB%?fzZN4Ea;5<)1`J)%xd~q& zMO_8=NVV^aRlWR{YQ#0KODBDPXH6bv7DJ%Fo)Gtu1;h355EET_9)(%w1;1UsUJ;>A zspA%($W>eb1x_$|Mhqj0k!bT1h%@ZhqT~!)LF!=y8$UYAvENw@bExq$<*bR`H@z;- z^zT=fnD6dQ59ZhO0~+ds9GH%Y_kHTfp7Ka8wxb_PsfYctYsDx2%5q>3y|?F3&0%Z< zrNbKC0eNmq1DKYM>2cM$@qXV`e`Q&($s?5gwngT|D;n7Crw?6m?~N^WmLw5eID|oU zg~3sD5XzaZtLNO+PYQnz!B|rUS-mlLoE64IyYlv8hL|qKy3-XHMuFM#Vpt*ai6lOg zVG&n2| zklq7>97qx=SVWs>l&2>rUs1cz8dmNpIPI=vugDa!;H6lx!AF~a3epdb=^MIrp8Ec8 zQ~wWmQy{!A&t8&(eE(M*?nutbSI$-vCv^YexNa7+g{biynbVY8)|2^crVE zO>afvSuN@iciJB;!)HsZ-lO_a;xfXf%biv{l06n>O<~MuwvGV>J;fyL&Vq<=Ma_NH z8a)RpB<&5n?2j`pfnR5~C?)>G8n=VxH$))pZfV`_>G$H^tqk9J$)4?|T^8j%N_g9V zc2yGj$qv=$4jp>PB<&{X3CoFski%VnqJwj5=L%UTw&m zjDkhLto0psRbi*I3Y~}*mN30Q#P76@WZ9h#L!S4m(?&i zi08634`(snJ&V!VPr^^QJ4YTntpj9>tL<0FiW$gg`mp72v|X$3;7SY94d_l z2PW!O6)iR@G6Q~$0dj=E7ny1cll<1(WT%9!&)0GFSyoCaxUv~Hq*WZ+cr3+Rww)JueNCEw zF`eJiOz}2T0Zy)ds`zltu?o&DoEtd1*0oH4MWi6P12FD=8`GPqleL6*jN3vW=H#O< z7yb!O#qmu$p@xeA{L z=%X*;?(G58V1&`w1N*bc*xP0a%shz+1eGOK9l=slsO0z)_(h3xcs>jGtzT1s^3aAK z`hudy7EsdFN)UCFF95QguA9004R%MJNV#EdP(O~*-xuT%(Xw^8zt>0 zj${r=g5;)SiCxg774!BLF=CIto`v>?WwU2^sP5V2Mx9OboWXV-U0$ifZmF}%!al;r z_W?Z2U{h!}qzbh)h4osMyt+!B0yS>k^=g$q%|XmJ5QuN%z$DD{im03V#tP5N+D$%r zARKC5U*b5a%D5L+NQbnTqUyzHQGz0Ve_|4iS~ay=w~o-7u^Q5d-oxup@IDKxzg82G zf3?s2T7Iz-YQNvB4NgDt{)XoCh7FY-Y%5${x|VL$8eZ8#*%@-?KPsSGsdd4e2(hkc zD7^#@-R)^EBD9D!XvB;9>Q{^ouvOZ0g@m+O-%2nGLs*ecEI+VBescqX2#w(u{Iygu zMq%9mQ&OIglJI(eI*T;#Bae_{;%tKBD;VDMcMicCNdJtJopQ~}V)f_&I?c*4BU^hi zFNAZyAwEARzMQkdsU6O*7_SdOxZhA2Qp($@6zrMDmNnUHp8gHYr-I};BYDc+A|%6~ zYKb>BwplZYAJLN1rZ&jOOzlM(Q+vWPHf3M`En=t|(M59RUqT%^-TzC((0?>>g$$kl zlj|3=cd|A7KXm_;CY*=%lJEB?-I#hTtv~@82l9SU+-U47*&nn+WQnXGPR{883StN< zj=Sgrk(4MhXdzGZA9TSf;`*IZ&^7?=4)K%HL{@~_YFg`-dbD-(`C}AEiUuewBSZpdTDx zlz-fBogX=i_V|L28i^Wiy4PR)aiguk1g6!rX+&G)X+HK*TizZll*+a<0zBF@ea`lt zrGL2?#pSA(k0z-=Bgf&&T76o?bI&Jx^-RXay=9eQPAkQ(Wnla1{B)L3pPoOo5m+Z^ zcV^=f?>a6^5BW2h3VG&O?FD!k_teuAxizVyFchFq5qq>yiZv6h5KScavrJ~M$*LJs z5@1KI03^bFIOXXSid_bcIW?IDtJ6P8iTJj>7lGSb*DEDTL$DZy=j~0tX>4lc$RK2- zPq(Ja!C7@Yr$bLRx29uTGzb!a>D$}cM_-OXjj{Cb zR;#6GU`RJ#N(?cNLvtqv3Y$qqb5D29BC-=Duk)bCnvOb~NX`7*z&S65$7#|enjYM@ z-KuL;N@9EwfHKkfbYlEJQf>c`Bbw zVJZXz)1^>0@^z1@W3^$bQ|X6sbbX|0xJ-(enWTqQLB(C9=$9uwI$3azb$+UyD-RFR#PvtpPbYT!-&Ynn;H zq$I}^c}G4`!8~)tNOwOS`5N*Pncv|0+gXSF6c3n(G0-s`tyT*r1}=$Ai9A=(DaC-n znqgvv1q!#3I4aiNs>#Vzq7l<*kQ|((L<~A^@>H`v{$)wCR}IyTQ@3;~v$7_u95c?)&L$a(_OLV>X!@8L_<|X(Yig;u>rh+A_X*RrWCYo6GWt*~A<8wZnZi$u4 zdC<#i=M!K=1W;ZZDFiNrY2-!eXB#&jfx7E^O;o6c!9GZ|NI)g$sLTAr2LX z3ofOJWsYuy*Q8)ED4*scwr6PlM%oVg7XGFzY5c(Z7Xv4N2SZgc-Hm<1vCS73MWCu# zI`S{rK+iG>w_UBq#HY#6oY8N0+q*)Toe}1PQ5oa9_@M&lAyXbv(+k(L?nrAj4N~^1)~%-(O3}lsjU3Z& zg%ADn!*a&Ih#y2lm2U$xLjhpE5m4{%=&nTAVGW>6IMdYAqf0@Y4N~;uyNdcg@AE0J zGAVY=N2l;P!w_thI59&&{5A)nJBB?QT)n?2N)EM~%9{+t?l0c|Bynkx3`v^q{=;%P z_DUVx$`9N-ou*;q4sDEmHOfV80%&vcr6Z@c+yk=w442b-#cE{MWFo8#m-Bi_FPAld zw#ypL2J7ikk$iQb(!}Yr#*FI+fA<)!LcaO5U|Cc3lqc0t;$<84hFOI&^?fdL7Cp}) z%@bt%MN-8Y{z<)b_1uwGTMUAQXPHE5?C4{ATD|s2%S~vzNU$M)!K@(adzD`%E>4}x z8Yy_j#S)JAyUQAbI?Q_U+Riy@s-P|^te{eeOPiq5g{g}qT=*tP`?b%8&)RY;+s;uF zMF_Mu&`%-NkSq7#&nA_Ij@hLu>Sp1DV4G%Q?3q*KX8DAzQ%B5J*x3uN7(|tF0SzDx z_F%-NP57w}hd_B9ilr;nQ#X|-+0SY?G?m_cj9B+7#gE;o-ABhxE#?(Q_bOhVxg*KJ zqQ6-BEq~-OUadWLfL2t*lS^@(t|T5Pn8;oej;QabFBDQBJb-; z6cIc{uR3^l%AwCmkUw|o;mw^0yhHwUUKs@BkD0r?R{F?+^i@itCQ#(89KqeyQpBFz zmGTx(9Jq+Jk7Buc3b8X@HYo%n(pEi5<~Iop>sou%XFa*jaO!gjbK_wNCaNzdpsn)#kR;8D`RlF($LuuuC~iPtBeF*v`YWz<1&t;A5WG-K&sovI;6mo7Dz^= zSJao*RRn5Blvy}AFs9dXwI^IOWHAlWWiRg6Wmc8N+i!QxM3f_XZKD_G4XoOl$A1m% zDMdSsWpe0)D$8mXrzLwgD0D<5C#@%|NonZ_#mw7E!VooFwok32m6RTeol0)elU3q% zpq@shhb5HI_M_RwkJ}DOSvb29H+OfH9Y)9eIb@Jvvl}zjcg4e@cZ4~NKI(EeoAc=_ zif&$!d0w5{hvle`b)>wA#o0^156)s&zU}5u#-9||dJeU5r)=wki<5Eh#-sg9ZaUb4-OrW9BGt;S znl*GazMi19wo_PP4*x~WwPpS>pm$iJ6JH;*)lN7~OWcrBEfgYBdBd;eYyR=-_MRUxx@h|c zN4l`47BPl)fh#jv>VV!#vo?S5ju|CLs;+CRnixKG{6vqXDFc0TuXBKuA3alZHB8-TK3HYBJs1 zHUT5J658%d?K`8GpF^W1bQbv6&)WKmeUSa^FhE9#*&Q@N@^7_!O45>S%$ClH8n+dq zCgLg|k-8Dj;#xi!j|0XI&#S6`1~c#g3or&qJCfPoy0TpBcOI3T*$1j#qMz%$Ppk!dGlwhWxu9$2#oY+^w${K_N{h z1_V|PZPDJv4Md-{FUtzjS>S~&srq`OmWvi)hD1G^L23h`pJC=@HI#Nu9p#n~TDp*y zUgEoWiGB7ncEZTMkd5VxVGstf3D_jdOczg}nHm^R()IPP8PxILq7X4r%G0Wl622lC z&GBz>bxnLBi*xUcBhU^7ft!zf`)Ps)E*wOm^s9?b@O{|>r7^AYxn3oE{1F*g|5(3c z#YNFgX)y(&TkPyq12`CAWBOnB2(tY+cE8NZ$3W3OGEeRgBIU3u>G3L#Ps@7;nCm!Itu}dqDBXZ~s+0}6B zEzAebRo{6lmOy{W4LjlB%nkDPz`_17no=ZX5sv(rsZY%a%Y5)avwl{HpqAgJh>V@l zyaNlrlzJx?LTm7Yi_uh{+@ZlBItfurfu4|tl-$*?=t)7i*EYB&W~^u(cdS6_5RTEE zt-;^OI|C!s%71ho*Sbp*TWCig`FGy(1IjfX~Z zRt7)D=O-f0nJyQ+mbnd2T;fKXb@-vh2K;p-@zmfQW za>0opV^0EYmZcoulqNz?`Q8dE?*Nkn?ES^XEjo08M)X_wJL|_=5HOVO1B&ZK_6tkw zm8$;+Xh?ZOpIkR12aAz(FtO(F4V(O^{bEbGc?5_c`00aI&`=c`@8UPOl`pg&@0`b3C}^&^gnB|y zD5oL?*OP;;?FsutGv#uDTsEFa&R?*UC~XMt>?pP>q&z6P__vMN*(f4<{mgdf3K@Eu zedNw5*xC5I39$GhG+1uenJLILBfC8p>{1K_`@ZzdJ@rh4OLBnZh1F&l(7$Gf8k$mXf7b zk*Qh}jFJYGf~12#L#NrkQ}8{MEu5H$@;R`9*vr#fRn18il>VOn<2rBtGYT8>}*&o@!8~>+LKxdH85A0M>7Bz z0ZaO7VspJs=D)kgvweSZ5+juXnI&Zh&!@i%YN><*49gGKdf8~H@}$b_il5t>pLClu zxofLD(_leM4sK>iXrf_m^^7U!Bbcff8V^2kctW#$ahIlZf2sRDdW$yWt>s*b{bgU$ z&A1E>C{J6_bwjVvcfieOJHRC=S4+f&NCOH_z8HTFZu$tbCf;fd?I;RQEQmj-S=J+>UdE9AC$AS?O-^3k6?asWmh)&ec&gyE|Wv+9^$-N5B=by^Tx&NW{Z9umue2oY6%ery8@L{h4r2v zFSCX9)Q+Jq+S*>Jg`I(eLnC$F0Z=_3*z6*muDU2YFg9eV>LMQ@9H@+7GcD)UPX<)j z^Etz?q{y)Jkt#TTpPq2PVyEg1rF@|tEJZH^fKA{0*?@G>F>;nP; z+b9ownX$*2r_ynrHxl5T%sJ)yz55>YW@DZfaiTvaj9jEi*`?9mG-7H}?j5TDLw^7R zxd3q~wOzS3>}QA1PbX@;KTDh>);C!$`(%*^6*geoNfWotHoe~UCpitI+>TU2~-$KQ~@*Of45x3KuvY6<*Vww5wqh zGh=G@>)o%a14)~7Vo0nwK2fVdjU0#aeWCquWlyfy+^hT@eV%On)XO8~n58ts>Vhoip8Z>u$@`TX;S+*+;=>|qLkm7<|+`wRgU-%5d=IF8W zgZAUhuCrFnt%9YLPR+v~GS5pOTHDg{Xgq>sf~?h?)cyk-XBc5Ro+fErI1>F~I_zyy zpA-$Vz9k0UYRg#-cbIj0jkY5T;C<6$7{012%L&COUgeEDPYR%fS~N z-*IXk*jI{&gePWrc(UixxJNN9n}Y)ZIUd)&AN)W&p>ZRVAS*vxU! zl1SCVgsAlx^T#K^I>bwsEgO zYp(S{m2g^rfukwKfhH$^l^6ELF8kDn&bHHwhehXc$R`yde8?Gy=}y;ZDi3;xtDU6a z-aK0`sxG-qJ&d^S?k_KfeT`3Dpl*0Ey)wlR2o}!?zgGF2i7M=L%DWAx?v;Gi=@ded zweY}l!;UehCfz0HX_Vw~gXXZ5R&kspAX3etq3mI+HOHm1Q2%EME#9t*)evjvpqkvb z8s>Y@UPdPdWcH#({-d+oI$lM$T{D+2+Q&oPL)7CSJik~w4G^g71`K7s`y$}&ykEz_ z_Kb`eGV~*GyX|Q3|7`>P6*cuo#Xqhq`hd9GvM`{L`U!X6TpuuAw`NY5hx>~Tb{E`7 z0@v)ACm^opHNPgia}^hN|J(vV8}!`0fQcOo zT?_@L++gv*h95Sxg1o=b7-NQNiaQ z$F`mjSf{?qf=z^ML~z|Hxf!ldGgMfr+!Bz*P(p*$J>@eJ<_FBvLDj7KYo1DETebc@zG4B|nZrq#*cOTt6D}C;}41AzAMN{-q;=u$P6d{@%W0)drZa7 z?2`TI<}*16DF%p`G>BMw7Mv-F9mXGI`AWcsv|w8-wD=M3DFVOxYvcVVbAfvi$7FpA z;~d@>^nbnW6P54{gd0`ScZbp24iY&d@|@19}QG848F@j#~pCHwiDH^tfnb=O)@(A^Alo*4riF0 z_~{#j>`ptKW2+$Ab*d06vZuNp_wl)|)laj9{dBv#cA~U3C&jvG%MY)nh3?7;ms4#A z-*yw5J<`)%3iG-aESOxAkGCpkt|0|Tfa`szIRFS33I29e=a6flEJBFMON}fok2Anf zFY%W59Ea>mn~CX&QJqv~*-P`f`eTaegnlLk{UplHEDB?j4?raVoR~^gl{b*Kh?qCr zk5?-*>1-(0=E?OlQNk7PnL!^zxei(y;&-wjOE-%cDc@2umlY1f0VF_h z;ZQSO0jcrTmVh0Np{k{trIcq_?g6qN5`UJ_SdWgnh{@Q`mLMQVFPUKZd&q__BaG9a z0eNk_xj~oxcsW-b>1f__la{OgK-{-lSz&=@>(PBegx|x$xKD<03Ct(7u&wEn$+E9I(Z$xHzW4D%P?_yCYOb)A4IC?tjT}KgqO&{5}xC zrQaj)u%c>B+Cw*K;K6Pu<`_U5<{H&L~g(W%3H1d(n z={{1e#JMXqXdpZPEn>BM0B;9*sr;>UVEdG~%(4i$)o$y*4 zf_5T#5TNeVpwlx@AP^TsGMc&a{$dHO9&8P81|P?ub`z+9sqO_2GXm-GDDf0}p@05~ z0RI97n-t}Ai4Mi=yE2!s=0*@N^X)2fd>IbnFHYr&Y>VCRq+|viBe8)azw_?|iGW3P zqER=F^2Cb-b#&B?JZB;NQ^Y9~{uOF%uNoGsI|JLuM-mNpV+u#mw{bN-H8uEH*%e`I z=kklEBjBH*;GbT{b7n`U3dcSbP=Cb9$lSw6?ekUpiuUA@?d{Jy!bKo#Qwz=T23IP> z8aL=c>77SRA87%@l@D{GJd$5m=7vGjy4@|{dVo+>Olg!F5fG&Xj0y2nFGG08~*=A+EsvMv3z~HySqE3ySuwv8tE311`(w} z1OZV>8l+P~q+0|eq*J6MCB8-PRq-O?|J%pcSA5v>o6~2`&d&61Et95shWQnk=OOP{ zKTHz9X~hx2Y2s~mQIF_7H0JM}Q*q~td~m$TA7L6~$&t0my+QVwGjxt1k!GE_1%2Yd zj0~}DU(5Z*78x%T<07f95d_BE*|@N5b^ab98oJa&=687qf%G zPU~poH|~9k7SuE|eY*UR&MSBHwbE>hl@QArCyQmank~uugxMOWR9r7^(nhH|Lz8wb z+(i!8blID=9rw@TSSu-{fio+C%?gZ3+M;HYsgOhL^ZoD3G&>>(Vs=Qa1w@h)M;S(b zBoHb|tZ8ynp z(=-ZTi5hU=oQ1We8?Z~pKRaY-9eUscTlS?gSX11N84a78#z3cL5vd^Pl4%*_#zBik z!Q~6#=H{NnkwL!wu|!fj-55O$gN89E)7Dz2wz%At)cQWb9M%#3(k8VNcVoTMwsu(7 zeShM`o;&-K9tj&v4vRZQf~OFTMBaG*A@UGT(un&K{SdfRvXAarBeK;A+&b82Mxq)IRazlzTL(h%?nNscjA%btL`lrr8h`;qQZO zo7?eMtP>w|vR0%hb>kiebjJ;{vfRvuf_yhdNp0jDvOaOwqJpN_#PtogIfZP5HRYR! z{7ymC-L}1svXHcndMBSwI|o^v$6g1i-EbLeqR%BDi*7ox=yeT&s4>-=-$=yRL>|ZO zs7U0wUB;de6Feu`Nha(1RuTVQhRTiG29!rB*0KCLQrYzO(JPRk0*PH6&Bf*z4`M1D za)y#Q30hv3a@4fRJ~EA}HnBl8^med|b`597e<+{zNN-xY zn;a_-V_^>u=AnCIv3z>_G^aF%Rr#&AxoqOMP{>h|;^Bwa+i6?qI#!#XR_wrot?&Bl zwHHA+EUW5#nM;7I#Da|1vqJ>)N+jT6fBNWlTbYXlA~RgFA-LfO#qtD0W;`*1ZduD9 zxzAq6Y+mnFE6^jNQ!~-ji&dzf^SYr2tqNs<(Yh%5HzE<_)^QDMIaHFx-gr8;KAXiE z8^M4R(PN#RV>TVIVpaUvxTB1fj;B0PwgT?s=NqYra`_acO)P2Tz2G+N~F z46|S3-o)oSlET!BAQIn4*?! zr#$v9QR2WB(vGPQ)Uctr>G%&oU5x&a@)PfnYL@e)|ke1wz;fZYyDkCBt zMWhlF^_V!6g+HrE8h}!W4oB}qL`dkrEeVQrW0(^H)6*|J<%xjeC_J@cRjXJ&U>3|7 z;z>=4wzh1w7*eEYyVbvDD5tB|XUjn5|GKgpll8$-3uTNcOHsqLj;?^=el#@oeyE!p z>hoR^O_&Mw@kexx2%M^3crtIL!Ul02>d4>UO!ZL6>$-dIju8+CVmYv-&_zT$rcEAa zBYl+lvwtrxfkq$oJbJCiVj()%DPduR*x zM_ro(chok>yz9HHl5~U4z`*La;9*j=qzbcH;DksbT9_kD0;}-tZlZCa)Ym)XaNQ(M zv&fGlS*$I{y;+s}{5I};j2zhTH+|Isk|FseOi7;?8h(4qkDM5}}g~Kq7Gc*ZG#bbnj1C=VK$2(hxp?VMzpLCm%Aq-m~6biA- z?k)-X2Ja>zouO5*0`Ic0)_B&3kBsj-!Hhj6WLL&G^y`|ui#pP9whQg#>`Be0pD?3? zTY`RffGEkHDSf|AH2&^Jws?Ri9O)Lj#1X>XWBv!ITLe_CJJR>DRnVAC4C@}U&0-$a z@UBVvpy^@afzrhWK29UsLN^avrpBwQG!% zZSV%Y?eL}=qR`Oh4zr_@9L#Bl)P3{v*$Qk?kkv3IK$5sb6CQOb8x=A z_^2=;4ml*_yVn*tWU2lBKyb5@qe@IPyh4%&7*J0wj7u_qmQ7T zHDsG=ZQ~J-`l*T`jK1dNY=%yFXjhEhXlxrm7|H{8%Q|}GQMssYW_Upj?6q2HEyQnRBu=tT)n4tnTq?7`t+G39N!$hUF$^z6@D1_gZ!Yrj)OYw!vN*u=bqj&wkPJeMyMM_VQnL3-UfBA zNavo+o+Lk9YW)I<%N({}NueLRi6&Ma&53U9(*F3#kb;_A7EZ3D#oO$+#_!*nHqF^Q zPiW#Ql$y&;RJ1`r478EO@8ZH^oz2Q^E2ju#CCJ`amRHDR-*)5qFml@6_ZTk{s;wu3 zC~_|Xr^8BF;l+$KBR8R@0*8fopoMn|qbENh+ith}y@&J3Hk<_&z79eXYaH3T z>O+blz0P;x=T5xzxxV0%F`DZw;l4m^k>-9&{r2rg5jb}ylQ73}dJBg&TB!Xf)B*8c z+T(%G6LF7{m9uYB+|ImM@F zKGFpW1}117sSG?u9z*CzLpfAkMe(>wbfvAU`SfKJeshw%xbKiwZo=g24N&ljBhxI4 zCj{8nbr<_o(^?gG+eDMw*d!epG`W|lmC_%e8merX9#dqmO|Kg3MoNg6aw1c53{%LM zW<}z!cXJxD#)M;~h=#*0Zsi6%owY>sDV*jw7HwkQqKAC-W({phc@w{`V^jExVOkxN ziAWRTgnDskCB=m5rrvP`uXdB&v0)SOrqpV%sI99HVou59#yD~}!AF%_)R4Y!j4+Nd zebpSnBH3)a*--7`9Cj$EFq2z(LKJv<7DBqUG+0(PEi26%RJoNpK8<*1hKA;o_6@-i zA`T%$3Wkcv#uhyUc5TpJ%tGMn#D{h5)Gi+mpWb{d@G=Li3zL|t)r>XeMtP{%GxvqM zxXu0AX?kOdz&wfmv7(R`UAp17<+1kocQc6*CHmTuOYLX!3p0neAB7pLCc;MG47}i2 zueTw4)=JJUtgBgLWkA}=nuuUN)Q&1##q{#7+l;%b=-QSXl-S*pdYHR+)!80kzF~N7 zwAaXdW)a(E6xseke5I9U2F&aOlAi2vreONV{Iu5KZKVd*4NGjT56}9i4I}nD>*xiX z{P|fwx@q3EwULA$I;;{MtWgy=5Xdti9BG$QDCA&VqFC)2?-`8VWw^CM(yit(;gey_ z!C|vYwiQ>1^r>o>{MF9-x`E1!hhFr-gIulHBjZy_-ceQi#N+wbRPrMR-4816xXA3l zJz3f4=R6{hPuEO7NgFUZd9P!@_zC_qV{AUY0L3_Dao}dN(xB!03M1%SwV+e(@P!@t zjz;&gPpInB2}wN~@7}3wD<9X}DA=!iI0WiZUrA3f8RQDjnsa1x0{;R=`m?s*%TU<} zKkH(OBM&l+ahtiZmyV2v zw;Z(I1~L(sf;ry4@t5Jnsy3%+Bk zsj|g!%U~7O`RHh-isQjBOR5;g@f-vGz|WJN!FcX?u-SQv2u=Gy*LKEjd2SbHX6;7D z$Q;23${|?cD0ep%$z?VQ+yGfp(jM-l~J@5J0ZXbMRSN4;g_Y;SAidvs#Qa z#&&4J`4ACXHyJRhv{oxEC;T}r(Tbd;Mnz14k)%dEi1YP*264rsrTY*QV6~vB`^qga z>vu-UX99vA6oP*weCbA}<)UFBs2gq7S|t`wosg%WA4eLg(H+G#4eQA`-xFams^5=* zo5=|1mto{C{uW+c?CMmo;#7Ej9W=N%cF3W<_yJM(_TA8(sQ5H;3>)g60D0t(nTgbk z0r#{g71eGMXF9-#tT$8{?+2Su_c(rT#j9eyyG5LQH{o+KEZhfb#EM6Xfmvki%h|37 zD2PbFA2dX$`%q#V*ob8NJX1+>Z8Zfj?^=R(VN3|A@}*$cSekjaeZskpTspJrx}Mj0 zK{pe-piF`8p*k_t!1wdsYuYBUG7|1O9>fp!R5GUbP1Zf8aPlQsI)q@_rnWteRME3I zcom<-*K$TjryOyX8?Mzei)qAb@$%S%L}*61m*v(Z?oAk7|0=5&`freU6ZBS2Xjb%_ zkx>Lt4umdt^b_*8mWP02gmHka1eN}WqUC(W{}l(<7N{BAI^Soow6-H@{ry|*yxlz7 zgQ&vR_y%1FM2xTjQASMSqZ?xg9q)R2S^4Ns8G^Bzpnd($vgi&-LOnQhGVp}JUf$U6 zO}W!s?XvOlp2Nlz_-JG9J-A1=936)lM)W5fono7`$YOH$8;*<=F#`itiGJHOPQa{ z>S?d}9P?;SEO17NsKiZr&J*pU%wxWL$CUdsCaqGKg^?~Vs;yXU&ng9PFyKVrn6Va` zSRo1v{dBXko4o~BqCaEA@R?DgXiJ=&VT-!_JE%!am$IX>JBqlRV_utB3{`g#=(UAH zPO^ehnN5%y6{m|MUz8|4wtUK>w6zE$X#`FqK;9zT2ct7@^m^ElE}$l{RsI99r}owAYK3cpqTN`kp71#QDE zhd0}he`j7O&h5SH?Aqa)=&^!(dKvWA?%K!){%R{Ogp(&)MSEXP^anXzOe>4Hhgp+w64n5)dTruBY`Wg+|$E_S}?x|-pS+c9S@h?8Acz2el@MG-IK}fd&mcf zum?6wh3W--0)Ec+r_Hz6cBqqJD`OSNbt(B+8fBo(^{CCX?hm?R7NM1w+;q)F(cFEf zZhGGc+Ym!g&zeiS9(U^wY03*nIgR^0BKaB}q77h&?;eh)94J1<6SVT|jWf-wnbBr8 zRl=ZG@GbVL#Sa?u=!8>-#af*qndkoaaQH0iq%PToJ|u+z!nURy4a;56t9_hQP1w$y zyin%zeKga1&=l+OQIs;Ux_-FRI?Qhx!3{+U2pkZh2=lGsNDJVI@ltsd$v@2S~9E*L{@O2uyo`O%rr@{F`G+DIn9b1ZL5}QReS9!+Fk(#<$XHISZq zkWC=A%a?ObUcXWII3dMg(eU0`KNWHfp2{Q4+uoj_tEgqeNSL7Ydg}y`{H?gupw$Z$ zZ{HQ`!P3I43=>J|I%pK+2QPfSrGRx7q*s+op3FDR(B=+Pw*K5kmwa<)GuDUH7ujd{vn$xZV@Cr;yJab89{#9TQfW+R_Nw z+|oTLndm?&LxKfU8Q}3&4e@*KfOl4crCNMjTAxKAEy2lEUiyilsVJ^$FR=@JG>D~N z5jAx00KDQ+EG@{Xx5``!>N&BqzJ!N-m;L=ep-(8$8@h?GiQwwu4=c_I{-E5q{O~{$t%n1tv~FSqT6?F$Kw@!ndV`@( z_1xyXZKAyp7bs1-CO6v69*0pZ@8AYqn|ySP5ecq19Q&F1b;)7f?v14S4e7cfJk4^| zwqy1BG7MTPY+lK{d>f=F=t*kU38?3ga~ZJ(H3*6$AmSB|C^7GoF231V82H*cZqqSnbTd>XTfM#mI4e@`@bH@)#O$KzS+?QQd6 z!|vHc3q!1tFECxj(i~BK??7;{mxIG?X(INpM%=^&-}LGk8Dq^$YIP`EWjAG-k$$P2 z9aF_9q5lAWD#r^-whD#KB}^-#tND@)(;1rU#k&O*Bn}UCZBg0W=u=pCAx%LG`siB= zlFY0)iQ*y_c4hdE6V9&c3)XFqHV22=BZEA1*mK&9gI09J_?uPqi@F9LBE?k3@1cu6 zRrpAw%(U>ig8bt>WpUzYlRinZfMHw6y+>qy!-pT)gCh^z_}$wZ`&3gW{6Ztw2eNKL zciLjSMpu@UkVcYS`jSE6x>|`ko84q52fnxH93I3aCj|ApXlb@cI?r*ua_hif=l*Vc z>u}rZm<5$wDM37@x=b{#G)a&j@P1M#HV99U8d%HPFvug=8R-5I^*iDE>1>m7+jW8H zb8QKYULUdWsa>La96fdPAfK|M)IzI~MLkt_`EWK9@@zZ+Gs;xp%@*t+CHF=fVm>6j zXxd(QkZ|M&zg~SC!nk+sDm1^! zUVfA@RWgD?qaa*uXM;oWBs(E@O8&#q1r2)qoTjN`TX*haGT4p+e5caDk8X;(3S&BY z!cY4LXb5eLQV*~{p}jXAqBTMVJ@tzg*Hr3?zzza;y0@0c_@a-(AvgKM&B#KSa(kvn z97jTXqDWD2ro6o7{d$@r_LQgcBun`F57djvDl6VpCz##fm$a6gS31V!XO3xG2u;`p@Poxl`8aS5i4XYVl|mNF+~|>E$0Af<9F?K% z*_zTVwTY@BtjTRT!P->_NYnk#xJt;B#x0m;lt^@?1^~bP)`Wd*xm`M!WpDN*V|{r6)Qi?g4}S)=70eSgC^WaNIv!I^1)zy}pYOv*AQ?$MpT7K$NrPm-(@H zk;X)Mo)U&bdUQ-(C0M~Y%-zSd%Wri`$!KJfcPwu5zd&W(6j_F8O&H}scbRIoT$gQv zV#T8*tWZ0Qn_`w3gL@6bc^Ca{oJgD5n%OfM^=eDE13~B=nW%-^0))b_2Gq$-G1Ur~ zo9YcEtvT`L(_Uk7UHed$`Rc^xaHufLAY(ct{hG{MQTLXgAT7D{-yknhMoa4dLg--< zYLso!d#o_;ueZWIA4o*TQPi%SwR$i?_L^7TVyDEmJ-69c@1w_dXmLqiitkQ*63>K%tHV#bFyFkKiEwyXBiB6V$$7@n;iQ|p8o>I+vlV!{xI;YP{Eu$%RqY8^MoP}bFoH0_Gh^Dq zXQ`bArPXGs5Z7B?q1i+h6njCg7Ma^klIq8W_d2(r%)7{S;?6;>pAaYp6xb{Z78{WP zu4u16t%o~#P{=;Ejl5AaM~JPHJr*vaVtw4*#>9p*7~Vsi#d~7?P~l=38JxqpA0*cu zB8$`bHenyRI@4nK49)Jc+kk6AO{p}4*3;h{@{fHVduZF9HO*y&*c%5@@%HJ`tSp~R z%I07N7O}r?fB_N%I%KQz2xOkY0YPvjDmf!@DlG8=u-1P9nL8Kee{CPEOu*dsS!n;7RSV?&)HRcD@gm zuTf#4-zTdEP@mT;FXX5Pg6K6ksT6R@KXPgCQwN)9E~tl7Fs_7ix#k*HcFHp(ZKLzwy!v~NT9KeZajO%JMZe?tgS;H`E=fQ%z@TQ~Ba_%JHK3MN({CZMR zsZKj5BGe7W=11TS{?p6u$KF0Syc9d3k<0_S@D%XB+y|-FtN>mfnVDPNloj^kY21a| zcvm`ZDNRFC;T`V>5n@^6NxBpa74#+%e>r^4rpsedyVbPdjo#dxWI{fVUT%)9RWyorrLgcrX38whzf7?xY#g_d@B~wq!s{$l z_YO^MOCvf#!tlsl-yDu%M=>qs6!9?O41X$$>To-_IIugRNOzk5F>}MKK~NFXz^(K# zQ|i|RX6!g)Z0O0oQVwOV3hwx5Je9Aoo{<)w;rX*S(X7bHQ%^pp5Lb$Q&VX<%k8_90 zfKAY}*`*=YZq@3E>~MeKCYV@DA5O5DXpV1T3|Wh&4MV+rBZ@B=lp`60MyZ;})O{aNnD(Mv@yCZ`1BB8yA%(gbk?!{S6@esN$eM2$$J*ylVU97k%pQ1QUZf;h~n zN_X5>{Dd2G^$X4Qw+Q+&Nbi*`AUWjo z;0&72GD9DH(Kr_M3QU!%%zrgFCn%kpZ3B{NUIu7kKQSjdO4&{R(`GsQK}R8ZPo-2}6frayRR+x1V-z zzZHq!BE1Wrp~LA%xe$^myKKnf;IrLQz~oQA|ccZ9!khWthaFN0yM z*PrX~p}{swbgaU<92^vRR|2l3JphnDhb)?%$W{)PgHS-Le@I;4Ub$}RM;wP z;t+C~#()wHvA|>KX7Wcz0UOvbUlwCgwuew>C@j!jUppTy_B>HTflB`%9oG3- zs`HcT(&Vdpk#@0@IAp6GQiloYbj-gDM*A@>$CHZ3F?D(@)QyCQr>?SJ9` zu?Z@73~L(u_y`m}$3eTqM)Z|e-`R4*vAe8_5IN{E?n{dTRc7?fg-Fkeg-<}JWO#DYh6plH zRi9U4pA4nIW%gGZ#pSGKCI_20!pmT;Kd5zr4neQqx0I+L|MF#uTBCNfM&qWdL57s7 zuB-eKyM>OZ8j_1fXheFz(!h(jmx($n>jPU`#?U>r8eCW;GZa)3ED>Qj#Dmq$3z#i} zv1(FM>Zq9cJaQd4na+sXc2Oi{W-A}LIUMIeZ`iqt-=C_bPGwGNJ6%@4v!U z=okvkz?>_!XC(5pk&ZKxc3&Eclrf9SBu^Uo5x9Aqv7_8UFyT#ujKkXUN%{}MIF!aO z*sI?6^04U6xk(_*cP5z+iKVK6rM?!gprzVyNB}3t>DKfi#dU1zxJ!v+)p)D>jir{+ z=bJ=N%9RRWkqgV^2M0OgWw7MXoxW@xZEqjJ5kB=6z}5etpQ=Ck{B)RwSCDwhM^Nb4 z`3_R>`#17Jcwatpt`EJSf@#q<%Sh#E($FdMCH>6Fx4^mx840Dz zNxD4eD4GrHEVSyan-Hmx$l^>Y`PRejo00Gw>uKtv!OJGr5idcW2Ya>m=S+4=7pt3SW_#01z_q+s3AjQSF-yD3q~4DAez*l{~A{5-n6;(fV>%=9N8cCtUTu16sdqwvsx zVqt`Lb@FCR2xMTQMyVf#5jB>>)GOe75L$4j#L%-eN+avQKApD=mFILL)(JwnmPqMI zIRt*|(p*c|%#O+Y3=J(E!IN;@S0^qktn&-VqJz@!8qRVEaCtVQF={YAoays5TkxQ} z_Z1%)-BzwWg`Qjf{EEjXcGGGC#0nbreL&vMHs!iZ5?D4o)l%mjR^arO=N`|Owa-)p zn}9v9ZZA;XcT`r-D|AdJ``m34rBIq;^xOxA=D%U;_T-r6S0yH)Ph z>{UFbQSdbDVO|O{>gr=oDt1fjxEr<4$kIhr;yp8R4+(92urqu6bCRNV9rXm})*L)- z>Z;bkmp`cqi#5WldW=Zj(i%mRpZrXX#`;Ky(2%=#S{=)Dw{10}*|yuRZJbkesR?6o zLo=GAYb~mBeuz`&*?~rTjz^WIT<*I7Ez-5@xgnDW8AFbB1q~x;6ck<7+E8!R^0Hh% z>rohXzA}J8rcfF-oR!T%?4n0u_w4p!EMyG5MUbqf>X_tPOYruzTRY};2#vJmUTGjP z*Jf%NH)*sF zuFgp*3HDSNb#5Y_P0-zDXzzGj%b9Pbx~hXJzuU}uTlngOUR-@S z?UGfuv{%!tn7f0Wc!}rhPixWc?W$g`fj0~V+imuGVYmb3yXZT#qq~nw)_nbtPQ6*V zSCEw+*$;L^tMlX}o#kN5&KaR!(!P7`?1wLzcgK^-LEe-i zC#UQw#fA11ZfSZ%6d+bhbqAShMU8dNYQvcD#GRW@3UH`1z$W zu=RZwg~?q=%7#Fe$vgW(*|p622LWU=8@Q`#;?c9^sva*t`zrZcI9CzDN*o79Owjtv zY!ecWwNbUm?H|No$ONL&kf~Kj1L22p?MPc|_)#19I-=)d@iQCXSQ`Ago)z+0PXg`y ztkp3WlX$tK$@mi1p^$;tlsfs9D0fa%N8~ZwM6h0AEvP-oGYMM;UNLgMwII*#3>ThO zbmtK*3?%nT4xeVTML~d9U**VzrgiHpx}9A}UhSd|892K*n|8-egGbh%i7mrN z>(yp?ZrfImf7uadKQh3YXXa}iA!@vN8sJ1rQh_TKtI9ZKH+;Icfe=F5ahshAR0!GR zb%c8J>^L&tS;-(CSgk}|a_;AyW0(B}mYo_|9r?HYPhJfKg^2~YvGR)c+ydpIs1sIp z6gnoW5`7!n1T0AyMT>khT4RtDU0y<@oup63B_$^anau|mCxGN8%co6CtpbNumW9;b zw}gcTBONUv5`Wmg5~gjE!Vgwyn8Nb%@T6iXr<`T5)q~4uK7mj1#!IsoT2D8$-wka? zN3K8c9o_7;gu}#hSn%j`y7WICqyV⪙J}(Os<^FeH|S1Qt!O~;$y(;~ zK;n8)*Etgh1y?uG38wma9mup^iwUd{A=xCMc2I*DSi3gS4gCAt9B3+K8UzV(u>s>< zo2oGPP4xWJ=|0dsChcjF^1}!)hlJ6oumjnl@gxW^)*L^oh><$2=Qm4_fmopxU{l4E4{Tx+QUpFal~lF&F0gzbPPw2|o6cX0A@M=G zVbgX_F`~8d;@Tm2#r%!7`(C%mhmZBZyRil3Q=pK9Ny1=)Gs$Wtd|~FEyKonK=L^YB z?%(NwQE17DtibYJ7ju%HNnUuR!M^3x8pKcc#0kQXY)lKUl;q{$&SH{rm()UbZ%Xo; zxKN}U{Dvb!j_D|B0eSdaY{-*{_bO�vB0{!f)Qf_AjL%>6H`>bb+SDnn;u5AU=S; zp<@RwJ#`TND4v_$6g}i5oe^Ksgm{5qrHE$Y^UA6me7tii8)s306B+T7SMH4;l~^(0 zHGOk2`cuzhT;3(5)x}tcaM76-ha`1A)F@eDaAI;~G7(4JSwPp+x{F5b!)`tz6t0R^ ziFlN7>=)DIqO=qGI$%J+Hg5&5_XefTUh#n(ZfvMhSG0VPnj3HMd>pa!a_Zag95CB> zJ*i0iWZ~DoXW!qS2i~FtCh5R@_xp8Wl5S_>Zf?)~?@al>zf1$%_Wfb|w_gU%IA!{F z#wnAPwYxdf_k&J$9lSuBLI5=P~(Gui2nZw zJ381gdD_{oYs&s$cOslDTcNtC6NxV3>ZWze7EbM8aW=&Xt6^v3MRO4xHS{>E_lEq|M z4ig?p{3QP7F1RqdlbW>0OR5-5;ks`kLdg%nPaeA}9b?srK$cn8^em1Wj0O~~j!4?{&p9`rb-nl`B*oRLTJshn9N zwUkwUZte(usYR}62BzhP*8 z>!}(QBuZwX?mR@u6OXHw^%xHI%_uVNP{n-=UHgRvF8-n+t1=hbEcU_TcGW^3JKj?^ z*7Q0<|Hwe(Ccak-l}Q!ynMY%yCGq!Z$By69JuTC1YkEptC|1@R+Ez7~B|xXv+8Z}n zjTH@f;8$F%)B;yGR$Ek1#OV8M)}oDFk`J9l!NA2M|DL;}oJE{-zTgNUd|ryQ%6xrQ zpd$SlIt}_hA8Zn zqjt1U8AZfL9EZS{XUiq9=~9Sj_h$5_4dU#-`-;%kA+G-x(nFFgp4Qa)f%lZXRxchj z5h%v=cBqzMmTu za^8EY(K8Euem*lDDa^V+(=ra-{l}y z{E~tBDc#Hi^C?Cx7MX?;;#4w`!thZ)znv`3iZSl9`^{Z9*5eGjKHQ_%?B!%;9Jkg< zxichE(6U%^n@CRQ1ZK*KNGWxPA1z`-bfNqR^{(SlfFfmd^zBFWng?GtSc^dT1wU*G zl!tzdM=jcpB-~)8O_1UDStUApHfGNxnLB-o5!k;_KNe?``rP@^;Jxm>!D-y3BbQX{ zvQX9M82D^-c0Qxv935Tr(@_OSytt!y{IOza+W|YtVJe4fXRf~NlUk#n8#m4Bw9zSp@o#pO5LTwfRo_ehgieWy$s$H-uRvIff$+H@ho)UrCTme*xL zYJoP|Fz3~iHx-9?>Z4lzdh+?N-Eu%FrE~4OlHU$4jxm?I-yUZgWwnLoyZJWgmPG(B z$FOYtF|Q=!z+$Od_7?KLK^z7oHrHI@3(qN;D-dJ-fRD`oKh`=rJ2;v zHQzuH-v1I7*o!EW3lrQsn+5~dYw48*+7ZtNb$n-TdV6}yuyUzeHrTSxcT+l^&9%~k zI>)MVd2)H3`{Sl}_47)5@c7+$3Z-{dP28b$8@o4@o)7fPnwXSkyhnM_53C+|^f^uW z{SN$z&}Wgj8F*zmP%sQ|IDichkn_&<*MI!;7wq>RP(ap2&VTrx^?uw21Hud3#&1~R zHVFJeF|eKyG}|DExEBKc|jgb>sQMxo_9`jqVq3JWsnT(8#j z`GK)NNHc+7A=j_Ge;=>4ouloQgQonK@K>pN{*khasTH8KD@hVxAX$4ANe2fn>#I2@ zxj^jTDq^M%c6JWumGh67@~dl~*UsOrbG>r{p8i|=boq8Hwbi#{sjsAZ(Sf|LqH1Mw zwGEN{JHD^#sJx1lv#Eu(?bX5~zd&vGDr$Bnj?U(0Zl+fg{1IHg$`2^{f0myq;C8kS zSBFZR3q!K{4TV3&8QKd}Qm>%$S0UzNKwbC^**``XvJ0al08WLuOiC9W_4-z;e=T}{ zjwu&o;PY!y{bM*Gxxo6;RYb25N$@UEqXafGxJ;0L4kEy`^q=e8Yz=-x=Nj?jqC-9U z4WYjX9T(kUfkWW$t|bBF^Q=%>4cG{eTOy0h@M7UO-0R{#)d4tcn8>{w9F;dv2Y> zHB`!gvepqO1*I=w3Jd)$=4ISo&S_M7sA?%7V=*8>{V2oEjYU`a52)uL;?97s|JmQ< zS9?M68o@sVNCT%bfdHoX{mSsy*13Z8*XS;9;&uC+{uRozbFQ;*SeNT- z+3roTDPRj8fX=x7MF74t{F`1~8$<$VHMrjXeY&JH^QLYF0K9Sk^GTQon((vcScN+Z{L{s~vZztK_x{Ki(AHBR zb^^1c?^lMu%fYqLeu^@e8=+)PF%vsr50SuBiQ_xQxtgTE{0ZfsisOevUk)J`{<;Yd z5Ss)5bJ5QGz(D^Xwtspv=Ma+C=C)>PU%S2(z>nsZK(%{0AOe$a-YOuueE?A60#C9) z$i7-nKLLKj(r~r5b)ir&alA71w3@H-6#c4b=MZf81X~P;guh^7b}8ggp>UD$#zmA@PJigGe|#J2?N|zqSqNMHc`~6#qvPXK8;X zF8zxpepBC+$gJ_X1Jks}z}f`q3x>**{`VlifXbQwCWD%h-B%X}lp_itUsMj}i@!%@ zzg#&@62na&fZBH#$SSHASS!r^dyud8{IvlN2v)(|)kFProQ0n;z03ZZNXKWXkGzYx+k03yx0m^!y!DH|0ayh`q zTQicHfSS4h1Gxww-f%qt2~SgV$MZVlasa_C)h=E@O)@|{V7b7-TJ!Y)u58=DdguxY z16p7J5{cLa@~-VykpJb;mbjap-PaDch?}d`)!na}ZY9JWAm|JL6|d+8K+Ntd0M8$N zNt-x-n?;KOPx!4}fsT~2gNrKwXKL=^;_#cv>?CXrk1BpxSx9lze{$<(&FLJ z0Dp(bzw);{_5T9AoWBmrgoz%2KL^0xg)fLx5eUYA$=^>i^~=#JbZVp00Lyfz__b|* zd*uqWA5Rm{ONH}J-_Me~9LjVtr#}~P;AQ&ezp&auiSl0V$LR}w|>_S zt~d6Niw1%{44}k|7IW|TTEIVT8~o39{Xk;TG_iI2P1BpXBzX7)2>PeMquGnkB5t2u z3s4g9I&SxF8ys`%;iE=)O@iP3h1v2(BH)_!~i_l@2%yc{(ctfIsK{^E?H824H*{}tia|M`Y|Rh67*H%sRaDDDXmT{tgTjVSrGkk1YD>w{w%dkcrl zx$8pNq!f#Uqge$fs$*!?%kT=&SV?SilJ% zzqdfa4EkDJfS3C~4RWzmgE6`m#<`V$OQOFQnq<}?V?F?K7YI!kV}-lvwLpMEz|tA0 z1XRr(9h?CbU9OuP5ub`4KsN#al-31{yKQwXnE%F#9|`7rVe$vFZ;GO?u?7Zk@}Huv z*5+D+P&RP}`rN;Z7~r6XsZ&5ZvEBbyuX1s?7Wp4ud|i%a)IyXN0k0|t76M#+C{vbs zEy(|f^>X!z6zMjY0Kwu2D7G(J$#&(naL!HfXZN}sW)6&d$pWwvc%ak~xnL!^UDty7 z260{KfxKsA9tXlZ6(GTjVbG!HT5$h_!JirXg9)~#w9cKcP!|SD5B>`r77SerQthAB zUldbR%xuE8a6q|=YRn-PCcd^HC9vfZ&P%AHMi-z-Rkfc;6KDPtg5cX z+SS^`*4oSbYb|=YscR#kSKJ4983ybRaH{WDhJQ0CuvHK6%ELb|FnoOv;rqJ%ANla> zOyBwU(P%S?hFui^cnApVV)jUexE8?A3HNuha#;^rBn2u>E|OoT$zssg0{PdRFBcNo z#Hi3AF#hw^4Hs>=6ytj2fn`E~D#U=f$Zt9ch}xs1M}Q90fb#2NtLY5)S`Zgvz;z?f z{<`!15TF2C08lS7W<+u=)UTFxW%U4$hZ>3jbO0%U)NnDvwlK@4NQ^os?@h%sD@5KaNHHbtO$_ z_+l9}K;Qa5hTt!KE&BgSrI(%G_ABc9m@Ih>AWHLKf7=KCU&b>0KPml>s6UP4yE$I& zUUY-hvco_!q6S<{{a<9@JHx+N`+o#KH<-VO=xl)n``}o>2!LlDlov*@uX}L>rvJR% zaQ<$>xh8+R!q_A;m*pK22uKj=ud@IFr{9zO-xe2#;{*#S0oudC`*j6{gWq3ZeYMPM zuPfevIs9@8a5pZ%5iS-<H$gtYKHGuhJWw>m3BQqRaI#mOWmy%id8Y978Mg*HFXyT64q4|6cJDyLNm>Vz=cq2 zVbcVwXN0-pkeY^QB-2w1DgS`3)~j8`DBX5T63|S{v|`r9YKH`sV1I|_d-s0#oqNu? zbZ6c$I&;6@`Tm~oocp`?R0%|z(}&BiJ5qu$Cw+uz@3(GoEH797O(4)sz%`q#8?J?) zT?o0&2-gurZnHxmmzma?(|m{B}^|a8uvm^H0Nx`w@9`66^e?CvbeF>K=SUVxO+yc08GpcK4Z^!`~+do_cG?uq)vfQ%`N7%;F!!`9QZ3dUIULulku+U z5{T|GBlwHDZdCyy zOgDaDoeW(UJr;IrfWK*XG^ei$1oMw*$92W7>=RjVc^=fD@=xEcB3~n$@yGv%yt)$~ zLG~Va zKt+DH?VUyvGYqdsEqlNlada4p&{&EXqypr8ipF*+OgZIWdClxog?#A1k3y8(qXMDV z#pD71_xFAAh7CK8lkm1OZ7Ry{Re`X~?=2bj@pla8v z)`g6)MnuN5SRq9d^kbuc9y&kT$Gm?2y$VF*I%E^w7Q7Lvf~RGZI2c=Adtu5x=yMIf zP@>mQ;TCaB0%e;$zIZ?C4*DL$#-rB*DhOKL5t3XxDuhKF3L&p$4L-3Swdtux6*V`~CY86uJzoiNst-lhm+u z+)gQBwyv@8?cG4@g88NrwBEC=qbYJ_-yPf5uoP{<@6mJ8ay|eQ8vVuzX>)Ww)0vYW zbY0ipnXndX?p0V-P>6;{)F56<+Q=jA_}WQ8rRiJtU&OkXFQy@Yy!&6LO6#y zX6SM&(%D{lPWO!9R7l8+O}{uNb}8b#1<^pO^aoF?f$AJ1pv1m;a!dPeY!{JHuZNQL3ndvo^f$~Te*^8eq-7X86@yWNqcIjb8lCNuhKJ z3w6(@V^S}|D%cP*SmpE2g>nsCP%~#1*ZZX)2fYelnaL!qz0X;aZ@@F79kcek`d7tBZ%wS%Xhy(0@x-Yvepj|3kLNT3k zwCl>taLFF@oT$w1%WkXfkjfS=;4=M#N5sS8VR$k}8z}oNeYi7z&xxfM=N;MFg`llR zze#Cr@qPJNJ?4~7tUk4QG3LrrK+z{!JAC+1e14~MFVi^%@yk~ImiUFKFyM5A6*XY? zKr2XkVt`qx**Kyora!c52a}X)73kknEWTE2?D>K0-I8EvMHe|mfmV=oOJF`wLDcrr zNMvImx*DodG`6*Q9jE)7Hq#!<$YE^erGL&`@hGEbi)qbsLk1>pMP2D1;QSn;D~as3 z#s~@0O(Nx#^z8IS-AEQ-1A_vmHKSUWB7JFmoS7tnHC50oKIZ>W6`6oUzA@CeF@0|g zA4}s1u%1PuNjYh%e3x||pY`|-6uPr8W~|GK_-nWQBUuGSpHr$7+^b*b-(CzWwt86c zICe^TVWMfKd&sH8zKvLSD0w92z77Nm-QKT$NQR|OybYUqVs!z0H7IkVzQ@i9sX1eOm-eyq%_9t`_*8CY?#XW3Hv&s#p{^==vM#^9+rtWb-r22err=_ zMHS3dh=N16MSUGIFf}^u;5M%f$Dz*sKuIP6=66Jf;8_N=DZ)({k$7v~=JdJ+z&_^D zyC&7h@WmwBf1df@&Rn4_?NoXef zE~hU{M(f%W0Wu`QlT)gqWgjkjCLVg>6$ry-bJY+Tnu}K`rO9??Hh$8KnzI+~r?bNB zAQ^~7`l|Qc{lf4fg4bpw4&D3T94>=$`bolET|cWdF$NhqILHVPHC_gwL|Q%R822ak z(o_>-fG&sYVr2j_zN*jN6%!fXV->7Jw+Z&d$zAfsg7C$wgS*zX1*8A#KiIgW&q|VExFVLHC72QywuvA zuWor2ky_(pOo#FLDj?Ez?uPnf{j;(65d}@4i2JhJLRP5&2-v8$nqH-SAUegv-{;rJ zM0-q3DnwM@!ZTMAu=w)@KW@5P6Bb}G>#YOohTsl3eqg^+`9t$(DDrn9?UM-Z^)1$M zEznQdfd|#qUllIKzRLnM95A%|vfIXQ>#H4jFayW-CeDu0b9Z?M{km?q+v<1g?oRAJ z_Sbl`@$_Bo=9QJYoAqb$Y9bJF7`^Kj_7_Uryx2FESG=ne;X2Vs`!2ij<`CDt-Jd<* z<#{vei`fZB=2EpSCN`uN3sTuD4<08Txq$d*9=SBI+hQFax| Date: Mon, 13 Apr 2015 20:43:24 -0700 Subject: [PATCH 082/128] [Minor][SparkR] Minor refactor and removes redundancy related to cleanClosure. 1. Only use `cleanClosure` in creation of RRDDs. Normally, user and developer do not need to call `cleanClosure` in their function definition. 2. Removes redundant code (e.g. unnecessary wrapper functions) related to `cleanClosure`. Author: hlin09 Closes #5495 from hlin09/cleanClosureFix and squashes the following commits: 74ec303 [hlin09] Minor refactor and removes redundancy. --- R/pkg/R/RDD.R | 16 ++++------------ R/pkg/R/pairRDD.R | 4 ---- 2 files changed, 4 insertions(+), 16 deletions(-) diff --git a/R/pkg/R/RDD.R b/R/pkg/R/RDD.R index d6a75007a6ad7..820027ef67e3b 100644 --- a/R/pkg/R/RDD.R +++ b/R/pkg/R/RDD.R @@ -85,7 +85,7 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) if (!inherits(prev, "PipelinedRDD") || !isPipelinable(prev)) { # This transformation is the first in its stage: - .Object@func <- func + .Object@func <- cleanClosure(func) .Object@prev_jrdd <- getJRDD(prev) .Object@env$prev_serializedMode <- prev@env$serializedMode # NOTE: We use prev_serializedMode to track the serialization mode of prev_JRDD @@ -94,7 +94,7 @@ setMethod("initialize", "PipelinedRDD", function(.Object, prev, func, jrdd_val) pipelinedFunc <- function(split, iterator) { func(split, prev@func(split, iterator)) } - .Object@func <- pipelinedFunc + .Object@func <- cleanClosure(pipelinedFunc) .Object@prev_jrdd <- prev@prev_jrdd # maintain the pipeline # Get the serialization mode of the parent RDD .Object@env$prev_serializedMode <- prev@env$prev_serializedMode @@ -144,17 +144,13 @@ setMethod("getJRDD", signature(rdd = "PipelinedRDD"), return(rdd@env$jrdd_val) } - computeFunc <- function(split, part) { - rdd@func(split, part) - } - packageNamesArr <- serialize(.sparkREnv[[".packages"]], connection = NULL) broadcastArr <- lapply(ls(.broadcastNames), function(name) { get(name, .broadcastNames) }) - serializedFuncArr <- serialize(computeFunc, connection = NULL) + serializedFuncArr <- serialize(rdd@func, connection = NULL) prev_jrdd <- rdd@prev_jrdd @@ -551,11 +547,7 @@ setMethod("mapPartitions", setMethod("lapplyPartitionsWithIndex", signature(X = "RDD", FUN = "function"), function(X, FUN) { - FUN <- cleanClosure(FUN) - closureCapturingFunc <- function(split, part) { - FUN(split, part) - } - PipelinedRDD(X, closureCapturingFunc) + PipelinedRDD(X, FUN) }) #' @rdname lapplyPartitionsWithIndex diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index c2396c32a7548..739d399f0820f 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -694,10 +694,6 @@ setMethod("cogroup", for (i in 1:rddsLen) { rdds[[i]] <- lapply(rdds[[i]], function(x) { list(x[[1]], list(i, x[[2]])) }) - # TODO(hao): As issue [SparkR-142] mentions, the right value of i - # will not be captured into UDF if getJRDD is not invoked. - # It should be resolved together with that issue. - getJRDD(rdds[[i]]) # Capture the closure. } union.rdd <- Reduce(unionRDD, rdds) group.func <- function(vlist) { From 971b95b0c9002bd541bcbe0da54a9967ba22588f Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 13 Apr 2015 21:18:05 -0700 Subject: [PATCH 083/128] [SPARK-5957][ML] better handling of parameters The design doc was posted on the JIRA page. Python changes will be in a follow-up PR. jkbradley 1. Use codegen for shared params. 1. Move shared params to package `ml.param.shared`. 1. Set default values in `Params` instead of in `Param`. 1. Add a few methods to `Params` and `ParamMap`. 1. Move schema handling to `SchemaUtils` from `Params`. - [x] check visibility of the methods added Author: Xiangrui Meng Closes #5431 from mengxr/SPARK-5957 and squashes the following commits: d19236d [Xiangrui Meng] fix test 26ae2d7 [Xiangrui Meng] re-gen code and mark clear protected 38b78c7 [Xiangrui Meng] update Param.toString and remove Params.explain() 409e2d5 [Xiangrui Meng] address comments 2d637bd [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5957 eec2264 [Xiangrui Meng] make get* public in Params 4090d95 [Xiangrui Meng] Merge remote-tracking branch 'apache/master' into SPARK-5957 4fee9e7 [Xiangrui Meng] re-gen shared params 2737c2d [Xiangrui Meng] rename SharedParamCodeGen to SharedParamsCodeGen e938f81 [Xiangrui Meng] update code to set default parameter values 28ed322 [Xiangrui Meng] merge master 55be1f3 [Xiangrui Meng] merge master d63b5cc [Xiangrui Meng] fix examples 29b004c [Xiangrui Meng] update ParamsSuite 94fd98e [Xiangrui Meng] fix explain params 48d0e84 [Xiangrui Meng] add remove and update explainParams 4ac6348 [Xiangrui Meng] move schema utils to SchemaUtils add a few methods to Params 0d9594e [Xiangrui Meng] add getOrElse to ParamMap eeeffe8 [Xiangrui Meng] map ++ paramMap => extractValues 0d3fc5b [Xiangrui Meng] setDefault after param a9dbf59 [Xiangrui Meng] minor updates d9302b8 [Xiangrui Meng] generate default values 1c72579 [Xiangrui Meng] pass test compile abb7a3b [Xiangrui Meng] update default values handling dcab97a [Xiangrui Meng] add codegen for shared params --- .../examples/ml/JavaDeveloperApiExample.java | 4 +- .../examples/ml/DeveloperApiExample.scala | 6 +- .../scala/org/apache/spark/ml/Estimator.scala | 2 +- .../scala/org/apache/spark/ml/Pipeline.scala | 10 +- .../org/apache/spark/ml/Transformer.scala | 5 +- .../spark/ml/classification/Classifier.scala | 17 +- .../classification/LogisticRegression.scala | 18 +- .../ProbabilisticClassifier.scala | 11 +- .../BinaryClassificationEvaluator.scala | 15 +- .../apache/spark/ml/feature/HashingTF.scala | 6 +- .../apache/spark/ml/feature/Normalizer.scala | 7 +- .../spark/ml/feature/StandardScaler.scala | 9 +- .../spark/ml/feature/StringIndexer.scala | 10 +- .../apache/spark/ml/feature/Tokenizer.scala | 16 +- .../spark/ml/feature/VectorAssembler.scala | 7 +- .../spark/ml/feature/VectorIndexer.scala | 25 +- .../spark/ml/impl/estimator/Predictor.scala | 16 +- .../org/apache/spark/ml/param/params.scala | 236 ++++++++++------ .../ml/param/shared/SharedParamsCodeGen.scala | 169 ++++++++++++ .../spark/ml/param/shared/sharedParams.scala | 259 ++++++++++++++++++ .../apache/spark/ml/param/sharedParams.scala | 173 ------------ .../apache/spark/ml/recommendation/ALS.scala | 49 ++-- .../ml/regression/LinearRegression.scala | 8 +- .../spark/ml/tuning/CrossValidator.scala | 18 +- .../apache/spark/ml/util/SchemaUtils.scala | 61 +++++ .../apache/spark/ml/param/ParamsSuite.scala | 47 +++- .../apache/spark/ml/param/TestParams.scala | 12 +- 27 files changed, 820 insertions(+), 396 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala create mode 100644 mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java index 19d0eb216848e..eaf00d09f550d 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaDeveloperApiExample.java @@ -116,7 +116,7 @@ class MyJavaLogisticRegression */ IntParam maxIter = new IntParam(this, "maxIter", "max number of iterations"); - int getMaxIter() { return (Integer) get(maxIter); } + int getMaxIter() { return (Integer) getOrDefault(maxIter); } public MyJavaLogisticRegression() { setMaxIter(100); @@ -211,7 +211,7 @@ public Vector predictRaw(Vector features) { public MyJavaLogisticRegressionModel copy() { MyJavaLogisticRegressionModel m = new MyJavaLogisticRegressionModel(parent_, fittingParamMap_, weights_); - Params$.MODULE$.inheritValues(this.paramMap(), this, m); + Params$.MODULE$.inheritValues(this.extractParamMap(), this, m); return m; } } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala index df26798e41b7b..2245fa429fda3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/DeveloperApiExample.scala @@ -99,7 +99,7 @@ private trait MyLogisticRegressionParams extends ClassifierParams { * class since the maxIter parameter is only used during training (not in the Model). */ val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") - def getMaxIter: Int = get(maxIter) + def getMaxIter: Int = getOrDefault(maxIter) } /** @@ -174,11 +174,11 @@ private class MyLogisticRegressionModel( * Create a copy of the model. * The copy is shallow, except for the embedded paramMap, which gets a deep copy. * - * This is used for the defaul implementation of [[transform()]]. + * This is used for the default implementation of [[transform()]]. */ override protected def copy(): MyLogisticRegressionModel = { val m = new MyLogisticRegressionModel(parent, fittingParamMap, weights) - Params.inheritValues(this.paramMap, this, m) + Params.inheritValues(extractParamMap(), this, m) m } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala index eff7ef925dfbd..d6b3503ebdd9a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Estimator.scala @@ -40,7 +40,7 @@ abstract class Estimator[M <: Model[M]] extends PipelineStage with Params { */ @varargs def fit(dataset: DataFrame, paramPairs: ParamPair[_]*): M = { - val map = new ParamMap().put(paramPairs: _*) + val map = ParamMap(paramPairs: _*) fit(dataset, map) } 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 a455341a1f723..8eddf79cdfe28 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -84,7 +84,7 @@ class Pipeline extends Estimator[PipelineModel] { /** param for pipeline stages */ val stages: Param[Array[PipelineStage]] = new Param(this, "stages", "stages of the pipeline") def setStages(value: Array[PipelineStage]): this.type = { set(stages, value); this } - def getStages: Array[PipelineStage] = get(stages) + def getStages: Array[PipelineStage] = getOrDefault(stages) /** * Fits the pipeline to the input dataset with additional parameters. If a stage is an @@ -101,7 +101,7 @@ class Pipeline extends Estimator[PipelineModel] { */ override def fit(dataset: DataFrame, paramMap: ParamMap): PipelineModel = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val theStages = map(stages) // Search for the last estimator. var indexOfLastEstimator = -1 @@ -138,7 +138,7 @@ class Pipeline extends Estimator[PipelineModel] { } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val theStages = map(stages) require(theStages.toSet.size == theStages.size, "Cannot have duplicate components in a pipeline.") @@ -177,14 +177,14 @@ class PipelineModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap - val map = (fittingParamMap ++ this.paramMap) ++ paramMap + val map = fittingParamMap ++ extractParamMap(paramMap) transformSchema(dataset.schema, map, logging = true) stages.foldLeft(dataset)((cur, transformer) => transformer.transform(cur, map)) } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { // Precedence of ParamMaps: paramMap > this.paramMap > fittingParamMap - val map = (fittingParamMap ++ this.paramMap) ++ paramMap + val map = fittingParamMap ++ extractParamMap(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 9a5848684b179..7fb87fe452ee6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Transformer.scala @@ -22,6 +22,7 @@ import scala.annotation.varargs import org.apache.spark.Logging import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ @@ -86,7 +87,7 @@ private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, O protected def validateInputType(inputType: DataType): Unit = {} override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputType = schema(map(inputCol)).dataType validateInputType(inputType) if (schema.fieldNames.contains(map(outputCol))) { @@ -99,7 +100,7 @@ private[ml] abstract class UnaryTransformer[IN, OUT, T <: UnaryTransformer[IN, O override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) dataset.withColumn(map(outputCol), callUDF(this.createTransformFunc(map), outputDataType, dataset(map(inputCol)))) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala index c5fc89f935432..29339c98f51cf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/Classifier.scala @@ -17,12 +17,14 @@ package org.apache.spark.ml.classification -import org.apache.spark.annotation.{DeveloperApi, AlphaComponent} +import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.impl.estimator.{PredictionModel, Predictor, PredictorParams} -import org.apache.spark.ml.param.{Params, ParamMap, HasRawPredictionCol} +import org.apache.spark.ml.param.{ParamMap, Params} +import org.apache.spark.ml.param.shared.HasRawPredictionCol +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} -import org.apache.spark.sql.functions._ import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} @@ -42,8 +44,8 @@ private[spark] trait ClassifierParams extends PredictorParams fitting: Boolean, featuresDataType: DataType): StructType = { val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) - val map = this.paramMap ++ paramMap - addOutputColumn(parentSchema, map(rawPredictionCol), new VectorUDT) + val map = extractParamMap(paramMap) + SchemaUtils.appendColumn(parentSchema, map(rawPredictionCol), new VectorUDT) } } @@ -67,8 +69,7 @@ private[spark] abstract class Classifier[ with ClassifierParams { /** @group setParam */ - def setRawPredictionCol(value: String): E = - set(rawPredictionCol, value).asInstanceOf[E] + def setRawPredictionCol(value: String): E = set(rawPredictionCol, value).asInstanceOf[E] // TODO: defaultEvaluator (follow-up PR) } @@ -109,7 +110,7 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Prepare model val tmpModel = if (paramMap.size != 0) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 34625745dd0a8..cc8b0721cf2b6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -19,11 +19,11 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.classification.LogisticRegressionWithLBFGS import org.apache.spark.mllib.linalg.{VectorUDT, BLAS, Vector, Vectors} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ -import org.apache.spark.sql.types.DoubleType import org.apache.spark.storage.StorageLevel @@ -31,8 +31,10 @@ import org.apache.spark.storage.StorageLevel * Params for logistic regression. */ private[classification] trait LogisticRegressionParams extends ProbabilisticClassifierParams - with HasRegParam with HasMaxIter with HasFitIntercept with HasThreshold + with HasRegParam with HasMaxIter with HasFitIntercept with HasThreshold { + setDefault(regParam -> 0.1, maxIter -> 100, threshold -> 0.5) +} /** * :: AlphaComponent :: @@ -45,10 +47,6 @@ class LogisticRegression extends ProbabilisticClassifier[Vector, LogisticRegression, LogisticRegressionModel] with LogisticRegressionParams { - setRegParam(0.1) - setMaxIter(100) - setThreshold(0.5) - /** @group setParam */ def setRegParam(value: Double): this.type = set(regParam, value) @@ -100,8 +98,6 @@ class LogisticRegressionModel private[ml] ( extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] with LogisticRegressionParams { - setThreshold(0.5) - /** @group setParam */ def setThreshold(value: Double): this.type = set(threshold, value) @@ -123,7 +119,7 @@ class LogisticRegressionModel private[ml] ( // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Output selected columns only. // This is a bit complicated since it tries to avoid repeated computation. @@ -184,7 +180,7 @@ class LogisticRegressionModel private[ml] ( * The behavior of this can be adjusted using [[threshold]]. */ override protected def predict(features: Vector): Double = { - if (score(features) > paramMap(threshold)) 1 else 0 + if (score(features) > getThreshold) 1 else 0 } override protected def predictProbabilities(features: Vector): Vector = { @@ -199,7 +195,7 @@ class LogisticRegressionModel private[ml] ( override protected def copy(): LogisticRegressionModel = { val m = new LogisticRegressionModel(parent, fittingParamMap, weights, intercept) - Params.inheritValues(this.paramMap, this, m) + Params.inheritValues(this.extractParamMap(), this, m) m } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala index bd8caac855981..10404548ccfde 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/ProbabilisticClassifier.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.classification import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} -import org.apache.spark.ml.param.{HasProbabilityCol, ParamMap, Params} +import org.apache.spark.ml.param.{ParamMap, Params} +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DataType, StructType} - /** * Params for probabilistic classification. */ @@ -37,8 +38,8 @@ private[classification] trait ProbabilisticClassifierParams fitting: Boolean, featuresDataType: DataType): StructType = { val parentSchema = super.validateAndTransformSchema(schema, paramMap, fitting, featuresDataType) - val map = this.paramMap ++ paramMap - addOutputColumn(parentSchema, map(probabilityCol), new VectorUDT) + val map = extractParamMap(paramMap) + SchemaUtils.appendColumn(parentSchema, map(probabilityCol), new VectorUDT) } } @@ -102,7 +103,7 @@ private[spark] abstract class ProbabilisticClassificationModel[ // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Prepare model val tmpModel = if (paramMap.size != 0) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index 2360f4479f1c2..c865eb9fe092d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -20,12 +20,13 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.Evaluator import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.mllib.evaluation.BinaryClassificationMetrics import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.types.DoubleType - /** * :: AlphaComponent :: * @@ -40,10 +41,10 @@ class BinaryClassificationEvaluator extends Evaluator with Params * @group param */ val metricName: Param[String] = new Param(this, "metricName", - "metric name in evaluation (areaUnderROC|areaUnderPR)", Some("areaUnderROC")) + "metric name in evaluation (areaUnderROC|areaUnderPR)") /** @group getParam */ - def getMetricName: String = get(metricName) + def getMetricName: String = getOrDefault(metricName) /** @group setParam */ def setMetricName(value: String): this.type = set(metricName, value) @@ -54,12 +55,14 @@ class BinaryClassificationEvaluator extends Evaluator with Params /** @group setParam */ def setLabelCol(value: String): this.type = set(labelCol, value) + setDefault(metricName -> "areaUnderROC") + override def evaluate(dataset: DataFrame, paramMap: ParamMap): Double = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val schema = dataset.schema - checkInputColumn(schema, map(rawPredictionCol), new VectorUDT) - checkInputColumn(schema, map(labelCol), DoubleType) + SchemaUtils.checkColumnType(schema, map(rawPredictionCol), new VectorUDT) + SchemaUtils.checkColumnType(schema, map(labelCol), DoubleType) // TODO: When dataset metadata has been implemented, check rawPredictionCol vector length = 2. val scoreAndLabels = dataset.select(map(rawPredictionCol), map(labelCol)) 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 fc4e12773c46d..b20f2fc49a8f6 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 @@ -35,14 +35,16 @@ class HashingTF extends UnaryTransformer[Iterable[_], Vector, HashingTF] { * number of features * @group param */ - val numFeatures = new IntParam(this, "numFeatures", "number of features", Some(1 << 18)) + val numFeatures = new IntParam(this, "numFeatures", "number of features") /** @group getParam */ - def getNumFeatures: Int = get(numFeatures) + def getNumFeatures: Int = getOrDefault(numFeatures) /** @group setParam */ def setNumFeatures(value: Int): this.type = set(numFeatures, value) + setDefault(numFeatures -> (1 << 18)) + override protected def createTransformFunc(paramMap: ParamMap): Iterable[_] => Vector = { val hashingTF = new feature.HashingTF(paramMap(numFeatures)) hashingTF.transform diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala index 05f91dc9105fe..decaeb0da6246 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Normalizer.scala @@ -35,14 +35,16 @@ class Normalizer extends UnaryTransformer[Vector, Vector, Normalizer] { * Normalization in L^p^ space, p = 2 by default. * @group param */ - val p = new DoubleParam(this, "p", "the p norm value", Some(2)) + val p = new DoubleParam(this, "p", "the p norm value") /** @group getParam */ - def getP: Double = get(p) + def getP: Double = getOrDefault(p) /** @group setParam */ def setP(value: Double): this.type = set(p, value) + setDefault(p -> 2.0) + override protected def createTransformFunc(paramMap: ParamMap): Vector => Vector = { val normalizer = new feature.Normalizer(paramMap(p)) normalizer.transform @@ -50,4 +52,3 @@ class Normalizer extends UnaryTransformer[Vector, Vector, Normalizer] { override protected def outputDataType: DataType = new VectorUDT() } - diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 1142aa4f8e73d..1b102619b3524 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -20,6 +20,7 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml._ import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.feature import org.apache.spark.mllib.linalg.{Vector, VectorUDT} import org.apache.spark.sql._ @@ -47,7 +48,7 @@ class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerP override def fit(dataset: DataFrame, paramMap: ParamMap): StandardScalerModel = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val input = dataset.select(map(inputCol)).map { case Row(v: Vector) => v } val scaler = new feature.StandardScaler().fit(input) val model = new StandardScalerModel(this, map, scaler) @@ -56,7 +57,7 @@ class StandardScaler extends Estimator[StandardScalerModel] with StandardScalerP } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputType = schema(map(inputCol)).dataType require(inputType.isInstanceOf[VectorUDT], s"Input column ${map(inputCol)} must be a vector column") @@ -86,13 +87,13 @@ class StandardScalerModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val scale = udf((v: Vector) => { scaler.transform(v) } : Vector) dataset.withColumn(map(outputCol), scale(col(map(inputCol)))) } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputType = schema(map(inputCol)).dataType require(inputType.isInstanceOf[VectorUDT], s"Input column ${map(inputCol)} must be a vector column") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index 61e6742e880d8..4d960df357fe9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -22,6 +22,8 @@ import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute.NominalAttribute import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.sql.DataFrame import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StringType, StructType} @@ -34,8 +36,8 @@ private[feature] trait StringIndexerBase extends Params with HasInputCol with Ha /** Validates and transforms the input schema. */ protected def validateAndTransformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap - checkInputColumn(schema, map(inputCol), StringType) + val map = extractParamMap(paramMap) + SchemaUtils.checkColumnType(schema, map(inputCol), StringType) val inputFields = schema.fields val outputColName = map(outputCol) require(inputFields.forall(_.name != outputColName), @@ -64,7 +66,7 @@ class StringIndexer extends Estimator[StringIndexerModel] with StringIndexerBase // TODO: handle unseen labels override def fit(dataset: DataFrame, paramMap: ParamMap): StringIndexerModel = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val counts = dataset.select(map(inputCol)).map(_.getString(0)).countByValue() val labels = counts.toSeq.sortBy(-_._2).map(_._1).toArray val model = new StringIndexerModel(this, map, labels) @@ -105,7 +107,7 @@ class StringIndexerModel private[ml] ( def setOutputCol(value: String): this.type = set(outputCol, value) override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val indexer = udf { label: String => if (labelToIndex.contains(label)) { labelToIndex(label) 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 68401e36950bd..376a004858b4c 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 @@ -56,39 +56,39 @@ class RegexTokenizer extends UnaryTransformer[String, Seq[String], RegexTokenize * param for minimum token length, default is one to avoid returning empty strings * @group param */ - val minTokenLength: IntParam = new IntParam(this, "minLength", "minimum token length", Some(1)) + val minTokenLength: IntParam = new IntParam(this, "minLength", "minimum token length") /** @group setParam */ def setMinTokenLength(value: Int): this.type = set(minTokenLength, value) /** @group getParam */ - def getMinTokenLength: Int = get(minTokenLength) + def getMinTokenLength: Int = getOrDefault(minTokenLength) /** * param sets regex as splitting on gaps (true) or matching tokens (false) * @group param */ - val gaps: BooleanParam = new BooleanParam( - this, "gaps", "Set regex to match gaps or tokens", Some(false)) + val gaps: BooleanParam = new BooleanParam(this, "gaps", "Set regex to match gaps or tokens") /** @group setParam */ def setGaps(value: Boolean): this.type = set(gaps, value) /** @group getParam */ - def getGaps: Boolean = get(gaps) + def getGaps: Boolean = getOrDefault(gaps) /** * param sets regex pattern used by tokenizer * @group param */ - val pattern: Param[String] = new Param( - this, "pattern", "regex pattern used for tokenizing", Some("\\p{L}+|[^\\p{L}\\s]+")) + val pattern: Param[String] = new Param(this, "pattern", "regex pattern used for tokenizing") /** @group setParam */ def setPattern(value: String): this.type = set(pattern, value) /** @group getParam */ - def getPattern: String = get(pattern) + def getPattern: String = getOrDefault(pattern) + + setDefault(minTokenLength -> 1, gaps -> false, pattern -> "\\p{L}+|[^\\p{L}\\s]+") override protected def createTransformFunc(paramMap: ParamMap): String => Seq[String] = { str => val re = paramMap(pattern).r diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index d1b8f7e6e9295..e567e069e7c0b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -22,7 +22,8 @@ import scala.collection.mutable.ArrayBuilder import org.apache.spark.SparkException import org.apache.spark.annotation.AlphaComponent import org.apache.spark.ml.Transformer -import org.apache.spark.ml.param.{HasInputCols, HasOutputCol, ParamMap} +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{Vector, VectorUDT, Vectors} import org.apache.spark.sql.{Column, DataFrame, Row} import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute @@ -44,7 +45,7 @@ class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { def setOutputCol(value: String): this.type = set(outputCol, value) override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val assembleFunc = udf { r: Row => VectorAssembler.assemble(r.toSeq: _*) } @@ -61,7 +62,7 @@ class VectorAssembler extends Transformer with HasInputCols with HasOutputCol { } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val inputColNames = map(inputCols) val outputColName = map(outputCol) val inputDataTypes = inputColNames.map(name => schema(name).dataType) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala index 8760960e19272..452faa06e2021 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorIndexer.scala @@ -18,10 +18,12 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.AlphaComponent +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute.{BinaryAttribute, NumericAttribute, NominalAttribute, Attribute, AttributeGroup} -import org.apache.spark.ml.param.{HasInputCol, HasOutputCol, IntParam, ParamMap, Params} +import org.apache.spark.ml.param.{IntParam, ParamMap, Params} +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{SparseVector, DenseVector, Vector, VectorUDT} import org.apache.spark.sql.{Row, DataFrame} import org.apache.spark.sql.functions.callUDF @@ -40,11 +42,12 @@ private[ml] trait VectorIndexerParams extends Params with HasInputCol with HasOu */ val maxCategories = new IntParam(this, "maxCategories", "Threshold for the number of values a categorical feature can take." + - " If a feature is found to have > maxCategories values, then it is declared continuous.", - Some(20)) + " If a feature is found to have > maxCategories values, then it is declared continuous.") /** @group getParam */ - def getMaxCategories: Int = get(maxCategories) + def getMaxCategories: Int = getOrDefault(maxCategories) + + setDefault(maxCategories -> 20) } /** @@ -101,7 +104,7 @@ class VectorIndexer extends Estimator[VectorIndexerModel] with VectorIndexerPara override def fit(dataset: DataFrame, paramMap: ParamMap): VectorIndexerModel = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val firstRow = dataset.select(map(inputCol)).take(1) require(firstRow.length == 1, s"VectorIndexer cannot be fit on an empty dataset.") val numFeatures = firstRow(0).getAs[Vector](0).size @@ -120,12 +123,12 @@ class VectorIndexer extends Estimator[VectorIndexerModel] with VectorIndexerPara override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { // We do not transfer feature metadata since we do not know what types of features we will // produce in transform(). - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val dataType = new VectorUDT require(map.contains(inputCol), s"VectorIndexer requires input column parameter: $inputCol") require(map.contains(outputCol), s"VectorIndexer requires output column parameter: $outputCol") - checkInputColumn(schema, map(inputCol), dataType) - addOutputColumn(schema, map(outputCol), dataType) + SchemaUtils.checkColumnType(schema, map(inputCol), dataType) + SchemaUtils.appendColumn(schema, map(outputCol), dataType) } } @@ -320,7 +323,7 @@ class VectorIndexerModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val newField = prepOutputField(dataset.schema, map) val newCol = callUDF(transformFunc, new VectorUDT, dataset(map(inputCol))) // For now, just check the first row of inputCol for vector length. @@ -334,13 +337,13 @@ class VectorIndexerModel private[ml] ( } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val dataType = new VectorUDT require(map.contains(inputCol), s"VectorIndexerModel requires input column parameter: $inputCol") require(map.contains(outputCol), s"VectorIndexerModel requires output column parameter: $outputCol") - checkInputColumn(schema, map(inputCol), dataType) + SchemaUtils.checkColumnType(schema, map(inputCol), dataType) val origAttrGroup = AttributeGroup.fromStructField(schema(map(inputCol))) val origNumFeatures: Option[Int] = if (origAttrGroup.attributes.nonEmpty) { diff --git a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala index dfb89cc8d4af3..195333a5cc47f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/impl/estimator/Predictor.scala @@ -18,8 +18,10 @@ package org.apache.spark.ml.impl.estimator import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} +import org.apache.spark.ml.util.SchemaUtils import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{VectorUDT, Vector} import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.rdd.RDD @@ -53,14 +55,14 @@ private[spark] trait PredictorParams extends Params paramMap: ParamMap, fitting: Boolean, featuresDataType: DataType): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // TODO: Support casting Array[Double] and Array[Float] to Vector when FeaturesType = Vector - checkInputColumn(schema, map(featuresCol), featuresDataType) + SchemaUtils.checkColumnType(schema, map(featuresCol), featuresDataType) if (fitting) { // TODO: Allow other numeric types - checkInputColumn(schema, map(labelCol), DoubleType) + SchemaUtils.checkColumnType(schema, map(labelCol), DoubleType) } - addOutputColumn(schema, map(predictionCol), DoubleType) + SchemaUtils.appendColumn(schema, map(predictionCol), DoubleType) } } @@ -98,7 +100,7 @@ private[spark] abstract class Predictor[ // This handles a few items such as schema validation. // Developers only need to implement train(). transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val model = train(dataset, map) Params.inheritValues(map, this, model) // copy params to model model @@ -141,7 +143,7 @@ private[spark] abstract class Predictor[ * and put it in an RDD with strong types. */ protected def extractLabeledPoints(dataset: DataFrame, paramMap: ParamMap): RDD[LabeledPoint] = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) dataset.select(map(labelCol), map(featuresCol)) .map { case Row(label: Double, features: Vector) => LabeledPoint(label, features) @@ -201,7 +203,7 @@ private[spark] abstract class PredictionModel[FeaturesType, M <: PredictionModel // Check schema transformSchema(dataset.schema, paramMap, logging = true) - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) // Prepare model val tmpModel = if (paramMap.size != 0) { 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 7d5178d0abb2d..849c60433c777 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,15 +17,14 @@ package org.apache.spark.ml.param +import java.lang.reflect.Modifier +import java.util.NoSuchElementException + import scala.annotation.varargs import scala.collection.mutable -import java.lang.reflect.Modifier - import org.apache.spark.annotation.{AlphaComponent, DeveloperApi} import org.apache.spark.ml.Identifiable -import org.apache.spark.sql.types.{DataType, StructField, StructType} - /** * :: AlphaComponent :: @@ -38,12 +37,7 @@ import org.apache.spark.sql.types.{DataType, StructField, StructType} * @tparam T param value type */ @AlphaComponent -class Param[T] ( - val parent: Params, - val name: String, - val doc: String, - val defaultValue: Option[T] = None) - extends Serializable { +class Param[T] (val parent: Params, val name: String, val doc: String) extends Serializable { /** * Creates a param pair with the given value (for Java). @@ -55,58 +49,55 @@ class Param[T] ( */ def ->(value: T): ParamPair[T] = ParamPair(this, value) + /** + * Converts this param's name, doc, and optionally its default value and the user-supplied + * value in its parent to string. + */ override def toString: String = { - if (defaultValue.isDefined) { - s"$name: $doc (default: ${defaultValue.get})" + val valueStr = if (parent.isDefined(this)) { + val defaultValueStr = parent.getDefault(this).map("default: " + _) + val currentValueStr = parent.get(this).map("current: " + _) + (defaultValueStr ++ currentValueStr).mkString("(", ", ", ")") } else { - s"$name: $doc" + "(undefined)" } + s"$name: $doc $valueStr" } } // specialize primitive-typed params because Java doesn't recognize scala.Double, scala.Int, ... /** Specialized version of [[Param[Double]]] for Java. */ -class DoubleParam(parent: Params, name: String, doc: String, defaultValue: Option[Double]) - extends Param[Double](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class DoubleParam(parent: Params, name: String, doc: String) + extends Param[Double](parent, name, doc) { override def w(value: Double): ParamPair[Double] = super.w(value) } /** Specialized version of [[Param[Int]]] for Java. */ -class IntParam(parent: Params, name: String, doc: String, defaultValue: Option[Int]) - extends Param[Int](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class IntParam(parent: Params, name: String, doc: String) + extends Param[Int](parent, name, doc) { override def w(value: Int): ParamPair[Int] = super.w(value) } /** Specialized version of [[Param[Float]]] for Java. */ -class FloatParam(parent: Params, name: String, doc: String, defaultValue: Option[Float]) - extends Param[Float](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class FloatParam(parent: Params, name: String, doc: String) + extends Param[Float](parent, name, doc) { override def w(value: Float): ParamPair[Float] = super.w(value) } /** Specialized version of [[Param[Long]]] for Java. */ -class LongParam(parent: Params, name: String, doc: String, defaultValue: Option[Long]) - extends Param[Long](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class LongParam(parent: Params, name: String, doc: String) + extends Param[Long](parent, name, doc) { override def w(value: Long): ParamPair[Long] = super.w(value) } /** Specialized version of [[Param[Boolean]]] for Java. */ -class BooleanParam(parent: Params, name: String, doc: String, defaultValue: Option[Boolean]) - extends Param[Boolean](parent, name, doc, defaultValue) { - - def this(parent: Params, name: String, doc: String) = this(parent, name, doc, None) +class BooleanParam(parent: Params, name: String, doc: String) + extends Param[Boolean](parent, name, doc) { override def w(value: Boolean): ParamPair[Boolean] = super.w(value) } @@ -124,8 +115,11 @@ case class ParamPair[T](param: Param[T], value: T) @AlphaComponent trait Params extends Identifiable with Serializable { - /** Returns all params. */ - def params: Array[Param[_]] = { + /** + * Returns all params sorted by their names. The default implementation uses Java reflection to + * list all public methods that have no arguments and return [[Param]]. + */ + lazy val params: Array[Param[_]] = { val methods = this.getClass.getMethods methods.filter { m => Modifier.isPublic(m.getModifiers) && @@ -153,25 +147,29 @@ trait Params extends Identifiable with Serializable { def explainParams(): String = params.mkString("\n") /** Checks whether a param is explicitly set. */ - def isSet(param: Param[_]): Boolean = { - require(param.parent.eq(this)) + final def isSet(param: Param[_]): Boolean = { + shouldOwn(param) paramMap.contains(param) } + /** Checks whether a param is explicitly set or has a default value. */ + final def isDefined(param: Param[_]): Boolean = { + shouldOwn(param) + defaultParamMap.contains(param) || paramMap.contains(param) + } + /** Gets a param by its name. */ - private[ml] def getParam(paramName: String): Param[Any] = { - val m = this.getClass.getMethod(paramName) - assert(Modifier.isPublic(m.getModifiers) && - classOf[Param[_]].isAssignableFrom(m.getReturnType) && - m.getParameterTypes.isEmpty) - m.invoke(this).asInstanceOf[Param[Any]] + def getParam(paramName: String): Param[Any] = { + params.find(_.name == paramName).getOrElse { + throw new NoSuchElementException(s"Param $paramName does not exist.") + }.asInstanceOf[Param[Any]] } /** * Sets a parameter in the embedded param map. */ - protected def set[T](param: Param[T], value: T): this.type = { - require(param.parent.eq(this)) + protected final def set[T](param: Param[T], value: T): this.type = { + shouldOwn(param) paramMap.put(param.asInstanceOf[Param[Any]], value) this } @@ -179,52 +177,102 @@ trait Params extends Identifiable with Serializable { /** * Sets a parameter (by name) in the embedded param map. */ - private[ml] def set(param: String, value: Any): this.type = { + protected final def set(param: String, value: Any): this.type = { set(getParam(param), value) } /** - * Gets the value of a parameter in the embedded param map. + * Optionally returns the user-supplied value of a param. + */ + final def get[T](param: Param[T]): Option[T] = { + shouldOwn(param) + paramMap.get(param) + } + + /** + * Clears the user-supplied value for the input param. + */ + protected final def clear(param: Param[_]): this.type = { + shouldOwn(param) + paramMap.remove(param) + this + } + + /** + * Gets the value of a param in the embedded param map or its default value. Throws an exception + * if neither is set. + */ + final def getOrDefault[T](param: Param[T]): T = { + shouldOwn(param) + get(param).orElse(getDefault(param)).get + } + + /** + * Sets a default value for a param. + * @param param param to set the default value. Make sure that this param is initialized before + * this method gets called. + * @param value the default value */ - protected def get[T](param: Param[T]): T = { - require(param.parent.eq(this)) - paramMap(param) + protected final def setDefault[T](param: Param[T], value: T): this.type = { + shouldOwn(param) + defaultParamMap.put(param, value) + this } /** - * Internal param map. + * Sets default values for a list of params. + * @param paramPairs a list of param pairs that specify params and their default values to set + * respectively. Make sure that the params are initialized before this method + * gets called. */ - protected val paramMap: ParamMap = ParamMap.empty + protected final def setDefault(paramPairs: ParamPair[_]*): this.type = { + paramPairs.foreach { p => + setDefault(p.param.asInstanceOf[Param[Any]], p.value) + } + this + } /** - * Check whether the given schema contains an input column. - * @param colName Input column name - * @param dataType Input column DataType + * Gets the default value of a parameter. */ - protected def checkInputColumn(schema: StructType, colName: String, dataType: DataType): Unit = { - val actualDataType = schema(colName).dataType - require(actualDataType.equals(dataType), s"Input column $colName must be of type $dataType" + - s" but was actually $actualDataType. Column param description: ${getParam(colName)}") + final def getDefault[T](param: Param[T]): Option[T] = { + shouldOwn(param) + defaultParamMap.get(param) } /** - * Add an output column to the given schema. - * This fails if the given output column already exists. - * @param schema Initial schema (not modified) - * @param colName Output column name. If this column name is an empy String "", this method - * returns the initial schema, unchanged. This allows users to disable output - * columns. - * @param dataType Output column DataType - */ - protected def addOutputColumn( - schema: StructType, - colName: String, - dataType: DataType): StructType = { - if (colName.length == 0) return schema - val fieldNames = schema.fieldNames - require(!fieldNames.contains(colName), s"Output column $colName already exists.") - val outputFields = schema.fields ++ Seq(StructField(colName, dataType, nullable = false)) - StructType(outputFields) + * Tests whether the input param has a default value set. + */ + final def hasDefault[T](param: Param[T]): Boolean = { + shouldOwn(param) + defaultParamMap.contains(param) + } + + /** + * Extracts the embedded default param values and user-supplied values, and then merges them with + * extra values from input into a flat param map, where the latter value is used if there exist + * conflicts, i.e., with ordering: default param values < user-supplied values < extraParamMap. + */ + protected final def extractParamMap(extraParamMap: ParamMap): ParamMap = { + defaultParamMap ++ paramMap ++ extraParamMap + } + + /** + * [[extractParamMap]] with no extra values. + */ + protected final def extractParamMap(): ParamMap = { + extractParamMap(ParamMap.empty) + } + + /** Internal param map for user-supplied values. */ + private val paramMap: ParamMap = ParamMap.empty + + /** Internal param map for default values. */ + private val defaultParamMap: ParamMap = ParamMap.empty + + /** Validates that the input param belongs to this instance. */ + private def shouldOwn(param: Param[_]): Unit = { + require(param.parent.eq(this), s"Param $param does not belong to $this.") } } @@ -261,12 +309,13 @@ private[spark] object Params { * A param to value map. */ @AlphaComponent -class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) extends Serializable { +final class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) + extends Serializable { /** * Creates an empty param map. */ - def this() = this(mutable.Map.empty[Param[Any], Any]) + def this() = this(mutable.Map.empty) /** * Puts a (param, value) pair (overwrites if the input param exists). @@ -288,12 +337,17 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten } /** - * Optionally returns the value associated with a param or its default. + * Optionally returns the value associated with a param. */ def get[T](param: Param[T]): Option[T] = { - map.get(param.asInstanceOf[Param[Any]]) - .orElse(param.defaultValue) - .asInstanceOf[Option[T]] + map.get(param.asInstanceOf[Param[Any]]).asInstanceOf[Option[T]] + } + + /** + * Returns the value associated with a param or a default value. + */ + def getOrElse[T](param: Param[T], default: T): T = { + get(param).getOrElse(default) } /** @@ -301,10 +355,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten * Raises a NoSuchElementException if there is no value associated with the input param. */ def apply[T](param: Param[T]): T = { - val value = get(param) - if (value.isDefined) { - value.get - } else { + get(param).getOrElse { throw new NoSuchElementException(s"Cannot find param ${param.name}.") } } @@ -316,6 +367,13 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten map.contains(param.asInstanceOf[Param[Any]]) } + /** + * Removes a key from this map and returns its value associated previously as an option. + */ + def remove[T](param: Param[T]): Option[T] = { + map.remove(param.asInstanceOf[Param[Any]]).asInstanceOf[Option[T]] + } + /** * Filters this param map for the given parent. */ @@ -325,7 +383,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten } /** - * Make a copy of this param map. + * Creates a copy of this param map. */ def copy: ParamMap = new ParamMap(map.clone()) @@ -337,7 +395,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten /** * Returns a new param map that contains parameters in this map and the given map, - * where the latter overwrites this if there exists conflicts. + * where the latter overwrites this if there exist conflicts. */ def ++(other: ParamMap): ParamMap = { // TODO: Provide a better method name for Java users. @@ -363,7 +421,7 @@ class ParamMap private[ml] (private val map: mutable.Map[Param[Any], Any]) exten } /** - * Number of param pairs in this set. + * Number of param pairs in this map. */ def size: Int = map.size } diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala new file mode 100644 index 0000000000000..95d7e64790c79 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/SharedParamsCodeGen.scala @@ -0,0 +1,169 @@ +/* + * 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.ml.param.shared + +import java.io.PrintWriter + +import scala.reflect.ClassTag + +/** + * Code generator for shared params (sharedParams.scala). Run under the Spark folder with + * {{{ + * build/sbt "mllib/runMain org.apache.spark.ml.param.shared.SharedParamsCodeGen" + * }}} + */ +private[shared] object SharedParamsCodeGen { + + def main(args: Array[String]): Unit = { + val params = Seq( + ParamDesc[Double]("regParam", "regularization parameter"), + ParamDesc[Int]("maxIter", "max number of iterations"), + ParamDesc[String]("featuresCol", "features column name", Some("\"features\"")), + ParamDesc[String]("labelCol", "label column name", Some("\"label\"")), + ParamDesc[String]("predictionCol", "prediction column name", Some("\"prediction\"")), + ParamDesc[String]("rawPredictionCol", "raw prediction (a.k.a. confidence) column name", + Some("\"rawPrediction\"")), + ParamDesc[String]("probabilityCol", + "column name for predicted class conditional probabilities", Some("\"probability\"")), + ParamDesc[Double]("threshold", "threshold in binary classification prediction"), + ParamDesc[String]("inputCol", "input column name"), + ParamDesc[Array[String]]("inputCols", "input column names"), + ParamDesc[String]("outputCol", "output column name"), + ParamDesc[Int]("checkpointInterval", "checkpoint interval"), + ParamDesc[Boolean]("fitIntercept", "whether to fit an intercept term", Some("true"))) + + val code = genSharedParams(params) + val file = "src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala" + val writer = new PrintWriter(file) + writer.write(code) + writer.close() + } + + /** Description of a param. */ + private case class ParamDesc[T: ClassTag]( + name: String, + doc: String, + defaultValueStr: Option[String] = None) { + + require(name.matches("[a-z][a-zA-Z0-9]*"), s"Param name $name is invalid.") + require(doc.nonEmpty) // TODO: more rigorous on doc + + def paramTypeName: String = { + val c = implicitly[ClassTag[T]].runtimeClass + c match { + case _ if c == classOf[Int] => "IntParam" + case _ if c == classOf[Long] => "LongParam" + case _ if c == classOf[Float] => "FloatParam" + case _ if c == classOf[Double] => "DoubleParam" + case _ if c == classOf[Boolean] => "BooleanParam" + case _ => s"Param[${getTypeString(c)}]" + } + } + + def valueTypeName: String = { + val c = implicitly[ClassTag[T]].runtimeClass + getTypeString(c) + } + + private def getTypeString(c: Class[_]): String = { + c match { + case _ if c == classOf[Int] => "Int" + case _ if c == classOf[Long] => "Long" + case _ if c == classOf[Float] => "Float" + case _ if c == classOf[Double] => "Double" + case _ if c == classOf[Boolean] => "Boolean" + case _ if c == classOf[String] => "String" + case _ if c.isArray => s"Array[${getTypeString(c.getComponentType)}]" + } + } + } + + /** Generates the HasParam trait code for the input param. */ + private def genHasParamTrait(param: ParamDesc[_]): String = { + val name = param.name + val Name = name(0).toUpper +: name.substring(1) + val Param = param.paramTypeName + val T = param.valueTypeName + val doc = param.doc + val defaultValue = param.defaultValueStr + val defaultValueDoc = defaultValue.map { v => + s" (default: $v)" + }.getOrElse("") + val setDefault = defaultValue.map { v => + s""" + | setDefault($name, $v) + |""".stripMargin + }.getOrElse("") + + s""" + |/** + | * :: DeveloperApi :: + | * Trait for shared param $name$defaultValueDoc. + | */ + |@DeveloperApi + |trait Has$Name extends Params { + | + | /** + | * Param for $doc. + | * @group param + | */ + | final val $name: $Param = new $Param(this, "$name", "$doc") + |$setDefault + | /** @group getParam */ + | final def get$Name: $T = getOrDefault($name) + |} + |""".stripMargin + } + + /** Generates Scala source code for the input params with header. */ + private def genSharedParams(params: Seq[ParamDesc[_]]): String = { + val header = + """/* + | * 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.ml.param.shared + | + |import org.apache.spark.annotation.DeveloperApi + |import org.apache.spark.ml.param._ + | + |// DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen. + | + |// scalastyle:off + |""".stripMargin + + val footer = "// scalastyle:on\n" + + val traits = params.map(genHasParamTrait).mkString + + header + traits + footer + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala new file mode 100644 index 0000000000000..72b08bf276483 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala @@ -0,0 +1,259 @@ +/* + * 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.ml.param.shared + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.ml.param._ + +// DO NOT MODIFY THIS FILE! It was generated by SharedParamsCodeGen. + +// scalastyle:off + +/** + * :: DeveloperApi :: + * Trait for shared param regParam. + */ +@DeveloperApi +trait HasRegParam extends Params { + + /** + * Param for regularization parameter. + * @group param + */ + final val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter") + + /** @group getParam */ + final def getRegParam: Double = getOrDefault(regParam) +} + +/** + * :: DeveloperApi :: + * Trait for shared param maxIter. + */ +@DeveloperApi +trait HasMaxIter extends Params { + + /** + * Param for max number of iterations. + * @group param + */ + final val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") + + /** @group getParam */ + final def getMaxIter: Int = getOrDefault(maxIter) +} + +/** + * :: DeveloperApi :: + * Trait for shared param featuresCol (default: "features"). + */ +@DeveloperApi +trait HasFeaturesCol extends Params { + + /** + * Param for features column name. + * @group param + */ + final val featuresCol: Param[String] = new Param[String](this, "featuresCol", "features column name") + + setDefault(featuresCol, "features") + + /** @group getParam */ + final def getFeaturesCol: String = getOrDefault(featuresCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param labelCol (default: "label"). + */ +@DeveloperApi +trait HasLabelCol extends Params { + + /** + * Param for label column name. + * @group param + */ + final val labelCol: Param[String] = new Param[String](this, "labelCol", "label column name") + + setDefault(labelCol, "label") + + /** @group getParam */ + final def getLabelCol: String = getOrDefault(labelCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param predictionCol (default: "prediction"). + */ +@DeveloperApi +trait HasPredictionCol extends Params { + + /** + * Param for prediction column name. + * @group param + */ + final val predictionCol: Param[String] = new Param[String](this, "predictionCol", "prediction column name") + + setDefault(predictionCol, "prediction") + + /** @group getParam */ + final def getPredictionCol: String = getOrDefault(predictionCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param rawPredictionCol (default: "rawPrediction"). + */ +@DeveloperApi +trait HasRawPredictionCol extends Params { + + /** + * Param for raw prediction (a.k.a. confidence) column name. + * @group param + */ + final val rawPredictionCol: Param[String] = new Param[String](this, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name") + + setDefault(rawPredictionCol, "rawPrediction") + + /** @group getParam */ + final def getRawPredictionCol: String = getOrDefault(rawPredictionCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param probabilityCol (default: "probability"). + */ +@DeveloperApi +trait HasProbabilityCol extends Params { + + /** + * Param for column name for predicted class conditional probabilities. + * @group param + */ + final val probabilityCol: Param[String] = new Param[String](this, "probabilityCol", "column name for predicted class conditional probabilities") + + setDefault(probabilityCol, "probability") + + /** @group getParam */ + final def getProbabilityCol: String = getOrDefault(probabilityCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param threshold. + */ +@DeveloperApi +trait HasThreshold extends Params { + + /** + * Param for threshold in binary classification prediction. + * @group param + */ + final val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in binary classification prediction") + + /** @group getParam */ + final def getThreshold: Double = getOrDefault(threshold) +} + +/** + * :: DeveloperApi :: + * Trait for shared param inputCol. + */ +@DeveloperApi +trait HasInputCol extends Params { + + /** + * Param for input column name. + * @group param + */ + final val inputCol: Param[String] = new Param[String](this, "inputCol", "input column name") + + /** @group getParam */ + final def getInputCol: String = getOrDefault(inputCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param inputCols. + */ +@DeveloperApi +trait HasInputCols extends Params { + + /** + * Param for input column names. + * @group param + */ + final val inputCols: Param[Array[String]] = new Param[Array[String]](this, "inputCols", "input column names") + + /** @group getParam */ + final def getInputCols: Array[String] = getOrDefault(inputCols) +} + +/** + * :: DeveloperApi :: + * Trait for shared param outputCol. + */ +@DeveloperApi +trait HasOutputCol extends Params { + + /** + * Param for output column name. + * @group param + */ + final val outputCol: Param[String] = new Param[String](this, "outputCol", "output column name") + + /** @group getParam */ + final def getOutputCol: String = getOrDefault(outputCol) +} + +/** + * :: DeveloperApi :: + * Trait for shared param checkpointInterval. + */ +@DeveloperApi +trait HasCheckpointInterval extends Params { + + /** + * Param for checkpoint interval. + * @group param + */ + final val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "checkpoint interval") + + /** @group getParam */ + final def getCheckpointInterval: Int = getOrDefault(checkpointInterval) +} + +/** + * :: DeveloperApi :: + * Trait for shared param fitIntercept (default: true). + */ +@DeveloperApi +trait HasFitIntercept extends Params { + + /** + * Param for whether to fit an intercept term. + * @group param + */ + final val fitIntercept: BooleanParam = new BooleanParam(this, "fitIntercept", "whether to fit an intercept term") + + setDefault(fitIntercept, true) + + /** @group getParam */ + final def getFitIntercept: Boolean = getOrDefault(fitIntercept) +} +// scalastyle:on diff --git a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala b/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala deleted file mode 100644 index 07e6eb417763d..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/param/sharedParams.scala +++ /dev/null @@ -1,173 +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.ml.param - -/* NOTE TO DEVELOPERS: - * If you mix these parameter traits into your algorithm, please add a setter method as well - * so that users may use a builder pattern: - * val myLearner = new MyLearner().setParam1(x).setParam2(y)... - */ - -private[ml] trait HasRegParam extends Params { - /** - * param for regularization parameter - * @group param - */ - val regParam: DoubleParam = new DoubleParam(this, "regParam", "regularization parameter") - - /** @group getParam */ - def getRegParam: Double = get(regParam) -} - -private[ml] trait HasMaxIter extends Params { - /** - * param for max number of iterations - * @group param - */ - val maxIter: IntParam = new IntParam(this, "maxIter", "max number of iterations") - - /** @group getParam */ - def getMaxIter: Int = get(maxIter) -} - -private[ml] trait HasFeaturesCol extends Params { - /** - * param for features column name - * @group param - */ - val featuresCol: Param[String] = - new Param(this, "featuresCol", "features column name", Some("features")) - - /** @group getParam */ - def getFeaturesCol: String = get(featuresCol) -} - -private[ml] trait HasLabelCol extends Params { - /** - * param for label column name - * @group param - */ - val labelCol: Param[String] = new Param(this, "labelCol", "label column name", Some("label")) - - /** @group getParam */ - def getLabelCol: String = get(labelCol) -} - -private[ml] trait HasPredictionCol extends Params { - /** - * param for prediction column name - * @group param - */ - val predictionCol: Param[String] = - new Param(this, "predictionCol", "prediction column name", Some("prediction")) - - /** @group getParam */ - def getPredictionCol: String = get(predictionCol) -} - -private[ml] trait HasRawPredictionCol extends Params { - /** - * param for raw prediction column name - * @group param - */ - val rawPredictionCol: Param[String] = - new Param(this, "rawPredictionCol", "raw prediction (a.k.a. confidence) column name", - Some("rawPrediction")) - - /** @group getParam */ - def getRawPredictionCol: String = get(rawPredictionCol) -} - -private[ml] trait HasProbabilityCol extends Params { - /** - * param for predicted class conditional probabilities column name - * @group param - */ - val probabilityCol: Param[String] = - new Param(this, "probabilityCol", "column name for predicted class conditional probabilities", - Some("probability")) - - /** @group getParam */ - def getProbabilityCol: String = get(probabilityCol) -} - -private[ml] trait HasFitIntercept extends Params { - /** - * param for fitting the intercept term, defaults to true - * @group param - */ - val fitIntercept: BooleanParam = - new BooleanParam(this, "fitIntercept", "indicates whether to fit an intercept term", Some(true)) - - /** @group getParam */ - def getFitIntercept: Boolean = get(fitIntercept) -} - -private[ml] trait HasThreshold extends Params { - /** - * param for threshold in (binary) prediction - * @group param - */ - val threshold: DoubleParam = new DoubleParam(this, "threshold", "threshold in prediction") - - /** @group getParam */ - def getThreshold: Double = get(threshold) -} - -private[ml] trait HasInputCol extends Params { - /** - * param for input column name - * @group param - */ - val inputCol: Param[String] = new Param(this, "inputCol", "input column name") - - /** @group getParam */ - def getInputCol: String = get(inputCol) -} - -private[ml] trait HasInputCols extends Params { - /** - * Param for input column names. - */ - val inputCols: Param[Array[String]] = new Param(this, "inputCols", "input column names") - - /** @group getParam */ - def getInputCols: Array[String] = get(inputCols) -} - -private[ml] trait HasOutputCol extends Params { - /** - * param for output column name - * @group param - */ - val outputCol: Param[String] = new Param(this, "outputCol", "output column name") - - /** @group getParam */ - def getOutputCol: String = get(outputCol) -} - -private[ml] trait HasCheckpointInterval extends Params { - /** - * param for checkpoint interval - * @group param - */ - val checkpointInterval: IntParam = new IntParam(this, "checkpointInterval", "checkpoint interval") - - /** @group getParam */ - def getCheckpointInterval: Int = get(checkpointInterval) -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala index 52c9e95d6012f..bd793beba35b6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala @@ -34,6 +34,7 @@ import org.apache.spark.{Logging, Partitioner} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.optimization.NNLS import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame @@ -54,86 +55,88 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR * Param for rank of the matrix factorization. * @group param */ - val rank = new IntParam(this, "rank", "rank of the factorization", Some(10)) + val rank = new IntParam(this, "rank", "rank of the factorization") /** @group getParam */ - def getRank: Int = get(rank) + def getRank: Int = getOrDefault(rank) /** * Param for number of user blocks. * @group param */ - val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks", Some(10)) + val numUserBlocks = new IntParam(this, "numUserBlocks", "number of user blocks") /** @group getParam */ - def getNumUserBlocks: Int = get(numUserBlocks) + def getNumUserBlocks: Int = getOrDefault(numUserBlocks) /** * Param for number of item blocks. * @group param */ val numItemBlocks = - new IntParam(this, "numItemBlocks", "number of item blocks", Some(10)) + new IntParam(this, "numItemBlocks", "number of item blocks") /** @group getParam */ - def getNumItemBlocks: Int = get(numItemBlocks) + def getNumItemBlocks: Int = getOrDefault(numItemBlocks) /** * Param to decide whether to use implicit preference. * @group param */ - val implicitPrefs = - new BooleanParam(this, "implicitPrefs", "whether to use implicit preference", Some(false)) + val implicitPrefs = new BooleanParam(this, "implicitPrefs", "whether to use implicit preference") /** @group getParam */ - def getImplicitPrefs: Boolean = get(implicitPrefs) + def getImplicitPrefs: Boolean = getOrDefault(implicitPrefs) /** * Param for the alpha parameter in the implicit preference formulation. * @group param */ - val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference", Some(1.0)) + val alpha = new DoubleParam(this, "alpha", "alpha for implicit preference") /** @group getParam */ - def getAlpha: Double = get(alpha) + def getAlpha: Double = getOrDefault(alpha) /** * Param for the column name for user ids. * @group param */ - val userCol = new Param[String](this, "userCol", "column name for user ids", Some("user")) + val userCol = new Param[String](this, "userCol", "column name for user ids") /** @group getParam */ - def getUserCol: String = get(userCol) + def getUserCol: String = getOrDefault(userCol) /** * Param for the column name for item ids. * @group param */ - val itemCol = - new Param[String](this, "itemCol", "column name for item ids", Some("item")) + val itemCol = new Param[String](this, "itemCol", "column name for item ids") /** @group getParam */ - def getItemCol: String = get(itemCol) + def getItemCol: String = getOrDefault(itemCol) /** * Param for the column name for ratings. * @group param */ - val ratingCol = new Param[String](this, "ratingCol", "column name for ratings", Some("rating")) + val ratingCol = new Param[String](this, "ratingCol", "column name for ratings") /** @group getParam */ - def getRatingCol: String = get(ratingCol) + def getRatingCol: String = getOrDefault(ratingCol) /** * Param for whether to apply nonnegativity constraints. * @group param */ val nonnegative = new BooleanParam( - this, "nonnegative", "whether to use nonnegative constraint for least squares", Some(false)) + this, "nonnegative", "whether to use nonnegative constraint for least squares") /** @group getParam */ - val getNonnegative: Boolean = get(nonnegative) + def getNonnegative: Boolean = getOrDefault(nonnegative) + + setDefault(rank -> 10, maxIter -> 10, regParam -> 0.1, numUserBlocks -> 10, numItemBlocks -> 10, + implicitPrefs -> false, alpha -> 1.0, userCol -> "user", itemCol -> "item", + ratingCol -> "rating", nonnegative -> false) /** * Validates and transforms the input schema. @@ -142,7 +145,7 @@ private[recommendation] trait ALSParams extends Params with HasMaxIter with HasR * @return output schema */ protected def validateAndTransformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) assert(schema(map(userCol)).dataType == IntegerType) assert(schema(map(itemCol)).dataType== IntegerType) val ratingType = schema(map(ratingCol)).dataType @@ -171,7 +174,7 @@ class ALSModel private[ml] ( override def transform(dataset: DataFrame, paramMap: ParamMap): DataFrame = { import dataset.sqlContext.implicits._ - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val users = userFactors.toDF("id", "features") val items = itemFactors.toDF("id", "features") @@ -283,7 +286,7 @@ class ALS extends Estimator[ALSModel] with ALSParams { setCheckpointInterval(10) override def fit(dataset: DataFrame, paramMap: ParamMap): ALSModel = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val ratings = dataset .select(col(map(userCol)), col(map(itemCol)), col(map(ratingCol)).cast(FloatType)) .map { row => diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index 65f6627a0c351..26ca7459c4fdf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -18,7 +18,8 @@ package org.apache.spark.ml.regression import org.apache.spark.annotation.AlphaComponent -import org.apache.spark.ml.param.{Params, ParamMap, HasMaxIter, HasRegParam} +import org.apache.spark.ml.param.{Params, ParamMap} +import org.apache.spark.ml.param.shared._ import org.apache.spark.mllib.linalg.{BLAS, Vector} import org.apache.spark.mllib.regression.LinearRegressionWithSGD import org.apache.spark.sql.DataFrame @@ -41,8 +42,7 @@ private[regression] trait LinearRegressionParams extends RegressorParams class LinearRegression extends Regressor[Vector, LinearRegression, LinearRegressionModel] with LinearRegressionParams { - setRegParam(0.1) - setMaxIter(100) + setDefault(regParam -> 0.1, maxIter -> 100) /** @group setParam */ def setRegParam(value: Double): this.type = set(regParam, value) @@ -93,7 +93,7 @@ class LinearRegressionModel private[ml] ( override protected def copy(): LinearRegressionModel = { val m = new LinearRegressionModel(parent, fittingParamMap, weights, intercept) - Params.inheritValues(this.paramMap, this, m) + Params.inheritValues(extractParamMap(), this, m) m } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala index 2eb1dac56f1e9..4bb4ed813c006 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/CrossValidator.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.types.StructType * Params for [[CrossValidator]] and [[CrossValidatorModel]]. */ private[ml] trait CrossValidatorParams extends Params { + /** * param for the estimator to be cross-validated * @group param @@ -38,7 +39,7 @@ private[ml] trait CrossValidatorParams extends Params { val estimator: Param[Estimator[_]] = new Param(this, "estimator", "estimator for selection") /** @group getParam */ - def getEstimator: Estimator[_] = get(estimator) + def getEstimator: Estimator[_] = getOrDefault(estimator) /** * param for estimator param maps @@ -48,7 +49,7 @@ private[ml] trait CrossValidatorParams extends Params { new Param(this, "estimatorParamMaps", "param maps for the estimator") /** @group getParam */ - def getEstimatorParamMaps: Array[ParamMap] = get(estimatorParamMaps) + def getEstimatorParamMaps: Array[ParamMap] = getOrDefault(estimatorParamMaps) /** * param for the evaluator for selection @@ -57,17 +58,18 @@ private[ml] trait CrossValidatorParams extends Params { val evaluator: Param[Evaluator] = new Param(this, "evaluator", "evaluator for selection") /** @group getParam */ - def getEvaluator: Evaluator = get(evaluator) + def getEvaluator: Evaluator = getOrDefault(evaluator) /** * param for number of folds for cross validation * @group param */ - val numFolds: IntParam = - new IntParam(this, "numFolds", "number of folds for cross validation", Some(3)) + val numFolds: IntParam = new IntParam(this, "numFolds", "number of folds for cross validation") /** @group getParam */ - def getNumFolds: Int = get(numFolds) + def getNumFolds: Int = getOrDefault(numFolds) + + setDefault(numFolds -> 3) } /** @@ -92,7 +94,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP def setNumFolds(value: Int): this.type = set(numFolds, value) override def fit(dataset: DataFrame, paramMap: ParamMap): CrossValidatorModel = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) val schema = dataset.schema transformSchema(dataset.schema, paramMap, logging = true) val sqlCtx = dataset.sqlContext @@ -130,7 +132,7 @@ class CrossValidator extends Estimator[CrossValidatorModel] with CrossValidatorP } override def transformSchema(schema: StructType, paramMap: ParamMap): StructType = { - val map = this.paramMap ++ paramMap + val map = extractParamMap(paramMap) map(estimator).transformSchema(schema, paramMap) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala new file mode 100644 index 0000000000000..0383bf0b382b7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/SchemaUtils.scala @@ -0,0 +1,61 @@ +/* + * 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.ml.util + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.types.{DataType, StructField, StructType} + +/** + * :: DeveloperApi :: + * Utils for handling schemas. + */ +@DeveloperApi +object SchemaUtils { + + // TODO: Move the utility methods to SQL. + + /** + * Check whether the given schema contains a column of the required data type. + * @param colName column name + * @param dataType required column data type + */ + def checkColumnType(schema: StructType, colName: String, dataType: DataType): Unit = { + val actualDataType = schema(colName).dataType + require(actualDataType.equals(dataType), + s"Column $colName must be of type $dataType but was actually $actualDataType.") + } + + /** + * Appends a new column to the input schema. This fails if the given output column already exists. + * @param schema input schema + * @param colName new column name. If this column name is an empty string "", this method returns + * the input schema unchanged. This allows users to disable output columns. + * @param dataType new column data type + * @return new schema with the input column appended + */ + def appendColumn( + schema: StructType, + colName: String, + dataType: DataType): StructType = { + if (colName.isEmpty) return schema + val fieldNames = schema.fieldNames + require(!fieldNames.contains(colName), s"Column $colName already exists.") + val outputFields = schema.fields :+ StructField(colName, dataType, nullable = false) + StructType(outputFields) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala index 1ce2987612378..88ea679eeaad5 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/ParamsSuite.scala @@ -21,19 +21,25 @@ import org.scalatest.FunSuite class ParamsSuite extends FunSuite { - val solver = new TestParams() - import solver.{inputCol, maxIter} - test("param") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + assert(maxIter.name === "maxIter") assert(maxIter.doc === "max number of iterations") - assert(maxIter.defaultValue.get === 100) assert(maxIter.parent.eq(solver)) - assert(maxIter.toString === "maxIter: max number of iterations (default: 100)") - assert(inputCol.defaultValue === None) + assert(maxIter.toString === "maxIter: max number of iterations (default: 10)") + + solver.setMaxIter(5) + assert(maxIter.toString === "maxIter: max number of iterations (default: 10, current: 5)") + + assert(inputCol.toString === "inputCol: input column name (undefined)") } test("param pair") { + val solver = new TestParams() + import solver.maxIter + val pair0 = maxIter -> 5 val pair1 = maxIter.w(5) val pair2 = ParamPair(maxIter, 5) @@ -44,10 +50,12 @@ class ParamsSuite extends FunSuite { } test("param map") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + val map0 = ParamMap.empty assert(!map0.contains(maxIter)) - assert(map0(maxIter) === maxIter.defaultValue.get) map0.put(maxIter, 10) assert(map0.contains(maxIter)) assert(map0(maxIter) === 10) @@ -78,23 +86,39 @@ class ParamsSuite extends FunSuite { } test("params") { + val solver = new TestParams() + import solver.{maxIter, inputCol} + val params = solver.params - assert(params.size === 2) + assert(params.length === 2) assert(params(0).eq(inputCol), "params must be ordered by name") assert(params(1).eq(maxIter)) + + assert(!solver.isSet(maxIter)) + assert(solver.isDefined(maxIter)) + assert(solver.getMaxIter === 10) + solver.setMaxIter(100) + assert(solver.isSet(maxIter)) + assert(solver.getMaxIter === 100) + assert(!solver.isSet(inputCol)) + assert(!solver.isDefined(inputCol)) + intercept[NoSuchElementException](solver.getInputCol) + assert(solver.explainParams() === Seq(inputCol, maxIter).mkString("\n")) + assert(solver.getParam("inputCol").eq(inputCol)) assert(solver.getParam("maxIter").eq(maxIter)) - intercept[NoSuchMethodException] { + intercept[NoSuchElementException] { solver.getParam("abc") } - assert(!solver.isSet(inputCol)) + intercept[IllegalArgumentException] { solver.validate() } solver.validate(ParamMap(inputCol -> "input")) solver.setInputCol("input") assert(solver.isSet(inputCol)) + assert(solver.isDefined(inputCol)) assert(solver.getInputCol === "input") solver.validate() intercept[IllegalArgumentException] { @@ -104,5 +128,8 @@ class ParamsSuite extends FunSuite { intercept[IllegalArgumentException] { solver.validate() } + + solver.clearMaxIter() + assert(!solver.isSet(maxIter)) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala index ce52f2f230085..8f9ab687c05cb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala @@ -20,17 +20,21 @@ package org.apache.spark.ml.param /** A subclass of Params for testing. */ class TestParams extends Params { - val maxIter = new IntParam(this, "maxIter", "max number of iterations", Some(100)) + val maxIter = new IntParam(this, "maxIter", "max number of iterations") def setMaxIter(value: Int): this.type = { set(maxIter, value); this } - def getMaxIter: Int = get(maxIter) + def getMaxIter: Int = getOrDefault(maxIter) val inputCol = new Param[String](this, "inputCol", "input column name") def setInputCol(value: String): this.type = { set(inputCol, value); this } - def getInputCol: String = get(inputCol) + def getInputCol: String = getOrDefault(inputCol) + + setDefault(maxIter -> 10) override def validate(paramMap: ParamMap): Unit = { - val m = this.paramMap ++ paramMap + val m = extractParamMap(paramMap) require(m(maxIter) >= 0) require(m.contains(inputCol)) } + + def clearMaxIter(): this.type = clear(maxIter) } From 77eeb10fd1f5beb558d6278d9443d267fc6c2f62 Mon Sep 17 00:00:00 2001 From: Brennon York Date: Mon, 13 Apr 2015 22:31:44 -0700 Subject: [PATCH 084/128] [WIP][HOTFIX][SPARK-4123]: Fix bug in PR dependency (all deps. removed issue) We're seeing a bug sporadically in the new PR dependency comparison test whereby it notes that *all* dependencies are removed. This happens when the current PR is built, but the final, sorted, dependency file is left blank. I believe this is an error either in the way the `git checkout` calls have been or an error within the `mvn` build for that PR (again, likely related to the `git checkout`). As such I've set the checkouts to now force (with `-f` flag) which is more in line with what Jenkins currently does on the initial checkout. Setting this as a WIP for now to trigger the build process myriad times to see if the issue still arises. Author: Brennon York Closes #5443 from brennonyork/HOTFIX2-SPARK-4123 and squashes the following commits: f2186be [Brennon York] added output for the various git commit refs 3f073d6 [Brennon York] removed the git checkouts piping to dev null 07765a6 [Brennon York] updated the diff logic to reference the filenames rather than hardlink e3f63c7 [Brennon York] added '-f' to the checkout flags for git 710c8d1 [Brennon York] added 30 minutes to the test benchmark --- dev/run-tests-jenkins | 4 ++++ dev/tests/pr_new_dependencies.sh | 8 ++++---- 2 files changed, 8 insertions(+), 4 deletions(-) diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index f6372835a6dbf..3c1c91a111357 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -161,6 +161,10 @@ pr_message="" # Ensure we save off the current HEAD to revert to current_pr_head="`git rev-parse HEAD`" +echo "HEAD: `git rev-parse HEAD`" +echo "GHPRB: $ghprbActualCommit" +echo "SHA1: $sha1" + # Run pull request tests for t in "${PR_TESTS[@]}"; do this_test="${FWDIR}/dev/tests/${t}.sh" diff --git a/dev/tests/pr_new_dependencies.sh b/dev/tests/pr_new_dependencies.sh index 370c7cc737bbd..fdfb3c62aff58 100755 --- a/dev/tests/pr_new_dependencies.sh +++ b/dev/tests/pr_new_dependencies.sh @@ -39,12 +39,12 @@ CURR_CP_FILE="my-classpath.txt" MASTER_CP_FILE="master-classpath.txt" # First switch over to the master branch -git checkout master &>/dev/null +git checkout -f master # Find and copy all pom.xml files into a *.gate file that we can check # against through various `git` changes find -name "pom.xml" -exec cp {} {}.gate \; # Switch back to the current PR -git checkout "${current_pr_head}" &>/dev/null +git checkout -f "${current_pr_head}" # Check if any *.pom files from the current branch are different from the master difference_q="" @@ -71,7 +71,7 @@ else sort > ${CURR_CP_FILE} # Checkout the master branch to compare against - git checkout master &>/dev/null + git checkout -f master ${MVN_BIN} clean package dependency:build-classpath -DskipTests 2>/dev/null | \ sed -n -e '/Building Spark Project Assembly/,$p' | \ @@ -84,7 +84,7 @@ else rev | \ sort > ${MASTER_CP_FILE} - DIFF_RESULTS="`diff my-classpath.txt master-classpath.txt`" + DIFF_RESULTS="`diff ${CURR_CP_FILE} ${MASTER_CP_FILE}`" if [ -z "${DIFF_RESULTS}" ]; then echo " * This patch does not change any dependencies." From 628a72f70ed06b8d7aee81cfb16070eb2c87b9cd Mon Sep 17 00:00:00 2001 From: Punyashloka Biswal Date: Tue, 14 Apr 2015 11:43:06 +0100 Subject: [PATCH 085/128] [SPARK-6731] Bump version of apache commons-math3 Version 3.1.1 is two years old and the newer version includes approximate percentile statistics (among other things). Author: Punyashloka Biswal Closes #5380 from punya/patch-1 and squashes the following commits: 226622b [Punyashloka Biswal] Bump version of apache commons-math3 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index d8881c213bf07..0b8d664dbfe1b 100644 --- a/pom.xml +++ b/pom.xml @@ -147,7 +147,7 @@ 1.8.3 1.1.0 4.2.6 - 3.1.1 + 3.4.1 ${project.build.directory}/spark-test-classpath.txt 2.10.4 2.10 From 51b306b930cfe03ad21af72a3a6ef31e6e626235 Mon Sep 17 00:00:00 2001 From: Erik van Oosten Date: Tue, 14 Apr 2015 12:39:56 +0100 Subject: [PATCH 086/128] SPARK-6878 [CORE] Fix for sum on empty RDD fails with exception Author: Erik van Oosten Closes #5489 from erikvanoosten/master and squashes the following commits: 1c91954 [Erik van Oosten] Rewrote double range matcher to an exact equality assert (SPARK-6878) f1708c9 [Erik van Oosten] Fix for sum on empty RDD fails with exception (SPARK-6878) --- .../scala/org/apache/spark/rdd/DoubleRDDFunctions.scala | 2 +- .../test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala | 6 ++++++ 2 files changed, 7 insertions(+), 1 deletion(-) 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 29ca3e9c4bd04..843a893235e56 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -31,7 +31,7 @@ import org.apache.spark.util.StatCounter class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** Add up the elements in this RDD. */ def sum(): Double = { - self.reduce(_ + _) + self.fold(0.0)(_ + _) } /** 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 97079382c716f..01039b9449daf 100644 --- a/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/DoubleRDDSuite.scala @@ -22,6 +22,12 @@ import org.scalatest.FunSuite import org.apache.spark._ class DoubleRDDSuite extends FunSuite with SharedSparkContext { + test("sum") { + assert(sc.parallelize(Seq.empty[Double]).sum() === 0.0) + assert(sc.parallelize(Seq(1.0)).sum() === 1.0) + assert(sc.parallelize(Seq(1.0, 2.0)).sum() === 3.0) + } + // Verify tests on the histogram functionality. We test with both evenly // and non-evenly spaced buckets as the bucket lookup function changes. test("WorksOnEmpty") { From 320bca4508e890b874c2eb7abb76a30ef14c932f Mon Sep 17 00:00:00 2001 From: Timothy Chen Date: Tue, 14 Apr 2015 11:48:12 -0700 Subject: [PATCH 087/128] [SPARK-6081] Support fetching http/https uris in driver runner. Currently if passed uris such as http/https, it won't able to fetch them as it only calls HadoopFs get. This fix utilizes the existing util method to fetch remote uris as well. Author: Timothy Chen Closes #4832 from tnachen/driver_remote and squashes the following commits: aa52cd6 [Timothy Chen] Support fetching remote uris in driver runner. --- .../spark/deploy/worker/DriverRunner.scala | 21 ++++++++++++------- .../apache/spark/deploy/worker/Worker.scala | 3 ++- .../spark/deploy/JsonProtocolSuite.scala | 7 ++++--- .../deploy/worker/DriverRunnerTest.scala | 7 ++++--- 4 files changed, 23 insertions(+), 15 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index e0948e16ef354..ef7a703bffe67 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -24,14 +24,14 @@ import scala.collection.JavaConversions._ import akka.actor.ActorRef import com.google.common.base.Charsets.UTF_8 import com.google.common.io.Files -import org.apache.hadoop.fs.{FileUtil, Path} +import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SecurityManager} import org.apache.spark.deploy.{DriverDescription, SparkHadoopUtil} import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState -import org.apache.spark.util.{Clock, SystemClock} +import org.apache.spark.util.{Utils, Clock, SystemClock} /** * Manages the execution of one driver, including automatically restarting the driver on failure. @@ -44,7 +44,8 @@ private[deploy] class DriverRunner( val sparkHome: File, val driverDesc: DriverDescription, val worker: ActorRef, - val workerUrl: String) + val workerUrl: String, + val securityManager: SecurityManager) extends Logging { @volatile private var process: Option[Process] = None @@ -136,12 +137,9 @@ private[deploy] class DriverRunner( * Will throw an exception if there are errors downloading the jar. */ private def downloadUserJar(driverDir: File): String = { - val jarPath = new Path(driverDesc.jarUrl) val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val jarFileSystem = jarPath.getFileSystem(hadoopConf) - val destPath = new File(driverDir.getAbsolutePath, jarPath.getName) val jarFileName = jarPath.getName val localJarFile = new File(driverDir, jarFileName) @@ -149,7 +147,14 @@ private[deploy] class DriverRunner( if (!localJarFile.exists()) { // May already exist if running multiple workers on one node logInfo(s"Copying user jar $jarPath to $destPath") - FileUtil.copy(jarFileSystem, jarPath, destPath, false, hadoopConf) + Utils.fetchFile( + driverDesc.jarUrl, + driverDir, + conf, + securityManager, + hadoopConf, + System.currentTimeMillis(), + useCache = false) } if (!localJarFile.exists()) { // Verify copy succeeded 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 c4c24a7866aa3..3ee2eb69e8a4e 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 @@ -436,7 +436,8 @@ private[worker] class Worker( sparkHome, driverDesc.copy(command = Worker.maybeUpdateSSLSettings(driverDesc.command, conf)), self, - akkaUrl) + akkaUrl, + securityMgr) drivers(driverId) = driver driver.start() diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 2071701b313db..b58d62567afe1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, RecoveryState, WorkerInfo} import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} class JsonProtocolSuite extends FunSuite { @@ -124,8 +124,9 @@ class JsonProtocolSuite extends FunSuite { } def createDriverRunner(): DriverRunner = { - new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), - createDriverDesc(), null, "akka://worker") + val conf = new SparkConf() + new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), + createDriverDesc(), null, "akka://worker", new SecurityManager(conf)) } def assertValidJson(json: JValue) { diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index aa6e4874cecde..2159fd8c16c6f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -25,7 +25,7 @@ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.FunSuite -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.{Command, DriverDescription} import org.apache.spark.util.Clock @@ -33,8 +33,9 @@ class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { val command = new Command("mainClass", Seq(), Map(), Seq(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) - new DriverRunner(new SparkConf(), "driverId", new File("workDir"), new File("sparkHome"), - driverDescription, null, "akka://1.2.3.4/worker/") + val conf = new SparkConf() + new DriverRunner(conf, "driverId", new File("workDir"), new File("sparkHome"), + driverDescription, null, "akka://1.2.3.4/worker/", new SecurityManager(conf)) } private def createProcessBuilderAndProcess(): (ProcessBuilderLike, Process) = { From f63b44a5c201d9678738a906462be9a6d7e3e8f8 Mon Sep 17 00:00:00 2001 From: WangTaoTheTonic Date: Tue, 14 Apr 2015 12:02:11 -0700 Subject: [PATCH 088/128] [SPARK-6894]spark.executor.extraLibraryOptions => spark.executor.extraLibraryPath https://issues.apache.org/jira/browse/SPARK-6894 cc vanzin Author: WangTaoTheTonic Closes #5506 from WangTaoTheTonic/SPARK-6894 and squashes the following commits: 4b7ced7 [WangTaoTheTonic] spark.executor.extraLibraryOptions => spark.executor.extraLibraryPath --- .../src/main/java/org/apache/spark/launcher/SparkLauncher.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java index b566507ee6061..d4cfeacb6ef18 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java +++ b/launcher/src/main/java/org/apache/spark/launcher/SparkLauncher.java @@ -52,7 +52,7 @@ public class SparkLauncher { /** Configuration key for the executor VM options. */ public static final String EXECUTOR_EXTRA_JAVA_OPTIONS = "spark.executor.extraJavaOptions"; /** Configuration key for the executor native library path. */ - public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryOptions"; + public static final String EXECUTOR_EXTRA_LIBRARY_PATH = "spark.executor.extraLibraryPath"; /** Configuration key for the number of executor CPU cores. */ public static final String EXECUTOR_CORES = "spark.executor.cores"; From dcf8a9f331c6193a62bbc9282bdc99663e23ca19 Mon Sep 17 00:00:00 2001 From: pankaj arora Date: Tue, 14 Apr 2015 12:06:46 -0700 Subject: [PATCH 089/128] [CORE] SPARK-6880: Fixed null check when all the dependent stages are cancelled due to previous stage failure Fixed null check when all the dependent stages are cancelled due to previous stage failure. This happens when one of the executor node goes down and all the dependent stages are cancelled. Author: pankaj arora Closes #5494 from pankajarora12/NEWBRANCH and squashes the following commits: 55ba5e3 [pankaj arora] [CORE] SPARK-6880: Fixed null check when all the dependent stages are cancelled due to previous stage failure 4575720 [pankaj arora] [CORE] SPARK-6880: Fixed null check when all the dependent stages are cancelled due to previous stage failure --- .../scala/org/apache/spark/scheduler/DAGScheduler.scala | 7 +------ 1 file changed, 1 insertion(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 508fe7b3303ca..4a32f8936fb0e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -818,12 +818,7 @@ class DAGScheduler( } } - val properties = if (jobIdToActiveJob.contains(jobId)) { - jobIdToActiveJob(stage.jobId).properties - } else { - // this stage will be assigned to "default" pool - null - } + val properties = jobIdToActiveJob.get(stage.jobId).map(_.properties).orNull runningStages += stage // SparkListenerStageSubmitted should be posted before testing whether tasks are From 25998e4d73bcc95ac85d9af71adfdc726ec89568 Mon Sep 17 00:00:00 2001 From: GuoQiang Li Date: Tue, 14 Apr 2015 12:56:47 -0700 Subject: [PATCH 090/128] [SPARK-2033] Automatically cleanup checkpoint Author: GuoQiang Li Closes #855 from witgo/cleanup_checkpoint_date and squashes the following commits: 1649850 [GuoQiang Li] review commit c0087e0 [GuoQiang Li] Automatically cleanup checkpoint --- .../org/apache/spark/ContextCleaner.scala | 44 ++++++++++++----- .../apache/spark/rdd/RDDCheckpointData.scala | 27 ++++++++-- .../apache/spark/ContextCleanerSuite.scala | 49 ++++++++++++++++++- 3 files changed, 102 insertions(+), 18 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 9b05c9623b704..715b259057569 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -22,7 +22,7 @@ import java.lang.ref.{ReferenceQueue, WeakReference} import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{RDDCheckpointData, RDD} import org.apache.spark.util.Utils /** @@ -33,6 +33,7 @@ private case class CleanRDD(rddId: Int) extends CleanupTask private case class CleanShuffle(shuffleId: Int) extends CleanupTask private case class CleanBroadcast(broadcastId: Long) extends CleanupTask private case class CleanAccum(accId: Long) extends CleanupTask +private case class CleanCheckpoint(rddId: Int) extends CleanupTask /** * A WeakReference associated with a CleanupTask. @@ -94,12 +95,12 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { @volatile private var stopped = false /** Attach a listener object to get information of when objects are cleaned. */ - def attachListener(listener: CleanerListener) { + def attachListener(listener: CleanerListener): Unit = { listeners += listener } /** Start the cleaner. */ - def start() { + def start(): Unit = { cleaningThread.setDaemon(true) cleaningThread.setName("Spark Context Cleaner") cleaningThread.start() @@ -108,7 +109,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { /** * Stop the cleaning thread and wait until the thread has finished running its current task. */ - def stop() { + def stop(): Unit = { stopped = true // Interrupt the cleaning thread, but wait until the current task has finished before // doing so. This guards against the race condition where a cleaning thread may @@ -121,7 +122,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Register a RDD for cleanup when it is garbage collected. */ - def registerRDDForCleanup(rdd: RDD[_]) { + def registerRDDForCleanup(rdd: RDD[_]): Unit = { registerForCleanup(rdd, CleanRDD(rdd.id)) } @@ -130,17 +131,22 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Register a ShuffleDependency for cleanup when it is garbage collected. */ - def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _, _]) { + def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _, _]): Unit = { registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId)) } /** Register a Broadcast for cleanup when it is garbage collected. */ - def registerBroadcastForCleanup[T](broadcast: Broadcast[T]) { + def registerBroadcastForCleanup[T](broadcast: Broadcast[T]): Unit = { registerForCleanup(broadcast, CleanBroadcast(broadcast.id)) } + /** Register a RDDCheckpointData for cleanup when it is garbage collected. */ + def registerRDDCheckpointDataForCleanup[T](rdd: RDD[_], parentId: Int): Unit = { + registerForCleanup(rdd, CleanCheckpoint(parentId)) + } + /** Register an object for cleanup. */ - private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask) { + private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask): Unit = { referenceBuffer += new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue) } @@ -164,6 +170,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) case CleanAccum(accId) => doCleanupAccum(accId, blocking = blockOnCleanupTasks) + case CleanCheckpoint(rddId) => + doCleanCheckpoint(rddId) } } } @@ -175,7 +183,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform RDD cleanup. */ - def doCleanupRDD(rddId: Int, blocking: Boolean) { + def doCleanupRDD(rddId: Int, blocking: Boolean): Unit = { try { logDebug("Cleaning RDD " + rddId) sc.unpersistRDD(rddId, blocking) @@ -187,7 +195,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform shuffle cleanup, asynchronously. */ - def doCleanupShuffle(shuffleId: Int, blocking: Boolean) { + def doCleanupShuffle(shuffleId: Int, blocking: Boolean): Unit = { try { logDebug("Cleaning shuffle " + shuffleId) mapOutputTrackerMaster.unregisterShuffle(shuffleId) @@ -200,7 +208,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform broadcast cleanup. */ - def doCleanupBroadcast(broadcastId: Long, blocking: Boolean) { + def doCleanupBroadcast(broadcastId: Long, blocking: Boolean): Unit = { try { logDebug(s"Cleaning broadcast $broadcastId") broadcastManager.unbroadcast(broadcastId, true, blocking) @@ -212,7 +220,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } /** Perform accumulator cleanup. */ - def doCleanupAccum(accId: Long, blocking: Boolean) { + def doCleanupAccum(accId: Long, blocking: Boolean): Unit = { try { logDebug("Cleaning accumulator " + accId) Accumulators.remove(accId) @@ -223,6 +231,18 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { } } + /** Perform checkpoint cleanup. */ + def doCleanCheckpoint(rddId: Int): Unit = { + try { + logDebug("Cleaning rdd checkpoint data " + rddId) + RDDCheckpointData.clearRDDCheckpointData(sc, rddId) + logInfo("Cleaned rdd checkpoint data " + rddId) + } + catch { + case e: Exception => logError("Error cleaning rdd checkpoint data " + rddId, e) + } + } + private def blockManagerMaster = sc.env.blockManager.master private def broadcastManager = sc.env.broadcastManager private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] diff --git a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala index 6afd63d537d75..1722c27e55003 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDDCheckpointData.scala @@ -21,7 +21,7 @@ import scala.reflect.ClassTag import org.apache.hadoop.fs.Path -import org.apache.spark.{Logging, Partition, SerializableWritable, SparkException} +import org.apache.spark._ import org.apache.spark.scheduler.{ResultTask, ShuffleMapTask} /** @@ -83,7 +83,7 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } // Create the output path for the checkpoint - val path = new Path(rdd.context.checkpointDir.get, "rdd-" + rdd.id) + val path = RDDCheckpointData.rddCheckpointDataPath(rdd.context, rdd.id).get val fs = path.getFileSystem(rdd.context.hadoopConfiguration) if (!fs.mkdirs(path)) { throw new SparkException("Failed to create checkpoint path " + path) @@ -92,8 +92,13 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) // Save to file, and reload it as an RDD val broadcastedConf = rdd.context.broadcast( new SerializableWritable(rdd.context.hadoopConfiguration)) - rdd.context.runJob(rdd, CheckpointRDD.writeToFile[T](path.toString, broadcastedConf) _) val newRDD = new CheckpointRDD[T](rdd.context, path.toString) + if (rdd.conf.getBoolean("spark.cleaner.referenceTracking.cleanCheckpoints", false)) { + rdd.context.cleaner.foreach { cleaner => + cleaner.registerRDDCheckpointDataForCleanup(newRDD, rdd.id) + } + } + rdd.context.runJob(rdd, CheckpointRDD.writeToFile[T](path.toString, broadcastedConf) _) if (newRDD.partitions.length != rdd.partitions.length) { throw new SparkException( "Checkpoint RDD " + newRDD + "(" + newRDD.partitions.length + ") has different " + @@ -130,5 +135,17 @@ private[spark] class RDDCheckpointData[T: ClassTag](@transient rdd: RDD[T]) } } -// Used for synchronization -private[spark] object RDDCheckpointData +private[spark] object RDDCheckpointData { + def rddCheckpointDataPath(sc: SparkContext, rddId: Int): Option[Path] = { + sc.checkpointDir.map { dir => new Path(dir, "rdd-" + rddId) } + } + + def clearRDDCheckpointData(sc: SparkContext, rddId: Int): Unit = { + rddCheckpointDataPath(sc, rddId).foreach { path => + val fs = path.getFileSystem(sc.hadoopConfiguration) + if (fs.exists(path)) { + fs.delete(path, true) + } + } + } +} diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 1de169d964d23..097e7076e5391 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -28,7 +28,8 @@ import org.scalatest.concurrent.{PatienceConfiguration, Eventually} import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.{RDDCheckpointData, RDD} import org.apache.spark.storage._ import org.apache.spark.shuffle.hash.HashShuffleManager import org.apache.spark.shuffle.sort.SortShuffleManager @@ -205,6 +206,52 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { postGCTester.assertCleanup() } + test("automatically cleanup checkpoint") { + val checkpointDir = java.io.File.createTempFile("temp", "") + checkpointDir.deleteOnExit() + checkpointDir.delete() + var rdd = newPairRDD + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + var rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).isDefined) + val path = RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get + val fs = path.getFileSystem(sc.hadoopConfiguration) + assert(fs.exists(path)) + + // the checkpoint is not cleaned by default (without the configuration set) + var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + assert(fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + + sc.stop() + val conf = new SparkConf().setMaster("local[2]").setAppName("cleanupCheckpoint"). + set("spark.cleaner.referenceTracking.cleanCheckpoints", "true") + sc = new SparkContext(conf) + rdd = newPairRDD + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + + // Test that GC causes checkpoint data cleanup after dereferencing the RDD + postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + assert(!fs.exists(RDDCheckpointData.rddCheckpointDataPath(sc, rddId).get)) + } + test("automatically cleanup RDD + shuffle + broadcast") { val numRdds = 100 val numBroadcasts = 4 // Broadcasts are more costly From 8f8dc45f6d4c8d7b740eaa3d2ea09d0b531af9dd Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 14 Apr 2015 13:32:06 -0700 Subject: [PATCH 091/128] SPARK-1706: Allow multiple executors per worker in Standalone mode resubmit of https://github.com/apache/spark/pull/636 for a totally different algorithm https://issues.apache.org/jira/browse/SPARK-1706 In current implementation, the user has to start multiple workers in a server for starting multiple executors in a server, which introduces additional overhead due to the more JVM processes... In this patch, I changed the scheduling logic in master to enable the user to start multiple executor processes within the same JVM process. 1. user configure spark.executor.maxCoreNumPerExecutor to suggest the maximum core he/she would like to allocate to each executor 2. Master assigns the executors to the workers with the major consideration on the memoryPerExecutor and the worker.freeMemory, and tries to allocate as many as possible cores to the executor ```min(min(memoryPerExecutor, worker.freeCore), maxLeftCoreToAssign)``` where ```maxLeftCoreToAssign = maxExecutorCanAssign * maxCoreNumPerExecutor``` --------------------------------------- Other small changes include change memoryPerSlave in ApplicationDescription to memoryPerExecutor, as "Slave" is overrided to represent both worker and executor in the documents... (we have some discussion on this before?) Author: CodingCat Closes #731 from CodingCat/SPARK-1706-2 and squashes the following commits: 6dee808 [CodingCat] change filter predicate fbeb7e5 [CodingCat] address the comments 940cb42 [CodingCat] avoid unnecessary allocation b8ca561 [CodingCat] revert a change 45967b4 [CodingCat] remove unused method 2eeff77 [CodingCat] stylistic fixes 12a1b32 [CodingCat] change the semantic of coresPerExecutor to exact core number f035423 [CodingCat] stylistic fix d9c1685 [CodingCat] remove unused var f595bd6 [CodingCat] recover some unintentional changes 63b3df9 [CodingCat] change the description of the parameter in the submit script 4cf61f1 [CodingCat] improve the code and docs ff011e2 [CodingCat] start multiple executors on the worker by rewriting startExeuctor logic 2c2bcc5 [CodingCat] fix wrong usage info 497ec2c [CodingCat] address andrew's comments 878402c [CodingCat] change the launching executor code f64a28d [CodingCat] typo fix 387f4ec [CodingCat] bug fix 35c462c [CodingCat] address Andrew's comments 0b64fea [CodingCat] fix compilation issue 19d3da7 [CodingCat] address the comments 5b81466 [CodingCat] remove outdated comments ec7d421 [CodingCat] test commit e5efabb [CodingCat] more java docs and consolidate canUse function a26096d [CodingCat] stylistic fix a5d629a [CodingCat] java doc b34ec0c [CodingCat] make master support multiple executors per worker --- .../spark/deploy/ApplicationDescription.scala | 9 +- .../apache/spark/deploy/JsonProtocol.scala | 4 +- .../org/apache/spark/deploy/SparkSubmit.scala | 2 + .../spark/deploy/SparkSubmitArguments.scala | 5 +- .../spark/deploy/master/ApplicationInfo.scala | 8 +- .../apache/spark/deploy/master/Master.scala | 117 +++++++++--------- .../deploy/master/ui/ApplicationPage.scala | 2 +- .../spark/deploy/master/ui/MasterPage.scala | 4 +- .../cluster/SparkDeploySchedulerBackend.scala | 7 +- docs/configuration.md | 11 ++ 10 files changed, 96 insertions(+), 73 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index b7ae9c1fc0a23..ae99432f5ce86 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -22,12 +22,13 @@ import java.net.URI private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], - val memoryPerSlave: Int, + val memoryPerExecutorMB: Int, val command: Command, var appUiUrl: String, val eventLogDir: Option[URI] = None, // short name of compression codec used when writing event logs, if any (e.g. lzf) - val eventLogCodec: Option[String] = None) + val eventLogCodec: Option[String] = None, + val coresPerExecutor: Option[Int] = None) extends Serializable { val user = System.getProperty("user.name", "") @@ -35,13 +36,13 @@ private[spark] class ApplicationDescription( def copy( name: String = name, maxCores: Option[Int] = maxCores, - memoryPerSlave: Int = memoryPerSlave, + memoryPerExecutorMB: Int = memoryPerExecutorMB, command: Command = command, appUiUrl: String = appUiUrl, eventLogDir: Option[URI] = eventLogDir, eventLogCodec: Option[String] = eventLogCodec): ApplicationDescription = new ApplicationDescription( - name, maxCores, memoryPerSlave, command, appUiUrl, eventLogDir, eventLogCodec) + name, maxCores, memoryPerExecutorMB, command, appUiUrl, eventLogDir, eventLogCodec) override def toString: String = "ApplicationDescription(" + name + ")" } diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index dfc5b97e6a6c8..2954f932b4f41 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -46,7 +46,7 @@ private[deploy] object JsonProtocol { ("name" -> obj.desc.name) ~ ("cores" -> obj.desc.maxCores) ~ ("user" -> obj.desc.user) ~ - ("memoryperslave" -> obj.desc.memoryPerSlave) ~ + ("memoryperslave" -> obj.desc.memoryPerExecutorMB) ~ ("submitdate" -> obj.submitDate.toString) ~ ("state" -> obj.state.toString) ~ ("duration" -> obj.duration) @@ -55,7 +55,7 @@ private[deploy] object JsonProtocol { def writeApplicationDescription(obj: ApplicationDescription): JObject = { ("name" -> obj.name) ~ ("cores" -> obj.maxCores) ~ - ("memoryperslave" -> obj.memoryPerSlave) ~ + ("memoryperslave" -> obj.memoryPerExecutorMB) ~ ("user" -> obj.user) ~ ("command" -> obj.command.toString) } 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 60bc243ebf40a..296a0764b8baf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -406,6 +406,8 @@ object SparkSubmit { OptionAssigner(args.jars, YARN, CLUSTER, clOption = "--addJars"), // Other options + OptionAssigner(args.executorCores, STANDALONE, ALL_DEPLOY_MODES, + sysProp = "spark.executor.cores"), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN, ALL_DEPLOY_MODES, sysProp = "spark.executor.memory"), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS, ALL_DEPLOY_MODES, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 03ecf3fd99ec5..faa8780288ea3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -482,10 +482,13 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | Spark standalone and Mesos only: | --total-executor-cores NUM Total cores for all executors. | + | Spark standalone and YARN only: + | --executor-cores NUM Number of cores per executor. (Default: 1 in YARN mode, + | or all available cores on the worker in standalone mode) + | | YARN-only: | --driver-cores NUM Number of cores used by the driver, only in cluster mode | (Default: 1). - | --executor-cores NUM Number of cores per executor (Default: 1). | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). | --num-executors NUM Number of executors to launch (Default: 2). | --archives ARCHIVES Comma separated list of archives to be extracted into the diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index bc5b293379f2b..f59d550d4f3b3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -75,9 +75,11 @@ private[deploy] class ApplicationInfo( } } - private[master] def addExecutor(worker: WorkerInfo, cores: Int, useID: Option[Int] = None): - ExecutorDesc = { - val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerSlave) + private[master] def addExecutor( + worker: WorkerInfo, + cores: Int, + useID: Option[Int] = None): ExecutorDesc = { + val exec = new ExecutorDesc(newExecutorId(useID), this, worker, cores, desc.memoryPerExecutorMB) executors(exec.id) = exec coresGranted += cores exec 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 9a5d5877da86d..c5a6b1beac9be 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 @@ -524,52 +524,28 @@ private[master] class Master( } /** - * Can an app use the given worker? True if the worker has enough memory and we haven't already - * launched an executor for the app on it (right now the standalone backend doesn't like having - * two executors on the same worker). - */ - private def canUse(app: ApplicationInfo, worker: WorkerInfo): Boolean = { - worker.memoryFree >= app.desc.memoryPerSlave && !worker.hasExecutor(app) - } - - /** - * Schedule the currently available resources among waiting apps. This method will be called - * every time a new app joins or resource availability changes. + * Schedule executors to be launched on the workers. + * + * There are two modes of launching executors. The first attempts to spread out an application's + * executors on as many workers as possible, while the second does the opposite (i.e. launch them + * on as few workers as possible). The former is usually better for data locality purposes and is + * the default. + * + * The number of cores assigned to each executor is configurable. When this is explicitly set, + * multiple executors from the same application may be launched on the same worker if the worker + * has enough cores and memory. Otherwise, each executor grabs all the cores available on the + * worker by default, in which case only one executor may be launched on each worker. */ - private def schedule() { - if (state != RecoveryState.ALIVE) { return } - - // First schedule drivers, they take strict precedence over applications - // Randomization helps balance drivers - val shuffledAliveWorkers = Random.shuffle(workers.toSeq.filter(_.state == WorkerState.ALIVE)) - val numWorkersAlive = shuffledAliveWorkers.size - var curPos = 0 - - for (driver <- waitingDrivers.toList) { // iterate over a copy of waitingDrivers - // We assign workers to each waiting driver in a round-robin fashion. For each driver, we - // start from the last worker that was assigned a driver, and continue onwards until we have - // explored all alive workers. - var launched = false - var numWorkersVisited = 0 - while (numWorkersVisited < numWorkersAlive && !launched) { - val worker = shuffledAliveWorkers(curPos) - numWorkersVisited += 1 - if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { - launchDriver(worker, driver) - waitingDrivers -= driver - launched = true - } - curPos = (curPos + 1) % numWorkersAlive - } - } - + private def startExecutorsOnWorkers(): Unit = { // Right now this is a very simple FIFO scheduler. We keep trying to fit in the first app // in the queue, then the second app, etc. if (spreadOutApps) { - // Try to spread out each app among all the nodes, until it has all its cores + // Try to spread out each app among all the workers, until it has all its cores for (app <- waitingApps if app.coresLeft > 0) { val usableWorkers = workers.toArray.filter(_.state == WorkerState.ALIVE) - .filter(canUse(app, _)).sortBy(_.coresFree).reverse + .filter(worker => worker.memoryFree >= app.desc.memoryPerExecutorMB && + worker.coresFree >= app.desc.coresPerExecutor.getOrElse(1)) + .sortBy(_.coresFree).reverse val numUsable = usableWorkers.length val assigned = new Array[Int](numUsable) // Number of cores to give on each node var toAssign = math.min(app.coresLeft, usableWorkers.map(_.coresFree).sum) @@ -582,32 +558,61 @@ private[master] class Master( pos = (pos + 1) % numUsable } // Now that we've decided how many cores to give on each node, let's actually give them - for (pos <- 0 until numUsable) { - if (assigned(pos) > 0) { - val exec = app.addExecutor(usableWorkers(pos), assigned(pos)) - launchExecutor(usableWorkers(pos), exec) - app.state = ApplicationState.RUNNING - } + for (pos <- 0 until numUsable if assigned(pos) > 0) { + allocateWorkerResourceToExecutors(app, assigned(pos), usableWorkers(pos)) } } } else { - // Pack each app into as few nodes as possible until we've assigned all its cores + // Pack each app into as few workers as possible until we've assigned all its cores for (worker <- workers if worker.coresFree > 0 && worker.state == WorkerState.ALIVE) { for (app <- waitingApps if app.coresLeft > 0) { - if (canUse(app, worker)) { - val coresToUse = math.min(worker.coresFree, app.coresLeft) - if (coresToUse > 0) { - val exec = app.addExecutor(worker, coresToUse) - launchExecutor(worker, exec) - app.state = ApplicationState.RUNNING - } - } + allocateWorkerResourceToExecutors(app, app.coresLeft, worker) + } + } + } + } + + /** + * Allocate a worker's resources to one or more executors. + * @param app the info of the application which the executors belong to + * @param coresToAllocate cores on this worker to be allocated to this application + * @param worker the worker info + */ + private def allocateWorkerResourceToExecutors( + app: ApplicationInfo, + coresToAllocate: Int, + worker: WorkerInfo): Unit = { + val memoryPerExecutor = app.desc.memoryPerExecutorMB + val coresPerExecutor = app.desc.coresPerExecutor.getOrElse(coresToAllocate) + var coresLeft = coresToAllocate + while (coresLeft >= coresPerExecutor && worker.memoryFree >= memoryPerExecutor) { + val exec = app.addExecutor(worker, coresPerExecutor) + coresLeft -= coresPerExecutor + launchExecutor(worker, exec) + app.state = ApplicationState.RUNNING + } + } + + /** + * Schedule the currently available resources among waiting apps. This method will be called + * every time a new app joins or resource availability changes. + */ + private def schedule(): Unit = { + if (state != RecoveryState.ALIVE) { return } + // Drivers take strict precedence over executors + val shuffledWorkers = Random.shuffle(workers) // Randomization helps balance drivers + for (worker <- shuffledWorkers if worker.state == WorkerState.ALIVE) { + for (driver <- waitingDrivers) { + if (worker.memoryFree >= driver.desc.mem && worker.coresFree >= driver.desc.cores) { + launchDriver(worker, driver) + waitingDrivers -= driver } } } + startExecutorsOnWorkers() } - private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc) { + private def launchExecutor(worker: WorkerInfo, exec: ExecutorDesc): Unit = { logInfo("Launching executor " + exec.fullId + " on worker " + worker.id) worker.addExecutor(exec) worker.actor ! LaunchExecutor(masterUrl, diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 761aa8f7b1ef6..273f077bd8f57 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -94,7 +94,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app")

  • Executor Memory: - {Utils.megabytesToString(app.desc.memoryPerSlave)} + {Utils.megabytesToString(app.desc.memoryPerExecutorMB)}
  • Submit Date: {app.submitDate}
  • State: {app.state}
  • diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 45412a35e9a7d..399f07399a0aa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -208,8 +208,8 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { {app.coresGranted} - - {Utils.megabytesToString(app.desc.memoryPerSlave)} + + {Utils.megabytesToString(app.desc.memoryPerExecutorMB)} {UIUtils.formatDate(app.submitDate)} {app.desc.user} diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 7eb3fdc19b5b8..ed5b7c1088196 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -82,12 +82,11 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts) val appUIAddress = sc.ui.map(_.appUIAddress).getOrElse("") - val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - appUIAddress, sc.eventLogDir, sc.eventLogCodec) - + val coresPerExecutor = conf.getOption("spark.executor.cores").map(_.toInt) + val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, + command, appUIAddress, sc.eventLogDir, sc.eventLogCodec, coresPerExecutor) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() - waitForRegistration() } diff --git a/docs/configuration.md b/docs/configuration.md index 7169ec295ef7f..d9e9e67026cbb 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -723,6 +723,17 @@ Apart from these, the following properties are also available, and may be useful this duration will be cleared as well. + + spark.executor.cores + 1 in YARN mode, all the available cores on the worker in standalone mode. + + The number of cores to use on each executor. For YARN and standalone mode only. + + In standalone mode, setting this parameter allows an application to run multiple executors on + the same worker, provided that there are enough cores on that worker. Otherwise, only one + executor per application will run on each worker. + + spark.default.parallelism From b075e4b720221a8204cae93468065a6708348830 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 13:34:44 -0700 Subject: [PATCH 092/128] [SPARK-6700] [yarn] Re-enable flaky test. Test runs have been successful on jenkins. So let's re-enable the test and look out for any failures, and fix things appropriately. Author: Marcelo Vanzin Closes #5459 from vanzin/SPARK-6700 and squashes the following commits: 2ead85b [Marcelo Vanzin] WIP: re-enable flaky test to catch failure in jenkins. --- .../scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index c06c0105670c0..76952e3341cc4 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -144,7 +144,7 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } // Enable this once fix SPARK-6700 - ignore("run Python application in yarn-cluster mode") { + test("run Python application in yarn-cluster mode") { val primaryPyFile = new File(tempDir, "test.py") Files.write(TEST_PYFILE, primaryPyFile, UTF_8) val pyFile = new File(tempDir, "test2.py") From 6adb8bcbf0a1a7bfe2990de18c59c66cd7a0aeb8 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 14 Apr 2015 13:40:07 -0700 Subject: [PATCH 093/128] [SPARK-6905] Upgrade to snappy-java 1.1.1.7 We should upgrade our snappy-java dependency to 1.1.1.7 in order to include a fix for a bug that results in worse compression in SnappyOutputStream (see https://github.com/xerial/snappy-java/issues/100). Author: Josh Rosen Closes #5512 from JoshRosen/snappy-1.1.1.7 and squashes the following commits: f1ac0f8 [Josh Rosen] Upgrade to snappy-java 1.1.1.7. --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 0b8d664dbfe1b..261292d5b6cde 100644 --- a/pom.xml +++ b/pom.xml @@ -156,7 +156,7 @@ 3.6.3 1.8.8 2.4.4 - 1.1.1.6 + 1.1.1.7 1.1.2 ${java.home} From 65774370a1275e25cd8a3357e397d116767793a9 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 13:41:38 -0700 Subject: [PATCH 094/128] [SPARK-5808] [build] Package pyspark files in sbt assembly. This turned out to be more complicated than I wanted because the layout of python/ doesn't really follow the usual maven conventions. So some extra code is needed to copy just the right things. Author: Marcelo Vanzin Closes #5461 from vanzin/SPARK-5808 and squashes the following commits: 7153dac [Marcelo Vanzin] Only try to create resource dir if it doesn't already exist. ee90e84 [Marcelo Vanzin] [SPARK-5808] [build] Package pyspark files in sbt assembly. --- project/SparkBuild.scala | 60 +++++++++++++++++++++++++++++++++++++++- 1 file changed, 59 insertions(+), 1 deletion(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5f51f4b58f97a..09b4976d10c26 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -import java.io.File +import java.io._ import scala.util.Properties import scala.collection.JavaConversions._ @@ -166,6 +166,9 @@ object SparkBuild extends PomBuild { /* Enable Assembly for all assembly projects */ assemblyProjects.foreach(enable(Assembly.settings)) + /* Package pyspark artifacts in the main assembly. */ + enable(PySparkAssembly.settings)(assembly) + /* Enable unidoc only for the root spark project */ enable(Unidoc.settings)(spark) @@ -316,6 +319,7 @@ object Hive { } object Assembly { + import sbtassembly.AssemblyUtils._ import sbtassembly.Plugin._ import AssemblyKeys._ @@ -347,6 +351,60 @@ object Assembly { ) } +object PySparkAssembly { + import sbtassembly.Plugin._ + import AssemblyKeys._ + + lazy val settings = Seq( + unmanagedJars in Compile += { BuildCommons.sparkHome / "python/lib/py4j-0.8.2.1-src.zip" }, + // Use a resource generator to copy all .py files from python/pyspark into a managed directory + // to be included in the assembly. We can't just add "python/" to the assembly's resource dir + // list since that will copy unneeded / unwanted files. + resourceGenerators in Compile <+= resourceManaged in Compile map { outDir: File => + val dst = new File(outDir, "pyspark") + if (!dst.isDirectory()) { + require(dst.mkdirs()) + } + + val src = new File(BuildCommons.sparkHome, "python/pyspark") + copy(src, dst) + } + ) + + private def copy(src: File, dst: File): Seq[File] = { + src.listFiles().flatMap { f => + val child = new File(dst, f.getName()) + if (f.isDirectory()) { + child.mkdir() + copy(f, child) + } else if (f.getName().endsWith(".py")) { + var in: Option[FileInputStream] = None + var out: Option[FileOutputStream] = None + try { + in = Some(new FileInputStream(f)) + out = Some(new FileOutputStream(child)) + + val bytes = new Array[Byte](1024) + var read = 0 + while (read >= 0) { + read = in.get.read(bytes) + if (read > 0) { + out.get.write(bytes, 0, read) + } + } + + Some(child) + } finally { + in.foreach(_.close()) + out.foreach(_.close()) + } + } else { + None + } + } + } +} + object Unidoc { import BuildCommons._ From 4d4b24927417b2c17810e94d6d46c37491c68869 Mon Sep 17 00:00:00 2001 From: Kousuke Saruta Date: Tue, 14 Apr 2015 14:00:49 -0700 Subject: [PATCH 095/128] [SPARK-6769][YARN][TEST] Usage of the ListenerBus in YarnClusterSuite is wrong In YarnClusterSuite, a test case uses `SaveExecutorInfo` to handle ExecutorAddedEvent as follows. ``` private class SaveExecutorInfo extends SparkListener { val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() override def onExecutorAdded(executor: SparkListenerExecutorAdded) { addedExecutorInfos(executor.executorId) = executor.executorInfo } } ... listener = new SaveExecutorInfo val sc = new SparkContext(new SparkConf() .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) sc.addSparkListener(listener) val status = new File(args(0)) var result = "failure" try { val data = sc.parallelize(1 to 4, 4).collect().toSet assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) data should be (Set(1, 2, 3, 4)) result = "success" } finally { sc.stop() Files.write(result, status, UTF_8) } ``` But, the usage is wrong because Executors will spawn during initializing SparkContext and SparkContext#addSparkListener should be invoked after the initialization, thus after Executors spawn, so SaveExecutorInfo cannot handle ExecutorAddedEvent. Following code refers the result of the handling ExecutorAddedEvent. Because of the reason above, we cannot reach the assertion. ``` // verify log urls are present listener.addedExecutorInfos.values.foreach { info => assert(info.logUrlMap.nonEmpty) } ``` Author: Kousuke Saruta Closes #5417 from sarutak/SPARK-6769 and squashes the following commits: 8adc8ba [Kousuke Saruta] Fixed compile error e258530 [Kousuke Saruta] Fixed style 591cf3e [Kousuke Saruta] Fixed style 48ec89a [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-6769 860c965 [Kousuke Saruta] Simplified code 207d325 [Kousuke Saruta] Added findListenersByClass method to ListenerBus 2408c84 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-6769 2d7e409 [Kousuke Saruta] Merge branch 'master' of git://git.apache.org/spark into SPARK-6769 3874adf [Kousuke Saruta] Fixed the usage of listener bus in LogUrlsStandaloneSuite 153a91b [Kousuke Saruta] Fixed the usage of listener bus in YarnClusterSuite --- .../org/apache/spark/util/ListenerBus.scala | 8 ++++++++ .../spark/deploy/LogUrlsStandaloneSuite.scala | 20 ++++++++++--------- .../spark/deploy/yarn/YarnClusterSuite.scala | 17 +++++++++------- 3 files changed, 29 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala index d60b8b9a31a9b..a725767d08cc2 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -19,9 +19,12 @@ package org.apache.spark.util import java.util.concurrent.CopyOnWriteArrayList +import scala.collection.JavaConversions._ +import scala.reflect.ClassTag import scala.util.control.NonFatal import org.apache.spark.Logging +import org.apache.spark.scheduler.SparkListener /** * An event bus which posts events to its listeners. @@ -64,4 +67,9 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { */ def onPostEvent(listener: L, event: E): Unit + private[spark] def findListenersByClass[T <: L : ClassTag](): Seq[T] = { + val c = implicitly[ClassTag[T]].runtimeClass + listeners.filter(_.getClass == c).map(_.asInstanceOf[T]).toSeq + } + } diff --git a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala index 9cdb42814ca32..c93d16f8a1586 100644 --- a/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/LogUrlsStandaloneSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy import java.net.URL +import scala.collection.JavaConversions._ import scala.collection.mutable import scala.io.Source @@ -65,16 +66,17 @@ class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { new MySparkConf().setAll(getAll) } } - val conf = new MySparkConf() + val conf = new MySparkConf().set( + "spark.extraListeners", classOf[SaveExecutorInfo].getName) sc = new SparkContext("local-cluster[2,1,512]", "test", conf) - val listener = new SaveExecutorInfo - sc.addSparkListener(listener) - // Trigger a job so that executors get added sc.parallelize(1 to 100, 4).map(_.toString).count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] + assert(listeners.size === 1) + val listener = listeners(0) listener.addedExecutorInfos.values.foreach { info => assert(info.logUrlMap.nonEmpty) info.logUrlMap.values.foreach { logUrl => @@ -82,12 +84,12 @@ class LogUrlsStandaloneSuite extends FunSuite with LocalSparkContext { } } } +} - private class SaveExecutorInfo extends SparkListener { - val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() +private[spark] class SaveExecutorInfo extends SparkListener { + val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() - override def onExecutorAdded(executor: SparkListenerExecutorAdded) { - addedExecutorInfos(executor.executorId) = executor.executorInfo - } + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { + addedExecutorInfos(executor.executorId) = executor.executorInfo } } diff --git a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 76952e3341cc4..a18c94d4ab4a8 100644 --- a/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -33,7 +33,7 @@ import org.scalatest.{BeforeAndAfterAll, FunSuite, Matchers} import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException, TestUtils} import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorAdded} +import org.apache.spark.scheduler.{SparkListenerJobStart, SparkListener, SparkListenerExecutorAdded} import org.apache.spark.util.Utils /** @@ -282,10 +282,10 @@ class YarnClusterSuite extends FunSuite with BeforeAndAfterAll with Matchers wit } -private class SaveExecutorInfo extends SparkListener { +private[spark] class SaveExecutorInfo extends SparkListener { val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() - override def onExecutorAdded(executor : SparkListenerExecutorAdded) { + override def onExecutorAdded(executor: SparkListenerExecutorAdded) { addedExecutorInfos(executor.executorId) = executor.executorInfo } } @@ -293,7 +293,6 @@ private class SaveExecutorInfo extends SparkListener { private object YarnClusterDriver extends Logging with Matchers { val WAIT_TIMEOUT_MILLIS = 10000 - var listener: SaveExecutorInfo = null def main(args: Array[String]): Unit = { if (args.length != 1) { @@ -306,10 +305,9 @@ private object YarnClusterDriver extends Logging with Matchers { System.exit(1) } - listener = new SaveExecutorInfo val sc = new SparkContext(new SparkConf() + .set("spark.extraListeners", classOf[SaveExecutorInfo].getName) .setAppName("yarn \"test app\" 'with quotes' and \\back\\slashes and $dollarSigns")) - sc.addSparkListener(listener) val status = new File(args(0)) var result = "failure" try { @@ -323,7 +321,12 @@ private object YarnClusterDriver extends Logging with Matchers { } // verify log urls are present - listener.addedExecutorInfos.values.foreach { info => + val listeners = sc.listenerBus.findListenersByClass[SaveExecutorInfo] + assert(listeners.size === 1) + val listener = listeners(0) + val executorInfos = listener.addedExecutorInfos.values + assert(executorInfos.nonEmpty) + executorInfos.foreach { info => assert(info.logUrlMap.nonEmpty) } } From a76b921a923ac37d3c73ee18d24df4bb611daba3 Mon Sep 17 00:00:00 2001 From: Josh Rosen Date: Tue, 14 Apr 2015 14:07:25 -0700 Subject: [PATCH 096/128] Revert "[SPARK-6352] [SQL] Add DirectParquetOutputCommitter" This reverts commit b29663eeea440b1d1a288d41b5ddf67e77c5bd54. I'm reverting this because it broke test compilation for the Hadoop 1.x profiles. --- .../DirectParquetOutputCommitter.scala | 66 ------------------- .../sql/parquet/ParquetTableOperations.scala | 22 ------- .../spark/sql/parquet/ParquetIOSuite.scala | 21 ------ 3 files changed, 109 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala deleted file mode 100644 index 25a66cb488103..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/DirectParquetOutputCommitter.scala +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.parquet - -import org.apache.hadoop.fs.Path -import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} -import org.apache.hadoop.mapreduce.lib.output.FileOutputCommitter - -import parquet.Log -import parquet.hadoop.util.ContextUtil -import parquet.hadoop.{ParquetFileReader, ParquetFileWriter, ParquetOutputCommitter} - -private[parquet] class DirectParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) - extends ParquetOutputCommitter(outputPath, context) { - val LOG = Log.getLog(classOf[ParquetOutputCommitter]) - - override def getWorkPath(): Path = outputPath - override def abortTask(taskContext: TaskAttemptContext): Unit = {} - override def commitTask(taskContext: TaskAttemptContext): Unit = {} - override def needsTaskCommit(taskContext: TaskAttemptContext): Boolean = true - override def setupJob(jobContext: JobContext): Unit = {} - override def setupTask(taskContext: TaskAttemptContext): Unit = {} - - override def commitJob(jobContext: JobContext) { - try { - val configuration = ContextUtil.getConfiguration(jobContext) - val fileSystem = outputPath.getFileSystem(configuration) - val outputStatus = fileSystem.getFileStatus(outputPath) - val footers = ParquetFileReader.readAllFootersInParallel(configuration, outputStatus) - try { - ParquetFileWriter.writeMetadataFile(configuration, outputPath, footers) - if (configuration.getBoolean("mapreduce.fileoutputcommitter.marksuccessfuljobs", true)) { - val successPath = new Path(outputPath, FileOutputCommitter.SUCCEEDED_FILE_NAME) - fileSystem.create(successPath).close() - } - } catch { - case e: Exception => { - LOG.warn("could not write summary file for " + outputPath, e) - val metadataPath = new Path(outputPath, ParquetFileWriter.PARQUET_METADATA_FILE) - if (fileSystem.exists(metadataPath)) { - fileSystem.delete(metadataPath, true) - } - } - } - } catch { - case e: Exception => LOG.warn("could not write summary file for " + outputPath, e) - } - } - -} - 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 3724bda829d30..1c868da23e060 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 @@ -379,8 +379,6 @@ private[sql] case class InsertIntoParquetTable( */ private[parquet] class AppendingParquetOutputFormat(offset: Int) extends parquet.hadoop.ParquetOutputFormat[Row] { - var committer: OutputCommitter = null - // override to accept existing directories as valid output directory override def checkOutputSpecs(job: JobContext): Unit = {} @@ -405,26 +403,6 @@ private[parquet] class AppendingParquetOutputFormat(offset: Int) private def getTaskAttemptID(context: TaskAttemptContext): TaskAttemptID = { context.getClass.getMethod("getTaskAttemptID").invoke(context).asInstanceOf[TaskAttemptID] } - - // override to create output committer from configuration - override def getOutputCommitter(context: TaskAttemptContext): OutputCommitter = { - if (committer == null) { - val output = getOutputPath(context) - val cls = context.getConfiguration.getClass("spark.sql.parquet.output.committer.class", - classOf[ParquetOutputCommitter], classOf[ParquetOutputCommitter]) - val ctor = cls.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) - committer = ctor.newInstance(output, context).asInstanceOf[ParquetOutputCommitter] - } - committer - } - - // FileOutputFormat.getOutputPath takes JobConf in hadoop-1 but JobContext in hadoop-2 - private def getOutputPath(context: TaskAttemptContext): Path = { - context.getConfiguration().get("mapred.output.dir") match { - case null => null - case name => new Path(name) - } - } } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala index 4d0bf7cf99cdf..97c0f439acf13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetIOSuite.scala @@ -381,27 +381,6 @@ class ParquetIOSuiteBase extends QueryTest with ParquetTest { } } } - - test("SPARK-6352 DirectParquetOutputCommitter") { - // Write to a parquet file and let it fail. - // _temporary should be missing if direct output committer works. - try { - configuration.set("spark.sql.parquet.output.committer.class", - "org.apache.spark.sql.parquet.DirectParquetOutputCommitter") - sqlContext.udf.register("div0", (x: Int) => x / 0) - withTempPath { dir => - intercept[org.apache.spark.SparkException] { - sqlContext.sql("select div0(1)").saveAsParquetFile(dir.getCanonicalPath) - } - val path = new Path(dir.getCanonicalPath, "_temporary") - val fs = path.getFileSystem(configuration) - assert(!fs.exists(path)) - } - } - finally { - configuration.unset("spark.sql.parquet.output.committer.class") - } - } } class ParquetDataSourceOnIOSuite extends ParquetIOSuiteBase with BeforeAndAfterAll { From 6de282e2de3cb69f9b746d03fde581429248824a Mon Sep 17 00:00:00 2001 From: zsxwing Date: Tue, 14 Apr 2015 16:51:36 -0700 Subject: [PATCH 097/128] [SPARK-6796][Streaming][WebUI] Add "Active Batches" and "Completed Batches" lists to StreamingPage This PR adds two lists, `Active Batches` and `Completed Batches`. Here is the screenshot: ![batch_list](https://cloud.githubusercontent.com/assets/1000778/7060458/d8898572-deb3-11e4-938b-6f8602c71a9f.png) Due to [SPARK-6766](https://issues.apache.org/jira/browse/SPARK-6766), I need to merge #5414 in my local machine to get the above screenshot. Author: zsxwing Closes #5434 from zsxwing/SPARK-6796 and squashes the following commits: be50fc6 [zsxwing] Fix the code style 51b792e [zsxwing] Fix the unit test 6f3078e [zsxwing] Make 'startTime' readable f40e0a9 [zsxwing] Merge branch 'master' into SPARK-6796 2525336 [zsxwing] Rename 'Processed batches' and 'Waiting batches' and also add links a69c091 [zsxwing] Show the number of total completed batches too a12ad7b [zsxwing] Change 'records' to 'events' in the UI 86b5e7f [zsxwing] Make BatchTableBase abstract b248787 [zsxwing] Add tests to verify the new tables d18ab7d [zsxwing] Fix the code style 6ceffb3 [zsxwing] Add "Active Batches" and "Completed Batches" lists to StreamingPage --- .../spark/streaming/ui/AllBatchesTable.scala | 114 ++++++++++++++++++ .../spark/streaming/ui/StreamingPage.scala | 44 +++++-- .../spark/streaming/UISeleniumSuite.scala | 11 ++ 3 files changed, 159 insertions(+), 10 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala new file mode 100644 index 0000000000000..df1c0a10704c3 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/AllBatchesTable.scala @@ -0,0 +1,114 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.streaming.ui + +import scala.xml.Node + +import org.apache.spark.streaming.scheduler.BatchInfo +import org.apache.spark.ui.UIUtils + +private[ui] abstract class BatchTableBase(tableId: String) { + + protected def columns: Seq[Node] = { + Batch Time + Input Size + Scheduling Delay + Processing Time + } + + protected def baseRow(batch: BatchInfo): Seq[Node] = { + val batchTime = batch.batchTime.milliseconds + val formattedBatchTime = UIUtils.formatDate(batch.batchTime.milliseconds) + val eventCount = batch.receivedBlockInfo.values.map { + receivers => receivers.map(_.numRecords).sum + }.sum + val schedulingDelay = batch.schedulingDelay + val formattedSchedulingDelay = schedulingDelay.map(UIUtils.formatDuration).getOrElse("-") + val processingTime = batch.processingDelay + val formattedProcessingTime = processingTime.map(UIUtils.formatDuration).getOrElse("-") + + {formattedBatchTime} + {eventCount.toString} events + + {formattedSchedulingDelay} + + + {formattedProcessingTime} + + } + + private def batchTable: Seq[Node] = { + + + {columns} + + + {renderRows} + +
    + } + + def toNodeSeq: Seq[Node] = { + batchTable + } + + /** + * Return HTML for all rows of this table. + */ + protected def renderRows: Seq[Node] +} + +private[ui] class ActiveBatchTable(runningBatches: Seq[BatchInfo], waitingBatches: Seq[BatchInfo]) + extends BatchTableBase("active-batches-table") { + + override protected def columns: Seq[Node] = super.columns ++ Status + + override protected def renderRows: Seq[Node] = { + // The "batchTime"s of "waitingBatches" must be greater than "runningBatches"'s, so display + // waiting batches before running batches + waitingBatches.flatMap(batch => {waitingBatchRow(batch)}) ++ + runningBatches.flatMap(batch => {runningBatchRow(batch)}) + } + + private def runningBatchRow(batch: BatchInfo): Seq[Node] = { + baseRow(batch) ++ processing + } + + private def waitingBatchRow(batch: BatchInfo): Seq[Node] = { + baseRow(batch) ++ queued + } +} + +private[ui] class CompletedBatchTable(batches: Seq[BatchInfo]) + extends BatchTableBase("completed-batches-table") { + + override protected def columns: Seq[Node] = super.columns ++ Total Delay + + override protected def renderRows: Seq[Node] = { + batches.flatMap(batch => {completedBatchRow(batch)}) + } + + private def completedBatchRow(batch: BatchInfo): Seq[Node] = { + val totalDelay = batch.totalDelay + val formattedTotalDelay = totalDelay.map(UIUtils.formatDuration).getOrElse("-") + baseRow(batch) ++ + + {formattedTotalDelay} + + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index b6dcb62bfeec8..07fa285642eec 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -41,7 +41,8 @@ private[ui] class StreamingPage(parent: StreamingTab) generateBasicStats() ++

    ++

    Statistics over last {listener.retainedCompletedBatches.size} processed batches

    ++ generateReceiverStats() ++ - generateBatchStatsTable() + generateBatchStatsTable() ++ + generateBatchListTables() } UIUtils.headerSparkPage("Streaming", content, parent, Some(5000)) } @@ -49,9 +50,10 @@ private[ui] class StreamingPage(parent: StreamingTab) /** Generate basic stats of the streaming program */ private def generateBasicStats(): Seq[Node] = { val timeSinceStart = System.currentTimeMillis() - startTime + // scalastyle:off
    + // scalastyle:on } /** Generate stats of data received by the receivers in the streaming program */ @@ -86,10 +89,10 @@ private[ui] class StreamingPage(parent: StreamingTab) "Receiver", "Status", "Location", - "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", - "Minimum rate\n[records/sec]", - "Median rate\n[records/sec]", - "Maximum rate\n[records/sec]", + "Events in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", + "Minimum rate\n[events/sec]", + "Median rate\n[events/sec]", + "Maximum rate\n[events/sec]", "Last Error" ) val dataRows = (0 until listener.numReceivers).map { receiverId => @@ -190,5 +193,26 @@ private[ui] class StreamingPage(parent: StreamingTab) } UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) } + + private def generateBatchListTables(): Seq[Node] = { + val runningBatches = listener.runningBatches.sortBy(_.batchTime.milliseconds).reverse + val waitingBatches = listener.waitingBatches.sortBy(_.batchTime.milliseconds).reverse + val completedBatches = listener.retainedCompletedBatches. + sortBy(_.batchTime.milliseconds).reverse + + val activeBatchesContent = { +

    Active Batches ({runningBatches.size + waitingBatches.size})

    ++ + new ActiveBatchTable(runningBatches, waitingBatches).toNodeSeq + } + + val completedBatchesContent = { +

    + Completed Batches (last {completedBatches.size} out of {listener.numTotalCompletedBatches}) +

    ++ + new CompletedBatchTable(completedBatches).toNodeSeq + } + + activeBatchesContent ++ completedBatchesContent + } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 998426ebb82e5..205ddf6dbe9b0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -75,6 +75,17 @@ class UISeleniumSuite val statisticText = findAll(cssSelector("li strong")).map(_.text).toSeq statisticText should contain("Network receivers:") statisticText should contain("Batch interval:") + + val h4Text = findAll(cssSelector("h4")).map(_.text).toSeq + h4Text should contain("Active Batches (0)") + h4Text should contain("Completed Batches (last 0 out of 0)") + + findAll(cssSelector("""#active-batches-table th""")).map(_.text).toSeq should be { + List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Status") + } + findAll(cssSelector("""#completed-batches-table th""")).map(_.text).toSeq should be { + List("Batch Time", "Input Size", "Scheduling Delay", "Processing Time", "Total Delay") + } } ssc.stop(false) From 9717389365772d218cd7c67f9a13c3440f3c6791 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 18:51:39 -0700 Subject: [PATCH 098/128] [SPARK-6890] [core] Fix launcher lib work with SPARK_PREPEND_CLASSES. The fix for SPARK-6406 broke the case where sub-processes are launched when SPARK_PREPEND_CLASSES is set, because the code now would only add the launcher's build directory to the sub-process's classpath instead of the complete assembly. This patch fixes the problem by having the launch scripts stash the assembly's location in an environment variable. This is not the prettiest solution, but it avoids having to plumb that location all the way through the Worker code that launches executors. The env variable is always set by the launch scripts, so users cannot override it. Author: Marcelo Vanzin Closes #5504 from vanzin/SPARK-6890 and squashes the following commits: 7aec921 [Marcelo Vanzin] Fix tests. ff87a60 [Marcelo Vanzin] Merge branch 'master' into SPARK-6890 31d3ce8 [Marcelo Vanzin] [SPARK-6890] [core] Fix launcher lib work with SPARK_PREPEND_CLASSES. --- bin/spark-class | 11 ++++- bin/spark-class2.cmd | 11 ++++- .../launcher/AbstractCommandBuilder.java | 44 +++++++++++++++++-- .../spark/launcher/CommandBuilderUtils.java | 1 + .../SparkSubmitCommandBuilderSuite.java | 15 ++++--- 5 files changed, 71 insertions(+), 11 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index c03946d92e2e4..c49d97ce5cf25 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -82,13 +82,22 @@ if [ $(command -v "$JAR_CMD") ] ; then fi fi +LAUNCH_CLASSPATH="$SPARK_ASSEMBLY_JAR" + +# Add the launcher build dir to the classpath if requested. +if [ -n "$SPARK_PREPEND_CLASSES" ]; then + LAUNCH_CLASSPATH="$SPARK_HOME/launcher/target/scala-$SPARK_SCALA_VERSION/classes:$LAUNCH_CLASSPATH" +fi + +export _SPARK_ASSEMBLY="$SPARK_ASSEMBLY_JAR" + # The launcher library will print arguments separated by a NULL character, to allow arguments with # characters that would be otherwise interpreted by the shell. Read that in a while loop, populating # an array that will be used to exec the final command. CMD=() while IFS= read -d '' -r ARG; do CMD+=("$ARG") -done < <("$RUNNER" -cp "$SPARK_ASSEMBLY_JAR" org.apache.spark.launcher.Main "$@") +done < <("$RUNNER" -cp "$LAUNCH_CLASSPATH" org.apache.spark.launcher.Main "$@") if [ "${CMD[0]}" = "usage" ]; then "${CMD[@]}" diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 4b3401d745f2a..3d068dd3a2739 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -46,13 +46,22 @@ if "%SPARK_ASSEMBLY_JAR%"=="0" ( exit /b 1 ) +set LAUNCH_CLASSPATH=%SPARK_ASSEMBLY_JAR% + +rem Add the launcher build dir to the classpath if requested. +if not "x%SPARK_PREPEND_CLASSES%"=="x" ( + set LAUNCH_CLASSPATH=%SPARK_HOME%\launcher\target\scala-%SPARK_SCALA_VERSION%\classes;%LAUNCH_CLASSPATH% +) + +set _SPARK_ASSEMBLY=%SPARK_ASSEMBLY_JAR% + rem Figure out where java is. set RUNNER=java if not "x%JAVA_HOME%"=="x" set RUNNER=%JAVA_HOME%\bin\java rem The launcher library prints the command to be executed in a single line suitable for being rem executed by the batch interpreter. So read all the output of the launcher into a variable. -for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %SPARK_ASSEMBLY_JAR% org.apache.spark.launcher.Main %*"') do ( +for /f "tokens=*" %%i in ('cmd /C ""%RUNNER%" -cp %LAUNCH_CLASSPATH% org.apache.spark.launcher.Main %*"') do ( set SPARK_CMD=%%i ) %SPARK_CMD% diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index d8279145d8e90..b8f02b961113d 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -186,12 +186,24 @@ List buildClassPath(String appClassPath) throws IOException { addToClassPath(cp, String.format("%s/core/target/jars/*", sparkHome)); } - final String assembly = AbstractCommandBuilder.class.getProtectionDomain().getCodeSource(). - getLocation().getPath(); + // We can't rely on the ENV_SPARK_ASSEMBLY variable to be set. Certain situations, such as + // when running unit tests, or user code that embeds Spark and creates a SparkContext + // with a local or local-cluster master, will cause this code to be called from an + // environment where that env variable is not guaranteed to exist. + // + // For the testing case, we rely on the test code to set and propagate the test classpath + // appropriately. + // + // For the user code case, we fall back to looking for the Spark assembly under SPARK_HOME. + // That duplicates some of the code in the shell scripts that look for the assembly, though. + String assembly = getenv(ENV_SPARK_ASSEMBLY); + if (assembly == null && isEmpty(getenv("SPARK_TESTING"))) { + assembly = findAssembly(); + } addToClassPath(cp, assembly); - // Datanucleus jars must be included on the classpath. Datanucleus jars do not work if only - // included in the uber jar as plugin.xml metadata is lost. Both sbt and maven will populate + // Datanucleus jars must be included on the classpath. Datanucleus jars do not work if only + // included in the uber jar as plugin.xml metadata is lost. Both sbt and maven will populate // "lib_managed/jars/" with the datanucleus jars when Spark is built with Hive File libdir; if (new File(sparkHome, "RELEASE").isFile()) { @@ -299,6 +311,30 @@ String getenv(String key) { return firstNonEmpty(childEnv.get(key), System.getenv(key)); } + private String findAssembly() { + String sparkHome = getSparkHome(); + File libdir; + if (new File(sparkHome, "RELEASE").isFile()) { + libdir = new File(sparkHome, "lib"); + checkState(libdir.isDirectory(), "Library directory '%s' does not exist.", + libdir.getAbsolutePath()); + } else { + libdir = new File(sparkHome, String.format("assembly/target/scala-%s", getScalaVersion())); + } + + final Pattern re = Pattern.compile("spark-assembly.*hadoop.*\\.jar"); + FileFilter filter = new FileFilter() { + @Override + public boolean accept(File file) { + return file.isFile() && re.matcher(file.getName()).matches(); + } + }; + File[] assemblies = libdir.listFiles(filter); + checkState(assemblies != null && assemblies.length > 0, "No assemblies found in '%s'.", libdir); + checkState(assemblies.length == 1, "Multiple assemblies found in '%s'.", libdir); + return assemblies[0].getAbsolutePath(); + } + private String getConfDir() { String confDir = getenv("SPARK_CONF_DIR"); return confDir != null ? confDir : join(File.separator, getSparkHome(), "conf"); diff --git a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java index f4ebc25bdd32b..8028e42ffb483 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java +++ b/launcher/src/main/java/org/apache/spark/launcher/CommandBuilderUtils.java @@ -30,6 +30,7 @@ class CommandBuilderUtils { static final String DEFAULT_MEM = "512m"; static final String DEFAULT_PROPERTIES_FILE = "spark-defaults.conf"; static final String ENV_SPARK_HOME = "SPARK_HOME"; + static final String ENV_SPARK_ASSEMBLY = "_SPARK_ASSEMBLY"; /** Returns whether the given string is null or empty. */ static boolean isEmpty(String s) { diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 626116a9e7477..97043a76cc612 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -98,7 +98,7 @@ public void testShellCliParser() throws Exception { parser.NAME, "appName"); - List args = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); + List args = newCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); List expected = Arrays.asList("spark-shell", "--app-arg", "bar", "--app-switch"); assertEquals(expected, args.subList(args.size() - expected.size(), args.size())); } @@ -110,7 +110,7 @@ public void testAlternateSyntaxParsing() throws Exception { parser.MASTER + "=foo", parser.DEPLOY_MODE + "=bar"); - List cmd = new SparkSubmitCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); + List cmd = newCommandBuilder(sparkSubmitArgs).buildSparkSubmitArgs(); assertEquals("org.my.Class", findArgValue(cmd, parser.CLASS)); assertEquals("foo", findArgValue(cmd, parser.MASTER)); assertEquals("bar", findArgValue(cmd, parser.DEPLOY_MODE)); @@ -153,7 +153,7 @@ private void testCmdBuilder(boolean isDriver) throws Exception { String deployMode = isDriver ? "client" : "cluster"; SparkSubmitCommandBuilder launcher = - new SparkSubmitCommandBuilder(Collections.emptyList()); + newCommandBuilder(Collections.emptyList()); launcher.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); launcher.master = "yarn"; @@ -273,10 +273,15 @@ private boolean findInStringList(String list, String sep, String needle) { return contains(needle, list.split(sep)); } - private List buildCommand(List args, Map env) throws Exception { + private SparkSubmitCommandBuilder newCommandBuilder(List args) { SparkSubmitCommandBuilder builder = new SparkSubmitCommandBuilder(args); builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_HOME, System.getProperty("spark.test.home")); - return builder.buildCommand(env); + builder.childEnv.put(CommandBuilderUtils.ENV_SPARK_ASSEMBLY, "dummy"); + return builder; + } + + private List buildCommand(List args, Map env) throws Exception { + return newCommandBuilder(args).buildCommand(env); } } From 30a6e0dcc0bd298731c1387546779cddcc16bc72 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 14 Apr 2015 18:52:48 -0700 Subject: [PATCH 099/128] [SPARK-5634] [core] Show correct message in HS when no incomplete apps f... ...ound. Author: Marcelo Vanzin Closes #5515 from vanzin/SPARK-5634 and squashes the following commits: f74ecf1 [Marcelo Vanzin] [SPARK-5634] [core] Show correct message in HS when no incomplete apps found. --- .../scala/org/apache/spark/deploy/history/HistoryPage.scala | 2 ++ 1 file changed, 2 insertions(+) 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 6e432d63c6b5a..3781b4e8c12bd 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 @@ -90,6 +90,8 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") ++ appTable + } else if (requestedIncomplete) { +

    No incomplete applications found!

    } else {

    No completed applications found!

    ++

    Did you specify the correct logging directory? From 6be918942c4078692d169d72fa9c358f6e98e85e Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Tue, 14 Apr 2015 23:47:16 -0700 Subject: [PATCH 100/128] [SPARK-6871][SQL] WITH clause in CTE can not following another WITH clause JIRA https://issues.apache.org/jira/browse/SPARK-6871 Author: Liang-Chi Hsieh Closes #5480 from viirya/no_cte_after_cte and squashes the following commits: 4da3712 [Liang-Chi Hsieh] Create new test. 40b38ed [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into no_cte_after_cte 0edf568 [Liang-Chi Hsieh] for comments. 6591b79 [Liang-Chi Hsieh] WITH clause in CTE can not following another WITH clause. --- .../apache/spark/sql/catalyst/SqlParser.scala | 18 +++++++++--------- .../org/apache/spark/sql/SQLQuerySuite.scala | 6 ++++++ 2 files changed, 15 insertions(+), 9 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 bc8d3751f6616..9a3531ceb3343 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -121,14 +121,14 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { } protected lazy val start: Parser[LogicalPlan] = - ( (select | ("(" ~> select <~ ")")) * - ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } - | INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } - | EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} - | UNION ~ DISTINCT.? ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } - ) - | insert - | cte + start1 | insert | cte + + protected lazy val start1: Parser[LogicalPlan] = + (select | ("(" ~> select <~ ")")) * + ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } + | INTERSECT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Intersect(q1, q2) } + | EXCEPT ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Except(q1, q2)} + | UNION ~ DISTINCT.? ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } ) protected lazy val select: Parser[LogicalPlan] = @@ -159,7 +159,7 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { } protected lazy val cte: Parser[LogicalPlan] = - WITH ~> rep1sep(ident ~ ( AS ~ "(" ~> start <~ ")"), ",") ~ start ^^ { + WITH ~> rep1sep(ident ~ ( AS ~ "(" ~> start1 <~ ")"), ",") ~ (start1 | insert) ^^ { case r ~ s => With(s, r.map({case n ~ s => (n, Subquery(n, s))}).toMap) } 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 73fb791c3ead7..0174aaee94246 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 @@ -431,6 +431,12 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { } + test("Allow only a single WITH clause per query") { + intercept[RuntimeException] { + sql("with q1 as (select * from testData) with q2 as (select * from q1) select * from q2") + } + } + test("date row") { checkAnswer(sql( """select cast("2015-01-28" as date) from testData limit 1"""), From 29aabdd6c20197adb16706823a8c7f48a0074352 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 15 Apr 2015 10:23:53 +0100 Subject: [PATCH 101/128] [HOTFIX] [SPARK-6896] [SQL] fix compile error in hive-thriftserver SPARK-6440 #5424 import guava but did not promote guava dependency to compile level. [INFO] compiler plugin: BasicArtifact(org.scalamacros,paradise_2.10.4,2.0.1,null) [info] Compiling 8 Scala sources to /root/projects/spark/sql/hive-thriftserver/target/scala-2.10/classes... [error] bad symbolic reference. A signature in Utils.class refers to term util [error] in package com.google.common which is not available. [error] It may be completely missing from the current classpath, or the version on [error] the classpath might be incompatible with the version used when compiling Utils.class. [error] [error] while compiling: /root/projects/spark/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala [error] during phase: erasure [error] library version: version 2.10.4 [error] compiler version: version 2.10.4 [error] reconstructed args: -deprecation -classpath Author: Daoyuan Wang Closes #5507 from adrian-wang/guava and squashes the following commits: c337dad [Daoyuan Wang] fix compile error --- sql/hive-thriftserver/pom.xml | 1 - 1 file changed, 1 deletion(-) diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index a96b1ffc26966..f38c796241df1 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -44,7 +44,6 @@ com.google.guava guava - runtime ${hive.group} From 6c5ed8a6d552abd967d27cdb94b68d46ccb57221 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 15 Apr 2015 15:17:58 +0100 Subject: [PATCH 102/128] SPARK-6861 [BUILD] Scalastyle config prevents building Maven child modules alone Move scalastyle-config.xml to dev/ (SBT config still doesn't work) to fix running mvn targets from subdirs; make scalastyle a verify stage target again in Maven; output results in target not project root; update to scalastyle 0.7.0 Author: Sean Owen Closes #5471 from srowen/SPARK-6861 and squashes the following commits: acac637 [Sean Owen] Oops, add back execution but leave it at the default verify phase 35a4fd2 [Sean Owen] Revert change to scalastyle-config.xml location, but return scalastyle Maven check to verify phase instead of package to get it farther out of the way, since the Maven invocation is optional c4fb42c [Sean Owen] Move scalastyle-config.xml to dev/ (SBT config still doesn't work) to fix running mvn targets from subdirs; make scalastyle a verify stage target again in Maven; output results in target not project root; update to scalastyle 0.7.0 --- pom.xml | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/pom.xml b/pom.xml index 261292d5b6cde..bcc2f57f1af5d 100644 --- a/pom.xml +++ b/pom.xml @@ -1447,7 +1447,7 @@ org.scalastyle scalastyle-maven-plugin - 0.4.0 + 0.7.0 false true @@ -1456,13 +1456,12 @@ ${basedir}/src/main/scala ${basedir}/src/test/scala scalastyle-config.xml - scalastyle-output.xml + ${basedir}/target/scalastyle-output.xml ${project.build.sourceEncoding} ${project.reporting.outputEncoding} - package check From f11288d5272bc18585b8cad4ee3bd59eade7c296 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 15 Apr 2015 12:58:02 -0700 Subject: [PATCH 103/128] [SPARK-6886] [PySpark] fix big closure with shuffle Currently, the created broadcast object will have same life cycle as RDD in Python. For multistage jobs, an PythonRDD will be created in JVM and the RDD in Python may be GCed, then the broadcast will be destroyed in JVM before the PythonRDD. This PR change to use PythonRDD to track the lifecycle of the broadcast object. It also have a refactor about getNumPartitions() to avoid unnecessary creation of PythonRDD, which could be heavy. cc JoshRosen Author: Davies Liu Closes #5496 from davies/big_closure and squashes the following commits: 9a0ea4c [Davies Liu] fix big closure with shuffle --- python/pyspark/rdd.py | 15 +++++---------- python/pyspark/tests.py | 6 ++---- 2 files changed, 7 insertions(+), 14 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index c9ac95d117574..93e658eded9e2 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1197,7 +1197,7 @@ def take(self, num): [91, 92, 93] """ items = [] - totalParts = self._jrdd.partitions().size() + totalParts = self.getNumPartitions() partsScanned = 0 while len(items) < num and partsScanned < totalParts: @@ -1260,7 +1260,7 @@ def isEmpty(self): >>> sc.parallelize([1]).isEmpty() False """ - return self._jrdd.partitions().size() == 0 or len(self.take(1)) == 0 + return self.getNumPartitions() == 0 or len(self.take(1)) == 0 def saveAsNewAPIHadoopDataset(self, conf, keyConverter=None, valueConverter=None): """ @@ -2235,11 +2235,9 @@ def _prepare_for_python_RDD(sc, command, obj=None): ser = CloudPickleSerializer() pickled_command = ser.dumps((command, sys.version_info[:2])) if len(pickled_command) > (1 << 20): # 1M + # The broadcast will have same life cycle as created PythonRDD broadcast = sc.broadcast(pickled_command) pickled_command = ser.dumps(broadcast) - # tracking the life cycle by obj - if obj is not None: - obj._broadcast = broadcast broadcast_vars = ListConverter().convert( [x._jbroadcast for x in sc._pickled_broadcast_vars], sc._gateway._gateway_client) @@ -2294,12 +2292,9 @@ def pipeline_func(split, iterator): self._jrdd_deserializer = self.ctx.serializer self._bypass_serializer = False self.partitioner = prev.partitioner if self.preservesPartitioning else None - self._broadcast = None - def __del__(self): - if self._broadcast: - self._broadcast.unpersist() - self._broadcast = None + def getNumPartitions(self): + return self._prev_jrdd.partitions().size() @property def _jrdd(self): diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index b938b9ce12395..ee67e80d539f8 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -550,10 +550,8 @@ def test_large_closure(self): data = [float(i) for i in xrange(N)] rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data)) self.assertEquals(N, rdd.first()) - self.assertTrue(rdd._broadcast is not None) - rdd = self.sc.parallelize(range(1), 1).map(lambda x: 1) - self.assertEqual(1, rdd.first()) - self.assertTrue(rdd._broadcast is None) + # regression test for SPARK-6886 + self.assertEqual(1, rdd.map(lambda x: (x, 1)).groupByKey().count()) def test_zip_with_different_serializers(self): a = self.sc.parallelize(range(5)) From b75b3070740803480d235b0c9a86673721344f30 Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 15 Apr 2015 13:00:19 -0700 Subject: [PATCH 104/128] [SPARK-6730][SQL] Allow using keyword as identifier in OPTIONS JIRA: https://issues.apache.org/jira/browse/SPARK-6730 It is very possible that keyword will be used as identifier in `OPTIONS`, this pr makes it works. However, another approach is that we can request that `OPTIONS` can't include keywords and has to use alternative identifier (e.g. table -> cassandraTable) if needed. If so, please let me know to close this pr. Thanks. Author: Liang-Chi Hsieh Closes #5520 from viirya/relax_options and squashes the following commits: 339fd68 [Liang-Chi Hsieh] Use regex parser. 92be11c [Liang-Chi Hsieh] Allow using keyword as identifier in OPTIONS. --- .../scala/org/apache/spark/sql/sources/ddl.scala | 15 ++++++++++++++- .../apache/spark/sql/sources/DDLTestSuite.scala | 11 ++++++----- 2 files changed, 20 insertions(+), 6 deletions(-) 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 319de710fbc3e..2e861b84b7133 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.sources import scala.language.existentials +import scala.util.matching.Regex import scala.language.implicitConversions import org.apache.spark.Logging @@ -155,7 +156,19 @@ private[sql] class DDLParser( protected lazy val className: Parser[String] = repsep(ident, ".") ^^ { case s => s.mkString(".")} - protected lazy val pair: Parser[(String, String)] = ident ~ stringLit ^^ { case k ~ v => (k,v) } + override implicit def regexToParser(regex: Regex): Parser[String] = acceptMatch( + s"identifier matching regex ${regex}", { + case lexical.Identifier(str) if regex.unapplySeq(str).isDefined => str + case lexical.Keyword(str) if regex.unapplySeq(str).isDefined => str + } + ) + + protected lazy val optionName: Parser[String] = "[_a-zA-Z][a-zA-Z0-9]*".r ^^ { + case name => name + } + + protected lazy val pair: Parser[(String, String)] = + optionName ~ stringLit ^^ { case k ~ v => (k,v) } protected lazy val column: Parser[StructField] = ident ~ dataType ~ (COMMENT ~> stringLit).? ^^ { case columnName ~ typ ~ cm => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala index 3f24a497390c1..ca25751b9583d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala @@ -25,17 +25,17 @@ class DDLScanSource extends RelationProvider { override def createRelation( sqlContext: SQLContext, parameters: Map[String, String]): BaseRelation = { - SimpleDDLScan(parameters("from").toInt, parameters("TO").toInt)(sqlContext) + SimpleDDLScan(parameters("from").toInt, parameters("TO").toInt, parameters("Table"))(sqlContext) } } -case class SimpleDDLScan(from: Int, to: Int)(@transient val sqlContext: SQLContext) +case class SimpleDDLScan(from: Int, to: Int, table: String)(@transient val sqlContext: SQLContext) extends BaseRelation with TableScan { override def schema: StructType = StructType(Seq( StructField("intType", IntegerType, nullable = false, - new MetadataBuilder().putString("comment", "test comment").build()), + new MetadataBuilder().putString("comment", s"test comment $table").build()), StructField("stringType", StringType, nullable = false), StructField("dateType", DateType, nullable = false), StructField("timestampType", TimestampType, nullable = false), @@ -73,7 +73,8 @@ class DDLTestSuite extends DataSourceTest { |USING org.apache.spark.sql.sources.DDLScanSource |OPTIONS ( | From '1', - | To '10' + | To '10', + | Table 'test1' |) """.stripMargin) } @@ -81,7 +82,7 @@ class DDLTestSuite extends DataSourceTest { sqlTest( "describe ddlPeople", Seq( - Row("intType", "int", "test comment"), + Row("intType", "int", "test comment test1"), Row("stringType", "string", ""), Row("dateType", "date", ""), Row("timestampType", "timestamp", ""), From e3e4e9a38b25174ed8bb460ba2b375813ebf3b4b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 15 Apr 2015 13:01:29 -0700 Subject: [PATCH 105/128] [SPARK-6800][SQL] Update doc for JDBCRelation's columnPartition JIRA https://issues.apache.org/jira/browse/SPARK-6800 Author: Liang-Chi Hsieh Closes #5488 from viirya/fix_jdbc_where and squashes the following commits: 51386c8 [Liang-Chi Hsieh] Update code comment. 1dcc929 [Liang-Chi Hsieh] Update document. 3eb74d6 [Liang-Chi Hsieh] Revert and modify doc. df11783 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into fix_jdbc_where 3e7db15 [Liang-Chi Hsieh] Fix wrong logic to generate WHERE clause for JDBC. --- docs/sql-programming-guide.md | 5 ++++- .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 4 ++-- 2 files changed, 6 insertions(+), 3 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 332618edf0c55..03500867df70f 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1371,7 +1371,10 @@ the Data Sources API. The following options are supported: These options must all be specified if any of them is specified. They describe how to partition the table when reading in parallel from multiple workers. - partitionColumn must be a numeric column from the table in question. + partitionColumn must be a numeric column from the table in question. Notice + that lowerBound and upperBound are just used to decide the + partition stride, not for filtering the rows in table. So all rows in the table will be + partitioned and returned. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index c25ef58e6f62a..b237fe684cdc1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -873,8 +873,8 @@ class SQLContext(@transient val sparkContext: SparkContext) * passed to this function. * * @param columnName the name of a column of integral type that will be used for partitioning. - * @param lowerBound the minimum value of `columnName` to retrieve - * @param upperBound the maximum value of `columnName` to retrieve + * @param lowerBound the minimum value of `columnName` used to decide partition stride + * @param upperBound the maximum value of `columnName` used to decide partition stride * @param numPartitions the number of partitions. the range `minValue`-`maxValue` will be split * evenly into this many partitions * From 785f95586b951d7b05481ee925fb95c20c4d6b6f Mon Sep 17 00:00:00 2001 From: Yin Huai Date: Wed, 15 Apr 2015 13:04:03 -0700 Subject: [PATCH 106/128] [SPARK-6887][SQL] ColumnBuilder misses FloatType https://issues.apache.org/jira/browse/SPARK-6887 Author: Yin Huai Closes #5499 from yhuai/inMemFloat and squashes the following commits: 84cba38 [Yin Huai] Add test. 4b75ba6 [Yin Huai] Add FloatType back. --- .../spark/sql/columnar/ColumnBuilder.scala | 1 + .../org/apache/spark/sql/QueryTest.scala | 3 + .../columnar/InMemoryColumnarQuerySuite.scala | 59 ++++++++++++++++++- 3 files changed, 62 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index c881747751520..00ed70430b84d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -153,6 +153,7 @@ private[sql] object ColumnBuilder { val builder: ColumnBuilder = dataType match { case IntegerType => new IntColumnBuilder case LongType => new LongColumnBuilder + case FloatType => new FloatColumnBuilder case DoubleType => new DoubleColumnBuilder case BooleanType => new BooleanColumnBuilder case ByteType => new ByteColumnBuilder diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 9a81fc5d72819..59f9508444f25 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -104,9 +104,12 @@ object QueryTest { // Converts data to types that we can do equality comparison using Scala collections. // For BigDecimal type, the Scala type has a better definition of equality test (similar to // Java's java.math.BigDecimal.compareTo). + // For binary arrays, we convert it to Seq to avoid of calling java.util.Arrays.equals for + // equality test. val converted: Seq[Row] = answer.map { s => Row.fromSeq(s.toSeq.map { case d: java.math.BigDecimal => BigDecimal(d) + case b: Array[Byte] => b.toSeq case o => o }) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 479210d1c9c43..56591d9dba29e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -17,11 +17,13 @@ package org.apache.spark.sql.columnar +import java.sql.{Date, Timestamp} + import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.implicits._ -import org.apache.spark.sql.types.{DecimalType, Decimal} +import org.apache.spark.sql.types._ import org.apache.spark.sql.{QueryTest, TestData} import org.apache.spark.storage.StorageLevel.MEMORY_ONLY @@ -132,4 +134,59 @@ class InMemoryColumnarQuerySuite extends QueryTest { sql("SELECT * FROM test_fixed_decimal"), (1 to 10).map(i => Row(Decimal(i, 15, 10).toJavaBigDecimal))) } + + test("test different data types") { + // Create the schema. + val struct = + StructType( + StructField("f1", FloatType, true) :: + StructField("f2", ArrayType(BooleanType), true) :: Nil) + val dataTypes = + Seq(StringType, BinaryType, NullType, BooleanType, + ByteType, ShortType, IntegerType, LongType, + FloatType, DoubleType, DecimalType.Unlimited, DecimalType(6, 5), + DateType, TimestampType, + ArrayType(IntegerType), MapType(StringType, LongType), struct) + val fields = dataTypes.zipWithIndex.map { case (dataType, index) => + StructField(s"col$index", dataType, true) + } + val allColumns = fields.map(_.name).mkString(",") + val schema = StructType(fields) + + // Create a RDD for the schema + val rdd = + sparkContext.parallelize((1 to 100), 10).map { i => + Row( + s"str${i}: test cache.", + s"binary${i}: test cache.".getBytes("UTF-8"), + null, + i % 2 == 0, + i.toByte, + i.toShort, + i, + Long.MaxValue - i.toLong, + (i + 0.25).toFloat, + (i + 0.75), + BigDecimal(Long.MaxValue.toString + ".12345"), + new java.math.BigDecimal(s"${i % 9 + 1}" + ".23456"), + new Date(i), + new Timestamp(i), + (1 to i).toSeq, + (0 to i).map(j => s"map_key_$j" -> (Long.MaxValue - j)).toMap, + Row((i - 0.25).toFloat, (1 to i).toSeq)) + } + createDataFrame(rdd, schema).registerTempTable("InMemoryCache_different_data_types") + // Cache the table. + sql("cache table InMemoryCache_different_data_types") + // Make sure the table is indeed cached. + val tableScan = table("InMemoryCache_different_data_types").queryExecution.executedPlan + assert( + isCached("InMemoryCache_different_data_types"), + "InMemoryCache_different_data_types should be cached.") + // Issue a query and check the results. + checkAnswer( + sql(s"SELECT DISTINCT ${allColumns} FROM InMemoryCache_different_data_types"), + table("InMemoryCache_different_data_types").collect()) + dropTempTable("InMemoryCache_different_data_types") + } } From 85842760dc4616577162f44cc0fa9db9bd23bd9c Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Wed, 15 Apr 2015 13:06:38 -0700 Subject: [PATCH 107/128] [SPARK-6638] [SQL] Improve performance of StringType in SQL This PR change the internal representation for StringType from java.lang.String to UTF8String, which is implemented use ArrayByte. This PR should not break any public API, Row.getString() will still return java.lang.String. This is the first step of improve the performance of String in SQL. cc rxin Author: Davies Liu Closes #5350 from davies/string and squashes the following commits: 3b7bfa8 [Davies Liu] fix schema of AddJar 2772f0d [Davies Liu] fix new test failure 6d776a9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 59025c8 [Davies Liu] address comments from @marmbrus 341ec2c [Davies Liu] turn off scala style check in UTF8StringSuite 744788f [Davies Liu] Merge branch 'master' of github.com:apache/spark into string b04a19c [Davies Liu] add comment for getString/setString 08d897b [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 5116b43 [Davies Liu] rollback unrelated changes 1314a37 [Davies Liu] address comments from Yin 867bf50 [Davies Liu] fix String filter push down 13d9d42 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 2089d24 [Davies Liu] add hashcode check back ac18ae6 [Davies Liu] address comment fd11364 [Davies Liu] optimize UTF8String 8d17f21 [Davies Liu] fix hive compatibility tests e5fa5b8 [Davies Liu] remove clone in UTF8String 28f3d81 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 28d6f32 [Davies Liu] refactor 537631c [Davies Liu] some comment about Date 9f4c194 [Davies Liu] convert data type for data source 956b0a4 [Davies Liu] fix hive tests 73e4363 [Davies Liu] Merge branch 'master' of github.com:apache/spark into string 9dc32d1 [Davies Liu] fix some hive tests 23a766c [Davies Liu] refactor 8b45864 [Davies Liu] fix codegen with UTF8String bb52e44 [Davies Liu] fix scala style c7dd4d2 [Davies Liu] fix some catalyst tests 38c303e [Davies Liu] fix python sql tests 5f9e120 [Davies Liu] fix sql tests 6b499ac [Davies Liu] fix style a85fb27 [Davies Liu] refactor d32abd1 [Davies Liu] fix utf8 for python api 4699c3a [Davies Liu] use Array[Byte] in UTF8String 21f67c6 [Davies Liu] cleanup 685fd07 [Davies Liu] use UTF8String instead of String for StringType --- python/pyspark/sql/dataframe.py | 10 +- .../main/scala/org/apache/spark/sql/Row.scala | 3 +- .../sql/catalyst/CatalystTypeConverters.scala | 37 +++ .../spark/sql/catalyst/ScalaReflection.scala | 1 + .../catalyst/analysis/HiveTypeCoercion.scala | 6 +- .../spark/sql/catalyst/expressions/Cast.scala | 36 +-- .../expressions/SpecificMutableRow.scala | 12 +- .../expressions/codegen/CodeGenerator.scala | 32 ++- .../codegen/GenerateProjection.scala | 46 ++-- .../sql/catalyst/expressions/generators.scala | 7 +- .../sql/catalyst/expressions/literals.scala | 7 +- .../sql/catalyst/expressions/predicates.scala | 3 +- .../spark/sql/catalyst/expressions/rows.scala | 14 +- .../expressions/stringOperations.scala | 90 ++++---- .../sql/catalyst/optimizer/Optimizer.scala | 21 +- .../apache/spark/sql/types/DateUtils.scala | 1 + .../apache/spark/sql/types/UTF8String.scala | 214 ++++++++++++++++++ .../apache/spark/sql/types/dataTypes.scala | 6 +- .../ExpressionEvaluationSuite.scala | 90 ++++---- .../GeneratedMutableEvaluationSuite.scala | 4 +- .../spark/sql/types/UTF8StringSuite.scala | 70 ++++++ .../org/apache/spark/sql/SQLContext.scala | 1 + .../spark/sql/columnar/ColumnStats.scala | 6 +- .../spark/sql/columnar/ColumnType.scala | 20 +- .../spark/sql/execution/ExistingRDD.scala | 31 ++- .../apache/spark/sql/execution/commands.scala | 13 +- .../spark/sql/execution/debug/package.scala | 2 +- .../spark/sql/execution/pythonUdfs.scala | 4 +- .../org/apache/spark/sql/jdbc/JDBCRDD.scala | 4 +- .../apache/spark/sql/jdbc/JDBCRelation.scala | 2 + .../org/apache/spark/sql/jdbc/jdbc.scala | 5 +- .../apache/spark/sql/json/JSONRelation.scala | 8 +- .../org/apache/spark/sql/json/JsonRDD.scala | 2 +- .../spark/sql/parquet/ParquetConverter.scala | 19 +- .../spark/sql/parquet/ParquetFilters.scala | 12 +- .../sql/parquet/ParquetTableSupport.scala | 7 +- .../apache/spark/sql/parquet/newParquet.scala | 11 +- .../sql/sources/DataSourceStrategy.scala | 37 +-- .../apache/spark/sql/sources/interfaces.scala | 10 + .../scala/org/apache/spark/sql/RowSuite.scala | 2 +- .../org/apache/spark/sql/SQLQuerySuite.scala | 10 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 8 +- .../sql/columnar/ColumnarTestUtils.scala | 4 +- .../spark/sql/sources/TableScanSuite.scala | 10 +- .../spark/sql/hive/HiveInspectors.scala | 22 +- .../spark/sql/hive/HiveStrategies.scala | 13 +- .../hive/execution/ScriptTransformation.scala | 17 +- .../spark/sql/hive/execution/commands.scala | 10 +- .../org/apache/spark/sql/hive/Shim12.scala | 4 +- .../org/apache/spark/sql/hive/Shim13.scala | 36 ++- 50 files changed, 742 insertions(+), 298 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index ef91a9c4f522d..f2c3b74a185cf 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -456,7 +456,7 @@ def join(self, other, joinExprs=None, joinType=None): One of `inner`, `outer`, `left_outer`, `right_outer`, `semijoin`. >>> df.join(df2, df.name == df2.name, 'outer').select(df.name, df2.height).collect() - [Row(name=None, height=80), Row(name=u'Bob', height=85), Row(name=u'Alice', height=None)] + [Row(name=None, height=80), Row(name=u'Alice', height=None), Row(name=u'Bob', height=85)] """ if joinExprs is None: @@ -637,9 +637,9 @@ def groupBy(self, *cols): >>> df.groupBy().avg().collect() [Row(AVG(age)=3.5)] >>> df.groupBy('name').agg({'age': 'mean'}).collect() - [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] + [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] >>> df.groupBy(df.name).avg().collect() - [Row(name=u'Bob', AVG(age)=5.0), Row(name=u'Alice', AVG(age)=2.0)] + [Row(name=u'Alice', AVG(age)=2.0), Row(name=u'Bob', AVG(age)=5.0)] """ jcols = ListConverter().convert([_to_java_column(c) for c in cols], self._sc._gateway._gateway_client) @@ -867,11 +867,11 @@ def agg(self, *exprs): >>> gdf = df.groupBy(df.name) >>> gdf.agg({"*": "count"}).collect() - [Row(name=u'Bob', COUNT(1)=1), Row(name=u'Alice', COUNT(1)=1)] + [Row(name=u'Alice', COUNT(1)=1), Row(name=u'Bob', COUNT(1)=1)] >>> from pyspark.sql import functions as F >>> gdf.agg(F.min(df.age)).collect() - [Row(MIN(age)=5), Row(MIN(age)=2)] + [Row(MIN(age)=2), Row(MIN(age)=5)] """ assert exprs, "exprs should not be empty" if len(exprs) == 1 and isinstance(exprs[0], dict): diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index d794f034f5578..ac8a782976465 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.util.hashing.MurmurHash3 import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.types.{StructType, DateUtils} +import org.apache.spark.sql.types.StructType object Row { /** @@ -257,6 +257,7 @@ trait Row extends Serializable { * * @throws ClassCastException when data type does not match. */ + // TODO(davies): This is not the right default implementation, we use Int as Date internally def getDate(i: Int): java.sql.Date = apply(i).asInstanceOf[java.sql.Date] /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 91976fef6dc0d..d4f9fdacda4fb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -77,6 +77,9 @@ object CatalystTypeConverters { } new GenericRowWithSchema(ar, structType) + case (d: String, _) => + UTF8String(d) + case (d: BigDecimal, _) => Decimal(d) @@ -175,6 +178,11 @@ object CatalystTypeConverters { case other => other } + case dataType: StringType => (item: Any) => extractOption(item) match { + case s: String => UTF8String(s) + case other => other + } + case _ => (item: Any) => extractOption(item) match { case d: BigDecimal => Decimal(d) @@ -184,6 +192,26 @@ object CatalystTypeConverters { } } + /** + * Converts Scala objects to catalyst rows / types. + * + * Note: This should be called before do evaluation on Row + * (It does not support UDT) + * This is used to create an RDD or test results with correct types for Catalyst. + */ + def convertToCatalyst(a: Any): Any = a match { + case s: String => UTF8String(s) + case d: java.sql.Date => DateUtils.fromJavaDate(d) + case d: BigDecimal => Decimal(d) + case d: java.math.BigDecimal => Decimal(d) + case seq: Seq[Any] => seq.map(convertToCatalyst) + case r: Row => Row(r.toSeq.map(convertToCatalyst): _*) + case arr: Array[Any] => arr.toSeq.map(convertToCatalyst).toArray + case m: Map[Any, Any] => + m.map { case (k, v) => (convertToCatalyst(k), convertToCatalyst(v)) }.toMap + case other => other + } + /** * Converts Catalyst types used internally in rows to standard Scala types * This method is slow, and for batch conversion you should be using converter @@ -211,6 +239,9 @@ object CatalystTypeConverters { case (i: Int, DateType) => DateUtils.toJavaDate(i) + case (s: UTF8String, StringType) => + s.toString() + case (other, _) => other } @@ -262,6 +293,12 @@ object CatalystTypeConverters { case other => other } + case StringType => + (item: Any) => item match { + case s: UTF8String => s.toString() + case other => other + } + case other => (item: Any) => item } 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 01d5c1512201a..d9521953cad73 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 @@ -138,6 +138,7 @@ trait ScalaReflection { // The data type can be determined without ambiguity. case obj: BooleanType.JvmType => BooleanType case obj: BinaryType.JvmType => BinaryType + case obj: String => StringType case obj: StringType.JvmType => StringType case obj: ByteType.JvmType => ByteType case obj: ShortType.JvmType => ShortType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index 3aeb964994d37..35c7f00d4e42a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -115,7 +115,7 @@ trait HiveTypeCoercion { * the appropriate numeric equivalent. */ object ConvertNaNs extends Rule[LogicalPlan] { - val stringNaN = Literal.create("NaN", StringType) + val stringNaN = Literal("NaN") def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressions { @@ -563,6 +563,10 @@ trait HiveTypeCoercion { case Sum(e @ TimestampType()) => Sum(Cast(e, DoubleType)) case Average(e @ TimestampType()) => Average(Cast(e, DoubleType)) + // Compatible with Hive + case Substring(e, start, len) if e.dataType != StringType => + Substring(Cast(e, StringType), start, len) + // Coalesce should return the first non-null value, which could be any column // from the list. So we need to make sure the return type is deterministic and // compatible with every child column. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index 31f1a5fdc7e53..adf941ab2a45f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -21,7 +21,6 @@ import java.sql.{Date, Timestamp} import java.text.{DateFormat, SimpleDateFormat} import org.apache.spark.Logging -import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.types._ /** Cast the child expression to the target data type. */ @@ -112,21 +111,21 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { - case BinaryType => buildCast[Array[Byte]](_, new String(_, "UTF-8")) - case DateType => buildCast[Int](_, d => DateUtils.toString(d)) - case TimestampType => buildCast[Timestamp](_, timestampToString) - case _ => buildCast[Any](_, _.toString) + case BinaryType => buildCast[Array[Byte]](_, UTF8String(_)) + case DateType => buildCast[Int](_, d => UTF8String(DateUtils.toString(d))) + case TimestampType => buildCast[Timestamp](_, t => UTF8String(timestampToString(t))) + case _ => buildCast[Any](_, o => UTF8String(o.toString)) } // BinaryConverter private[this] def castToBinary(from: DataType): Any => Any = from match { - case StringType => buildCast[String](_, _.getBytes("UTF-8")) + case StringType => buildCast[UTF8String](_, _.getBytes) } // UDFToBoolean private[this] def castToBoolean(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, _.length() != 0) + buildCast[UTF8String](_, _.length() != 0) case TimestampType => buildCast[Timestamp](_, t => t.getTime() != 0 || t.getNanos() != 0) case DateType => @@ -151,8 +150,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // TimestampConverter private[this] def castToTimestamp(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => { + buildCast[UTF8String](_, utfs => { // Throw away extra if more than 9 decimal places + val s = utfs.toString val periodIdx = s.indexOf(".") var n = s if (periodIdx != -1 && n.length() - periodIdx > 9) { @@ -227,8 +227,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // DateConverter private[this] def castToDate(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => - try DateUtils.fromJavaDate(Date.valueOf(s)) + buildCast[UTF8String](_, s => + try DateUtils.fromJavaDate(Date.valueOf(s.toString)) catch { case _: java.lang.IllegalArgumentException => null } ) case TimestampType => @@ -245,7 +245,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // LongConverter private[this] def castToLong(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toLong catch { + buildCast[UTF8String](_, s => try s.toString.toLong catch { case _: NumberFormatException => null }) case BooleanType => @@ -261,7 +261,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // IntConverter private[this] def castToInt(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toInt catch { + buildCast[UTF8String](_, s => try s.toString.toInt catch { case _: NumberFormatException => null }) case BooleanType => @@ -277,7 +277,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // ShortConverter private[this] def castToShort(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toShort catch { + buildCast[UTF8String](_, s => try s.toString.toShort catch { case _: NumberFormatException => null }) case BooleanType => @@ -293,7 +293,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // ByteConverter private[this] def castToByte(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toByte catch { + buildCast[UTF8String](_, s => try s.toString.toByte catch { case _: NumberFormatException => null }) case BooleanType => @@ -323,7 +323,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w private[this] def castToDecimal(from: DataType, target: DecimalType): Any => Any = from match { case StringType => - buildCast[String](_, s => try changePrecision(Decimal(s.toDouble), target) catch { + buildCast[UTF8String](_, s => try { + changePrecision(Decimal(s.toString.toDouble), target) + } catch { case _: NumberFormatException => null }) case BooleanType => @@ -348,7 +350,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // DoubleConverter private[this] def castToDouble(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toDouble catch { + buildCast[UTF8String](_, s => try s.toString.toDouble catch { case _: NumberFormatException => null }) case BooleanType => @@ -364,7 +366,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression w // FloatConverter private[this] def castToFloat(from: DataType): Any => Any = from match { case StringType => - buildCast[String](_, s => try s.toFloat catch { + buildCast[UTF8String](_, s => try s.toString.toFloat catch { case _: NumberFormatException => null }) case BooleanType => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala index 47b6f358ed1b1..3475ed05f4454 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SpecificMutableRow.scala @@ -230,13 +230,17 @@ final class SpecificMutableRow(val values: Array[MutableValue]) extends MutableR new GenericRow(newValues) } - override def update(ordinal: Int, value: Any): Unit = { - if (value == null) setNullAt(ordinal) else values(ordinal).update(value) + override def update(ordinal: Int, value: Any) { + if (value == null) { + setNullAt(ordinal) + } else { + values(ordinal).update(value) + } } - override def setString(ordinal: Int, value: String): Unit = update(ordinal, value) + override def setString(ordinal: Int, value: String): Unit = update(ordinal, UTF8String(value)) - override def getString(ordinal: Int): String = apply(ordinal).asInstanceOf[String] + override def getString(ordinal: Int): String = apply(ordinal).toString override def setInt(ordinal: Int, value: Int): Unit = { val currentValue = values(ordinal).asInstanceOf[MutableInt] 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 d141354a0f427..be2c101d63a63 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 @@ -216,10 +216,11 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val $primitiveTerm: ${termForType(dataType)} = $value """.children - case expressions.Literal(value: String, dataType) => + case expressions.Literal(value: UTF8String, dataType) => q""" val $nullTerm = ${value == null} - val $primitiveTerm: ${termForType(dataType)} = $value + val $primitiveTerm: ${termForType(dataType)} = + org.apache.spark.sql.types.UTF8String(${value.getBytes}) """.children case expressions.Literal(value: Int, dataType) => @@ -243,11 +244,14 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin if($nullTerm) ${defaultPrimitive(StringType)} else - new String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) + org.apache.spark.sql.types.UTF8String(${eval.primitiveTerm}.asInstanceOf[Array[Byte]]) """.children case Cast(child @ DateType(), StringType) => - child.castOrNull(c => q"org.apache.spark.sql.types.DateUtils.toString($c)", StringType) + child.castOrNull(c => + q"""org.apache.spark.sql.types.UTF8String( + org.apache.spark.sql.types.DateUtils.toString($c))""", + StringType) case Cast(child @ NumericType(), IntegerType) => child.castOrNull(c => q"$c.toInt", IntegerType) @@ -272,9 +276,18 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin if($nullTerm) ${defaultPrimitive(StringType)} else - ${eval.primitiveTerm}.toString + org.apache.spark.sql.types.UTF8String(${eval.primitiveTerm}.toString) """.children + case EqualTo(e1: BinaryType, e2: BinaryType) => + (e1, e2).evaluateAs (BooleanType) { + case (eval1, eval2) => + q""" + java.util.Arrays.equals($eval1.asInstanceOf[Array[Byte]], + $eval2.asInstanceOf[Array[Byte]]) + """ + } + case EqualTo(e1, e2) => (e1, e2).evaluateAs (BooleanType) { case (eval1, eval2) => q"$eval1 == $eval2" } @@ -597,7 +610,8 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin val localLogger = log val localLoggerTree = reify { localLogger } q""" - $localLoggerTree.debug(${e.toString} + ": " + (if($nullTerm) "null" else $primitiveTerm)) + $localLoggerTree.debug( + ${e.toString} + ": " + (if ($nullTerm) "null" else $primitiveTerm.toString)) """ :: Nil } else { Nil @@ -608,6 +622,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin protected def getColumn(inputRow: TermName, dataType: DataType, ordinal: Int) = { dataType match { + case StringType => q"$inputRow($ordinal).asInstanceOf[org.apache.spark.sql.types.UTF8String]" case dt @ NativeType() => q"$inputRow.${accessorForType(dt)}($ordinal)" case _ => q"$inputRow.apply($ordinal).asInstanceOf[${termForType(dataType)}]" } @@ -619,6 +634,7 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin ordinal: Int, value: TermName) = { dataType match { + case StringType => q"$destinationRow.update($ordinal, $value)" case dt @ NativeType() => q"$destinationRow.${mutatorForType(dt)}($ordinal, $value)" case _ => q"$destinationRow.update($ordinal, $value)" } @@ -642,13 +658,13 @@ abstract class CodeGenerator[InType <: AnyRef, OutType <: AnyRef] extends Loggin case DoubleType => "Double" case FloatType => "Float" case BooleanType => "Boolean" - case StringType => "String" + case StringType => "org.apache.spark.sql.types.UTF8String" } protected def defaultPrimitive(dt: DataType) = dt match { case BooleanType => ru.Literal(Constant(false)) case FloatType => ru.Literal(Constant(-1.0.toFloat)) - case StringType => ru.Literal(Constant("")) + case StringType => q"""org.apache.spark.sql.types.UTF8String("")""" case ShortType => ru.Literal(Constant(-1.toShort)) case LongType => ru.Literal(Constant(-1L)) case ByteType => ru.Literal(Constant(-1.toByte)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala index 69397a73a8880..6f572ff959fb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateProjection.scala @@ -111,36 +111,54 @@ object GenerateProjection extends CodeGenerator[Seq[Expression], Projection] { val specificAccessorFunctions = NativeType.all.map { dataType => val ifStatements = expressions.zipWithIndex.flatMap { - case (e, i) if e.dataType == dataType => + // getString() is not used by expressions + case (e, i) if e.dataType == dataType && dataType != StringType => val elementName = newTermName(s"c$i") // TODO: The string of ifs gets pretty inefficient as the row grows in size. // TODO: Optional null checks? q"if(i == $i) return $elementName" :: Nil case _ => Nil } - - q""" - override def ${accessorForType(dataType)}(i: Int):${termForType(dataType)} = { - ..$ifStatements; - $accessorFailure - }""" + dataType match { + // Row() need this interface to compile + case StringType => + q""" + override def getString(i: Int): String = { + $accessorFailure + }""" + case other => + q""" + override def ${accessorForType(dataType)}(i: Int): ${termForType(dataType)} = { + ..$ifStatements; + $accessorFailure + }""" + } } val specificMutatorFunctions = NativeType.all.map { dataType => val ifStatements = expressions.zipWithIndex.flatMap { - case (e, i) if e.dataType == dataType => + // setString() is not used by expressions + case (e, i) if e.dataType == dataType && dataType != StringType => val elementName = newTermName(s"c$i") // TODO: The string of ifs gets pretty inefficient as the row grows in size. // TODO: Optional null checks? q"if(i == $i) { nullBits($i) = false; $elementName = value; return }" :: Nil case _ => Nil } - - q""" - override def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}): Unit = { - ..$ifStatements; - $accessorFailure - }""" + dataType match { + case StringType => + // MutableRow() need this interface to compile + q""" + override def setString(i: Int, value: String) { + $accessorFailure + }""" + case other => + q""" + override def ${mutatorForType(dataType)}(i: Int, value: ${termForType(dataType)}) { + ..$ifStatements; + $accessorFailure + }""" + } } val hashValues = expressions.zipWithIndex.map { case (e,i) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index 860b72fad38b3..67caadb839ff9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import scala.collection.Map -import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, trees} import org.apache.spark.sql.types._ /** @@ -85,8 +85,11 @@ case class UserDefinedGenerator( override protected def makeOutput(): Seq[Attribute] = schema override def eval(input: Row): TraversableOnce[Row] = { + // TODO(davies): improve this + // Convert the objects into Scala Type before calling function, we need schema to support UDT + val inputSchema = StructType(children.map(e => StructField(e.simpleString, e.dataType, true))) val inputRow = new InterpretedProjection(children) - function(inputRow(input)) + function(CatalystTypeConverters.convertToScala(inputRow(input), inputSchema).asInstanceOf[Row]) } override def toString: String = s"UserDefinedGenerator(${children.mkString(",")})" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 0e2d593e94124..18cba4cc46707 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.types._ object Literal { @@ -29,7 +30,7 @@ object Literal { case f: Float => Literal(f, FloatType) case b: Byte => Literal(b, ByteType) case s: Short => Literal(s, ShortType) - case s: String => Literal(s, StringType) + case s: String => Literal(UTF8String(s), StringType) case b: Boolean => Literal(b, BooleanType) case d: BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) case d: java.math.BigDecimal => Literal(Decimal(d), DecimalType.Unlimited) @@ -42,7 +43,9 @@ object Literal { throw new RuntimeException("Unsupported literal type " + v.getClass + " " + v) } - def create(v: Any, dataType: DataType): Literal = Literal(v, dataType) + def create(v: Any, dataType: DataType): Literal = { + Literal(CatalystTypeConverters.convertToCatalyst(v), dataType) + } } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 7e47cb3fffe12..fcd6352079b4d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -179,8 +179,7 @@ case class EqualTo(left: Expression, right: Expression) extends BinaryComparison val r = right.eval(input) if (r == null) null else if (left.dataType != BinaryType) l == r - else BinaryType.ordering.compare( - l.asInstanceOf[Array[Byte]], r.asInstanceOf[Array[Byte]]) == 0 + else java.util.Arrays.equals(l.asInstanceOf[Array[Byte]], r.asInstanceOf[Array[Byte]]) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 0a275b84086cf..1b62e17ff47fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.types.{StructType, NativeType} +import org.apache.spark.sql.types.{UTF8String, StructType, NativeType} /** @@ -37,6 +37,7 @@ trait MutableRow extends Row { def setByte(ordinal: Int, value: Byte) def setFloat(ordinal: Int, value: Float) def setString(ordinal: Int, value: String) + // TODO(davies): add setDate() and setDecimal() } /** @@ -114,9 +115,15 @@ class GenericRow(protected[sql] val values: Array[Any]) extends Row { } override def getString(i: Int): String = { - values(i).asInstanceOf[String] + values(i) match { + case null => null + case s: String => s + case utf8: UTF8String => utf8.toString + } } + // TODO(davies): add getDate and getDecimal + // Custom hashCode function that matches the efficient code generated version. override def hashCode: Int = { var result: Int = 37 @@ -189,8 +196,7 @@ class GenericMutableRow(v: Array[Any]) extends GenericRow(v) with MutableRow { override def setFloat(ordinal: Int, value: Float): Unit = { values(ordinal) = value } override def setInt(ordinal: Int, value: Int): Unit = { values(ordinal) = value } override def setLong(ordinal: Int, value: Long): Unit = { values(ordinal) = value } - override def setString(ordinal: Int, value: String): Unit = { values(ordinal) = value } - + override def setString(ordinal: Int, value: String) { values(ordinal) = UTF8String(value)} override def setNullAt(i: Int): Unit = { values(i) = null } override def setShort(ordinal: Int, value: Short): Unit = { values(ordinal) = value } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala index acfbbace608ef..d597bf7ce756a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala @@ -19,11 +19,8 @@ package org.apache.spark.sql.catalyst.expressions import java.util.regex.Pattern -import scala.collection.IndexedSeqOptimized - - import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.types.{BinaryType, BooleanType, DataType, StringType} +import org.apache.spark.sql.types._ trait StringRegexExpression { self: BinaryExpression => @@ -60,38 +57,17 @@ trait StringRegexExpression { if(r == null) { null } else { - val regex = pattern(r.asInstanceOf[String]) + val regex = pattern(r.asInstanceOf[UTF8String].toString) if(regex == null) { null } else { - matches(regex, l.asInstanceOf[String]) + matches(regex, l.asInstanceOf[UTF8String].toString) } } } } } -trait CaseConversionExpression { - self: UnaryExpression => - - type EvaluatedType = Any - - def convert(v: String): String - - override def foldable: Boolean = child.foldable - def nullable: Boolean = child.nullable - def dataType: DataType = StringType - - override def eval(input: Row): Any = { - val evaluated = child.eval(input) - if (evaluated == null) { - null - } else { - convert(evaluated.toString) - } - } -} - /** * Simple RegEx pattern matching function */ @@ -134,12 +110,33 @@ case class RLike(left: Expression, right: Expression) override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) } +trait CaseConversionExpression { + self: UnaryExpression => + + type EvaluatedType = Any + + def convert(v: UTF8String): UTF8String + + override def foldable: Boolean = child.foldable + def nullable: Boolean = child.nullable + def dataType: DataType = StringType + + override def eval(input: Row): Any = { + val evaluated = child.eval(input) + if (evaluated == null) { + null + } else { + convert(evaluated.asInstanceOf[UTF8String]) + } + } +} + /** * A function that converts the characters of a string to uppercase. */ case class Upper(child: Expression) extends UnaryExpression with CaseConversionExpression { - override def convert(v: String): String = v.toUpperCase() + override def convert(v: UTF8String): UTF8String = v.toUpperCase override def toString: String = s"Upper($child)" } @@ -149,7 +146,7 @@ case class Upper(child: Expression) extends UnaryExpression with CaseConversionE */ case class Lower(child: Expression) extends UnaryExpression with CaseConversionExpression { - override def convert(v: String): String = v.toLowerCase() + override def convert(v: UTF8String): UTF8String = v.toLowerCase override def toString: String = s"Lower($child)" } @@ -162,15 +159,16 @@ trait StringComparison { override def nullable: Boolean = left.nullable || right.nullable - def compare(l: String, r: String): Boolean + def compare(l: UTF8String, r: UTF8String): Boolean override def eval(input: Row): Any = { - val leftEval = left.eval(input).asInstanceOf[String] + val leftEval = left.eval(input) if(leftEval == null) { null } else { - val rightEval = right.eval(input).asInstanceOf[String] - if (rightEval == null) null else compare(leftEval, rightEval) + val rightEval = right.eval(input) + if (rightEval == null) null + else compare(leftEval.asInstanceOf[UTF8String], rightEval.asInstanceOf[UTF8String]) } } @@ -184,7 +182,7 @@ trait StringComparison { */ case class Contains(left: Expression, right: Expression) extends BinaryPredicate with StringComparison { - override def compare(l: String, r: String): Boolean = l.contains(r) + override def compare(l: UTF8String, r: UTF8String): Boolean = l.contains(r) } /** @@ -192,7 +190,7 @@ case class Contains(left: Expression, right: Expression) */ case class StartsWith(left: Expression, right: Expression) extends BinaryPredicate with StringComparison { - override def compare(l: String, r: String): Boolean = l.startsWith(r) + override def compare(l: UTF8String, r: UTF8String): Boolean = l.startsWith(r) } /** @@ -200,7 +198,7 @@ case class StartsWith(left: Expression, right: Expression) */ case class EndsWith(left: Expression, right: Expression) extends BinaryPredicate with StringComparison { - override def compare(l: String, r: String): Boolean = l.endsWith(r) + override def compare(l: UTF8String, r: UTF8String): Boolean = l.endsWith(r) } /** @@ -224,9 +222,7 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends override def children: Seq[Expression] = str :: pos :: len :: Nil @inline - def slice[T, C <: Any](str: C, startPos: Int, sliceLen: Int) - (implicit ev: (C=>IndexedSeqOptimized[T,_])): Any = { - val len = str.length + def slicePos(startPos: Int, sliceLen: Int, length: () => Int): (Int, Int) = { // Hive and SQL use one-based indexing for SUBSTR arguments but also accept zero and // negative indices for start positions. If a start index i is greater than 0, it // refers to element i-1 in the sequence. If a start index i is less than 0, it refers @@ -235,7 +231,7 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends val start = startPos match { case pos if pos > 0 => pos - 1 - case neg if neg < 0 => len + neg + case neg if neg < 0 => length() + neg case _ => 0 } @@ -244,12 +240,11 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends case x => start + x } - str.slice(start, end) + (start, end) } override def eval(input: Row): Any = { val string = str.eval(input) - val po = pos.eval(input) val ln = len.eval(input) @@ -257,11 +252,14 @@ case class Substring(str: Expression, pos: Expression, len: Expression) extends null } else { val start = po.asInstanceOf[Int] - val length = ln.asInstanceOf[Int] - + val length = ln.asInstanceOf[Int] string match { - case ba: Array[Byte] => slice(ba, start, length) - case other => slice(other.toString, start, length) + case ba: Array[Byte] => + val (st, end) = slicePos(start, length, () => ba.length) + ba.slice(st, end) + case s: UTF8String => + val (st, end) = slicePos(start, length, () => s.length) + s.slice(st, end) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala index 93e69d409cb91..7c80634d2c852 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -198,14 +198,19 @@ object LikeSimplification extends Rule[LogicalPlan] { val equalTo = "([^_%]*)".r def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case Like(l, Literal(startsWith(pattern), StringType)) if !pattern.endsWith("\\") => - StartsWith(l, Literal(pattern)) - case Like(l, Literal(endsWith(pattern), StringType)) => - EndsWith(l, Literal(pattern)) - case Like(l, Literal(contains(pattern), StringType)) if !pattern.endsWith("\\") => - Contains(l, Literal(pattern)) - case Like(l, Literal(equalTo(pattern), StringType)) => - EqualTo(l, Literal(pattern)) + case Like(l, Literal(utf, StringType)) => + utf.toString match { + case startsWith(pattern) if !pattern.endsWith("\\") => + StartsWith(l, Literal(pattern)) + case endsWith(pattern) => + EndsWith(l, Literal(pattern)) + case contains(pattern) if !pattern.endsWith("\\") => + Contains(l, Literal(pattern)) + case equalTo(pattern) => + EqualTo(l, Literal(pattern)) + case _ => + Like(l, Literal.create(utf, StringType)) + } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala index 504fb05842505..d36a49159b87f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateUtils.scala @@ -40,6 +40,7 @@ object DateUtils { millisToDays(d.getTime) } + // we should use the exact day as Int, for example, (year, month, day) -> day def millisToDays(millisLocal: Long): Int = { ((millisLocal + LOCAL_TIMEZONE.get().getOffset(millisLocal)) / MILLIS_PER_DAY).toInt } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala new file mode 100644 index 0000000000000..fc02ba6c9c43e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/UTF8String.scala @@ -0,0 +1,214 @@ +/* +* 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.types + +import java.util.Arrays + +/** + * A UTF-8 String, as internal representation of StringType in SparkSQL + * + * A String encoded in UTF-8 as an Array[Byte], which can be used for comparison, + * search, see http://en.wikipedia.org/wiki/UTF-8 for details. + * + * Note: This is not designed for general use cases, should not be used outside SQL. + */ + +final class UTF8String extends Ordered[UTF8String] with Serializable { + + private[this] var bytes: Array[Byte] = _ + + /** + * Update the UTF8String with String. + */ + def set(str: String): UTF8String = { + bytes = str.getBytes("utf-8") + this + } + + /** + * Update the UTF8String with Array[Byte], which should be encoded in UTF-8 + */ + def set(bytes: Array[Byte]): UTF8String = { + this.bytes = bytes + this + } + + /** + * Return the number of bytes for a code point with the first byte as `b` + * @param b The first byte of a code point + */ + @inline + private[this] def numOfBytes(b: Byte): Int = { + val offset = (b & 0xFF) - 192 + if (offset >= 0) UTF8String.bytesOfCodePointInUTF8(offset) else 1 + } + + /** + * Return the number of code points in it. + * + * This is only used by Substring() when `start` is negative. + */ + def length(): Int = { + var len = 0 + var i: Int = 0 + while (i < bytes.length) { + i += numOfBytes(bytes(i)) + len += 1 + } + len + } + + def getBytes: Array[Byte] = { + bytes + } + + /** + * Return a substring of this, + * @param start the position of first code point + * @param until the position after last code point + */ + def slice(start: Int, until: Int): UTF8String = { + if (until <= start || start >= bytes.length || bytes == null) { + new UTF8String + } + + var c = 0 + var i: Int = 0 + while (c < start && i < bytes.length) { + i += numOfBytes(bytes(i)) + c += 1 + } + var j = i + while (c < until && j < bytes.length) { + j += numOfBytes(bytes(j)) + c += 1 + } + UTF8String(Arrays.copyOfRange(bytes, i, j)) + } + + def contains(sub: UTF8String): Boolean = { + val b = sub.getBytes + if (b.length == 0) { + return true + } + var i: Int = 0 + while (i <= bytes.length - b.length) { + // In worst case, it's O(N*K), but should works fine with SQL + if (bytes(i) == b(0) && Arrays.equals(Arrays.copyOfRange(bytes, i, i + b.length), b)) { + return true + } + i += 1 + } + false + } + + def startsWith(prefix: UTF8String): Boolean = { + val b = prefix.getBytes + if (b.length > bytes.length) { + return false + } + Arrays.equals(Arrays.copyOfRange(bytes, 0, b.length), b) + } + + def endsWith(suffix: UTF8String): Boolean = { + val b = suffix.getBytes + if (b.length > bytes.length) { + return false + } + Arrays.equals(Arrays.copyOfRange(bytes, bytes.length - b.length, bytes.length), b) + } + + def toUpperCase(): UTF8String = { + // upper case depends on locale, fallback to String. + UTF8String(toString().toUpperCase) + } + + def toLowerCase(): UTF8String = { + // lower case depends on locale, fallback to String. + UTF8String(toString().toLowerCase) + } + + override def toString(): String = { + new String(bytes, "utf-8") + } + + override def clone(): UTF8String = new UTF8String().set(this.bytes) + + override def compare(other: UTF8String): Int = { + var i: Int = 0 + val b = other.getBytes + while (i < bytes.length && i < b.length) { + val res = bytes(i).compareTo(b(i)) + if (res != 0) return res + i += 1 + } + bytes.length - b.length + } + + override def compareTo(other: UTF8String): Int = { + compare(other) + } + + override def equals(other: Any): Boolean = other match { + case s: UTF8String => + Arrays.equals(bytes, s.getBytes) + case s: String => + // This is only used for Catalyst unit tests + // fail fast + bytes.length >= s.length && length() == s.length && toString() == s + case _ => + false + } + + override def hashCode(): Int = { + Arrays.hashCode(bytes) + } +} + +object UTF8String { + // number of tailing bytes in a UTF8 sequence for a code point + // see http://en.wikipedia.org/wiki/UTF-8, 192-256 of Byte 1 + private[types] val bytesOfCodePointInUTF8: Array[Int] = Array(2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, + 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, 3, + 4, 4, 4, 4, 4, 4, 4, 4, + 5, 5, 5, 5, + 6, 6, 6, 6) + + /** + * Create a UTF-8 String from String + */ + def apply(s: String): UTF8String = { + if (s != null) { + new UTF8String().set(s) + } else{ + null + } + } + + /** + * Create a UTF-8 String from Array[Byte], which should be encoded in UTF-8 + */ + def apply(bytes: Array[Byte]): UTF8String = { + if (bytes != null) { + new UTF8String().set(bytes) + } else { + null + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala index cdf2bc68d9c5e..c6fb22c26bd3c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/dataTypes.scala @@ -350,7 +350,7 @@ class StringType private() extends NativeType with PrimitiveType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "StringType$" in byte code. // Defined with a private constructor so the companion object is the only possible instantiation. - private[sql] type JvmType = String + private[sql] type JvmType = UTF8String @transient private[sql] lazy val tag = ScalaReflectionLock.synchronized { typeTag[JvmType] } private[sql] val ordering = implicitly[Ordering[JvmType]] @@ -1196,8 +1196,8 @@ abstract class UserDefinedType[UserType] extends DataType with Serializable { /** * Convert the user type to a SQL datum * - * TODO: Can we make this take obj: UserType? The issue is in ScalaReflection.convertToCatalyst, - * where we need to convert Any to UserType. + * TODO: Can we make this take obj: UserType? The issue is in + * CatalystTypeConverters.convertToCatalyst, where we need to convert Any to UserType. */ def serialize(obj: Any): Any 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 d4362a91d992c..76298f03c94ae 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 @@ -25,8 +25,9 @@ import org.scalactic.TripleEqualsSupport.Spread import org.scalatest.FunSuite import org.scalatest.Matchers._ -import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.UnresolvedGetField +import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.types._ @@ -59,6 +60,10 @@ class ExpressionEvaluationBaseSuite extends FunSuite { class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { + def create_row(values: Any*): Row = { + new GenericRow(values.map(CatalystTypeConverters.convertToCatalyst).toArray) + } + test("literals") { checkEvaluation(Literal(1), 1) checkEvaluation(Literal(true), true) @@ -265,24 +270,23 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("LIKE Non-literal Regular Expression") { val regEx = 'a.string.at(0) - checkEvaluation("abcd" like regEx, null, new GenericRow(Array[Any](null))) - checkEvaluation("abdef" like regEx, true, new GenericRow(Array[Any]("abdef"))) - checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a\\__b"))) - checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a_%b"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a\\__b"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a%\\%b"))) - checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a%\\%b"))) - checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a%"))) - checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("**"))) - checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) - checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) - checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) - checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a_b"))) - checkEvaluation("ab" like regEx, true, new GenericRow(Array[Any]("a%b"))) - checkEvaluation("a\nb" like regEx, true, new GenericRow(Array[Any]("a%b"))) - - checkEvaluation(Literal.create(null, StringType) like regEx, null, - new GenericRow(Array[Any]("bc%"))) + checkEvaluation("abcd" like regEx, null, create_row(null)) + checkEvaluation("abdef" like regEx, true, create_row("abdef")) + checkEvaluation("a_%b" like regEx, true, create_row("a\\__b")) + checkEvaluation("addb" like regEx, true, create_row("a_%b")) + checkEvaluation("addb" like regEx, false, create_row("a\\__b")) + checkEvaluation("addb" like regEx, false, create_row("a%\\%b")) + checkEvaluation("a_%b" like regEx, true, create_row("a%\\%b")) + checkEvaluation("addb" like regEx, true, create_row("a%")) + checkEvaluation("addb" like regEx, false, create_row("**")) + checkEvaluation("abc" like regEx, true, create_row("a%")) + checkEvaluation("abc" like regEx, false, create_row("b%")) + checkEvaluation("abc" like regEx, false, create_row("bc%")) + checkEvaluation("a\nb" like regEx, true, create_row("a_b")) + checkEvaluation("ab" like regEx, true, create_row("a%b")) + checkEvaluation("a\nb" like regEx, true, create_row("a%b")) + + checkEvaluation(Literal.create(null, StringType) like regEx, null, create_row("bc%")) } test("RLIKE literal Regular Expression") { @@ -313,14 +317,14 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("RLIKE Non-literal Regular Expression") { val regEx = 'a.string.at(0) - checkEvaluation("abdef" rlike regEx, true, new GenericRow(Array[Any]("abdef"))) - checkEvaluation("abbbbc" rlike regEx, true, new GenericRow(Array[Any]("a.*c"))) - checkEvaluation("fofo" rlike regEx, true, new GenericRow(Array[Any]("^fo"))) - checkEvaluation("fo\no" rlike regEx, true, new GenericRow(Array[Any]("^fo\no$"))) - checkEvaluation("Bn" rlike regEx, true, new GenericRow(Array[Any]("^Ba*n"))) + checkEvaluation("abdef" rlike regEx, true, create_row("abdef")) + checkEvaluation("abbbbc" rlike regEx, true, create_row("a.*c")) + checkEvaluation("fofo" rlike regEx, true, create_row("^fo")) + checkEvaluation("fo\no" rlike regEx, true, create_row("^fo\no$")) + checkEvaluation("Bn" rlike regEx, true, create_row("^Ba*n")) intercept[java.util.regex.PatternSyntaxException] { - evaluate("abbbbc" rlike regEx, new GenericRow(Array[Any]("**"))) + evaluate("abbbbc" rlike regEx, create_row("**")) } } @@ -763,7 +767,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("null checking") { - val row = new GenericRow(Array[Any]("^Ba*n", null, true, null)) + val row = create_row("^Ba*n", null, true, null) val c1 = 'a.string.at(0) val c2 = 'a.string.at(1) val c3 = 'a.boolean.at(2) @@ -803,7 +807,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("case when") { - val row = new GenericRow(Array[Any](null, false, true, "a", "b", "c")) + val row = create_row(null, false, true, "a", "b", "c") val c1 = 'a.boolean.at(0) val c2 = 'a.boolean.at(1) val c3 = 'a.boolean.at(2) @@ -846,13 +850,13 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("complex type") { - val row = new GenericRow(Array[Any]( - "^Ba*n", // 0 - null.asInstanceOf[String], // 1 - new GenericRow(Array[Any]("aa", "bb")), // 2 - Map("aa"->"bb"), // 3 - Seq("aa", "bb") // 4 - )) + val row = create_row( + "^Ba*n", // 0 + null.asInstanceOf[UTF8String], // 1 + create_row("aa", "bb"), // 2 + Map("aa"->"bb"), // 3 + Seq("aa", "bb") // 4 + ) val typeS = StructType( StructField("a", StringType, true) :: StructField("b", StringType, true) :: Nil @@ -909,7 +913,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("arithmetic") { - val row = new GenericRow(Array[Any](1, 2, 3, null)) + val row = create_row(1, 2, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) @@ -934,7 +938,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("fractional arithmetic") { - val row = new GenericRow(Array[Any](1.1, 2.0, 3.1, null)) + val row = create_row(1.1, 2.0, 3.1, null) val c1 = 'a.double.at(0) val c2 = 'a.double.at(1) val c3 = 'a.double.at(2) @@ -958,7 +962,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("BinaryComparison") { - val row = new GenericRow(Array[Any](1, 2, 3, null, 3, null)) + val row = create_row(1, 2, 3, null, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) @@ -988,7 +992,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("StringComparison") { - val row = new GenericRow(Array[Any]("abc", null)) + val row = create_row("abc", null) val c1 = 'a.string.at(0) val c2 = 'a.string.at(1) @@ -1009,7 +1013,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { } test("Substring") { - val row = new GenericRow(Array[Any]("example", "example".toArray.map(_.toByte))) + val row = create_row("example", "example".toArray.map(_.toByte)) val s = 'a.string.at(0) @@ -1053,7 +1057,7 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { // substring(null, _, _) -> null checkEvaluation(Substring(s, Literal.create(100, IntegerType), Literal.create(4, IntegerType)), - null, new GenericRow(Array[Any](null))) + null, create_row(null)) // substring(_, null, _) -> null checkEvaluation(Substring(s, Literal.create(null, IntegerType), Literal.create(4, IntegerType)), @@ -1102,20 +1106,20 @@ class ExpressionEvaluationSuite extends ExpressionEvaluationBaseSuite { test("SQRT") { val inputSequence = (1 to (1<<24) by 511).map(_ * (1L<<24)) val expectedResults = inputSequence.map(l => math.sqrt(l.toDouble)) - val rowSequence = inputSequence.map(l => new GenericRow(Array[Any](l.toDouble))) + val rowSequence = inputSequence.map(l => create_row(l.toDouble)) val d = 'a.double.at(0) for ((row, expected) <- rowSequence zip expectedResults) { checkEvaluation(Sqrt(d), expected, row) } - checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, new GenericRow(Array[Any](null))) + checkEvaluation(Sqrt(Literal.create(null, DoubleType)), null, create_row(null)) checkEvaluation(Sqrt(-1), null, EmptyRow) checkEvaluation(Sqrt(-1.5), null, EmptyRow) } test("Bitwise operations") { - val row = new GenericRow(Array[Any](1, 2, 3, null)) + val row = create_row(1, 2, 3, null) val c1 = 'a.int.at(0) val c2 = 'a.int.at(1) val c3 = 'a.int.at(2) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala index 275ea2627ebcd..bcc0c404d2cfb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/GeneratedMutableEvaluationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions.codegen._ /** @@ -43,7 +43,7 @@ class GeneratedMutableEvaluationSuite extends ExpressionEvaluationSuite { } val actual = plan(inputRow) - val expectedRow = new GenericRow(Array[Any](expected)) + val expectedRow = new GenericRow(Array[Any](CatalystTypeConverters.convertToCatalyst(expected))) if (actual.hashCode() != expectedRow.hashCode()) { fail( s""" diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala new file mode 100644 index 0000000000000..a22aa6f244c48 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/UTF8StringSuite.scala @@ -0,0 +1,70 @@ +/* +* 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.types + +import org.scalatest.FunSuite + +// scalastyle:off +class UTF8StringSuite extends FunSuite { + test("basic") { + def check(str: String, len: Int) { + + assert(UTF8String(str).length == len) + assert(UTF8String(str.getBytes("utf8")).length() == len) + + assert(UTF8String(str) == str) + assert(UTF8String(str.getBytes("utf8")) == str) + assert(UTF8String(str).toString == str) + assert(UTF8String(str.getBytes("utf8")).toString == str) + assert(UTF8String(str.getBytes("utf8")) == UTF8String(str)) + + assert(UTF8String(str).hashCode() == UTF8String(str.getBytes("utf8")).hashCode()) + } + + check("hello", 5) + check("世 界", 3) + } + + test("contains") { + assert(UTF8String("hello").contains(UTF8String("ello"))) + assert(!UTF8String("hello").contains(UTF8String("vello"))) + assert(UTF8String("大千世界").contains(UTF8String("千世"))) + assert(!UTF8String("大千世界").contains(UTF8String("世千"))) + } + + test("prefix") { + assert(UTF8String("hello").startsWith(UTF8String("hell"))) + assert(!UTF8String("hello").startsWith(UTF8String("ell"))) + assert(UTF8String("大千世界").startsWith(UTF8String("大千"))) + assert(!UTF8String("大千世界").startsWith(UTF8String("千"))) + } + + test("suffix") { + assert(UTF8String("hello").endsWith(UTF8String("ello"))) + assert(!UTF8String("hello").endsWith(UTF8String("ellov"))) + assert(UTF8String("大千世界").endsWith(UTF8String("世界"))) + assert(!UTF8String("大千世界").endsWith(UTF8String("世"))) + } + + test("slice") { + assert(UTF8String("hello").slice(1, 3) == UTF8String("el")) + assert(UTF8String("大千世界").slice(0, 1) == UTF8String("大")) + assert(UTF8String("大千世界").slice(1, 3) == UTF8String("千世")) + assert(UTF8String("大千世界").slice(3, 5) == UTF8String("界")) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index b237fe684cdc1..89a4faf35e0d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -1195,6 +1195,7 @@ class SQLContext(@transient val sparkContext: SparkContext) case FloatType => true case DateType => true case TimestampType => true + case StringType => true case ArrayType(_, _) => true case MapType(_, _, _) => true case StructType(_) => true diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 87a6631da8300..b0f983c180673 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -216,13 +216,13 @@ private[sql] class IntColumnStats extends ColumnStats { } private[sql] class StringColumnStats extends ColumnStats { - protected var upper: String = null - protected var lower: String = null + protected var upper: UTF8String = null + protected var lower: UTF8String = null override def gatherStats(row: Row, ordinal: Int): Unit = { super.gatherStats(row, ordinal) if (!row.isNullAt(ordinal)) { - val value = row.getString(ordinal) + val value = row(ordinal).asInstanceOf[UTF8String] if (upper == null || value.compareTo(upper) > 0) upper = value if (lower == null || value.compareTo(lower) < 0) lower = value sizeInBytes += STRING.actualSize(row, ordinal) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index c47497e0662d9..1b9e0df2dcb5e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import scala.reflect.runtime.universe.TypeTag @@ -312,26 +312,28 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { row.getString(ordinal).getBytes("utf-8").length + 4 } - override def append(v: String, buffer: ByteBuffer): Unit = { - val stringBytes = v.getBytes("utf-8") + override def append(v: UTF8String, buffer: ByteBuffer): Unit = { + val stringBytes = v.getBytes buffer.putInt(stringBytes.length).put(stringBytes, 0, stringBytes.length) } - override def extract(buffer: ByteBuffer): String = { + override def extract(buffer: ByteBuffer): UTF8String = { val length = buffer.getInt() val stringBytes = new Array[Byte](length) buffer.get(stringBytes, 0, length) - new String(stringBytes, "utf-8") + UTF8String(stringBytes) } - override def setField(row: MutableRow, ordinal: Int, value: String): Unit = { - row.setString(ordinal, value) + override def setField(row: MutableRow, ordinal: Int, value: UTF8String): Unit = { + row.update(ordinal, value) } - override def getField(row: Row, ordinal: Int): String = row.getString(ordinal) + override def getField(row: Row, ordinal: Int): UTF8String = { + row(ordinal).asInstanceOf[UTF8String] + } override def copyField(from: Row, fromOrdinal: Int, to: MutableRow, toOrdinal: Int): Unit = { - to.setString(toOrdinal, from.getString(fromOrdinal)) + to.update(toOrdinal, from(fromOrdinal)) } } 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 656bdd7212f56..1fd387eec7e57 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,12 +19,12 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.expressions.{SpecificMutableRow, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, GenericMutableRow, SpecificMutableRow} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Statistics} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{Row, SQLContext} /** * :: DeveloperApi :: @@ -54,6 +54,33 @@ object RDDConversions { } } } + + /** + * Convert the objects inside Row into the types Catalyst expected. + */ + def rowToRowRdd(data: RDD[Row], schema: StructType): RDD[Row] = { + data.mapPartitions { iterator => + if (iterator.isEmpty) { + Iterator.empty + } else { + val bufferedIterator = iterator.buffered + val mutableRow = new GenericMutableRow(bufferedIterator.head.toSeq.toArray) + val schemaFields = schema.fields.toArray + val converters = schemaFields.map { + f => CatalystTypeConverters.createToCatalystConverter(f.dataType) + } + bufferedIterator.map { r => + var i = 0 + while (i < mutableRow.length) { + mutableRow(i) = converters(i)(r(i)) + i += 1 + } + + mutableRow + } + } + } + } } /** Logical plan node for scanning data from an RDD. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala index fad7a281dc1e2..99f24910fd61f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/commands.scala @@ -20,12 +20,13 @@ package org.apache.spark.sql.execution import org.apache.spark.Logging import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD -import org.apache.spark.sql.types.{BooleanType, StructField, StructType, StringType} -import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row, Attribute} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types._ +import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext} /** * A logical command that is executed for its side-effects. `RunnableCommand`s are @@ -61,7 +62,11 @@ case class ExecutedCommand(cmd: RunnableCommand) extends SparkPlan { override def executeTake(limit: Int): Array[Row] = sideEffectResult.take(limit).toArray - override def execute(): RDD[Row] = sqlContext.sparkContext.parallelize(sideEffectResult, 1) + override def execute(): RDD[Row] = { + val converted = sideEffectResult.map(r => + CatalystTypeConverters.convertToCatalyst(r, schema).asInstanceOf[Row]) + sqlContext.sparkContext.parallelize(converted, 1) + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala index e916e68e58b5d..710787096e6cb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug/package.scala @@ -164,7 +164,7 @@ package object debug { case (_: Long, LongType) => case (_: Int, IntegerType) => - case (_: String, StringType) => + case (_: UTF8String, StringType) => case (_: Float, FloatType) => case (_: Byte, ByteType) => case (_: Short, ShortType) => 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 5b308d88d4cdf..7a43bfd8bc8d9 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 @@ -140,6 +140,7 @@ object EvaluatePython { case (ud, udt: UserDefinedType[_]) => toJava(udt.serialize(ud), udt.sqlType) case (date: Int, DateType) => DateUtils.toJavaDate(date) + case (s: UTF8String, StringType) => s.toString // Pyrolite can handle Timestamp and Decimal case (other, _) => other @@ -192,7 +193,8 @@ object EvaluatePython { case (c: Long, IntegerType) => c.toInt case (c: Int, LongType) => c.toLong case (c: Double, FloatType) => c.toFloat - case (c, StringType) if !c.isInstanceOf[String] => c.toString + case (c: String, StringType) => UTF8String(c) + case (c, StringType) if !c.isInstanceOf[String] => UTF8String(c.toString) case (c, _) => c } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 463e1dcc268bc..b9022fcd9e3ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -233,7 +233,7 @@ private[sql] class JDBCRDD( * Converts value to SQL expression. */ private def compileValue(value: Any): Any = value match { - case stringValue: String => s"'${escapeSql(stringValue)}'" + case stringValue: UTF8String => s"'${escapeSql(stringValue.toString)}'" case _ => value } @@ -349,12 +349,14 @@ private[sql] class JDBCRDD( val pos = i + 1 conversions(i) match { case BooleanConversion => mutableRow.setBoolean(i, rs.getBoolean(pos)) + // TODO(davies): convert Date into Int case DateConversion => mutableRow.update(i, rs.getDate(pos)) case DecimalConversion => mutableRow.update(i, rs.getBigDecimal(pos)) case DoubleConversion => mutableRow.setDouble(i, rs.getDouble(pos)) case FloatConversion => mutableRow.setFloat(i, rs.getFloat(pos)) case IntegerConversion => mutableRow.setInt(i, rs.getInt(pos)) case LongConversion => mutableRow.setLong(i, rs.getLong(pos)) + // TODO(davies): use getBytes for better performance, if the encoding is UTF-8 case StringConversion => mutableRow.setString(i, rs.getString(pos)) case TimestampConversion => mutableRow.update(i, rs.getTimestamp(pos)) case BinaryConversion => mutableRow.update(i, rs.getBytes(pos)) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index 4fa84dc076f7e..99b755c9f25d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -130,6 +130,8 @@ private[sql] case class JDBCRelation( extends BaseRelation with PrunedFilteredScan { + override val needConversion: Boolean = false + override val schema: StructType = JDBCRDD.resolveTable(url, table, properties) override def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala index 34f864f5fda7a..d4e0abc040bc6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/jdbc.scala @@ -18,11 +18,8 @@ package org.apache.spark.sql import java.sql.{Connection, DriverManager, PreparedStatement} -import org.apache.spark.{Logging, Partition} -import org.apache.spark.sql._ -import org.apache.spark.sql.sources.LogicalRelation -import org.apache.spark.sql.jdbc.{JDBCPartitioningInfo, JDBCRelation, JDBCPartition} +import org.apache.spark.Logging import org.apache.spark.sql.types._ package object jdbc { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala index f4c99b4b56606..e3352d02787fd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JSONRelation.scala @@ -20,12 +20,12 @@ package org.apache.spark.sql.json import java.io.IOException import org.apache.hadoop.fs.Path + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row - -import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} import org.apache.spark.sql.sources._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.{DataFrame, SQLContext, SaveMode} private[sql] class DefaultSource @@ -113,6 +113,8 @@ private[sql] case class JSONRelation( // TODO: Support partitioned JSON relation. private def baseRDD = sqlContext.sparkContext.textFile(path) + override val needConversion: Boolean = false + override val schema = userSpecifiedSchema.getOrElse( JsonRDD.nullTypeToStringType( JsonRDD.inferSchema( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index b1e8521383756..29de7401dda71 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -409,7 +409,7 @@ private[sql] object JsonRDD extends Logging { null } else { desiredType match { - case StringType => toString(value) + case StringType => UTF8String(toString(value)) case _ if value == null || value == "" => null // guard the non string type case IntegerType => value.asInstanceOf[IntegerType.JvmType] case LongType => toLong(value) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala index 43ca359b51735..bc108e37dfb0f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetConverter.scala @@ -219,8 +219,8 @@ private[parquet] abstract class CatalystConverter extends GroupConverter { protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = updateField(fieldIndex, value.getBytes) - protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = - updateField(fieldIndex, value) + protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = + updateField(fieldIndex, UTF8String(value)) protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = updateField(fieldIndex, readTimestamp(value)) @@ -418,8 +418,8 @@ private[parquet] class CatalystPrimitiveRowConverter( override protected[parquet] def updateBinary(fieldIndex: Int, value: Binary): Unit = current.update(fieldIndex, value.getBytes) - override protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = - current.setString(fieldIndex, value) + override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = + current.update(fieldIndex, UTF8String(value)) override protected[parquet] def updateTimestamp(fieldIndex: Int, value: Binary): Unit = current.update(fieldIndex, readTimestamp(value)) @@ -475,19 +475,18 @@ private[parquet] class CatalystPrimitiveConverter( private[parquet] class CatalystPrimitiveStringConverter(parent: CatalystConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { - private[this] var dict: Array[String] = null + private[this] var dict: Array[Array[Byte]] = null override def hasDictionarySupport: Boolean = true override def setDictionary(dictionary: Dictionary):Unit = - dict = Array.tabulate(dictionary.getMaxId + 1) {dictionary.decodeToBinary(_).toStringUsingUTF8} - + dict = Array.tabulate(dictionary.getMaxId + 1) { dictionary.decodeToBinary(_).getBytes } override def addValueFromDictionary(dictionaryId: Int): Unit = parent.updateString(fieldIndex, dict(dictionaryId)) override def addBinary(value: Binary): Unit = - parent.updateString(fieldIndex, value.toStringUsingUTF8) + parent.updateString(fieldIndex, value.getBytes) } private[parquet] object CatalystArrayConverter { @@ -714,9 +713,9 @@ private[parquet] class CatalystNativeArrayConverter( elements += 1 } - override protected[parquet] def updateString(fieldIndex: Int, value: String): Unit = { + override protected[parquet] def updateString(fieldIndex: Int, value: Array[Byte]): Unit = { checkGrowBuffer() - buffer(elements) = value.asInstanceOf[NativeType] + buffer(elements) = UTF8String(value).asInstanceOf[NativeType] elements += 1 } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala index 0357dcc4688be..5eb1c6abc2432 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetFilters.scala @@ -55,7 +55,7 @@ private[sql] object ParquetFilters { case StringType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[UTF8String].getBytes)).orNull) case BinaryType => (n: String, v: Any) => FilterApi.eq( binaryColumn(n), @@ -76,7 +76,7 @@ private[sql] object ParquetFilters { case StringType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), - Option(v).map(s => Binary.fromString(s.asInstanceOf[String])).orNull) + Option(v).map(s => Binary.fromByteArray(s.asInstanceOf[UTF8String].getBytes)).orNull) case BinaryType => (n: String, v: Any) => FilterApi.notEq( binaryColumn(n), @@ -94,7 +94,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.lt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.lt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.lt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -111,7 +111,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.ltEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.ltEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.ltEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -128,7 +128,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gt(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.gt(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.gt(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) @@ -145,7 +145,7 @@ private[sql] object ParquetFilters { (n: String, v: Any) => FilterApi.gtEq(doubleColumn(n), v.asInstanceOf[java.lang.Double]) case StringType => (n: String, v: Any) => - FilterApi.gtEq(binaryColumn(n), Binary.fromString(v.asInstanceOf[String])) + FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[UTF8String].getBytes)) case BinaryType => (n: String, v: Any) => FilterApi.gtEq(binaryColumn(n), Binary.fromByteArray(v.asInstanceOf[Array[Byte]])) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala index 5a1b15490d273..e05a4c20b0d41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -198,10 +198,7 @@ private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { if (value != null) { schema match { case StringType => writer.addBinary( - Binary.fromByteArray( - value.asInstanceOf[String].getBytes("utf-8") - ) - ) + Binary.fromByteArray(value.asInstanceOf[UTF8String].getBytes)) case BinaryType => writer.addBinary( Binary.fromByteArray(value.asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(value.asInstanceOf[Int]) @@ -349,7 +346,7 @@ private[parquet] class MutableRowWriteSupport extends RowWriteSupport { index: Int): Unit = { ctype match { case StringType => writer.addBinary( - Binary.fromByteArray(record(index).asInstanceOf[String].getBytes("utf-8"))) + Binary.fromByteArray(record(index).asInstanceOf[UTF8String].getBytes)) case BinaryType => writer.addBinary( Binary.fromByteArray(record(index).asInstanceOf[Array[Byte]])) case IntegerType => writer.addInteger(record.getInt(index)) 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 20fdf5e58ef82..af7b3c81ae7b2 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 @@ -33,7 +33,6 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.{InputSplit, Job, JobContext} - import parquet.filter2.predicate.FilterApi import parquet.format.converter.ParquetMetadataConverter import parquet.hadoop.metadata.CompressionCodecName @@ -45,13 +44,13 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD, RDD} -import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, expressions} import org.apache.spark.sql.parquet.ParquetTypesConverter._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{IntegerType, StructField, StructType, _} import org.apache.spark.sql.{DataFrame, Row, SQLConf, SQLContext, SaveMode} -import org.apache.spark.{Logging, Partition => SparkPartition, SerializableWritable, SparkException, TaskContext} +import org.apache.spark.{Logging, SerializableWritable, SparkException, TaskContext, Partition => SparkPartition} /** * Allows creation of Parquet based tables using the syntax: @@ -409,6 +408,9 @@ private[sql] case class ParquetRelation2( file.getName == ParquetFileWriter.PARQUET_METADATA_FILE } + // Skip type conversion + override val needConversion: Boolean = false + // TODO Should calculate per scan size // It's common that a query only scans a fraction of a large Parquet file. Returning size of the // whole Parquet file disables some optimizations in this case (e.g. broadcast join). @@ -550,7 +552,8 @@ private[sql] case class ParquetRelation2( baseRDD.mapPartitionsWithInputSplit { case (split: ParquetInputSplit, iterator) => val partValues = selectedPartitions.collectFirst { - case p if split.getPath.getParent.toString == p.path => p.values + case p if split.getPath.getParent.toString == p.path => + CatalystTypeConverters.convertToCatalyst(p.values).asInstanceOf[Row] }.get val requiredPartOrdinal = partitionKeyLocations.keys.toSeq diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala index 34d048e426d10..b3d71f687a60a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/DataSourceStrategy.scala @@ -23,7 +23,8 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.types.{UTF8String, StringType} import org.apache.spark.sql.{Row, Strategy, execution, sources} /** @@ -53,7 +54,7 @@ private[sql] object DataSourceStrategy extends Strategy { (a, _) => t.buildScan(a)) :: Nil case l @ LogicalRelation(t: TableScan) => - execution.PhysicalRDD(l.output, t.buildScan()) :: Nil + createPhysicalRDD(l.relation, l.output, t.buildScan()) :: Nil case i @ logical.InsertIntoTable( l @ LogicalRelation(t: InsertableRelation), part, query, overwrite, false) if part.isEmpty => @@ -102,20 +103,30 @@ private[sql] object DataSourceStrategy extends Strategy { projectList.asInstanceOf[Seq[Attribute]] // Safe due to if above. .map(relation.attributeMap) // Match original case of attributes. - val scan = - execution.PhysicalRDD( - projectList.map(_.toAttribute), + val scan = createPhysicalRDD(relation.relation, projectList.map(_.toAttribute), scanBuilder(requestedColumns, pushedFilters)) filterCondition.map(execution.Filter(_, scan)).getOrElse(scan) } else { val requestedColumns = (projectSet ++ filterSet).map(relation.attributeMap).toSeq - val scan = - execution.PhysicalRDD(requestedColumns, scanBuilder(requestedColumns, pushedFilters)) + val scan = createPhysicalRDD(relation.relation, requestedColumns, + scanBuilder(requestedColumns, pushedFilters)) execution.Project(projectList, filterCondition.map(execution.Filter(_, scan)).getOrElse(scan)) } } + private[this] def createPhysicalRDD( + relation: BaseRelation, + output: Seq[Attribute], + rdd: RDD[Row]): SparkPlan = { + val converted = if (relation.needConversion) { + execution.RDDConversions.rowToRowRdd(rdd, relation.schema) + } else { + rdd + } + execution.PhysicalRDD(output, converted) + } + /** * Selects Catalyst predicate [[Expression]]s which are convertible into data source [[Filter]]s, * and convert them. @@ -167,14 +178,14 @@ private[sql] object DataSourceStrategy extends Strategy { case expressions.Not(child) => translate(child).map(sources.Not) - case expressions.StartsWith(a: Attribute, Literal(v: String, StringType)) => - Some(sources.StringStartsWith(a.name, v)) + case expressions.StartsWith(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringStartsWith(a.name, v.toString)) - case expressions.EndsWith(a: Attribute, Literal(v: String, StringType)) => - Some(sources.StringEndsWith(a.name, v)) + case expressions.EndsWith(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringEndsWith(a.name, v.toString)) - case expressions.Contains(a: Attribute, Literal(v: String, StringType)) => - Some(sources.StringContains(a.name, v)) + case expressions.Contains(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringContains(a.name, v.toString)) case _ => None } 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 8f9946a5a801e..ca53dcdb92c52 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 @@ -126,6 +126,16 @@ abstract class BaseRelation { * could lead to execution plans that are suboptimal (i.e. broadcasting a very large table). */ def sizeInBytes: Long = sqlContext.conf.defaultSizeInBytes + + /** + * Whether does it need to convert the objects in Row to internal representation, for example: + * java.lang.String -> UTF8String + * java.lang.Decimal -> Decimal + * + * Note: The internal representation is not stable across releases and thus data sources outside + * of Spark SQL should leave this as true. + */ + def needConversion: Boolean = true } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 36465cc2fa11a..bf6cf1321a056 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -30,7 +30,7 @@ class RowSuite extends FunSuite { test("create row") { val expected = new GenericMutableRow(4) expected.update(0, 2147483647) - expected.update(1, "this is a string") + expected.setString(1, "this is a string") expected.update(2, false) expected.update(3, null) val actual1 = Row(2147483647, "this is a string", false, null) 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 0174aaee94246..4c48dca44498b 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 @@ -17,18 +17,14 @@ package org.apache.spark.sql -import org.apache.spark.sql.execution.GeneratedAggregate -import org.apache.spark.sql.test.TestSQLContext import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql.TestData._ import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.functions._ -import org.apache.spark.sql.catalyst.errors.TreeNodeException -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.types._ - -import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} +import org.apache.spark.sql.types._ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 5f08834f73c6b..c86ef338fc644 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -65,7 +65,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(FIXED_DECIMAL(15, 10), Decimal(0, 15, 10), 8) checkActualSize(BOOLEAN, true, 1) - checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) + checkActualSize(STRING, UTF8String("hello"), 4 + "hello".getBytes("utf-8").length) checkActualSize(DATE, 0, 4) checkActualSize(TIMESTAMP, new Timestamp(0L), 12) @@ -108,8 +108,8 @@ class ColumnTypeSuite extends FunSuite with Logging { testNativeColumnType[StringType.type]( STRING, - (buffer: ByteBuffer, string: String) => { - val bytes = string.getBytes("utf-8") + (buffer: ByteBuffer, string: UTF8String) => { + val bytes = string.getBytes buffer.putInt(bytes.length) buffer.put(bytes) }, @@ -117,7 +117,7 @@ class ColumnTypeSuite extends FunSuite with Logging { val length = buffer.getInt() val bytes = new Array[Byte](length) buffer.get(bytes) - new String(bytes, "utf-8") + UTF8String(bytes) }) testColumnType[BinaryType.type, Array[Byte]]( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala index b301818a008e7..f76314b9dab5e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -24,7 +24,7 @@ import scala.util.Random import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.types.{Decimal, DataType, NativeType} +import org.apache.spark.sql.types.{UTF8String, DataType, Decimal, NativeType} object ColumnarTestUtils { def makeNullRow(length: Int): GenericMutableRow = { @@ -48,7 +48,7 @@ object ColumnarTestUtils { case FLOAT => Random.nextFloat() case DOUBLE => Random.nextDouble() case FIXED_DECIMAL(precision, scale) => Decimal(Random.nextLong() % 100, precision, scale) - case STRING => Random.nextString(Random.nextInt(32)) + case STRING => UTF8String(Random.nextString(Random.nextInt(32))) case BOOLEAN => Random.nextBoolean() case BINARY => randomBytes(Random.nextInt(32)) case DATE => Random.nextInt() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala index 60c8c00bda4d5..3b47b8adf313b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/TableScanSuite.scala @@ -74,7 +74,7 @@ case class AllDataTypesScan( i.toDouble, new java.math.BigDecimal(i), new java.math.BigDecimal(i), - new Date((i + 1) * 8640000), + new Date(1970, 1, 1), new Timestamp(20000 + i), s"varchar_$i", Seq(i, i + 1), @@ -82,7 +82,7 @@ case class AllDataTypesScan( Map(i -> i.toString), Map(Map(s"str_$i" -> i.toFloat) -> Row(i.toLong)), Row(i, i.toString), - Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date((i + 2) * 8640000))))) + Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date(1970, 1, i + 1))))) } } } @@ -103,7 +103,7 @@ class TableScanSuite extends DataSourceTest { i.toDouble, new java.math.BigDecimal(i), new java.math.BigDecimal(i), - new Date((i + 1) * 8640000), + new Date(1970, 1, 1), new Timestamp(20000 + i), s"varchar_$i", Seq(i, i + 1), @@ -111,7 +111,7 @@ class TableScanSuite extends DataSourceTest { Map(i -> i.toString), Map(Map(s"str_$i" -> i.toFloat) -> Row(i.toLong)), Row(i, i.toString), - Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date((i + 2) * 8640000))))) + Row(Seq(s"str_$i", s"str_${i + 1}"), Row(Seq(new Date(1970, 1, i + 1))))) }.toSeq before { @@ -266,7 +266,7 @@ class TableScanSuite extends DataSourceTest { sqlTest( "SELECT structFieldComplex.Value.`value_(2)` FROM tableWithSchema", - (1 to 10).map(i => Row(Seq(new Date((i + 2) * 8640000)))).toSeq) + (1 to 10).map(i => Row(Seq(new Date(1970, 1, i + 1)))).toSeq) test("Caching") { // Cached Query Execution diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala index 921c6194c7b76..74ae984f34866 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveInspectors.scala @@ -34,7 +34,7 @@ import scala.collection.JavaConversions._ * 1. The Underlying data type in catalyst and in Hive * In catalyst: * Primitive => - * java.lang.String + * UTF8String * int / scala.Int * boolean / scala.Boolean * float / scala.Float @@ -239,9 +239,10 @@ private[hive] trait HiveInspectors { */ def unwrap(data: Any, oi: ObjectInspector): Any = oi match { case coi: ConstantObjectInspector if coi.getWritableConstantValue == null => null - case poi: WritableConstantStringObjectInspector => poi.getWritableConstantValue.toString + case poi: WritableConstantStringObjectInspector => + UTF8String(poi.getWritableConstantValue.toString) case poi: WritableConstantHiveVarcharObjectInspector => - poi.getWritableConstantValue.getHiveVarchar.getValue + UTF8String(poi.getWritableConstantValue.getHiveVarchar.getValue) case poi: WritableConstantHiveDecimalObjectInspector => HiveShim.toCatalystDecimal( PrimitiveObjectInspectorFactory.javaHiveDecimalObjectInspector, @@ -284,10 +285,13 @@ private[hive] trait HiveInspectors { case pi: PrimitiveObjectInspector => pi match { // We think HiveVarchar is also a String case hvoi: HiveVarcharObjectInspector if hvoi.preferWritable() => - hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue - case hvoi: HiveVarcharObjectInspector => hvoi.getPrimitiveJavaObject(data).getValue + UTF8String(hvoi.getPrimitiveWritableObject(data).getHiveVarchar.getValue) + case hvoi: HiveVarcharObjectInspector => + UTF8String(hvoi.getPrimitiveJavaObject(data).getValue) case x: StringObjectInspector if x.preferWritable() => - x.getPrimitiveWritableObject(data).toString + UTF8String(x.getPrimitiveWritableObject(data).toString) + case x: StringObjectInspector => + UTF8String(x.getPrimitiveJavaObject(data)) case x: IntObjectInspector if x.preferWritable() => x.get(data) case x: BooleanObjectInspector if x.preferWritable() => x.get(data) case x: FloatObjectInspector if x.preferWritable() => x.get(data) @@ -340,7 +344,9 @@ private[hive] trait HiveInspectors { */ protected def wrapperFor(oi: ObjectInspector): Any => Any = oi match { case _: JavaHiveVarcharObjectInspector => - (o: Any) => new HiveVarchar(o.asInstanceOf[String], o.asInstanceOf[String].size) + (o: Any) => + val s = o.asInstanceOf[UTF8String].toString + new HiveVarchar(s, s.size) case _: JavaHiveDecimalObjectInspector => (o: Any) => HiveShim.createDecimal(o.asInstanceOf[Decimal].toJavaBigDecimal) @@ -409,7 +415,7 @@ private[hive] trait HiveInspectors { case x: PrimitiveObjectInspector => x match { // TODO we don't support the HiveVarcharObjectInspector yet. case _: StringObjectInspector if x.preferWritable() => HiveShim.getStringWritable(a) - case _: StringObjectInspector => a.asInstanceOf[java.lang.String] + case _: StringObjectInspector => a.asInstanceOf[UTF8String].toString() case _: IntObjectInspector if x.preferWritable() => HiveShim.getIntWritable(a) case _: IntObjectInspector => a.asInstanceOf[java.lang.Integer] case _: BooleanObjectInspector if x.preferWritable() => HiveShim.getBooleanWritable(a) 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 1ccb0c279c60e..a6f4fbe8aba06 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 @@ -17,24 +17,21 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.catalyst.expressions.Row - import scala.collection.JavaConversions._ import org.apache.spark.annotation.Experimental import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GeneratePredicate +import org.apache.spark.sql.catalyst.expressions.{Row, _} 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.sources.DescribeCommand -import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand} -import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand, _} import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.sources.{CreateTableUsingAsSelect, CreateTableUsing} +import org.apache.spark.sql.sources.{CreateTableUsing, CreateTableUsingAsSelect, DescribeCommand} import org.apache.spark.sql.types.StringType @@ -131,7 +128,7 @@ private[hive] trait HiveStrategies { val partitionValues = part.getValues var i = 0 while (i < partitionValues.size()) { - inputData(i) = partitionValues(i) + inputData(i) = CatalystTypeConverters.convertToCatalyst(partitionValues(i)) i += 1 } pruningCondition(inputData) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala index 8efed7f0299bf..cab0fdd35723a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformation.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.hive.execution -import java.io.{BufferedReader, InputStreamReader} -import java.io.{DataInputStream, DataOutputStream, EOFException} +import java.io.{BufferedReader, DataInputStream, DataOutputStream, EOFException, InputStreamReader} import java.util.Properties import scala.collection.JavaConversions._ @@ -28,12 +27,13 @@ import org.apache.hadoop.hive.serde2.AbstractSerDe import org.apache.hadoop.hive.serde2.objectinspector._ import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.CatalystTypeConverters import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.ScriptInputOutputSchema import org.apache.spark.sql.execution._ -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} import org.apache.spark.sql.hive.HiveShim._ +import org.apache.spark.sql.hive.{HiveContext, HiveInspectors} +import org.apache.spark.sql.types.DataType import org.apache.spark.util.Utils /** @@ -121,14 +121,13 @@ case class ScriptTransformation( if (outputSerde == null) { val prevLine = curLine curLine = reader.readLine() - if (!ioschema.schemaLess) { - new GenericRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD")) + new GenericRow(CatalystTypeConverters.convertToCatalyst( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"))) .asInstanceOf[Array[Any]]) } else { - new GenericRow( - prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2) + new GenericRow(CatalystTypeConverters.convertToCatalyst( + prevLine.split(ioschema.outputRowFormatMap("TOK_TABLEROWFORMATFIELD"), 2)) .asInstanceOf[Array[Any]]) } } else { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala index 902a12785e3e9..a40a1e53117cd 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/commands.scala @@ -22,11 +22,11 @@ import org.apache.spark.sql.catalyst.analysis.EliminateSubQueries import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.sources._ import org.apache.spark.sql.{SaveMode, DataFrame, SQLContext} -import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.hive.HiveContext -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.types._ /** * Analyzes the given table in the current database to generate statistics, which will be @@ -76,6 +76,12 @@ case class DropTable( private[hive] case class AddJar(path: String) extends RunnableCommand { + override val output: Seq[Attribute] = { + val schema = StructType( + StructField("result", IntegerType, false) :: Nil) + schema.toAttributes + } + override def run(sqlContext: SQLContext): Seq[Row] = { val hiveContext = sqlContext.asInstanceOf[HiveContext] hiveContext.runSqlHive(s"ADD JAR $path") 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 0ed93c2c5b1fa..33e96eaabfbf6 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 @@ -41,7 +41,7 @@ import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, TypeInfoFactory} import org.apache.hadoop.io.{NullWritable, Writable} import org.apache.hadoop.mapred.InputFormat -import org.apache.spark.sql.types.{Decimal, DecimalType} +import org.apache.spark.sql.types.{UTF8String, Decimal, DecimalType} private[hive] case class HiveFunctionWrapper(functionClassName: String) extends java.io.Serializable { @@ -135,7 +135,7 @@ private[hive] object HiveShim { PrimitiveCategory.VOID, null) def getStringWritable(value: Any): hadoopIo.Text = - if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]) + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[UTF8String].toString) def getIntWritable(value: Any): hadoopIo.IntWritable = if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]) 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 7577309900209..d331c210e8939 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 @@ -17,37 +17,35 @@ package org.apache.spark.sql.hive -import java.util -import java.util.{ArrayList => JArrayList} -import java.util.Properties import java.rmi.server.UID +import java.util.{Properties, ArrayList => JArrayList} import scala.collection.JavaConversions._ import scala.language.implicitConversions +import com.esotericsoftware.kryo.Kryo import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.{NullWritable, Writable} -import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.hive.common.StatsSetupConst -import org.apache.hadoop.hive.common.`type`.{HiveDecimal} +import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Context -import org.apache.hadoop.hive.ql.metadata.{Table, Hive, Partition} +import org.apache.hadoop.hive.ql.exec.{UDF, Utilities} +import org.apache.hadoop.hive.ql.metadata.{Hive, Partition, Table} import org.apache.hadoop.hive.ql.plan.{CreateTableDesc, FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory import org.apache.hadoop.hive.serde.serdeConstants -import org.apache.hadoop.hive.serde2.typeinfo.{TypeInfo, DecimalTypeInfo, TypeInfoFactory} -import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} -import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, PrimitiveObjectInspector, ObjectInspector} -import org.apache.hadoop.hive.serde2.{Deserializer, ColumnProjectionUtils} -import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.hive.serde2.avro.AvroGenericRecordWritable +import org.apache.hadoop.hive.serde2.objectinspector.primitive.{HiveDecimalObjectInspector, PrimitiveObjectInspectorFactory} +import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspector, ObjectInspectorConverters, PrimitiveObjectInspector} +import org.apache.hadoop.hive.serde2.typeinfo.{DecimalTypeInfo, TypeInfo, TypeInfoFactory} +import org.apache.hadoop.hive.serde2.{ColumnProjectionUtils, Deserializer, io => hiveIo} +import org.apache.hadoop.io.{NullWritable, Writable} +import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.{io => hadoopIo} import org.apache.spark.Logging -import org.apache.spark.sql.types.{Decimal, DecimalType} - +import org.apache.spark.sql.types.{Decimal, DecimalType, UTF8String} /** * This class provides the UDF creation and also the UDF instance serialization and @@ -63,18 +61,14 @@ private[hive] case class HiveFunctionWrapper(var functionClassName: String) // 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[java.io.InputStream], classOf[Class[_]]) method.setAccessible(true) @@ -87,7 +81,7 @@ private[hive] case class HiveFunctionWrapper(var functionClassName: String) "serializeObjectByKryo", classOf[Kryo], classOf[Object], - classOf[OutputStream]) + classOf[java.io.OutputStream]) method.setAccessible(true) method @@ -224,7 +218,7 @@ private[hive] object HiveShim { TypeInfoFactory.voidTypeInfo, null) def getStringWritable(value: Any): hadoopIo.Text = - if (value == null) null else new hadoopIo.Text(value.asInstanceOf[String]) + if (value == null) null else new hadoopIo.Text(value.asInstanceOf[UTF8String].toString) def getIntWritable(value: Any): hadoopIo.IntWritable = if (value == null) null else new hadoopIo.IntWritable(value.asInstanceOf[Int]) From cf38fe04f8782ff4573ae106ec0de8e8d183cb2b Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Wed, 15 Apr 2015 13:15:58 -0700 Subject: [PATCH 108/128] [SPARK-6844][SQL] Clean up accumulators used in InMemoryRelation when it is uncached JIRA: https://issues.apache.org/jira/browse/SPARK-6844 Author: Liang-Chi Hsieh Closes #5475 from viirya/cache_memory_leak and squashes the following commits: 0b41235 [Liang-Chi Hsieh] fix style. dc1d5d5 [Liang-Chi Hsieh] For comments. 78af229 [Liang-Chi Hsieh] Merge remote-tracking branch 'upstream/master' into cache_memory_leak 26c9bb6 [Liang-Chi Hsieh] Add configuration to enable in-memory table scan accumulators. 1c3b06e [Liang-Chi Hsieh] Clean up accumulators used in InMemoryRelation when it is uncached. --- .../org/apache/spark/sql/CacheManager.scala | 2 +- .../columnar/InMemoryColumnarTableScan.scala | 47 ++++++++++++++----- .../apache/spark/sql/CachedTableSuite.scala | 18 +++++++ .../columnar/PartitionBatchPruningSuite.scala | 2 + 4 files changed, 55 insertions(+), 14 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala index ca4a127120b37..18584c2dcf797 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/CacheManager.scala @@ -112,7 +112,7 @@ private[sql] class CacheManager(sqlContext: SQLContext) extends Logging { val planToCache = query.queryExecution.analyzed val dataIndex = cachedData.indexWhere(cd => planToCache.sameResult(cd.plan)) require(dataIndex >= 0, s"Table $query is not cached.") - cachedData(dataIndex).cachedRepresentation.cachedColumnBuffers.unpersist(blocking) + cachedData(dataIndex).cachedRepresentation.uncache(blocking) cachedData.remove(dataIndex) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index 6eee0c86d6a1c..d9b6fb43ab83d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -19,13 +19,15 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import org.apache.spark.Accumulator +import org.apache.spark.{Accumulable, Accumulator, Accumulators} import org.apache.spark.sql.catalyst.expressions import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap import org.apache.spark.rdd.RDD import org.apache.spark.sql.Row +import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ @@ -53,11 +55,16 @@ private[sql] case class InMemoryRelation( child: SparkPlan, tableName: Option[String])( private var _cachedColumnBuffers: RDD[CachedBatch] = null, - private var _statistics: Statistics = null) + private var _statistics: Statistics = null, + private var _batchStats: Accumulable[ArrayBuffer[Row], Row] = null) extends LogicalPlan with MultiInstanceRelation { - private val batchStats = - child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[Row]) + private val batchStats: Accumulable[ArrayBuffer[Row], Row] = + if (_batchStats == null) { + child.sqlContext.sparkContext.accumulableCollection(ArrayBuffer.empty[Row]) + } else { + _batchStats + } val partitionStatistics = new PartitionStatistics(output) @@ -161,7 +168,7 @@ private[sql] case class InMemoryRelation( def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { InMemoryRelation( newOutput, useCompression, batchSize, storageLevel, child, tableName)( - _cachedColumnBuffers, statisticsToBePropagated) + _cachedColumnBuffers, statisticsToBePropagated, batchStats) } override def children: Seq[LogicalPlan] = Seq.empty @@ -175,13 +182,20 @@ private[sql] case class InMemoryRelation( child, tableName)( _cachedColumnBuffers, - statisticsToBePropagated).asInstanceOf[this.type] + statisticsToBePropagated, + batchStats).asInstanceOf[this.type] } def cachedColumnBuffers: RDD[CachedBatch] = _cachedColumnBuffers override protected def otherCopyArgs: Seq[AnyRef] = - Seq(_cachedColumnBuffers, statisticsToBePropagated) + Seq(_cachedColumnBuffers, statisticsToBePropagated, batchStats) + + private[sql] def uncache(blocking: Boolean): Unit = { + Accumulators.remove(batchStats.id) + cachedColumnBuffers.unpersist(blocking) + _cachedColumnBuffers = null + } } private[sql] case class InMemoryColumnarTableScan( @@ -244,15 +258,20 @@ private[sql] case class InMemoryColumnarTableScan( } } + lazy val enableAccumulators: Boolean = + sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean + // Accumulators used for testing purposes - val readPartitions: Accumulator[Int] = sparkContext.accumulator(0) - val readBatches: Accumulator[Int] = sparkContext.accumulator(0) + lazy val readPartitions: Accumulator[Int] = sparkContext.accumulator(0) + lazy val readBatches: Accumulator[Int] = sparkContext.accumulator(0) private val inMemoryPartitionPruningEnabled = sqlContext.conf.inMemoryPartitionPruning override def execute(): RDD[Row] = { - readPartitions.setValue(0) - readBatches.setValue(0) + if (enableAccumulators) { + readPartitions.setValue(0) + readBatches.setValue(0) + } relation.cachedColumnBuffers.mapPartitions { cachedBatchIterator => val partitionFilter = newPredicate( @@ -302,7 +321,7 @@ private[sql] case class InMemoryColumnarTableScan( } } - if (rows.hasNext) { + if (rows.hasNext && enableAccumulators) { readPartitions += 1 } @@ -321,7 +340,9 @@ private[sql] case class InMemoryColumnarTableScan( logInfo(s"Skipping partition based on stats $statsString") false } else { - readBatches += 1 + if (enableAccumulators) { + readBatches += 1 + } true } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index f7b5f08beb92f..01e3b8671071e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -22,6 +22,7 @@ import scala.language.{implicitConversions, postfixOps} import org.scalatest.concurrent.Eventually._ +import org.apache.spark.Accumulators import org.apache.spark.sql.TestData._ import org.apache.spark.sql.columnar._ import org.apache.spark.sql.test.TestSQLContext._ @@ -297,4 +298,21 @@ class CachedTableSuite extends QueryTest { sql("Clear CACHE") assert(cacheManager.isEmpty) } + + test("Clear accumulators when uncacheTable to prevent memory leaking") { + val accsSize = Accumulators.originals.size + + sql("SELECT key FROM testData LIMIT 10").registerTempTable("t1") + sql("SELECT key FROM testData LIMIT 5").registerTempTable("t2") + cacheTable("t1") + cacheTable("t2") + sql("SELECT * FROM t1").count() + sql("SELECT * FROM t2").count() + sql("SELECT * FROM t1").count() + sql("SELECT * FROM t2").count() + uncacheTable("t1") + uncacheTable("t2") + + assert(accsSize >= Accumulators.originals.size) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala index e57bb06e7263b..2a0b701cad7fa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/PartitionBatchPruningSuite.scala @@ -39,6 +39,8 @@ class PartitionBatchPruningSuite extends FunSuite with BeforeAndAfterAll with Be // Enable in-memory partition pruning setConf(SQLConf.IN_MEMORY_PARTITION_PRUNING, "true") + // Enable in-memory table scan accumulators + setConf("spark.sql.inMemoryTableScanStatistics.enable", "true") } override protected def afterAll(): Unit = { From 557a797a273f1668065806cba53e19e6134a66d3 Mon Sep 17 00:00:00 2001 From: sboeschhuawei Date: Wed, 15 Apr 2015 13:28:10 -0700 Subject: [PATCH 109/128] [SPARK-6937][MLLIB] Fixed bug in PICExample in which the radius were not being accepted on c... Tiny bug in PowerIterationClusteringExample in which radius not accepted from command line Author: sboeschhuawei Closes #5531 from javadba/picsub and squashes the following commits: 2aab8cf [sboeschhuawei] Fixed bug in PICExample in which the radius were not being accepted on command line --- .../examples/mllib/PowerIterationClusteringExample.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala index 9f22d40c15f3f..6d8b806569dfd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/PowerIterationClusteringExample.scala @@ -65,7 +65,7 @@ object PowerIterationClusteringExample { def main(args: Array[String]) { val defaultParams = Params() - val parser = new OptionParser[Params]("PIC Circles") { + val parser = new OptionParser[Params]("PowerIterationClusteringExample") { head("PowerIterationClusteringExample: an example PIC app using concentric circles.") opt[Int]('k', "k") .text(s"number of circles (/clusters), default: ${defaultParams.k}") @@ -76,9 +76,9 @@ object PowerIterationClusteringExample { opt[Int]("maxIterations") .text(s"number of iterations, default: ${defaultParams.maxIterations}") .action((x, c) => c.copy(maxIterations = x)) - opt[Int]('r', "r") + opt[Double]('r', "r") .text(s"radius of outermost circle, default: ${defaultParams.outerRadius}") - .action((x, c) => c.copy(numPoints = x)) + .action((x, c) => c.copy(outerRadius = x)) } parser.parse(args, defaultParams).map { params => @@ -154,3 +154,4 @@ object PowerIterationClusteringExample { coeff * math.exp(expCoeff * ssquares) } } + From 4754e16f4746ebd882b2ce7f1efc6e4d4408922c Mon Sep 17 00:00:00 2001 From: Wenchen Fan Date: Wed, 15 Apr 2015 13:39:12 -0700 Subject: [PATCH 110/128] [SPARK-6898][SQL] completely support special chars in column names Even if we wrap column names in backticks like `` `a#$b.c` ``, we still handle the "." inside column name specially. I think it's fragile to use a special char to split name parts, why not put name parts in `UnresolvedAttribute` directly? Author: Wenchen Fan This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #5511 from cloud-fan/6898 and squashes the following commits: 48e3e57 [Wenchen Fan] more style fix 820dc45 [Wenchen Fan] do not ignore newName in UnresolvedAttribute d81ad43 [Wenchen Fan] fix style 11699d6 [Wenchen Fan] completely support special chars in column names --- .../apache/spark/sql/catalyst/SqlParser.scala | 4 +-- .../sql/catalyst/analysis/Analyzer.scala | 13 ++++----- .../sql/catalyst/analysis/CheckAnalysis.scala | 6 ++++- .../sql/catalyst/analysis/unresolved.scala | 14 ++++++++-- .../catalyst/plans/logical/LogicalPlan.scala | 27 +++++++++---------- .../sql/catalyst/analysis/AnalysisSuite.scala | 2 -- .../org/apache/spark/sql/DataFrame.scala | 4 +-- .../org/apache/spark/sql/SQLQuerySuite.scala | 13 ++++++--- .../org/apache/spark/sql/hive/HiveQl.scala | 2 +- 9 files changed, 52 insertions(+), 33 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 9a3531ceb3343..0af969cc5cc67 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -381,13 +381,13 @@ class SqlParser extends AbstractSparkSQLParser with DataTypeParser { | "(" ~> expression <~ ")" | function | dotExpressionHeader - | ident ^^ UnresolvedAttribute + | ident ^^ {case i => UnresolvedAttribute.quoted(i)} | signedPrimary | "~" ~> expression ^^ BitwiseNot ) protected lazy val dotExpressionHeader: Parser[Expression] = (ident <~ ".") ~ ident ~ rep("." ~> ident) ^^ { - case i1 ~ i2 ~ rest => UnresolvedAttribute((Seq(i1, i2) ++ rest).mkString(".")) + case i1 ~ i2 ~ rest => UnresolvedAttribute(Seq(i1, i2) ++ rest) } } 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 8b68b0df35f48..cb49e5ad5586f 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 @@ -297,14 +297,15 @@ class Analyzer( case q: LogicalPlan => logTrace(s"Attempting to resolve ${q.simpleString}") q transformExpressionsUp { - case u @ UnresolvedAttribute(name) if resolver(name, VirtualColumn.groupingIdName) && + case u @ UnresolvedAttribute(nameParts) if nameParts.length == 1 && + resolver(nameParts(0), VirtualColumn.groupingIdName) && q.isInstanceOf[GroupingAnalytics] => // Resolve the virtual column GROUPING__ID for the operator GroupingAnalytics q.asInstanceOf[GroupingAnalytics].gid - case u @ UnresolvedAttribute(name) => + case u @ UnresolvedAttribute(nameParts) => // Leave unchanged if resolution fails. Hopefully will be resolved next round. val result = - withPosition(u) { q.resolveChildren(name, resolver).getOrElse(u) } + withPosition(u) { q.resolveChildren(nameParts, resolver).getOrElse(u) } logDebug(s"Resolving $u to $result") result case UnresolvedGetField(child, fieldName) if child.resolved => @@ -383,12 +384,12 @@ class Analyzer( child: LogicalPlan, grandchild: LogicalPlan): (Seq[SortOrder], Seq[Attribute]) = { // Find any attributes that remain unresolved in the sort. - val unresolved: Seq[String] = - ordering.flatMap(_.collect { case UnresolvedAttribute(name) => name }) + val unresolved: Seq[Seq[String]] = + ordering.flatMap(_.collect { case UnresolvedAttribute(nameParts) => nameParts }) // Create a map from name, to resolved attributes, when the desired name can be found // prior to the projection. - val resolved: Map[String, NamedExpression] = + val resolved: Map[Seq[String], NamedExpression] = unresolved.flatMap(u => grandchild.resolve(u, resolver).map(a => u -> a)).toMap // Construct a set that contains all of the attributes that we need to evaluate the diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala index fa02111385c06..1155dac28fc78 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CheckAnalysis.scala @@ -46,8 +46,12 @@ trait CheckAnalysis { operator transformExpressionsUp { case a: Attribute if !a.resolved => if (operator.childrenResolved) { + val nameParts = a match { + case UnresolvedAttribute(nameParts) => nameParts + case _ => Seq(a.name) + } // Throw errors for specific problems with get field. - operator.resolveChildren(a.name, resolver, throwErrors = true) + operator.resolveChildren(nameParts, resolver, throwErrors = true) } val from = operator.inputSet.map(_.name).mkString(", ") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index 300e9ba187bc5..3f567e3e8b2a6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -49,7 +49,12 @@ case class UnresolvedRelation( /** * Holds the name of an attribute that has yet to be resolved. */ -case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNode[Expression] { +case class UnresolvedAttribute(nameParts: Seq[String]) + extends Attribute with trees.LeafNode[Expression] { + + def name: String = + nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def nullable: Boolean = throw new UnresolvedException(this, "nullable") @@ -59,7 +64,7 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def newInstance(): UnresolvedAttribute = this override def withNullability(newNullability: Boolean): UnresolvedAttribute = this override def withQualifiers(newQualifiers: Seq[String]): UnresolvedAttribute = this - override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute(name) + override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute.quoted(newName) // Unresolved attributes are transient at compile time and don't get evaluated during execution. override def eval(input: Row = null): EvaluatedType = @@ -68,6 +73,11 @@ case class UnresolvedAttribute(name: String) extends Attribute with trees.LeafNo override def toString: String = s"'$name" } +object UnresolvedAttribute { + def apply(name: String): UnresolvedAttribute = new UnresolvedAttribute(name.split("\\.")) + def quoted(name: String): UnresolvedAttribute = new UnresolvedAttribute(Seq(name)) +} + case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def foldable: Boolean = throw new UnresolvedException(this, "foldable") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 579a0fb8d3f93..ae4620a4e5abf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -19,12 +19,11 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.Logging import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.analysis.{EliminateSubQueries, UnresolvedGetField, Resolver} +import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, EliminateSubQueries, Resolver} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.trees -import org.apache.spark.sql.types.{ArrayType, StructType, StructField} abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { @@ -111,10 +110,10 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ def resolveChildren( - name: String, + nameParts: Seq[String], resolver: Resolver, throwErrors: Boolean = false): Option[NamedExpression] = - resolve(name, children.flatMap(_.output), resolver, throwErrors) + resolve(nameParts, children.flatMap(_.output), resolver, throwErrors) /** * Optionally resolves the given string to a [[NamedExpression]] based on the output of this @@ -122,10 +121,10 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * `[scope].AttributeName.[nested].[fields]...`. */ def resolve( - name: String, + nameParts: Seq[String], resolver: Resolver, throwErrors: Boolean = false): Option[NamedExpression] = - resolve(name, output, resolver, throwErrors) + resolve(nameParts, output, resolver, throwErrors) /** * Resolve the given `name` string against the given attribute, returning either 0 or 1 match. @@ -135,7 +134,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * See the comment above `candidates` variable in resolve() for semantics the returned data. */ private def resolveAsTableColumn( - nameParts: Array[String], + nameParts: Seq[String], resolver: Resolver, attribute: Attribute): Option[(Attribute, List[String])] = { assert(nameParts.length > 1) @@ -155,7 +154,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { * See the comment above `candidates` variable in resolve() for semantics the returned data. */ private def resolveAsColumn( - nameParts: Array[String], + nameParts: Seq[String], resolver: Resolver, attribute: Attribute): Option[(Attribute, List[String])] = { if (resolver(attribute.name, nameParts.head)) { @@ -167,13 +166,11 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { /** Performs attribute resolution given a name and a sequence of possible attributes. */ protected def resolve( - name: String, + nameParts: Seq[String], input: Seq[Attribute], resolver: Resolver, throwErrors: Boolean): Option[NamedExpression] = { - val parts = name.split("\\.") - // A sequence of possible candidate matches. // Each candidate is a tuple. The first element is a resolved attribute, followed by a list // of parts that are to be resolved. @@ -182,9 +179,9 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // and the second element will be List("c"). var candidates: Seq[(Attribute, List[String])] = { // If the name has 2 or more parts, try to resolve it as `table.column` first. - if (parts.length > 1) { + if (nameParts.length > 1) { input.flatMap { option => - resolveAsTableColumn(parts, resolver, option) + resolveAsTableColumn(nameParts, resolver, option) } } else { Seq.empty @@ -194,10 +191,12 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging { // If none of attributes match `table.column` pattern, we try to resolve it as a column. if (candidates.isEmpty) { candidates = input.flatMap { candidate => - resolveAsColumn(parts, resolver, candidate) + resolveAsColumn(nameParts, resolver, candidate) } } + def name = UnresolvedAttribute(nameParts).name + candidates.distinct match { // One match, no nested fields, use it. case Seq((a, Nil)) => Some(a) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 6e3d6b9263e86..e10ddfdf5127c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -27,8 +27,6 @@ import org.apache.spark.sql.types._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import scala.collection.immutable - class AnalysisSuite extends FunSuite with BeforeAndAfter { val caseSensitiveCatalog = new SimpleCatalog(true) val caseInsensitiveCatalog = new SimpleCatalog(false) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 94ae2d65fd0e4..3235f85d5bbd2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -158,7 +158,7 @@ class DataFrame private[sql]( } protected[sql] def resolve(colName: String): NamedExpression = { - queryExecution.analyzed.resolve(colName, sqlContext.analyzer.resolver).getOrElse { + queryExecution.analyzed.resolve(colName.split("\\."), sqlContext.analyzer.resolver).getOrElse { throw new AnalysisException( s"""Cannot resolve column name "$colName" among (${schema.fieldNames.mkString(", ")})""") } @@ -166,7 +166,7 @@ class DataFrame private[sql]( protected[sql] def numericColumns: Seq[Expression] = { schema.fields.filter(_.dataType.isInstanceOf[NumericType]).map { n => - queryExecution.analyzed.resolve(n.name, sqlContext.analyzer.resolver).get + queryExecution.analyzed.resolve(n.name.split("\\."), sqlContext.analyzer.resolver).get } } 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 4c48dca44498b..d739e550f3e56 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 @@ -19,14 +19,13 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.TestData._ import org.apache.spark.sql.execution.GeneratedAggregate import org.apache.spark.sql.functions._ +import org.apache.spark.sql.TestData._ import org.apache.spark.sql.test.TestSQLContext import org.apache.spark.sql.test.TestSQLContext.{udf => _, _} import org.apache.spark.sql.types._ - class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { // Make sure the tables are loaded. TestData @@ -1125,7 +1124,7 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { val data = sparkContext.parallelize( Seq("""{"key?number1": "value1", "key.number2": "value2"}""")) jsonRDD(data).registerTempTable("records") - sql("SELECT `key?number1` FROM records") + sql("SELECT `key?number1`, `key.number2` FROM records") } test("SPARK-3814 Support Bitwise & operator") { @@ -1225,4 +1224,12 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { checkAnswer(sql("SELECT a.b[0] FROM t ORDER BY c0.a"), Row(1)) checkAnswer(sql("SELECT b[0].a FROM t ORDER BY c0.a"), Row(1)) } + + test("SPARK-6898: complete support for special chars in column names") { + jsonRDD(sparkContext.makeRDD( + """{"a": {"c.b": 1}, "b.$q": [{"a@!.q": 1}], "q.w": {"w.i&": [1]}}""" :: Nil)) + .registerTempTable("t") + + checkAnswer(sql("SELECT a.`c.b`, `b.$q`[0].`a@!.q`, `q.w`.`w.i&`[0] FROM t"), Row(1, 1, 1)) + } } 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 53a204b8c2932..fd305eb480e63 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 @@ -1101,7 +1101,7 @@ https://cwiki.apache.org/confluence/display/Hive/Enhanced+Aggregation%2C+Cube%2C case Token(".", qualifier :: Token(attr, Nil) :: Nil) => nodeToExpr(qualifier) match { case UnresolvedAttribute(qualifierName) => - UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)) + UnresolvedAttribute(qualifierName :+ cleanIdentifier(attr)) case other => UnresolvedGetField(other, attr) } From 585638e81ce09a72b9e7f95d38e0d432cfa02456 Mon Sep 17 00:00:00 2001 From: Daoyuan Wang Date: Wed, 15 Apr 2015 14:06:10 -0700 Subject: [PATCH 111/128] [SPARK-2213] [SQL] sort merge join for spark sql Thanks for the initial work from Ishiihara in #3173 This PR introduce a new join method of sort merge join, which firstly ensure that keys of same value are in the same partition, and inside each partition the Rows are sorted by key. Then we can run down both sides together, find matched rows using [sort merge join](http://en.wikipedia.org/wiki/Sort-merge_join). In this way, we don't have to store the whole hash table of one side as hash join, thus we have less memory usage. Also, this PR would benefit from #3438 , making the sorting phrase much more efficient. We introduced a new configuration of "spark.sql.planner.sortMergeJoin" to switch between this(`true`) and ShuffledHashJoin(`false`), probably we want the default value of it be `false` at first. Author: Daoyuan Wang Author: Michael Armbrust This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #5208 from adrian-wang/smj and squashes the following commits: 2493b9f [Daoyuan Wang] fix style 5049d88 [Daoyuan Wang] propagate rowOrdering for RangePartitioning f91a2ae [Daoyuan Wang] yin's comment: use external sort if option is enabled, add comments f515cd2 [Daoyuan Wang] yin's comment: outputOrdering, join suite refine ec8061b [Daoyuan Wang] minor change 413fd24 [Daoyuan Wang] Merge pull request #3 from marmbrus/pr/5208 952168a [Michael Armbrust] add type 5492884 [Michael Armbrust] copy when ordering 7ddd656 [Michael Armbrust] Cleanup addition of ordering requirements b198278 [Daoyuan Wang] inherit ordering in project c8e82a3 [Daoyuan Wang] fix style 6e897dd [Daoyuan Wang] hide boundReference from manually construct RowOrdering for key compare in smj 8681d73 [Daoyuan Wang] refactor Exchange and fix copy for sorting 2875ef2 [Daoyuan Wang] fix changed configuration 61d7f49 [Daoyuan Wang] add omitted comment 00a4430 [Daoyuan Wang] fix bug 078d69b [Daoyuan Wang] address comments: add comments, do sort in shuffle, and others 3af6ba5 [Daoyuan Wang] use buffer for only one side 171001f [Daoyuan Wang] change default outputordering 47455c9 [Daoyuan Wang] add apache license ... a28277f [Daoyuan Wang] fix style 645c70b [Daoyuan Wang] address comments using sort 068c35d [Daoyuan Wang] fix new style and add some tests 925203b [Daoyuan Wang] address comments 07ce92f [Daoyuan Wang] fix ArrayIndexOutOfBound 42fca0e [Daoyuan Wang] code clean e3ec096 [Daoyuan Wang] fix comment style.. 2edd235 [Daoyuan Wang] fix outputpartitioning 57baa40 [Daoyuan Wang] fix sort eval bug 303b6da [Daoyuan Wang] fix several errors 95db7ad [Daoyuan Wang] fix brackets for if-statement 4464f16 [Daoyuan Wang] fix error 880d8e9 [Daoyuan Wang] sort merge join for spark sql --- .../spark/sql/catalyst/expressions/rows.scala | 10 +- .../plans/physical/partitioning.scala | 13 ++ .../scala/org/apache/spark/sql/SQLConf.scala | 8 + .../org/apache/spark/sql/SQLContext.scala | 2 +- .../apache/spark/sql/execution/Exchange.scala | 148 ++++++++++++--- .../spark/sql/execution/SparkPlan.scala | 6 + .../spark/sql/execution/SparkStrategies.scala | 11 +- .../spark/sql/execution/basicOperators.scala | 10 ++ .../sql/execution/joins/SortMergeJoin.scala | 169 ++++++++++++++++++ .../org/apache/spark/sql/JoinSuite.scala | 28 ++- .../SortMergeCompatibilitySuite.scala | 162 +++++++++++++++++ 11 files changed, 534 insertions(+), 33 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala create mode 100644 sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala index 1b62e17ff47fd..b6ec7d3417ef8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/rows.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.types.{UTF8String, StructType, NativeType} - +import org.apache.spark.sql.types.{UTF8String, DataType, StructType, NativeType} /** * An extended interface to [[Row]] that allows the values for each column to be updated. Setting @@ -239,3 +238,10 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { return 0 } } + +object RowOrdering { + def forSchema(dataTypes: Seq[DataType]): RowOrdering = + new RowOrdering(dataTypes.zipWithIndex.map { + case(dt, index) => new SortOrder(BoundReference(index, dt, nullable = true), Ascending) + }) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index 288c11f69fe22..fb4217a44807b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -94,6 +94,9 @@ sealed trait Partitioning { * only compatible if the `numPartitions` of them is the same. */ def compatibleWith(other: Partitioning): Boolean + + /** Returns the expressions that are used to key the partitioning. */ + def keyExpressions: Seq[Expression] } case class UnknownPartitioning(numPartitions: Int) extends Partitioning { @@ -106,6 +109,8 @@ case class UnknownPartitioning(numPartitions: Int) extends Partitioning { case UnknownPartitioning(_) => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } case object SinglePartition extends Partitioning { @@ -117,6 +122,8 @@ case object SinglePartition extends Partitioning { case SinglePartition => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } case object BroadcastPartitioning extends Partitioning { @@ -128,6 +135,8 @@ case object BroadcastPartitioning extends Partitioning { case SinglePartition => true case _ => false } + + override def keyExpressions: Seq[Expression] = Nil } /** @@ -158,6 +167,8 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) case _ => false } + override def keyExpressions: Seq[Expression] = expressions + override def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } @@ -200,6 +211,8 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) case _ => false } + override def keyExpressions: Seq[Expression] = ordering.map(_.child) + override def eval(input: Row): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index ee641bdfeb2d7..5c65f04ee8497 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -47,6 +47,7 @@ private[spark] object SQLConf { // Options that control which operators can be chosen by the query planner. These should be // considered hints and may be ignored by future versions of Spark SQL. val EXTERNAL_SORT = "spark.sql.planner.externalSort" + val SORTMERGE_JOIN = "spark.sql.planner.sortMergeJoin" // This is only used for the thriftserver val THRIFTSERVER_POOL = "spark.sql.thriftserver.scheduler.pool" @@ -128,6 +129,13 @@ private[sql] class SQLConf extends Serializable { /** When true the planner will use the external sort, which may spill to disk. */ private[spark] def externalSortEnabled: Boolean = getConf(EXTERNAL_SORT, "false").toBoolean + /** + * Sort merge join would sort the two side of join first, and then iterate both sides together + * only once to get all matches. Using sort merge join can save a lot of memory usage compared + * to HashJoin. + */ + private[spark] def sortMergeJoinEnabled: Boolean = getConf(SORTMERGE_JOIN, "false").toBoolean + /** * When set to true, Spark SQL will use the Scala compiler at runtime to generate custom bytecode * that evaluates expressions found in queries. In general this custom code runs much faster diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 89a4faf35e0d2..f9f3eb2e03817 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -1081,7 +1081,7 @@ class SQLContext(@transient val sparkContext: SparkContext) @transient protected[sql] val prepareForExecution = new RuleExecutor[SparkPlan] { val batches = - Batch("Add exchange", Once, AddExchange(self)) :: Nil + Batch("Add exchange", Once, EnsureRequirements(self)) :: Nil } protected[sql] def openSession(): SQLSession = { 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 437408d30bfd2..518fc9e57c708 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 @@ -19,24 +19,42 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.sql.catalyst.expressions import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.sql.{SQLContext, Row} import org.apache.spark.sql.catalyst.errors.attachTree -import org.apache.spark.sql.catalyst.expressions.{Attribute, RowOrdering} +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair +object Exchange { + /** + * Returns true when the ordering expressions are a subset of the key. + * if true, ShuffledRDD can use `setKeyOrdering(orderingKey)` to sort within [[Exchange]]. + */ + def canSortWithShuffle(partitioning: Partitioning, desiredOrdering: Seq[SortOrder]): Boolean = { + desiredOrdering.map(_.child).toSet.subsetOf(partitioning.keyExpressions.toSet) + } +} + /** * :: DeveloperApi :: + * Performs a shuffle that will result in the desired `newPartitioning`. Optionally sorts each + * resulting partition based on expressions from the partition key. It is invalid to construct an + * exchange operator with a `newOrdering` that cannot be calculated using the partitioning key. */ @DeveloperApi -case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { +case class Exchange( + newPartitioning: Partitioning, + newOrdering: Seq[SortOrder], + child: SparkPlan) + extends UnaryNode { override def outputPartitioning: Partitioning = newPartitioning + override def outputOrdering: Seq[SortOrder] = newOrdering + override def output: Seq[Attribute] = child.output /** We must copy rows when sort based shuffle is on */ @@ -45,6 +63,20 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una private val bypassMergeThreshold = child.sqlContext.sparkContext.conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + private val keyOrdering = { + if (newOrdering.nonEmpty) { + val key = newPartitioning.keyExpressions + val boundOrdering = newOrdering.map { o => + val ordinal = key.indexOf(o.child) + if (ordinal == -1) sys.error(s"Invalid ordering on $o requested for $newPartitioning") + o.copy(child = BoundReference(ordinal, o.child.dataType, o.child.nullable)) + } + new RowOrdering(boundOrdering) + } else { + null // Ordering will not be used + } + } + override def execute(): RDD[Row] = attachTree(this , "execute") { newPartitioning match { case HashPartitioning(expressions, numPartitions) => @@ -56,7 +88,9 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una // 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) { + val willMergeSort = sortBasedShuffleOn && numPartitions > bypassMergeThreshold + + val rdd = if (willMergeSort || newOrdering.nonEmpty) { child.execute().mapPartitions { iter => val hashExpressions = newMutableProjection(expressions, child.output)() iter.map(r => (hashExpressions(r).copy(), r.copy())) @@ -69,12 +103,17 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una } } val part = new HashPartitioner(numPartitions) - val shuffled = new ShuffledRDD[Row, Row, Row](rdd, part) + val shuffled = + if (newOrdering.nonEmpty) { + new ShuffledRDD[Row, Row, Row](rdd, part).setKeyOrdering(keyOrdering) + } else { + new ShuffledRDD[Row, Row, Row](rdd, part) + } shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._2) case RangePartitioning(sortingExpressions, numPartitions) => - val rdd = if (sortBasedShuffleOn) { + val rdd = if (sortBasedShuffleOn || newOrdering.nonEmpty) { child.execute().mapPartitions { iter => iter.map(row => (row.copy(), null))} } else { child.execute().mapPartitions { iter => @@ -87,7 +126,12 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una implicit val ordering = new RowOrdering(sortingExpressions, child.output) val part = new RangePartitioner(numPartitions, rdd, ascending = true) - val shuffled = new ShuffledRDD[Row, Null, Null](rdd, part) + val shuffled = + if (newOrdering.nonEmpty) { + new ShuffledRDD[Row, Null, Null](rdd, part).setKeyOrdering(keyOrdering) + } else { + new ShuffledRDD[Row, Null, Null](rdd, part) + } shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._1) @@ -120,27 +164,34 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una * Ensures that the [[org.apache.spark.sql.catalyst.plans.physical.Partitioning Partitioning]] * of input data meets the * [[org.apache.spark.sql.catalyst.plans.physical.Distribution Distribution]] requirements for - * each operator by inserting [[Exchange]] Operators where required. + * each operator by inserting [[Exchange]] Operators where required. Also ensure that the + * required input partition ordering requirements are met. */ -private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPlan] { +private[sql] case class EnsureRequirements(sqlContext: SQLContext) extends Rule[SparkPlan] { // TODO: Determine the number of partitions. def numPartitions: Int = sqlContext.conf.numShufflePartitions def apply(plan: SparkPlan): SparkPlan = plan.transformUp { case operator: SparkPlan => - // Check if every child's outputPartitioning satisfies the corresponding + // True iff every child's outputPartitioning satisfies the corresponding // required data distribution. def meetsRequirements: Boolean = - !operator.requiredChildDistribution.zip(operator.children).map { + operator.requiredChildDistribution.zip(operator.children).forall { case (required, child) => val valid = child.outputPartitioning.satisfies(required) logDebug( s"${if (valid) "Valid" else "Invalid"} distribution," + s"required: $required current: ${child.outputPartitioning}") valid - }.exists(!_) + } - // Check if outputPartitionings of children are compatible with each other. + // True iff any of the children are incorrectly sorted. + def needsAnySort: Boolean = + operator.requiredChildOrdering.zip(operator.children).exists { + case (required, child) => required.nonEmpty && required != child.outputOrdering + } + + // True iff outputPartitionings of children are compatible with each other. // It is possible that every child satisfies its required data distribution // but two children have incompatible outputPartitionings. For example, // A dataset is range partitioned by "a.asc" (RangePartitioning) and another @@ -157,28 +208,69 @@ private[sql] case class AddExchange(sqlContext: SQLContext) extends Rule[SparkPl case Seq(a,b) => a compatibleWith b }.exists(!_) - // Check if the partitioning we want to ensure is the same as the child's output - // partitioning. If so, we do not need to add the Exchange operator. - def addExchangeIfNecessary(partitioning: Partitioning, child: SparkPlan): SparkPlan = - if (child.outputPartitioning != partitioning) Exchange(partitioning, child) else child + // Adds Exchange or Sort operators as required + def addOperatorsIfNecessary( + partitioning: Partitioning, + rowOrdering: Seq[SortOrder], + child: SparkPlan): SparkPlan = { + val needSort = rowOrdering.nonEmpty && child.outputOrdering != rowOrdering + val needsShuffle = child.outputPartitioning != partitioning + val canSortWithShuffle = Exchange.canSortWithShuffle(partitioning, rowOrdering) + + if (needSort && needsShuffle && canSortWithShuffle) { + Exchange(partitioning, rowOrdering, child) + } else { + val withShuffle = if (needsShuffle) { + Exchange(partitioning, Nil, child) + } else { + child + } - if (meetsRequirements && compatible) { + val withSort = if (needSort) { + if (sqlContext.conf.externalSortEnabled) { + ExternalSort(rowOrdering, global = false, withShuffle) + } else { + Sort(rowOrdering, global = false, withShuffle) + } + } else { + withShuffle + } + + withSort + } + } + + if (meetsRequirements && compatible && !needsAnySort) { operator } else { // At least one child does not satisfies its required data distribution or // at least one child's outputPartitioning is not compatible with another child's // outputPartitioning. In this case, we need to add Exchange operators. - val repartitionedChildren = operator.requiredChildDistribution.zip(operator.children).map { - case (AllTuples, child) => - addExchangeIfNecessary(SinglePartition, child) - case (ClusteredDistribution(clustering), child) => - addExchangeIfNecessary(HashPartitioning(clustering, numPartitions), child) - case (OrderedDistribution(ordering), child) => - addExchangeIfNecessary(RangePartitioning(ordering, numPartitions), child) - case (UnspecifiedDistribution, child) => child - case (dist, _) => sys.error(s"Don't know how to ensure $dist") + val requirements = + (operator.requiredChildDistribution, operator.requiredChildOrdering, operator.children) + + val fixedChildren = requirements.zipped.map { + case (AllTuples, rowOrdering, child) => + addOperatorsIfNecessary(SinglePartition, rowOrdering, child) + case (ClusteredDistribution(clustering), rowOrdering, child) => + addOperatorsIfNecessary(HashPartitioning(clustering, numPartitions), rowOrdering, child) + case (OrderedDistribution(ordering), rowOrdering, child) => + addOperatorsIfNecessary(RangePartitioning(ordering, numPartitions), rowOrdering, child) + + case (UnspecifiedDistribution, Seq(), child) => + child + case (UnspecifiedDistribution, rowOrdering, child) => + if (sqlContext.conf.externalSortEnabled) { + ExternalSort(rowOrdering, global = false, child) + } else { + Sort(rowOrdering, global = false, child) + } + + case (dist, ordering, _) => + sys.error(s"Don't know how to ensure $dist with ordering $ordering") } - operator.withNewChildren(repartitionedChildren) + + operator.withNewChildren(fixedChildren) } } } 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 fabcf6b4a0570..e159ffe66cb24 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 @@ -72,6 +72,12 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ def requiredChildDistribution: Seq[Distribution] = Seq.fill(children.size)(UnspecifiedDistribution) + /** Specifies how data is ordered in each partition. */ + def outputOrdering: Seq[SortOrder] = Nil + + /** Specifies sort order for each partition requirements on the input data for this operator. */ + def requiredChildOrdering: Seq[Seq[SortOrder]] = Seq.fill(children.size)(Nil) + /** * Runs this query returning the result as an RDD. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index 5b99e40c2f491..e687d01f57520 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -90,6 +90,14 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { left.statistics.sizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold => makeBroadcastHashJoin(leftKeys, rightKeys, left, right, condition, joins.BuildLeft) + // If the sort merge join option is set, we want to use sort merge join prior to hashjoin + // for now let's support inner join first, then add outer join + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) + if sqlContext.conf.sortMergeJoinEnabled => + val mergeJoin = + joins.SortMergeJoin(leftKeys, rightKeys, planLater(left), planLater(right)) + condition.map(Filter(_, mergeJoin)).getOrElse(mergeJoin) :: Nil + case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left, right) => val buildSide = if (right.statistics.sizeInBytes <= left.statistics.sizeInBytes) { @@ -309,7 +317,8 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.OneRowRelation => execution.PhysicalRDD(Nil, singleRowRdd) :: Nil case logical.Repartition(expressions, child) => - execution.Exchange(HashPartitioning(expressions, numPartitions), planLater(child)) :: Nil + execution.Exchange( + HashPartitioning(expressions, numPartitions), Nil, planLater(child)) :: Nil case e @ EvaluatePython(udf, child, _) => BatchPythonEvaluation(udf, e.output, planLater(child)) :: Nil case LogicalRDD(output, rdd) => PhysicalRDD(output, rdd) :: Nil diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index f8221f41bc6c3..308dae236a5ed 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -41,6 +41,8 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends val resuableProjection = buildProjection() iter.map(resuableProjection) } + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** @@ -55,6 +57,8 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def execute(): RDD[Row] = child.execute().mapPartitions { iter => iter.filter(conditionEvaluator) } + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering } /** @@ -147,6 +151,8 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. override def execute(): RDD[Row] = sparkContext.makeRDD(collectData(), 1) + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** @@ -172,6 +178,8 @@ case class Sort( } override def output: Seq[Attribute] = child.output + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** @@ -202,6 +210,8 @@ case class ExternalSort( } override def output: Seq[Attribute] = child.output + + override def outputOrdering: Seq[SortOrder] = sortOrder } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala new file mode 100644 index 0000000000000..b5123668ba11e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala @@ -0,0 +1,169 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.joins + +import java.util.NoSuchElementException + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.{BinaryNode, SparkPlan} +import org.apache.spark.util.collection.CompactBuffer + +/** + * :: DeveloperApi :: + * Performs an sort merge join of two child relations. + */ +@DeveloperApi +case class SortMergeJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode { + + override def output: Seq[Attribute] = left.output ++ right.output + + override def outputPartitioning: Partitioning = left.outputPartitioning + + override def requiredChildDistribution: Seq[Distribution] = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + // this is to manually construct an ordering that can be used to compare keys from both sides + private val keyOrdering: RowOrdering = RowOrdering.forSchema(leftKeys.map(_.dataType)) + + override def outputOrdering: Seq[SortOrder] = requiredOrders(leftKeys) + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = + requiredOrders(leftKeys) :: requiredOrders(rightKeys) :: Nil + + @transient protected lazy val leftKeyGenerator = newProjection(leftKeys, left.output) + @transient protected lazy val rightKeyGenerator = newProjection(rightKeys, right.output) + + private def requiredOrders(keys: Seq[Expression]): Seq[SortOrder] = + keys.map(SortOrder(_, Ascending)) + + override def execute(): RDD[Row] = { + val leftResults = left.execute().map(_.copy()) + val rightResults = right.execute().map(_.copy()) + + leftResults.zipPartitions(rightResults) { (leftIter, rightIter) => + new Iterator[Row] { + // Mutable per row objects. + private[this] val joinRow = new JoinedRow5 + private[this] var leftElement: Row = _ + private[this] var rightElement: Row = _ + private[this] var leftKey: Row = _ + private[this] var rightKey: Row = _ + private[this] var rightMatches: CompactBuffer[Row] = _ + private[this] var rightPosition: Int = -1 + private[this] var stop: Boolean = false + private[this] var matchKey: Row = _ + + // initialize iterator + initialize() + + override final def hasNext: Boolean = nextMatchingPair() + + override final def next(): Row = { + if (hasNext) { + // we are using the buffered right rows and run down left iterator + val joinedRow = joinRow(leftElement, rightMatches(rightPosition)) + rightPosition += 1 + if (rightPosition >= rightMatches.size) { + rightPosition = 0 + fetchLeft() + if (leftElement == null || keyOrdering.compare(leftKey, matchKey) != 0) { + stop = false + rightMatches = null + } + } + joinedRow + } else { + // no more result + throw new NoSuchElementException + } + } + + private def fetchLeft() = { + if (leftIter.hasNext) { + leftElement = leftIter.next() + leftKey = leftKeyGenerator(leftElement) + } else { + leftElement = null + } + } + + private def fetchRight() = { + if (rightIter.hasNext) { + rightElement = rightIter.next() + rightKey = rightKeyGenerator(rightElement) + } else { + rightElement = null + } + } + + private def initialize() = { + fetchLeft() + fetchRight() + } + + /** + * Searches the right iterator for the next rows that have matches in left side, and store + * them in a buffer. + * + * @return true if the search is successful, and false if the right iterator runs out of + * tuples. + */ + private def nextMatchingPair(): Boolean = { + if (!stop && rightElement != null) { + // run both side to get the first match pair + while (!stop && leftElement != null && rightElement != null) { + val comparing = keyOrdering.compare(leftKey, rightKey) + // for inner join, we need to filter those null keys + stop = comparing == 0 && !leftKey.anyNull + if (comparing > 0 || rightKey.anyNull) { + fetchRight() + } else if (comparing < 0 || leftKey.anyNull) { + fetchLeft() + } + } + rightMatches = new CompactBuffer[Row]() + if (stop) { + stop = false + // iterate the right side to buffer all rows that matches + // as the records should be ordered, exit when we meet the first that not match + while (!stop && rightElement != null) { + rightMatches += rightElement + fetchRight() + stop = keyOrdering.compare(leftKey, rightKey) != 0 + } + if (rightMatches.size > 0) { + rightPosition = 0 + matchKey = leftKey + } + } + } + rightMatches != null && rightMatches.size > 0 + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index e4dee87849fd4..037d392c1f929 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -51,6 +51,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { case j: CartesianProduct => j case j: BroadcastNestedLoopJoin => j case j: BroadcastLeftSemiJoinHash => j + case j: SortMergeJoin => j } assert(operators.size === 1) @@ -62,6 +63,7 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { test("join operator selection") { cacheManager.clearCache() + val SORTMERGEJOIN_ENABLED: Boolean = conf.sortMergeJoinEnabled Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", classOf[LeftSemiJoinHash]), ("SELECT * FROM testData LEFT SEMI JOIN testData2", classOf[LeftSemiJoinBNL]), @@ -91,17 +93,41 @@ class JoinSuite extends QueryTest with BeforeAndAfterEach { ("SELECT * FROM testData full JOIN testData2 ON (key * a != key + a)", classOf[BroadcastNestedLoopJoin]) ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + try { + conf.setConf("spark.sql.planner.sortMergeJoin", "true") + Seq( + ("SELECT * FROM testData JOIN testData2 ON key = a", classOf[SortMergeJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a and key = 2", classOf[SortMergeJoin]), + ("SELECT * FROM testData JOIN testData2 ON key = a where key = 2", classOf[SortMergeJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + } finally { + conf.setConf("spark.sql.planner.sortMergeJoin", SORTMERGEJOIN_ENABLED.toString) + } } test("broadcasted hash join operator selection") { cacheManager.clearCache() sql("CACHE TABLE testData") + val SORTMERGEJOIN_ENABLED: Boolean = conf.sortMergeJoinEnabled Seq( ("SELECT * FROM testData join testData2 ON key = a", classOf[BroadcastHashJoin]), ("SELECT * FROM testData join testData2 ON key = a and key = 2", classOf[BroadcastHashJoin]), - ("SELECT * FROM testData join testData2 ON key = a where key = 2", classOf[BroadcastHashJoin]) + ("SELECT * FROM testData join testData2 ON key = a where key = 2", + classOf[BroadcastHashJoin]) ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + try { + conf.setConf("spark.sql.planner.sortMergeJoin", "true") + Seq( + ("SELECT * FROM testData join testData2 ON key = a", classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a and key = 2", + classOf[BroadcastHashJoin]), + ("SELECT * FROM testData join testData2 ON key = a where key = 2", + classOf[BroadcastHashJoin]) + ).foreach { case (query, joinClass) => assertJoin(query, joinClass) } + } finally { + conf.setConf("spark.sql.planner.sortMergeJoin", SORTMERGEJOIN_ENABLED.toString) + } sql("UNCACHE TABLE testData") } diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala new file mode 100644 index 0000000000000..65d070bd3cbde --- /dev/null +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/SortMergeCompatibilitySuite.scala @@ -0,0 +1,162 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.SQLConf +import org.apache.spark.sql.hive.test.TestHive + +/** + * Runs the test cases that are included in the hive distribution with sort merge join is true. + */ +class SortMergeCompatibilitySuite extends HiveCompatibilitySuite { + override def beforeAll() { + super.beforeAll() + TestHive.setConf(SQLConf.SORTMERGE_JOIN, "true") + } + + override def afterAll() { + TestHive.setConf(SQLConf.SORTMERGE_JOIN, "false") + super.afterAll() + } + + override def whiteList = Seq( + "auto_join0", + "auto_join1", + "auto_join10", + "auto_join11", + "auto_join12", + "auto_join13", + "auto_join14", + "auto_join14_hadoop20", + "auto_join15", + "auto_join17", + "auto_join18", + "auto_join19", + "auto_join2", + "auto_join20", + "auto_join21", + "auto_join22", + "auto_join23", + "auto_join24", + "auto_join25", + "auto_join26", + "auto_join27", + "auto_join28", + "auto_join3", + "auto_join30", + "auto_join31", + "auto_join32", + "auto_join4", + "auto_join5", + "auto_join6", + "auto_join7", + "auto_join8", + "auto_join9", + "auto_join_filters", + "auto_join_nulls", + "auto_join_reordering_values", + "auto_smb_mapjoin_14", + "auto_sortmerge_join_1", + "auto_sortmerge_join_10", + "auto_sortmerge_join_11", + "auto_sortmerge_join_12", + "auto_sortmerge_join_13", + "auto_sortmerge_join_14", + "auto_sortmerge_join_15", + "auto_sortmerge_join_16", + "auto_sortmerge_join_2", + "auto_sortmerge_join_3", + "auto_sortmerge_join_4", + "auto_sortmerge_join_5", + "auto_sortmerge_join_6", + "auto_sortmerge_join_7", + "auto_sortmerge_join_8", + "auto_sortmerge_join_9", + "correlationoptimizer1", + "correlationoptimizer10", + "correlationoptimizer11", + "correlationoptimizer13", + "correlationoptimizer14", + "correlationoptimizer15", + "correlationoptimizer2", + "correlationoptimizer3", + "correlationoptimizer4", + "correlationoptimizer6", + "correlationoptimizer7", + "correlationoptimizer8", + "correlationoptimizer9", + "join0", + "join1", + "join10", + "join11", + "join12", + "join13", + "join14", + "join14_hadoop20", + "join15", + "join16", + "join17", + "join18", + "join19", + "join2", + "join20", + "join21", + "join22", + "join23", + "join24", + "join25", + "join26", + "join27", + "join28", + "join29", + "join3", + "join30", + "join31", + "join32", + "join32_lessSize", + "join33", + "join34", + "join35", + "join36", + "join37", + "join38", + "join39", + "join4", + "join40", + "join41", + "join5", + "join6", + "join7", + "join8", + "join9", + "join_1to1", + "join_array", + "join_casesensitive", + "join_empty", + "join_filters", + "join_hive_626", + "join_map_ppr", + "join_nulls", + "join_nullsafe", + "join_rc", + "join_reorder2", + "join_reorder3", + "join_reorder4", + "join_star" + ) +} From d5f1b9650b6e46cf6a9d61f01cda0df0cda5b1c9 Mon Sep 17 00:00:00 2001 From: Isaias Barroso Date: Wed, 15 Apr 2015 22:40:52 +0100 Subject: [PATCH 112/128] [SPARK-2312] Logging Unhandled messages The previous solution has changed based on https://github.com/apache/spark/pull/2048 discussions. Author: Isaias Barroso Closes #2055 from isaias/SPARK-2312 and squashes the following commits: f61d9e6 [Isaias Barroso] Change Log level for unhandled message to debug f341777 [Isaias Barroso] [SPARK-2312] Logging Unhandled messages --- .../scala/org/apache/spark/util/ActorLogReceive.scala | 8 +++++++- 1 file changed, 7 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala b/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala index 332d0cbb2dc0c..81a7cbde01ce5 100644 --- a/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala +++ b/core/src/main/scala/org/apache/spark/util/ActorLogReceive.scala @@ -43,7 +43,13 @@ private[spark] trait ActorLogReceive { private val _receiveWithLogging = receiveWithLogging - override def isDefinedAt(o: Any): Boolean = _receiveWithLogging.isDefinedAt(o) + override def isDefinedAt(o: Any): Boolean = { + val handled = _receiveWithLogging.isDefinedAt(o) + if (!handled) { + log.debug(s"Received unexpected actor system event: $o") + } + handled + } override def apply(o: Any): Unit = { if (log.isDebugEnabled) { From 8a53de16fc8208358b76d0f3d45538e0304bcc8e Mon Sep 17 00:00:00 2001 From: Max Seiden Date: Wed, 15 Apr 2015 16:15:11 -0700 Subject: [PATCH 113/128] [SPARK-5277][SQL] - SparkSqlSerializer doesn't always register user specified KryoRegistrators [SPARK-5277][SQL] - SparkSqlSerializer doesn't always register user specified KryoRegistrators There were a few places where new SparkSqlSerializer instances were created with new, empty SparkConfs resulting in user specified registrators sometimes not getting initialized. The fix is to try and pull a conf from the SparkEnv, and construct a new conf (that loads defaults) if one cannot be found. The changes touched: 1) SparkSqlSerializer's resource pool (this appears to fix the issue in the comment) 2) execution.Exchange (for all of the partitioners) 3) execution.Limit (for the HashPartitioner) A few tests were added to ColumnTypeSuite, ensuring that a custom registrator and serde is initialized and used when in-memory columns are written. Author: Max Seiden This patch had conflicts when merged, resolved by Committer: Michael Armbrust Closes #5237 from mhseiden/sql_udt_kryo and squashes the following commits: 3175c2f [Max Seiden] [SPARK-5277][SQL] - address code review comments e5011fb [Max Seiden] [SPARK-5277][SQL] - SparkSqlSerializer does not register user specified KryoRegistrators --- .../apache/spark/sql/execution/Exchange.scala | 9 +-- .../sql/execution/SparkSqlSerializer.scala | 7 +-- .../spark/sql/execution/basicOperators.scala | 2 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 62 ++++++++++++++++++- 4 files changed, 68 insertions(+), 12 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala index 518fc9e57c708..69a620e1ec929 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 @@ -78,6 +78,8 @@ case class Exchange( } override def execute(): RDD[Row] = attachTree(this , "execute") { + lazy val sparkConf = child.sqlContext.sparkContext.getConf + newPartitioning match { case HashPartitioning(expressions, numPartitions) => // TODO: Eliminate redundant expressions in grouping key and value. @@ -109,7 +111,7 @@ case class Exchange( } else { new ShuffledRDD[Row, Row, Row](rdd, part) } - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) shuffled.map(_._2) case RangePartitioning(sortingExpressions, numPartitions) => @@ -132,8 +134,7 @@ case class Exchange( } else { new ShuffledRDD[Row, Null, Null](rdd, part) } - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) - + shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) shuffled.map(_._1) case SinglePartition => @@ -151,7 +152,7 @@ case class Exchange( } val partitioner = new HashPartitioner(1) val shuffled = new ShuffledRDD[Null, Row, Row](rdd, partitioner) - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.setSerializer(new SparkSqlSerializer(sparkConf)) shuffled.map(_._2) case _ => sys.error(s"Exchange not implemented for $newPartitioning") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 914f387dec78f..eea15aff5dbcf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -65,12 +65,9 @@ private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(co private[execution] class KryoResourcePool(size: Int) extends ResourcePool[SerializerInstance](size) { - val ser: KryoSerializer = { + val ser: SparkSqlSerializer = { val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) - // TODO (lian) Using KryoSerializer here is workaround, needs further investigation - // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization - // related error. - new KryoSerializer(sparkConf) + new SparkSqlSerializer(sparkConf) } def newInstance(): SerializerInstance = ser.newInstance() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala index 308dae236a5ed..d286fe81bee5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -121,7 +121,7 @@ case class Limit(limit: Int, child: SparkPlan) } val part = new HashPartitioner(1) val shuffled = new ShuffledRDD[Boolean, Row, Row](rdd, part) - shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.setSerializer(new SparkSqlSerializer(child.sqlContext.sparkContext.getConf)) shuffled.mapPartitions(_.take(limit).map(_._2)) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index c86ef338fc644..b48bed1871c50 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -20,9 +20,12 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer import java.sql.Timestamp +import com.esotericsoftware.kryo.{Serializer, Kryo} +import com.esotericsoftware.kryo.io.{Input, Output} +import org.apache.spark.serializer.KryoRegistrator import org.scalatest.FunSuite -import org.apache.spark.Logging +import org.apache.spark.{SparkConf, Logging} import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer @@ -73,7 +76,7 @@ class ColumnTypeSuite extends FunSuite with Logging { checkActualSize(BINARY, binary, 4 + 4) val generic = Map(1 -> "a") - checkActualSize(GENERIC, SparkSqlSerializer.serialize(generic), 4 + 11) + checkActualSize(GENERIC, SparkSqlSerializer.serialize(generic), 4 + 8) } testNativeColumnType[BooleanType.type]( @@ -158,6 +161,41 @@ class ColumnTypeSuite extends FunSuite with Logging { } } + test("CUSTOM") { + val conf = new SparkConf() + conf.set("spark.kryo.registrator", "org.apache.spark.sql.columnar.Registrator") + val serializer = new SparkSqlSerializer(conf).newInstance() + + val buffer = ByteBuffer.allocate(512) + val obj = CustomClass(Int.MaxValue,Long.MaxValue) + val serializedObj = serializer.serialize(obj).array() + + GENERIC.append(serializer.serialize(obj).array(), buffer) + buffer.rewind() + + val length = buffer.getInt + assert(length === serializedObj.length) + assert(13 == length) // id (1) + int (4) + long (8) + + val genericSerializedObj = SparkSqlSerializer.serialize(obj) + assert(length != genericSerializedObj.length) + assert(length < genericSerializedObj.length) + + assertResult(obj, "Custom deserialized object didn't equal the original object") { + val bytes = new Array[Byte](length) + buffer.get(bytes, 0, length) + serializer.deserialize(ByteBuffer.wrap(bytes)) + } + + buffer.rewind() + buffer.putInt(serializedObj.length).put(serializedObj) + + assertResult(obj, "Custom deserialized object didn't equal the original object") { + buffer.rewind() + serializer.deserialize(ByteBuffer.wrap(GENERIC.extract(buffer))) + } + } + def testNativeColumnType[T <: NativeType]( columnType: NativeColumnType[T], putter: (ByteBuffer, T#JvmType) => Unit, @@ -229,3 +267,23 @@ class ColumnTypeSuite extends FunSuite with Logging { } } } + +private[columnar] final case class CustomClass(a: Int, b: Long) + +private[columnar] object CustomerSerializer extends Serializer[CustomClass] { + override def write(kryo: Kryo, output: Output, t: CustomClass) { + output.writeInt(t.a) + output.writeLong(t.b) + } + override def read(kryo: Kryo, input: Input, aClass: Class[CustomClass]): CustomClass = { + val a = input.readInt() + val b = input.readLong() + CustomClass(a,b) + } +} + +private[columnar] final class Registrator extends KryoRegistrator { + override def registerClasses(kryo: Kryo) { + kryo.register(classOf[CustomClass], CustomerSerializer) + } +} From 52c3439a8a107ce1fc10e4f0b59fd7881e851622 Mon Sep 17 00:00:00 2001 From: Juliet Hougland Date: Wed, 15 Apr 2015 21:52:25 -0700 Subject: [PATCH 114/128] SPARK-6938: All require statements now have an informative error message. This pr adds informative error messages to all require statements in the Vectors class that did not previously have them. This references [SPARK-6938](https://issues.apache.org/jira/browse/SPARK-6938). Author: Juliet Hougland Closes #5532 from jhlch/SPARK-6938 and squashes the following commits: ab321bb [Juliet Hougland] Remove braces from string interpolation when not required. 1221f94 [Juliet Hougland] All require statements now have an informative error message. --- .../org/apache/spark/mllib/linalg/Vectors.scala | 15 ++++++++++----- 1 file changed, 10 insertions(+), 5 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 328dbe2ce11fa..4ef171f4f0419 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 @@ -227,7 +227,7 @@ object Vectors { * @param elements vector elements in (index, value) pairs. */ def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { - require(size > 0) + require(size > 0, "The size of the requested sparse vector must be greater than 0.") val (indices, values) = elements.sortBy(_._1).unzip var prev = -1 @@ -235,7 +235,8 @@ object Vectors { require(prev < i, s"Found duplicate indices: $i.") prev = i } - require(prev < size) + require(prev < size, s"You may not write an element to index $prev because the declared " + + s"size of your vector is $size") new SparseVector(size, indices.toArray, values.toArray) } @@ -309,7 +310,8 @@ object Vectors { * @return norm in L^p^ space. */ def norm(vector: Vector, p: Double): Double = { - require(p >= 1.0) + require(p >= 1.0, "To compute the p-norm of the vector, we require that you specify a p>=1. " + + s"You specified p=$p.") val values = vector match { case DenseVector(vs) => vs case SparseVector(n, ids, vs) => vs @@ -360,7 +362,8 @@ object Vectors { * @return squared distance between two Vectors. */ def sqdist(v1: Vector, v2: Vector): Double = { - require(v1.size == v2.size, "vector dimension mismatch") + require(v1.size == v2.size, s"Vector dimensions do not match: Dim(v1)=${v1.size} and Dim(v2)" + + s"=${v2.size}.") var squaredDistance = 0.0 (v1, v2) match { case (v1: SparseVector, v2: SparseVector) => @@ -518,7 +521,9 @@ class SparseVector( val indices: Array[Int], val values: Array[Double]) extends Vector { - require(indices.length == values.length) + require(indices.length == values.length, "Sparse vectors require that the dimension of the" + + s" indices match the dimension of the values. You provided ${indices.size} indices and " + + s" ${values.size} values.") override def toString: String = "(%s,%s,%s)".format(size, indices.mkString("[", ",", "]"), values.mkString("[", ",", "]")) From 57cd1e86d1d450f85fc9e296aff498a940452113 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 15 Apr 2015 23:49:42 -0700 Subject: [PATCH 115/128] [SPARK-6893][ML] default pipeline parameter handling in python Same as #5431 but for Python. jkbradley Author: Xiangrui Meng Closes #5534 from mengxr/SPARK-6893 and squashes the following commits: d3b519b [Xiangrui Meng] address comments ebaccc6 [Xiangrui Meng] style update fce244e [Xiangrui Meng] update explainParams with test 4d6b07a [Xiangrui Meng] add tests 5294500 [Xiangrui Meng] update default param handling in python --- .../org/apache/spark/ml/Identifiable.scala | 2 +- .../apache/spark/ml/param/TestParams.scala | 9 +- python/pyspark/ml/classification.py | 3 +- python/pyspark/ml/feature.py | 19 +-- python/pyspark/ml/param/__init__.py | 146 +++++++++++++++--- ...d_params.py => _shared_params_code_gen.py} | 42 ++--- python/pyspark/ml/param/shared.py | 106 ++++++------- python/pyspark/ml/pipeline.py | 6 +- python/pyspark/ml/tests.py | 52 ++++++- python/pyspark/ml/util.py | 4 +- python/pyspark/ml/wrapper.py | 2 +- 11 files changed, 270 insertions(+), 121 deletions(-) rename python/pyspark/ml/param/{_gen_shared_params.py => _shared_params_code_gen.py} (70%) diff --git a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala b/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala index a50090671ae48..a1d49095c24ac 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Identifiable.scala @@ -25,7 +25,7 @@ import java.util.UUID private[ml] trait Identifiable extends Serializable { /** - * A unique id for the object. The default implementation concatenates the class name, "-", and 8 + * A unique id for the object. The default implementation concatenates the class name, "_", and 8 * random hex chars. */ private[ml] val uid: String = diff --git a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala index 8f9ab687c05cb..641b64b42a5e7 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/param/TestParams.scala @@ -17,16 +17,13 @@ package org.apache.spark.ml.param +import org.apache.spark.ml.param.shared.{HasInputCol, HasMaxIter} + /** A subclass of Params for testing. */ -class TestParams extends Params { +class TestParams extends Params with HasMaxIter with HasInputCol { - val maxIter = new IntParam(this, "maxIter", "max number of iterations") def setMaxIter(value: Int): this.type = { set(maxIter, value); this } - def getMaxIter: Int = getOrDefault(maxIter) - - val inputCol = new Param[String](this, "inputCol", "input column name") def setInputCol(value: String): this.type = { set(inputCol, value); this } - def getInputCol: String = getOrDefault(inputCol) setDefault(maxIter -> 10) diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index 7f42de531f3b4..d7bc09fd77adb 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -59,6 +59,7 @@ def __init__(self, featuresCol="features", labelCol="label", predictionCol="pred maxIter=100, regParam=0.1) """ super(LogisticRegression, self).__init__() + self._setDefault(maxIter=100, regParam=0.1) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @@ -71,7 +72,7 @@ def setParams(self, featuresCol="features", labelCol="label", predictionCol="pre Sets params for logistic regression. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) def _create_model(self, java_model): return LogisticRegressionModel(java_model) diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 1cfcd019dfb18..263fe2a5bcc41 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -52,22 +52,22 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): _java_class = "org.apache.spark.ml.feature.Tokenizer" @keyword_only - def __init__(self, inputCol="input", outputCol="output"): + def __init__(self, inputCol=None, outputCol=None): """ - __init__(self, inputCol="input", outputCol="output") + __init__(self, inputCol=None, outputCol=None) """ super(Tokenizer, self).__init__() kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only - def setParams(self, inputCol="input", outputCol="output"): + def setParams(self, inputCol=None, outputCol=None): """ setParams(self, inputCol="input", outputCol="output") Sets params for this Tokenizer. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) @inherit_doc @@ -91,22 +91,23 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): _java_class = "org.apache.spark.ml.feature.HashingTF" @keyword_only - def __init__(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): + def __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None): """ - __init__(self, numFeatures=1 << 18, inputCol="input", outputCol="output") + __init__(self, numFeatures=1 << 18, inputCol=None, outputCol=None) """ super(HashingTF, self).__init__() + self._setDefault(numFeatures=1 << 18) kwargs = self.__init__._input_kwargs self.setParams(**kwargs) @keyword_only - def setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output"): + def setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None): """ - setParams(self, numFeatures=1 << 18, inputCol="input", outputCol="output") + setParams(self, numFeatures=1 << 18, inputCol=None, outputCol=None) Sets params for this HashingTF. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) if __name__ == "__main__": diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index e3a53dd780c4c..5c62620562a84 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -25,23 +25,21 @@ class Param(object): """ - A param with self-contained documentation and optionally default value. + A param with self-contained documentation. """ - def __init__(self, parent, name, doc, defaultValue=None): - if not isinstance(parent, Identifiable): - raise ValueError("Parent must be identifiable but got type %s." % type(parent).__name__) + def __init__(self, parent, name, doc): + if not isinstance(parent, Params): + raise ValueError("Parent must be a Params but got type %s." % type(parent).__name__) self.parent = parent self.name = str(name) self.doc = str(doc) - self.defaultValue = defaultValue def __str__(self): - return str(self.parent) + "-" + self.name + return str(self.parent) + "__" + self.name def __repr__(self): - return "Param(parent=%r, name=%r, doc=%r, defaultValue=%r)" % \ - (self.parent, self.name, self.doc, self.defaultValue) + return "Param(parent=%r, name=%r, doc=%r)" % (self.parent, self.name, self.doc) class Params(Identifiable): @@ -52,26 +50,128 @@ class Params(Identifiable): __metaclass__ = ABCMeta - def __init__(self): - super(Params, self).__init__() - #: embedded param map - self.paramMap = {} + #: internal param map for user-supplied values param map + paramMap = {} + + #: internal param map for default values + defaultParamMap = {} @property def params(self): """ - Returns all params. The default implementation uses - :py:func:`dir` to get all attributes of type + Returns all params ordered by name. The default implementation + uses :py:func:`dir` to get all attributes of type :py:class:`Param`. """ return filter(lambda attr: isinstance(attr, Param), [getattr(self, x) for x in dir(self) if x != "params"]) - def _merge_params(self, params): - paramMap = self.paramMap.copy() - paramMap.update(params) + def _explain(self, param): + """ + Explains a single param and returns its name, doc, and optional + default value and user-supplied value in a string. + """ + param = self._resolveParam(param) + values = [] + if self.isDefined(param): + if param in self.defaultParamMap: + values.append("default: %s" % self.defaultParamMap[param]) + if param in self.paramMap: + values.append("current: %s" % self.paramMap[param]) + else: + values.append("undefined") + valueStr = "(" + ", ".join(values) + ")" + return "%s: %s %s" % (param.name, param.doc, valueStr) + + def explainParams(self): + """ + Returns the documentation of all params with their optionally + default values and user-supplied values. + """ + return "\n".join([self._explain(param) for param in self.params]) + + def getParam(self, paramName): + """ + Gets a param by its name. + """ + param = getattr(self, paramName) + if isinstance(param, Param): + return param + else: + raise ValueError("Cannot find param with name %s." % paramName) + + def isSet(self, param): + """ + Checks whether a param is explicitly set by user. + """ + param = self._resolveParam(param) + return param in self.paramMap + + def hasDefault(self, param): + """ + Checks whether a param has a default value. + """ + param = self._resolveParam(param) + return param in self.defaultParamMap + + def isDefined(self, param): + """ + Checks whether a param is explicitly set by user or has a default value. + """ + return self.isSet(param) or self.hasDefault(param) + + def getOrDefault(self, param): + """ + Gets the value of a param in the user-supplied param map or its + default value. Raises an error if either is set. + """ + if isinstance(param, Param): + if param in self.paramMap: + return self.paramMap[param] + else: + return self.defaultParamMap[param] + elif isinstance(param, str): + return self.getOrDefault(self.getParam(param)) + else: + raise KeyError("Cannot recognize %r as a param." % param) + + def extractParamMap(self, extraParamMap={}): + """ + Extracts the embedded default param values and user-supplied + values, and then merges them with extra values from input into + a flat param map, where the latter value is used if there exist + conflicts, i.e., with ordering: default param values < + user-supplied values < extraParamMap. + :param extraParamMap: extra param values + :return: merged param map + """ + paramMap = self.defaultParamMap.copy() + paramMap.update(self.paramMap) + paramMap.update(extraParamMap) return paramMap + def _shouldOwn(self, param): + """ + Validates that the input param belongs to this Params instance. + """ + if param.parent is not self: + raise ValueError("Param %r does not belong to %r." % (param, self)) + + def _resolveParam(self, param): + """ + Resolves a param and validates the ownership. + :param param: param name or the param instance, which must + belong to this Params instance + :return: resolved param instance + """ + if isinstance(param, Param): + self._shouldOwn(param) + return param + elif isinstance(param, str): + return self.getParam(param) + else: + raise ValueError("Cannot resolve %r as a param." % param) + @staticmethod def _dummy(): """ @@ -81,10 +181,18 @@ def _dummy(): dummy.uid = "undefined" return dummy - def _set_params(self, **kwargs): + def _set(self, **kwargs): """ - Sets params. + Sets user-supplied params. """ for param, value in kwargs.iteritems(): self.paramMap[getattr(self, param)] = value return self + + def _setDefault(self, **kwargs): + """ + Sets default params. + """ + for param, value in kwargs.iteritems(): + self.defaultParamMap[getattr(self, param)] = value + return self diff --git a/python/pyspark/ml/param/_gen_shared_params.py b/python/pyspark/ml/param/_shared_params_code_gen.py similarity index 70% rename from python/pyspark/ml/param/_gen_shared_params.py rename to python/pyspark/ml/param/_shared_params_code_gen.py index 5eb81106f116c..55f422497672f 100644 --- a/python/pyspark/ml/param/_gen_shared_params.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -32,29 +32,34 @@ # limitations under the License. #""" +# Code generator for shared params (shared.py). Run under this folder with: +# python _shared_params_code_gen.py > shared.py -def _gen_param_code(name, doc, defaultValue): + +def _gen_param_code(name, doc, defaultValueStr): """ Generates Python code for a shared param class. :param name: param name :param doc: param doc - :param defaultValue: string representation of the param + :param defaultValueStr: string representation of the default value :return: code string """ # TODO: How to correctly inherit instance attributes? template = '''class Has$Name(Params): """ - Params with $name. + Mixin for param $name: $doc. """ # a placeholder to make it appear in the generated doc - $name = Param(Params._dummy(), "$name", "$doc", $defaultValue) + $name = Param(Params._dummy(), "$name", "$doc") def __init__(self): super(Has$Name, self).__init__() #: param for $doc - self.$name = Param(self, "$name", "$doc", $defaultValue) + self.$name = Param(self, "$name", "$doc") + if $defaultValueStr is not None: + self._setDefault($name=$defaultValueStr) def set$Name(self, value): """ @@ -67,32 +72,29 @@ def get$Name(self): """ Gets the value of $name or its default value. """ - if self.$name in self.paramMap: - return self.paramMap[self.$name] - else: - return self.$name.defaultValue''' + return self.getOrDefault(self.$name)''' - upperCamelName = name[0].upper() + name[1:] + Name = name[0].upper() + name[1:] return template \ .replace("$name", name) \ - .replace("$Name", upperCamelName) \ + .replace("$Name", Name) \ .replace("$doc", doc) \ - .replace("$defaultValue", defaultValue) + .replace("$defaultValueStr", str(defaultValueStr)) if __name__ == "__main__": print header - print "\n# DO NOT MODIFY. The code is generated by _gen_shared_params.py.\n" + print "\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n" print "from pyspark.ml.param import Param, Params\n\n" shared = [ - ("maxIter", "max number of iterations", "100"), - ("regParam", "regularization constant", "0.1"), + ("maxIter", "max number of iterations", None), + ("regParam", "regularization constant", None), ("featuresCol", "features column name", "'features'"), ("labelCol", "label column name", "'label'"), ("predictionCol", "prediction column name", "'prediction'"), - ("inputCol", "input column name", "'input'"), - ("outputCol", "output column name", "'output'"), - ("numFeatures", "number of features", "1 << 18")] + ("inputCol", "input column name", None), + ("outputCol", "output column name", None), + ("numFeatures", "number of features", None)] code = [] - for name, doc, defaultValue in shared: - code.append(_gen_param_code(name, doc, defaultValue)) + for name, doc, defaultValueStr in shared: + code.append(_gen_param_code(name, doc, defaultValueStr)) print "\n\n\n".join(code) diff --git a/python/pyspark/ml/param/shared.py b/python/pyspark/ml/param/shared.py index 586822f2de423..13b6749998ad0 100644 --- a/python/pyspark/ml/param/shared.py +++ b/python/pyspark/ml/param/shared.py @@ -15,23 +15,25 @@ # limitations under the License. # -# DO NOT MODIFY. The code is generated by _gen_shared_params.py. +# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py. from pyspark.ml.param import Param, Params class HasMaxIter(Params): """ - Params with maxIter. + Mixin for param maxIter: max number of iterations. """ # a placeholder to make it appear in the generated doc - maxIter = Param(Params._dummy(), "maxIter", "max number of iterations", 100) + maxIter = Param(Params._dummy(), "maxIter", "max number of iterations") def __init__(self): super(HasMaxIter, self).__init__() #: param for max number of iterations - self.maxIter = Param(self, "maxIter", "max number of iterations", 100) + self.maxIter = Param(self, "maxIter", "max number of iterations") + if None is not None: + self._setDefault(maxIter=None) def setMaxIter(self, value): """ @@ -44,24 +46,23 @@ def getMaxIter(self): """ Gets the value of maxIter or its default value. """ - if self.maxIter in self.paramMap: - return self.paramMap[self.maxIter] - else: - return self.maxIter.defaultValue + return self.getOrDefault(self.maxIter) class HasRegParam(Params): """ - Params with regParam. + Mixin for param regParam: regularization constant. """ # a placeholder to make it appear in the generated doc - regParam = Param(Params._dummy(), "regParam", "regularization constant", 0.1) + regParam = Param(Params._dummy(), "regParam", "regularization constant") def __init__(self): super(HasRegParam, self).__init__() #: param for regularization constant - self.regParam = Param(self, "regParam", "regularization constant", 0.1) + self.regParam = Param(self, "regParam", "regularization constant") + if None is not None: + self._setDefault(regParam=None) def setRegParam(self, value): """ @@ -74,24 +75,23 @@ def getRegParam(self): """ Gets the value of regParam or its default value. """ - if self.regParam in self.paramMap: - return self.paramMap[self.regParam] - else: - return self.regParam.defaultValue + return self.getOrDefault(self.regParam) class HasFeaturesCol(Params): """ - Params with featuresCol. + Mixin for param featuresCol: features column name. """ # a placeholder to make it appear in the generated doc - featuresCol = Param(Params._dummy(), "featuresCol", "features column name", 'features') + featuresCol = Param(Params._dummy(), "featuresCol", "features column name") def __init__(self): super(HasFeaturesCol, self).__init__() #: param for features column name - self.featuresCol = Param(self, "featuresCol", "features column name", 'features') + self.featuresCol = Param(self, "featuresCol", "features column name") + if 'features' is not None: + self._setDefault(featuresCol='features') def setFeaturesCol(self, value): """ @@ -104,24 +104,23 @@ def getFeaturesCol(self): """ Gets the value of featuresCol or its default value. """ - if self.featuresCol in self.paramMap: - return self.paramMap[self.featuresCol] - else: - return self.featuresCol.defaultValue + return self.getOrDefault(self.featuresCol) class HasLabelCol(Params): """ - Params with labelCol. + Mixin for param labelCol: label column name. """ # a placeholder to make it appear in the generated doc - labelCol = Param(Params._dummy(), "labelCol", "label column name", 'label') + labelCol = Param(Params._dummy(), "labelCol", "label column name") def __init__(self): super(HasLabelCol, self).__init__() #: param for label column name - self.labelCol = Param(self, "labelCol", "label column name", 'label') + self.labelCol = Param(self, "labelCol", "label column name") + if 'label' is not None: + self._setDefault(labelCol='label') def setLabelCol(self, value): """ @@ -134,24 +133,23 @@ def getLabelCol(self): """ Gets the value of labelCol or its default value. """ - if self.labelCol in self.paramMap: - return self.paramMap[self.labelCol] - else: - return self.labelCol.defaultValue + return self.getOrDefault(self.labelCol) class HasPredictionCol(Params): """ - Params with predictionCol. + Mixin for param predictionCol: prediction column name. """ # a placeholder to make it appear in the generated doc - predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name", 'prediction') + predictionCol = Param(Params._dummy(), "predictionCol", "prediction column name") def __init__(self): super(HasPredictionCol, self).__init__() #: param for prediction column name - self.predictionCol = Param(self, "predictionCol", "prediction column name", 'prediction') + self.predictionCol = Param(self, "predictionCol", "prediction column name") + if 'prediction' is not None: + self._setDefault(predictionCol='prediction') def setPredictionCol(self, value): """ @@ -164,24 +162,23 @@ def getPredictionCol(self): """ Gets the value of predictionCol or its default value. """ - if self.predictionCol in self.paramMap: - return self.paramMap[self.predictionCol] - else: - return self.predictionCol.defaultValue + return self.getOrDefault(self.predictionCol) class HasInputCol(Params): """ - Params with inputCol. + Mixin for param inputCol: input column name. """ # a placeholder to make it appear in the generated doc - inputCol = Param(Params._dummy(), "inputCol", "input column name", 'input') + inputCol = Param(Params._dummy(), "inputCol", "input column name") def __init__(self): super(HasInputCol, self).__init__() #: param for input column name - self.inputCol = Param(self, "inputCol", "input column name", 'input') + self.inputCol = Param(self, "inputCol", "input column name") + if None is not None: + self._setDefault(inputCol=None) def setInputCol(self, value): """ @@ -194,24 +191,23 @@ def getInputCol(self): """ Gets the value of inputCol or its default value. """ - if self.inputCol in self.paramMap: - return self.paramMap[self.inputCol] - else: - return self.inputCol.defaultValue + return self.getOrDefault(self.inputCol) class HasOutputCol(Params): """ - Params with outputCol. + Mixin for param outputCol: output column name. """ # a placeholder to make it appear in the generated doc - outputCol = Param(Params._dummy(), "outputCol", "output column name", 'output') + outputCol = Param(Params._dummy(), "outputCol", "output column name") def __init__(self): super(HasOutputCol, self).__init__() #: param for output column name - self.outputCol = Param(self, "outputCol", "output column name", 'output') + self.outputCol = Param(self, "outputCol", "output column name") + if None is not None: + self._setDefault(outputCol=None) def setOutputCol(self, value): """ @@ -224,24 +220,23 @@ def getOutputCol(self): """ Gets the value of outputCol or its default value. """ - if self.outputCol in self.paramMap: - return self.paramMap[self.outputCol] - else: - return self.outputCol.defaultValue + return self.getOrDefault(self.outputCol) class HasNumFeatures(Params): """ - Params with numFeatures. + Mixin for param numFeatures: number of features. """ # a placeholder to make it appear in the generated doc - numFeatures = Param(Params._dummy(), "numFeatures", "number of features", 1 << 18) + numFeatures = Param(Params._dummy(), "numFeatures", "number of features") def __init__(self): super(HasNumFeatures, self).__init__() #: param for number of features - self.numFeatures = Param(self, "numFeatures", "number of features", 1 << 18) + self.numFeatures = Param(self, "numFeatures", "number of features") + if None is not None: + self._setDefault(numFeatures=None) def setNumFeatures(self, value): """ @@ -254,7 +249,4 @@ def getNumFeatures(self): """ Gets the value of numFeatures or its default value. """ - if self.numFeatures in self.paramMap: - return self.paramMap[self.numFeatures] - else: - return self.numFeatures.defaultValue + return self.getOrDefault(self.numFeatures) diff --git a/python/pyspark/ml/pipeline.py b/python/pyspark/ml/pipeline.py index 83880a5afcd1d..d94ecfff09f66 100644 --- a/python/pyspark/ml/pipeline.py +++ b/python/pyspark/ml/pipeline.py @@ -124,10 +124,10 @@ def setParams(self, stages=[]): Sets params for Pipeline. """ kwargs = self.setParams._input_kwargs - return self._set_params(**kwargs) + return self._set(**kwargs) def fit(self, dataset, params={}): - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) stages = paramMap[self.stages] for stage in stages: if not (isinstance(stage, Estimator) or isinstance(stage, Transformer)): @@ -164,7 +164,7 @@ def __init__(self, transformers): self.transformers = transformers def transform(self, dataset, params={}): - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) for t in self.transformers: dataset = t.transform(dataset, paramMap) return dataset diff --git a/python/pyspark/ml/tests.py b/python/pyspark/ml/tests.py index b627c2b4e930b..3a42bcf723894 100644 --- a/python/pyspark/ml/tests.py +++ b/python/pyspark/ml/tests.py @@ -33,6 +33,7 @@ from pyspark.tests import ReusedPySparkTestCase as PySparkTestCase from pyspark.sql import DataFrame from pyspark.ml.param import Param +from pyspark.ml.param.shared import HasMaxIter, HasInputCol from pyspark.ml.pipeline import Transformer, Estimator, Pipeline @@ -46,7 +47,7 @@ class MockTransformer(Transformer): def __init__(self): super(MockTransformer, self).__init__() - self.fake = Param(self, "fake", "fake", None) + self.fake = Param(self, "fake", "fake") self.dataset_index = None self.fake_param_value = None @@ -62,7 +63,7 @@ class MockEstimator(Estimator): def __init__(self): super(MockEstimator, self).__init__() - self.fake = Param(self, "fake", "fake", None) + self.fake = Param(self, "fake", "fake") self.dataset_index = None self.fake_param_value = None self.model = None @@ -111,5 +112,52 @@ def test_pipeline(self): self.assertEqual(6, dataset.index) +class TestParams(HasMaxIter, HasInputCol): + """ + A subclass of Params mixed with HasMaxIter and HasInputCol. + """ + + def __init__(self): + super(TestParams, self).__init__() + self._setDefault(maxIter=10) + + +class ParamTests(PySparkTestCase): + + def test_param(self): + testParams = TestParams() + maxIter = testParams.maxIter + self.assertEqual(maxIter.name, "maxIter") + self.assertEqual(maxIter.doc, "max number of iterations") + self.assertTrue(maxIter.parent is testParams) + + def test_params(self): + testParams = TestParams() + maxIter = testParams.maxIter + inputCol = testParams.inputCol + + params = testParams.params + self.assertEqual(params, [inputCol, maxIter]) + + self.assertTrue(testParams.hasDefault(maxIter)) + self.assertFalse(testParams.isSet(maxIter)) + self.assertTrue(testParams.isDefined(maxIter)) + self.assertEqual(testParams.getMaxIter(), 10) + testParams.setMaxIter(100) + self.assertTrue(testParams.isSet(maxIter)) + self.assertEquals(testParams.getMaxIter(), 100) + + self.assertFalse(testParams.hasDefault(inputCol)) + self.assertFalse(testParams.isSet(inputCol)) + self.assertFalse(testParams.isDefined(inputCol)) + with self.assertRaises(KeyError): + testParams.getInputCol() + + self.assertEquals( + testParams.explainParams(), + "\n".join(["inputCol: input column name (undefined)", + "maxIter: max number of iterations (default: 10, current: 100)"])) + + if __name__ == "__main__": unittest.main() diff --git a/python/pyspark/ml/util.py b/python/pyspark/ml/util.py index 6f7f39c40eb5a..d3cb100a9efa5 100644 --- a/python/pyspark/ml/util.py +++ b/python/pyspark/ml/util.py @@ -40,8 +40,8 @@ class Identifiable(object): def __init__(self): #: A unique id for the object. The default implementation - #: concatenates the class name, "-", and 8 random hex chars. - self.uid = type(self).__name__ + "-" + uuid.uuid4().hex[:8] + #: concatenates the class name, "_", and 8 random hex chars. + self.uid = type(self).__name__ + "_" + uuid.uuid4().hex[:8] def __repr__(self): return self.uid diff --git a/python/pyspark/ml/wrapper.py b/python/pyspark/ml/wrapper.py index 31a66b3d2f730..394f23c5e9b12 100644 --- a/python/pyspark/ml/wrapper.py +++ b/python/pyspark/ml/wrapper.py @@ -64,7 +64,7 @@ def _transfer_params_to_java(self, params, java_obj): :param params: additional params (overwriting embedded values) :param java_obj: Java object to receive the params """ - paramMap = self._merge_params(params) + paramMap = self.extractParamMap(params) for param in self.params: if param in paramMap: java_obj.set(param.name, paramMap[param]) From 8370550593f3549e90ace446961281dad0cd7498 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 16 Apr 2015 10:39:02 +0100 Subject: [PATCH 116/128] [Streaming][minor] Remove additional quote and unneeded imports Author: jerryshao Closes #5540 from jerryshao/minor-fix and squashes the following commits: ebaa646 [jerryshao] Minor fix --- .../apache/spark/examples/streaming/DirectKafkaWordCount.scala | 2 +- .../main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala | 3 +-- 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala index 1c8a20bf8f1ae..11a8cf09533ce 100644 --- a/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala +++ b/examples/scala-2.10/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala @@ -41,7 +41,7 @@ object DirectKafkaWordCount { | is a list of one or more Kafka brokers | is a list of one or more kafka topics to consume from | - """".stripMargin) + """.stripMargin) System.exit(1) } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala index a0b8a0c565210..a1b4a12e5d6a0 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala @@ -23,10 +23,9 @@ import org.apache.spark.{Logging, Partition, SparkContext, SparkException, TaskC import org.apache.spark.rdd.RDD import org.apache.spark.util.NextIterator -import java.util.Properties import kafka.api.{FetchRequestBuilder, FetchResponse} import kafka.common.{ErrorMapping, TopicAndPartition} -import kafka.consumer.{ConsumerConfig, SimpleConsumer} +import kafka.consumer.SimpleConsumer import kafka.message.{MessageAndMetadata, MessageAndOffset} import kafka.serializer.Decoder import kafka.utils.VerifiableProperties From 6179a948371897cecb7322ebda366c2de8ecaedd Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 16 Apr 2015 10:45:32 +0100 Subject: [PATCH 117/128] SPARK-4783 [CORE] System.exit() calls in SparkContext disrupt applications embedding Spark Avoid `System.exit(1)` in `TaskSchedulerImpl` and convert to `SparkException`; ensure scheduler calls `sc.stop()` even when this exception is thrown. CC mateiz aarondav as those who may have last touched this code. Author: Sean Owen Closes #5492 from srowen/SPARK-4783 and squashes the following commits: 60dc682 [Sean Owen] Avoid System.exit(1) in TaskSchedulerImpl and convert to SparkException; ensure scheduler calls sc.stop() even when this exception is thrown --- .../org/apache/spark/scheduler/TaskSchedulerImpl.scala | 5 ++--- .../scheduler/cluster/SparkDeploySchedulerBackend.scala | 9 ++++++--- 2 files changed, 8 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 2362cc7240039..ecc8bf189986d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -394,7 +394,7 @@ private[spark] class TaskSchedulerImpl( def error(message: String) { synchronized { - if (activeTaskSets.size > 0) { + if (activeTaskSets.nonEmpty) { // Have each task set throw a SparkException with the error for ((taskSetId, manager) <- activeTaskSets) { try { @@ -407,8 +407,7 @@ private[spark] class TaskSchedulerImpl( // No task sets are active but we still got an error. Just exit since this // must mean the error is during registration. // It might be good to do something smarter here in the future. - logError("Exiting due to error from cluster scheduler: " + message) - System.exit(1) + throw new SparkException(s"Exiting due to error from cluster scheduler: $message") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index ed5b7c1088196..ccf1dc5af6120 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -118,9 +118,12 @@ private[spark] class SparkDeploySchedulerBackend( notifyContext() if (!stopping) { logError("Application has been killed. Reason: " + reason) - scheduler.error(reason) - // Ensure the application terminates, as we can no longer run jobs. - sc.stop() + try { + scheduler.error(reason) + } finally { + // Ensure the application terminates, as we can no longer run jobs. + sc.stop() + } } } From de4fa6b6d12e2bee0307ffba2abfca0c33f15e45 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Thu, 16 Apr 2015 10:48:31 +0100 Subject: [PATCH 118/128] [SPARK-4194] [core] Make SparkContext initialization exception-safe. SparkContext has a very long constructor, where multiple things are initialized, multiple threads are spawned, and multiple opportunities for exceptions to be thrown exist. If one of these happens at an innoportune time, lots of garbage tends to stick around. This patch re-organizes SparkContext so that its internal state is initialized in a big "try" block. The fields keeping state are now completely private to SparkContext, and are "vars", because Scala doesn't allow you to initialize a val later. The existing API interface is kept by turning vals into defs (which works because Scala guarantees the same binary interface for those). On top of that, a few things in other areas were changed to avoid more things leaking: - Executor was changed to explicitly wait for the heartbeat thread to stop. LocalBackend was changed to wait for the "StopExecutor" message to be received, since otherwise there could be a race between that message arriving and the actor system being shut down. - ConnectionManager could possibly hang during shutdown, because an interrupt at the wrong moment could cause the selector thread to still call select and then wait forever. So also wake up the selector so that this situation is avoided. Author: Marcelo Vanzin Closes #5335 from vanzin/SPARK-4194 and squashes the following commits: 746b661 [Marcelo Vanzin] Fix borked merge. 80fc00e [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 408dada [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 2621609 [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 6b73fcb [Marcelo Vanzin] Scalastyle. c671c46 [Marcelo Vanzin] Fix merge. 3979aad [Marcelo Vanzin] Merge branch 'master' into SPARK-4194 8caa8b3 [Marcelo Vanzin] [SPARK-4194] [core] Make SparkContext initialization exception-safe. 071f16e [Marcelo Vanzin] Nits. 27456b9 [Marcelo Vanzin] More exception safety. a0b0881 [Marcelo Vanzin] Stop alloc manager before scheduler. 5545d83 [Marcelo Vanzin] [SPARK-6650] [core] Stop ExecutorAllocationManager when context stops. --- .../scala/org/apache/spark/SparkContext.scala | 505 ++++++++++-------- .../org/apache/spark/executor/Executor.scala | 33 +- .../spark/network/nio/ConnectionManager.scala | 7 +- .../spark/scheduler/TaskSchedulerImpl.scala | 3 +- .../spark/scheduler/local/LocalBackend.scala | 19 +- .../ExecutorAllocationManagerSuite.scala | 6 - 6 files changed, 329 insertions(+), 244 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3f1a7dd99d635..e106c5c4bef60 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -31,6 +31,7 @@ import scala.collection.JavaConversions._ import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.reflect.{ClassTag, classTag} +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -50,9 +51,10 @@ import org.apache.spark.executor.{ExecutorEndpoint, TriggerThreadDump} import org.apache.spark.input.{StreamInputFormat, PortableDataStream, WholeTextFileInputFormat, FixedLengthBinaryInputFormat} import org.apache.spark.io.CompressionCodec +import org.apache.spark.metrics.MetricsSystem import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ -import org.apache.spark.rpc.RpcAddress +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SparkDeploySchedulerBackend, SimrSchedulerBackend} @@ -192,8 +194,42 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // log out Spark Version in Spark driver log logInfo(s"Running Spark version $SPARK_VERSION") - private[spark] val conf = config.clone() - conf.validateSettings() + /* ------------------------------------------------------------------------------------- * + | Private variables. These variables keep the internal state of the context, and are | + | not accessible by the outside world. They're mutable since we want to initialize all | + | of them to some neutral value ahead of time, so that calling "stop()" while the | + | constructor is still running is safe. | + * ------------------------------------------------------------------------------------- */ + + private var _conf: SparkConf = _ + private var _eventLogDir: Option[URI] = None + private var _eventLogCodec: Option[String] = None + private var _env: SparkEnv = _ + private var _metadataCleaner: MetadataCleaner = _ + private var _jobProgressListener: JobProgressListener = _ + private var _statusTracker: SparkStatusTracker = _ + private var _progressBar: Option[ConsoleProgressBar] = None + private var _ui: Option[SparkUI] = None + private var _hadoopConfiguration: Configuration = _ + private var _executorMemory: Int = _ + private var _schedulerBackend: SchedulerBackend = _ + private var _taskScheduler: TaskScheduler = _ + private var _heartbeatReceiver: RpcEndpointRef = _ + @volatile private var _dagScheduler: DAGScheduler = _ + private var _applicationId: String = _ + private var _eventLogger: Option[EventLoggingListener] = None + private var _executorAllocationManager: Option[ExecutorAllocationManager] = None + private var _cleaner: Option[ContextCleaner] = None + private var _listenerBusStarted: Boolean = false + private var _jars: Seq[String] = _ + private var _files: Seq[String] = _ + + /* ------------------------------------------------------------------------------------- * + | Accessors and public fields. These provide access to the internal state of the | + | context. | + * ------------------------------------------------------------------------------------- */ + + private[spark] def conf: SparkConf = _conf /** * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be @@ -201,65 +237,24 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli */ def getConf: SparkConf = conf.clone() - if (!conf.contains("spark.master")) { - throw new SparkException("A master URL must be set in your configuration") - } - if (!conf.contains("spark.app.name")) { - throw new SparkException("An application name must be set in your configuration") - } - - if (conf.getBoolean("spark.logConf", false)) { - logInfo("Spark configuration:\n" + conf.toDebugString) - } - - // Set Spark driver host and port system properties - conf.setIfMissing("spark.driver.host", Utils.localHostName()) - conf.setIfMissing("spark.driver.port", "0") - - val jars: Seq[String] = - conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten - - val files: Seq[String] = - conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten - - val master = conf.get("spark.master") - val appName = conf.get("spark.app.name") + def jars: Seq[String] = _jars + def files: Seq[String] = _files + def master: String = _conf.get("spark.master") + def appName: String = _conf.get("spark.app.name") - private[spark] val isEventLogEnabled = conf.getBoolean("spark.eventLog.enabled", false) - private[spark] val eventLogDir: Option[URI] = { - if (isEventLogEnabled) { - val unresolvedDir = conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR) - .stripSuffix("/") - Some(Utils.resolveURI(unresolvedDir)) - } else { - None - } - } - private[spark] val eventLogCodec: Option[String] = { - val compress = conf.getBoolean("spark.eventLog.compress", false) - if (compress && isEventLogEnabled) { - Some(CompressionCodec.getCodecName(conf)).map(CompressionCodec.getShortName) - } else { - None - } - } + private[spark] def isEventLogEnabled: Boolean = _conf.getBoolean("spark.eventLog.enabled", false) + private[spark] def eventLogDir: Option[URI] = _eventLogDir + private[spark] def eventLogCodec: Option[String] = _eventLogCodec // Generate the random name for a temp folder in Tachyon // Add a timestamp as the suffix here to make it more safe val tachyonFolderName = "spark-" + randomUUID.toString() - conf.set("spark.tachyonStore.folderName", tachyonFolderName) - val isLocal = (master == "local" || master.startsWith("local[")) - - if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + def isLocal: Boolean = (master == "local" || master.startsWith("local[")) // An asynchronous listener bus for Spark events private[spark] val listenerBus = new LiveListenerBus - conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) - - // Create the Spark execution environment (cache, map output tracker, etc) - // This function allows components created by SparkEnv to be mocked in unit tests: private[spark] def createSparkEnv( conf: SparkConf, @@ -268,8 +263,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli SparkEnv.createDriverEnv(conf, isLocal, listenerBus) } - private[spark] val env = createSparkEnv(conf, isLocal, listenerBus) - SparkEnv.set(env) + private[spark] def env: SparkEnv = _env // Used to store a URL for each static file/jar together with the file's local timestamp private[spark] val addedFiles = HashMap[String, Long]() @@ -277,35 +271,14 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli // Keeps track of all persisted RDDs private[spark] val persistentRdds = new TimeStampedWeakValueHashMap[Int, RDD[_]] - private[spark] val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) - + private[spark] def metadataCleaner: MetadataCleaner = _metadataCleaner + private[spark] def jobProgressListener: JobProgressListener = _jobProgressListener - private[spark] val jobProgressListener = new JobProgressListener(conf) - listenerBus.addListener(jobProgressListener) + def statusTracker: SparkStatusTracker = _statusTracker - val statusTracker = new SparkStatusTracker(this) + private[spark] def progressBar: Option[ConsoleProgressBar] = _progressBar - private[spark] val progressBar: Option[ConsoleProgressBar] = - if (conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) { - Some(new ConsoleProgressBar(this)) - } else { - None - } - - // Initialize the Spark UI - private[spark] val ui: Option[SparkUI] = - if (conf.getBoolean("spark.ui.enabled", true)) { - Some(SparkUI.createLiveUI(this, conf, listenerBus, jobProgressListener, - env.securityManager,appName)) - } else { - // For tests, do not enable the UI - None - } - - // Bind the UI before starting the task scheduler to communicate - // the bound port to the cluster manager properly - ui.foreach(_.bind()) + private[spark] def ui: Option[SparkUI] = _ui /** * A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. @@ -313,134 +286,248 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * '''Note:''' As it will be reused in all Hadoop RDDs, it's better not to modify it unless you * plan to set some global configurations for all Hadoop RDDs. */ - val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(conf) + def hadoopConfiguration: Configuration = _hadoopConfiguration + + private[spark] def executorMemory: Int = _executorMemory + + // Environment variables to pass to our executors. + private[spark] val executorEnvs = HashMap[String, String]() + + // Set SPARK_USER for user who is running SparkContext. + val sparkUser = Utils.getCurrentUserName() - // Add each JAR given through the constructor - if (jars != null) { - jars.foreach(addJar) + private[spark] def schedulerBackend: SchedulerBackend = _schedulerBackend + private[spark] def schedulerBackend_=(sb: SchedulerBackend): Unit = { + _schedulerBackend = sb } - if (files != null) { - files.foreach(addFile) + private[spark] def taskScheduler: TaskScheduler = _taskScheduler + private[spark] def taskScheduler_=(ts: TaskScheduler): Unit = { + _taskScheduler = ts } + private[spark] def dagScheduler: DAGScheduler = _dagScheduler + private[spark] def dagScheduler_=(ds: DAGScheduler): Unit = { + _dagScheduler = ds + } + + def applicationId: String = _applicationId + + def metricsSystem: MetricsSystem = if (_env != null) _env.metricsSystem else null + + private[spark] def eventLogger: Option[EventLoggingListener] = _eventLogger + + private[spark] def executorAllocationManager: Option[ExecutorAllocationManager] = + _executorAllocationManager + + private[spark] def cleaner: Option[ContextCleaner] = _cleaner + + private[spark] var checkpointDir: Option[String] = None + + // Thread Local variable that can be used by users to pass information down the stack + private val localProperties = new InheritableThreadLocal[Properties] { + override protected def childValue(parent: Properties): Properties = new Properties(parent) + override protected def initialValue(): Properties = new Properties() + } + + /* ------------------------------------------------------------------------------------- * + | Initialization. This code initializes the context in a manner that is exception-safe. | + | All internal fields holding state are initialized here, and any error prompts the | + | stop() method to be called. | + * ------------------------------------------------------------------------------------- */ + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") value } - private[spark] val executorMemory = conf.getOption("spark.executor.memory") - .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) - .orElse(Option(System.getenv("SPARK_MEM")).map(warnSparkMem)) - .map(Utils.memoryStringToMb) - .getOrElse(512) + try { + _conf = config.clone() + _conf.validateSettings() - // Environment variables to pass to our executors. - private[spark] val executorEnvs = HashMap[String, String]() + if (!_conf.contains("spark.master")) { + throw new SparkException("A master URL must be set in your configuration") + } + if (!_conf.contains("spark.app.name")) { + throw new SparkException("An application name must be set in your configuration") + } - // Convert java options to env vars as a work around - // since we can't set env vars directly in sbt. - for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) - value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { - executorEnvs(envKey) = value - } - Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => - executorEnvs("SPARK_PREPEND_CLASSES") = v - } - // The Mesos scheduler backend relies on this environment variable to set executor memory. - // TODO: Set this only in the Mesos scheduler. - executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" - executorEnvs ++= conf.getExecutorEnv + if (_conf.getBoolean("spark.logConf", false)) { + logInfo("Spark configuration:\n" + _conf.toDebugString) + } - // Set SPARK_USER for user who is running SparkContext. - val sparkUser = Utils.getCurrentUserName() - executorEnvs("SPARK_USER") = sparkUser + // Set Spark driver host and port system properties + _conf.setIfMissing("spark.driver.host", Utils.localHostName()) + _conf.setIfMissing("spark.driver.port", "0") - // We need to register "HeartbeatReceiver" before "createTaskScheduler" because Executor will - // retrieve "HeartbeatReceiver" in the constructor. (SPARK-6640) - private val heartbeatReceiver = env.rpcEnv.setupEndpoint( - HeartbeatReceiver.ENDPOINT_NAME, new HeartbeatReceiver(this)) + _conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) - // Create and start the scheduler - private[spark] var (schedulerBackend, taskScheduler) = - SparkContext.createTaskScheduler(this, master) + _jars =_conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten + _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)) + .toSeq.flatten - heartbeatReceiver.send(TaskSchedulerIsSet) + _eventLogDir = + if (isEventLogEnabled) { + val unresolvedDir = conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR) + .stripSuffix("/") + Some(Utils.resolveURI(unresolvedDir)) + } else { + None + } - @volatile private[spark] var dagScheduler: DAGScheduler = _ - try { - dagScheduler = new DAGScheduler(this) - } catch { - case e: Exception => { - try { - stop() - } finally { - throw new SparkException("Error while constructing DAGScheduler", e) + _eventLogCodec = { + val compress = _conf.getBoolean("spark.eventLog.compress", false) + if (compress && isEventLogEnabled) { + Some(CompressionCodec.getCodecName(_conf)).map(CompressionCodec.getShortName) + } else { + None } } - } - // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's - // constructor - taskScheduler.start() + _conf.set("spark.tachyonStore.folderName", tachyonFolderName) - val applicationId: String = taskScheduler.applicationId() - conf.set("spark.app.id", applicationId) + if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") - env.blockManager.initialize(applicationId) + // Create the Spark execution environment (cache, map output tracker, etc) + _env = createSparkEnv(_conf, isLocal, listenerBus) + SparkEnv.set(_env) - val metricsSystem = env.metricsSystem + _metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, _conf) - // The metrics system for Driver need to be set spark.app.id to app ID. - // So it should start after we get app ID from the task scheduler and set spark.app.id. - metricsSystem.start() - // Attach the driver metrics servlet handler to the web ui after the metrics system is started. - metricsSystem.getServletHandlers.foreach(handler => ui.foreach(_.attachHandler(handler))) + _jobProgressListener = new JobProgressListener(_conf) + listenerBus.addListener(jobProgressListener) - // Optionally log Spark events - private[spark] val eventLogger: Option[EventLoggingListener] = { - if (isEventLogEnabled) { - val logger = - new EventLoggingListener(applicationId, eventLogDir.get, conf, hadoopConfiguration) - logger.start() - listenerBus.addListener(logger) - Some(logger) - } else None - } + _statusTracker = new SparkStatusTracker(this) - // 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 (dynamicAllocationEnabled) { - assert(supportDynamicAllocation, - "Dynamic allocation of executors is currently only supported in YARN mode") - Some(new ExecutorAllocationManager(this, listenerBus, conf)) - } else { - None + _progressBar = + if (_conf.getBoolean("spark.ui.showConsoleProgress", true) && !log.isInfoEnabled) { + Some(new ConsoleProgressBar(this)) + } else { + None + } + + _ui = + if (conf.getBoolean("spark.ui.enabled", true)) { + Some(SparkUI.createLiveUI(this, _conf, listenerBus, _jobProgressListener, + _env.securityManager,appName)) + } else { + // For tests, do not enable the UI + None + } + // Bind the UI before starting the task scheduler to communicate + // the bound port to the cluster manager properly + _ui.foreach(_.bind()) + + _hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(_conf) + + // Add each JAR given through the constructor + if (jars != null) { + jars.foreach(addJar) } - executorAllocationManager.foreach(_.start()) - private[spark] val cleaner: Option[ContextCleaner] = { - if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { - Some(new ContextCleaner(this)) - } else { - None + if (files != null) { + files.foreach(addFile) } - } - cleaner.foreach(_.start()) - setupAndStartListenerBus() - postEnvironmentUpdate() - postApplicationStart() + _executorMemory = _conf.getOption("spark.executor.memory") + .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) + .orElse(Option(System.getenv("SPARK_MEM")) + .map(warnSparkMem)) + .map(Utils.memoryStringToMb) + .getOrElse(512) + + // Convert java options to env vars as a work around + // since we can't set env vars directly in sbt. + for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) + value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { + executorEnvs(envKey) = value + } + Option(System.getenv("SPARK_PREPEND_CLASSES")).foreach { v => + executorEnvs("SPARK_PREPEND_CLASSES") = v + } + // The Mesos scheduler backend relies on this environment variable to set executor memory. + // TODO: Set this only in the Mesos scheduler. + executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" + executorEnvs ++= _conf.getExecutorEnv + executorEnvs("SPARK_USER") = sparkUser + + // We need to register "HeartbeatReceiver" before "createTaskScheduler" because Executor will + // retrieve "HeartbeatReceiver" in the constructor. (SPARK-6640) + _heartbeatReceiver = env.rpcEnv.setupEndpoint( + HeartbeatReceiver.ENDPOINT_NAME, new HeartbeatReceiver(this)) + + // Create and start the scheduler + val (sched, ts) = SparkContext.createTaskScheduler(this, master) + _schedulerBackend = sched + _taskScheduler = ts + _dagScheduler = new DAGScheduler(this) + _heartbeatReceiver.send(TaskSchedulerIsSet) + + // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's + // constructor + _taskScheduler.start() + + _applicationId = _taskScheduler.applicationId() + _conf.set("spark.app.id", _applicationId) + _env.blockManager.initialize(_applicationId) + + // The metrics system for Driver need to be set spark.app.id to app ID. + // So it should start after we get app ID from the task scheduler and set spark.app.id. + metricsSystem.start() + // Attach the driver metrics servlet handler to the web ui after the metrics system is started. + metricsSystem.getServletHandlers.foreach(handler => ui.foreach(_.attachHandler(handler))) + + _eventLogger = + if (isEventLogEnabled) { + val logger = + new EventLoggingListener(_applicationId, _eventLogDir.get, _conf, _hadoopConfiguration) + logger.start() + listenerBus.addListener(logger) + Some(logger) + } else { + None + } - private[spark] var checkpointDir: Option[String] = None + // Optionally scale number of executors dynamically based on workload. Exposed for testing. + val dynamicAllocationEnabled = _conf.getBoolean("spark.dynamicAllocation.enabled", false) + _executorAllocationManager = + if (dynamicAllocationEnabled) { + assert(supportDynamicAllocation, + "Dynamic allocation of executors is currently only supported in YARN mode") + Some(new ExecutorAllocationManager(this, listenerBus, _conf)) + } else { + None + } + _executorAllocationManager.foreach(_.start()) - // Thread Local variable that can be used by users to pass information down the stack - private val localProperties = new InheritableThreadLocal[Properties] { - override protected def childValue(parent: Properties): Properties = new Properties(parent) - override protected def initialValue(): Properties = new Properties() + _cleaner = + if (_conf.getBoolean("spark.cleaner.referenceTracking", true)) { + Some(new ContextCleaner(this)) + } else { + None + } + _cleaner.foreach(_.start()) + + setupAndStartListenerBus() + postEnvironmentUpdate() + postApplicationStart() + + // Post init + _taskScheduler.postStartHook() + _env.metricsSystem.registerSource(new DAGSchedulerSource(dagScheduler)) + _env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager)) + } catch { + case NonFatal(e) => + logError("Error initializing SparkContext.", e) + try { + stop() + } catch { + case NonFatal(inner) => + logError("Error stopping SparkContext after init error.", inner) + } finally { + throw e + } } /** @@ -544,19 +631,6 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, null) } - // Post init - taskScheduler.postStartHook() - - private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler) - private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager) - - private def initDriverMetrics() { - SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) - SparkEnv.get.metricsSystem.registerSource(blockManagerSource) - } - - initDriverMetrics() - // Methods for creating RDDs /** Distribute a local Scala collection to form an RDD. @@ -1146,7 +1220,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * this application is supported. This is currently only available for YARN. */ private[spark] def supportDynamicAllocation = - master.contains("yarn") || dynamicAllocationTesting + master.contains("yarn") || _conf.getBoolean("spark.dynamicAllocation.testing", false) /** * :: DeveloperApi :: @@ -1163,7 +1237,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli * This is currently only supported in YARN mode. Return whether the request is received. */ private[spark] override def requestTotalExecutors(numExecutors: Int): Boolean = { - assert(master.contains("yarn") || dynamicAllocationTesting, + assert(supportDynamicAllocation, "Requesting executors is currently only supported in YARN mode") schedulerBackend match { case b: CoarseGrainedSchedulerBackend => @@ -1403,28 +1477,40 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli def stop() { // Use the stopping variable to ensure no contention for the stop scenario. // Still track the stopped variable for use elsewhere in the code. - if (!stopped.compareAndSet(false, true)) { logInfo("SparkContext already stopped.") return } - + postApplicationEnd() - ui.foreach(_.stop()) - env.metricsSystem.report() - metadataCleaner.cancel() - cleaner.foreach(_.stop()) - executorAllocationManager.foreach(_.stop()) - dagScheduler.stop() - dagScheduler = null - listenerBus.stop() - eventLogger.foreach(_.stop()) - env.rpcEnv.stop(heartbeatReceiver) - progressBar.foreach(_.stop()) - taskScheduler = null + _ui.foreach(_.stop()) + if (env != null) { + env.metricsSystem.report() + } + if (metadataCleaner != null) { + metadataCleaner.cancel() + } + _cleaner.foreach(_.stop()) + _executorAllocationManager.foreach(_.stop()) + if (_dagScheduler != null) { + _dagScheduler.stop() + _dagScheduler = null + } + if (_listenerBusStarted) { + listenerBus.stop() + _listenerBusStarted = false + } + _eventLogger.foreach(_.stop()) + if (env != null && _heartbeatReceiver != null) { + env.rpcEnv.stop(_heartbeatReceiver) + } + _progressBar.foreach(_.stop()) + _taskScheduler = null // TODO: Cache.stop()? - env.stop() - SparkEnv.set(null) + if (_env != null) { + _env.stop() + SparkEnv.set(null) + } SparkContext.clearActiveContext() logInfo("Successfully stopped SparkContext") } @@ -1749,6 +1835,7 @@ class SparkContext(config: SparkConf) extends Logging with ExecutorAllocationCli } listenerBus.start(this) + _listenerBusStarted = true } /** Post the application start event */ @@ -2152,7 +2239,7 @@ object SparkContext extends Logging { master match { case "local" => val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, 1) + val backend = new LocalBackend(sc.getConf, scheduler, 1) scheduler.initialize(backend) (backend, scheduler) @@ -2164,7 +2251,7 @@ object SparkContext extends Logging { throw new SparkException(s"Asked to run locally with $threadCount threads") } val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, threadCount) + val backend = new LocalBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) @@ -2174,7 +2261,7 @@ object SparkContext extends Logging { // local[N, M] means exactly N threads with M failures val threadCount = if (threads == "*") localCpuCount else threads.toInt val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) - val backend = new LocalBackend(scheduler, threadCount) + val backend = new LocalBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) 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 516f619529c48..1b5fdeba28ee2 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -21,7 +21,7 @@ import java.io.File import java.lang.management.ManagementFactory import java.net.URL import java.nio.ByteBuffer -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.{ConcurrentHashMap, Executors, TimeUnit} import scala.collection.JavaConversions._ import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -60,8 +60,6 @@ private[spark] class Executor( private val conf = env.conf - @volatile private var isStopped = false - // No ip or host:port - just hostname Utils.checkHost(executorHostname, "Expected executed slave to be a hostname") // must not have port specified. @@ -114,6 +112,10 @@ private[spark] class Executor( // Maintains the list of running tasks. private val runningTasks = new ConcurrentHashMap[Long, TaskRunner] + // Executor for the heartbeat task. + private val heartbeater = Executors.newSingleThreadScheduledExecutor( + Utils.namedThreadFactory("driver-heartbeater")) + startDriverHeartbeater() def launchTask( @@ -138,7 +140,8 @@ private[spark] class Executor( def stop(): Unit = { env.metricsSystem.report() env.rpcEnv.stop(executorEndpoint) - isStopped = true + heartbeater.shutdown() + heartbeater.awaitTermination(10, TimeUnit.SECONDS) threadPool.shutdown() if (!isLocal) { env.stop() @@ -432,23 +435,17 @@ private[spark] class Executor( } /** - * Starts a thread to report heartbeat and partial metrics for active tasks to driver. - * This thread stops running when the executor is stopped. + * Schedules a task to report heartbeat and partial metrics for active tasks to driver. */ private def startDriverHeartbeater(): Unit = { val intervalMs = conf.getTimeAsMs("spark.executor.heartbeatInterval", "10s") - val thread = new Thread() { - override def run() { - // Sleep a random interval so the heartbeats don't end up in sync - Thread.sleep(intervalMs + (math.random * intervalMs).asInstanceOf[Int]) - while (!isStopped) { - reportHeartBeat() - Thread.sleep(intervalMs) - } - } + + // Wait a random interval so the heartbeats don't end up in sync + val initialDelay = intervalMs + (math.random * intervalMs).asInstanceOf[Int] + + val heartbeatTask = new Runnable() { + override def run(): Unit = Utils.logUncaughtExceptions(reportHeartBeat()) } - thread.setDaemon(true) - thread.setName("driver-heartbeater") - thread.start() + heartbeater.scheduleAtFixedRate(heartbeatTask, initialDelay, intervalMs, TimeUnit.MILLISECONDS) } } diff --git a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala index 8e3c30fc3d781..5a74c13b38bf7 100644 --- a/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/nio/ConnectionManager.scala @@ -86,11 +86,11 @@ private[nio] class ConnectionManager( conf.get("spark.network.timeout", "120s")) // Get the thread counts from the Spark Configuration. - // + // // Even though the ThreadPoolExecutor constructor takes both a minimum and maximum value, // we only query for the minimum value because we are using LinkedBlockingDeque. - // - // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is + // + // The JavaDoc for ThreadPoolExecutor points out that when using a LinkedBlockingDeque (which is // an unbounded queue) no more than corePoolSize threads will ever be created, so only the "min" // parameter is necessary. private val handlerThreadCount = conf.getInt("spark.core.connection.handler.threads.min", 20) @@ -989,6 +989,7 @@ private[nio] class ConnectionManager( def stop() { ackTimeoutMonitor.stop() + selector.wakeup() selectorThread.interrupt() selectorThread.join() selector.close() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index ecc8bf189986d..13a52d836f32f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -142,11 +142,10 @@ private[spark] class TaskSchedulerImpl( if (!isLocal && conf.getBoolean("spark.speculation", false)) { logInfo("Starting speculative execution thread") - import sc.env.actorSystem.dispatcher sc.env.actorSystem.scheduler.schedule(SPECULATION_INTERVAL_MS milliseconds, SPECULATION_INTERVAL_MS milliseconds) { Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() } - } + }(sc.env.actorSystem.dispatcher) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 70a477a6895cc..50ba0b9d5a612 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -20,12 +20,12 @@ package org.apache.spark.scheduler.local import java.nio.ByteBuffer import java.util.concurrent.{Executors, TimeUnit} -import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcEndpointRef, RpcEnv} -import org.apache.spark.util.Utils -import org.apache.spark.{Logging, SparkContext, SparkEnv, TaskState} +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} +import org.apache.spark.rpc.{ThreadSafeRpcEndpoint, RpcCallContext, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{SchedulerBackend, TaskSchedulerImpl, WorkerOffer} +import org.apache.spark.util.Utils private case class ReviveOffers() @@ -71,11 +71,15 @@ private[spark] class LocalEndpoint( case KillTask(taskId, interruptThread) => executor.killTask(taskId, interruptThread) + } + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { case StopExecutor => executor.stop() + context.reply(true) } + def reviveOffers() { val offers = Seq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores)) val tasks = scheduler.resourceOffers(offers).flatten @@ -104,8 +108,11 @@ private[spark] class LocalEndpoint( * master all run in the same JVM. It sits behind a TaskSchedulerImpl and handles launching tasks * on a single Executor (created by the LocalBackend) running locally. */ -private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: Int) - extends SchedulerBackend with ExecutorBackend { +private[spark] class LocalBackend( + conf: SparkConf, + scheduler: TaskSchedulerImpl, + val totalCores: Int) + extends SchedulerBackend with ExecutorBackend with Logging { private val appId = "local-" + System.currentTimeMillis var localEndpoint: RpcEndpointRef = null @@ -116,7 +123,7 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: } override def stop() { - localEndpoint.send(StopExecutor) + localEndpoint.sendWithReply(StopExecutor) } override def reviveOffers() { diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 6b3049b28cd5e..22acc270b983e 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -56,19 +56,13 @@ class ExecutorAllocationManagerSuite extends FunSuite with LocalSparkContext wit // Min < 0 val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "-1") intercept[SparkException] { contexts += new SparkContext(conf1) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() // Max < 0 val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "-1") intercept[SparkException] { contexts += new SparkContext(conf2) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() // Both min and max, but min > max intercept[SparkException] { createSparkContext(2, 1) } - SparkEnv.get.stop() - SparkContext.clearActiveContext() // Both min and max, and min == max val sc1 = createSparkContext(1, 1) From 3ae37b93a7c299bd8b22a36248035bca5de3422f Mon Sep 17 00:00:00 2001 From: Jin Adachi Date: Thu, 16 Apr 2015 23:41:04 +0800 Subject: [PATCH 119/128] [SPARK-6694][SQL]SparkSQL CLI must be able to specify an option --database on the command line. SparkSQL CLI has an option --database as follows. But, the option --database is ignored. ``` $ spark-sql --help : CLI options: : --database Specify the database to use ``` Author: Jin Adachi Author: adachij Closes #5345 from adachij2002/SPARK-6694 and squashes the following commits: 8659084 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 0301eb9 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 df81086 [Jin Adachi] Modify code style. 846f83e [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 dbe8c63 [Jin Adachi] Change file permission to 644. 7b58f42 [Jin Adachi] Merge branch 'master' of https://github.com/apache/spark into SPARK-6694 c581d06 [Jin Adachi] Add an option --database test db56122 [Jin Adachi] Merge branch 'SPARK-6694' of https://github.com/adachij2002/spark into SPARK-6694 ee09fa5 [adachij] Merge branch 'master' into SPARK-6694 c804c03 [adachij] SparkSQL CLI must be able to specify an option --database on the command line. --- .../hive/thriftserver/SparkSQLCLIDriver.scala | 3 ++ .../sql/hive/thriftserver/CliSuite.scala | 45 +++++++++++++++---- 2 files changed, 39 insertions(+), 9 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 62c061bef690a..85281c6d73a3b 100644 --- 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 @@ -145,6 +145,9 @@ private[hive] object SparkSQLCLIDriver { case e: UnsupportedEncodingException => System.exit(3) } + // use the specified database if specified + cli.processSelectDatabase(sessionState); + // Execute -i init files (always in silent mode) cli.processInitFiles(sessionState) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 6d1d7c3a4e698..b070fa8eaa469 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -25,22 +25,31 @@ import scala.concurrent.{Await, Promise} import scala.sys.process.{Process, ProcessLogger} import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.apache.spark.Logging import org.apache.spark.util.Utils -class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { +class CliSuite extends FunSuite with BeforeAndAfter with Logging { + val warehousePath = Utils.createTempDir() + val metastorePath = Utils.createTempDir() + + before { + warehousePath.delete() + metastorePath.delete() + } + + after { + warehousePath.delete() + metastorePath.delete() + } + def runCliWithin( timeout: FiniteDuration, extraArgs: Seq[String] = Seq.empty)( - queriesAndExpectedAnswers: (String, String)*) { + queriesAndExpectedAnswers: (String, String)*): Unit = { val (queries, expectedAnswers) = queriesAndExpectedAnswers.unzip - val warehousePath = Utils.createTempDir() - warehousePath.delete() - val metastorePath = Utils.createTempDir() - metastorePath.delete() val cliScript = "../../bin/spark-sql".split("/").mkString(File.separator) val command = { @@ -95,8 +104,6 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { """.stripMargin, cause) throw cause } finally { - warehousePath.delete() - metastorePath.delete() process.destroy() } } @@ -124,4 +131,24 @@ class CliSuite extends FunSuite with BeforeAndAfterAll with Logging { test("Single command with -e") { runCliWithin(1.minute, Seq("-e", "SHOW DATABASES;"))("" -> "OK") } + + test("Single command with --database") { + runCliWithin(1.minute)( + "CREATE DATABASE hive_test_db;" + -> "OK", + "USE hive_test_db;" + -> "OK", + "CREATE TABLE hive_test(key INT, val STRING);" + -> "OK", + "SHOW TABLES;" + -> "Time taken: " + ) + + runCliWithin(1.minute, Seq("--database", "hive_test_db", "-e", "SHOW TABLES;"))( + "" + -> "OK", + "" + -> "hive_test" + ) + } } From ef3fb801ae971656ed9cd1b0ab95bc5a1548adbd Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 16 Apr 2015 13:45:55 -0500 Subject: [PATCH 120/128] [SPARK-6934][Core] Use 'spark.akka.askTimeout' for the ask timeout Fixed my mistake in #4588 Author: zsxwing Closes #5529 from zsxwing/SPARK-6934 and squashes the following commits: 9890b2d [zsxwing] Use 'spark.akka.askTimeout' for the ask timeout --- core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala index e259867c14040..f2c1c86af767e 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEnv.scala @@ -284,7 +284,7 @@ private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) private[this] val maxRetries = conf.getInt("spark.akka.num.retries", 3) private[this] val retryWaitMs = conf.getLong("spark.akka.retry.wait", 3000) - private[this] val defaultTimeout = conf.getLong("spark.akka.lookupTimeout", 30) seconds + private[this] val defaultAskTimeout = conf.getLong("spark.akka.askTimeout", 30) seconds /** * return the address for the [[RpcEndpointRef]] @@ -304,7 +304,8 @@ private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) * * This method only sends the message once and never retries. */ - def sendWithReply[T: ClassTag](message: Any): Future[T] = sendWithReply(message, defaultTimeout) + def sendWithReply[T: ClassTag](message: Any): Future[T] = + sendWithReply(message, defaultAskTimeout) /** * Send a message to the corresponding [[RpcEndpoint.receiveAndReply)]] and return a `Future` to @@ -327,7 +328,7 @@ private[spark] abstract class RpcEndpointRef(@transient conf: SparkConf) * @tparam T type of the reply message * @return the reply message from the corresponding [[RpcEndpoint]] */ - def askWithReply[T: ClassTag](message: Any): T = askWithReply(message, defaultTimeout) + def askWithReply[T: ClassTag](message: Any): T = askWithReply(message, defaultAskTimeout) /** * Send a message to the corresponding [[RpcEndpoint.receive]] and get its result within a From 55f553a979db925aa0c3559f7e80b99d2bf3feb4 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Thu, 16 Apr 2015 13:06:34 -0700 Subject: [PATCH 121/128] [SPARK-6855] [SPARKR] Set R includes to get the right collate order. This prevents tools like devtools::document creating invalid collate orders Author: Shivaram Venkataraman Closes #5462 from shivaram/collate-order and squashes the following commits: f3db562 [Shivaram Venkataraman] Set R includes to get the right collate order. This prevents tools like devtools::document creating invalid collate orders --- R/pkg/DESCRIPTION | 6 +++--- R/pkg/R/DataFrame.R | 2 +- R/pkg/R/column.R | 2 +- R/pkg/R/group.R | 3 +++ R/pkg/R/jobj.R | 3 +++ R/pkg/R/pairRDD.R | 2 ++ 6 files changed, 13 insertions(+), 5 deletions(-) diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index 1842b97d43651..052f68c6c24e2 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -17,19 +17,19 @@ License: Apache License (== 2.0) Collate: 'generics.R' 'jobj.R' - 'SQLTypes.R' 'RDD.R' 'pairRDD.R' + 'SQLTypes.R' 'column.R' 'group.R' 'DataFrame.R' 'SQLContext.R' + 'backend.R' 'broadcast.R' + 'client.R' 'context.R' 'deserialize.R' 'serialize.R' 'sparkR.R' - 'backend.R' - 'client.R' 'utils.R' 'zzz.R' diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index feafd56909a67..044fdb4d01223 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -17,7 +17,7 @@ # DataFrame.R - DataFrame class and methods implemented in S4 OO classes -#' @include jobj.R SQLTypes.R RDD.R pairRDD.R column.R group.R +#' @include generics.R jobj.R SQLTypes.R RDD.R pairRDD.R column.R group.R NULL setOldClass("jobj") diff --git a/R/pkg/R/column.R b/R/pkg/R/column.R index e196305186b9a..b282001d8b6b5 100644 --- a/R/pkg/R/column.R +++ b/R/pkg/R/column.R @@ -17,7 +17,7 @@ # Column Class -#' @include generics.R jobj.R +#' @include generics.R jobj.R SQLTypes.R NULL setOldClass("jobj") diff --git a/R/pkg/R/group.R b/R/pkg/R/group.R index 09fc0a7abe48a..855fbdfc7c4ca 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -17,6 +17,9 @@ # group.R - GroupedData class and methods implemented in S4 OO classes +#' @include generics.R jobj.R SQLTypes.R column.R +NULL + setOldClass("jobj") #' @title S4 class that represents a GroupedData diff --git a/R/pkg/R/jobj.R b/R/pkg/R/jobj.R index 4180f146b7fbc..a8a25230b636d 100644 --- a/R/pkg/R/jobj.R +++ b/R/pkg/R/jobj.R @@ -18,6 +18,9 @@ # References to objects that exist on the JVM backend # are maintained using the jobj. +#' @include generics.R +NULL + # Maintain a reference count of Java object references # This allows us to GC the java object when it is safe .validJobjs <- new.env(parent = emptyenv()) diff --git a/R/pkg/R/pairRDD.R b/R/pkg/R/pairRDD.R index 739d399f0820f..5d64822859d1f 100644 --- a/R/pkg/R/pairRDD.R +++ b/R/pkg/R/pairRDD.R @@ -16,6 +16,8 @@ # # Operations supported on RDDs contains pairs (i.e key, value) +#' @include generics.R jobj.R RDD.R +NULL ############ Actions and Transformations ############ From 04e44b37cc04f62fbf9e08c7076349e0a4d12ea8 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 16 Apr 2015 16:20:57 -0700 Subject: [PATCH 122/128] [SPARK-4897] [PySpark] Python 3 support This PR update PySpark to support Python 3 (tested with 3.4). Known issue: unpickle array from Pyrolite is broken in Python 3, those tests are skipped. TODO: ec2/spark-ec2.py is not fully tested with python3. Author: Davies Liu Author: twneale Author: Josh Rosen Closes #5173 from davies/python3 and squashes the following commits: d7d6323 [Davies Liu] fix tests 6c52a98 [Davies Liu] fix mllib test 99e334f [Davies Liu] update timeout b716610 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 cafd5ec [Davies Liu] adddress comments from @mengxr bf225d7 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 179fc8d [Davies Liu] tuning flaky tests 8c8b957 [Davies Liu] fix ResourceWarning in Python 3 5c57c95 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 4006829 [Davies Liu] fix test 2fc0066 [Davies Liu] add python3 path 71535e9 [Davies Liu] fix xrange and divide 5a55ab4 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 125f12c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ed498c8 [Davies Liu] fix compatibility with python 3 820e649 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 e8ce8c9 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ad7c374 [Davies Liu] fix mllib test and warning ef1fc2f [Davies Liu] fix tests 4eee14a [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 20112ff [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 59bb492 [Davies Liu] fix tests 1da268c [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 ca0fdd3 [Davies Liu] fix code style 9563a15 [Davies Liu] add imap back for python 2 0b1ec04 [Davies Liu] make python examples work with Python 3 d2fd566 [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 a716d34 [Davies Liu] test with python 3.4 f1700e8 [Davies Liu] fix test in python3 671b1db [Davies Liu] fix test in python3 692ff47 [Davies Liu] fix flaky test 7b9699f [Davies Liu] invalidate import cache for Python 3.3+ 9c58497 [Davies Liu] fix kill worker 309bfbf [Davies Liu] keep compatibility 5707476 [Davies Liu] cleanup, fix hash of string in 3.3+ 8662d5b [Davies Liu] Merge branch 'master' of github.com:apache/spark into python3 f53e1f0 [Davies Liu] fix tests 70b6b73 [Davies Liu] compile ec2/spark_ec2.py in python 3 a39167e [Davies Liu] support customize class in __main__ 814c77b [Davies Liu] run unittests with python 3 7f4476e [Davies Liu] mllib tests passed d737924 [Davies Liu] pass ml tests 375ea17 [Davies Liu] SQL tests pass 6cc42a9 [Davies Liu] rename 431a8de [Davies Liu] streaming tests pass 78901a7 [Davies Liu] fix hash of serializer in Python 3 24b2f2e [Davies Liu] pass all RDD tests 35f48fe [Davies Liu] run future again 1eebac2 [Davies Liu] fix conflict in ec2/spark_ec2.py 6e3c21d [Davies Liu] make cloudpickle work with Python3 2fb2db3 [Josh Rosen] Guard more changes behind sys.version; still doesn't run 1aa5e8f [twneale] Turned out `pickle.DictionaryType is dict` == True, so swapped it out 7354371 [twneale] buffer --> memoryview I'm not super sure if this a valid change, but the 2.7 docs recommend using memoryview over buffer where possible, so hoping it'll work. b69ccdf [twneale] Uses the pure python pickle._Pickler instead of c-extension _pickle.Pickler. It appears pyspark 2.7 uses the pure python pickler as well, so this shouldn't degrade pickling performance (?). f40d925 [twneale] xrange --> range e104215 [twneale] Replaces 2.7 types.InstsanceType with 3.4 `object`....could be horribly wrong depending on how types.InstanceType is used elsewhere in the package--see http://bugs.python.org/issue8206 79de9d0 [twneale] Replaces python2.7 `file` with 3.4 _io.TextIOWrapper 2adb42d [Josh Rosen] Fix up some import differences between Python 2 and 3 854be27 [Josh Rosen] Run `futurize` on Python code: 7c5b4ce [Josh Rosen] Remove Python 3 check in shell.py. --- bin/pyspark | 1 + bin/spark-submit | 3 + bin/spark-submit2.cmd | 3 + dev/run-tests | 2 + dev/run-tests-jenkins | 2 +- ec2/spark_ec2.py | 262 ++++---- examples/src/main/python/als.py | 15 +- examples/src/main/python/avro_inputformat.py | 9 +- .../src/main/python/cassandra_inputformat.py | 8 +- .../src/main/python/cassandra_outputformat.py | 6 +- examples/src/main/python/hbase_inputformat.py | 8 +- .../src/main/python/hbase_outputformat.py | 6 +- examples/src/main/python/kmeans.py | 11 +- .../src/main/python/logistic_regression.py | 20 +- .../ml/simple_text_classification_pipeline.py | 20 +- .../src/main/python/mllib/correlations.py | 19 +- .../src/main/python/mllib/dataset_example.py | 13 +- .../main/python/mllib/decision_tree_runner.py | 29 +- .../python/mllib/gaussian_mixture_model.py | 9 +- .../python/mllib/gradient_boosted_trees.py | 7 +- examples/src/main/python/mllib/kmeans.py | 5 +- .../main/python/mllib/logistic_regression.py | 9 +- .../python/mllib/random_forest_example.py | 9 +- .../python/mllib/random_rdd_generation.py | 21 +- .../src/main/python/mllib/sampled_rdds.py | 29 +- examples/src/main/python/mllib/word2vec.py | 5 +- examples/src/main/python/pagerank.py | 16 +- .../src/main/python/parquet_inputformat.py | 7 +- examples/src/main/python/pi.py | 5 +- examples/src/main/python/sort.py | 6 +- examples/src/main/python/sql.py | 4 +- examples/src/main/python/status_api_demo.py | 10 +- .../main/python/streaming/hdfs_wordcount.py | 3 +- .../main/python/streaming/kafka_wordcount.py | 3 +- .../python/streaming/network_wordcount.py | 3 +- .../recoverable_network_wordcount.py | 11 +- .../python/streaming/sql_network_wordcount.py | 5 +- .../streaming/stateful_network_wordcount.py | 3 +- .../src/main/python/transitive_closure.py | 10 +- examples/src/main/python/wordcount.py | 6 +- .../MatrixFactorizationModelWrapper.scala | 9 +- .../mllib/api/python/PythonMLLibAPI.scala | 39 +- python/pyspark/accumulators.py | 9 +- python/pyspark/broadcast.py | 37 +- python/pyspark/cloudpickle.py | 577 +++++------------- python/pyspark/conf.py | 9 +- python/pyspark/context.py | 42 +- python/pyspark/daemon.py | 36 +- python/pyspark/heapq3.py | 24 +- python/pyspark/java_gateway.py | 2 +- python/pyspark/join.py | 1 + python/pyspark/ml/classification.py | 4 +- python/pyspark/ml/feature.py | 22 +- python/pyspark/ml/param/__init__.py | 8 +- .../ml/param/_shared_params_code_gen.py | 10 +- python/pyspark/mllib/__init__.py | 11 +- python/pyspark/mllib/classification.py | 7 +- python/pyspark/mllib/clustering.py | 18 +- python/pyspark/mllib/common.py | 19 +- python/pyspark/mllib/feature.py | 18 +- python/pyspark/mllib/fpm.py | 2 + python/pyspark/mllib/linalg.py | 48 +- python/pyspark/mllib/rand.py | 33 +- python/pyspark/mllib/recommendation.py | 7 +- python/pyspark/mllib/stat/_statistics.py | 25 +- python/pyspark/mllib/tests.py | 20 +- python/pyspark/mllib/tree.py | 15 +- python/pyspark/mllib/util.py | 26 +- python/pyspark/profiler.py | 10 +- python/pyspark/rdd.py | 189 +++--- python/pyspark/rddsampler.py | 4 +- python/pyspark/serializers.py | 101 ++- python/pyspark/shell.py | 16 +- python/pyspark/shuffle.py | 126 ++-- python/pyspark/sql/__init__.py | 15 +- python/pyspark/sql/{types.py => _types.py} | 49 +- python/pyspark/sql/context.py | 32 +- python/pyspark/sql/dataframe.py | 63 +- python/pyspark/sql/functions.py | 6 +- python/pyspark/sql/tests.py | 11 +- python/pyspark/statcounter.py | 4 +- python/pyspark/streaming/context.py | 5 +- python/pyspark/streaming/dstream.py | 51 +- python/pyspark/streaming/kafka.py | 8 +- python/pyspark/streaming/tests.py | 39 +- python/pyspark/streaming/util.py | 6 +- python/pyspark/tests.py | 327 +++++----- python/pyspark/worker.py | 16 +- python/run-tests | 15 +- python/test_support/userlib-0.1-py2.7.egg | Bin 1945 -> 0 bytes python/test_support/userlib-0.1.zip | Bin 0 -> 668 bytes 91 files changed, 1398 insertions(+), 1396 deletions(-) rename python/pyspark/sql/{types.py => _types.py} (97%) delete mode 100644 python/test_support/userlib-0.1-py2.7.egg create mode 100644 python/test_support/userlib-0.1.zip diff --git a/bin/pyspark b/bin/pyspark index 776b28dc41099..8acad6113797d 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -89,6 +89,7 @@ export PYTHONSTARTUP="$SPARK_HOME/python/pyspark/shell.py" if [[ -n "$SPARK_TESTING" ]]; then unset YARN_CONF_DIR unset HADOOP_CONF_DIR + export PYTHONHASHSEED=0 if [[ -n "$PYSPARK_DOC_TEST" ]]; then exec "$PYSPARK_DRIVER_PYTHON" -m doctest $1 else diff --git a/bin/spark-submit b/bin/spark-submit index bcff78edd51ca..0e0afe71a0f05 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -19,6 +19,9 @@ SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" +# disable randomized hash for string in Python 3.3+ +export PYTHONHASHSEED=0 + # Only define a usage function if an upstream script hasn't done so. if ! type -t usage >/dev/null 2>&1; then usage() { diff --git a/bin/spark-submit2.cmd b/bin/spark-submit2.cmd index 08ddb185742d2..d3fc4a5cc3f6e 100644 --- a/bin/spark-submit2.cmd +++ b/bin/spark-submit2.cmd @@ -20,6 +20,9 @@ rem rem This is the entry point for running Spark submit. To avoid polluting the rem environment, it just launches a new cmd to do the real work. +rem disable randomized hash for string in Python 3.3+ +set PYTHONHASHSEED=0 + set CLASS=org.apache.spark.deploy.SparkSubmit call %~dp0spark-class2.cmd %CLASS% %* set SPARK_ERROR_LEVEL=%ERRORLEVEL% diff --git a/dev/run-tests b/dev/run-tests index bb21ab6c9aa04..861d1671182c2 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -235,6 +235,8 @@ echo "=========================================================================" CURRENT_BLOCK=$BLOCK_PYSPARK_UNIT_TESTS +# add path for python 3 in jenkins +export PATH="${PATH}:/home/anaonda/envs/py3k/bin" ./python/run-tests echo "" diff --git a/dev/run-tests-jenkins b/dev/run-tests-jenkins index 3c1c91a111357..030f2cdddb350 100755 --- a/dev/run-tests-jenkins +++ b/dev/run-tests-jenkins @@ -47,7 +47,7 @@ COMMIT_URL="https://github.com/apache/spark/commit/${ghprbActualCommit}" # GitHub doesn't auto-link short hashes when submitted via the API, unfortunately. :( SHORT_COMMIT_HASH="${ghprbActualCommit:0:7}" -TESTS_TIMEOUT="120m" # format: http://linux.die.net/man/1/timeout +TESTS_TIMEOUT="150m" # format: http://linux.die.net/man/1/timeout # Array to capture all tests to run on the pull request. These tests are held under the #+ dev/tests/ directory. diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 0c1f24761d0de..87c0818279713 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -19,7 +19,7 @@ # limitations under the License. # -from __future__ import with_statement +from __future__ import with_statement, print_function import hashlib import itertools @@ -37,12 +37,17 @@ import tempfile import textwrap import time -import urllib2 import warnings from datetime import datetime from optparse import OptionParser from sys import stderr +if sys.version < "3": + from urllib2 import urlopen, Request, HTTPError +else: + from urllib.request import urlopen, Request + from urllib.error import HTTPError + SPARK_EC2_VERSION = "1.2.1" SPARK_EC2_DIR = os.path.dirname(os.path.realpath(__file__)) @@ -88,10 +93,10 @@ def setup_external_libs(libs): SPARK_EC2_LIB_DIR = os.path.join(SPARK_EC2_DIR, "lib") if not os.path.exists(SPARK_EC2_LIB_DIR): - print "Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format( + print("Downloading external libraries that spark-ec2 needs from PyPI to {path}...".format( path=SPARK_EC2_LIB_DIR - ) - print "This should be a one-time operation." + )) + print("This should be a one-time operation.") os.mkdir(SPARK_EC2_LIB_DIR) for lib in libs: @@ -100,8 +105,8 @@ def setup_external_libs(libs): if not os.path.isdir(lib_dir): tgz_file_path = os.path.join(SPARK_EC2_LIB_DIR, versioned_lib_name + ".tar.gz") - print " - Downloading {lib}...".format(lib=lib["name"]) - download_stream = urllib2.urlopen( + print(" - Downloading {lib}...".format(lib=lib["name"])) + download_stream = urlopen( "{prefix}/{first_letter}/{lib_name}/{lib_name}-{lib_version}.tar.gz".format( prefix=PYPI_URL_PREFIX, first_letter=lib["name"][:1], @@ -113,13 +118,13 @@ def setup_external_libs(libs): tgz_file.write(download_stream.read()) with open(tgz_file_path) as tar: if hashlib.md5(tar.read()).hexdigest() != lib["md5"]: - print >> stderr, "ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]) + print("ERROR: Got wrong md5sum for {lib}.".format(lib=lib["name"]), file=stderr) sys.exit(1) tar = tarfile.open(tgz_file_path) tar.extractall(path=SPARK_EC2_LIB_DIR) tar.close() os.remove(tgz_file_path) - print " - Finished downloading {lib}.".format(lib=lib["name"]) + print(" - Finished downloading {lib}.".format(lib=lib["name"])) sys.path.insert(1, lib_dir) @@ -299,12 +304,12 @@ def parse_args(): if home_dir is None or not os.path.isfile(home_dir + '/.boto'): if not os.path.isfile('/etc/boto.cfg'): if os.getenv('AWS_ACCESS_KEY_ID') is None: - print >> stderr, ("ERROR: The environment variable AWS_ACCESS_KEY_ID " + - "must be set") + print("ERROR: The environment variable AWS_ACCESS_KEY_ID must be set", + file=stderr) sys.exit(1) if os.getenv('AWS_SECRET_ACCESS_KEY') is None: - print >> stderr, ("ERROR: The environment variable AWS_SECRET_ACCESS_KEY " + - "must be set") + print("ERROR: The environment variable AWS_SECRET_ACCESS_KEY must be set", + file=stderr) sys.exit(1) return (opts, action, cluster_name) @@ -316,7 +321,7 @@ def get_or_make_group(conn, name, vpc_id): if len(group) > 0: return group[0] else: - print "Creating security group " + name + print("Creating security group " + name) return conn.create_security_group(name, "Spark EC2 group", vpc_id) @@ -324,18 +329,19 @@ def get_validate_spark_version(version, repo): if "." in version: version = version.replace("v", "") if version not in VALID_SPARK_VERSIONS: - print >> stderr, "Don't know about Spark version: {v}".format(v=version) + print("Don't know about Spark version: {v}".format(v=version), file=stderr) sys.exit(1) return version else: github_commit_url = "{repo}/commit/{commit_hash}".format(repo=repo, commit_hash=version) - request = urllib2.Request(github_commit_url) + request = Request(github_commit_url) request.get_method = lambda: 'HEAD' try: - response = urllib2.urlopen(request) - except urllib2.HTTPError, e: - print >> stderr, "Couldn't validate Spark commit: {url}".format(url=github_commit_url) - print >> stderr, "Received HTTP response code of {code}.".format(code=e.code) + response = urlopen(request) + except HTTPError as e: + print("Couldn't validate Spark commit: {url}".format(url=github_commit_url), + file=stderr) + print("Received HTTP response code of {code}.".format(code=e.code), file=stderr) sys.exit(1) return version @@ -394,8 +400,7 @@ def get_spark_ami(opts): instance_type = EC2_INSTANCE_TYPES[opts.instance_type] else: instance_type = "pvm" - print >> stderr,\ - "Don't recognize %s, assuming type is pvm" % opts.instance_type + print("Don't recognize %s, assuming type is pvm" % opts.instance_type, file=stderr) # URL prefix from which to fetch AMI information ami_prefix = "{r}/{b}/ami-list".format( @@ -404,10 +409,10 @@ def get_spark_ami(opts): ami_path = "%s/%s/%s" % (ami_prefix, opts.region, instance_type) try: - ami = urllib2.urlopen(ami_path).read().strip() - print "Spark AMI: " + ami + ami = urlopen(ami_path).read().strip() + print("Spark AMI: " + ami) except: - print >> stderr, "Could not resolve AMI at: " + ami_path + print("Could not resolve AMI at: " + ami_path, file=stderr) sys.exit(1) return ami @@ -419,11 +424,11 @@ def get_spark_ami(opts): # Fails if there already instances running in the cluster's groups. def launch_cluster(conn, opts, cluster_name): if opts.identity_file is None: - print >> stderr, "ERROR: Must provide an identity file (-i) for ssh connections." + print("ERROR: Must provide an identity file (-i) for ssh connections.", file=stderr) sys.exit(1) if opts.key_pair is None: - print >> stderr, "ERROR: Must provide a key pair name (-k) to use on instances." + print("ERROR: Must provide a key pair name (-k) to use on instances.", file=stderr) sys.exit(1) user_data_content = None @@ -431,7 +436,7 @@ def launch_cluster(conn, opts, cluster_name): with open(opts.user_data) as user_data_file: user_data_content = user_data_file.read() - print "Setting up security groups..." + print("Setting up security groups...") master_group = get_or_make_group(conn, cluster_name + "-master", opts.vpc_id) slave_group = get_or_make_group(conn, cluster_name + "-slaves", opts.vpc_id) authorized_address = opts.authorized_address @@ -497,8 +502,8 @@ def launch_cluster(conn, opts, cluster_name): 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 running in " + - "group %s or %s" % (master_group.name, slave_group.name)) + print("ERROR: There are already instances running in group %s or %s" % + (master_group.name, slave_group.name), file=stderr) sys.exit(1) # Figure out Spark AMI @@ -511,12 +516,12 @@ def launch_cluster(conn, opts, cluster_name): additional_group_ids = [sg.id for sg in conn.get_all_security_groups() if opts.additional_security_group in (sg.name, sg.id)] - print "Launching instances..." + print("Launching instances...") try: image = conn.get_all_images(image_ids=[opts.ami])[0] except: - print >> stderr, "Could not find AMI " + opts.ami + print("Could not find AMI " + opts.ami, file=stderr) sys.exit(1) # Create block device mapping so that we can add EBS volumes if asked to. @@ -542,8 +547,8 @@ def launch_cluster(conn, opts, cluster_name): # Launch slaves if opts.spot_price is not None: # Launch spot instances with the requested price - print ("Requesting %d slaves as spot instances with price $%.3f" % - (opts.slaves, opts.spot_price)) + print("Requesting %d slaves as spot instances with price $%.3f" % + (opts.slaves, opts.spot_price)) zones = get_zones(conn, opts) num_zones = len(zones) i = 0 @@ -566,7 +571,7 @@ def launch_cluster(conn, opts, cluster_name): my_req_ids += [req.id for req in slave_reqs] i += 1 - print "Waiting for spot instances to be granted..." + print("Waiting for spot instances to be granted...") try: while True: time.sleep(10) @@ -579,24 +584,24 @@ def launch_cluster(conn, opts, cluster_name): 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 + print("All %d slaves granted" % opts.slaves) reservations = conn.get_all_reservations(active_instance_ids) slave_nodes = [] for r in reservations: slave_nodes += r.instances break else: - print "%d of %d slaves granted, waiting longer" % ( - len(active_instance_ids), opts.slaves) + print("%d of %d slaves granted, waiting longer" % ( + len(active_instance_ids), opts.slaves)) except: - print "Canceling spot instance requests" + print("Canceling spot instance requests") conn.cancel_spot_instance_requests(my_req_ids) # Log a warning if any of these requests actually launched instances: (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) running = len(master_nodes) + len(slave_nodes) if running: - print >> stderr, ("WARNING: %d instances are still running" % running) + print(("WARNING: %d instances are still running" % running), file=stderr) sys.exit(0) else: # Launch non-spot instances @@ -618,16 +623,16 @@ def launch_cluster(conn, opts, cluster_name): placement_group=opts.placement_group, user_data=user_data_content) slave_nodes += slave_res.instances - print "Launched {s} slave{plural_s} in {z}, regid = {r}".format( - s=num_slaves_this_zone, - plural_s=('' if num_slaves_this_zone == 1 else 's'), - z=zone, - r=slave_res.id) + print("Launched {s} slave{plural_s} in {z}, regid = {r}".format( + s=num_slaves_this_zone, + plural_s=('' if num_slaves_this_zone == 1 else 's'), + z=zone, + r=slave_res.id)) i += 1 # Launch or resume masters if existing_masters: - print "Starting master..." + print("Starting master...") for inst in existing_masters: if inst.state not in ["shutting-down", "terminated"]: inst.start() @@ -650,10 +655,10 @@ def launch_cluster(conn, opts, cluster_name): user_data=user_data_content) master_nodes = master_res.instances - print "Launched master in %s, regid = %s" % (zone, master_res.id) + print("Launched master in %s, regid = %s" % (zone, master_res.id)) # This wait time corresponds to SPARK-4983 - print "Waiting for AWS to propagate instance metadata..." + print("Waiting for AWS to propagate instance metadata...") time.sleep(5) # Give the instances descriptive names for master in master_nodes: @@ -674,8 +679,8 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): Get the EC2 instances in an existing cluster if available. Returns a tuple of lists of EC2 instance objects for the masters and slaves. """ - print "Searching for existing cluster {c} in region {r}...".format( - c=cluster_name, r=opts.region) + print("Searching for existing cluster {c} in region {r}...".format( + c=cluster_name, r=opts.region)) def get_instances(group_names): """ @@ -693,16 +698,15 @@ def get_instances(group_names): slave_instances = get_instances([cluster_name + "-slaves"]) if any((master_instances, slave_instances)): - print "Found {m} master{plural_m}, {s} slave{plural_s}.".format( - m=len(master_instances), - plural_m=('' if len(master_instances) == 1 else 's'), - s=len(slave_instances), - plural_s=('' if len(slave_instances) == 1 else 's')) + print("Found {m} master{plural_m}, {s} slave{plural_s}.".format( + m=len(master_instances), + plural_m=('' if len(master_instances) == 1 else 's'), + s=len(slave_instances), + plural_s=('' if len(slave_instances) == 1 else 's'))) if not master_instances and die_on_error: - print >> sys.stderr, \ - "ERROR: Could not find a master for cluster {c} in region {r}.".format( - c=cluster_name, r=opts.region) + print("ERROR: Could not find a master for cluster {c} in region {r}.".format( + c=cluster_name, r=opts.region), file=sys.stderr) sys.exit(1) return (master_instances, slave_instances) @@ -713,7 +717,7 @@ def get_instances(group_names): def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): master = get_dns_name(master_nodes[0], opts.private_ips) if deploy_ssh_key: - print "Generating cluster's SSH key on master..." + print("Generating cluster's SSH key on master...") key_setup = """ [ -f ~/.ssh/id_rsa ] || (ssh-keygen -q -t rsa -N '' -f ~/.ssh/id_rsa && @@ -721,10 +725,10 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): """ ssh(master, opts, key_setup) dot_ssh_tar = ssh_read(master, opts, ['tar', 'c', '.ssh']) - print "Transferring cluster's SSH key to slaves..." + print("Transferring cluster's SSH key to slaves...") for slave in slave_nodes: slave_address = get_dns_name(slave, opts.private_ips) - print slave_address + print(slave_address) ssh_write(slave_address, opts, ['tar', 'x'], dot_ssh_tar) modules = ['spark', 'ephemeral-hdfs', 'persistent-hdfs', @@ -738,8 +742,8 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): # NOTE: We should clone the repository before running deploy_files to # prevent ec2-variables.sh from being overwritten - print "Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format( - r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch) + print("Cloning spark-ec2 scripts from {r}/tree/{b} on master...".format( + r=opts.spark_ec2_git_repo, b=opts.spark_ec2_git_branch)) ssh( host=master, opts=opts, @@ -749,7 +753,7 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): b=opts.spark_ec2_git_branch) ) - print "Deploying files to master..." + print("Deploying files to master...") deploy_files( conn=conn, root_dir=SPARK_EC2_DIR + "/" + "deploy.generic", @@ -760,25 +764,25 @@ def setup_cluster(conn, master_nodes, slave_nodes, opts, deploy_ssh_key): ) if opts.deploy_root_dir is not None: - print "Deploying {s} to master...".format(s=opts.deploy_root_dir) + print("Deploying {s} to master...".format(s=opts.deploy_root_dir)) deploy_user_files( root_dir=opts.deploy_root_dir, opts=opts, master_nodes=master_nodes ) - print "Running setup on master..." + print("Running setup on master...") setup_spark_cluster(master, opts) - print "Done!" + print("Done!") def setup_spark_cluster(master, opts): ssh(master, opts, "chmod u+x spark-ec2/setup.sh") ssh(master, opts, "spark-ec2/setup.sh") - print "Spark standalone cluster started at http://%s:8080" % master + print("Spark standalone cluster started at http://%s:8080" % master) if opts.ganglia: - print "Ganglia started at http://%s:5080/ganglia" % master + print("Ganglia started at http://%s:5080/ganglia" % master) def is_ssh_available(host, opts, print_ssh_output=True): @@ -795,7 +799,7 @@ def is_ssh_available(host, opts, print_ssh_output=True): if s.returncode != 0 and print_ssh_output: # extra leading newline is for spacing in wait_for_cluster_state() - print textwrap.dedent("""\n + print(textwrap.dedent("""\n Warning: SSH connection error. (This could be temporary.) Host: {h} SSH return code: {r} @@ -804,7 +808,7 @@ def is_ssh_available(host, opts, print_ssh_output=True): h=host, r=s.returncode, o=cmd_output.strip() - ) + )) return s.returncode == 0 @@ -865,10 +869,10 @@ def wait_for_cluster_state(conn, opts, cluster_instances, cluster_state): sys.stdout.write("\n") end_time = datetime.now() - print "Cluster is now in '{s}' state. Waited {t} seconds.".format( + print("Cluster is now in '{s}' state. Waited {t} seconds.".format( s=cluster_state, t=(end_time - start_time).seconds - ) + )) # Get number of local disks available for a given EC2 instance type. @@ -916,8 +920,8 @@ def get_num_disks(instance_type): if instance_type in disks_by_instance: return disks_by_instance[instance_type] else: - print >> stderr, ("WARNING: Don't know number of disks on instance type %s; assuming 1" - % instance_type) + print("WARNING: Don't know number of disks on instance type %s; assuming 1" + % instance_type, file=stderr) return 1 @@ -951,7 +955,7 @@ def deploy_files(conn, root_dir, opts, master_nodes, slave_nodes, modules): # Spark-only custom deploy spark_v = "%s|%s" % (opts.spark_git_repo, opts.spark_version) tachyon_v = "" - print "Deploying Spark via git hash; Tachyon won't be set up" + print("Deploying Spark via git hash; Tachyon won't be set up") modules = filter(lambda x: x != "tachyon", modules) master_addresses = [get_dns_name(i, opts.private_ips) for i in master_nodes] @@ -1067,8 +1071,8 @@ def ssh(host, opts, command): "--key-pair parameters and try again.".format(host)) else: raise e - print >> stderr, \ - "Error executing remote command, retrying after 30 seconds: {0}".format(e) + print("Error executing remote command, retrying after 30 seconds: {0}".format(e), + file=stderr) time.sleep(30) tries = tries + 1 @@ -1107,8 +1111,8 @@ def ssh_write(host, opts, command, arguments): elif tries > 5: raise RuntimeError("ssh_write failed with error %s" % proc.returncode) else: - print >> stderr, \ - "Error {0} while executing remote command, retrying after 30 seconds".format(status) + print("Error {0} while executing remote command, retrying after 30 seconds". + format(status), file=stderr) time.sleep(30) tries = tries + 1 @@ -1162,42 +1166,41 @@ def real_main(): if opts.identity_file is not None: if not os.path.exists(opts.identity_file): - print >> stderr,\ - "ERROR: The identity file '{f}' doesn't exist.".format(f=opts.identity_file) + print("ERROR: The identity file '{f}' doesn't exist.".format(f=opts.identity_file), + file=stderr) sys.exit(1) file_mode = os.stat(opts.identity_file).st_mode if not (file_mode & S_IRUSR) or not oct(file_mode)[-2:] == '00': - print >> stderr, "ERROR: The identity file must be accessible only by you." - print >> stderr, 'You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file) + print("ERROR: The identity file must be accessible only by you.", file=stderr) + print('You can fix this with: chmod 400 "{f}"'.format(f=opts.identity_file), + file=stderr) sys.exit(1) if opts.instance_type not in EC2_INSTANCE_TYPES: - print >> stderr, "Warning: Unrecognized EC2 instance type for instance-type: {t}".format( - t=opts.instance_type) + print("Warning: Unrecognized EC2 instance type for instance-type: {t}".format( + t=opts.instance_type), file=stderr) if opts.master_instance_type != "": if opts.master_instance_type not in EC2_INSTANCE_TYPES: - print >> stderr, \ - "Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format( - t=opts.master_instance_type) + print("Warning: Unrecognized EC2 instance type for master-instance-type: {t}".format( + t=opts.master_instance_type), file=stderr) # Since we try instance types even if we can't resolve them, we check if they resolve first # and, if they do, see if they resolve to the same virtualization type. if opts.instance_type in EC2_INSTANCE_TYPES and \ opts.master_instance_type in EC2_INSTANCE_TYPES: if EC2_INSTANCE_TYPES[opts.instance_type] != \ EC2_INSTANCE_TYPES[opts.master_instance_type]: - print >> stderr, \ - "Error: spark-ec2 currently does not support having a master and slaves " + \ - "with different AMI virtualization types." - print >> stderr, "master instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.master_instance_type]) - print >> stderr, "slave instance virtualization type: {t}".format( - t=EC2_INSTANCE_TYPES[opts.instance_type]) + print("Error: spark-ec2 currently does not support having a master and slaves " + "with different AMI virtualization types.", file=stderr) + print("master instance virtualization type: {t}".format( + t=EC2_INSTANCE_TYPES[opts.master_instance_type]), file=stderr) + print("slave instance virtualization type: {t}".format( + t=EC2_INSTANCE_TYPES[opts.instance_type]), file=stderr) sys.exit(1) if opts.ebs_vol_num > 8: - print >> stderr, "ebs-vol-num cannot be greater than 8" + print("ebs-vol-num cannot be greater than 8", file=stderr) sys.exit(1) # Prevent breaking ami_prefix (/, .git and startswith checks) @@ -1206,23 +1209,22 @@ def real_main(): opts.spark_ec2_git_repo.endswith(".git") or \ not opts.spark_ec2_git_repo.startswith("https://github.com") or \ not opts.spark_ec2_git_repo.endswith("spark-ec2"): - print >> stderr, "spark-ec2-git-repo must be a github repo and it must not have a " \ - "trailing / or .git. " \ - "Furthermore, we currently only support forks named spark-ec2." + print("spark-ec2-git-repo must be a github repo and it must not have a trailing / or .git. " + "Furthermore, we currently only support forks named spark-ec2.", file=stderr) sys.exit(1) if not (opts.deploy_root_dir is None or (os.path.isabs(opts.deploy_root_dir) and os.path.isdir(opts.deploy_root_dir) and os.path.exists(opts.deploy_root_dir))): - print >> stderr, "--deploy-root-dir must be an absolute path to a directory that exists " \ - "on the local file system" + print("--deploy-root-dir must be an absolute path to a directory that exists " + "on the local file system", file=stderr) sys.exit(1) try: conn = ec2.connect_to_region(opts.region) except Exception as e: - print >> stderr, (e) + print((e), file=stderr) sys.exit(1) # Select an AZ at random if it was not specified. @@ -1231,7 +1233,7 @@ def real_main(): if action == "launch": if opts.slaves <= 0: - print >> sys.stderr, "ERROR: You have to start at least 1 slave" + print("ERROR: You have to start at least 1 slave", file=sys.stderr) sys.exit(1) if opts.resume: (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) @@ -1250,18 +1252,18 @@ def real_main(): conn, opts, cluster_name, die_on_error=False) if any(master_nodes + slave_nodes): - print "The following instances will be terminated:" + print("The following instances will be terminated:") for inst in master_nodes + slave_nodes: - print "> %s" % get_dns_name(inst, opts.private_ips) - print "ALL DATA ON ALL NODES WILL BE LOST!!" + print("> %s" % get_dns_name(inst, opts.private_ips)) + print("ALL DATA ON ALL NODES WILL BE LOST!!") msg = "Are you sure you want to destroy the cluster {c}? (y/N) ".format(c=cluster_name) response = raw_input(msg) if response == "y": - print "Terminating master..." + print("Terminating master...") for inst in master_nodes: inst.terminate() - print "Terminating slaves..." + print("Terminating slaves...") for inst in slave_nodes: inst.terminate() @@ -1274,16 +1276,16 @@ def real_main(): cluster_instances=(master_nodes + slave_nodes), cluster_state='terminated' ) - print "Deleting security groups (this will take some time)..." + print("Deleting security groups (this will take some time)...") attempt = 1 while attempt <= 3: - print "Attempt %d" % attempt + print("Attempt %d" % attempt) groups = [g for g in conn.get_all_security_groups() if g.name in group_names] success = True # Delete individual rules in all groups before deleting groups to # remove dependencies between them for group in groups: - print "Deleting rules in security group " + group.name + print("Deleting rules in security group " + group.name) for rule in group.rules: for grant in rule.grants: success &= group.revoke(ip_protocol=rule.ip_protocol, @@ -1298,10 +1300,10 @@ def real_main(): try: # It is needed to use group_id to make it work with VPC conn.delete_security_group(group_id=group.id) - print "Deleted security group %s" % group.name + print("Deleted security group %s" % group.name) except boto.exception.EC2ResponseError: success = False - print "Failed to delete security group %s" % group.name + print("Failed to delete security group %s" % group.name) # Unfortunately, group.revoke() returns True even if a rule was not # deleted, so this needs to be rerun if something fails @@ -1311,17 +1313,16 @@ def real_main(): attempt += 1 if not success: - print "Failed to delete all security groups after 3 tries." - print "Try re-running in a few minutes." + print("Failed to delete all security groups after 3 tries.") + print("Try re-running in a few minutes.") elif action == "login": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) if not master_nodes[0].public_dns_name and not opts.private_ips: - print "Master has no public DNS name. Maybe you meant to specify " \ - "--private-ips?" + print("Master has no public DNS name. Maybe you meant to specify --private-ips?") else: master = get_dns_name(master_nodes[0], opts.private_ips) - print "Logging into master " + master + "..." + print("Logging into master " + master + "...") proxy_opt = [] if opts.proxy_port is not None: proxy_opt = ['-D', opts.proxy_port] @@ -1336,19 +1337,18 @@ def real_main(): if response == "y": (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) - print "Rebooting slaves..." + print("Rebooting slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: - print "Rebooting " + inst.id + print("Rebooting " + inst.id) inst.reboot() elif action == "get-master": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) if not master_nodes[0].public_dns_name and not opts.private_ips: - print "Master has no public DNS name. Maybe you meant to specify " \ - "--private-ips?" + print("Master has no public DNS name. Maybe you meant to specify --private-ips?") else: - print get_dns_name(master_nodes[0], opts.private_ips) + print(get_dns_name(master_nodes[0], opts.private_ips)) elif action == "stop": response = raw_input( @@ -1361,11 +1361,11 @@ def real_main(): if response == "y": (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name, die_on_error=False) - print "Stopping master..." + print("Stopping master...") for inst in master_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.stop() - print "Stopping slaves..." + print("Stopping slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: if inst.spot_instance_request_id: @@ -1375,11 +1375,11 @@ def real_main(): elif action == "start": (master_nodes, slave_nodes) = get_existing_cluster(conn, opts, cluster_name) - print "Starting slaves..." + print("Starting slaves...") for inst in slave_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.start() - print "Starting master..." + print("Starting master...") for inst in master_nodes: if inst.state not in ["shutting-down", "terminated"]: inst.start() @@ -1403,15 +1403,15 @@ def real_main(): setup_cluster(conn, master_nodes, slave_nodes, opts, False) else: - print >> stderr, "Invalid action: %s" % action + print("Invalid action: %s" % action, file=stderr) sys.exit(1) def main(): try: real_main() - except UsageError, e: - print >> stderr, "\nError:\n", e + except UsageError as e: + print("\nError:\n", e, file=stderr) sys.exit(1) diff --git a/examples/src/main/python/als.py b/examples/src/main/python/als.py index 70b6146e39a87..1c3a787bd0e94 100755 --- a/examples/src/main/python/als.py +++ b/examples/src/main/python/als.py @@ -21,7 +21,8 @@ This example requires numpy (http://www.numpy.org/) """ -from os.path import realpath +from __future__ import print_function + import sys import numpy as np @@ -57,9 +58,9 @@ def update(i, vec, mat, ratings): Usage: als [M] [U] [F] [iterations] [partitions]" """ - print >> sys.stderr, """WARN: This is a naive implementation of ALS and is given as an + print("""WARN: This is a naive implementation of ALS and is given as an example. Please use the ALS method found in pyspark.mllib.recommendation for more - conventional use.""" + conventional use.""", file=sys.stderr) sc = SparkContext(appName="PythonALS") M = int(sys.argv[1]) if len(sys.argv) > 1 else 100 @@ -68,8 +69,8 @@ def update(i, vec, mat, ratings): ITERATIONS = int(sys.argv[4]) if len(sys.argv) > 4 else 5 partitions = int(sys.argv[5]) if len(sys.argv) > 5 else 2 - print "Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % \ - (M, U, F, ITERATIONS, partitions) + print("Running ALS with M=%d, U=%d, F=%d, iters=%d, partitions=%d\n" % + (M, U, F, ITERATIONS, partitions)) R = matrix(rand(M, F)) * matrix(rand(U, F).T) ms = matrix(rand(M, F)) @@ -95,7 +96,7 @@ def update(i, vec, mat, ratings): usb = sc.broadcast(us) error = rmse(R, ms, us) - print "Iteration %d:" % i - print "\nRMSE: %5.4f\n" % error + print("Iteration %d:" % i) + print("\nRMSE: %5.4f\n" % error) sc.stop() diff --git a/examples/src/main/python/avro_inputformat.py b/examples/src/main/python/avro_inputformat.py index 4626bbb7e3b02..da368ac628a49 100644 --- a/examples/src/main/python/avro_inputformat.py +++ b/examples/src/main/python/avro_inputformat.py @@ -15,9 +15,12 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext +from functools import reduce """ Read data file users.avro in local Spark distro: @@ -49,7 +52,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 2 and len(sys.argv) != 3: - print >> sys.stderr, """ + print(""" Usage: avro_inputformat [reader_schema_file] Run with example jar: @@ -57,7 +60,7 @@ /path/to/examples/avro_inputformat.py [reader_schema_file] Assumes you have Avro data stored in . Reader schema can be optionally specified in [reader_schema_file]. - """ + """, file=sys.stderr) exit(-1) path = sys.argv[1] @@ -77,6 +80,6 @@ conf=conf) output = avro_rdd.map(lambda x: x[0]).collect() for k in output: - print k + print(k) sc.stop() diff --git a/examples/src/main/python/cassandra_inputformat.py b/examples/src/main/python/cassandra_inputformat.py index 05f34b74df45a..93ca0cfcc9302 100644 --- a/examples/src/main/python/cassandra_inputformat.py +++ b/examples/src/main/python/cassandra_inputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -47,14 +49,14 @@ """ if __name__ == "__main__": if len(sys.argv) != 4: - print >> sys.stderr, """ + print(""" Usage: cassandra_inputformat Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \ /path/to/examples/cassandra_inputformat.py Assumes you have some data in Cassandra already, running on , in and - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] @@ -77,6 +79,6 @@ conf=conf) output = cass_rdd.collect() for (k, v) in output: - print (k, v) + print((k, v)) sc.stop() diff --git a/examples/src/main/python/cassandra_outputformat.py b/examples/src/main/python/cassandra_outputformat.py index d144539e58b8f..5d643eac92f94 100644 --- a/examples/src/main/python/cassandra_outputformat.py +++ b/examples/src/main/python/cassandra_outputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -46,7 +48,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 7: - print >> sys.stderr, """ + print(""" Usage: cassandra_outputformat Run with example jar: @@ -60,7 +62,7 @@ ... fname text, ... lname text ... ); - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] diff --git a/examples/src/main/python/hbase_inputformat.py b/examples/src/main/python/hbase_inputformat.py index 3b16010f1cb97..e17819d5feb76 100644 --- a/examples/src/main/python/hbase_inputformat.py +++ b/examples/src/main/python/hbase_inputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -47,14 +49,14 @@ """ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, """ + print(""" Usage: hbase_inputformat Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \ /path/to/examples/hbase_inputformat.py
    Assumes you have some data in HBase already, running on , in
    - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] @@ -74,6 +76,6 @@ conf=conf) output = hbase_rdd.collect() for (k, v) in output: - print (k, v) + print((k, v)) sc.stop() diff --git a/examples/src/main/python/hbase_outputformat.py b/examples/src/main/python/hbase_outputformat.py index abb425b1f886a..9e5641789a976 100644 --- a/examples/src/main/python/hbase_outputformat.py +++ b/examples/src/main/python/hbase_outputformat.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -40,7 +42,7 @@ """ if __name__ == "__main__": if len(sys.argv) != 7: - print >> sys.stderr, """ + print(""" Usage: hbase_outputformat
    Run with example jar: @@ -48,7 +50,7 @@ /path/to/examples/hbase_outputformat.py Assumes you have created
    with column family in HBase running on already - """ + """, file=sys.stderr) exit(-1) host = sys.argv[1] diff --git a/examples/src/main/python/kmeans.py b/examples/src/main/python/kmeans.py index 86ef6f32c84e8..19391506463f0 100755 --- a/examples/src/main/python/kmeans.py +++ b/examples/src/main/python/kmeans.py @@ -22,6 +22,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import sys @@ -47,12 +48,12 @@ def closestPoint(p, centers): if __name__ == "__main__": if len(sys.argv) != 4: - print >> sys.stderr, "Usage: kmeans " + print("Usage: kmeans ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of KMeans Clustering and is given + print("""WARN: This is a naive implementation of KMeans Clustering and is given as an example! Please refer to examples/src/main/python/mllib/kmeans.py for an example on - how to use MLlib's KMeans implementation.""" + how to use MLlib's KMeans implementation.""", file=sys.stderr) sc = SparkContext(appName="PythonKMeans") lines = sc.textFile(sys.argv[1]) @@ -69,13 +70,13 @@ def closestPoint(p, centers): pointStats = closest.reduceByKey( lambda (x1, y1), (x2, y2): (x1 + x2, y1 + y2)) newPoints = pointStats.map( - lambda (x, (y, z)): (x, y / z)).collect() + lambda xy: (xy[0], xy[1][0] / xy[1][1])).collect() tempDist = sum(np.sum((kPoints[x] - y) ** 2) for (x, y) in newPoints) for (x, y) in newPoints: kPoints[x] = y - print "Final centers: " + str(kPoints) + print("Final centers: " + str(kPoints)) sc.stop() diff --git a/examples/src/main/python/logistic_regression.py b/examples/src/main/python/logistic_regression.py index 3aa56b0528168..b318b7d87bfdc 100755 --- a/examples/src/main/python/logistic_regression.py +++ b/examples/src/main/python/logistic_regression.py @@ -22,10 +22,8 @@ In practice, one may prefer to use the LogisticRegression algorithm in MLlib, as shown in examples/src/main/python/mllib/logistic_regression.py. """ +from __future__ import print_function -from collections import namedtuple -from math import exp -from os.path import realpath import sys import numpy as np @@ -42,19 +40,19 @@ def readPointBatch(iterator): strs = list(iterator) matrix = np.zeros((len(strs), D + 1)) - for i in xrange(len(strs)): - matrix[i] = np.fromstring(strs[i].replace(',', ' '), dtype=np.float32, sep=' ') + for i, s in enumerate(strs): + matrix[i] = np.fromstring(s.replace(',', ' '), dtype=np.float32, sep=' ') return [matrix] if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: logistic_regression " + print("Usage: logistic_regression ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of Logistic Regression and is + print("""WARN: This is a naive implementation of Logistic Regression and is given as an example! Please refer to examples/src/main/python/mllib/logistic_regression.py - to see how MLlib's implementation is used.""" + to see how MLlib's implementation is used.""", file=sys.stderr) sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).mapPartitions(readPointBatch).cache() @@ -62,7 +60,7 @@ def readPointBatch(iterator): # Initialize w to a random value w = 2 * np.random.ranf(size=D) - 1 - print "Initial w: " + str(w) + print("Initial w: " + str(w)) # Compute logistic regression gradient for a matrix of data points def gradient(matrix, w): @@ -76,9 +74,9 @@ def add(x, y): return x for i in range(iterations): - print "On iteration %i" % (i + 1) + print("On iteration %i" % (i + 1)) w -= points.map(lambda m: gradient(m, w)).reduce(add) - print "Final w: " + str(w) + print("Final w: " + str(w)) sc.stop() diff --git a/examples/src/main/python/ml/simple_text_classification_pipeline.py b/examples/src/main/python/ml/simple_text_classification_pipeline.py index c73edb7fd6b20..fab21f003b233 100644 --- a/examples/src/main/python/ml/simple_text_classification_pipeline.py +++ b/examples/src/main/python/ml/simple_text_classification_pipeline.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + from pyspark import SparkContext from pyspark.ml import Pipeline from pyspark.ml.classification import LogisticRegression @@ -37,10 +39,10 @@ # Prepare training documents, which are labeled. LabeledDocument = Row("id", "text", "label") - training = sc.parallelize([(0L, "a b c d e spark", 1.0), - (1L, "b d", 0.0), - (2L, "spark f g h", 1.0), - (3L, "hadoop mapreduce", 0.0)]) \ + training = sc.parallelize([(0, "a b c d e spark", 1.0), + (1, "b d", 0.0), + (2, "spark f g h", 1.0), + (3, "hadoop mapreduce", 0.0)]) \ .map(lambda x: LabeledDocument(*x)).toDF() # Configure an ML pipeline, which consists of tree stages: tokenizer, hashingTF, and lr. @@ -54,16 +56,16 @@ # Prepare test documents, which are unlabeled. Document = Row("id", "text") - test = sc.parallelize([(4L, "spark i j k"), - (5L, "l m n"), - (6L, "mapreduce spark"), - (7L, "apache hadoop")]) \ + test = sc.parallelize([(4, "spark i j k"), + (5, "l m n"), + (6, "mapreduce spark"), + (7, "apache hadoop")]) \ .map(lambda x: Document(*x)).toDF() # Make predictions on test documents and print columns of interest. prediction = model.transform(test) selected = prediction.select("id", "text", "prediction") for row in selected.collect(): - print row + print(row) sc.stop() diff --git a/examples/src/main/python/mllib/correlations.py b/examples/src/main/python/mllib/correlations.py index 4218eca822a99..0e13546b88e67 100755 --- a/examples/src/main/python/mllib/correlations.py +++ b/examples/src/main/python/mllib/correlations.py @@ -18,6 +18,7 @@ """ Correlations using MLlib. """ +from __future__ import print_function import sys @@ -29,7 +30,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: correlations ()" + print("Usage: correlations ()", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonCorrelations") if len(sys.argv) == 2: @@ -41,20 +42,20 @@ points = MLUtils.loadLibSVMFile(sc, filepath)\ .map(lambda lp: LabeledPoint(lp.label, lp.features.toArray())) - print - print 'Summary of data file: ' + filepath - print '%d data points' % points.count() + print() + print('Summary of data file: ' + filepath) + print('%d data points' % points.count()) # Statistics (correlations) - print - print 'Correlation (%s) between label and each feature' % corrType - print 'Feature\tCorrelation' + print() + print('Correlation (%s) between label and each feature' % corrType) + print('Feature\tCorrelation') numFeatures = points.take(1)[0].features.size labelRDD = points.map(lambda lp: lp.label) for i in range(numFeatures): featureRDD = points.map(lambda lp: lp.features[i]) corr = Statistics.corr(labelRDD, featureRDD, corrType) - print '%d\t%g' % (i, corr) - print + print('%d\t%g' % (i, corr)) + print() sc.stop() diff --git a/examples/src/main/python/mllib/dataset_example.py b/examples/src/main/python/mllib/dataset_example.py index fcbf56cbf0c52..e23ecc0c5d302 100644 --- a/examples/src/main/python/mllib/dataset_example.py +++ b/examples/src/main/python/mllib/dataset_example.py @@ -19,6 +19,7 @@ An example of how to use DataFrame as a dataset for ML. Run with:: bin/spark-submit examples/src/main/python/mllib/dataset_example.py """ +from __future__ import print_function import os import sys @@ -32,16 +33,16 @@ def summarize(dataset): - print "schema: %s" % dataset.schema().json() + print("schema: %s" % dataset.schema().json()) labels = dataset.map(lambda r: r.label) - print "label average: %f" % labels.mean() + print("label average: %f" % labels.mean()) features = dataset.map(lambda r: r.features) summary = Statistics.colStats(features) - print "features average: %r" % summary.mean() + print("features average: %r" % summary.mean()) if __name__ == "__main__": if len(sys.argv) > 2: - print >> sys.stderr, "Usage: dataset_example.py " + print("Usage: dataset_example.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="DatasetExample") sqlContext = SQLContext(sc) @@ -54,9 +55,9 @@ def summarize(dataset): summarize(dataset0) tempdir = tempfile.NamedTemporaryFile(delete=False).name os.unlink(tempdir) - print "Save dataset as a Parquet file to %s." % tempdir + print("Save dataset as a Parquet file to %s." % tempdir) dataset0.saveAsParquetFile(tempdir) - print "Load it back and summarize it again." + print("Load it back and summarize it again.") dataset1 = sqlContext.parquetFile(tempdir).setName("dataset1").cache() summarize(dataset1) shutil.rmtree(tempdir) diff --git a/examples/src/main/python/mllib/decision_tree_runner.py b/examples/src/main/python/mllib/decision_tree_runner.py index fccabd841b139..513ed8fd51450 100755 --- a/examples/src/main/python/mllib/decision_tree_runner.py +++ b/examples/src/main/python/mllib/decision_tree_runner.py @@ -20,6 +20,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import numpy import os @@ -83,18 +84,17 @@ def reindexClassLabels(data): numClasses = len(classCounts) # origToNewLabels: class --> index in 0,...,numClasses-1 if (numClasses < 2): - print >> sys.stderr, \ - "Dataset for classification should have at least 2 classes." + \ - " The given dataset had only %d classes." % numClasses + print("Dataset for classification should have at least 2 classes." + " The given dataset had only %d classes." % numClasses, file=sys.stderr) exit(1) origToNewLabels = dict([(sortedClasses[i], i) for i in range(0, numClasses)]) - print "numClasses = %d" % numClasses - print "Per-class example fractions, counts:" - print "Class\tFrac\tCount" + print("numClasses = %d" % numClasses) + print("Per-class example fractions, counts:") + print("Class\tFrac\tCount") for c in sortedClasses: frac = classCounts[c] / (numExamples + 0.0) - print "%g\t%g\t%d" % (c, frac, classCounts[c]) + print("%g\t%g\t%d" % (c, frac, classCounts[c])) if (sortedClasses[0] == 0 and sortedClasses[-1] == numClasses - 1): return (data, origToNewLabels) @@ -105,8 +105,7 @@ def reindexClassLabels(data): def usage(): - print >> sys.stderr, \ - "Usage: decision_tree_runner [libsvm format data filepath]" + print("Usage: decision_tree_runner [libsvm format data filepath]", file=sys.stderr) exit(1) @@ -133,13 +132,13 @@ def usage(): model = DecisionTree.trainClassifier(reindexedData, numClasses=numClasses, categoricalFeaturesInfo=categoricalFeaturesInfo) # Print learned tree and stats. - print "Trained DecisionTree for classification:" - print " Model numNodes: %d" % model.numNodes() - print " Model depth: %d" % model.depth() - print " Training accuracy: %g" % getAccuracy(model, reindexedData) + print("Trained DecisionTree for classification:") + 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() + print(model.toDebugString()) else: - print model + print(model) sc.stop() diff --git a/examples/src/main/python/mllib/gaussian_mixture_model.py b/examples/src/main/python/mllib/gaussian_mixture_model.py index a2cd626c9f19d..2cb8010cdc07f 100644 --- a/examples/src/main/python/mllib/gaussian_mixture_model.py +++ b/examples/src/main/python/mllib/gaussian_mixture_model.py @@ -18,7 +18,8 @@ """ A Gaussian Mixture Model clustering program using MLlib. """ -import sys +from __future__ import print_function + import random import argparse import numpy as np @@ -59,7 +60,7 @@ def parseVector(line): model = GaussianMixture.train(data, args.k, args.convergenceTol, args.maxIterations, args.seed) for i in range(args.k): - print ("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu, - "sigma = ", model.gaussians[i].sigma.toArray()) - print ("Cluster labels (first 100): ", model.predict(data).take(100)) + print(("weight = ", model.weights[i], "mu = ", model.gaussians[i].mu, + "sigma = ", model.gaussians[i].sigma.toArray())) + print(("Cluster labels (first 100): ", model.predict(data).take(100))) sc.stop() diff --git a/examples/src/main/python/mllib/gradient_boosted_trees.py b/examples/src/main/python/mllib/gradient_boosted_trees.py index e647773ad9060..781bd61c9d2b5 100644 --- a/examples/src/main/python/mllib/gradient_boosted_trees.py +++ b/examples/src/main/python/mllib/gradient_boosted_trees.py @@ -18,6 +18,7 @@ """ Gradient boosted Trees classification and regression using MLlib. """ +from __future__ import print_function import sys @@ -34,7 +35,7 @@ def testClassification(trainingData, testData): # 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() \ + testErr = labelsAndPredictions.filter(lambda v_p: v_p[0] != v_p[1]).count() \ / float(testData.count()) print('Test Error = ' + str(testErr)) print('Learned classification ensemble model:') @@ -49,7 +50,7 @@ def testRegression(trainingData, testData): # 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() \ + testMSE = labelsAndPredictions.map(lambda vp: (vp[0] - vp[1]) * (vp[0] - vp[1])).sum() \ / float(testData.count()) print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression ensemble model:') @@ -58,7 +59,7 @@ def testRegression(trainingData, testData): if __name__ == "__main__": if len(sys.argv) > 1: - print >> sys.stderr, "Usage: gradient_boosted_trees" + print("Usage: gradient_boosted_trees", file=sys.stderr) exit(1) sc = SparkContext(appName="PythonGradientBoostedTrees") diff --git a/examples/src/main/python/mllib/kmeans.py b/examples/src/main/python/mllib/kmeans.py index 2eeb1abeeb12b..f901a87fa63ac 100755 --- a/examples/src/main/python/mllib/kmeans.py +++ b/examples/src/main/python/mllib/kmeans.py @@ -20,6 +20,7 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function import sys @@ -34,12 +35,12 @@ def parseVector(line): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: kmeans " + print("Usage: kmeans ", file=sys.stderr) exit(-1) sc = SparkContext(appName="KMeans") lines = sc.textFile(sys.argv[1]) data = lines.map(parseVector) k = int(sys.argv[2]) model = KMeans.train(data, k) - print "Final centers: " + str(model.clusterCenters) + print("Final centers: " + str(model.clusterCenters)) sc.stop() diff --git a/examples/src/main/python/mllib/logistic_regression.py b/examples/src/main/python/mllib/logistic_regression.py index 8cae27fc4a52d..d4f1d34e2d8cf 100755 --- a/examples/src/main/python/mllib/logistic_regression.py +++ b/examples/src/main/python/mllib/logistic_regression.py @@ -20,11 +20,10 @@ This example requires NumPy (http://www.numpy.org/). """ +from __future__ import print_function -from math import exp import sys -import numpy as np from pyspark import SparkContext from pyspark.mllib.regression import LabeledPoint from pyspark.mllib.classification import LogisticRegressionWithSGD @@ -42,12 +41,12 @@ def parsePoint(line): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: logistic_regression " + print("Usage: logistic_regression ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonLR") points = sc.textFile(sys.argv[1]).map(parsePoint) iterations = int(sys.argv[2]) model = LogisticRegressionWithSGD.train(points, iterations) - print "Final weights: " + str(model.weights) - print "Final intercept: " + str(model.intercept) + print("Final weights: " + str(model.weights)) + print("Final intercept: " + str(model.intercept)) sc.stop() diff --git a/examples/src/main/python/mllib/random_forest_example.py b/examples/src/main/python/mllib/random_forest_example.py index d3c24f7664329..4cfdad868c66e 100755 --- a/examples/src/main/python/mllib/random_forest_example.py +++ b/examples/src/main/python/mllib/random_forest_example.py @@ -22,6 +22,7 @@ For information on multiclass classification, please refer to the decision_tree_runner.py example. """ +from __future__ import print_function import sys @@ -43,7 +44,7 @@ def testClassification(trainingData, testData): # 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()\ + testErr = labelsAndPredictions.filter(lambda v_p: v_p[0] != v_p[1]).count()\ / float(testData.count()) print('Test Error = ' + str(testErr)) print('Learned classification forest model:') @@ -62,8 +63,8 @@ def testRegression(trainingData, testData): # 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()) + testMSE = labelsAndPredictions.map(lambda v_p1: (v_p1[0] - v_p1[1]) * (v_p1[0] - v_p1[1]))\ + .sum() / float(testData.count()) print('Test Mean Squared Error = ' + str(testMSE)) print('Learned regression forest model:') print(model.toDebugString()) @@ -71,7 +72,7 @@ def testRegression(trainingData, testData): if __name__ == "__main__": if len(sys.argv) > 1: - print >> sys.stderr, "Usage: random_forest_example" + print("Usage: random_forest_example", file=sys.stderr) exit(1) sc = SparkContext(appName="PythonRandomForestExample") diff --git a/examples/src/main/python/mllib/random_rdd_generation.py b/examples/src/main/python/mllib/random_rdd_generation.py index 1e8892741e714..729bae30b152c 100755 --- a/examples/src/main/python/mllib/random_rdd_generation.py +++ b/examples/src/main/python/mllib/random_rdd_generation.py @@ -18,6 +18,7 @@ """ Randomly generated RDDs. """ +from __future__ import print_function import sys @@ -27,7 +28,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: random_rdd_generation" + print("Usage: random_rdd_generation", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonRandomRDDGeneration") @@ -37,19 +38,19 @@ # Example: RandomRDDs.normalRDD normalRDD = RandomRDDs.normalRDD(sc, numExamples) - print 'Generated RDD of %d examples sampled from the standard normal distribution'\ - % normalRDD.count() - print ' First 5 samples:' + print('Generated RDD of %d examples sampled from the standard normal distribution' + % normalRDD.count()) + print(' First 5 samples:') for sample in normalRDD.take(5): - print ' ' + str(sample) - print + print(' ' + str(sample)) + print() # Example: RandomRDDs.normalVectorRDD normalVectorRDD = RandomRDDs.normalVectorRDD(sc, numRows=numExamples, numCols=2) - print 'Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count() - print ' First 5 samples:' + print('Generated RDD of %d examples of length-2 vectors.' % normalVectorRDD.count()) + print(' First 5 samples:') for sample in normalVectorRDD.take(5): - print ' ' + str(sample) - print + print(' ' + str(sample)) + print() sc.stop() diff --git a/examples/src/main/python/mllib/sampled_rdds.py b/examples/src/main/python/mllib/sampled_rdds.py index 92af3af5ebd1e..b7033ab7daeb3 100755 --- a/examples/src/main/python/mllib/sampled_rdds.py +++ b/examples/src/main/python/mllib/sampled_rdds.py @@ -18,6 +18,7 @@ """ Randomly sampled RDDs. """ +from __future__ import print_function import sys @@ -27,7 +28,7 @@ if __name__ == "__main__": if len(sys.argv) not in [1, 2]: - print >> sys.stderr, "Usage: sampled_rdds " + print("Usage: sampled_rdds ", file=sys.stderr) exit(-1) if len(sys.argv) == 2: datapath = sys.argv[1] @@ -41,24 +42,24 @@ examples = MLUtils.loadLibSVMFile(sc, datapath) numExamples = examples.count() if numExamples == 0: - print >> sys.stderr, "Error: Data file had no samples to load." + print("Error: Data file had no samples to load.", file=sys.stderr) exit(1) - print 'Loaded data with %d examples from file: %s' % (numExamples, datapath) + print('Loaded data with %d examples from file: %s' % (numExamples, datapath)) # Example: RDD.sample() and RDD.takeSample() expectedSampleSize = int(numExamples * fraction) - print 'Sampling RDD using fraction %g. Expected sample size = %d.' \ - % (fraction, expectedSampleSize) + print('Sampling RDD using fraction %g. Expected sample size = %d.' + % (fraction, expectedSampleSize)) sampledRDD = examples.sample(withReplacement=True, fraction=fraction) - print ' RDD.sample(): sample has %d examples' % sampledRDD.count() + print(' RDD.sample(): sample has %d examples' % sampledRDD.count()) sampledArray = examples.takeSample(withReplacement=True, num=expectedSampleSize) - print ' RDD.takeSample(): sample has %d examples' % len(sampledArray) + print(' RDD.takeSample(): sample has %d examples' % len(sampledArray)) - print + print() # Example: RDD.sampleByKey() keyedRDD = examples.map(lambda lp: (int(lp.label), lp.features)) - print ' Keyed data using label (Int) as key ==> Orig' + print(' Keyed data using label (Int) as key ==> Orig') # Count examples per label in original data. keyCountsA = keyedRDD.countByKey() @@ -69,18 +70,18 @@ sampledByKeyRDD = keyedRDD.sampleByKey(withReplacement=True, fractions=fractions) keyCountsB = sampledByKeyRDD.countByKey() sizeB = sum(keyCountsB.values()) - print ' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' \ - % sizeB + print(' Sampled %d examples using approximate stratified sampling (by label). ==> Sample' + % sizeB) # Compare samples - print ' \tFractions of examples with key' - print 'Key\tOrig\tSample' + print(' \tFractions of examples with key') + print('Key\tOrig\tSample') for k in sorted(keyCountsA.keys()): fracA = keyCountsA[k] / float(numExamples) if sizeB != 0: fracB = keyCountsB.get(k, 0) / float(sizeB) else: fracB = 0 - print '%d\t%g\t%g' % (k, fracA, fracB) + print('%d\t%g\t%g' % (k, fracA, fracB)) sc.stop() diff --git a/examples/src/main/python/mllib/word2vec.py b/examples/src/main/python/mllib/word2vec.py index 99fef4276a369..40d1b887927e0 100644 --- a/examples/src/main/python/mllib/word2vec.py +++ b/examples/src/main/python/mllib/word2vec.py @@ -23,6 +23,7 @@ # grep -o -E '\w+(\W+\w+){0,15}' text8 > text8_lines # This was done so that the example can be run in local mode +from __future__ import print_function import sys @@ -34,7 +35,7 @@ if __name__ == "__main__": if len(sys.argv) < 2: - print USAGE + print(USAGE) sys.exit("Argument for file not provided") file_path = sys.argv[1] sc = SparkContext(appName='Word2Vec') @@ -46,5 +47,5 @@ synonyms = model.findSynonyms('china', 40) for word, cosine_distance in synonyms: - print "{}: {}".format(word, cosine_distance) + print("{}: {}".format(word, cosine_distance)) sc.stop() diff --git a/examples/src/main/python/pagerank.py b/examples/src/main/python/pagerank.py index a5f25d78c1146..2fdc9773d4eb1 100755 --- a/examples/src/main/python/pagerank.py +++ b/examples/src/main/python/pagerank.py @@ -19,6 +19,7 @@ This is an example implementation of PageRank. For more conventional use, Please refer to PageRank implementation provided by graphx """ +from __future__ import print_function import re import sys @@ -42,11 +43,12 @@ def parseNeighbors(urls): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: pagerank " + print("Usage: pagerank ", file=sys.stderr) exit(-1) - print >> sys.stderr, """WARN: This is a naive implementation of PageRank and is - given as an example! Please refer to PageRank implementation provided by graphx""" + print("""WARN: This is a naive implementation of PageRank and is + given as an example! Please refer to PageRank implementation provided by graphx""", + file=sys.stderr) # Initialize the spark context. sc = SparkContext(appName="PythonPageRank") @@ -62,19 +64,19 @@ def parseNeighbors(urls): links = lines.map(lambda urls: parseNeighbors(urls)).distinct().groupByKey().cache() # Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. - ranks = links.map(lambda (url, neighbors): (url, 1.0)) + ranks = links.map(lambda url_neighbors: (url_neighbors[0], 1.0)) # Calculates and updates URL ranks continuously using PageRank algorithm. - for iteration in xrange(int(sys.argv[2])): + for iteration in range(int(sys.argv[2])): # Calculates URL contributions to the rank of other URLs. contribs = links.join(ranks).flatMap( - lambda (url, (urls, rank)): computeContribs(urls, rank)) + lambda url_urls_rank: computeContribs(url_urls_rank[1][0], url_urls_rank[1][1])) # Re-calculates URL ranks based on neighbor contributions. ranks = contribs.reduceByKey(add).mapValues(lambda rank: rank * 0.85 + 0.15) # Collects all URL ranks and dump them to console. for (link, rank) in ranks.collect(): - print "%s has rank: %s." % (link, rank) + print("%s has rank: %s." % (link, rank)) sc.stop() diff --git a/examples/src/main/python/parquet_inputformat.py b/examples/src/main/python/parquet_inputformat.py index fa4c20ab20281..96ddac761d698 100644 --- a/examples/src/main/python/parquet_inputformat.py +++ b/examples/src/main/python/parquet_inputformat.py @@ -1,3 +1,4 @@ +from __future__ import print_function # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -35,14 +36,14 @@ """ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, """ + print(""" Usage: parquet_inputformat.py Run with example jar: ./bin/spark-submit --driver-class-path /path/to/example/jar \\ /path/to/examples/parquet_inputformat.py Assumes you have Parquet data stored in . - """ + """, file=sys.stderr) exit(-1) path = sys.argv[1] @@ -56,6 +57,6 @@ valueConverter='org.apache.spark.examples.pythonconverters.IndexedRecordToJavaConverter') output = parquet_rdd.map(lambda x: x[1]).collect() for k in output: - print k + print(k) sc.stop() diff --git a/examples/src/main/python/pi.py b/examples/src/main/python/pi.py index a7c74e969cdb9..92e5cf45abc8b 100755 --- a/examples/src/main/python/pi.py +++ b/examples/src/main/python/pi.py @@ -1,3 +1,4 @@ +from __future__ import print_function # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -35,7 +36,7 @@ def f(_): y = random() * 2 - 1 return 1 if x ** 2 + y ** 2 < 1 else 0 - count = sc.parallelize(xrange(1, n + 1), partitions).map(f).reduce(add) - print "Pi is roughly %f" % (4.0 * count / n) + count = sc.parallelize(range(1, n + 1), partitions).map(f).reduce(add) + print("Pi is roughly %f" % (4.0 * count / n)) sc.stop() diff --git a/examples/src/main/python/sort.py b/examples/src/main/python/sort.py index bb686f17518a0..f6b0ecb02c100 100755 --- a/examples/src/main/python/sort.py +++ b/examples/src/main/python/sort.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from pyspark import SparkContext @@ -22,7 +24,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: sort " + print("Usage: sort ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonSort") lines = sc.textFile(sys.argv[1], 1) @@ -33,6 +35,6 @@ # In reality, we wouldn't want to collect all the data to the driver node. output = sortedCount.collect() for (num, unitcount) in output: - print num + print(num) sc.stop() diff --git a/examples/src/main/python/sql.py b/examples/src/main/python/sql.py index d89361f324917..87d7b088f077b 100644 --- a/examples/src/main/python/sql.py +++ b/examples/src/main/python/sql.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import os from pyspark import SparkContext @@ -68,6 +70,6 @@ teenagers = sqlContext.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") for each in teenagers.collect(): - print each[0] + print(each[0]) sc.stop() diff --git a/examples/src/main/python/status_api_demo.py b/examples/src/main/python/status_api_demo.py index a33bdc475a06d..49b7902185aaa 100644 --- a/examples/src/main/python/status_api_demo.py +++ b/examples/src/main/python/status_api_demo.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import time import threading import Queue @@ -52,15 +54,15 @@ def run(): ids = status.getJobIdsForGroup() for id in ids: job = status.getJobInfo(id) - print "Job", id, "status: ", job.status + print("Job", id, "status: ", job.status) for sid in job.stageIds: info = status.getStageInfo(sid) if info: - print "Stage %d: %d tasks total (%d active, %d complete)" % \ - (sid, info.numTasks, info.numActiveTasks, info.numCompletedTasks) + print("Stage %d: %d tasks total (%d active, %d complete)" % + (sid, info.numTasks, info.numActiveTasks, info.numCompletedTasks)) time.sleep(1) - print "Job results are:", result.get() + print("Job results are:", result.get()) sc.stop() if __name__ == "__main__": diff --git a/examples/src/main/python/streaming/hdfs_wordcount.py b/examples/src/main/python/streaming/hdfs_wordcount.py index f7ffb5379681e..f815dd26823d1 100644 --- a/examples/src/main/python/streaming/hdfs_wordcount.py +++ b/examples/src/main/python/streaming/hdfs_wordcount.py @@ -25,6 +25,7 @@ Then create a text file in `localdir` and the words in the file will get counted. """ +from __future__ import print_function import sys @@ -33,7 +34,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: hdfs_wordcount.py " + print("Usage: hdfs_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingHDFSWordCount") diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py index 51e1ff822fc55..b178e7899b5e1 100644 --- a/examples/src/main/python/streaming/kafka_wordcount.py +++ b/examples/src/main/python/streaming/kafka_wordcount.py @@ -27,6 +27,7 @@ spark-streaming-kafka-assembly-*.jar examples/src/main/python/streaming/kafka_wordcount.py \ localhost:2181 test` """ +from __future__ import print_function import sys @@ -36,7 +37,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: kafka_wordcount.py " + print("Usage: kafka_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingKafkaWordCount") diff --git a/examples/src/main/python/streaming/network_wordcount.py b/examples/src/main/python/streaming/network_wordcount.py index cfa9c1ff5bfbc..2b48bcfd55db0 100644 --- a/examples/src/main/python/streaming/network_wordcount.py +++ b/examples/src/main/python/streaming/network_wordcount.py @@ -25,6 +25,7 @@ and then run the example `$ bin/spark-submit examples/src/main/python/streaming/network_wordcount.py localhost 9999` """ +from __future__ import print_function import sys @@ -33,7 +34,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: network_wordcount.py " + print("Usage: network_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingNetworkWordCount") ssc = StreamingContext(sc, 1) diff --git a/examples/src/main/python/streaming/recoverable_network_wordcount.py b/examples/src/main/python/streaming/recoverable_network_wordcount.py index fc6827c82bf9b..ac91f0a06b172 100644 --- a/examples/src/main/python/streaming/recoverable_network_wordcount.py +++ b/examples/src/main/python/streaming/recoverable_network_wordcount.py @@ -35,6 +35,7 @@ checkpoint data exists in ~/checkpoint/, then it will create StreamingContext from the checkpoint data. """ +from __future__ import print_function import os import sys @@ -46,7 +47,7 @@ def createContext(host, port, outputPath): # If you do not see this printed, that means the StreamingContext has been loaded # from the new checkpoint - print "Creating new context" + print("Creating new context") if os.path.exists(outputPath): os.remove(outputPath) sc = SparkContext(appName="PythonStreamingRecoverableNetworkWordCount") @@ -60,8 +61,8 @@ def createContext(host, port, outputPath): def echo(time, rdd): counts = "Counts at time %s %s" % (time, rdd.collect()) - print counts - print "Appending to " + os.path.abspath(outputPath) + print(counts) + print("Appending to " + os.path.abspath(outputPath)) with open(outputPath, 'a') as f: f.write(counts + "\n") @@ -70,8 +71,8 @@ def echo(time, rdd): if __name__ == "__main__": if len(sys.argv) != 5: - print >> sys.stderr, "Usage: recoverable_network_wordcount.py "\ - " " + print("Usage: recoverable_network_wordcount.py " + " ", file=sys.stderr) exit(-1) host, port, checkpoint, output = sys.argv[1:] ssc = StreamingContext.getOrCreate(checkpoint, diff --git a/examples/src/main/python/streaming/sql_network_wordcount.py b/examples/src/main/python/streaming/sql_network_wordcount.py index f89bc562d856b..da90c07dbd82f 100644 --- a/examples/src/main/python/streaming/sql_network_wordcount.py +++ b/examples/src/main/python/streaming/sql_network_wordcount.py @@ -27,6 +27,7 @@ and then run the example `$ bin/spark-submit examples/src/main/python/streaming/sql_network_wordcount.py localhost 9999` """ +from __future__ import print_function import os import sys @@ -44,7 +45,7 @@ def getSqlContextInstance(sparkContext): if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: sql_network_wordcount.py " + print("Usage: sql_network_wordcount.py ", file=sys.stderr) exit(-1) host, port = sys.argv[1:] sc = SparkContext(appName="PythonSqlNetworkWordCount") @@ -57,7 +58,7 @@ def getSqlContextInstance(sparkContext): # Convert RDDs of the words DStream to DataFrame and run SQL query def process(time, rdd): - print "========= %s =========" % str(time) + print("========= %s =========" % str(time)) try: # Get the singleton instance of SQLContext diff --git a/examples/src/main/python/streaming/stateful_network_wordcount.py b/examples/src/main/python/streaming/stateful_network_wordcount.py index 18a9a5a452ffb..16ef646b7c42e 100644 --- a/examples/src/main/python/streaming/stateful_network_wordcount.py +++ b/examples/src/main/python/streaming/stateful_network_wordcount.py @@ -29,6 +29,7 @@ `$ bin/spark-submit examples/src/main/python/streaming/stateful_network_wordcount.py \ localhost 9999` """ +from __future__ import print_function import sys @@ -37,7 +38,7 @@ if __name__ == "__main__": if len(sys.argv) != 3: - print >> sys.stderr, "Usage: stateful_network_wordcount.py " + print("Usage: stateful_network_wordcount.py ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonStreamingStatefulNetworkWordCount") ssc = StreamingContext(sc, 1) diff --git a/examples/src/main/python/transitive_closure.py b/examples/src/main/python/transitive_closure.py index 00a281bfb6506..7bf5fb6ddfe29 100755 --- a/examples/src/main/python/transitive_closure.py +++ b/examples/src/main/python/transitive_closure.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from random import Random @@ -49,20 +51,20 @@ def generateGraph(): # the graph to obtain the path (x, z). # Because join() joins on keys, the edges are stored in reversed order. - edges = tc.map(lambda (x, y): (y, x)) + edges = tc.map(lambda x_y: (x_y[1], x_y[0])) - oldCount = 0L + oldCount = 0 nextCount = tc.count() while True: oldCount = nextCount # Perform the join, obtaining an RDD of (y, (z, x)) pairs, # then project the result to obtain the new (x, z) paths. - new_edges = tc.join(edges).map(lambda (_, (a, b)): (b, a)) + new_edges = tc.join(edges).map(lambda __a_b: (__a_b[1][1], __a_b[1][0])) tc = tc.union(new_edges).distinct().cache() nextCount = tc.count() if nextCount == oldCount: break - print "TC has %i edges" % tc.count() + print("TC has %i edges" % tc.count()) sc.stop() diff --git a/examples/src/main/python/wordcount.py b/examples/src/main/python/wordcount.py index ae6cd13b83d92..7c0143607b61d 100755 --- a/examples/src/main/python/wordcount.py +++ b/examples/src/main/python/wordcount.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import sys from operator import add @@ -23,7 +25,7 @@ if __name__ == "__main__": if len(sys.argv) != 2: - print >> sys.stderr, "Usage: wordcount " + print("Usage: wordcount ", file=sys.stderr) exit(-1) sc = SparkContext(appName="PythonWordCount") lines = sc.textFile(sys.argv[1], 1) @@ -32,6 +34,6 @@ .reduceByKey(add) output = counts.collect() for (word, count) in output: - print "%s: %i" % (word, count) + print("%s: %i" % (word, count)) sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala index ecd3b16598438..534edac56bc5a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/MatrixFactorizationModelWrapper.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.api.python import org.apache.spark.api.java.JavaRDD +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.recommendation.{MatrixFactorizationModel, Rating} import org.apache.spark.rdd.RDD @@ -31,10 +32,14 @@ private[python] class MatrixFactorizationModelWrapper(model: MatrixFactorization predict(SerDe.asTupleRDD(userAndProducts.rdd)) def getUserFeatures: RDD[Array[Any]] = { - SerDe.fromTuple2RDD(userFeatures.asInstanceOf[RDD[(Any, Any)]]) + SerDe.fromTuple2RDD(userFeatures.map { + case (user, feature) => (user, Vectors.dense(feature)) + }.asInstanceOf[RDD[(Any, Any)]]) } def getProductFeatures: RDD[Array[Any]] = { - SerDe.fromTuple2RDD(productFeatures.asInstanceOf[RDD[(Any, Any)]]) + SerDe.fromTuple2RDD(productFeatures.map { + case (product, feature) => (product, Vectors.dense(feature)) + }.asInstanceOf[RDD[(Any, Any)]]) } } 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 ab15f0f36a14b..f976d2f97b043 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 @@ -28,7 +28,6 @@ import scala.reflect.ClassTag import net.razorvine.pickle._ -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} import org.apache.spark.api.python.SerDeUtil import org.apache.spark.mllib.classification._ @@ -40,15 +39,15 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.random.{RandomRDDs => RG} import org.apache.spark.mllib.recommendation._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} import org.apache.spark.mllib.stat.correlation.CorrelationNames import org.apache.spark.mllib.stat.distribution.MultivariateGaussian import org.apache.spark.mllib.stat.test.ChiSqTestResult -import org.apache.spark.mllib.tree.{GradientBoostedTrees, RandomForest, DecisionTree} -import org.apache.spark.mllib.tree.configuration.{BoostingStrategy, Algo, Strategy} +import org.apache.spark.mllib.stat.{MultivariateStatisticalSummary, Statistics} +import org.apache.spark.mllib.tree.configuration.{Algo, BoostingStrategy, Strategy} import org.apache.spark.mllib.tree.impurity._ import org.apache.spark.mllib.tree.loss.Losses -import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel, RandomForestModel, DecisionTreeModel} +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTreesModel, RandomForestModel} +import org.apache.spark.mllib.tree.{DecisionTree, GradientBoostedTrees, RandomForest} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel @@ -279,7 +278,7 @@ private[python] class PythonMLLibAPI extends Serializable { data: JavaRDD[LabeledPoint], lambda: Double): JList[Object] = { val model = NaiveBayes.train(data.rdd, lambda) - List(Vectors.dense(model.labels), Vectors.dense(model.pi), model.theta). + List(Vectors.dense(model.labels), Vectors.dense(model.pi), model.theta.map(Vectors.dense)). map(_.asInstanceOf[Object]).asJava } @@ -335,7 +334,7 @@ private[python] class PythonMLLibAPI extends Serializable { mu += model.gaussians(i).mu sigma += model.gaussians(i).sigma } - List(wt.toArray, mu.toArray, sigma.toArray).map(_.asInstanceOf[Object]).asJava + List(Vectors.dense(wt.toArray), mu.toArray, sigma.toArray).map(_.asInstanceOf[Object]).asJava } finally { data.rdd.unpersist(blocking = false) } @@ -346,20 +345,20 @@ private[python] class PythonMLLibAPI extends Serializable { */ def predictSoftGMM( data: JavaRDD[Vector], - wt: Object, + wt: Vector, mu: Array[Object], - si: Array[Object]): RDD[Array[Double]] = { + si: Array[Object]): RDD[Vector] = { - val weight = wt.asInstanceOf[Array[Double]] + val weight = wt.toArray val mean = mu.map(_.asInstanceOf[DenseVector]) val sigma = si.map(_.asInstanceOf[DenseMatrix]) val gaussians = Array.tabulate(weight.length){ i => new MultivariateGaussian(mean(i), sigma(i)) } val model = new GaussianMixtureModel(weight, gaussians) - model.predictSoft(data) + model.predictSoft(data).map(Vectors.dense) } - + /** * Java stub for Python mllib ALS.train(). This stub returns a handle * to the Java object instead of the content of the Java object. Extra care @@ -936,6 +935,14 @@ private[spark] object SerDe extends Serializable { out.write(code) } + protected def getBytes(obj: Object): Array[Byte] = { + if (obj.getClass.isArray) { + obj.asInstanceOf[Array[Byte]] + } else { + obj.asInstanceOf[String].getBytes(LATIN1) + } + } + private[python] def saveState(obj: Object, out: OutputStream, pickler: Pickler) } @@ -961,7 +968,7 @@ private[spark] object SerDe extends Serializable { if (args.length != 1) { throw new PickleException("should be 1") } - val bytes = args(0).asInstanceOf[String].getBytes(LATIN1) + val bytes = getBytes(args(0)) val bb = ByteBuffer.wrap(bytes, 0, bytes.length) bb.order(ByteOrder.nativeOrder()) val db = bb.asDoubleBuffer() @@ -994,7 +1001,7 @@ private[spark] object SerDe extends Serializable { if (args.length != 3) { throw new PickleException("should be 3") } - val bytes = args(2).asInstanceOf[String].getBytes(LATIN1) + val bytes = getBytes(args(2)) val n = bytes.length / 8 val values = new Array[Double](n) val order = ByteOrder.nativeOrder() @@ -1031,8 +1038,8 @@ private[spark] object SerDe extends Serializable { throw new PickleException("should be 3") } val size = args(0).asInstanceOf[Int] - val indiceBytes = args(1).asInstanceOf[String].getBytes(LATIN1) - val valueBytes = args(2).asInstanceOf[String].getBytes(LATIN1) + val indiceBytes = getBytes(args(1)) + val valueBytes = getBytes(args(2)) val n = indiceBytes.length / 4 val indices = new Array[Int](n) val values = new Array[Double](n) diff --git a/python/pyspark/accumulators.py b/python/pyspark/accumulators.py index ccbca67656c8d..7271809e43880 100644 --- a/python/pyspark/accumulators.py +++ b/python/pyspark/accumulators.py @@ -54,7 +54,7 @@ ... def zero(self, value): ... return [0.0] * len(value) ... def addInPlace(self, val1, val2): -... for i in xrange(len(val1)): +... for i in range(len(val1)): ... val1[i] += val2[i] ... return val1 >>> va = sc.accumulator([1.0, 2.0, 3.0], VectorAccumulatorParam()) @@ -86,9 +86,13 @@ Exception:... """ +import sys import select import struct -import SocketServer +if sys.version < '3': + import SocketServer +else: + import socketserver as SocketServer import threading from pyspark.cloudpickle import CloudPickler from pyspark.serializers import read_int, PickleSerializer @@ -247,6 +251,7 @@ class AccumulatorServer(SocketServer.TCPServer): def shutdown(self): self.server_shutdown = True SocketServer.TCPServer.shutdown(self) + self.server_close() def _start_update_server(): diff --git a/python/pyspark/broadcast.py b/python/pyspark/broadcast.py index 6b8a8b256a891..3de4615428bb6 100644 --- a/python/pyspark/broadcast.py +++ b/python/pyspark/broadcast.py @@ -16,10 +16,15 @@ # import os -import cPickle +import sys import gc from tempfile import NamedTemporaryFile +if sys.version < '3': + import cPickle as pickle +else: + import pickle + unicode = str __all__ = ['Broadcast'] @@ -70,33 +75,19 @@ def __init__(self, sc=None, value=None, pickle_registry=None, path=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) + pickle.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 + # pickle.load() may create lots of objects, disable GC + # temporary for better performance + gc.disable() + try: + return pickle.load(f) + finally: + gc.enable() @property def value(self): diff --git a/python/pyspark/cloudpickle.py b/python/pyspark/cloudpickle.py index bb0783555aa77..9ef93071d2e77 100644 --- a/python/pyspark/cloudpickle.py +++ b/python/pyspark/cloudpickle.py @@ -40,164 +40,126 @@ NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. """ - +from __future__ import print_function import operator import os +import io import pickle import struct import sys import types from functools import partial import itertools -from copy_reg import _extension_registry, _inverted_registry, _extension_cache -import new import dis import traceback -import platform - -PyImp = platform.python_implementation() - -import logging -cloudLog = logging.getLogger("Cloud.Transport") +if sys.version < '3': + from pickle import Pickler + try: + from cStringIO import StringIO + except ImportError: + from StringIO import StringIO + PY3 = False +else: + types.ClassType = type + from pickle import _Pickler as Pickler + from io import BytesIO as StringIO + PY3 = True #relevant opcodes -STORE_GLOBAL = chr(dis.opname.index('STORE_GLOBAL')) -DELETE_GLOBAL = chr(dis.opname.index('DELETE_GLOBAL')) -LOAD_GLOBAL = chr(dis.opname.index('LOAD_GLOBAL')) +STORE_GLOBAL = dis.opname.index('STORE_GLOBAL') +DELETE_GLOBAL = dis.opname.index('DELETE_GLOBAL') +LOAD_GLOBAL = dis.opname.index('LOAD_GLOBAL') GLOBAL_OPS = [STORE_GLOBAL, DELETE_GLOBAL, LOAD_GLOBAL] +HAVE_ARGUMENT = dis.HAVE_ARGUMENT +EXTENDED_ARG = dis.EXTENDED_ARG -HAVE_ARGUMENT = chr(dis.HAVE_ARGUMENT) -EXTENDED_ARG = chr(dis.EXTENDED_ARG) - -if PyImp == "PyPy": - # register builtin type in `new` - new.method = types.MethodType - -try: - from cStringIO import StringIO -except ImportError: - from StringIO import StringIO -# These helper functions were copied from PiCloud's util module. def islambda(func): - return getattr(func,'func_name') == '' + return getattr(func,'__name__') == '' -def xrange_params(xrangeobj): - """Returns a 3 element tuple describing the xrange start, step, and len - respectively - Note: Only guarentees that elements of xrange are the same. parameters may - be different. - e.g. xrange(1,1) is interpretted as xrange(0,0); both behave the same - though w/ iteration - """ - - xrange_len = len(xrangeobj) - if not xrange_len: #empty - return (0,1,0) - start = xrangeobj[0] - if xrange_len == 1: #one element - return start, 1, 1 - return (start, xrangeobj[1] - xrangeobj[0], xrange_len) - -#debug variables intended for developer use: -printSerialization = False -printMemoization = False +_BUILTIN_TYPE_NAMES = {} +for k, v in types.__dict__.items(): + if type(v) is type: + _BUILTIN_TYPE_NAMES[v] = k -useForcedImports = True #Should I use forced imports for tracking? +def _builtin_type(name): + return getattr(types, name) -class CloudPickler(pickle.Pickler): +class CloudPickler(Pickler): - dispatch = pickle.Pickler.dispatch.copy() - savedForceImports = False - savedDjangoEnv = False #hack tro transport django environment + dispatch = Pickler.dispatch.copy() - def __init__(self, file, protocol=None, min_size_to_save= 0): - pickle.Pickler.__init__(self,file,protocol) - self.modules = set() #set of modules needed to depickle - self.globals_ref = {} # map ids to dictionary. used to ensure that functions can share global env + def __init__(self, file, protocol=None): + Pickler.__init__(self, file, protocol) + # set of modules to unpickle + self.modules = set() + # map ids to dictionary. used to ensure that functions can share global env + self.globals_ref = {} def dump(self, obj): - # note: not thread safe - # minimal side-effects, so not fixing - recurse_limit = 3000 - base_recurse = sys.getrecursionlimit() - if base_recurse < recurse_limit: - sys.setrecursionlimit(recurse_limit) self.inject_addons() try: - return pickle.Pickler.dump(self, obj) - except RuntimeError, e: + return Pickler.dump(self, obj) + except RuntimeError as e: if 'recursion' in e.args[0]: - msg = """Could not pickle object as excessively deep recursion required. - Try _fast_serialization=2 or contact PiCloud support""" + msg = """Could not pickle object as excessively deep recursion required.""" raise pickle.PicklingError(msg) - finally: - new_recurse = sys.getrecursionlimit() - if new_recurse == recurse_limit: - sys.setrecursionlimit(base_recurse) + + def save_memoryview(self, obj): + """Fallback to save_string""" + Pickler.save_string(self, str(obj)) def save_buffer(self, obj): """Fallback to save_string""" - pickle.Pickler.save_string(self,str(obj)) - dispatch[buffer] = save_buffer + Pickler.save_string(self,str(obj)) + if PY3: + dispatch[memoryview] = save_memoryview + else: + dispatch[buffer] = save_buffer - #block broken objects - def save_unsupported(self, obj, pack=None): + def save_unsupported(self, obj): raise pickle.PicklingError("Cannot pickle objects of type %s" % type(obj)) dispatch[types.GeneratorType] = save_unsupported - #python2.6+ supports slice pickling. some py2.5 extensions might as well. We just test it - try: - slice(0,1).__reduce__() - except TypeError: #can't pickle - - dispatch[slice] = save_unsupported - - #itertools objects do not pickle! + # itertools objects do not pickle! for v in itertools.__dict__.values(): if type(v) is type: dispatch[v] = save_unsupported - - def save_dict(self, obj): - """hack fix - If the dict is a global, deal with it in a special way - """ - #print 'saving', obj - if obj is __builtins__: - self.save_reduce(_get_module_builtins, (), obj=obj) - else: - pickle.Pickler.save_dict(self, obj) - dispatch[pickle.DictionaryType] = save_dict - - - def save_module(self, obj, pack=struct.pack): + def save_module(self, obj): """ Save a module as an import """ - #print 'try save import', obj.__name__ self.modules.add(obj) - self.save_reduce(subimport,(obj.__name__,), obj=obj) - dispatch[types.ModuleType] = save_module #new type + self.save_reduce(subimport, (obj.__name__,), obj=obj) + dispatch[types.ModuleType] = save_module - def save_codeobject(self, obj, pack=struct.pack): + def save_codeobject(self, obj): """ Save a code object """ - #print 'try to save codeobj: ', obj - args = ( - obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, - obj.co_consts, obj.co_names, obj.co_varnames, obj.co_filename, obj.co_name, - obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, obj.co_cellvars - ) + if PY3: + args = ( + obj.co_argcount, obj.co_kwonlyargcount, obj.co_nlocals, obj.co_stacksize, + obj.co_flags, obj.co_code, obj.co_consts, obj.co_names, obj.co_varnames, + obj.co_filename, obj.co_name, obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, + obj.co_cellvars + ) + else: + args = ( + obj.co_argcount, obj.co_nlocals, obj.co_stacksize, obj.co_flags, obj.co_code, + obj.co_consts, obj.co_names, obj.co_varnames, obj.co_filename, obj.co_name, + obj.co_firstlineno, obj.co_lnotab, obj.co_freevars, obj.co_cellvars + ) self.save_reduce(types.CodeType, args, obj=obj) - dispatch[types.CodeType] = save_codeobject #new type + dispatch[types.CodeType] = save_codeobject - def save_function(self, obj, name=None, pack=struct.pack): + def save_function(self, obj, name=None): """ Registered with the dispatch to handle all function types. Determines what kind of function obj is (e.g. lambda, defined at @@ -205,12 +167,14 @@ def save_function(self, obj, name=None, pack=struct.pack): """ write = self.write - name = obj.__name__ + if name is None: + name = obj.__name__ modname = pickle.whichmodule(obj, name) - #print 'which gives %s %s %s' % (modname, obj, name) + # print('which gives %s %s %s' % (modname, obj, name)) try: themodule = sys.modules[modname] - except KeyError: # eval'd items such as namedtuple give invalid items for their function __module__ + except KeyError: + # eval'd items such as namedtuple give invalid items for their function __module__ modname = '__main__' if modname == '__main__': @@ -221,37 +185,18 @@ def save_function(self, obj, name=None, pack=struct.pack): if getattr(themodule, name, None) is obj: return self.save_global(obj, name) - if not self.savedDjangoEnv: - #hack for django - if we detect the settings module, we transport it - django_settings = os.environ.get('DJANGO_SETTINGS_MODULE', '') - if django_settings: - django_mod = sys.modules.get(django_settings) - if django_mod: - cloudLog.debug('Transporting django settings %s during save of %s', django_mod, name) - self.savedDjangoEnv = True - self.modules.add(django_mod) - write(pickle.MARK) - self.save_reduce(django_settings_load, (django_mod.__name__,), obj=django_mod) - write(pickle.POP_MARK) - - # if func is lambda, def'ed at prompt, is in main, or is nested, then # we'll pickle the actual function object rather than simply saving a # reference (as is done in default pickler), via save_function_tuple. - if islambda(obj) or obj.func_code.co_filename == '' or themodule is None: - #Force server to import modules that have been imported in main - modList = None - if themodule is None and not self.savedForceImports: - mainmod = sys.modules['__main__'] - if useForcedImports and hasattr(mainmod,'___pyc_forcedImports__'): - modList = list(mainmod.___pyc_forcedImports__) - self.savedForceImports = True - self.save_function_tuple(obj, modList) + if islambda(obj) or obj.__code__.co_filename == '' or themodule is None: + #print("save global", islambda(obj), obj.__code__.co_filename, modname, themodule) + self.save_function_tuple(obj) return - else: # func is nested + else: + # func is nested klass = getattr(themodule, name, None) if klass is None or klass is not obj: - self.save_function_tuple(obj, [themodule]) + self.save_function_tuple(obj) return if obj.__dict__: @@ -266,7 +211,7 @@ def save_function(self, obj, name=None, pack=struct.pack): self.memoize(obj) dispatch[types.FunctionType] = save_function - def save_function_tuple(self, func, forced_imports): + def save_function_tuple(self, func): """ Pickles an actual func object. A func comprises: code, globals, defaults, closure, and dict. We @@ -281,19 +226,6 @@ def save_function_tuple(self, func, forced_imports): save = self.save write = self.write - # save the modules (if any) - if forced_imports: - write(pickle.MARK) - save(_modules_to_main) - #print 'forced imports are', forced_imports - - forced_names = map(lambda m: m.__name__, forced_imports) - save((forced_names,)) - - #save((forced_imports,)) - write(pickle.REDUCE) - write(pickle.POP_MARK) - code, f_globals, defaults, closure, dct, base_globals = self.extract_func_data(func) save(_fill_function) # skeleton function updater @@ -318,6 +250,8 @@ def extract_code_globals(co): Find all globals names read or written to by codeblock co """ code = co.co_code + if not PY3: + code = [ord(c) for c in code] names = co.co_names out_names = set() @@ -327,18 +261,18 @@ def extract_code_globals(co): while i < n: op = code[i] - i = i+1 + i += 1 if op >= HAVE_ARGUMENT: - oparg = ord(code[i]) + ord(code[i+1])*256 + extended_arg + oparg = code[i] + code[i+1] * 256 + extended_arg extended_arg = 0 - i = i+2 + i += 2 if op == EXTENDED_ARG: - extended_arg = oparg*65536L + extended_arg = oparg*65536 if op in GLOBAL_OPS: out_names.add(names[oparg]) - #print 'extracted', out_names, ' from ', names - if co.co_consts: # see if nested function have any global refs + # see if nested function have any global refs + if co.co_consts: for const in co.co_consts: if type(const) is types.CodeType: out_names |= CloudPickler.extract_code_globals(const) @@ -350,46 +284,28 @@ def extract_func_data(self, func): Turn the function into a tuple of data necessary to recreate it: code, globals, defaults, closure, dict """ - code = func.func_code + code = func.__code__ # extract all global ref's - func_global_refs = CloudPickler.extract_code_globals(code) + func_global_refs = self.extract_code_globals(code) # process all variables referenced by global environment f_globals = {} for var in func_global_refs: - #Some names, such as class functions are not global - we don't need them - if func.func_globals.has_key(var): - f_globals[var] = func.func_globals[var] + if var in func.__globals__: + f_globals[var] = func.__globals__[var] # defaults requires no processing - defaults = func.func_defaults - - def get_contents(cell): - try: - return cell.cell_contents - except ValueError, e: #cell is empty error on not yet assigned - raise pickle.PicklingError('Function to be pickled has free variables that are referenced before assignment in enclosing scope') - + defaults = func.__defaults__ # process closure - if func.func_closure: - closure = map(get_contents, func.func_closure) - else: - closure = [] + closure = [c.cell_contents for c in func.__closure__] if func.__closure__ else [] # save the dict - dct = func.func_dict - - if printSerialization: - outvars = ['code: ' + str(code) ] - outvars.append('globals: ' + str(f_globals)) - outvars.append('defaults: ' + str(defaults)) - outvars.append('closure: ' + str(closure)) - print 'function ', func, 'is extracted to: ', ', '.join(outvars) + dct = func.__dict__ - base_globals = self.globals_ref.get(id(func.func_globals), {}) - self.globals_ref[id(func.func_globals)] = base_globals + base_globals = self.globals_ref.get(id(func.__globals__), {}) + self.globals_ref[id(func.__globals__)] = base_globals return (code, f_globals, defaults, closure, dct, base_globals) @@ -400,8 +316,9 @@ def save_builtin_function(self, obj): dispatch[types.BuiltinFunctionType] = save_builtin_function def save_global(self, obj, name=None, pack=struct.pack): - write = self.write - memo = self.memo + if obj.__module__ == "__builtin__" or obj.__module__ == "builtins": + if obj in _BUILTIN_TYPE_NAMES: + return self.save_reduce(_builtin_type, (_BUILTIN_TYPE_NAMES[obj],), obj=obj) if name is None: name = obj.__name__ @@ -410,98 +327,57 @@ def save_global(self, obj, name=None, pack=struct.pack): if modname is None: modname = pickle.whichmodule(obj, name) - try: - __import__(modname) - themodule = sys.modules[modname] - except (ImportError, KeyError, AttributeError): #should never occur - raise pickle.PicklingError( - "Can't pickle %r: Module %s cannot be found" % - (obj, modname)) - if modname == '__main__': themodule = None - - if themodule: + else: + __import__(modname) + themodule = sys.modules[modname] self.modules.add(themodule) - sendRef = True - typ = type(obj) - #print 'saving', obj, typ - try: - try: #Deal with case when getattribute fails with exceptions - klass = getattr(themodule, name) - except (AttributeError): - if modname == '__builtin__': #new.* are misrepeported - modname = 'new' - __import__(modname) - themodule = sys.modules[modname] - try: - klass = getattr(themodule, name) - except AttributeError, a: - # print themodule, name, obj, type(obj) - raise pickle.PicklingError("Can't pickle builtin %s" % obj) - else: - raise + if hasattr(themodule, name) and getattr(themodule, name) is obj: + return Pickler.save_global(self, obj, name) - except (ImportError, KeyError, AttributeError): - if typ == types.TypeType or typ == types.ClassType: - sendRef = False - else: #we can't deal with this - raise - else: - if klass is not obj and (typ == types.TypeType or typ == types.ClassType): - sendRef = False - if not sendRef: - #note: Third party types might crash this - add better checks! - d = dict(obj.__dict__) #copy dict proxy to a dict - if not isinstance(d.get('__dict__', None), property): # don't extract dict that are properties - d.pop('__dict__',None) - d.pop('__weakref__',None) + typ = type(obj) + if typ is not obj and isinstance(obj, (type, types.ClassType)): + d = dict(obj.__dict__) # copy dict proxy to a dict + if not isinstance(d.get('__dict__', None), property): + # don't extract dict that are properties + d.pop('__dict__', None) + d.pop('__weakref__', None) # hack as __new__ is stored differently in the __dict__ new_override = d.get('__new__', None) if new_override: d['__new__'] = obj.__new__ - self.save_reduce(type(obj),(obj.__name__,obj.__bases__, - d),obj=obj) - #print 'internal reduce dask %s %s' % (obj, d) - return - - if self.proto >= 2: - code = _extension_registry.get((modname, name)) - if code: - assert code > 0 - if code <= 0xff: - write(pickle.EXT1 + chr(code)) - elif code <= 0xffff: - write("%c%c%c" % (pickle.EXT2, code&0xff, code>>8)) - else: - write(pickle.EXT4 + pack("> sys.stderr, 'Cloud not import django settings %s:' % (name) - print_exec(sys.stderr) - if modified_env: - del os.environ['DJANGO_SETTINGS_MODULE'] - else: - #add project directory to sys,path: - if hasattr(module,'__file__'): - dirname = os.path.split(module.__file__)[0] + '/' - sys.path.append(dirname) # restores function attributes def _restore_attr(obj, attr): @@ -851,13 +636,16 @@ def _restore_attr(obj, attr): setattr(obj, key, val) return obj + def _get_module_builtins(): return pickle.__builtins__ + def print_exec(stream): ei = sys.exc_info() traceback.print_exception(ei[0], ei[1], ei[2], None, stream) + def _modules_to_main(modList): """Force every module in modList to be placed into main""" if not modList: @@ -868,22 +656,16 @@ def _modules_to_main(modList): if type(modname) is str: try: mod = __import__(modname) - except Exception, i: #catch all... - sys.stderr.write('warning: could not import %s\n. Your function may unexpectedly error due to this import failing; \ -A version mismatch is likely. Specific error was:\n' % modname) + except Exception as e: + sys.stderr.write('warning: could not import %s\n. ' + 'Your function may unexpectedly error due to this import failing;' + 'A version mismatch is likely. Specific error was:\n' % modname) print_exec(sys.stderr) else: - setattr(main,mod.__name__, mod) - else: - #REVERSE COMPATIBILITY FOR CLOUD CLIENT 1.5 (WITH EPD) - #In old version actual module was sent - setattr(main,modname.__name__, modname) + setattr(main, mod.__name__, mod) -#object generators: -def _build_xrange(start, step, len): - """Built xrange explicitly""" - return xrange(start, start + step*len, step) +#object generators: def _genpartial(func, args, kwds): if not args: args = () @@ -891,22 +673,26 @@ def _genpartial(func, args, kwds): kwds = {} return partial(func, *args, **kwds) + def _fill_function(func, globals, defaults, dict): """ Fills in the rest of function data into the skeleton function object that were created via _make_skel_func(). """ - func.func_globals.update(globals) - func.func_defaults = defaults - func.func_dict = dict + func.__globals__.update(globals) + func.__defaults__ = defaults + func.__dict__ = dict return func + def _make_cell(value): - return (lambda: value).func_closure[0] + return (lambda: value).__closure__[0] + def _reconstruct_closure(values): return tuple([_make_cell(v) for v in values]) + def _make_skel_func(code, closures, base_globals = None): """ Creates a skeleton function object that contains just the provided code and the correct number of cells in func_closure. All other @@ -928,40 +714,3 @@ def _make_skel_func(code, closures, base_globals = None): def _getobject(modname, attribute): mod = __import__(modname, fromlist=[attribute]) return mod.__dict__[attribute] - -def _generateImage(size, mode, str_rep): - """Generate image from string representation""" - import Image - i = Image.new(mode, size) - i.fromstring(str_rep) - return i - -def _lazyloadImage(fp): - import Image - fp.seek(0) #works in almost any case - return Image.open(fp) - -"""Timeseries""" -def _genTimeSeries(reduce_args, state): - import scikits.timeseries.tseries as ts - from numpy import ndarray - from numpy.ma import MaskedArray - - - time_series = ts._tsreconstruct(*reduce_args) - - #from setstate modified - (ver, shp, typ, isf, raw, msk, flv, dsh, dtm, dtyp, frq, infodict) = state - #print 'regenerating %s' % dtyp - - MaskedArray.__setstate__(time_series, (ver, shp, typ, isf, raw, msk, flv)) - _dates = time_series._dates - #_dates.__setstate__((ver, dsh, typ, isf, dtm, frq)) #use remote typ - ndarray.__setstate__(_dates,(dsh,dtyp, isf, dtm)) - _dates.freq = frq - _dates._cachedinfo.update(dict(full=None, hasdups=None, steps=None, - toobj=None, toord=None, tostr=None)) - # Update the _optinfo dictionary - time_series._optinfo.update(infodict) - return time_series - diff --git a/python/pyspark/conf.py b/python/pyspark/conf.py index dc7cd0bce56f3..924da3eecf214 100644 --- a/python/pyspark/conf.py +++ b/python/pyspark/conf.py @@ -44,7 +44,7 @@ >>> conf.get("spark.executorEnv.VAR1") u'value1' ->>> print conf.toDebugString() +>>> print(conf.toDebugString()) spark.executorEnv.VAR1=value1 spark.executorEnv.VAR3=value3 spark.executorEnv.VAR4=value4 @@ -56,6 +56,13 @@ __all__ = ['SparkConf'] +import sys +import re + +if sys.version > '3': + unicode = str + __doc__ = re.sub(r"(\W|^)[uU](['])", r'\1\2', __doc__) + class SparkConf(object): diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 78dccc40470e3..1dc2fec0ae5c8 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + import os import shutil import sys @@ -32,11 +34,14 @@ from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ PairDeserializer, AutoBatchedSerializer, NoOpSerializer from pyspark.storagelevel import StorageLevel -from pyspark.rdd import RDD, _load_from_socket +from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix from pyspark.traceback_utils import CallSite, first_spark_call from pyspark.status import StatusTracker from pyspark.profiler import ProfilerCollector, BasicProfiler +if sys.version > '3': + xrange = range + __all__ = ['SparkContext'] @@ -133,7 +138,7 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, if sparkHome: self._conf.setSparkHome(sparkHome) if environment: - for key, value in environment.iteritems(): + for key, value in environment.items(): self._conf.setExecutorEnv(key, value) for key, value in DEFAULT_CONFIGS.items(): self._conf.setIfMissing(key, value) @@ -153,6 +158,10 @@ def _do_init(self, master, appName, sparkHome, pyFiles, environment, batchSize, if k.startswith("spark.executorEnv."): varName = k[len("spark.executorEnv."):] self.environment[varName] = v + if sys.version >= '3.3' and 'PYTHONHASHSEED' not in os.environ: + # disable randomness of hash of string in worker, if this is not + # launched by spark-submit + self.environment["PYTHONHASHSEED"] = "0" # Create the Java SparkContext through Py4J self._jsc = jsc or self._initialize_context(self._conf._jconf) @@ -323,7 +332,7 @@ def parallelize(self, c, numSlices=None): start0 = c[0] def getStart(split): - return start0 + (split * size / numSlices) * step + return start0 + int((split * size / numSlices)) * step def f(split, iterator): return xrange(getStart(split), getStart(split + 1), step) @@ -357,6 +366,7 @@ def pickleFile(self, name, minPartitions=None): minPartitions = minPartitions or self.defaultMinPartitions return RDD(self._jsc.objectFile(name, minPartitions), self) + @ignore_unicode_prefix def textFile(self, name, minPartitions=None, use_unicode=True): """ Read a text file from HDFS, a local file system (available on all @@ -369,7 +379,7 @@ def textFile(self, name, minPartitions=None, use_unicode=True): >>> path = os.path.join(tempdir, "sample-text.txt") >>> with open(path, "w") as testFile: - ... testFile.write("Hello world!") + ... _ = testFile.write("Hello world!") >>> textFile = sc.textFile(path) >>> textFile.collect() [u'Hello world!'] @@ -378,6 +388,7 @@ def textFile(self, name, minPartitions=None, use_unicode=True): return RDD(self._jsc.textFile(name, minPartitions), self, UTF8Deserializer(use_unicode)) + @ignore_unicode_prefix def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): """ Read a directory of text files from HDFS, a local file system @@ -411,9 +422,9 @@ def wholeTextFiles(self, path, minPartitions=None, use_unicode=True): >>> dirPath = os.path.join(tempdir, "files") >>> os.mkdir(dirPath) >>> with open(os.path.join(dirPath, "1.txt"), "w") as file1: - ... file1.write("1") + ... _ = file1.write("1") >>> with open(os.path.join(dirPath, "2.txt"), "w") as file2: - ... file2.write("2") + ... _ = file2.write("2") >>> textFiles = sc.wholeTextFiles(dirPath) >>> sorted(textFiles.collect()) [(u'.../1.txt', u'1'), (u'.../2.txt', u'2')] @@ -456,7 +467,7 @@ def _dictToJavaMap(self, d): jm = self._jvm.java.util.HashMap() if not d: d = {} - for k, v in d.iteritems(): + for k, v in d.items(): jm[k] = v return jm @@ -608,6 +619,7 @@ def _checkpointFile(self, name, input_deserializer): jrdd = self._jsc.checkpointFile(name) return RDD(jrdd, self, input_deserializer) + @ignore_unicode_prefix def union(self, rdds): """ Build the union of a list of RDDs. @@ -618,7 +630,7 @@ def union(self, rdds): >>> path = os.path.join(tempdir, "union-text.txt") >>> with open(path, "w") as testFile: - ... testFile.write("Hello") + ... _ = testFile.write("Hello") >>> textFile = sc.textFile(path) >>> textFile.collect() [u'Hello'] @@ -677,7 +689,7 @@ def addFile(self, path): >>> from pyspark import SparkFiles >>> path = os.path.join(tempdir, "test.txt") >>> with open(path, "w") as testFile: - ... testFile.write("100") + ... _ = testFile.write("100") >>> sc.addFile(path) >>> def func(iterator): ... with open(SparkFiles.get("test.txt")) as testFile: @@ -705,11 +717,13 @@ def addPyFile(self, path): """ self.addFile(path) (dirname, filename) = os.path.split(path) # dirname may be directory or HDFS/S3 prefix - if filename[-4:].lower() in self.PACKAGE_EXTENSIONS: self._python_includes.append(filename) # for tests in local mode sys.path.insert(1, os.path.join(SparkFiles.getRootDirectory(), filename)) + if sys.version > '3': + import importlib + importlib.invalidate_caches() def setCheckpointDir(self, dirName): """ @@ -744,7 +758,7 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): The application can use L{SparkContext.cancelJobGroup} to cancel all running jobs in this group. - >>> import thread, threading + >>> import threading >>> from time import sleep >>> result = "Not Set" >>> lock = threading.Lock() @@ -763,10 +777,10 @@ def setJobGroup(self, groupId, description, interruptOnCancel=False): ... sleep(5) ... sc.cancelJobGroup("job_to_cancel") >>> supress = lock.acquire() - >>> supress = thread.start_new_thread(start_job, (10,)) - >>> supress = thread.start_new_thread(stop_job, tuple()) + >>> supress = threading.Thread(target=start_job, args=(10,)).start() + >>> supress = threading.Thread(target=stop_job).start() >>> supress = lock.acquire() - >>> print result + >>> print(result) Cancelled If interruptOnCancel is set to true for the job group, then job cancellation will result diff --git a/python/pyspark/daemon.py b/python/pyspark/daemon.py index 93885985fe377..7f06d4288c872 100644 --- a/python/pyspark/daemon.py +++ b/python/pyspark/daemon.py @@ -24,9 +24,10 @@ import traceback import time import gc -from errno import EINTR, ECHILD, EAGAIN +from errno import EINTR, EAGAIN from socket import AF_INET, SOCK_STREAM, SOMAXCONN from signal import SIGHUP, SIGTERM, SIGCHLD, SIG_DFL, SIG_IGN, SIGINT + from pyspark.worker import main as worker_main from pyspark.serializers import read_int, write_int @@ -53,8 +54,8 @@ def worker(sock): # Read the socket using fdopen instead of socket.makefile() because the latter # seems to be very slow; note that we need to dup() the file descriptor because # otherwise writes also cause a seek that makes us miss data on the read side. - infile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) - outfile = os.fdopen(os.dup(sock.fileno()), "a+", 65536) + infile = os.fdopen(os.dup(sock.fileno()), "rb", 65536) + outfile = os.fdopen(os.dup(sock.fileno()), "wb", 65536) exit_code = 0 try: worker_main(infile, outfile) @@ -68,17 +69,6 @@ def worker(sock): return exit_code -# Cleanup zombie children -def cleanup_dead_children(): - try: - while True: - pid, _ = os.waitpid(0, os.WNOHANG) - if not pid: - break - except: - pass - - def manager(): # Create a new process group to corral our children os.setpgid(0, 0) @@ -88,8 +78,12 @@ def manager(): listen_sock.bind(('127.0.0.1', 0)) listen_sock.listen(max(1024, SOMAXCONN)) listen_host, listen_port = listen_sock.getsockname() - write_int(listen_port, sys.stdout) - sys.stdout.flush() + + # re-open stdin/stdout in 'wb' mode + stdin_bin = os.fdopen(sys.stdin.fileno(), 'rb', 4) + stdout_bin = os.fdopen(sys.stdout.fileno(), 'wb', 4) + write_int(listen_port, stdout_bin) + stdout_bin.flush() def shutdown(code): signal.signal(SIGTERM, SIG_DFL) @@ -101,6 +95,7 @@ def handle_sigterm(*args): shutdown(1) signal.signal(SIGTERM, handle_sigterm) # Gracefully exit on SIGTERM signal.signal(SIGHUP, SIG_IGN) # Don't die on SIGHUP + signal.signal(SIGCHLD, SIG_IGN) reuse = os.environ.get("SPARK_REUSE_WORKER") @@ -115,12 +110,9 @@ def handle_sigterm(*args): else: raise - # cleanup in signal handler will cause deadlock - cleanup_dead_children() - if 0 in ready_fds: try: - worker_pid = read_int(sys.stdin) + worker_pid = read_int(stdin_bin) except EOFError: # Spark told us to exit by closing stdin shutdown(0) @@ -145,7 +137,7 @@ def handle_sigterm(*args): time.sleep(1) pid = os.fork() # error here will shutdown daemon else: - outfile = sock.makefile('w') + outfile = sock.makefile(mode='wb') write_int(e.errno, outfile) # Signal that the fork failed outfile.flush() outfile.close() @@ -157,7 +149,7 @@ def handle_sigterm(*args): listen_sock.close() try: # Acknowledge that the fork was successful - outfile = sock.makefile("w") + outfile = sock.makefile(mode="wb") write_int(os.getpid(), outfile) outfile.flush() outfile.close() diff --git a/python/pyspark/heapq3.py b/python/pyspark/heapq3.py index bc441f138f7fc..4ef2afe03544f 100644 --- a/python/pyspark/heapq3.py +++ b/python/pyspark/heapq3.py @@ -627,51 +627,49 @@ def merge(iterables, key=None, reverse=False): if key is None: for order, it in enumerate(map(iter, iterables)): try: - next = it.next - h_append([next(), order * direction, next]) + h_append([next(it), order * direction, it]) except StopIteration: pass _heapify(h) while len(h) > 1: try: while True: - value, order, next = s = h[0] + value, order, it = s = h[0] yield value - s[0] = next() # raises StopIteration when exhausted + s[0] = next(it) # raises StopIteration when exhausted _heapreplace(h, s) # restore heap condition except StopIteration: _heappop(h) # remove empty iterator if h: # fast case when only a single iterator remains - value, order, next = h[0] + value, order, it = h[0] yield value - for value in next.__self__: + for value in it: yield value return for order, it in enumerate(map(iter, iterables)): try: - next = it.next - value = next() - h_append([key(value), order * direction, value, next]) + value = next(it) + h_append([key(value), order * direction, value, it]) except StopIteration: pass _heapify(h) while len(h) > 1: try: while True: - key_value, order, value, next = s = h[0] + key_value, order, value, it = s = h[0] yield value - value = next() + value = next(it) s[0] = key(value) s[2] = value _heapreplace(h, s) except StopIteration: _heappop(h) if h: - key_value, order, value, next = h[0] + key_value, order, value, it = h[0] yield value - for value in next.__self__: + for value in it: yield value diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index 2a5e84a7dfdb4..45bc38f7e61f8 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -69,7 +69,7 @@ def preexec_func(): if callback_socket in readable: gateway_connection = callback_socket.accept()[0] # Determine which ephemeral port the server started on: - gateway_port = read_int(gateway_connection.makefile()) + gateway_port = read_int(gateway_connection.makefile(mode="rb")) gateway_connection.close() callback_socket.close() if gateway_port is None: diff --git a/python/pyspark/join.py b/python/pyspark/join.py index c3491defb2b29..94df3990164d6 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -32,6 +32,7 @@ """ from pyspark.resultiterable import ResultIterable +from functools import reduce def _do_python_join(rdd, other, numPartitions, dispatch): diff --git a/python/pyspark/ml/classification.py b/python/pyspark/ml/classification.py index d7bc09fd77adb..45754bc9d4b10 100644 --- a/python/pyspark/ml/classification.py +++ b/python/pyspark/ml/classification.py @@ -39,10 +39,10 @@ class LogisticRegression(JavaEstimator, HasFeaturesCol, HasLabelCol, HasPredicti >>> lr = LogisticRegression(maxIter=5, regParam=0.01) >>> model = lr.fit(df) >>> test0 = sc.parallelize([Row(features=Vectors.dense(-1.0))]).toDF() - >>> print model.transform(test0).head().prediction + >>> model.transform(test0).head().prediction 0.0 >>> test1 = sc.parallelize([Row(features=Vectors.sparse(1, [0], [1.0]))]).toDF() - >>> print model.transform(test1).head().prediction + >>> model.transform(test1).head().prediction 1.0 >>> lr.setParams("vector") Traceback (most recent call last): diff --git a/python/pyspark/ml/feature.py b/python/pyspark/ml/feature.py index 263fe2a5bcc41..4e4614b859ac6 100644 --- a/python/pyspark/ml/feature.py +++ b/python/pyspark/ml/feature.py @@ -15,6 +15,7 @@ # limitations under the License. # +from pyspark.rdd import ignore_unicode_prefix from pyspark.ml.param.shared import HasInputCol, HasOutputCol, HasNumFeatures from pyspark.ml.util import keyword_only from pyspark.ml.wrapper import JavaTransformer @@ -24,6 +25,7 @@ @inherit_doc +@ignore_unicode_prefix class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): """ A tokenizer that converts the input string to lowercase and then @@ -32,15 +34,15 @@ class Tokenizer(JavaTransformer, HasInputCol, HasOutputCol): >>> from pyspark.sql import Row >>> df = sc.parallelize([Row(text="a b c")]).toDF() >>> tokenizer = Tokenizer(inputCol="text", outputCol="words") - >>> print tokenizer.transform(df).head() + >>> tokenizer.transform(df).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) >>> # Change a parameter. - >>> print tokenizer.setParams(outputCol="tokens").transform(df).head() + >>> tokenizer.setParams(outputCol="tokens").transform(df).head() Row(text=u'a b c', tokens=[u'a', u'b', u'c']) >>> # Temporarily modify a parameter. - >>> print tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() + >>> tokenizer.transform(df, {tokenizer.outputCol: "words"}).head() Row(text=u'a b c', words=[u'a', u'b', u'c']) - >>> print tokenizer.transform(df).head() + >>> tokenizer.transform(df).head() Row(text=u'a b c', tokens=[u'a', u'b', u'c']) >>> # Must use keyword arguments to specify params. >>> tokenizer.setParams("text") @@ -79,13 +81,13 @@ class HashingTF(JavaTransformer, HasInputCol, HasOutputCol, HasNumFeatures): >>> from pyspark.sql import Row >>> df = sc.parallelize([Row(words=["a", "b", "c"])]).toDF() >>> hashingTF = HashingTF(numFeatures=10, inputCol="words", outputCol="features") - >>> print hashingTF.transform(df).head().features - (10,[7,8,9],[1.0,1.0,1.0]) - >>> print hashingTF.setParams(outputCol="freqs").transform(df).head().freqs - (10,[7,8,9],[1.0,1.0,1.0]) + >>> hashingTF.transform(df).head().features + SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) + >>> hashingTF.setParams(outputCol="freqs").transform(df).head().freqs + SparseVector(10, {7: 1.0, 8: 1.0, 9: 1.0}) >>> params = {hashingTF.numFeatures: 5, hashingTF.outputCol: "vector"} - >>> print hashingTF.transform(df, params).head().vector - (5,[2,3,4],[1.0,1.0,1.0]) + >>> hashingTF.transform(df, params).head().vector + SparseVector(5, {2: 1.0, 3: 1.0, 4: 1.0}) """ _java_class = "org.apache.spark.ml.feature.HashingTF" diff --git a/python/pyspark/ml/param/__init__.py b/python/pyspark/ml/param/__init__.py index 5c62620562a84..9fccb65675185 100644 --- a/python/pyspark/ml/param/__init__.py +++ b/python/pyspark/ml/param/__init__.py @@ -63,8 +63,8 @@ def params(self): uses :py:func:`dir` to get all attributes of type :py:class:`Param`. """ - return filter(lambda attr: isinstance(attr, Param), - [getattr(self, x) for x in dir(self) if x != "params"]) + return list(filter(lambda attr: isinstance(attr, Param), + [getattr(self, x) for x in dir(self) if x != "params"])) def _explain(self, param): """ @@ -185,7 +185,7 @@ def _set(self, **kwargs): """ Sets user-supplied params. """ - for param, value in kwargs.iteritems(): + for param, value in kwargs.items(): self.paramMap[getattr(self, param)] = value return self @@ -193,6 +193,6 @@ def _setDefault(self, **kwargs): """ Sets default params. """ - for param, value in kwargs.iteritems(): + for param, value in kwargs.items(): self.defaultParamMap[getattr(self, param)] = value return self diff --git a/python/pyspark/ml/param/_shared_params_code_gen.py b/python/pyspark/ml/param/_shared_params_code_gen.py index 55f422497672f..6a3192465d66d 100644 --- a/python/pyspark/ml/param/_shared_params_code_gen.py +++ b/python/pyspark/ml/param/_shared_params_code_gen.py @@ -15,6 +15,8 @@ # limitations under the License. # +from __future__ import print_function + header = """# # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -82,9 +84,9 @@ def get$Name(self): .replace("$defaultValueStr", str(defaultValueStr)) if __name__ == "__main__": - print header - print "\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n" - print "from pyspark.ml.param import Param, Params\n\n" + print(header) + print("\n# DO NOT MODIFY THIS FILE! It was generated by _shared_params_code_gen.py.\n") + print("from pyspark.ml.param import Param, Params\n\n") shared = [ ("maxIter", "max number of iterations", None), ("regParam", "regularization constant", None), @@ -97,4 +99,4 @@ def get$Name(self): code = [] for name, doc, defaultValueStr in shared: code.append(_gen_param_code(name, doc, defaultValueStr)) - print "\n\n\n".join(code) + print("\n\n\n".join(code)) diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index f2ef573fe9f6f..07507b2ad0d05 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -18,6 +18,7 @@ """ Python bindings for MLlib. """ +from __future__ import absolute_import # MLlib currently needs NumPy 1.4+, so complain if lower @@ -29,7 +30,9 @@ 'recommendation', 'regression', 'stat', 'tree', 'util'] import sys -import rand as random -random.__name__ = 'random' -random.RandomRDDs.__module__ = __name__ + '.random' -sys.modules[__name__ + '.random'] = random +from . import rand as random +modname = __name__ + '.random' +random.__name__ = modname +random.RandomRDDs.__module__ = modname +sys.modules[modname] = random +del modname, sys diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 2466e8ac43458..eda0b60f8b1e7 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -510,9 +510,10 @@ def save(self, sc, path): def load(cls, sc, path): java_model = sc._jvm.org.apache.spark.mllib.classification.NaiveBayesModel.load( sc._jsc.sc(), path) - py_labels = _java2py(sc, java_model.labels()) - py_pi = _java2py(sc, java_model.pi()) - py_theta = _java2py(sc, java_model.theta()) + # Can not unpickle array.array from Pyrolite in Python3 with "bytes" + py_labels = _java2py(sc, java_model.labels(), "latin1") + py_pi = _java2py(sc, java_model.pi(), "latin1") + py_theta = _java2py(sc, java_model.theta(), "latin1") return NaiveBayesModel(py_labels, py_pi, numpy.array(py_theta)) diff --git a/python/pyspark/mllib/clustering.py b/python/pyspark/mllib/clustering.py index 464f49aeee3cd..abbb7cf60eece 100644 --- a/python/pyspark/mllib/clustering.py +++ b/python/pyspark/mllib/clustering.py @@ -15,6 +15,12 @@ # limitations under the License. # +import sys +import array as pyarray + +if sys.version > '3': + xrange = range + from numpy import array from pyspark import RDD @@ -55,8 +61,8 @@ class KMeansModel(Saveable, Loader): True >>> model.predict(sparse_data[2]) == model.predict(sparse_data[3]) True - >>> type(model.clusterCenters) - + >>> isinstance(model.clusterCenters, list) + True >>> import os, tempfile >>> path = tempfile.mkdtemp() >>> model.save(sc, path) @@ -90,7 +96,7 @@ def predict(self, x): return best def save(self, sc, path): - java_centers = _py2java(sc, map(_convert_to_vector, self.centers)) + java_centers = _py2java(sc, [_convert_to_vector(c) for c in self.centers]) java_model = sc._jvm.org.apache.spark.mllib.clustering.KMeansModel(java_centers) java_model.save(sc._jsc.sc(), path) @@ -133,7 +139,7 @@ class GaussianMixtureModel(object): ... 5.7048, 4.6567, 5.5026, ... 4.5605, 5.2043, 6.2734]).reshape(5, 3)) >>> model = GaussianMixture.train(clusterdata_2, 2, convergenceTol=0.0001, - ... maxIterations=150, seed=10) + ... maxIterations=150, seed=10) >>> labels = model.predict(clusterdata_2).collect() >>> labels[0]==labels[1]==labels[2] True @@ -168,8 +174,8 @@ def predictSoft(self, x): if isinstance(x, RDD): means, sigmas = zip(*[(g.mu, g.sigma) for g in self.gaussians]) membership_matrix = callMLlibFunc("predictSoftGMM", x.map(_convert_to_vector), - self.weights, means, sigmas) - return membership_matrix + _convert_to_vector(self.weights), means, sigmas) + return membership_matrix.map(lambda x: pyarray.array('d', x)) class GaussianMixture(object): diff --git a/python/pyspark/mllib/common.py b/python/pyspark/mllib/common.py index a539d2f2846f9..ba6058978880a 100644 --- a/python/pyspark/mllib/common.py +++ b/python/pyspark/mllib/common.py @@ -15,6 +15,11 @@ # limitations under the License. # +import sys +if sys.version >= '3': + long = int + unicode = str + import py4j.protocol from py4j.protocol import Py4JJavaError from py4j.java_gateway import JavaObject @@ -36,7 +41,7 @@ def _new_smart_decode(obj): if isinstance(obj, float): - s = unicode(obj) + s = str(obj) return _float_str_mapping.get(s, s) return _old_smart_decode(obj) @@ -74,15 +79,15 @@ def _py2java(sc, obj): obj = ListConverter().convert([_py2java(sc, x) for x in obj], sc._gateway._gateway_client) elif isinstance(obj, JavaObject): pass - elif isinstance(obj, (int, long, float, bool, basestring)): + elif isinstance(obj, (int, long, float, bool, bytes, unicode)): pass else: - bytes = bytearray(PickleSerializer().dumps(obj)) - obj = sc._jvm.SerDe.loads(bytes) + data = bytearray(PickleSerializer().dumps(obj)) + obj = sc._jvm.SerDe.loads(data) return obj -def _java2py(sc, r): +def _java2py(sc, r, encoding="bytes"): if isinstance(r, JavaObject): clsName = r.getClass().getSimpleName() # convert RDD into JavaRDD @@ -102,8 +107,8 @@ def _java2py(sc, r): except Py4JJavaError: pass # not pickable - if isinstance(r, bytearray): - r = PickleSerializer().loads(str(r)) + if isinstance(r, (bytearray, bytes)): + r = PickleSerializer().loads(bytes(r), encoding=encoding) return r diff --git a/python/pyspark/mllib/feature.py b/python/pyspark/mllib/feature.py index 8be819aceec24..1140539a24e95 100644 --- a/python/pyspark/mllib/feature.py +++ b/python/pyspark/mllib/feature.py @@ -23,12 +23,17 @@ import sys import warnings import random +import binascii +if sys.version >= '3': + basestring = str + unicode = str from py4j.protocol import Py4JJavaError -from pyspark import RDD, SparkContext +from pyspark import SparkContext +from pyspark.rdd import RDD, ignore_unicode_prefix from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper -from pyspark.mllib.linalg import Vectors, Vector, _convert_to_vector +from pyspark.mllib.linalg import Vectors, _convert_to_vector __all__ = ['Normalizer', 'StandardScalerModel', 'StandardScaler', 'HashingTF', 'IDFModel', 'IDF', 'Word2Vec', 'Word2VecModel'] @@ -206,7 +211,7 @@ class HashingTF(object): >>> htf = HashingTF(100) >>> doc = "a a b b c d".split(" ") >>> htf.transform(doc) - SparseVector(100, {1: 1.0, 14: 1.0, 31: 2.0, 44: 2.0}) + SparseVector(100, {...}) """ def __init__(self, numFeatures=1 << 20): """ @@ -360,6 +365,7 @@ def getVectors(self): return self.call("getVectors") +@ignore_unicode_prefix class Word2Vec(object): """ Word2Vec creates vector representation of words in a text corpus. @@ -382,7 +388,7 @@ class Word2Vec(object): >>> sentence = "a b " * 100 + "a c " * 10 >>> localDoc = [sentence, sentence] >>> doc = sc.parallelize(localDoc).map(lambda line: line.split(" ")) - >>> model = Word2Vec().setVectorSize(10).setSeed(42L).fit(doc) + >>> model = Word2Vec().setVectorSize(10).setSeed(42).fit(doc) >>> syms = model.findSynonyms("a", 2) >>> [s[0] for s in syms] @@ -400,7 +406,7 @@ def __init__(self): self.learningRate = 0.025 self.numPartitions = 1 self.numIterations = 1 - self.seed = random.randint(0, sys.maxint) + self.seed = random.randint(0, sys.maxsize) self.minCount = 5 def setVectorSize(self, vectorSize): @@ -459,7 +465,7 @@ def fit(self, data): raise TypeError("data should be an RDD of list of string") jmodel = callMLlibFunc("trainWord2Vec", data, int(self.vectorSize), float(self.learningRate), int(self.numPartitions), - int(self.numIterations), long(self.seed), + int(self.numIterations), int(self.seed), int(self.minCount)) return Word2VecModel(jmodel) diff --git a/python/pyspark/mllib/fpm.py b/python/pyspark/mllib/fpm.py index 3aa6d79d7093c..628ccc01cf3cc 100644 --- a/python/pyspark/mllib/fpm.py +++ b/python/pyspark/mllib/fpm.py @@ -16,12 +16,14 @@ # from pyspark import SparkContext +from pyspark.rdd import ignore_unicode_prefix from pyspark.mllib.common import JavaModelWrapper, callMLlibFunc, inherit_doc __all__ = ['FPGrowth', 'FPGrowthModel'] @inherit_doc +@ignore_unicode_prefix class FPGrowthModel(JavaModelWrapper): """ diff --git a/python/pyspark/mllib/linalg.py b/python/pyspark/mllib/linalg.py index a80320c52d1d0..38b3aa3ad460e 100644 --- a/python/pyspark/mllib/linalg.py +++ b/python/pyspark/mllib/linalg.py @@ -25,7 +25,13 @@ import sys import array -import copy_reg + +if sys.version >= '3': + basestring = str + xrange = range + import copyreg as copy_reg +else: + import copy_reg import numpy as np @@ -57,7 +63,7 @@ def fast_pickle_array(ar): def _convert_to_vector(l): if isinstance(l, Vector): return l - elif type(l) in (array.array, np.array, np.ndarray, list, tuple): + elif type(l) in (array.array, np.array, np.ndarray, list, tuple, xrange): return DenseVector(l) elif _have_scipy and scipy.sparse.issparse(l): assert l.shape[1] == 1, "Expected column vector" @@ -88,7 +94,7 @@ def _vector_size(v): """ if isinstance(v, Vector): return len(v) - elif type(v) in (array.array, list, tuple): + elif type(v) in (array.array, list, tuple, xrange): return len(v) elif type(v) == np.ndarray: if v.ndim == 1 or (v.ndim == 2 and v.shape[1] == 1): @@ -193,7 +199,7 @@ class DenseVector(Vector): DenseVector([1.0, 0.0]) """ def __init__(self, ar): - if isinstance(ar, basestring): + if isinstance(ar, bytes): ar = np.frombuffer(ar, dtype=np.float64) elif not isinstance(ar, np.ndarray): ar = np.array(ar, dtype=np.float64) @@ -321,11 +327,13 @@ def func(self, other): __sub__ = _delegate("__sub__") __mul__ = _delegate("__mul__") __div__ = _delegate("__div__") + __truediv__ = _delegate("__truediv__") __mod__ = _delegate("__mod__") __radd__ = _delegate("__radd__") __rsub__ = _delegate("__rsub__") __rmul__ = _delegate("__rmul__") __rdiv__ = _delegate("__rdiv__") + __rtruediv__ = _delegate("__rtruediv__") __rmod__ = _delegate("__rmod__") @@ -344,12 +352,12 @@ def __init__(self, size, *args): :param args: Non-zero entries, as a dictionary, list of tupes, or two sorted lists containing indices and values. - >>> print SparseVector(4, {1: 1.0, 3: 5.5}) - (4,[1,3],[1.0,5.5]) - >>> print SparseVector(4, [(1, 1.0), (3, 5.5)]) - (4,[1,3],[1.0,5.5]) - >>> print SparseVector(4, [1, 3], [1.0, 5.5]) - (4,[1,3],[1.0,5.5]) + >>> SparseVector(4, {1: 1.0, 3: 5.5}) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> SparseVector(4, [(1, 1.0), (3, 5.5)]) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> SparseVector(4, [1, 3], [1.0, 5.5]) + SparseVector(4, {1: 1.0, 3: 5.5}) """ self.size = int(size) assert 1 <= len(args) <= 2, "must pass either 2 or 3 arguments" @@ -361,8 +369,8 @@ def __init__(self, size, *args): 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: - if isinstance(args[0], basestring): - assert isinstance(args[1], str), "values should be string too" + if isinstance(args[0], bytes): + assert isinstance(args[1], bytes), "values should be string too" if args[0]: self.indices = np.frombuffer(args[0], np.int32) self.values = np.frombuffer(args[1], np.float64) @@ -591,12 +599,12 @@ def sparse(size, *args): :param args: Non-zero entries, as a dictionary, list of tupes, or two sorted lists containing indices and values. - >>> print Vectors.sparse(4, {1: 1.0, 3: 5.5}) - (4,[1,3],[1.0,5.5]) - >>> print Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) - (4,[1,3],[1.0,5.5]) - >>> print Vectors.sparse(4, [1, 3], [1.0, 5.5]) - (4,[1,3],[1.0,5.5]) + >>> Vectors.sparse(4, {1: 1.0, 3: 5.5}) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> Vectors.sparse(4, [(1, 1.0), (3, 5.5)]) + SparseVector(4, {1: 1.0, 3: 5.5}) + >>> Vectors.sparse(4, [1, 3], [1.0, 5.5]) + SparseVector(4, {1: 1.0, 3: 5.5}) """ return SparseVector(size, *args) @@ -645,7 +653,7 @@ def _convert_to_array(array_like, dtype): """ Convert Matrix attributes which are array-like or buffer to array. """ - if isinstance(array_like, basestring): + if isinstance(array_like, bytes): return np.frombuffer(array_like, dtype=dtype) return np.asarray(array_like, dtype=dtype) @@ -677,7 +685,7 @@ def toArray(self): def toSparse(self): """Convert to SparseMatrix""" indices = np.nonzero(self.values)[0] - colCounts = np.bincount(indices / self.numRows) + colCounts = np.bincount(indices // self.numRows) colPtrs = np.cumsum(np.hstack( (0, colCounts, np.zeros(self.numCols - colCounts.size)))) values = self.values[indices] diff --git a/python/pyspark/mllib/rand.py b/python/pyspark/mllib/rand.py index 20ee9d78bf5b0..06fbc0eb6aef0 100644 --- a/python/pyspark/mllib/rand.py +++ b/python/pyspark/mllib/rand.py @@ -88,10 +88,10 @@ def normalRDD(sc, size, numPartitions=None, seed=None): :param seed: Random seed (default: a random long integer). :return: RDD of float comprised of i.i.d. samples ~ N(0.0, 1.0). - >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1L) + >>> x = RandomRDDs.normalRDD(sc, 1000, seed=1) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - 0.0) < 0.1 True >>> abs(stats.stdev() - 1.0) < 0.1 @@ -118,10 +118,10 @@ def logNormalRDD(sc, mean, std, size, numPartitions=None, seed=None): >>> std = 1.0 >>> expMean = exp(mean + 0.5 * std * std) >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std)) - >>> x = RandomRDDs.logNormalRDD(sc, mean, std, 1000, seed=2L) + >>> x = RandomRDDs.logNormalRDD(sc, mean, std, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - expMean) < 0.5 True >>> from math import sqrt @@ -145,10 +145,10 @@ def poissonRDD(sc, mean, size, numPartitions=None, seed=None): :return: RDD of float comprised of i.i.d. samples ~ Pois(mean). >>> mean = 100.0 - >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=2L) + >>> x = RandomRDDs.poissonRDD(sc, mean, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - mean) < 0.5 True >>> from math import sqrt @@ -171,10 +171,10 @@ def exponentialRDD(sc, mean, size, numPartitions=None, seed=None): :return: RDD of float comprised of i.i.d. samples ~ Exp(mean). >>> mean = 2.0 - >>> x = RandomRDDs.exponentialRDD(sc, mean, 1000, seed=2L) + >>> x = RandomRDDs.exponentialRDD(sc, mean, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - mean) < 0.5 True >>> from math import sqrt @@ -202,10 +202,10 @@ def gammaRDD(sc, shape, scale, size, numPartitions=None, seed=None): >>> scale = 2.0 >>> expMean = shape * scale >>> expStd = sqrt(shape * scale * scale) - >>> x = RandomRDDs.gammaRDD(sc, shape, scale, 1000, seed=2L) + >>> x = RandomRDDs.gammaRDD(sc, shape, scale, 1000, seed=2) >>> stats = x.stats() >>> stats.count() - 1000L + 1000 >>> abs(stats.mean() - expMean) < 0.5 True >>> abs(stats.stdev() - expStd) < 0.5 @@ -254,7 +254,7 @@ def normalVectorRDD(sc, numRows, numCols, numPartitions=None, seed=None): :return: RDD of Vector with vectors containing i.i.d. samples ~ `N(0.0, 1.0)`. >>> import numpy as np - >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1L).collect()) + >>> mat = np.matrix(RandomRDDs.normalVectorRDD(sc, 100, 100, seed=1).collect()) >>> mat.shape (100, 100) >>> abs(mat.mean() - 0.0) < 0.1 @@ -286,8 +286,8 @@ def logNormalVectorRDD(sc, mean, std, numRows, numCols, numPartitions=None, seed >>> std = 1.0 >>> expMean = exp(mean + 0.5 * std * std) >>> expStd = sqrt((exp(std * std) - 1.0) * exp(2.0 * mean + std * std)) - >>> mat = np.matrix(RandomRDDs.logNormalVectorRDD(sc, mean, std, \ - 100, 100, seed=1L).collect()) + >>> m = RandomRDDs.logNormalVectorRDD(sc, mean, std, 100, 100, seed=1).collect() + >>> mat = np.matrix(m) >>> mat.shape (100, 100) >>> abs(mat.mean() - expMean) < 0.1 @@ -315,7 +315,7 @@ def poissonVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=None): >>> import numpy as np >>> mean = 100.0 - >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1L) + >>> rdd = RandomRDDs.poissonVectorRDD(sc, mean, 100, 100, seed=1) >>> mat = np.mat(rdd.collect()) >>> mat.shape (100, 100) @@ -345,7 +345,7 @@ def exponentialVectorRDD(sc, mean, numRows, numCols, numPartitions=None, seed=No >>> import numpy as np >>> mean = 0.5 - >>> rdd = RandomRDDs.exponentialVectorRDD(sc, mean, 100, 100, seed=1L) + >>> rdd = RandomRDDs.exponentialVectorRDD(sc, mean, 100, 100, seed=1) >>> mat = np.mat(rdd.collect()) >>> mat.shape (100, 100) @@ -380,8 +380,7 @@ def gammaVectorRDD(sc, shape, scale, numRows, numCols, numPartitions=None, seed= >>> scale = 2.0 >>> expMean = shape * scale >>> expStd = sqrt(shape * scale * scale) - >>> mat = np.matrix(RandomRDDs.gammaVectorRDD(sc, shape, scale, \ - 100, 100, seed=1L).collect()) + >>> mat = np.matrix(RandomRDDs.gammaVectorRDD(sc, shape, scale, 100, 100, seed=1).collect()) >>> mat.shape (100, 100) >>> abs(mat.mean() - expMean) < 0.1 diff --git a/python/pyspark/mllib/recommendation.py b/python/pyspark/mllib/recommendation.py index c5c4c13dae105..80e0a356bb78a 100644 --- a/python/pyspark/mllib/recommendation.py +++ b/python/pyspark/mllib/recommendation.py @@ -15,6 +15,7 @@ # limitations under the License. # +import array from collections import namedtuple from pyspark import SparkContext @@ -104,14 +105,14 @@ def predictAll(self, user_product): assert isinstance(user_product, RDD), "user_product should be RDD of (user, product)" first = user_product.first() assert len(first) == 2, "user_product should be RDD of (user, product)" - user_product = user_product.map(lambda (u, p): (int(u), int(p))) + user_product = user_product.map(lambda u_p: (int(u_p[0]), int(u_p[1]))) return self.call("predict", user_product) def userFeatures(self): - return self.call("getUserFeatures") + return self.call("getUserFeatures").mapValues(lambda v: array.array('d', v)) def productFeatures(self): - return self.call("getProductFeatures") + return self.call("getProductFeatures").mapValues(lambda v: array.array('d', v)) @classmethod def load(cls, sc, path): diff --git a/python/pyspark/mllib/stat/_statistics.py b/python/pyspark/mllib/stat/_statistics.py index 1d83e9d483f8e..b475be4b4d953 100644 --- a/python/pyspark/mllib/stat/_statistics.py +++ b/python/pyspark/mllib/stat/_statistics.py @@ -15,7 +15,7 @@ # limitations under the License. # -from pyspark import RDD +from pyspark.rdd import RDD, ignore_unicode_prefix from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper from pyspark.mllib.linalg import Matrix, _convert_to_vector from pyspark.mllib.regression import LabeledPoint @@ -38,7 +38,7 @@ def variance(self): return self.call("variance").toArray() def count(self): - return self.call("count") + return int(self.call("count")) def numNonzeros(self): return self.call("numNonzeros").toArray() @@ -78,7 +78,7 @@ def colStats(rdd): >>> cStats.variance() array([ 4., 13., 0., 25.]) >>> cStats.count() - 3L + 3 >>> cStats.numNonzeros() array([ 3., 2., 0., 3.]) >>> cStats.max() @@ -124,20 +124,20 @@ def corr(x, y=None, method=None): >>> rdd = sc.parallelize([Vectors.dense([1, 0, 0, -2]), Vectors.dense([4, 5, 0, 3]), ... Vectors.dense([6, 7, 0, 8]), Vectors.dense([9, 0, 0, 1])]) >>> pearsonCorr = Statistics.corr(rdd) - >>> print str(pearsonCorr).replace('nan', 'NaN') + >>> print(str(pearsonCorr).replace('nan', 'NaN')) [[ 1. 0.05564149 NaN 0.40047142] [ 0.05564149 1. NaN 0.91359586] [ NaN NaN 1. NaN] [ 0.40047142 0.91359586 NaN 1. ]] >>> spearmanCorr = Statistics.corr(rdd, method="spearman") - >>> print str(spearmanCorr).replace('nan', 'NaN') + >>> print(str(spearmanCorr).replace('nan', 'NaN')) [[ 1. 0.10540926 NaN 0.4 ] [ 0.10540926 1. NaN 0.9486833 ] [ NaN NaN 1. NaN] [ 0.4 0.9486833 NaN 1. ]] >>> try: ... Statistics.corr(rdd, "spearman") - ... print "Method name as second argument without 'method=' shouldn't be allowed." + ... print("Method name as second argument without 'method=' shouldn't be allowed.") ... except TypeError: ... pass """ @@ -153,6 +153,7 @@ def corr(x, y=None, method=None): return callMLlibFunc("corr", x.map(float), y.map(float), method) @staticmethod + @ignore_unicode_prefix def chiSqTest(observed, expected=None): """ .. note:: Experimental @@ -188,11 +189,11 @@ def chiSqTest(observed, expected=None): >>> from pyspark.mllib.linalg import Vectors, Matrices >>> observed = Vectors.dense([4, 6, 5]) >>> pearson = Statistics.chiSqTest(observed) - >>> print pearson.statistic + >>> print(pearson.statistic) 0.4 >>> pearson.degreesOfFreedom 2 - >>> print round(pearson.pValue, 4) + >>> print(round(pearson.pValue, 4)) 0.8187 >>> pearson.method u'pearson' @@ -202,12 +203,12 @@ def chiSqTest(observed, expected=None): >>> observed = Vectors.dense([21, 38, 43, 80]) >>> expected = Vectors.dense([3, 5, 7, 20]) >>> pearson = Statistics.chiSqTest(observed, expected) - >>> print round(pearson.pValue, 4) + >>> print(round(pearson.pValue, 4)) 0.0027 >>> data = [40.0, 24.0, 29.0, 56.0, 32.0, 42.0, 31.0, 10.0, 0.0, 30.0, 15.0, 12.0] >>> chi = Statistics.chiSqTest(Matrices.dense(3, 4, data)) - >>> print round(chi.statistic, 4) + >>> print(round(chi.statistic, 4)) 21.9958 >>> data = [LabeledPoint(0.0, Vectors.dense([0.5, 10.0])), @@ -218,9 +219,9 @@ def chiSqTest(observed, expected=None): ... LabeledPoint(1.0, Vectors.dense([3.5, 40.0])),] >>> rdd = sc.parallelize(data, 4) >>> chi = Statistics.chiSqTest(rdd) - >>> print chi[0].statistic + >>> print(chi[0].statistic) 0.75 - >>> print chi[1].statistic + >>> print(chi[1].statistic) 1.5 """ if isinstance(observed, RDD): diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index 8eaddcf8b9b5e..c6ed5acd1770e 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -72,11 +72,11 @@ 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))) + nv = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvec))) self.assertEqual(v, nv) vs = [v] * 100 jvecs = self.sc._jvm.SerDe.loads(bytearray(ser.dumps(vs))) - nvs = ser.loads(str(self.sc._jvm.SerDe.dumps(jvecs))) + nvs = ser.loads(bytes(self.sc._jvm.SerDe.dumps(jvecs))) self.assertEqual(vs, nvs) def test_serialize(self): @@ -412,11 +412,11 @@ def test_col_norms(self): self.assertEqual(10, len(summary.normL1())) self.assertEqual(10, len(summary.normL2())) - data2 = self.sc.parallelize(xrange(10)).map(lambda x: Vectors.dense(x)) + data2 = self.sc.parallelize(range(10)).map(lambda x: Vectors.dense(x)) summary2 = Statistics.colStats(data2) self.assertEqual(array([45.0]), summary2.normL1()) import math - expectedNormL2 = math.sqrt(sum(map(lambda x: x*x, xrange(10)))) + expectedNormL2 = math.sqrt(sum(map(lambda x: x*x, range(10)))) self.assertTrue(math.fabs(summary2.normL2()[0] - expectedNormL2) < 1e-14) @@ -438,11 +438,11 @@ def test_serialization(self): def test_infer_schema(self): sqlCtx = SQLContext(self.sc) rdd = self.sc.parallelize([LabeledPoint(1.0, self.dv1), LabeledPoint(0.0, self.sv1)]) - srdd = sqlCtx.inferSchema(rdd) - schema = srdd.schema + df = rdd.toDF() + schema = df.schema field = [f for f in schema.fields if f.name == "features"][0] self.assertEqual(field.dataType, self.udt) - vectors = srdd.map(lambda p: p.features).collect() + vectors = df.map(lambda p: p.features).collect() self.assertEqual(len(vectors), 2) for v in vectors: if isinstance(v, SparseVector): @@ -695,7 +695,7 @@ def test_right_number_of_results(self): class SerDeTest(PySparkTestCase): def test_to_java_object_rdd(self): # SPARK-6660 - data = RandomRDDs.uniformRDD(self.sc, 10, 5, seed=0L) + data = RandomRDDs.uniformRDD(self.sc, 10, 5, seed=0) self.assertEqual(_to_java_object_rdd(data).count(), 10) @@ -771,7 +771,7 @@ def test_model_transform(self): if __name__ == "__main__": if not _have_scipy: - print "NOTE: Skipping SciPy tests as it does not seem to be installed" + print("NOTE: Skipping SciPy tests as it does not seem to be installed") unittest.main() if not _have_scipy: - print "NOTE: SciPy tests were skipped as it does not seem to be installed" + print("NOTE: SciPy tests were skipped as it does not seem to be installed") diff --git a/python/pyspark/mllib/tree.py b/python/pyspark/mllib/tree.py index a7a4d2aaf855b..0fe6e4fabe43a 100644 --- a/python/pyspark/mllib/tree.py +++ b/python/pyspark/mllib/tree.py @@ -163,14 +163,16 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, ... LabeledPoint(1.0, [3.0]) ... ] >>> model = DecisionTree.trainClassifier(sc.parallelize(data), 2, {}) - >>> print model, # it already has newline + >>> print(model) DecisionTreeModel classifier of depth 1 with 3 nodes - >>> print model.toDebugString(), # it already has newline + + >>> print(model.toDebugString()) DecisionTreeModel classifier of depth 1 with 3 nodes If (feature 0 <= 0.0) Predict: 0.0 Else (feature 0 > 0.0) Predict: 1.0 + >>> model.predict(array([1.0])) 1.0 >>> model.predict(array([0.0])) @@ -318,9 +320,10 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, 3 >>> model.totalNumNodes() 7 - >>> print model, + >>> print(model) TreeEnsembleModel classifier with 3 trees - >>> print model.toDebugString(), + + >>> print(model.toDebugString()) TreeEnsembleModel classifier with 3 trees Tree 0: @@ -335,6 +338,7 @@ def trainClassifier(cls, data, numClasses, categoricalFeaturesInfo, numTrees, Predict: 0.0 Else (feature 0 > 1.0) Predict: 1.0 + >>> model.predict([2.0]) 1.0 >>> model.predict([0.0]) @@ -483,8 +487,9 @@ def trainClassifier(cls, data, categoricalFeaturesInfo, 100 >>> model.totalNumNodes() 300 - >>> print model, # it already has newline + >>> print(model) # it already has newline TreeEnsembleModel classifier with 100 trees + >>> model.predict([2.0]) 1.0 >>> model.predict([0.0]) diff --git a/python/pyspark/mllib/util.py b/python/pyspark/mllib/util.py index c5c3468eb95e9..16a90db146ef0 100644 --- a/python/pyspark/mllib/util.py +++ b/python/pyspark/mllib/util.py @@ -15,10 +15,14 @@ # limitations under the License. # +import sys import numpy as np import warnings -from pyspark.mllib.common import callMLlibFunc, JavaModelWrapper, inherit_doc +if sys.version > '3': + xrange = range + +from pyspark.mllib.common import callMLlibFunc, inherit_doc from pyspark.mllib.linalg import Vectors, SparseVector, _convert_to_vector @@ -94,22 +98,16 @@ def loadLibSVMFile(sc, path, numFeatures=-1, minPartitions=None, multiclass=None >>> from pyspark.mllib.util import MLUtils >>> from pyspark.mllib.regression import LabeledPoint >>> tempFile = NamedTemporaryFile(delete=True) - >>> tempFile.write("+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") + >>> _ = tempFile.write(b"+1 1:1.0 3:2.0 5:3.0\\n-1\\n-1 2:4.0 4:5.0 6:6.0") >>> tempFile.flush() >>> examples = MLUtils.loadLibSVMFile(sc, tempFile.name).collect() >>> tempFile.close() - >>> type(examples[0]) == LabeledPoint - True - >>> print examples[0] - (1.0,(6,[0,2,4],[1.0,2.0,3.0])) - >>> type(examples[1]) == LabeledPoint - True - >>> print examples[1] - (-1.0,(6,[],[])) - >>> type(examples[2]) == LabeledPoint - True - >>> print examples[2] - (-1.0,(6,[1,3,5],[4.0,5.0,6.0])) + >>> examples[0] + LabeledPoint(1.0, (6,[0,2,4],[1.0,2.0,3.0])) + >>> examples[1] + LabeledPoint(-1.0, (6,[],[])) + >>> examples[2] + LabeledPoint(-1.0, (6,[1,3,5],[4.0,5.0,6.0])) """ from pyspark.mllib.regression import LabeledPoint if multiclass is not None: diff --git a/python/pyspark/profiler.py b/python/pyspark/profiler.py index 4408996db0790..d18daaabfcb3c 100644 --- a/python/pyspark/profiler.py +++ b/python/pyspark/profiler.py @@ -84,11 +84,11 @@ class Profiler(object): >>> from pyspark import BasicProfiler >>> class MyCustomProfiler(BasicProfiler): ... def show(self, id): - ... print "My custom profiles for RDD:%s" % id + ... print("My custom profiles for RDD:%s" % id) ... >>> conf = SparkConf().set("spark.python.profile", "true") >>> sc = SparkContext('local', 'test', conf=conf, profiler_cls=MyCustomProfiler) - >>> sc.parallelize(list(range(1000))).map(lambda x: 2 * x).take(10) + >>> sc.parallelize(range(1000)).map(lambda x: 2 * x).take(10) [0, 2, 4, 6, 8, 10, 12, 14, 16, 18] >>> sc.show_profiles() My custom profiles for RDD:1 @@ -111,9 +111,9 @@ def show(self, id): """ Print the profile stats to stdout, id is the RDD id """ stats = self.stats() if stats: - print "=" * 60 - print "Profile of RDD" % id - print "=" * 60 + print("=" * 60) + print("Profile of RDD" % id) + print("=" * 60) stats.sort_stats("time", "cumulative").print_stats() def dump(self, id, path): diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 93e658eded9e2..d9cdbb666f92a 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -16,21 +16,29 @@ # import copy -from collections import defaultdict -from itertools import chain, ifilter, imap -import operator import sys +import os +import re +import operator import shlex -from subprocess import Popen, PIPE -from tempfile import NamedTemporaryFile -from threading import Thread import warnings import heapq import bisect import random import socket +from subprocess import Popen, PIPE +from tempfile import NamedTemporaryFile +from threading import Thread +from collections import defaultdict +from itertools import chain +from functools import reduce from math import sqrt, log, isinf, isnan, pow, ceil +if sys.version > '3': + basestring = unicode = str +else: + from itertools import imap as map, ifilter as filter + from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, \ PickleSerializer, pack_long, AutoBatchedSerializer @@ -50,20 +58,21 @@ __all__ = ["RDD"] -# TODO: for Python 3.3+, PYTHONHASHSEED should be reset to disable randomized -# hash for string def portable_hash(x): """ - This function returns consistant hash code for builtin types, especially + This function returns consistent hash code for builtin types, especially for None and tuple with None. - The algrithm is similar to that one used by CPython 2.7 + The algorithm is similar to that one used by CPython 2.7 >>> portable_hash(None) 0 >>> portable_hash((None, 1)) & 0xffffffff 219750521 """ + if sys.version >= '3.3' and 'PYTHONHASHSEED' not in os.environ: + raise Exception("Randomness of hash of string should be disabled via PYTHONHASHSEED") + if x is None: return 0 if isinstance(x, tuple): @@ -71,7 +80,7 @@ def portable_hash(x): for i in x: h ^= portable_hash(i) h *= 1000003 - h &= sys.maxint + h &= sys.maxsize h ^= len(x) if h == -1: h = -2 @@ -123,6 +132,19 @@ def _load_from_socket(port, serializer): sock.close() +def ignore_unicode_prefix(f): + """ + Ignore the 'u' prefix of string in doc tests, to make it works + in both python 2 and 3 + """ + if sys.version >= '3': + # the representation of unicode string in Python 3 does not have prefix 'u', + # so remove the prefix 'u' for doc tests + literal_re = re.compile(r"(\W|^)[uU](['])", re.UNICODE) + f.__doc__ = literal_re.sub(r'\1\2', f.__doc__) + return f + + class Partitioner(object): def __init__(self, numPartitions, partitionFunc): self.numPartitions = numPartitions @@ -251,7 +273,7 @@ def map(self, f, preservesPartitioning=False): [('a', 1), ('b', 1), ('c', 1)] """ def func(_, iterator): - return imap(f, iterator) + return map(f, iterator) return self.mapPartitionsWithIndex(func, preservesPartitioning) def flatMap(self, f, preservesPartitioning=False): @@ -266,7 +288,7 @@ def flatMap(self, f, preservesPartitioning=False): [(2, 2), (2, 2), (3, 3), (3, 3), (4, 4), (4, 4)] """ def func(s, iterator): - return chain.from_iterable(imap(f, iterator)) + return chain.from_iterable(map(f, iterator)) return self.mapPartitionsWithIndex(func, preservesPartitioning) def mapPartitions(self, f, preservesPartitioning=False): @@ -329,7 +351,7 @@ def filter(self, f): [2, 4] """ def func(iterator): - return ifilter(f, iterator) + return filter(f, iterator) return self.mapPartitions(func, True) def distinct(self, numPartitions=None): @@ -341,7 +363,7 @@ def distinct(self, numPartitions=None): """ return self.map(lambda x: (x, None)) \ .reduceByKey(lambda x, _: x, numPartitions) \ - .map(lambda (x, _): x) + .map(lambda x: x[0]) def sample(self, withReplacement, fraction, seed=None): """ @@ -354,8 +376,8 @@ def sample(self, withReplacement, fraction, seed=None): :param seed: seed for the random number generator >>> rdd = sc.parallelize(range(100), 4) - >>> rdd.sample(False, 0.1, 81).count() - 10 + >>> 6 <= rdd.sample(False, 0.1, 81).count() <= 14 + True """ assert fraction >= 0.0, "Negative fraction value: %s" % fraction return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) @@ -368,12 +390,14 @@ def randomSplit(self, weights, seed=None): :param seed: random seed :return: split RDDs in a list - >>> rdd = sc.parallelize(range(5), 1) + >>> rdd = sc.parallelize(range(500), 1) >>> rdd1, rdd2 = rdd.randomSplit([2, 3], 17) - >>> rdd1.collect() - [1, 3] - >>> rdd2.collect() - [0, 2, 4] + >>> len(rdd1.collect() + rdd2.collect()) + 500 + >>> 150 < rdd1.count() < 250 + True + >>> 250 < rdd2.count() < 350 + True """ s = float(sum(weights)) cweights = [0.0] @@ -416,7 +440,7 @@ def takeSample(self, withReplacement, num, seed=None): rand.shuffle(samples) return samples - maxSampleSize = sys.maxint - int(numStDev * sqrt(sys.maxint)) + maxSampleSize = sys.maxsize - int(numStDev * sqrt(sys.maxsize)) if num > maxSampleSize: raise ValueError( "Sample size cannot be greater than %d." % maxSampleSize) @@ -430,7 +454,7 @@ def takeSample(self, withReplacement, num, seed=None): # See: scala/spark/RDD.scala while len(samples) < num: # TODO: add log warning for when more than one iteration was run - seed = rand.randint(0, sys.maxint) + seed = rand.randint(0, sys.maxsize) samples = self.sample(withReplacement, fraction, seed).collect() rand.shuffle(samples) @@ -507,7 +531,7 @@ def intersection(self, other): """ return self.map(lambda v: (v, None)) \ .cogroup(other.map(lambda v: (v, None))) \ - .filter(lambda (k, vs): all(vs)) \ + .filter(lambda k_vs: all(k_vs[1])) \ .keys() def _reserialize(self, serializer=None): @@ -549,7 +573,7 @@ def repartitionAndSortWithinPartitions(self, numPartitions=None, partitionFunc=p def sortPartition(iterator): sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted - return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) + return iter(sort(iterator, key=lambda k_v: keyfunc(k_v[0]), reverse=(not ascending))) return self.partitionBy(numPartitions, partitionFunc).mapPartitions(sortPartition, True) @@ -579,7 +603,7 @@ def sortByKey(self, ascending=True, numPartitions=None, keyfunc=lambda x: x): def sortPartition(iterator): sort = ExternalSorter(memory * 0.9, serializer).sorted if spill else sorted - return iter(sort(iterator, key=lambda (k, v): keyfunc(k), reverse=(not ascending))) + return iter(sort(iterator, key=lambda kv: keyfunc(kv[0]), reverse=(not ascending))) if numPartitions == 1: if self.getNumPartitions() > 1: @@ -594,12 +618,12 @@ def sortPartition(iterator): return self # empty RDD maxSampleSize = numPartitions * 20.0 # constant from Spark's RangePartitioner fraction = min(maxSampleSize / max(rddSize, 1), 1.0) - samples = self.sample(False, fraction, 1).map(lambda (k, v): k).collect() + samples = self.sample(False, fraction, 1).map(lambda kv: kv[0]).collect() samples = sorted(samples, key=keyfunc) # we have numPartitions many parts but one of the them has # an implicit boundary - bounds = [samples[len(samples) * (i + 1) / numPartitions] + bounds = [samples[int(len(samples) * (i + 1) / numPartitions)] for i in range(0, numPartitions - 1)] def rangePartitioner(k): @@ -662,12 +686,13 @@ def groupBy(self, f, numPartitions=None): """ return self.map(lambda x: (f(x), x)).groupByKey(numPartitions) + @ignore_unicode_prefix def pipe(self, command, env={}): """ Return an RDD created by piping elements to a forked external process. >>> sc.parallelize(['1', '2', '', '3']).pipe('cat').collect() - ['1', '2', '', '3'] + [u'1', u'2', u'', u'3'] """ def func(iterator): pipe = Popen( @@ -675,17 +700,18 @@ def func(iterator): def pipe_objs(out): for obj in iterator: - out.write(str(obj).rstrip('\n') + '\n') + s = str(obj).rstrip('\n') + '\n' + out.write(s.encode('utf-8')) out.close() Thread(target=pipe_objs, args=[pipe.stdin]).start() - return (x.rstrip('\n') for x in iter(pipe.stdout.readline, '')) + return (x.rstrip(b'\n').decode('utf-8') for x in iter(pipe.stdout.readline, b'')) return self.mapPartitions(func) def foreach(self, f): """ Applies a function to all elements of this RDD. - >>> def f(x): print x + >>> def f(x): print(x) >>> sc.parallelize([1, 2, 3, 4, 5]).foreach(f) """ def processPartition(iterator): @@ -700,7 +726,7 @@ def foreachPartition(self, f): >>> def f(iterator): ... for x in iterator: - ... print x + ... print(x) >>> sc.parallelize([1, 2, 3, 4, 5]).foreachPartition(f) """ def func(it): @@ -874,7 +900,7 @@ def aggregatePartition(iterator): # aggregation. while numPartitions > scale + numPartitions / scale: numPartitions /= scale - curNumPartitions = numPartitions + curNumPartitions = int(numPartitions) def mapPartition(i, iterator): for obj in iterator: @@ -984,7 +1010,7 @@ def histogram(self, buckets): (('a', 'b', 'c'), [2, 2]) """ - if isinstance(buckets, (int, long)): + if isinstance(buckets, int): if buckets < 1: raise ValueError("number of buckets must be >= 1") @@ -1020,6 +1046,7 @@ def minmax(a, b): raise ValueError("Can not generate buckets with infinite value") # keep them as integer if possible + inc = int(inc) if inc * buckets != maxv - minv: inc = (maxv - minv) * 1.0 / buckets @@ -1137,7 +1164,7 @@ def countPartition(iterator): yield counts def mergeMaps(m1, m2): - for k, v in m2.iteritems(): + for k, v in m2.items(): m1[k] += v return m1 return self.mapPartitions(countPartition).reduce(mergeMaps) @@ -1378,8 +1405,8 @@ def saveAsPickleFile(self, path, batchSize=10): >>> tmpFile = NamedTemporaryFile(delete=True) >>> tmpFile.close() >>> sc.parallelize([1, 2, 'spark', 'rdd']).saveAsPickleFile(tmpFile.name, 3) - >>> sorted(sc.pickleFile(tmpFile.name, 5).collect()) - [1, 2, 'rdd', 'spark'] + >>> sorted(sc.pickleFile(tmpFile.name, 5).map(str).collect()) + ['1', '2', 'rdd', 'spark'] """ if batchSize == 0: ser = AutoBatchedSerializer(PickleSerializer()) @@ -1387,6 +1414,7 @@ def saveAsPickleFile(self, path, batchSize=10): ser = BatchedSerializer(PickleSerializer(), batchSize) self._reserialize(ser)._jrdd.saveAsObjectFile(path) + @ignore_unicode_prefix def saveAsTextFile(self, path, compressionCodecClass=None): """ Save this RDD as a text file, using string representations of elements. @@ -1418,12 +1446,13 @@ def saveAsTextFile(self, path, compressionCodecClass=None): >>> codec = "org.apache.hadoop.io.compress.GzipCodec" >>> sc.parallelize(['foo', 'bar']).saveAsTextFile(tempFile3.name, codec) >>> from fileinput import input, hook_compressed - >>> ''.join(sorted(input(glob(tempFile3.name + "/part*.gz"), openhook=hook_compressed))) - 'bar\\nfoo\\n' + >>> result = sorted(input(glob(tempFile3.name + "/part*.gz"), openhook=hook_compressed)) + >>> b''.join(result).decode('utf-8') + u'bar\\nfoo\\n' """ def func(split, iterator): for x in iterator: - if not isinstance(x, basestring): + if not isinstance(x, (unicode, bytes)): x = unicode(x) if isinstance(x, unicode): x = x.encode("utf-8") @@ -1458,7 +1487,7 @@ def keys(self): >>> m.collect() [1, 3] """ - return self.map(lambda (k, v): k) + return self.map(lambda x: x[0]) def values(self): """ @@ -1468,7 +1497,7 @@ def values(self): >>> m.collect() [2, 4] """ - return self.map(lambda (k, v): v) + return self.map(lambda x: x[1]) def reduceByKey(self, func, numPartitions=None): """ @@ -1507,7 +1536,7 @@ def reducePartition(iterator): yield m def mergeMaps(m1, m2): - for k, v in m2.iteritems(): + for k, v in m2.items(): m1[k] = func(m1[k], v) if k in m1 else v return m1 return self.mapPartitions(reducePartition).reduce(mergeMaps) @@ -1604,8 +1633,8 @@ def partitionBy(self, numPartitions, partitionFunc=portable_hash): >>> pairs = sc.parallelize([1, 2, 3, 4, 2, 4, 1]).map(lambda x: (x, x)) >>> sets = pairs.partitionBy(2).glom().collect() - >>> set(sets[0]).intersection(set(sets[1])) - set([]) + >>> len(set(sets[0]).intersection(set(sets[1]))) + 0 """ if numPartitions is None: numPartitions = self._defaultReducePartitions() @@ -1637,22 +1666,22 @@ def add_shuffle_key(split, iterator): if (c % 1000 == 0 and get_used_memory() > limit or c > batch): n, size = len(buckets), 0 - for split in buckets.keys(): + for split in list(buckets.keys()): yield pack_long(split) d = outputSerializer.dumps(buckets[split]) del buckets[split] yield d size += len(d) - avg = (size / n) >> 20 + avg = int(size / n) >> 20 # let 1M < avg < 10M if avg < 1: batch *= 1.5 elif avg > 10: - batch = max(batch / 1.5, 1) + batch = max(int(batch / 1.5), 1) c = 0 - for split, items in buckets.iteritems(): + for split, items in buckets.items(): yield pack_long(split) yield outputSerializer.dumps(items) @@ -1707,7 +1736,7 @@ def combineLocally(iterator): merger = ExternalMerger(agg, memory * 0.9, serializer) \ if spill else InMemoryMerger(agg) merger.mergeValues(iterator) - return merger.iteritems() + return merger.items() locally_combined = self.mapPartitions(combineLocally, preservesPartitioning=True) shuffled = locally_combined.partitionBy(numPartitions) @@ -1716,7 +1745,7 @@ def _mergeCombiners(iterator): merger = ExternalMerger(agg, memory, serializer) \ if spill else InMemoryMerger(agg) merger.mergeCombiners(iterator) - return merger.iteritems() + return merger.items() return shuffled.mapPartitions(_mergeCombiners, preservesPartitioning=True) @@ -1745,7 +1774,7 @@ def foldByKey(self, zeroValue, func, numPartitions=None): >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) >>> from operator import add - >>> rdd.foldByKey(0, add).collect() + >>> sorted(rdd.foldByKey(0, add).collect()) [('a', 2), ('b', 1)] """ def createZero(): @@ -1769,10 +1798,10 @@ def groupByKey(self, numPartitions=None): sum or average) over each key, using reduceByKey or aggregateByKey will provide much better performance. - >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> sorted(x.groupByKey().mapValues(len).collect()) + >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) + >>> sorted(rdd.groupByKey().mapValues(len).collect()) [('a', 2), ('b', 1)] - >>> sorted(x.groupByKey().mapValues(list).collect()) + >>> sorted(rdd.groupByKey().mapValues(list).collect()) [('a', [1, 1]), ('b', [1])] """ def createCombiner(x): @@ -1795,7 +1824,7 @@ def combine(iterator): merger = ExternalMerger(agg, memory * 0.9, serializer) \ if spill else InMemoryMerger(agg) merger.mergeValues(iterator) - return merger.iteritems() + return merger.items() locally_combined = self.mapPartitions(combine, preservesPartitioning=True) shuffled = locally_combined.partitionBy(numPartitions) @@ -1804,7 +1833,7 @@ def groupByKey(it): merger = ExternalGroupBy(agg, memory, serializer)\ if spill else InMemoryMerger(agg) merger.mergeCombiners(it) - return merger.iteritems() + return merger.items() return shuffled.mapPartitions(groupByKey, True).mapValues(ResultIterable) @@ -1819,7 +1848,7 @@ def flatMapValues(self, f): >>> x.flatMapValues(f).collect() [('a', 'x'), ('a', 'y'), ('a', 'z'), ('b', 'p'), ('b', 'r')] """ - flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) + flat_map_fn = lambda kv: ((kv[0], x) for x in f(kv[1])) return self.flatMap(flat_map_fn, preservesPartitioning=True) def mapValues(self, f): @@ -1833,7 +1862,7 @@ def mapValues(self, f): >>> x.mapValues(f).collect() [('a', 3), ('b', 1)] """ - map_values_fn = lambda (k, v): (k, f(v)) + map_values_fn = lambda kv: (kv[0], f(kv[1])) return self.map(map_values_fn, preservesPartitioning=True) def groupWith(self, other, *others): @@ -1844,8 +1873,7 @@ def groupWith(self, other, *others): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) >>> z = sc.parallelize([("b", 42)]) - >>> map((lambda (x,y): (x, (list(y[0]), list(y[1]), list(y[2]), list(y[3])))), \ - sorted(list(w.groupWith(x, y, z).collect()))) + >>> [(x, tuple(map(list, y))) for x, y in sorted(list(w.groupWith(x, y, z).collect()))] [('a', ([5], [1], [2], [])), ('b', ([6], [4], [], [42]))] """ @@ -1860,7 +1888,7 @@ def cogroup(self, other, numPartitions=None): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) - >>> map((lambda (x,y): (x, (list(y[0]), list(y[1])))), sorted(list(x.cogroup(y).collect()))) + >>> [(x, tuple(map(list, y))) for x, y in sorted(list(x.cogroup(y).collect()))] [('a', ([1], [2])), ('b', ([4], []))] """ return python_cogroup((self, other), numPartitions) @@ -1896,8 +1924,9 @@ def subtractByKey(self, other, numPartitions=None): >>> sorted(x.subtractByKey(y).collect()) [('b', 4), ('b', 5)] """ - def filter_func((key, vals)): - return vals[0] and not vals[1] + def filter_func(pair): + key, (val1, val2) = pair + return val1 and not val2 return self.cogroup(other, numPartitions).filter(filter_func).flatMapValues(lambda x: x[0]) def subtract(self, other, numPartitions=None): @@ -1919,8 +1948,8 @@ def keyBy(self, f): >>> x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x) >>> y = sc.parallelize(zip(range(0,5), range(0,5))) - >>> map((lambda (x,y): (x, (list(y[0]), (list(y[1]))))), sorted(x.cogroup(y).collect())) - [(0, ([0], [0])), (1, ([1], [1])), (2, ([], [2])), (3, ([], [3])), (4, ([2], [4]))] + >>> [(x, list(map(list, y))) for x, y in sorted(x.cogroup(y).collect())] + [(0, [[0], [0]]), (1, [[1], [1]]), (2, [[], [2]]), (3, [[], [3]]), (4, [[2], [4]])] """ return self.map(lambda x: (f(x), x)) @@ -2049,17 +2078,18 @@ def name(self): """ Return the name of this RDD. """ - name_ = self._jrdd.name() - if name_: - return name_.encode('utf-8') + n = self._jrdd.name() + if n: + return n + @ignore_unicode_prefix def setName(self, name): """ Assign a name to this RDD. - >>> rdd1 = sc.parallelize([1,2]) + >>> rdd1 = sc.parallelize([1, 2]) >>> rdd1.setName('RDD1').name() - 'RDD1' + u'RDD1' """ self._jrdd.setName(name) return self @@ -2121,7 +2151,7 @@ def lookup(self, key): >>> sorted.lookup(1024) [] """ - values = self.filter(lambda (k, v): k == key).values() + values = self.filter(lambda kv: kv[0] == key).values() if self.partitioner is not None: return self.ctx.runJob(values, lambda x: x, [self.partitioner(key)], False) @@ -2159,7 +2189,7 @@ def sumApprox(self, timeout, confidence=0.95): or meet the confidence. >>> rdd = sc.parallelize(range(1000), 10) - >>> r = sum(xrange(1000)) + >>> r = sum(range(1000)) >>> (rdd.sumApprox(1000) - r) / r < 0.05 True """ @@ -2176,7 +2206,7 @@ def meanApprox(self, timeout, confidence=0.95): or meet the confidence. >>> rdd = sc.parallelize(range(1000), 10) - >>> r = sum(xrange(1000)) / 1000.0 + >>> r = sum(range(1000)) / 1000.0 >>> (rdd.meanApprox(1000) - r) / r < 0.05 True """ @@ -2201,10 +2231,10 @@ def countApproxDistinct(self, relativeSD=0.05): It must be greater than 0.000017. >>> n = sc.parallelize(range(1000)).map(str).countApproxDistinct() - >>> 950 < n < 1050 + >>> 900 < n < 1100 True >>> n = sc.parallelize([i % 20 for i in range(1000)]).countApproxDistinct() - >>> 18 < n < 22 + >>> 16 < n < 24 True """ if relativeSD < 0.000017: @@ -2223,8 +2253,7 @@ def toLocalIterator(self): >>> [x for x in rdd.toLocalIterator()] [0, 1, 2, 3, 4, 5, 6, 7, 8, 9] """ - partitions = xrange(self.getNumPartitions()) - for partition in partitions: + for partition in range(self.getNumPartitions()): rows = self.context.runJob(self, lambda x: x, [partition]) for row in rows: yield row diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index 459e1427803cb..fe8f87324804b 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -23,7 +23,7 @@ class RDDSamplerBase(object): def __init__(self, withReplacement, seed=None): - self._seed = seed if seed is not None else random.randint(0, sys.maxint) + self._seed = seed if seed is not None else random.randint(0, sys.maxsize) self._withReplacement = withReplacement self._random = None @@ -31,7 +31,7 @@ def initRandomGenerator(self, split): self._random = random.Random(self._seed ^ split) # mixing because the initial seeds are close to each other - for _ in xrange(10): + for _ in range(10): self._random.randint(0, 1) def getUniformSample(self): diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 4afa82f4b2973..d8cdcda3a3783 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -49,16 +49,24 @@ >>> sc.stop() """ -import cPickle -from itertools import chain, izip, product +import sys +from itertools import chain, product import marshal import struct -import sys import types import collections import zlib import itertools +if sys.version < '3': + import cPickle as pickle + protocol = 2 + from itertools import izip as zip +else: + import pickle + protocol = 3 + xrange = range + from pyspark import cloudpickle @@ -97,7 +105,7 @@ def _load_stream_without_unbatching(self, stream): # subclasses should override __eq__ as appropriate. def __eq__(self, other): - return isinstance(other, self.__class__) + return isinstance(other, self.__class__) and self.__dict__ == other.__dict__ def __ne__(self, other): return not self.__eq__(other) @@ -212,10 +220,6 @@ def load_stream(self, stream): def _load_stream_without_unbatching(self, stream): return self.serializer.load_stream(stream) - def __eq__(self, other): - return (isinstance(other, BatchedSerializer) and - other.serializer == self.serializer and other.batchSize == self.batchSize) - def __repr__(self): return "BatchedSerializer(%s, %d)" % (str(self.serializer), self.batchSize) @@ -233,14 +237,14 @@ def __init__(self, serializer, batchSize=10): def _batched(self, iterator): n = self.batchSize for key, values in iterator: - for i in xrange(0, len(values), n): + for i in range(0, len(values), n): yield key, values[i:i + n] def load_stream(self, stream): return self.serializer.load_stream(stream) def __repr__(self): - return "FlattenedValuesSerializer(%d)" % self.batchSize + return "FlattenedValuesSerializer(%s, %d)" % (self.serializer, self.batchSize) class AutoBatchedSerializer(BatchedSerializer): @@ -270,12 +274,8 @@ def dump_stream(self, iterator, stream): elif size > best * 10 and batch > 1: batch /= 2 - def __eq__(self, other): - return (isinstance(other, AutoBatchedSerializer) and - other.serializer == self.serializer and other.bestSize == self.bestSize) - def __repr__(self): - return "AutoBatchedSerializer(%s)" % str(self.serializer) + return "AutoBatchedSerializer(%s)" % self.serializer class CartesianDeserializer(FramedSerializer): @@ -285,6 +285,7 @@ class CartesianDeserializer(FramedSerializer): """ def __init__(self, key_ser, val_ser): + FramedSerializer.__init__(self) self.key_ser = key_ser self.val_ser = val_ser @@ -293,7 +294,7 @@ def prepare_keys_values(self, stream): val_stream = self.val_ser._load_stream_without_unbatching(stream) key_is_batched = isinstance(self.key_ser, BatchedSerializer) val_is_batched = isinstance(self.val_ser, BatchedSerializer) - for (keys, vals) in izip(key_stream, val_stream): + for (keys, vals) in zip(key_stream, val_stream): keys = keys if key_is_batched else [keys] vals = vals if val_is_batched else [vals] yield (keys, vals) @@ -303,10 +304,6 @@ def load_stream(self, stream): for pair in product(keys, vals): yield pair - def __eq__(self, other): - return (isinstance(other, CartesianDeserializer) and - self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __repr__(self): return "CartesianDeserializer(%s, %s)" % \ (str(self.key_ser), str(self.val_ser)) @@ -318,22 +315,14 @@ class PairDeserializer(CartesianDeserializer): Deserializes the JavaRDD zip() of two PythonRDDs. """ - def __init__(self, key_ser, val_ser): - self.key_ser = key_ser - self.val_ser = val_ser - def load_stream(self, stream): for (keys, vals) in self.prepare_keys_values(stream): if len(keys) != len(vals): raise ValueError("Can not deserialize RDD with different number of items" " in pair: (%d, %d)" % (len(keys), len(vals))) - for pair in izip(keys, vals): + for pair in zip(keys, vals): yield pair - def __eq__(self, other): - return (isinstance(other, PairDeserializer) and - self.key_ser == other.key_ser and self.val_ser == other.val_ser) - def __repr__(self): return "PairDeserializer(%s, %s)" % (str(self.key_ser), str(self.val_ser)) @@ -382,8 +371,8 @@ def _hijack_namedtuple(): global _old_namedtuple # or it will put in closure def _copy_func(f): - return types.FunctionType(f.func_code, f.func_globals, f.func_name, - f.func_defaults, f.func_closure) + return types.FunctionType(f.__code__, f.__globals__, f.__name__, + f.__defaults__, f.__closure__) _old_namedtuple = _copy_func(collections.namedtuple) @@ -392,15 +381,15 @@ def namedtuple(*args, **kwargs): return _hack_namedtuple(cls) # replace namedtuple with new one - collections.namedtuple.func_globals["_old_namedtuple"] = _old_namedtuple - collections.namedtuple.func_globals["_hack_namedtuple"] = _hack_namedtuple - collections.namedtuple.func_code = namedtuple.func_code + collections.namedtuple.__globals__["_old_namedtuple"] = _old_namedtuple + collections.namedtuple.__globals__["_hack_namedtuple"] = _hack_namedtuple + collections.namedtuple.__code__ = namedtuple.__code__ collections.namedtuple.__hijack = 1 # hack the cls already generated by namedtuple # those created in other module can be pickled as normal, # so only hack those in __main__ module - for n, o in sys.modules["__main__"].__dict__.iteritems(): + for n, o in sys.modules["__main__"].__dict__.items(): if (type(o) is type and o.__base__ is tuple and hasattr(o, "_fields") and "__reduce__" not in o.__dict__): @@ -413,7 +402,7 @@ def namedtuple(*args, **kwargs): class PickleSerializer(FramedSerializer): """ - Serializes objects using Python's cPickle serializer: + Serializes objects using Python's pickle serializer: http://docs.python.org/2/library/pickle.html @@ -422,10 +411,14 @@ class PickleSerializer(FramedSerializer): """ def dumps(self, obj): - return cPickle.dumps(obj, 2) + return pickle.dumps(obj, protocol) - def loads(self, obj): - return cPickle.loads(obj) + if sys.version >= '3': + def loads(self, obj, encoding="bytes"): + return pickle.loads(obj, encoding=encoding) + else: + def loads(self, obj, encoding=None): + return pickle.loads(obj) class CloudPickleSerializer(PickleSerializer): @@ -454,7 +447,7 @@ def loads(self, obj): class AutoSerializer(FramedSerializer): """ - Choose marshal or cPickle as serialization protocol automatically + Choose marshal or pickle as serialization protocol automatically """ def __init__(self): @@ -463,19 +456,19 @@ def __init__(self): def dumps(self, obj): if self._type is not None: - return 'P' + cPickle.dumps(obj, -1) + return b'P' + pickle.dumps(obj, -1) try: - return 'M' + marshal.dumps(obj) + return b'M' + marshal.dumps(obj) except Exception: - self._type = 'P' - return 'P' + cPickle.dumps(obj, -1) + self._type = b'P' + return b'P' + pickle.dumps(obj, -1) def loads(self, obj): _type = obj[0] - if _type == 'M': + if _type == b'M': return marshal.loads(obj[1:]) - elif _type == 'P': - return cPickle.loads(obj[1:]) + elif _type == b'P': + return pickle.loads(obj[1:]) else: raise ValueError("invalid sevialization type: %s" % _type) @@ -495,8 +488,8 @@ def dumps(self, obj): def loads(self, obj): return self.serializer.loads(zlib.decompress(obj)) - def __eq__(self, other): - return isinstance(other, CompressedSerializer) and self.serializer == other.serializer + def __repr__(self): + return "CompressedSerializer(%s)" % self.serializer class UTF8Deserializer(Serializer): @@ -505,7 +498,7 @@ class UTF8Deserializer(Serializer): Deserializes streams written by String.getBytes. """ - def __init__(self, use_unicode=False): + def __init__(self, use_unicode=True): self.use_unicode = use_unicode def loads(self, stream): @@ -526,13 +519,13 @@ def load_stream(self, stream): except EOFError: return - def __eq__(self, other): - return isinstance(other, UTF8Deserializer) and self.use_unicode == other.use_unicode + def __repr__(self): + return "UTF8Deserializer(%s)" % self.use_unicode def read_long(stream): length = stream.read(8) - if length == "": + if not length: raise EOFError return struct.unpack("!q", length)[0] @@ -547,7 +540,7 @@ def pack_long(value): def read_int(stream): length = stream.read(4) - if length == "": + if not length: raise EOFError return struct.unpack("!i", length)[0] diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 81aa970a32f76..144cdf0b0cdd5 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -21,13 +21,6 @@ This file is designed to be launched as a PYTHONSTARTUP script. """ -import sys -if sys.version_info[0] != 2: - print("Error: Default Python used is Python%s" % sys.version_info.major) - print("\tSet env variable PYSPARK_PYTHON to Python2 binary and re-run it.") - sys.exit(1) - - import atexit import os import platform @@ -53,9 +46,14 @@ try: # Try to access HiveConf, it will raise exception if Hive is not added sc._jvm.org.apache.hadoop.hive.conf.HiveConf() - sqlCtx = sqlContext = HiveContext(sc) + sqlContext = HiveContext(sc) except py4j.protocol.Py4JError: - sqlCtx = sqlContext = SQLContext(sc) + sqlContext = SQLContext(sc) +except TypeError: + sqlContext = SQLContext(sc) + +# for compatibility +sqlCtx = sqlContext print("""Welcome to ____ __ diff --git a/python/pyspark/shuffle.py b/python/pyspark/shuffle.py index 8a6fc627eb383..b54baa57ec28a 100644 --- a/python/pyspark/shuffle.py +++ b/python/pyspark/shuffle.py @@ -78,8 +78,8 @@ def _get_local_dirs(sub): # global stats -MemoryBytesSpilled = 0L -DiskBytesSpilled = 0L +MemoryBytesSpilled = 0 +DiskBytesSpilled = 0 class Aggregator(object): @@ -126,7 +126,7 @@ def mergeCombiners(self, iterator): """ Merge the combined items by mergeCombiner """ raise NotImplementedError - def iteritems(self): + def items(self): """ Return the merged items ad iterator """ raise NotImplementedError @@ -156,9 +156,9 @@ def mergeCombiners(self, iterator): for k, v in iterator: d[k] = comb(d[k], v) if k in d else v - def iteritems(self): - """ Return the merged items as iterator """ - return self.data.iteritems() + def items(self): + """ Return the merged items ad iterator """ + return iter(self.data.items()) def _compressed_serializer(self, serializer=None): @@ -208,15 +208,15 @@ class ExternalMerger(Merger): >>> agg = SimpleAggregator(lambda x, y: x + y) >>> merger = ExternalMerger(agg, 10) >>> N = 10000 - >>> merger.mergeValues(zip(xrange(N), xrange(N))) + >>> merger.mergeValues(zip(range(N), range(N))) >>> assert merger.spills > 0 - >>> sum(v for k,v in merger.iteritems()) + >>> sum(v for k,v in merger.items()) 49995000 >>> merger = ExternalMerger(agg, 10) - >>> merger.mergeCombiners(zip(xrange(N), xrange(N))) + >>> merger.mergeCombiners(zip(range(N), range(N))) >>> assert merger.spills > 0 - >>> sum(v for k,v in merger.iteritems()) + >>> sum(v for k,v in merger.items()) 49995000 """ @@ -335,10 +335,10 @@ def _spill(self): # above limit at the first time. # open all the files for writing - streams = [open(os.path.join(path, str(i)), 'w') + streams = [open(os.path.join(path, str(i)), 'wb') for i in range(self.partitions)] - for k, v in self.data.iteritems(): + for k, v in self.data.items(): h = self._partition(k) # put one item in batch, make it compatible with load_stream # it will increase the memory if dump them in batch @@ -354,9 +354,9 @@ def _spill(self): else: for i in range(self.partitions): p = os.path.join(path, str(i)) - with open(p, "w") as f: + with open(p, "wb") as f: # dump items in batch - self.serializer.dump_stream(self.pdata[i].iteritems(), f) + self.serializer.dump_stream(iter(self.pdata[i].items()), f) self.pdata[i].clear() DiskBytesSpilled += os.path.getsize(p) @@ -364,10 +364,10 @@ def _spill(self): gc.collect() # release the memory as much as possible MemoryBytesSpilled += (used_memory - get_used_memory()) << 20 - def iteritems(self): + def items(self): """ Return all merged items as iterator """ if not self.pdata and not self.spills: - return self.data.iteritems() + return iter(self.data.items()) return self._external_items() def _external_items(self): @@ -398,7 +398,8 @@ def _merged_items(self, index): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) # do not check memory during merging - self.mergeCombiners(self.serializer.load_stream(open(p)), 0) + with open(p, "rb") as f: + self.mergeCombiners(self.serializer.load_stream(f), 0) # limit the total partitions if (self.scale * self.partitions < self.MAX_TOTAL_PARTITIONS @@ -408,7 +409,7 @@ def _merged_items(self, index): gc.collect() # release the memory as much as possible return self._recursive_merged_items(index) - return self.data.iteritems() + return self.data.items() def _recursive_merged_items(self, index): """ @@ -426,7 +427,8 @@ def _recursive_merged_items(self, index): for j in range(self.spills): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) - m.mergeCombiners(self.serializer.load_stream(open(p)), 0) + with open(p, 'rb') as f: + m.mergeCombiners(self.serializer.load_stream(f), 0) if get_used_memory() > limit: m._spill() @@ -451,7 +453,7 @@ class ExternalSorter(object): >>> sorter = ExternalSorter(1) # 1M >>> import random - >>> l = range(1024) + >>> l = list(range(1024)) >>> random.shuffle(l) >>> sorted(l) == list(sorter.sorted(l)) True @@ -499,9 +501,16 @@ def sorted(self, iterator, key=None, reverse=False): # sort them inplace will save memory current_chunk.sort(key=key, reverse=reverse) path = self._get_path(len(chunks)) - with open(path, 'w') as f: + with open(path, 'wb') as f: self.serializer.dump_stream(current_chunk, f) - chunks.append(self.serializer.load_stream(open(path))) + + def load(f): + for v in self.serializer.load_stream(f): + yield v + # close the file explicit once we consume all the items + # to avoid ResourceWarning in Python3 + f.close() + chunks.append(load(open(path, 'rb'))) current_chunk = [] gc.collect() limit = self._next_limit() @@ -527,7 +536,7 @@ class ExternalList(object): ExternalList can have many items which cannot be hold in memory in the same time. - >>> l = ExternalList(range(100)) + >>> l = ExternalList(list(range(100))) >>> len(l) 100 >>> l.append(10) @@ -555,11 +564,11 @@ def __init__(self, values): def __getstate__(self): if self._file is not None: self._file.flush() - f = os.fdopen(os.dup(self._file.fileno())) - f.seek(0) - serialized = f.read() + with os.fdopen(os.dup(self._file.fileno()), "rb") as f: + f.seek(0) + serialized = f.read() else: - serialized = '' + serialized = b'' return self.values, self.count, serialized def __setstate__(self, item): @@ -575,7 +584,7 @@ def __iter__(self): if self._file is not None: self._file.flush() # read all items from disks first - with os.fdopen(os.dup(self._file.fileno()), 'r') as f: + with os.fdopen(os.dup(self._file.fileno()), 'rb') as f: f.seek(0) for v in self._ser.load_stream(f): yield v @@ -598,11 +607,16 @@ def _open_file(self): d = dirs[id(self) % len(dirs)] if not os.path.exists(d): os.makedirs(d) - p = os.path.join(d, str(id)) - self._file = open(p, "w+", 65536) + p = os.path.join(d, str(id(self))) + self._file = open(p, "wb+", 65536) self._ser = BatchedSerializer(CompressedSerializer(PickleSerializer()), 1024) os.unlink(p) + def __del__(self): + if self._file: + self._file.close() + self._file = None + def _spill(self): """ dump the values into disk """ global MemoryBytesSpilled, DiskBytesSpilled @@ -651,33 +665,28 @@ class GroupByKey(object): """ Group a sorted iterator as [(k1, it1), (k2, it2), ...] - >>> k = [i/3 for i in range(6)] + >>> k = [i // 3 for i in range(6)] >>> v = [[i] for i in range(6)] - >>> g = GroupByKey(iter(zip(k, v))) + >>> g = GroupByKey(zip(k, v)) >>> [(k, list(it)) for k, it in g] [(0, [0, 1, 2]), (1, [3, 4, 5])] """ def __init__(self, iterator): - self.iterator = iter(iterator) - self.next_item = None + self.iterator = iterator def __iter__(self): - return self - - def next(self): - key, value = self.next_item if self.next_item else next(self.iterator) - values = ExternalListOfList([value]) - try: - while True: - k, v = next(self.iterator) - if k != key: - self.next_item = (k, v) - break + key, values = None, None + for k, v in self.iterator: + if values is not None and k == key: values.append(v) - except StopIteration: - self.next_item = None - return key, values + else: + if values is not None: + yield (key, values) + key = k + values = ExternalListOfList([v]) + if values is not None: + yield (key, values) class ExternalGroupBy(ExternalMerger): @@ -744,7 +753,7 @@ def _spill(self): # above limit at the first time. # open all the files for writing - streams = [open(os.path.join(path, str(i)), 'w') + streams = [open(os.path.join(path, str(i)), 'wb') for i in range(self.partitions)] # If the number of keys is small, then the overhead of sort is small @@ -756,7 +765,7 @@ def _spill(self): h = self._partition(k) self.serializer.dump_stream([(k, self.data[k])], streams[h]) else: - for k, v in self.data.iteritems(): + for k, v in self.data.items(): h = self._partition(k) self.serializer.dump_stream([(k, v)], streams[h]) @@ -771,14 +780,14 @@ def _spill(self): else: for i in range(self.partitions): p = os.path.join(path, str(i)) - with open(p, "w") as f: + with open(p, "wb") as f: # dump items in batch if self._sorted: # sort by key only (stable) - sorted_items = sorted(self.pdata[i].iteritems(), key=operator.itemgetter(0)) + sorted_items = sorted(self.pdata[i].items(), key=operator.itemgetter(0)) self.serializer.dump_stream(sorted_items, f) else: - self.serializer.dump_stream(self.pdata[i].iteritems(), f) + self.serializer.dump_stream(self.pdata[i].items(), f) self.pdata[i].clear() DiskBytesSpilled += os.path.getsize(p) @@ -792,7 +801,7 @@ def _merged_items(self, index): # if the memory can not hold all the partition, # then use sort based merge. Because of compression, # the data on disks will be much smaller than needed memory - if (size >> 20) >= self.memory_limit / 10: + if size >= self.memory_limit << 17: # * 1M / 8 return self._merge_sorted_items(index) self.data = {} @@ -800,15 +809,18 @@ def _merged_items(self, index): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) # do not check memory during merging - self.mergeCombiners(self.serializer.load_stream(open(p)), 0) - return self.data.iteritems() + with open(p, "rb") as f: + self.mergeCombiners(self.serializer.load_stream(f), 0) + return self.data.items() def _merge_sorted_items(self, index): """ load a partition from disk, then sort and group by key """ def load_partition(j): path = self._get_spill_dir(j) p = os.path.join(path, str(index)) - return self.serializer.load_stream(open(p, 'r', 65536)) + with open(p, 'rb', 65536) as f: + for v in self.serializer.load_stream(f): + yield v disk_items = [load_partition(j) for j in range(self.spills)] diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index 65abb24eed823..6d54b9e49ed10 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -37,9 +37,22 @@ - L{types} List of data types available. """ +from __future__ import absolute_import + +# fix the module name conflict for Python 3+ +import sys +from . import _types as types +modname = __name__ + '.types' +types.__name__ = modname +# update the __module__ for all objects, make them picklable +for v in types.__dict__.values(): + if hasattr(v, "__module__") and v.__module__.endswith('._types'): + v.__module__ = modname +sys.modules[modname] = types +del modname, sys -from pyspark.sql.context import SQLContext, HiveContext from pyspark.sql.types import Row +from pyspark.sql.context import SQLContext, HiveContext from pyspark.sql.dataframe import DataFrame, GroupedData, Column, SchemaRDD, DataFrameNaFunctions __all__ = [ diff --git a/python/pyspark/sql/types.py b/python/pyspark/sql/_types.py similarity index 97% rename from python/pyspark/sql/types.py rename to python/pyspark/sql/_types.py index ef76d84c00481..492c0cbdcf693 100644 --- a/python/pyspark/sql/types.py +++ b/python/pyspark/sql/_types.py @@ -15,6 +15,7 @@ # limitations under the License. # +import sys import decimal import datetime import keyword @@ -25,6 +26,9 @@ from array import array from operator import itemgetter +if sys.version >= "3": + long = int + unicode = str __all__ = [ "DataType", "NullType", "StringType", "BinaryType", "BooleanType", "DateType", @@ -410,7 +414,7 @@ def fromJson(cls, json): split = pyUDT.rfind(".") pyModule = pyUDT[:split] pyClass = pyUDT[split+1:] - m = __import__(pyModule, globals(), locals(), [pyClass], -1) + m = __import__(pyModule, globals(), locals(), [pyClass]) UDT = getattr(m, pyClass) return UDT() @@ -419,10 +423,9 @@ def __eq__(self, other): _all_primitive_types = dict((v.typeName(), v) - for v in globals().itervalues() - if type(v) is PrimitiveTypeSingleton and - v.__base__ == PrimitiveType) - + for v in list(globals().values()) + if (type(v) is type or type(v) is PrimitiveTypeSingleton) + and v.__base__ == PrimitiveType) _all_complex_types = dict((v.typeName(), v) for v in [ArrayType, MapType, StructType]) @@ -486,10 +489,10 @@ def _parse_datatype_json_string(json_string): def _parse_datatype_json_value(json_value): - if type(json_value) is unicode: + if not isinstance(json_value, dict): if json_value in _all_primitive_types.keys(): return _all_primitive_types[json_value]() - elif json_value == u'decimal': + elif json_value == 'decimal': return DecimalType() elif _FIXED_DECIMAL.match(json_value): m = _FIXED_DECIMAL.match(json_value) @@ -511,10 +514,8 @@ def _parse_datatype_json_value(json_value): type(None): NullType, bool: BooleanType, int: LongType, - long: LongType, float: DoubleType, str: StringType, - unicode: StringType, bytearray: BinaryType, decimal.Decimal: DecimalType, datetime.date: DateType, @@ -522,6 +523,12 @@ def _parse_datatype_json_value(json_value): datetime.time: TimestampType, } +if sys.version < "3": + _type_mappings.update({ + unicode: StringType, + long: LongType, + }) + def _infer_type(obj): """Infer the DataType from obj @@ -541,7 +548,7 @@ def _infer_type(obj): return dataType() if isinstance(obj, dict): - for key, value in obj.iteritems(): + for key, value in obj.items(): if key is not None and value is not None: return MapType(_infer_type(key), _infer_type(value), True) else: @@ -565,10 +572,10 @@ def _infer_schema(row): items = sorted(row.items()) elif isinstance(row, (tuple, list)): - if hasattr(row, "_fields"): # namedtuple - items = zip(row._fields, tuple(row)) - elif hasattr(row, "__fields__"): # Row + if hasattr(row, "__fields__"): # Row items = zip(row.__fields__, tuple(row)) + elif hasattr(row, "_fields"): # namedtuple + items = zip(row._fields, tuple(row)) else: names = ['_%d' % i for i in range(1, len(row) + 1)] items = zip(names, row) @@ -647,7 +654,7 @@ def converter(obj): if isinstance(obj, dict): return tuple(c(obj.get(n)) for n, c in zip(names, converters)) elif isinstance(obj, tuple): - if hasattr(obj, "_fields") or hasattr(obj, "__fields__"): + if hasattr(obj, "__fields__") or hasattr(obj, "_fields"): return tuple(c(v) for c, v in zip(converters, obj)) elif all(isinstance(x, tuple) and len(x) == 2 for x in obj): # k-v pairs d = dict(obj) @@ -733,12 +740,12 @@ def _create_converter(dataType): if isinstance(dataType, ArrayType): conv = _create_converter(dataType.elementType) - return lambda row: map(conv, row) + return lambda row: [conv(v) for v in row] elif isinstance(dataType, MapType): kconv = _create_converter(dataType.keyType) vconv = _create_converter(dataType.valueType) - return lambda row: dict((kconv(k), vconv(v)) for k, v in row.iteritems()) + return lambda row: dict((kconv(k), vconv(v)) for k, v in row.items()) elif isinstance(dataType, NullType): return lambda x: None @@ -881,7 +888,7 @@ def _infer_schema_type(obj, dataType): >>> _infer_schema_type(row, schema) StructType...a,ArrayType...b,MapType(StringType,...c,LongType... """ - if dataType is NullType(): + if isinstance(dataType, NullType): return _infer_type(obj) if not obj: @@ -892,7 +899,7 @@ def _infer_schema_type(obj, dataType): return ArrayType(eType, True) elif isinstance(dataType, MapType): - k, v = obj.iteritems().next() + k, v = next(iter(obj.items())) return MapType(_infer_schema_type(k, dataType.keyType), _infer_schema_type(v, dataType.valueType)) @@ -935,7 +942,7 @@ def _verify_type(obj, dataType): >>> _verify_type(None, StructType([])) >>> _verify_type("", StringType()) >>> _verify_type(0, LongType()) - >>> _verify_type(range(3), ArrayType(ShortType())) + >>> _verify_type(list(range(3)), ArrayType(ShortType())) >>> _verify_type(set(), ArrayType(StringType())) # doctest: +IGNORE_EXCEPTION_DETAIL Traceback (most recent call last): ... @@ -976,7 +983,7 @@ def _verify_type(obj, dataType): _verify_type(i, dataType.elementType) elif isinstance(dataType, MapType): - for k, v in obj.iteritems(): + for k, v in obj.items(): _verify_type(k, dataType.keyType) _verify_type(v, dataType.valueType) @@ -1213,6 +1220,8 @@ def __getattr__(self, item): return self[idx] except IndexError: raise AttributeError(item) + except ValueError: + raise AttributeError(item) def __reduce__(self): if hasattr(self, "__fields__"): diff --git a/python/pyspark/sql/context.py b/python/pyspark/sql/context.py index e8529a8f8e3a4..c90afc326ca0e 100644 --- a/python/pyspark/sql/context.py +++ b/python/pyspark/sql/context.py @@ -15,14 +15,19 @@ # limitations under the License. # +import sys import warnings import json -from itertools import imap + +if sys.version >= '3': + basestring = unicode = str +else: + from itertools import imap as map from py4j.protocol import Py4JError from py4j.java_collections import MapConverter -from pyspark.rdd import RDD, _prepare_for_python_RDD +from pyspark.rdd import RDD, _prepare_for_python_RDD, ignore_unicode_prefix from pyspark.serializers import AutoBatchedSerializer, PickleSerializer from pyspark.sql.types import Row, StringType, StructType, _verify_type, \ _infer_schema, _has_nulltype, _merge_type, _create_converter, _python_to_sql_converter @@ -62,31 +67,27 @@ class SQLContext(object): A SQLContext can be used create :class:`DataFrame`, register :class:`DataFrame` as tables, execute SQL over tables, cache tables, and read parquet files. - When created, :class:`SQLContext` adds a method called ``toDF`` to :class:`RDD`, - which could be used to convert an RDD into a DataFrame, it's a shorthand for - :func:`SQLContext.createDataFrame`. - :param sparkContext: The :class:`SparkContext` backing this SQLContext. :param sqlContext: An optional JVM Scala SQLContext. If set, we do not instantiate a new SQLContext in the JVM, instead we make all calls to this object. """ + @ignore_unicode_prefix def __init__(self, sparkContext, sqlContext=None): """Creates a new SQLContext. >>> from datetime import datetime >>> sqlContext = SQLContext(sc) - >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1L, + >>> allTypes = sc.parallelize([Row(i=1, s="string", d=1.0, l=1, ... b=True, list=[1, 2, 3], dict={"s": 0}, row=Row(a=1), ... time=datetime(2014, 8, 1, 14, 1, 5))]) >>> df = allTypes.toDF() >>> df.registerTempTable("allTypes") >>> sqlContext.sql('select i+1, d+1, not b, list[1], dict["s"], time, row.a ' ... 'from allTypes where b and i > 0').collect() - [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0...8, 1, 14, 1, 5), a=1)] - >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, - ... x.row.a, x.list)).collect() - [(1, u'string', 1.0, 1, True, ...(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] + [Row(c0=2, c1=2.0, c2=False, c3=2, c4=0, time=datetime.datetime(2014, 8, 1, 14, 1, 5), a=1)] + >>> df.map(lambda x: (x.i, x.s, x.d, x.l, x.b, x.time, x.row.a, x.list)).collect() + [(1, u'string', 1.0, 1, True, datetime.datetime(2014, 8, 1, 14, 1, 5), 1, [1, 2, 3])] """ self._sc = sparkContext self._jsc = self._sc._jsc @@ -122,6 +123,7 @@ def udf(self): """Returns a :class:`UDFRegistration` for UDF registration.""" return UDFRegistration(self) + @ignore_unicode_prefix def registerFunction(self, name, f, returnType=StringType()): """Registers a lambda function as a UDF so it can be used in SQL statements. @@ -147,7 +149,7 @@ def registerFunction(self, name, f, returnType=StringType()): >>> sqlContext.sql("SELECT stringLengthInt('test')").collect() [Row(c0=4)] """ - func = lambda _, it: imap(lambda x: f(*x), it) + func = lambda _, it: map(lambda x: f(*x), it) ser = AutoBatchedSerializer(PickleSerializer()) command = (func, None, ser, ser) pickled_cmd, bvars, env, includes = _prepare_for_python_RDD(self._sc, command, self) @@ -185,6 +187,7 @@ def _inferSchema(self, rdd, samplingRatio=None): schema = rdd.map(_infer_schema).reduce(_merge_type) return schema + @ignore_unicode_prefix def inferSchema(self, rdd, samplingRatio=None): """::note: Deprecated in 1.3, use :func:`createDataFrame` instead. """ @@ -195,6 +198,7 @@ def inferSchema(self, rdd, samplingRatio=None): return self.createDataFrame(rdd, None, samplingRatio) + @ignore_unicode_prefix def applySchema(self, rdd, schema): """::note: Deprecated in 1.3, use :func:`createDataFrame` instead. """ @@ -208,6 +212,7 @@ def applySchema(self, rdd, schema): return self.createDataFrame(rdd, schema) + @ignore_unicode_prefix def createDataFrame(self, data, schema=None, samplingRatio=None): """ Creates a :class:`DataFrame` from an :class:`RDD` of :class:`tuple`/:class:`list`, @@ -380,6 +385,7 @@ def jsonFile(self, path, schema=None, samplingRatio=1.0): df = self._ssql_ctx.jsonFile(path, scala_datatype) return DataFrame(df, self) + @ignore_unicode_prefix def jsonRDD(self, rdd, schema=None, samplingRatio=1.0): """Loads an RDD storing one JSON object per string as a :class:`DataFrame`. @@ -477,6 +483,7 @@ def createExternalTable(self, tableName, path=None, source=None, joptions) return DataFrame(df, self) + @ignore_unicode_prefix def sql(self, sqlQuery): """Returns a :class:`DataFrame` representing the result of the given query. @@ -497,6 +504,7 @@ def table(self, tableName): """ return DataFrame(self._ssql_ctx.table(tableName), self) + @ignore_unicode_prefix def tables(self, dbName=None): """Returns a :class:`DataFrame` containing names of tables in the given database. diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index f2c3b74a185cf..d76504f986270 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -16,14 +16,19 @@ # import sys -import itertools import warnings import random +if sys.version >= '3': + basestring = unicode = str + long = int +else: + from itertools import imap as map + from py4j.java_collections import ListConverter, MapConverter from pyspark.context import SparkContext -from pyspark.rdd import RDD, _load_from_socket +from pyspark.rdd import RDD, _load_from_socket, ignore_unicode_prefix from pyspark.serializers import BatchedSerializer, PickleSerializer, UTF8Deserializer from pyspark.storagelevel import StorageLevel from pyspark.traceback_utils import SCCallSiteSync @@ -65,19 +70,20 @@ def __init__(self, jdf, sql_ctx): self._sc = sql_ctx and sql_ctx._sc self.is_cached = False self._schema = None # initialized lazily + self._lazy_rdd = None @property def rdd(self): """Returns the content as an :class:`pyspark.RDD` of :class:`Row`. """ - if not hasattr(self, '_lazy_rdd'): + if self._lazy_rdd is None: jrdd = self._jdf.javaToPython() rdd = RDD(jrdd, self.sql_ctx._sc, BatchedSerializer(PickleSerializer())) schema = self.schema def applySchema(it): cls = _create_cls(schema) - return itertools.imap(cls, it) + return map(cls, it) self._lazy_rdd = rdd.mapPartitions(applySchema) @@ -89,13 +95,14 @@ def na(self): """ return DataFrameNaFunctions(self) - def toJSON(self, use_unicode=False): + @ignore_unicode_prefix + def toJSON(self, use_unicode=True): """Converts a :class:`DataFrame` into a :class:`RDD` of string. Each row is turned into a JSON document as one element in the returned RDD. >>> df.toJSON().first() - '{"age":2,"name":"Alice"}' + u'{"age":2,"name":"Alice"}' """ rdd = self._jdf.toJSON() return RDD(rdd.toJavaRDD(), self._sc, UTF8Deserializer(use_unicode)) @@ -228,7 +235,7 @@ def printSchema(self): |-- name: string (nullable = true) """ - print (self._jdf.schema().treeString()) + print(self._jdf.schema().treeString()) def explain(self, extended=False): """Prints the (logical and physical) plans to the console for debugging purpose. @@ -250,9 +257,9 @@ def explain(self, extended=False): == RDD == """ if extended: - print self._jdf.queryExecution().toString() + print(self._jdf.queryExecution().toString()) else: - print self._jdf.queryExecution().executedPlan().toString() + print(self._jdf.queryExecution().executedPlan().toString()) def isLocal(self): """Returns ``True`` if the :func:`collect` and :func:`take` methods can be run locally @@ -270,7 +277,7 @@ def show(self, n=20): 2 Alice 5 Bob """ - print self._jdf.showString(n).encode('utf8', 'ignore') + print(self._jdf.showString(n)) def __repr__(self): return "DataFrame[%s]" % (", ".join("%s: %s" % c for c in self.dtypes)) @@ -279,10 +286,11 @@ def count(self): """Returns the number of rows in this :class:`DataFrame`. >>> df.count() - 2L + 2 """ - return self._jdf.count() + return int(self._jdf.count()) + @ignore_unicode_prefix def collect(self): """Returns all the records as a list of :class:`Row`. @@ -295,6 +303,7 @@ def collect(self): cls = _create_cls(self.schema) return [cls(r) for r in rs] + @ignore_unicode_prefix def limit(self, num): """Limits the result count to the number specified. @@ -306,6 +315,7 @@ def limit(self, num): jdf = self._jdf.limit(num) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def take(self, num): """Returns the first ``num`` rows as a :class:`list` of :class:`Row`. @@ -314,6 +324,7 @@ def take(self, num): """ return self.limit(num).collect() + @ignore_unicode_prefix def map(self, f): """ Returns a new :class:`RDD` by applying a the ``f`` function to each :class:`Row`. @@ -324,6 +335,7 @@ def map(self, f): """ return self.rdd.map(f) + @ignore_unicode_prefix def flatMap(self, f): """ Returns a new :class:`RDD` by first applying the ``f`` function to each :class:`Row`, and then flattening the results. @@ -353,7 +365,7 @@ def foreach(self, f): This is a shorthand for ``df.rdd.foreach()``. >>> def f(person): - ... print person.name + ... print(person.name) >>> df.foreach(f) """ return self.rdd.foreach(f) @@ -365,7 +377,7 @@ def foreachPartition(self, f): >>> def f(people): ... for person in people: - ... print person.name + ... print(person.name) >>> df.foreachPartition(f) """ return self.rdd.foreachPartition(f) @@ -412,7 +424,7 @@ def distinct(self): """Returns a new :class:`DataFrame` containing the distinct rows in this :class:`DataFrame`. >>> df.distinct().count() - 2L + 2 """ return DataFrame(self._jdf.distinct(), self.sql_ctx) @@ -420,10 +432,10 @@ def sample(self, withReplacement, fraction, seed=None): """Returns a sampled subset of this :class:`DataFrame`. >>> df.sample(False, 0.5, 97).count() - 1L + 1 """ assert fraction >= 0.0, "Negative fraction value: %s" % fraction - seed = seed if seed is not None else random.randint(0, sys.maxint) + seed = seed if seed is not None else random.randint(0, sys.maxsize) rdd = self._jdf.sample(withReplacement, fraction, long(seed)) return DataFrame(rdd, self.sql_ctx) @@ -437,6 +449,7 @@ def dtypes(self): return [(str(f.name), f.dataType.simpleString()) for f in self.schema.fields] @property + @ignore_unicode_prefix def columns(self): """Returns all column names as a list. @@ -445,6 +458,7 @@ def columns(self): """ return [f.name for f in self.schema.fields] + @ignore_unicode_prefix def join(self, other, joinExprs=None, joinType=None): """Joins with another :class:`DataFrame`, using the given join expression. @@ -470,6 +484,7 @@ def join(self, other, joinExprs=None, joinType=None): jdf = self._jdf.join(other._jdf, joinExprs._jc, joinType) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def sort(self, *cols): """Returns a new :class:`DataFrame` sorted by the specified column(s). @@ -513,6 +528,7 @@ def describe(self, *cols): jdf = self._jdf.describe(self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols)) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def head(self, n=None): """ Returns the first ``n`` rows as a list of :class:`Row`, @@ -528,6 +544,7 @@ def head(self, n=None): return rs[0] if rs else None return self.take(n) + @ignore_unicode_prefix def first(self): """Returns the first row as a :class:`Row`. @@ -536,6 +553,7 @@ def first(self): """ return self.head() + @ignore_unicode_prefix def __getitem__(self, item): """Returns the column as a :class:`Column`. @@ -567,6 +585,7 @@ def __getattr__(self, name): jc = self._jdf.apply(name) return Column(jc) + @ignore_unicode_prefix def select(self, *cols): """Projects a set of expressions and returns a new :class:`DataFrame`. @@ -598,6 +617,7 @@ def selectExpr(self, *expr): jdf = self._jdf.selectExpr(self._sc._jvm.PythonUtils.toSeq(jexpr)) return DataFrame(jdf, self.sql_ctx) + @ignore_unicode_prefix def filter(self, condition): """Filters rows using the given condition. @@ -626,6 +646,7 @@ def filter(self, condition): where = filter + @ignore_unicode_prefix def groupBy(self, *cols): """Groups the :class:`DataFrame` using the specified columns, so we can run aggregation on them. See :class:`GroupedData` @@ -775,6 +796,7 @@ def fillna(self, value, subset=None): cols = self.sql_ctx._sc._jvm.PythonUtils.toSeq(cols) return DataFrame(self._jdf.na().fill(value, cols), self.sql_ctx) + @ignore_unicode_prefix def withColumn(self, colName, col): """Returns a new :class:`DataFrame` by adding a column. @@ -786,6 +808,7 @@ def withColumn(self, colName, col): """ return self.select('*', col.alias(colName)) + @ignore_unicode_prefix def withColumnRenamed(self, existing, new): """REturns a new :class:`DataFrame` by renaming an existing column. @@ -852,6 +875,7 @@ def __init__(self, jdf, sql_ctx): self._jdf = jdf self.sql_ctx = sql_ctx + @ignore_unicode_prefix def agg(self, *exprs): """Compute aggregates and returns the result as a :class:`DataFrame`. @@ -1041,11 +1065,13 @@ def __init__(self, jc): __sub__ = _bin_op("minus") __mul__ = _bin_op("multiply") __div__ = _bin_op("divide") + __truediv__ = _bin_op("divide") __mod__ = _bin_op("mod") __radd__ = _bin_op("plus") __rsub__ = _reverse_op("minus") __rmul__ = _bin_op("multiply") __rdiv__ = _reverse_op("divide") + __rtruediv__ = _reverse_op("divide") __rmod__ = _reverse_op("mod") # logistic operators @@ -1075,6 +1101,7 @@ def __init__(self, jc): startswith = _bin_op("startsWith") endswith = _bin_op("endsWith") + @ignore_unicode_prefix def substr(self, startPos, length): """ Return a :class:`Column` which is a substring of the column @@ -1097,6 +1124,7 @@ def substr(self, startPos, length): __getslice__ = substr + @ignore_unicode_prefix def inSet(self, *cols): """ A boolean expression that is evaluated to true if the value of this expression is contained by the evaluated values of the arguments. @@ -1131,6 +1159,7 @@ def alias(self, alias): """ return Column(getattr(self._jc, "as")(alias)) + @ignore_unicode_prefix def cast(self, dataType): """ Convert the column into type `dataType` diff --git a/python/pyspark/sql/functions.py b/python/pyspark/sql/functions.py index daeb6916b58bc..1d6536952810f 100644 --- a/python/pyspark/sql/functions.py +++ b/python/pyspark/sql/functions.py @@ -18,8 +18,10 @@ """ A collections of builtin functions """ +import sys -from itertools import imap +if sys.version < "3": + from itertools import imap as map from py4j.java_collections import ListConverter @@ -116,7 +118,7 @@ def __init__(self, func, returnType): def _create_judf(self): f = self.func # put it in closure `func` - func = lambda _, it: imap(lambda x: f(*x), it) + func = lambda _, it: map(lambda x: f(*x), it) ser = AutoBatchedSerializer(PickleSerializer()) command = (func, None, ser, ser) sc = SparkContext._active_spark_context diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index b3a6a2c6a9229..7c09a0cfe30ab 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -157,13 +157,13 @@ def test_udf2(self): self.assertEqual(4, res[0]) def test_udf_with_array_type(self): - d = [Row(l=range(3), d={"key": range(5)})] + d = [Row(l=list(range(3)), d={"key": list(range(5))})] rdd = self.sc.parallelize(d) self.sqlCtx.createDataFrame(rdd).registerTempTable("test") self.sqlCtx.registerFunction("copylist", lambda l: list(l), ArrayType(IntegerType())) self.sqlCtx.registerFunction("maplen", lambda d: len(d), IntegerType()) [(l1, l2)] = self.sqlCtx.sql("select copylist(l), maplen(d) from test").collect() - self.assertEqual(range(3), l1) + self.assertEqual(list(range(3)), l1) self.assertEqual(1, l2) def test_broadcast_in_udf(self): @@ -266,7 +266,7 @@ def test_infer_nested_schema(self): def test_apply_schema(self): from datetime import date, datetime - rdd = self.sc.parallelize([(127, -128L, -32768, 32767, 2147483647L, 1.0, + rdd = self.sc.parallelize([(127, -128, -32768, 32767, 2147483647, 1.0, date(2010, 1, 1), datetime(2010, 1, 1, 1, 1, 1), {"a": 1}, (2,), [1, 2, 3], None)]) schema = StructType([ @@ -309,7 +309,7 @@ def test_apply_schema(self): def test_struct_in_map(self): d = [Row(m={Row(i=1): Row(s="")})] df = self.sc.parallelize(d).toDF() - k, v = df.head().m.items()[0] + k, v = list(df.head().m.items())[0] self.assertEqual(1, k.i) self.assertEqual("", v.s) @@ -554,6 +554,9 @@ def setUpClass(cls): except py4j.protocol.Py4JError: cls.sqlCtx = None return + except TypeError: + cls.sqlCtx = None + return os.unlink(cls.tempdir.name) _scala_HiveContext =\ cls.sc._jvm.org.apache.spark.sql.hive.test.TestHiveContext(cls.sc._jsc.sc()) diff --git a/python/pyspark/statcounter.py b/python/pyspark/statcounter.py index 1e597d64e03fe..944fa414b0c0e 100644 --- a/python/pyspark/statcounter.py +++ b/python/pyspark/statcounter.py @@ -31,7 +31,7 @@ class StatCounter(object): def __init__(self, values=[]): - self.n = 0L # Running count of our values + self.n = 0 # Running count of our values self.mu = 0.0 # Running mean of our values self.m2 = 0.0 # Running variance numerator (sum of (x - mean)^2) self.maxValue = float("-inf") @@ -87,7 +87,7 @@ def copy(self): return copy.deepcopy(self) def count(self): - return self.n + return int(self.n) def mean(self): return self.mu diff --git a/python/pyspark/streaming/context.py b/python/pyspark/streaming/context.py index 2c73083c9f9a8..4590c58839266 100644 --- a/python/pyspark/streaming/context.py +++ b/python/pyspark/streaming/context.py @@ -14,6 +14,9 @@ # See the License for the specific language governing permissions and # limitations under the License. # + +from __future__ import print_function + import os import sys @@ -157,7 +160,7 @@ def getOrCreate(cls, checkpointPath, setupFunc): try: jssc = gw.jvm.JavaStreamingContext(checkpointPath) except Exception: - print >>sys.stderr, "failed to load StreamingContext from checkpoint" + print("failed to load StreamingContext from checkpoint", file=sys.stderr) raise jsc = jssc.sparkContext() diff --git a/python/pyspark/streaming/dstream.py b/python/pyspark/streaming/dstream.py index 3fa42444239f7..ff097985fae3e 100644 --- a/python/pyspark/streaming/dstream.py +++ b/python/pyspark/streaming/dstream.py @@ -15,11 +15,15 @@ # limitations under the License. # -from itertools import chain, ifilter, imap +import sys import operator import time +from itertools import chain from datetime import datetime +if sys.version < "3": + from itertools import imap as map, ifilter as filter + from py4j.protocol import Py4JJavaError from pyspark import RDD @@ -76,7 +80,7 @@ def filter(self, f): Return a new DStream containing only the elements that satisfy predicate. """ def func(iterator): - return ifilter(f, iterator) + return filter(f, iterator) return self.mapPartitions(func, True) def flatMap(self, f, preservesPartitioning=False): @@ -85,7 +89,7 @@ def flatMap(self, f, preservesPartitioning=False): this DStream, and then flattening the results """ def func(s, iterator): - return chain.from_iterable(imap(f, iterator)) + return chain.from_iterable(map(f, iterator)) return self.mapPartitionsWithIndex(func, preservesPartitioning) def map(self, f, preservesPartitioning=False): @@ -93,7 +97,7 @@ def map(self, f, preservesPartitioning=False): Return a new DStream by applying a function to each element of DStream. """ def func(iterator): - return imap(f, iterator) + return map(f, iterator) return self.mapPartitions(func, preservesPartitioning) def mapPartitions(self, f, preservesPartitioning=False): @@ -150,7 +154,7 @@ def foreachRDD(self, func): """ Apply a function to each RDD in this DStream. """ - if func.func_code.co_argcount == 1: + if func.__code__.co_argcount == 1: old_func = func func = lambda t, rdd: old_func(rdd) jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer) @@ -165,14 +169,14 @@ def pprint(self, num=10): """ def takeAndPrint(time, rdd): taken = rdd.take(num + 1) - print "-------------------------------------------" - print "Time: %s" % time - print "-------------------------------------------" + print("-------------------------------------------") + print("Time: %s" % time) + print("-------------------------------------------") for record in taken[:num]: - print record + print(record) if len(taken) > num: - print "..." - print + print("...") + print() self.foreachRDD(takeAndPrint) @@ -181,7 +185,7 @@ def mapValues(self, f): Return a new DStream by applying a map function to the value of each key-value pairs in this DStream without changing the key. """ - map_values_fn = lambda (k, v): (k, f(v)) + map_values_fn = lambda kv: (kv[0], f(kv[1])) return self.map(map_values_fn, preservesPartitioning=True) def flatMapValues(self, f): @@ -189,7 +193,7 @@ def flatMapValues(self, f): Return a new DStream by applying a flatmap function to the value of each key-value pairs in this DStream without changing the key. """ - flat_map_fn = lambda (k, v): ((k, x) for x in f(v)) + flat_map_fn = lambda kv: ((kv[0], x) for x in f(kv[1])) return self.flatMap(flat_map_fn, preservesPartitioning=True) def glom(self): @@ -286,10 +290,10 @@ def transform(self, func): `func` can have one argument of `rdd`, or have two arguments of (`time`, `rdd`) """ - if func.func_code.co_argcount == 1: + if func.__code__.co_argcount == 1: oldfunc = func func = lambda t, rdd: oldfunc(rdd) - assert func.func_code.co_argcount == 2, "func should take one or two arguments" + assert func.__code__.co_argcount == 2, "func should take one or two arguments" return TransformedDStream(self, func) def transformWith(self, func, other, keepSerializer=False): @@ -300,10 +304,10 @@ def transformWith(self, func, other, keepSerializer=False): `func` can have two arguments of (`rdd_a`, `rdd_b`) or have three arguments of (`time`, `rdd_a`, `rdd_b`) """ - if func.func_code.co_argcount == 2: + if func.__code__.co_argcount == 2: oldfunc = func func = lambda t, a, b: oldfunc(a, b) - assert func.func_code.co_argcount == 3, "func should take two or three arguments" + assert func.__code__.co_argcount == 3, "func should take two or three arguments" jfunc = TransformFunction(self._sc, func, self._jrdd_deserializer, other._jrdd_deserializer) dstream = self._sc._jvm.PythonTransformed2DStream(self._jdstream.dstream(), other._jdstream.dstream(), jfunc) @@ -460,7 +464,7 @@ def reduceByWindow(self, reduceFunc, invReduceFunc, windowDuration, slideDuratio keyed = self.map(lambda x: (1, x)) reduced = keyed.reduceByKeyAndWindow(reduceFunc, invReduceFunc, windowDuration, slideDuration, 1) - return reduced.map(lambda (k, v): v) + return reduced.map(lambda kv: kv[1]) def countByWindow(self, windowDuration, slideDuration): """ @@ -489,7 +493,7 @@ def countByValueAndWindow(self, windowDuration, slideDuration, numPartitions=Non keyed = self.map(lambda x: (x, 1)) counted = keyed.reduceByKeyAndWindow(operator.add, operator.sub, windowDuration, slideDuration, numPartitions) - return counted.filter(lambda (k, v): v > 0).count() + return counted.filter(lambda kv: kv[1] > 0).count() def groupByKeyAndWindow(self, windowDuration, slideDuration, numPartitions=None): """ @@ -548,7 +552,8 @@ def reduceFunc(t, a, b): def invReduceFunc(t, a, b): b = b.reduceByKey(func, numPartitions) joined = a.leftOuterJoin(b, numPartitions) - return joined.mapValues(lambda (v1, v2): invFunc(v1, v2) if v2 is not None else v1) + return joined.mapValues(lambda kv: invFunc(kv[0], kv[1]) + if kv[1] is not None else kv[0]) jreduceFunc = TransformFunction(self._sc, reduceFunc, reduced._jrdd_deserializer) if invReduceFunc: @@ -579,9 +584,9 @@ def reduceFunc(t, a, b): g = b.groupByKey(numPartitions).mapValues(lambda vs: (list(vs), None)) else: g = a.cogroup(b.partitionBy(numPartitions), numPartitions) - g = g.mapValues(lambda (va, vb): (list(vb), list(va)[0] if len(va) else None)) - state = g.mapValues(lambda (vs, s): updateFunc(vs, s)) - return state.filter(lambda (k, v): v is not None) + g = g.mapValues(lambda ab: (list(ab[1]), list(ab[0])[0] if len(ab[0]) else None)) + state = g.mapValues(lambda vs_s: updateFunc(vs_s[0], vs_s[1])) + return state.filter(lambda k_v: k_v[1] is not None) jreduceFunc = TransformFunction(self._sc, reduceFunc, self._sc.serializer, self._jrdd_deserializer) diff --git a/python/pyspark/streaming/kafka.py b/python/pyspark/streaming/kafka.py index f083ed149effb..7a7b6e1d9a527 100644 --- a/python/pyspark/streaming/kafka.py +++ b/python/pyspark/streaming/kafka.py @@ -67,10 +67,10 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, .loadClass("org.apache.spark.streaming.kafka.KafkaUtilsPythonHelper") helper = helperClass.newInstance() jstream = helper.createStream(ssc._jssc, jparam, jtopics, jlevel) - except Py4JJavaError, e: + except Py4JJavaError as e: # TODO: use --jar once it also work on driver if 'ClassNotFoundException' in str(e.java_exception): - print """ + print(""" ________________________________________________________________________________________________ Spark Streaming's Kafka libraries not found in class path. Try one of the following. @@ -88,8 +88,8 @@ def createStream(ssc, zkQuorum, groupId, topics, kafkaParams={}, ________________________________________________________________________________________________ -""" % (ssc.sparkContext.version, ssc.sparkContext.version) +""" % (ssc.sparkContext.version, ssc.sparkContext.version)) raise e ser = PairDeserializer(NoOpSerializer(), NoOpSerializer()) stream = DStream(jstream, ssc, ser) - return stream.map(lambda (k, v): (keyDecoder(k), valueDecoder(v))) + return stream.map(lambda k_v: (keyDecoder(k_v[0]), valueDecoder(k_v[1]))) diff --git a/python/pyspark/streaming/tests.py b/python/pyspark/streaming/tests.py index 9b4635e49020b..06d22154373bc 100644 --- a/python/pyspark/streaming/tests.py +++ b/python/pyspark/streaming/tests.py @@ -22,6 +22,7 @@ import unittest import tempfile import struct +from functools import reduce from py4j.java_collections import MapConverter @@ -51,7 +52,7 @@ def wait_for(self, result, n): while len(result) < n and time.time() - start_time < self.timeout: time.sleep(0.01) if len(result) < n: - print "timeout after", self.timeout + print("timeout after", self.timeout) def _take(self, dstream, n): """ @@ -131,7 +132,7 @@ def test_map(self): def func(dstream): return dstream.map(str) - expected = map(lambda x: map(str, x), input) + expected = [list(map(str, x)) for x in input] self._test_func(input, func, expected) def test_flatMap(self): @@ -140,8 +141,8 @@ def test_flatMap(self): def func(dstream): return dstream.flatMap(lambda x: (x, x * 2)) - expected = map(lambda x: list(chain.from_iterable((map(lambda y: [y, y * 2], x)))), - input) + expected = [list(chain.from_iterable((map(lambda y: [y, y * 2], x)))) + for x in input] self._test_func(input, func, expected) def test_filter(self): @@ -150,7 +151,7 @@ def test_filter(self): def func(dstream): return dstream.filter(lambda x: x % 2 == 0) - expected = map(lambda x: filter(lambda y: y % 2 == 0, x), input) + expected = [[y for y in x if y % 2 == 0] for x in input] self._test_func(input, func, expected) def test_count(self): @@ -159,7 +160,7 @@ def test_count(self): def func(dstream): return dstream.count() - expected = map(lambda x: [len(x)], input) + expected = [[len(x)] for x in input] self._test_func(input, func, expected) def test_reduce(self): @@ -168,7 +169,7 @@ def test_reduce(self): def func(dstream): return dstream.reduce(operator.add) - expected = map(lambda x: [reduce(operator.add, x)], input) + expected = [[reduce(operator.add, x)] for x in input] self._test_func(input, func, expected) def test_reduceByKey(self): @@ -185,27 +186,27 @@ def func(dstream): def test_mapValues(self): """Basic operation test for DStream.mapValues.""" input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], - [("", 4), (1, 1), (2, 2), (3, 3)], + [(0, 4), (1, 1), (2, 2), (3, 3)], [(1, 1), (2, 1), (3, 1), (4, 1)]] def func(dstream): return dstream.mapValues(lambda x: x + 10) expected = [[("a", 12), ("b", 12), ("c", 11), ("d", 11)], - [("", 14), (1, 11), (2, 12), (3, 13)], + [(0, 14), (1, 11), (2, 12), (3, 13)], [(1, 11), (2, 11), (3, 11), (4, 11)]] self._test_func(input, func, expected, sort=True) def test_flatMapValues(self): """Basic operation test for DStream.flatMapValues.""" input = [[("a", 2), ("b", 2), ("c", 1), ("d", 1)], - [("", 4), (1, 1), (2, 1), (3, 1)], + [(0, 4), (1, 1), (2, 1), (3, 1)], [(1, 1), (2, 1), (3, 1), (4, 1)]] def func(dstream): return dstream.flatMapValues(lambda x: (x, x + 10)) expected = [[("a", 2), ("a", 12), ("b", 2), ("b", 12), ("c", 1), ("c", 11), ("d", 1), ("d", 11)], - [("", 4), ("", 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)], + [(0, 4), (0, 14), (1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11)], [(1, 1), (1, 11), (2, 1), (2, 11), (3, 1), (3, 11), (4, 1), (4, 11)]] self._test_func(input, func, expected) @@ -233,7 +234,7 @@ def f(iterator): def test_countByValue(self): """Basic operation test for DStream.countByValue.""" - input = [range(1, 5) * 2, range(5, 7) + range(5, 9), ["a", "a", "b", ""]] + input = [list(range(1, 5)) * 2, list(range(5, 7)) + list(range(5, 9)), ["a", "a", "b", ""]] def func(dstream): return dstream.countByValue() @@ -285,7 +286,7 @@ def test_union(self): def func(d1, d2): return d1.union(d2) - expected = [range(6), range(6), range(6)] + expected = [list(range(6)), list(range(6)), list(range(6))] self._test_func(input1, func, expected, input2=input2) def test_cogroup(self): @@ -424,7 +425,7 @@ class StreamingContextTests(PySparkStreamingTestCase): duration = 0.1 def _add_input_stream(self): - inputs = map(lambda x: range(1, x), range(101)) + inputs = [range(1, x) for x in range(101)] stream = self.ssc.queueStream(inputs) self._collect(stream, 1, block=False) @@ -441,7 +442,7 @@ def test_stop_multiple_times(self): self.ssc.stop() def test_queue_stream(self): - input = [range(i + 1) for i in range(3)] + input = [list(range(i + 1)) for i in range(3)] dstream = self.ssc.queueStream(input) result = self._collect(dstream, 3) self.assertEqual(input, result) @@ -457,13 +458,13 @@ def test_text_file_stream(self): with open(os.path.join(d, name), "w") as f: f.writelines(["%d\n" % i for i in range(10)]) self.wait_for(result, 2) - self.assertEqual([range(10), range(10)], result) + self.assertEqual([list(range(10)), list(range(10))], result) def test_binary_records_stream(self): d = tempfile.mkdtemp() self.ssc = StreamingContext(self.sc, self.duration) dstream = self.ssc.binaryRecordsStream(d, 10).map( - lambda v: struct.unpack("10b", str(v))) + lambda v: struct.unpack("10b", bytes(v))) result = self._collect(dstream, 2, block=False) self.ssc.start() for name in ('a', 'b'): @@ -471,10 +472,10 @@ def test_binary_records_stream(self): with open(os.path.join(d, name), "wb") as f: f.write(bytearray(range(10))) self.wait_for(result, 2) - self.assertEqual([range(10), range(10)], map(lambda v: list(v[0]), result)) + self.assertEqual([list(range(10)), list(range(10))], [list(v[0]) for v in result]) def test_union(self): - input = [range(i + 1) for i in range(3)] + input = [list(range(i + 1)) for i in range(3)] dstream = self.ssc.queueStream(input) dstream2 = self.ssc.queueStream(input) dstream3 = self.ssc.union(dstream, dstream2) diff --git a/python/pyspark/streaming/util.py b/python/pyspark/streaming/util.py index 86ee5aa04f252..34291f30a5652 100644 --- a/python/pyspark/streaming/util.py +++ b/python/pyspark/streaming/util.py @@ -91,9 +91,9 @@ def dumps(self, id): except Exception: traceback.print_exc() - def loads(self, bytes): + def loads(self, data): try: - f, deserializers = self.serializer.loads(str(bytes)) + f, deserializers = self.serializer.loads(bytes(data)) return TransformFunction(self.ctx, f, *deserializers) except Exception: traceback.print_exc() @@ -116,7 +116,7 @@ def rddToFileName(prefix, suffix, timestamp): """ if isinstance(timestamp, datetime): seconds = time.mktime(timestamp.timetuple()) - timestamp = long(seconds * 1000) + timestamp.microsecond / 1000 + timestamp = int(seconds * 1000) + timestamp.microsecond // 1000 if suffix is None: return prefix + "-" + str(timestamp) else: diff --git a/python/pyspark/tests.py b/python/pyspark/tests.py index ee67e80d539f8..75f39d9e75f38 100644 --- a/python/pyspark/tests.py +++ b/python/pyspark/tests.py @@ -19,8 +19,8 @@ Unit tests for PySpark; additional tests are implemented as doctests in individual modules. """ + from array import array -from fileinput import input from glob import glob import os import re @@ -45,6 +45,9 @@ sys.exit(1) else: import unittest + if sys.version_info[0] >= 3: + xrange = range + basestring = str from pyspark.conf import SparkConf @@ -52,7 +55,9 @@ from pyspark.rdd import RDD from pyspark.files import SparkFiles from pyspark.serializers import read_int, BatchedSerializer, MarshalSerializer, PickleSerializer, \ - CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer + CloudPickleSerializer, CompressedSerializer, UTF8Deserializer, NoOpSerializer, \ + PairDeserializer, CartesianDeserializer, AutoBatchedSerializer, AutoSerializer, \ + FlattenedValuesSerializer from pyspark.shuffle import Aggregator, InMemoryMerger, ExternalMerger, ExternalSorter from pyspark import shuffle from pyspark.profiler import BasicProfiler @@ -81,7 +86,7 @@ class MergerTests(unittest.TestCase): def setUp(self): self.N = 1 << 12 self.l = [i for i in xrange(self.N)] - self.data = zip(self.l, self.l) + self.data = list(zip(self.l, self.l)) self.agg = Aggregator(lambda x: [x], lambda x, y: x.append(y) or x, lambda x, y: x.extend(y) or x) @@ -89,45 +94,45 @@ def setUp(self): def test_in_memory(self): m = InMemoryMerger(self.agg) m.mergeValues(self.data) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = InMemoryMerger(self.agg) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + m.mergeCombiners(map(lambda x_y: (x_y[0], [x_y[1]]), self.data)) + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) def test_small_dataset(self): m = ExternalMerger(self.agg, 1000) m.mergeValues(self.data) self.assertEqual(m.spills, 0) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = ExternalMerger(self.agg, 1000) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data)) + m.mergeCombiners(map(lambda x_y1: (x_y1[0], [x_y1[1]]), self.data)) self.assertEqual(m.spills, 0) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) def test_medium_dataset(self): - m = ExternalMerger(self.agg, 30) + m = ExternalMerger(self.agg, 20) m.mergeValues(self.data) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N))) m = ExternalMerger(self.agg, 10) - m.mergeCombiners(map(lambda (x, y): (x, [y]), self.data * 3)) + m.mergeCombiners(map(lambda x_y2: (x_y2[0], [x_y2[1]]), self.data * 3)) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(sum(v) for k, v in m.iteritems()), + self.assertEqual(sum(sum(v) for k, v in m.items()), sum(xrange(self.N)) * 3) def test_huge_dataset(self): - m = ExternalMerger(self.agg, 10, partitions=3) - m.mergeCombiners(map(lambda (k, v): (k, [str(v)]), self.data * 10)) + m = ExternalMerger(self.agg, 5, partitions=3) + m.mergeCombiners(map(lambda k_v: (k_v[0], [str(k_v[1])]), self.data * 10)) self.assertTrue(m.spills >= 1) - self.assertEqual(sum(len(v) for k, v in m.iteritems()), + self.assertEqual(sum(len(v) for k, v in m.items()), self.N * 10) m._cleanup() @@ -144,55 +149,55 @@ def gen_gs(N, step=1): self.assertEqual(1, len(list(gen_gs(1)))) self.assertEqual(2, len(list(gen_gs(2)))) self.assertEqual(100, len(list(gen_gs(100)))) - self.assertEqual(range(1, 101), [k for k, _ in gen_gs(100)]) - self.assertTrue(all(range(k) == list(vs) for k, vs in gen_gs(100))) + self.assertEqual(list(range(1, 101)), [k for k, _ in gen_gs(100)]) + self.assertTrue(all(list(range(k)) == list(vs) for k, vs in gen_gs(100))) for k, vs in gen_gs(50002, 10000): self.assertEqual(k, len(vs)) - self.assertEqual(range(k), list(vs)) + self.assertEqual(list(range(k)), list(vs)) ser = PickleSerializer() l = ser.loads(ser.dumps(list(gen_gs(50002, 30000)))) for k, vs in l: self.assertEqual(k, len(vs)) - self.assertEqual(range(k), list(vs)) + self.assertEqual(list(range(k)), list(vs)) class SorterTests(unittest.TestCase): def test_in_memory_sort(self): - l = range(1024) + l = list(range(1024)) random.shuffle(l) sorter = ExternalSorter(1024) - self.assertEquals(sorted(l), list(sorter.sorted(l))) - self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) - self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) - self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), - list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + self.assertEqual(sorted(l), list(sorter.sorted(l))) + self.assertEqual(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertEqual(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertEqual(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) def test_external_sort(self): - l = range(1024) + l = list(range(1024)) random.shuffle(l) sorter = ExternalSorter(1) - self.assertEquals(sorted(l), list(sorter.sorted(l))) + self.assertEqual(sorted(l), list(sorter.sorted(l))) self.assertGreater(shuffle.DiskBytesSpilled, 0) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) + self.assertEqual(sorted(l, reverse=True), list(sorter.sorted(l, reverse=True))) self.assertGreater(shuffle.DiskBytesSpilled, last) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) + self.assertEqual(sorted(l, key=lambda x: -x), list(sorter.sorted(l, key=lambda x: -x))) self.assertGreater(shuffle.DiskBytesSpilled, last) last = shuffle.DiskBytesSpilled - self.assertEquals(sorted(l, key=lambda x: -x, reverse=True), - list(sorter.sorted(l, key=lambda x: -x, reverse=True))) + self.assertEqual(sorted(l, key=lambda x: -x, reverse=True), + list(sorter.sorted(l, key=lambda x: -x, reverse=True))) self.assertGreater(shuffle.DiskBytesSpilled, last) def test_external_sort_in_rdd(self): conf = SparkConf().set("spark.python.worker.memory", "1m") sc = SparkContext(conf=conf) - l = range(10240) + l = list(range(10240)) random.shuffle(l) - rdd = sc.parallelize(l, 10) - self.assertEquals(sorted(l), rdd.sortBy(lambda x: x).collect()) + rdd = sc.parallelize(l, 2) + self.assertEqual(sorted(l), rdd.sortBy(lambda x: x).collect()) sc.stop() @@ -200,11 +205,11 @@ class SerializationTestCase(unittest.TestCase): def test_namedtuple(self): from collections import namedtuple - from cPickle import dumps, loads + from pickle import dumps, loads P = namedtuple("P", "x y") p1 = P(1, 3) p2 = loads(dumps(p1, 2)) - self.assertEquals(p1, p2) + self.assertEqual(p1, p2) def test_itemgetter(self): from operator import itemgetter @@ -246,7 +251,7 @@ def test_pickling_file_handles(self): ser = CloudPickleSerializer() out1 = sys.stderr out2 = ser.loads(ser.dumps(out1)) - self.assertEquals(out1, out2) + self.assertEqual(out1, out2) def test_func_globals(self): @@ -263,19 +268,36 @@ def __reduce__(self): def foo(): sys.exit(0) - self.assertTrue("exit" in foo.func_code.co_names) + self.assertTrue("exit" in foo.__code__.co_names) ser.dumps(foo) def test_compressed_serializer(self): ser = CompressedSerializer(PickleSerializer()) - from StringIO import StringIO + try: + from StringIO import StringIO + except ImportError: + from io import BytesIO as 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))) ser.dump_stream(range(1000), io) io.seek(0) - self.assertEqual(["abc", u"123", range(5)] + range(1000), list(ser.load_stream(io))) + self.assertEqual(["abc", u"123", range(5)] + list(range(1000)), list(ser.load_stream(io))) + io.close() + + def test_hash_serializer(self): + hash(NoOpSerializer()) + hash(UTF8Deserializer()) + hash(PickleSerializer()) + hash(MarshalSerializer()) + hash(AutoSerializer()) + hash(BatchedSerializer(PickleSerializer())) + hash(AutoBatchedSerializer(MarshalSerializer())) + hash(PairDeserializer(NoOpSerializer(), UTF8Deserializer())) + hash(CartesianDeserializer(NoOpSerializer(), UTF8Deserializer())) + hash(CompressedSerializer(PickleSerializer())) + hash(FlattenedValuesSerializer(PickleSerializer())) class PySparkTestCase(unittest.TestCase): @@ -340,7 +362,7 @@ def test_checkpoint_and_restore(self): self.assertTrue(flatMappedRDD.getCheckpointFile() is not None) recovered = self.sc._checkpointFile(flatMappedRDD.getCheckpointFile(), flatMappedRDD._jrdd_deserializer) - self.assertEquals([1, 2, 3, 4], recovered.collect()) + self.assertEqual([1, 2, 3, 4], recovered.collect()) class AddFileTests(PySparkTestCase): @@ -356,8 +378,7 @@ def test_add_py_file(self): def func(x): from userlibrary import UserClass return UserClass().hello() - self.assertRaises(Exception, - self.sc.parallelize(range(2)).map(func).first) + self.assertRaises(Exception, self.sc.parallelize(range(2)).map(func).first) log4j.LogManager.getRootLogger().setLevel(old_level) # Add the file, so the job should now succeed: @@ -372,7 +393,7 @@ def test_add_file_locally(self): download_path = SparkFiles.get("hello.txt") self.assertNotEqual(path, download_path) with open(download_path) as test_file: - self.assertEquals("Hello World!\n", test_file.readline()) + self.assertEqual("Hello World!\n", test_file.readline()) def test_add_py_file_locally(self): # To ensure that we're actually testing addPyFile's effects, check that @@ -381,7 +402,7 @@ def func(): from userlibrary import UserClass self.assertRaises(ImportError, func) path = os.path.join(SPARK_HOME, "python/test_support/userlibrary.py") - self.sc.addFile(path) + self.sc.addPyFile(path) from userlibrary import UserClass self.assertEqual("Hello World!", UserClass().hello()) @@ -391,7 +412,7 @@ def test_add_egg_file_locally(self): def func(): from userlib import UserClass self.assertRaises(ImportError, func) - path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1-py2.7.egg") + path = os.path.join(SPARK_HOME, "python/test_support/userlib-0.1.zip") self.sc.addPyFile(path) from userlib import UserClass self.assertEqual("Hello World from inside a package!", UserClass().hello()) @@ -427,8 +448,9 @@ def test_save_as_textfile_with_unicode(self): tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsTextFile(tempFile.name) - raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) - self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + raw_contents = b''.join(open(p, 'rb').read() + for p in glob(tempFile.name + "/part-0000*")) + self.assertEqual(x, raw_contents.strip().decode("utf-8")) def test_save_as_textfile_with_utf8(self): x = u"\u00A1Hola, mundo!" @@ -436,19 +458,20 @@ def test_save_as_textfile_with_utf8(self): tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsTextFile(tempFile.name) - raw_contents = ''.join(input(glob(tempFile.name + "/part-0000*"))) - self.assertEqual(x, unicode(raw_contents.strip(), "utf-8")) + raw_contents = b''.join(open(p, 'rb').read() + for p in glob(tempFile.name + "/part-0000*")) + self.assertEqual(x, raw_contents.strip().decode('utf8')) def test_transforming_cartesian_result(self): # Regression test for SPARK-1034 rdd1 = self.sc.parallelize([1, 2]) rdd2 = self.sc.parallelize([3, 4]) cart = rdd1.cartesian(rdd2) - result = cart.map(lambda (x, y): x + y).collect() + result = cart.map(lambda x_y3: x_y3[0] + x_y3[1]).collect() def test_transforming_pickle_file(self): # Regression test for SPARK-2601 - data = self.sc.parallelize(["Hello", "World!"]) + data = self.sc.parallelize([u"Hello", u"World!"]) tempFile = tempfile.NamedTemporaryFile(delete=True) tempFile.close() data.saveAsPickleFile(tempFile.name) @@ -461,13 +484,13 @@ def test_cartesian_on_textfile(self): a = self.sc.textFile(path) result = a.cartesian(a).collect() (x, y) = result[0] - self.assertEqual("Hello World!", x.strip()) - self.assertEqual("Hello World!", y.strip()) + self.assertEqual(u"Hello World!", x.strip()) + self.assertEqual(u"Hello World!", y.strip()) def test_deleting_input_files(self): # Regression test for SPARK-1025 tempFile = tempfile.NamedTemporaryFile(delete=False) - tempFile.write("Hello World!") + tempFile.write(b"Hello World!") tempFile.close() data = self.sc.textFile(tempFile.name) filtered_data = data.filter(lambda x: True) @@ -510,21 +533,21 @@ def test_namedtuple_in_rdd(self): jon = Person(1, "Jon", "Doe") jane = Person(2, "Jane", "Doe") theDoes = self.sc.parallelize([jon, jane]) - self.assertEquals([jon, jane], theDoes.collect()) + self.assertEqual([jon, jane], theDoes.collect()) def test_large_broadcast(self): N = 100000 data = [[float(i) for i in range(300)] for i in range(N)] bdata = self.sc.broadcast(data) # 270MB m = self.sc.parallelize(range(1), 1).map(lambda x: len(bdata.value)).sum() - self.assertEquals(N, m) + self.assertEqual(N, m) def test_multiple_broadcasts(self): N = 1 << 21 b1 = self.sc.broadcast(set(range(N))) # multiple blocks in JVM - r = range(1 << 15) + r = list(range(1 << 15)) random.shuffle(r) - s = str(r) + s = str(r).encode() checksum = hashlib.md5(s).hexdigest() b2 = self.sc.broadcast(s) r = list(set(self.sc.parallelize(range(10), 10).map( @@ -535,7 +558,7 @@ def test_multiple_broadcasts(self): self.assertEqual(checksum, csum) random.shuffle(r) - s = str(r) + s = str(r).encode() checksum = hashlib.md5(s).hexdigest() b2 = self.sc.broadcast(s) r = list(set(self.sc.parallelize(range(10), 10).map( @@ -549,7 +572,7 @@ def test_large_closure(self): N = 1000000 data = [float(i) for i in xrange(N)] rdd = self.sc.parallelize(range(1), 1).map(lambda x: len(data)) - self.assertEquals(N, rdd.first()) + self.assertEqual(N, rdd.first()) # regression test for SPARK-6886 self.assertEqual(1, rdd.map(lambda x: (x, 1)).groupByKey().count()) @@ -590,15 +613,15 @@ def test_zip_with_different_number_of_items(self): # same total number of items, but different distributions a = self.sc.parallelize([2, 3], 2).flatMap(range) b = self.sc.parallelize([3, 2], 2).flatMap(range) - self.assertEquals(a.count(), b.count()) + self.assertEqual(a.count(), b.count()) self.assertRaises(Exception, lambda: a.zip(b).count()) def test_count_approx_distinct(self): rdd = self.sc.parallelize(range(1000)) - self.assertTrue(950 < rdd.countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.04) < 1050) - self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.04) < 1050) + self.assertTrue(950 < rdd.countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(float).countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(str).countApproxDistinct(0.03) < 1050) + self.assertTrue(950 < rdd.map(lambda x: (x, -x)).countApproxDistinct(0.03) < 1050) rdd = self.sc.parallelize([i % 20 for i in range(1000)], 7) self.assertTrue(18 < rdd.countApproxDistinct() < 22) @@ -612,59 +635,59 @@ def test_count_approx_distinct(self): def test_histogram(self): # empty rdd = self.sc.parallelize([]) - self.assertEquals([0], rdd.histogram([0, 10])[1]) - self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([0], rdd.histogram([0, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1]) self.assertRaises(ValueError, lambda: rdd.histogram(1)) # out of range rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0], rdd.histogram([0, 10])[1]) - self.assertEquals([0, 0], rdd.histogram((0, 4, 10))[1]) + self.assertEqual([0], rdd.histogram([0, 10])[1]) + self.assertEqual([0, 0], rdd.histogram((0, 4, 10))[1]) # in range with one bucket rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals([4], rdd.histogram([0, 10])[1]) - self.assertEquals([3, 1], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([4], rdd.histogram([0, 10])[1]) + self.assertEqual([3, 1], rdd.histogram([0, 4, 10])[1]) # in range with one bucket exact match - self.assertEquals([4], rdd.histogram([1, 4])[1]) + self.assertEqual([4], rdd.histogram([1, 4])[1]) # out of range with two buckets rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0, 0], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 5, 10])[1]) # out of range with two uneven buckets rdd = self.sc.parallelize([10.01, -0.01]) - self.assertEquals([0, 0], rdd.histogram([0, 4, 10])[1]) + self.assertEqual([0, 0], rdd.histogram([0, 4, 10])[1]) # in range with two buckets rdd = self.sc.parallelize([1, 2, 3, 5, 6]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1]) # in range with two bucket and None rdd = self.sc.parallelize([1, 2, 3, 5, 6, None, float('nan')]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 10])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 10])[1]) # in range with two uneven buckets rdd = self.sc.parallelize([1, 2, 3, 5, 6]) - self.assertEquals([3, 2], rdd.histogram([0, 5, 11])[1]) + self.assertEqual([3, 2], rdd.histogram([0, 5, 11])[1]) # mixed range with two uneven buckets rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.0, 11.01]) - self.assertEquals([4, 3], rdd.histogram([0, 5, 11])[1]) + self.assertEqual([4, 3], rdd.histogram([0, 5, 11])[1]) # mixed range with four uneven buckets rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1]) - self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) # mixed range with uneven buckets and NaN rdd = self.sc.parallelize([-0.01, 0.0, 1, 2, 3, 5, 6, 11.01, 12.0, 199.0, 200.0, 200.1, None, float('nan')]) - self.assertEquals([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) + self.assertEqual([4, 2, 1, 3], rdd.histogram([0.0, 5.0, 11.0, 12.0, 200.0])[1]) # out of range with infinite buckets rdd = self.sc.parallelize([10.01, -0.01, float('nan'), float("inf")]) - self.assertEquals([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) + self.assertEqual([1, 2], rdd.histogram([float('-inf'), 0, float('inf')])[1]) # invalid buckets self.assertRaises(ValueError, lambda: rdd.histogram([])) @@ -674,25 +697,25 @@ def test_histogram(self): # without buckets rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals(([1, 4], [4]), rdd.histogram(1)) + self.assertEqual(([1, 4], [4]), rdd.histogram(1)) # without buckets single element rdd = self.sc.parallelize([1]) - self.assertEquals(([1, 1], [1]), rdd.histogram(1)) + self.assertEqual(([1, 1], [1]), rdd.histogram(1)) # without bucket no range rdd = self.sc.parallelize([1] * 4) - self.assertEquals(([1, 1], [4]), rdd.histogram(1)) + self.assertEqual(([1, 1], [4]), rdd.histogram(1)) # without buckets basic two rdd = self.sc.parallelize(range(1, 5)) - self.assertEquals(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) + self.assertEqual(([1, 2.5, 4], [2, 2]), rdd.histogram(2)) # without buckets with more requested than elements rdd = self.sc.parallelize([1, 2]) buckets = [1 + 0.2 * i for i in range(6)] hist = [1, 0, 0, 0, 1] - self.assertEquals((buckets, hist), rdd.histogram(5)) + self.assertEqual((buckets, hist), rdd.histogram(5)) # invalid RDDs rdd = self.sc.parallelize([1, float('inf')]) @@ -702,15 +725,8 @@ def test_histogram(self): # string rdd = self.sc.parallelize(["ab", "ac", "b", "bd", "ef"], 2) - self.assertEquals([2, 2], rdd.histogram(["a", "b", "c"])[1]) - self.assertEquals((["ab", "ef"], [5]), rdd.histogram(1)) - self.assertRaises(TypeError, lambda: rdd.histogram(2)) - - # mixed RDD - rdd = self.sc.parallelize([1, 4, "ab", "ac", "b"], 2) - self.assertEquals([1, 1], rdd.histogram([0, 4, 10])[1]) - self.assertEquals([2, 1], rdd.histogram(["a", "b", "c"])[1]) - self.assertEquals(([1, "b"], [5]), rdd.histogram(1)) + self.assertEqual([2, 2], rdd.histogram(["a", "b", "c"])[1]) + self.assertEqual((["ab", "ef"], [5]), rdd.histogram(1)) self.assertRaises(TypeError, lambda: rdd.histogram(2)) def test_repartitionAndSortWithinPartitions(self): @@ -718,31 +734,31 @@ def test_repartitionAndSortWithinPartitions(self): repartitioned = rdd.repartitionAndSortWithinPartitions(2, lambda key: key % 2) partitions = repartitioned.glom().collect() - self.assertEquals(partitions[0], [(0, 5), (0, 8), (2, 6)]) - self.assertEquals(partitions[1], [(1, 3), (3, 8), (3, 8)]) + self.assertEqual(partitions[0], [(0, 5), (0, 8), (2, 6)]) + self.assertEqual(partitions[1], [(1, 3), (3, 8), (3, 8)]) def test_distinct(self): rdd = self.sc.parallelize((1, 2, 3)*10, 10) - self.assertEquals(rdd.getNumPartitions(), 10) - self.assertEquals(rdd.distinct().count(), 3) + self.assertEqual(rdd.getNumPartitions(), 10) + self.assertEqual(rdd.distinct().count(), 3) result = rdd.distinct(5) - self.assertEquals(result.getNumPartitions(), 5) - self.assertEquals(result.count(), 3) + self.assertEqual(result.getNumPartitions(), 5) + self.assertEqual(result.count(), 3) def test_external_group_by_key(self): - self.sc._conf.set("spark.python.worker.memory", "5m") + self.sc._conf.set("spark.python.worker.memory", "1m") N = 200001 kv = self.sc.parallelize(range(N)).map(lambda x: (x % 3, x)) gkv = kv.groupByKey().cache() self.assertEqual(3, gkv.count()) - filtered = gkv.filter(lambda (k, vs): k == 1) + filtered = gkv.filter(lambda kv: kv[0] == 1) self.assertEqual(1, filtered.count()) - self.assertEqual([(1, N/3)], filtered.mapValues(len).collect()) - self.assertEqual([(N/3, N/3)], + self.assertEqual([(1, N // 3)], filtered.mapValues(len).collect()) + self.assertEqual([(N // 3, N // 3)], filtered.values().map(lambda x: (len(x), len(list(x)))).collect()) result = filtered.collect()[0][1] - self.assertEqual(N/3, len(result)) - self.assertTrue(isinstance(result.data, shuffle.ExternalList)) + self.assertEqual(N // 3, len(result)) + self.assertTrue(isinstance(result.data, shuffle.ExternalListOfList)) def test_sort_on_empty_rdd(self): self.assertEqual([], self.sc.parallelize(zip([], [])).sortByKey().collect()) @@ -767,7 +783,7 @@ def test_null_in_rdd(self): rdd = RDD(jrdd, self.sc, UTF8Deserializer()) self.assertEqual([u"a", None, u"b"], rdd.collect()) rdd = RDD(jrdd, self.sc, NoOpSerializer()) - self.assertEqual(["a", None, "b"], rdd.collect()) + self.assertEqual([b"a", None, b"b"], rdd.collect()) def test_multiple_python_java_RDD_conversions(self): # Regression test for SPARK-5361 @@ -813,14 +829,14 @@ def test_narrow_dependency_in_join(self): self.sc.setJobGroup("test3", "test", True) d = sorted(parted.cogroup(parted).collect()) self.assertEqual(10, len(d)) - self.assertEqual([[0], [0]], map(list, d[0][1])) + self.assertEqual([[0], [0]], list(map(list, d[0][1]))) jobId = tracker.getJobIdsForGroup("test3")[0] self.assertEqual(2, len(tracker.getJobInfo(jobId).stageIds)) self.sc.setJobGroup("test4", "test", True) d = sorted(parted.cogroup(rdd).collect()) self.assertEqual(10, len(d)) - self.assertEqual([[0], [0]], map(list, d[0][1])) + self.assertEqual([[0], [0]], list(map(list, d[0][1]))) jobId = tracker.getJobIdsForGroup("test4")[0] self.assertEqual(3, len(tracker.getJobInfo(jobId).stageIds)) @@ -906,6 +922,7 @@ def tearDownClass(cls): ReusedPySparkTestCase.tearDownClass() shutil.rmtree(cls.tempdir.name) + @unittest.skipIf(sys.version >= "3", "serialize array of byte") def test_sequencefiles(self): basepath = self.tempdir.name ints = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfint/", @@ -954,15 +971,16 @@ def test_sequencefiles(self): en = [(1, None), (1, None), (2, None), (2, None), (2, None), (3, None)] self.assertEqual(nulls, en) - maps = sorted(self.sc.sequenceFile(basepath + "/sftestdata/sfmap/", - "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.MapWritable").collect()) + maps = self.sc.sequenceFile(basepath + "/sftestdata/sfmap/", + "org.apache.hadoop.io.IntWritable", + "org.apache.hadoop.io.MapWritable").collect() em = [(1, {}), (1, {3.0: u'bb'}), (2, {1.0: u'aa'}), (2, {1.0: u'cc'}), (3, {2.0: u'dd'})] - self.assertEqual(maps, em) + for v in maps: + self.assertTrue(v in em) # arrays get pickled to tuples by default tuples = sorted(self.sc.sequenceFile( @@ -1089,8 +1107,8 @@ def test_converters(self): def test_binary_files(self): path = os.path.join(self.tempdir.name, "binaryfiles") os.mkdir(path) - data = "short binary data" - with open(os.path.join(path, "part-0000"), 'w') as f: + data = b"short binary data" + with open(os.path.join(path, "part-0000"), 'wb') as f: f.write(data) [(p, d)] = self.sc.binaryFiles(path).collect() self.assertTrue(p.endswith("part-0000")) @@ -1103,7 +1121,7 @@ def test_binary_records(self): for i in range(100): f.write('%04d' % i) result = self.sc.binaryRecords(path, 4).map(int).collect() - self.assertEqual(range(100), result) + self.assertEqual(list(range(100)), result) class OutputFormatTests(ReusedPySparkTestCase): @@ -1115,6 +1133,7 @@ def setUp(self): def tearDown(self): shutil.rmtree(self.tempdir.name, ignore_errors=True) + @unittest.skipIf(sys.version >= "3", "serialize array of byte") def test_sequencefiles(self): basepath = self.tempdir.name ei = [(1, u'aa'), (1, u'aa'), (2, u'aa'), (2, u'bb'), (2, u'bb'), (3, u'cc')] @@ -1155,8 +1174,9 @@ def test_sequencefiles(self): (2, {1.0: u'cc'}), (3, {2.0: u'dd'})] self.sc.parallelize(em).saveAsSequenceFile(basepath + "/sfmap/") - maps = sorted(self.sc.sequenceFile(basepath + "/sfmap/").collect()) - self.assertEqual(maps, em) + maps = self.sc.sequenceFile(basepath + "/sfmap/").collect() + for v in maps: + self.assertTrue(v, em) def test_oldhadoop(self): basepath = self.tempdir.name @@ -1168,12 +1188,13 @@ def test_oldhadoop(self): "org.apache.hadoop.mapred.SequenceFileOutputFormat", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable") - result = sorted(self.sc.hadoopFile( + result = self.sc.hadoopFile( basepath + "/oldhadoop/", "org.apache.hadoop.mapred.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", - "org.apache.hadoop.io.MapWritable").collect()) - self.assertEqual(result, dict_data) + "org.apache.hadoop.io.MapWritable").collect() + for v in result: + self.assertTrue(v, dict_data) conf = { "mapred.output.format.class": "org.apache.hadoop.mapred.SequenceFileOutputFormat", @@ -1183,12 +1204,13 @@ def test_oldhadoop(self): } self.sc.parallelize(dict_data).saveAsHadoopDataset(conf) input_conf = {"mapred.input.dir": basepath + "/olddataset/"} - old_dataset = sorted(self.sc.hadoopRDD( + result = self.sc.hadoopRDD( "org.apache.hadoop.mapred.SequenceFileInputFormat", "org.apache.hadoop.io.IntWritable", "org.apache.hadoop.io.MapWritable", - conf=input_conf).collect()) - self.assertEqual(old_dataset, dict_data) + conf=input_conf).collect() + for v in result: + self.assertTrue(v, dict_data) def test_newhadoop(self): basepath = self.tempdir.name @@ -1223,6 +1245,7 @@ def test_newhadoop(self): conf=input_conf).collect()) self.assertEqual(new_dataset, data) + @unittest.skipIf(sys.version >= "3", "serialize of array") def test_newhadoop_with_array(self): basepath = self.tempdir.name # use custom ArrayWritable types and converters to handle arrays @@ -1303,7 +1326,7 @@ def test_reserialization(self): basepath = self.tempdir.name x = range(1, 5) y = range(1001, 1005) - data = zip(x, y) + data = list(zip(x, y)) rdd = self.sc.parallelize(x).zip(self.sc.parallelize(y)) rdd.saveAsSequenceFile(basepath + "/reserialize/sequence") result1 = sorted(self.sc.sequenceFile(basepath + "/reserialize/sequence").collect()) @@ -1354,7 +1377,7 @@ def connect(self, port): sock = socket(AF_INET, SOCK_STREAM) sock.connect(('127.0.0.1', port)) # send a split index of -1 to shutdown the worker - sock.send("\xFF\xFF\xFF\xFF") + sock.send(b"\xFF\xFF\xFF\xFF") sock.close() return True @@ -1395,7 +1418,6 @@ def test_termination_sigterm(self): class WorkerTests(PySparkTestCase): - def test_cancel_task(self): temp = tempfile.NamedTemporaryFile(delete=True) temp.close() @@ -1410,7 +1432,7 @@ def sleep(x): # start job in background thread def run(): - self.sc.parallelize(range(1)).foreach(sleep) + self.sc.parallelize(range(1), 1).foreach(sleep) import threading t = threading.Thread(target=run) t.daemon = True @@ -1419,7 +1441,8 @@ def run(): daemon_pid, worker_pid = 0, 0 while True: if os.path.exists(path): - data = open(path).read().split(' ') + with open(path) as f: + data = f.read().split(' ') daemon_pid, worker_pid = map(int, data) break time.sleep(0.1) @@ -1455,7 +1478,7 @@ def raise_exception(_): def test_after_jvm_exception(self): tempFile = tempfile.NamedTemporaryFile(delete=False) - tempFile.write("Hello World!") + tempFile.write(b"Hello World!") tempFile.close() data = self.sc.textFile(tempFile.name, 1) filtered_data = data.filter(lambda x: True) @@ -1577,12 +1600,12 @@ def test_single_script(self): |from pyspark import SparkContext | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect() + |print(sc.parallelize([1, 2, 3]).map(lambda x: x * 2).collect()) """) proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 4, 6]", out) + self.assertIn("[2, 4, 6]", out.decode('utf-8')) def test_script_with_local_functions(self): """Submit and test a single script file calling a global function""" @@ -1593,12 +1616,12 @@ def test_script_with_local_functions(self): | return x * 3 | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(foo).collect() + |print(sc.parallelize([1, 2, 3]).map(foo).collect()) """) proc = subprocess.Popen([self.sparkSubmit, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[3, 6, 9]", out) + self.assertIn("[3, 6, 9]", out.decode('utf-8')) def test_module_dependency(self): """Submit and test a script with a dependency on another module""" @@ -1607,7 +1630,7 @@ def test_module_dependency(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) zip = self.createFileInZip("mylib.py", """ |def myfunc(x): @@ -1617,7 +1640,7 @@ def test_module_dependency(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_module_dependency_on_cluster(self): """Submit and test a script with a dependency on another module on a cluster""" @@ -1626,7 +1649,7 @@ def test_module_dependency_on_cluster(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) zip = self.createFileInZip("mylib.py", """ |def myfunc(x): @@ -1637,7 +1660,7 @@ def test_module_dependency_on_cluster(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_package_dependency(self): """Submit and test a script with a dependency on a Spark Package""" @@ -1646,14 +1669,14 @@ def test_package_dependency(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) self.create_spark_package("a:mylib:0.1") proc = subprocess.Popen([self.sparkSubmit, "--packages", "a:mylib:0.1", "--repositories", "file:" + self.programDir, script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_package_dependency_on_cluster(self): """Submit and test a script with a dependency on a Spark Package on a cluster""" @@ -1662,7 +1685,7 @@ def test_package_dependency_on_cluster(self): |from mylib import myfunc | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(myfunc).collect() + |print(sc.parallelize([1, 2, 3]).map(myfunc).collect()) """) self.create_spark_package("a:mylib:0.1") proc = subprocess.Popen([self.sparkSubmit, "--packages", "a:mylib:0.1", "--repositories", @@ -1670,7 +1693,7 @@ def test_package_dependency_on_cluster(self): "local-cluster[1,1,512]", script], stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 3, 4]", out) + self.assertIn("[2, 3, 4]", out.decode('utf-8')) def test_single_script_on_cluster(self): """Submit and test a single script on a cluster""" @@ -1681,7 +1704,7 @@ def test_single_script_on_cluster(self): | return x * 2 | |sc = SparkContext() - |print sc.parallelize([1, 2, 3]).map(foo).collect() + |print(sc.parallelize([1, 2, 3]).map(foo).collect()) """) # this will fail if you have different spark.executor.memory # in conf/spark-defaults.conf @@ -1690,7 +1713,7 @@ def test_single_script_on_cluster(self): stdout=subprocess.PIPE) out, err = proc.communicate() self.assertEqual(0, proc.returncode) - self.assertIn("[2, 4, 6]", out) + self.assertIn("[2, 4, 6]", out.decode('utf-8')) class ContextTests(unittest.TestCase): @@ -1765,7 +1788,7 @@ class SciPyTests(PySparkTestCase): def test_serialize(self): from scipy.special import gammaln x = range(1, 5) - expected = map(gammaln, x) + expected = list(map(gammaln, x)) observed = self.sc.parallelize(x).map(gammaln).collect() self.assertEqual(expected, observed) @@ -1786,11 +1809,11 @@ def test_statcounter_array(self): if __name__ == "__main__": if not _have_scipy: - print "NOTE: Skipping SciPy tests as it does not seem to be installed" + print("NOTE: Skipping SciPy tests as it does not seem to be installed") if not _have_numpy: - print "NOTE: Skipping NumPy tests as it does not seem to be installed" + print("NOTE: Skipping NumPy tests as it does not seem to be installed") unittest.main() if not _have_scipy: - print "NOTE: SciPy tests were skipped as it does not seem to be installed" + print("NOTE: SciPy tests were skipped as it does not seem to be installed") if not _have_numpy: - print "NOTE: NumPy tests were skipped as it does not seem to be installed" + print("NOTE: NumPy tests were skipped as it does not seem to be installed") diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 452d6fabdcc17..fbdaf3a5814cd 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -18,6 +18,7 @@ """ Worker that receives input from Piped RDD. """ +from __future__ import print_function import os import sys import time @@ -37,9 +38,9 @@ def report_times(outfile, boot, init, finish): write_int(SpecialLengths.TIMING_DATA, outfile) - write_long(1000 * boot, outfile) - write_long(1000 * init, outfile) - write_long(1000 * finish, outfile) + write_long(int(1000 * boot), outfile) + write_long(int(1000 * init), outfile) + write_long(int(1000 * finish), outfile) def add_path(path): @@ -72,6 +73,9 @@ def main(infile, outfile): for _ in range(num_python_includes): filename = utf8_deserializer.loads(infile) add_path(os.path.join(spark_files_dir, filename)) + if sys.version > '3': + import importlib + importlib.invalidate_caches() # fetch names and values of broadcast variables num_broadcast_variables = read_int(infile) @@ -106,14 +110,14 @@ def process(): except Exception: try: write_int(SpecialLengths.PYTHON_EXCEPTION_THROWN, outfile) - write_with_length(traceback.format_exc(), outfile) + write_with_length(traceback.format_exc().encode("utf-8"), outfile) except IOError: # JVM close the socket pass except Exception: # Write the error to stderr if it happened while serializing - print >> sys.stderr, "PySpark worker failed with exception:" - print >> sys.stderr, traceback.format_exc() + print("PySpark worker failed with exception:", file=sys.stderr) + print(traceback.format_exc(), file=sys.stderr) exit(-1) finish_time = time.time() report_times(outfile, boot_time, init_time, finish_time) diff --git a/python/run-tests b/python/run-tests index f3a07d8aba562..ed3e819ef30c1 100755 --- a/python/run-tests +++ b/python/run-tests @@ -66,7 +66,7 @@ function run_core_tests() { function run_sql_tests() { echo "Run sql tests ..." - run_test "pyspark/sql/types.py" + run_test "pyspark/sql/_types.py" run_test "pyspark/sql/context.py" run_test "pyspark/sql/dataframe.py" run_test "pyspark/sql/functions.py" @@ -136,6 +136,19 @@ run_mllib_tests run_ml_tests run_streaming_tests +# Try to test with Python 3 +if [ $(which python3.4) ]; then + export PYSPARK_PYTHON="python3.4" + echo "Testing with Python3.4 version:" + $PYSPARK_PYTHON --version + + run_core_tests + run_sql_tests + run_mllib_tests + run_ml_tests + run_streaming_tests +fi + # Try to test with PyPy if [ $(which pypy) ]; then export PYSPARK_PYTHON="pypy" diff --git a/python/test_support/userlib-0.1-py2.7.egg b/python/test_support/userlib-0.1-py2.7.egg deleted file mode 100644 index 1674c9cb2227e160aec55c74e9ba1ddba850f580..0000000000000000000000000000000000000000 GIT binary patch literal 0 HcmV?d00001 literal 1945 zcmWIWW@Zs#U|`^2Sdq%-{OPC4zkVQZ3lQ@IacOaCQBG!(er{rBo?bzvZr?^OCPM+2 z-!(^Pyju5>Ifws9kif>ilh5?x&a}Ph`t|+UfpA{q4-(mHcXF|6ID3k0S`rmJ^Pu27 zX2pP|PDj;+_Dt5>5pw-nXZI!d?|Lntzt1q@eSGhRp0S;^U`hJyZ6+W7I>sGuc2o~I z;$-<*VantEcQTf&4@>aB-66eQMnKwrV%L&?hZb7x-!-%B+8>4hxZ7LO_?&A_9oL=$ zbn|W?2Kfl)_W1bByv&mLc%b`}lmDGLz4eV;d4LhKhymk9W&!bThE5TMq?8kh2`5rh zI+9Wnd=is5t|hQ}rTk-O;&oG4-6HJKBDP%2^0|tb`0vbyFVikRj(5)S6>%#*g z4>4Xznml`c(5%T>?3=boqzIt-qG4rZelQ~gLjn^6g8-5*pfQlVbi#eF!v-Slm#_J* zt#~NyvVK)UwfP*@J=O@3Im_WVolA58N~>g61`pSxe0_vE)< z^ZdsjUjEk5J^74tL*7Q7qYqkq3~ia@R@xk@{p5N=T`@m+{)dK?i}POB9N~!PDabP2 znxJE9d~uu8)sXNlS7bEW#Y9#FH7t}rbS~rCraZfArw&~@*7N!K)x8r0<{n>Yd*t

    waS|`|O?l3T$=A=7qdQyq0*MfDE zS-ZRgRd|K9^1{m+-Ws>d>^fX$@A>54M%x1?ijPX2D3<;<+wF7fk=$4Lh4pO9-@=?k}n_0dd;3x|M^$py6en$SghCm=SC!>>%c6)9BG#!3k-N^AQnc7 z0HkE(t$pfz=n3C5XM#_h(LJHB7099Mr(1N+Q%CO^6IygS{lCb@1vG^Nh{b@|)!kj! z)6dOcza+mPJ}0#-HAkN)2L?f8-Bp`(* z^5{v|nXIY_yAzriEjv{+f4c7E5CpoBkx7IZ6hzp|O`rrAY-t2hu#yOxhmkd7E4Uz9 zfrW769wg03=`R`G1oT1!VL}ry>7ZGUq8nR^N9bk-CO(*MB>T~=M^EGk0|J2tz!MQl zl1DcKJ*gwi=tnjKmhkau2c>%$*wT0qSv$5|fNm6eCO{bVAK56REP-wUdYVO;Fo^{z YJYXp}z?+o~q=XX)eSyWk87qhf04R`+K>z>% diff --git a/python/test_support/userlib-0.1.zip b/python/test_support/userlib-0.1.zip new file mode 100644 index 0000000000000000000000000000000000000000..496e1349aa9674cfa4f232ea8b5e17fcdb49f4fd GIT binary patch literal 668 zcmWIWW@h1H0D&_>m2O}Ll;B{HVJIz5Ey~GE(hm*cWMDr3PAF9Jy-;Xr1vdjD%U4DQ z1{M*Z+5or-kw6n(?`LOH28x2PFp>%J@tJv%$WKZ+A#ed zj*Lul%(&u00_Z;w5MX%g2%-@&!wQKRw17l54KrRKrU7G+VN0VEl4&Th1T+{+3_%RW k6#>WwZ$dH{9vwhKLD7NXT2?loY6fN?`~#%BKu%)-0QB3qpa1{> literal 0 HcmV?d00001 From 5fe43433529346788e8c343d338a5b7dc169cf58 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?=E4=BA=91=E5=B3=A4?= Date: Thu, 16 Apr 2015 17:32:42 -0700 Subject: [PATCH 123/128] SPARK-6927 [SQL] Sorting Error when codegen on MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fix this error by adding BinaryType comparor in GenerateOrdering. JIRA https://issues.apache.org/jira/browse/SPARK-6927 Author: 云峤 Closes #5524 from kaka1992/fix-codegen-sort and squashes the following commits: d7e2afe [云峤] fix codegen sorting error --- .../codegen/GenerateOrdering.scala | 14 ++++++++++++- .../org/apache/spark/sql/SQLQuerySuite.scala | 20 +++++++++++++++++++ 2 files changed, 33 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 0db29eb404bd1..fc2a2b60703e4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.Logging import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{StringType, NumericType} +import org.apache.spark.sql.types.{BinaryType, StringType, NumericType} /** * Generates bytecode for an [[Ordering]] of [[Row Rows]] for a given set of @@ -43,6 +43,18 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[Row]] wit val evalB = expressionEvaluator(order.child) val compare = order.child.dataType match { + case BinaryType => + q""" + val x = ${if (order.direction == Ascending) evalA.primitiveTerm else evalB.primitiveTerm} + val y = ${if (order.direction != Ascending) evalB.primitiveTerm else evalA.primitiveTerm} + var i = 0 + while (i < x.length && i < y.length) { + val res = x(i).compareTo(y(i)) + if (res != 0) return res + i = i+1 + } + return x.length - y.length + """ case _: NumericType => q""" val comp = ${evalA.primitiveTerm} - ${evalB.primitiveTerm} 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 d739e550f3e56..9e02e69fda3f2 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 @@ -398,6 +398,26 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { setConf(SQLConf.EXTERNAL_SORT, before.toString) } + test("SPARK-6927 sorting with codegen on") { + val externalbefore = conf.externalSortEnabled + val codegenbefore = conf.codegenEnabled + setConf(SQLConf.EXTERNAL_SORT, "false") + setConf(SQLConf.CODEGEN_ENABLED, "true") + sortTest() + setConf(SQLConf.EXTERNAL_SORT, externalbefore.toString) + setConf(SQLConf.CODEGEN_ENABLED, codegenbefore.toString) + } + + test("SPARK-6927 external sorting with codegen on") { + val externalbefore = conf.externalSortEnabled + val codegenbefore = conf.codegenEnabled + setConf(SQLConf.CODEGEN_ENABLED, "true") + setConf(SQLConf.EXTERNAL_SORT, "true") + sortTest() + setConf(SQLConf.EXTERNAL_SORT, externalbefore.toString) + setConf(SQLConf.CODEGEN_ENABLED, codegenbefore.toString) + } + test("limit") { checkAnswer( sql("SELECT * FROM testData LIMIT 10"), From 6183b5e2caedd074073d0f6cb6609a634e2f5194 Mon Sep 17 00:00:00 2001 From: Davies Liu Date: Thu, 16 Apr 2015 17:33:57 -0700 Subject: [PATCH 124/128] [SPARK-6911] [SQL] improve accessor for nested types Support access columns by index in Python: ``` >>> df[df[0] > 3].collect() [Row(age=5, name=u'Bob')] ``` Access items in ArrayType or MapType ``` >>> df.select(df.l.getItem(0), df.d.getItem("key")).show() >>> df.select(df.l[0], df.d["key"]).show() ``` Access field in StructType ``` >>> df.select(df.r.getField("b")).show() >>> df.select(df.r.a).show() ``` Author: Davies Liu Closes #5513 from davies/access and squashes the following commits: e04d5a0 [Davies Liu] Update run-tests-jenkins 7ada9eb [Davies Liu] update timeout d125ac4 [Davies Liu] check column name, improve scala tests 6b62540 [Davies Liu] fix test db15b42 [Davies Liu] Merge branch 'master' of github.com:apache/spark into access 6c32e79 [Davies Liu] add scala tests 11f1df3 [Davies Liu] improve accessor for nested types --- python/pyspark/sql/dataframe.py | 49 +++++++++++++++++-- python/pyspark/sql/tests.py | 18 +++++++ .../scala/org/apache/spark/sql/Column.scala | 7 +-- .../org/apache/spark/sql/DataFrameSuite.scala | 6 +++ .../scala/org/apache/spark/sql/TestData.scala | 9 ++-- 5 files changed, 76 insertions(+), 13 deletions(-) diff --git a/python/pyspark/sql/dataframe.py b/python/pyspark/sql/dataframe.py index d76504f986270..b9a3e6cfe7f49 100644 --- a/python/pyspark/sql/dataframe.py +++ b/python/pyspark/sql/dataframe.py @@ -563,16 +563,23 @@ def __getitem__(self, item): [Row(name=u'Alice', age=2), Row(name=u'Bob', age=5)] >>> df[ df.age > 3 ].collect() [Row(age=5, name=u'Bob')] + >>> df[df[0] > 3].collect() + [Row(age=5, name=u'Bob')] """ if isinstance(item, basestring): + if item not in self.columns: + raise IndexError("no such column: %s" % item) jc = self._jdf.apply(item) return Column(jc) elif isinstance(item, Column): return self.filter(item) - elif isinstance(item, list): + elif isinstance(item, (list, tuple)): return self.select(*item) + elif isinstance(item, int): + jc = self._jdf.apply(self.columns[item]) + return Column(jc) else: - raise IndexError("unexpected index: %s" % item) + raise TypeError("unexpected type: %s" % type(item)) def __getattr__(self, name): """Returns the :class:`Column` denoted by ``name``. @@ -580,8 +587,8 @@ def __getattr__(self, name): >>> df.select(df.age).collect() [Row(age=2), Row(age=5)] """ - if name.startswith("__"): - raise AttributeError(name) + if name not in self.columns: + raise AttributeError("No such column: %s" % name) jc = self._jdf.apply(name) return Column(jc) @@ -1093,7 +1100,39 @@ def __init__(self, jc): # container operators __contains__ = _bin_op("contains") __getitem__ = _bin_op("getItem") - getField = _bin_op("getField", "An expression that gets a field by name in a StructField.") + + def getItem(self, key): + """An expression that gets an item at position `ordinal` out of a list, + or gets an item by key out of a dict. + + >>> df = sc.parallelize([([1, 2], {"key": "value"})]).toDF(["l", "d"]) + >>> df.select(df.l.getItem(0), df.d.getItem("key")).show() + l[0] d[key] + 1 value + >>> df.select(df.l[0], df.d["key"]).show() + l[0] d[key] + 1 value + """ + return self[key] + + def getField(self, name): + """An expression that gets a field by name in a StructField. + + >>> from pyspark.sql import Row + >>> df = sc.parallelize([Row(r=Row(a=1, b="b"))]).toDF() + >>> df.select(df.r.getField("b")).show() + r.b + b + >>> df.select(df.r.a).show() + r.a + 1 + """ + return Column(self._jc.getField(name)) + + def __getattr__(self, item): + if item.startswith("__"): + raise AttributeError(item) + return self.getField(item) # string methods rlike = _bin_op("rlike") diff --git a/python/pyspark/sql/tests.py b/python/pyspark/sql/tests.py index 7c09a0cfe30ab..6691e8c8dc44b 100644 --- a/python/pyspark/sql/tests.py +++ b/python/pyspark/sql/tests.py @@ -426,6 +426,24 @@ def test_help_command(self): pydoc.render_doc(df.foo) pydoc.render_doc(df.take(1)) + def test_access_column(self): + df = self.df + self.assertTrue(isinstance(df.key, Column)) + self.assertTrue(isinstance(df['key'], Column)) + self.assertTrue(isinstance(df[0], Column)) + self.assertRaises(IndexError, lambda: df[2]) + self.assertRaises(IndexError, lambda: df["bad_key"]) + self.assertRaises(TypeError, lambda: df[{}]) + + def test_access_nested_types(self): + df = self.sc.parallelize([Row(l=[1], r=Row(a=1, b="b"), d={"k": "v"})]).toDF() + self.assertEqual(1, df.select(df.l[0]).first()[0]) + self.assertEqual(1, df.select(df.l.getItem(0)).first()[0]) + self.assertEqual(1, df.select(df.r.a).first()[0]) + self.assertEqual("b", df.select(df.r.getField("b")).first()[0]) + self.assertEqual("v", df.select(df.d["k"]).first()[0]) + self.assertEqual("v", df.select(df.d.getItem("k")).first()[0]) + def test_infer_long_type(self): longrow = [Row(f1='a', f2=100000000000000)] df = self.sc.parallelize(longrow).toDF() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala index 3cd7adf8cab5e..edb229c059e6b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/Column.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/Column.scala @@ -515,14 +515,15 @@ class Column(protected[sql] val expr: Expression) extends Logging { def rlike(literal: String): Column = RLike(expr, lit(literal).expr) /** - * An expression that gets an item at position `ordinal` out of an array. + * An expression that gets an item at position `ordinal` out of an array, + * or gets a value by key `key` in a [[MapType]]. * * @group expr_ops */ - def getItem(ordinal: Int): Column = GetItem(expr, Literal(ordinal)) + def getItem(key: Any): Column = GetItem(expr, Literal(key)) /** - * An expression that gets a field by name in a [[StructField]]. + * An expression that gets a field by name in a [[StructType]]. * * @group expr_ops */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index b26e22f6229fe..34b2cb054a3e7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -86,6 +86,12 @@ class DataFrameSuite extends QueryTest { TestSQLContext.setConf(SQLConf.DATAFRAME_EAGER_ANALYSIS, oldSetting.toString) } + test("access complex data") { + assert(complexData.filter(complexData("a").getItem(0) === 2).count() == 1) + assert(complexData.filter(complexData("m").getItem("1") === 1).count() == 1) + assert(complexData.filter(complexData("s").getField("key") === 1).count() == 1) + } + test("table scan") { checkAnswer( testData, 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 637f59b2e68ca..225b51bd73d6c 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 @@ -20,9 +20,8 @@ package org.apache.spark.sql import java.sql.Timestamp import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.functions._ -import org.apache.spark.sql.test._ import org.apache.spark.sql.test.TestSQLContext.implicits._ +import org.apache.spark.sql.test._ case class TestData(key: Int, value: String) @@ -199,11 +198,11 @@ object TestData { Salary(1, 1000.0) :: Nil).toDF() salary.registerTempTable("salary") - case class ComplexData(m: Map[Int, String], s: TestData, a: Seq[Int], b: Boolean) + case class ComplexData(m: Map[String, Int], s: TestData, a: Seq[Int], b: Boolean) val complexData = TestSQLContext.sparkContext.parallelize( - ComplexData(Map(1 -> "1"), TestData(1, "1"), Seq(1), true) - :: ComplexData(Map(2 -> "2"), TestData(2, "2"), Seq(2), false) + ComplexData(Map("1" -> 1), TestData(1, "1"), Seq(1), true) + :: ComplexData(Map("2" -> 2), TestData(2, "2"), Seq(2), false) :: Nil).toDF() complexData.registerTempTable("complexData") } From d96608674f6c2ff3abb13c65d80c1a3872206710 Mon Sep 17 00:00:00 2001 From: scwf Date: Thu, 16 Apr 2015 17:35:51 -0700 Subject: [PATCH 125/128] [SQL][Minor] Fix foreachUp of treenode `foreachUp` should runs the given function recursively on [[children]] then on this node(just like transformUp). The current implementation does not follow this. This will leads to checkanalysis do not check from bottom of logical tree. Author: scwf Author: Fei Wang Closes #5518 from scwf/patch-1 and squashes the following commits: 18e28b2 [scwf] added a test case 1ccbfa8 [Fei Wang] fix foreachUp --- .../apache/spark/sql/catalyst/trees/TreeNode.scala | 2 +- .../spark/sql/catalyst/trees/TreeNodeSuite.scala | 12 ++++++++++++ 2 files changed, 13 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index a2df51e598a2b..97502ed3afe72 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -85,7 +85,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { * @param f the function to be applied to each node in the tree. */ def foreachUp(f: BaseType => Unit): Unit = { - children.foreach(_.foreach(f)) + children.foreach(_.foreachUp(f)) f(this) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 4eb8708335dcf..6b393327cc97a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -117,5 +117,17 @@ class TreeNodeSuite extends FunSuite { assert(transformed.origin.startPosition.isDefined) } + test("foreach up") { + val actual = new ArrayBuffer[String]() + val expected = Seq("1", "2", "3", "4", "-", "*", "+") + val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) + expression foreachUp { + case b: BinaryExpression => actual.append(b.symbol); + case l: Literal => actual.append(l.toString); + } + + assert(expected === actual) + } + } From 1e43851d6455f65b850ea0327d0e92f65395d23f Mon Sep 17 00:00:00 2001 From: Liang-Chi Hsieh Date: Thu, 16 Apr 2015 17:50:20 -0700 Subject: [PATCH 126/128] [SPARK-6899][SQL] Fix type mismatch when using codegen with Average on DecimalType JIRA https://issues.apache.org/jira/browse/SPARK-6899 Author: Liang-Chi Hsieh Closes #5517 from viirya/fix_codegen_average and squashes the following commits: 8ae5f65 [Liang-Chi Hsieh] Add the case of DecimalType.Unlimited to Average. --- .../spark/sql/catalyst/expressions/aggregates.scala | 2 +- .../test/scala/org/apache/spark/sql/DataFrameSuite.scala | 9 +++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 14a855054b94d..f3830c6d3bcf2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -326,7 +326,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN override def asPartial: SplitEvaluation = { child.dataType match { - case DecimalType.Fixed(_, _) => + case DecimalType.Fixed(_, _) | DecimalType.Unlimited => // Turn the child to unlimited decimals for calculation, before going back to fixed val partialSum = Alias(Sum(Cast(child, DecimalType.Unlimited)), "PartialSum")() val partialCount = Alias(Count(child), "PartialCount")() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 34b2cb054a3e7..44a7d1e7bbb6a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -537,4 +537,13 @@ class DataFrameSuite extends QueryTest { val df = TestSQLContext.createDataFrame(rowRDD, schema) df.rdd.collect() } + + test("SPARK-6899") { + val originalValue = TestSQLContext.conf.codegenEnabled + TestSQLContext.setConf(SQLConf.CODEGEN_ENABLED, "true") + checkAnswer( + decimalData.agg(avg('a)), + Row(new java.math.BigDecimal(2.0))) + TestSQLContext.setConf(SQLConf.CODEGEN_ENABLED, originalValue.toString) + } } From e5949c287ed19e78b6eecc61c3e88a07ad452eb9 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Apr 2015 17:59:49 -0700 Subject: [PATCH 127/128] [SPARK-6966][SQL] Use correct ClassLoader for JDBC Driver Otherwise we cannot add jars with drivers after the fact. Author: Michael Armbrust Closes #5543 from marmbrus/jdbcClassloader and squashes the following commits: d9930f3 [Michael Armbrust] fix imports 73d0614 [Michael Armbrust] [SPARK-6966][SQL] Use correct ClassLoader for JDBC Driver --- .../main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala index 99b755c9f25d0..5f480083d5a49 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRelation.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.StructType +import org.apache.spark.util.Utils /** * Data corresponding to one partition of a JDBCRDD. @@ -99,7 +100,7 @@ private[sql] class DefaultSource extends RelationProvider { val upperBound = parameters.getOrElse("upperBound", null) val numPartitions = parameters.getOrElse("numPartitions", null) - if (driver != null) Class.forName(driver) + if (driver != null) Utils.getContextOrSparkClassLoader.loadClass(driver) if (partitionColumn != null && (lowerBound == null || upperBound == null || numPartitions == null)) { From 8220d5265f1bbea9dfdaeec4f2d06d7fe24c0bc3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 16 Apr 2015 21:49:26 -0500 Subject: [PATCH 128/128] [SPARK-6972][SQL] Add Coalesce to DataFrame Author: Michael Armbrust Closes #5545 from marmbrus/addCoalesce and squashes the following commits: 9fdf3f6 [Michael Armbrust] [SPARK-6972][SQL] Add Coalesce to DataFrame --- .../scala/org/apache/spark/sql/DataFrame.scala | 14 ++++++++++++++ .../main/scala/org/apache/spark/sql/RDDApi.scala | 2 ++ .../org/apache/spark/sql/DataFrameSuite.scala | 8 ++++++++ 3 files changed, 24 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 3235f85d5bbd2..17c21f6e3a0e9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -908,6 +908,20 @@ class DataFrame private[sql]( schema, needsConversion = false) } + /** + * Returns a new [[DataFrame]] that has exactly `numPartitions` partitions. + * Similar to coalesce defined on an [[RDD]], this operation results in a narrow dependency, e.g. + * if you go from 1000 partitions to 100 partitions, there will not be a shuffle, instead each of + * the 100 new partitions will claim 10 of the current partitions. + * @group rdd + */ + override def coalesce(numPartitions: Int): DataFrame = { + sqlContext.createDataFrame( + queryExecution.toRdd.coalesce(numPartitions), + schema, + needsConversion = false) + } + /** * Returns a new [[DataFrame]] that contains only the unique rows from this [[DataFrame]]. * @group dfops diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala index ba4373f0124b4..63dbab19947c0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RDDApi.scala @@ -61,5 +61,7 @@ private[sql] trait RDDApi[T] { def repartition(numPartitions: Int): DataFrame + def coalesce(numPartitions: Int): DataFrame + def distinct: DataFrame } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala index 44a7d1e7bbb6a..3250ab476aeb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSuite.scala @@ -178,6 +178,14 @@ class DataFrameSuite extends QueryTest { testData.select('key).collect().toSeq) } + test("coalesce") { + assert(testData.select('key).coalesce(1).rdd.partitions.size === 1) + + checkAnswer( + testData.select('key).coalesce(1).select('key), + testData.select('key).collect().toSeq) + } + test("groupBy") { checkAnswer( testData2.groupBy("a").agg($"a", sum($"b")),